From 5d7a77c20771a803f34855f0e31fe58ad6566677 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 22 Sep 2020 17:23:46 +0800 Subject: [PATCH 0001/1088] add mutation support for StorageMemory --- src/Storages/StorageMemory.cpp | 50 +++++++++++++++++++++++++++++++++- src/Storages/StorageMemory.h | 2 ++ 2 files changed, 51 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 25e232dc4ad..9686fbe3062 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -2,8 +2,9 @@ #include -#include +#include #include +#include #include #include @@ -203,6 +204,53 @@ void StorageMemory::drop() data.clear(); } +static inline void columnUpdate(Block & old_block, const Block & new_block) +{ + for (const auto & it : new_block) + { + auto col_name = it.name; + auto & col_with_type_name = old_block.getByName(col_name); + col_with_type_name.column = it.column; + } +} + +void StorageMemory::mutate(const MutationCommands & commands, const Context & context) +{ + auto metadata_snapshot_ = getInMemoryMetadataPtr(); + auto storage_id_ = getStorageID(); + auto storage_ptr_ = DatabaseCatalog::instance().getTable(storage_id_, context); + auto interpreter = std::make_unique(storage_ptr_, metadata_snapshot_, commands, context, true); + auto in = interpreter->execute(); + + in->readPrefix(); + BlocksList out; + Block block; + while (block = in->read()) + { + out.push_back(block); + } + in->readSuffix(); + + std::lock_guard lock(mutex); + + // all column affected + if (interpreter->isAffectingAllColumns()) + { + std::swap(data, out); + } + else + { + auto data_it = data.begin(); + auto out_it = out.begin(); + while (data_it != data.end() && out_it != out.end()) + { + columnUpdate(*data_it, *out_it); + ++data_it; + ++out_it; + } + } +} + void StorageMemory::truncate( const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) { diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index e67e3015028..f0d5d7aadec 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -43,6 +43,8 @@ public: void drop() override; + void mutate(const MutationCommands & commands, const Context & context) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override; std::optional totalRows() const override; From c0b5f847d97cf0826036a61c1fc154060e4afd50 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 22 Sep 2020 17:29:57 +0800 Subject: [PATCH 0002/1088] add test fix fix --- src/Storages/StorageMemory.cpp | 12 +++++------ ...ation_support_for_storage_memory.reference | 14 +++++++++++++ ...97_mutation_support_for_storage_memory.sql | 20 +++++++++++++++++++ 3 files changed, 40 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/01497_mutation_support_for_storage_memory.reference create mode 100644 tests/queries/0_stateless/01497_mutation_support_for_storage_memory.sql diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 9686fbe3062..72b8fd78d65 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -216,22 +216,22 @@ static inline void columnUpdate(Block & old_block, const Block & new_block) void StorageMemory::mutate(const MutationCommands & commands, const Context & context) { - auto metadata_snapshot_ = getInMemoryMetadataPtr(); - auto storage_id_ = getStorageID(); - auto storage_ptr_ = DatabaseCatalog::instance().getTable(storage_id_, context); - auto interpreter = std::make_unique(storage_ptr_, metadata_snapshot_, commands, context, true); + auto metadata_snapshot = getInMemoryMetadataPtr(); + auto storage = getStorageID(); + auto storage_ptr = DatabaseCatalog::instance().getTable(storage, context); + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context, true); auto in = interpreter->execute(); in->readPrefix(); BlocksList out; Block block; - while (block = in->read()) + while ((block = in->read())) { out.push_back(block); } in->readSuffix(); - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); // all column affected if (interpreter->isAffectingAllColumns()) diff --git a/tests/queries/0_stateless/01497_mutation_support_for_storage_memory.reference b/tests/queries/0_stateless/01497_mutation_support_for_storage_memory.reference new file mode 100644 index 00000000000..5aaf21f9137 --- /dev/null +++ b/tests/queries/0_stateless/01497_mutation_support_for_storage_memory.reference @@ -0,0 +1,14 @@ +1 1 +2 2 +3 3 +4 4 +5 5 +100 1 +2 2 +3 3 +4 4 +5 5 +2 2 +3 3 +4 4 +5 5 diff --git a/tests/queries/0_stateless/01497_mutation_support_for_storage_memory.sql b/tests/queries/0_stateless/01497_mutation_support_for_storage_memory.sql new file mode 100644 index 00000000000..408487ed205 --- /dev/null +++ b/tests/queries/0_stateless/01497_mutation_support_for_storage_memory.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS defaults; +CREATE TABLE defaults +( + n Int32, + s String +)ENGINE = Memory(); + +INSERT INTO defaults VALUES(1, '1') (2, '2') (3, '3') (4, '4') (5, '5'); + +SELECT * FROM defaults; + +ALTER TABLE defaults UPDATE n = 100 WHERE s = '1'; + +SELECT * FROM defaults; + +ALTER TABLE defaults DELETE WHERE n = 100; + +SELECT * FROM defaults; + +DROP TABLE defaults; From 8094ba713eeed92dc4f58d029a0d4a3e87bf1506 Mon Sep 17 00:00:00 2001 From: myrrc Date: Mon, 28 Sep 2020 17:33:52 +0300 Subject: [PATCH 0003/1088] first steps: changed allowed types --- .../AggregateFunctionAvgWeighted.cpp | 29 ++++++++++++++----- .../AggregateFunctionAvgWeighted.h | 1 + 2 files changed, 23 insertions(+), 7 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp index 6722a94cdc6..10698332a14 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp @@ -17,27 +17,41 @@ namespace template struct AvgWeighted { - using FieldType = std::conditional_t, - std::conditional_t, Decimal256, Decimal128>, - NearestFieldType>; - // using FieldType = std::conditional_t, Decimal128, NearestFieldType>; + using FieldType = std::conditional_t< + IsDecimalNumber, + std::conditional_t, + Decimal256, + Decimal128>, + NearestFieldType>; + using Function = AggregateFunctionAvgWeighted>; }; template using AggregateFuncAvgWeighted = typename AvgWeighted::Function; +bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) +{ + return (isInteger(left) || isFloat(left)) && (isInteger(right) || isFloat(right)); +} + AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name, const DataTypes & argument_types, const Array & parameters) { assertNoParameters(name, parameters); assertBinary(name, argument_types); AggregateFunctionPtr res; + const auto data_type = static_cast(argument_types[0]); const auto data_type_weight = static_cast(argument_types[1]); - if (!data_type->equals(*data_type_weight)) - throw Exception("Different types " + data_type->getName() + " and " + data_type_weight->getName() + " of arguments for aggregate function " + name, - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!allowTypes(data_type, data_type_weight)) + throw Exception( + "Types " + data_type->getName() + + " and " + data_type_weight->getName() + + " are non-conforming as arguments for aggregate function " + name, + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (isDecimal(data_type)) res.reset(createWithDecimalType(*data_type, *data_type, argument_types)); else @@ -46,6 +60,7 @@ AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name if (!res) throw Exception("Illegal type " + data_type->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + return res; } diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index 8eb619585c7..53a0ae50fb4 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -11,6 +11,7 @@ public: using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { const auto & values = static_cast(*columns[0]); From 3ddcf590f69a790f518cd5c5e1dfb8db3adac08b Mon Sep 17 00:00:00 2001 From: hchen9 Date: Mon, 28 Sep 2020 16:32:50 -0700 Subject: [PATCH 0004/1088] Disable optimize_trivial_count_query when select_sequential_consistency = 0 --- src/Interpreters/TreeRewriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 0b2f8ac3eb7..b119788b083 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -619,7 +619,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.ast_join = select_query->join(); if (result.optimize_trivial_count) - result.optimize_trivial_count = settings.optimize_trivial_count_query && + result.optimize_trivial_count = settings.optimize_trivial_count_query && settings.select_sequential_consistency && !select_query->where() && !select_query->prewhere() && !select_query->groupBy() && !select_query->having() && !select_query->sampleSize() && !select_query->sampleOffset() && !select_query->final() && (tables_with_columns.size() < 2 || isLeft(result.analyzed_join->kind())); From fda6bde428d645b9a1d68c9e510508ac532ec641 Mon Sep 17 00:00:00 2001 From: hchen9 Date: Mon, 28 Sep 2020 17:05:22 -0700 Subject: [PATCH 0005/1088] improve change --- src/Interpreters/InterpreterSelectQuery.cpp | 2 ++ src/Interpreters/TreeRewriter.cpp | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 887f4795bcb..8c0643f78d7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1112,12 +1112,14 @@ void InterpreterSelectQuery::executeFetchColumns( /// Optimization for trivial query like SELECT count() FROM table. bool optimize_trivial_count = syntax_analyzer_result->optimize_trivial_count + && settings.select_sequential_consistency && storage && !filter_info && processing_stage == QueryProcessingStage::FetchColumns && query_analyzer->hasAggregation() && (query_analyzer->aggregates().size() == 1) && typeid_cast(query_analyzer->aggregates()[0].function.get()); + if (optimize_trivial_count) { diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index b119788b083..0b2f8ac3eb7 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -619,7 +619,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.ast_join = select_query->join(); if (result.optimize_trivial_count) - result.optimize_trivial_count = settings.optimize_trivial_count_query && settings.select_sequential_consistency && + result.optimize_trivial_count = settings.optimize_trivial_count_query && !select_query->where() && !select_query->prewhere() && !select_query->groupBy() && !select_query->having() && !select_query->sampleSize() && !select_query->sampleOffset() && !select_query->final() && (tables_with_columns.size() < 2 || isLeft(result.analyzed_join->kind())); From a5ac39b564745c6a1a652e933de62ab4317ac3e5 Mon Sep 17 00:00:00 2001 From: hchen9 Date: Wed, 30 Sep 2020 16:47:42 -0700 Subject: [PATCH 0006/1088] Pass Context parameter for IStorage.totalRows and IStorage.totalBytes --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Storages/IStorage.h | 4 ++-- src/Storages/StorageBuffer.cpp | 6 +++--- src/Storages/StorageBuffer.h | 4 ++-- src/Storages/StorageMemory.cpp | 4 ++-- src/Storages/StorageMemory.h | 4 ++-- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageMergeTree.h | 4 ++-- src/Storages/StorageNull.h | 4 ++-- src/Storages/System/StorageSystemTables.cpp | 4 ++-- 10 files changed, 20 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8c0643f78d7..96844223b1c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1125,7 +1125,7 @@ void InterpreterSelectQuery::executeFetchColumns( { const auto & desc = query_analyzer->aggregates()[0]; const auto & func = desc.function; - std::optional num_rows = storage->totalRows(); + std::optional num_rows = storage->totalRows(context); if (num_rows) { AggregateFunctionCount & agg_count = static_cast(*func); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index dc7c684d5b4..7ddab4fa8a4 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -455,7 +455,7 @@ public: /// - For total_rows column in system.tables /// /// Does takes underlying Storage (if any) into account. - virtual std::optional totalRows() const { return {}; } + virtual std::optional totalRows(const Context &) const { return {}; } /// If it is possible to quickly determine exact number of bytes for the table on storage: /// - memory (approximated, resident) @@ -470,7 +470,7 @@ public: /// Memory part should be estimated as a resident memory size. /// In particular, alloctedBytes() is preferable over bytes() /// when considering in-memory blocks. - virtual std::optional totalBytes() const { return {}; } + virtual std::optional totalBytes(const Context &) const { return {}; } /// Number of rows INSERTed since server start. /// diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 5b9957f4ed4..720ca90327d 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -793,13 +793,13 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const S } } -std::optional StorageBuffer::totalRows() const +std::optional StorageBuffer::totalRows(const Context & context) const { std::optional underlying_rows; auto underlying = DatabaseCatalog::instance().tryGetTable(destination_id, global_context); if (underlying) - underlying_rows = underlying->totalRows(); + underlying_rows = underlying->totalRows(context); if (!underlying_rows) return underlying_rows; @@ -812,7 +812,7 @@ std::optional StorageBuffer::totalRows() const return rows + *underlying_rows; } -std::optional StorageBuffer::totalBytes() const +std::optional StorageBuffer::totalBytes(const Context & /*context*/) const { UInt64 bytes = 0; for (const auto & buffer : buffers) diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 8f1354399ef..593e394fb46 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -94,8 +94,8 @@ public: /// The structure of the subordinate table is not checked and does not change. void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override; - std::optional totalRows() const override; - std::optional totalBytes() const override; + std::optional totalRows(const Context & context) const override; + std::optional totalBytes(const Context & context) const override; std::optional lifetimeRows() const override { return writes.rows; } std::optional lifetimeBytes() const override { return writes.bytes; } diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 25e232dc4ad..0be49ee7bd3 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -210,7 +210,7 @@ void StorageMemory::truncate( data.clear(); } -std::optional StorageMemory::totalRows() const +std::optional StorageMemory::totalRows(const Context &) const { UInt64 rows = 0; std::lock_guard lock(mutex); @@ -219,7 +219,7 @@ std::optional StorageMemory::totalRows() const return rows; } -std::optional StorageMemory::totalBytes() const +std::optional StorageMemory::totalBytes(const Context &) const { UInt64 bytes = 0; std::lock_guard lock(mutex); diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index e67e3015028..586be298808 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -45,8 +45,8 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override; - std::optional totalRows() const override; - std::optional totalBytes() const override; + std::optional totalRows(const Context &) const override; + std::optional totalBytes(const Context &) const override; /** Delays initialization of StorageMemory::read() until the first read is actually happen. * Usually, fore code like this: diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 347474753dc..fc907edb43e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -184,12 +184,12 @@ Pipe StorageMergeTree::read( context, max_block_size, num_streams); } -std::optional StorageMergeTree::totalRows() const +std::optional StorageMergeTree::totalRows(const Context &) const { return getTotalActiveSizeInRows(); } -std::optional StorageMergeTree::totalBytes() const +std::optional StorageMergeTree::totalBytes(const Context &) const { return getTotalActiveSizeInBytes(); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 5662f9e0088..162321668a2 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -46,8 +46,8 @@ public: size_t max_block_size, unsigned num_streams) override; - std::optional totalRows() const override; - std::optional totalBytes() const override; + std::optional totalRows(const Context &) const override; + std::optional totalBytes(const Context &) const override; BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index b5387c6b924..db2d169b7dd 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -46,11 +46,11 @@ public: void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override; - std::optional totalRows() const override + std::optional totalRows(const Context &) const override { return {0}; } - std::optional totalBytes() const override + std::optional totalBytes(const Context &) const override { return {0}; } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 0ad961ad7d8..dabf4685b29 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -430,7 +430,7 @@ protected: if (columns_mask[src_index++]) { assert(table != nullptr); - auto total_rows = table->totalRows(); + auto total_rows = table->totalRows(context); if (total_rows) res_columns[res_index++]->insert(*total_rows); else @@ -440,7 +440,7 @@ protected: if (columns_mask[src_index++]) { assert(table != nullptr); - auto total_bytes = table->totalBytes(); + auto total_bytes = table->totalBytes(context); if (total_bytes) res_columns[res_index++]->insert(*total_bytes); else From b4494578d77add388528ee786c37ddb7276ae12e Mon Sep 17 00:00:00 2001 From: hchen9 Date: Wed, 30 Sep 2020 16:50:58 -0700 Subject: [PATCH 0007/1088] Support select_sequential_consistency in StorageReplicatedMergeTree.totalRows and totalBytes --- src/Interpreters/InterpreterSelectQuery.cpp | 2 -- src/Storages/StorageReplicatedMergeTree.cpp | 28 ++++++++++++++------- src/Storages/StorageReplicatedMergeTree.h | 6 ++--- 3 files changed, 22 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 96844223b1c..e179d9325fb 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1112,14 +1112,12 @@ void InterpreterSelectQuery::executeFetchColumns( /// Optimization for trivial query like SELECT count() FROM table. bool optimize_trivial_count = syntax_analyzer_result->optimize_trivial_count - && settings.select_sequential_consistency && storage && !filter_info && processing_stage == QueryProcessingStage::FetchColumns && query_analyzer->hasAggregation() && (query_analyzer->aggregates().size() == 1) && typeid_cast(query_analyzer->aggregates()[0].function.get()); - if (optimize_trivial_count) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 608d983a21e..0fca98ff987 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3518,34 +3518,44 @@ Pipe StorageReplicatedMergeTree::read( template -void StorageReplicatedMergeTree::foreachCommittedParts(const Func & func) const +void StorageReplicatedMergeTree::foreachCommittedParts(const Func & func, const Context & context) const { - auto max_added_blocks = getMaxAddedBlocks(); + std::optional max_added_blocks = {}; + + /** Synchronously go to ZooKeeper when select_sequential_consistency enabled */ + if (context.getSettingsRef().select_sequential_consistency) + { + max_added_blocks = getMaxAddedBlocks(); + } + auto lock = lockParts(); for (const auto & part : getDataPartsStateRange(DataPartState::Committed)) { if (part->isEmpty()) continue; - auto blocks_iterator = max_added_blocks.find(part->info.partition_id); - if (blocks_iterator == max_added_blocks.end() || part->info.max_block > blocks_iterator->second) - continue; + if (max_added_blocks) + { + auto blocks_iterator = max_added_blocks->find(part->info.partition_id); + if (blocks_iterator == max_added_blocks->end() || part->info.max_block > blocks_iterator->second) + continue; + } func(part); } } -std::optional StorageReplicatedMergeTree::totalRows() const +std::optional StorageReplicatedMergeTree::totalRows(const Context & context) const { UInt64 res = 0; - foreachCommittedParts([&res](auto & part) { res += part->rows_count; }); + foreachCommittedParts([&res](auto & part) { res += part->rows_count; }, context); return res; } -std::optional StorageReplicatedMergeTree::totalBytes() const +std::optional StorageReplicatedMergeTree::totalBytes(const Context & context) const { UInt64 res = 0; - foreachCommittedParts([&res](auto & part) { res += part->getBytesOnDisk(); }); + foreachCommittedParts([&res](auto & part) { res += part->getBytesOnDisk(); }, context); return res; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 2bc9265331d..fd8ddf7c441 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -96,8 +96,8 @@ public: size_t max_block_size, unsigned num_streams) override; - std::optional totalRows() const override; - std::optional totalBytes() const override; + std::optional totalRows(const Context & context) const override; + std::optional totalBytes(const Context & context) const override; BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; @@ -305,7 +305,7 @@ private: bool other_replicas_fixed_granularity = false; template - void foreachCommittedParts(const Func & func) const; + void foreachCommittedParts(const Func & func, const Context & context) const; /** Creates the minimum set of nodes in ZooKeeper and create first replica. * Returns true if was created, false if exists. From 68888408da6e44715cc257ce31e6cbe3f6a3b93e Mon Sep 17 00:00:00 2001 From: hchen9 Date: Thu, 1 Oct 2020 21:23:41 -0700 Subject: [PATCH 0008/1088] Pass context ptr to ref --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0924fa2bdde..7b9c7856bd8 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1121,7 +1121,7 @@ void InterpreterSelectQuery::executeFetchColumns( { const auto & desc = query_analyzer->aggregates()[0]; const auto & func = desc.function; - std::optional num_rows = storage->totalRows(context); + std::optional num_rows = storage->totalRows(*context); if (num_rows) { AggregateFunctionCount & agg_count = static_cast(*func); From 279c6878392386f3225987c6c9bcf4fc1aec6e42 Mon Sep 17 00:00:00 2001 From: hchen9 Date: Fri, 2 Oct 2020 10:31:47 -0700 Subject: [PATCH 0009/1088] Add test case for trivial_count with select_sequential_consistency = 0 --- ...hout_select_sequence_consistency.reference | 3 ++ ...nt_without_select_sequence_consistency.sql | 36 +++++++++++++++++++ 2 files changed, 39 insertions(+) create mode 100644 tests/queries/0_stateless/01513_select_count_without_select_sequence_consistency.reference create mode 100644 tests/queries/0_stateless/01513_select_count_without_select_sequence_consistency.sql diff --git a/tests/queries/0_stateless/01513_select_count_without_select_sequence_consistency.reference b/tests/queries/0_stateless/01513_select_count_without_select_sequence_consistency.reference new file mode 100644 index 00000000000..16930b0384a --- /dev/null +++ b/tests/queries/0_stateless/01513_select_count_without_select_sequence_consistency.reference @@ -0,0 +1,3 @@ +3 +4 +4 \ No newline at end of file diff --git a/tests/queries/0_stateless/01513_select_count_without_select_sequence_consistency.sql b/tests/queries/0_stateless/01513_select_count_without_select_sequence_consistency.sql new file mode 100644 index 00000000000..69c45905655 --- /dev/null +++ b/tests/queries/0_stateless/01513_select_count_without_select_sequence_consistency.sql @@ -0,0 +1,36 @@ +SET send_logs_level = 'fatal'; + +DROP TABLE IF EXISTS quorum1; +DROP TABLE IF EXISTS quorum2; +DROP TABLE IF EXISTS quorum3; + +CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01513/sequence_consistency', '1') ORDER BY x PARTITION BY y; +CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01513/sequence_consistency', '2') ORDER BY x PARTITION BY y; +CREATE TABLE quorum3(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01513/sequence_consistency', '3') ORDER BY x PARTITION BY y; + +INSERT INTO quorum1 VALUES (1, '1990-11-15'); +INSERT INTO quorum1 VALUES (2, '1990-11-15'); +INSERT INTO quorum1 VALUES (3, '2020-12-16'); + +SYSTEM SYNC REPLICA quorum2; +SYSTEM SYNC REPLICA quorum3; + +SET select_sequential_consistency=0; +SET optimize_trivial_count_query=1; +SET insert_quorum=2; + +SYSTEM STOP FETCHES quorum1; + +INSERT INTO quorum2 VALUES (4, toDate('2020-12-16')); + +SYSTEM SYNC REPLICA quorum3; + +-- Should read local committed parts without throwing error code: 289. DB::Exception: Replica doesn't have part 20201216_1_1_0 which was successfully written to quorum of other replicas. +SELECT count() FROM quorum1; + +SELECT count() FROM quorum2; +SELECT count() FROM quorum3; + +DROP TABLE quorum1; +DROP TABLE quorum2; +DROP TABLE quorum3; From 572e3f6da4923d05e3fed9ef1f3fd1d10a33e057 Mon Sep 17 00:00:00 2001 From: hchen9 Date: Fri, 2 Oct 2020 10:36:47 -0700 Subject: [PATCH 0010/1088] Rename test files --- ... => 01513_count_without_select_sequence_consistency.reference} | 0 ...cy.sql => 01513_count_without_select_sequence_consistency.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01513_select_count_without_select_sequence_consistency.reference => 01513_count_without_select_sequence_consistency.reference} (100%) rename tests/queries/0_stateless/{01513_select_count_without_select_sequence_consistency.sql => 01513_count_without_select_sequence_consistency.sql} (100%) diff --git a/tests/queries/0_stateless/01513_select_count_without_select_sequence_consistency.reference b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency.reference similarity index 100% rename from tests/queries/0_stateless/01513_select_count_without_select_sequence_consistency.reference rename to tests/queries/0_stateless/01513_count_without_select_sequence_consistency.reference diff --git a/tests/queries/0_stateless/01513_select_count_without_select_sequence_consistency.sql b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency.sql similarity index 100% rename from tests/queries/0_stateless/01513_select_count_without_select_sequence_consistency.sql rename to tests/queries/0_stateless/01513_count_without_select_sequence_consistency.sql From cc2cb76295bf490387a1f8231f60b5a03c8e1d38 Mon Sep 17 00:00:00 2001 From: hchen9 Date: Fri, 2 Oct 2020 11:17:28 -0700 Subject: [PATCH 0011/1088] Add newline in result reference --- .../01513_count_without_select_sequence_consistency.reference | 2 +- .../01513_count_without_select_sequence_consistency.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01513_count_without_select_sequence_consistency.reference b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency.reference index 16930b0384a..4990ad10152 100644 --- a/tests/queries/0_stateless/01513_count_without_select_sequence_consistency.reference +++ b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency.reference @@ -1,3 +1,3 @@ 3 4 -4 \ No newline at end of file +4 diff --git a/tests/queries/0_stateless/01513_count_without_select_sequence_consistency.sql b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency.sql index 69c45905655..8bf7e6a6931 100644 --- a/tests/queries/0_stateless/01513_count_without_select_sequence_consistency.sql +++ b/tests/queries/0_stateless/01513_count_without_select_sequence_consistency.sql @@ -25,7 +25,7 @@ INSERT INTO quorum2 VALUES (4, toDate('2020-12-16')); SYSTEM SYNC REPLICA quorum3; --- Should read local committed parts without throwing error code: 289. DB::Exception: Replica doesn't have part 20201216_1_1_0 which was successfully written to quorum of other replicas. +-- Should read local committed parts instead of throwing error code: 289. DB::Exception: Replica doesn't have part 20201216_1_1_0 which was successfully written to quorum of other replicas. SELECT count() FROM quorum1; SELECT count() FROM quorum2; From 76182694b0bc836d51a9ec470d0adea3456cc489 Mon Sep 17 00:00:00 2001 From: hchen9 Date: Sat, 3 Oct 2020 14:07:21 -0700 Subject: [PATCH 0012/1088] Fix integration test case test_reload_zookeeper --- tests/integration/test_reload_zookeeper/configs/users.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_reload_zookeeper/configs/users.xml b/tests/integration/test_reload_zookeeper/configs/users.xml index 6061af8e33d..59802e82698 100644 --- a/tests/integration/test_reload_zookeeper/configs/users.xml +++ b/tests/integration/test_reload_zookeeper/configs/users.xml @@ -2,6 +2,8 @@ + 1 + 1 From 347f5c99b1dbe9e087a3251ceaa1bc150dfa0239 Mon Sep 17 00:00:00 2001 From: hchen9 Date: Sat, 3 Oct 2020 14:18:24 -0700 Subject: [PATCH 0013/1088] Set user_configs = configs/users.xml --- tests/integration/test_reload_zookeeper/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_reload_zookeeper/test.py b/tests/integration/test_reload_zookeeper/test.py index 66df5a1a126..ba2c81cfc73 100644 --- a/tests/integration/test_reload_zookeeper/test.py +++ b/tests/integration/test_reload_zookeeper/test.py @@ -8,7 +8,7 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper.xml') -node = cluster.add_instance('node', with_zookeeper=True) +node = cluster.add_instance('node', with_zookeeper=True, user_configs=["configs/users.xml"]) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) ZK_CONFIG_PATH = os.path.join(SCRIPT_DIR, 'configs/zookeeper.xml') From ea10291758fbf799a590a8bc43adfa0e0d124c17 Mon Sep 17 00:00:00 2001 From: hchen9 Date: Sat, 3 Oct 2020 21:03:46 -0700 Subject: [PATCH 0014/1088] Reformat code --- tests/integration/test_reload_zookeeper/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_reload_zookeeper/test.py b/tests/integration/test_reload_zookeeper/test.py index ba2c81cfc73..872510f3f08 100644 --- a/tests/integration/test_reload_zookeeper/test.py +++ b/tests/integration/test_reload_zookeeper/test.py @@ -8,7 +8,8 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper.xml') -node = cluster.add_instance('node', with_zookeeper=True, user_configs=["configs/users.xml"]) +node = cluster.add_instance('node', with_zookeeper=True, + user_configs=["configs/users.xml"]) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) ZK_CONFIG_PATH = os.path.join(SCRIPT_DIR, 'configs/zookeeper.xml') From b96c1326fc68fee516af78bdd3f8ba82246468b6 Mon Sep 17 00:00:00 2001 From: hchen9 Date: Sun, 4 Oct 2020 00:18:43 -0700 Subject: [PATCH 0015/1088] Trigger test again --- tests/integration/test_reload_zookeeper/test.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/test_reload_zookeeper/test.py b/tests/integration/test_reload_zookeeper/test.py index 872510f3f08..ba2c81cfc73 100644 --- a/tests/integration/test_reload_zookeeper/test.py +++ b/tests/integration/test_reload_zookeeper/test.py @@ -8,8 +8,7 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper.xml') -node = cluster.add_instance('node', with_zookeeper=True, - user_configs=["configs/users.xml"]) +node = cluster.add_instance('node', with_zookeeper=True, user_configs=["configs/users.xml"]) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) ZK_CONFIG_PATH = os.path.join(SCRIPT_DIR, 'configs/zookeeper.xml') From cbf3af401d906a3b342f2b8cc54a7508e933d8ea Mon Sep 17 00:00:00 2001 From: hchen9 Date: Sun, 4 Oct 2020 02:42:03 -0700 Subject: [PATCH 0016/1088] Fix comments --- src/Storages/StorageReplicatedMergeTree.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 35fa7af63eb..095019a25db 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3543,7 +3543,9 @@ void StorageReplicatedMergeTree::foreachCommittedParts(const Func & func, const { std::optional max_added_blocks = {}; - /** Synchronously go to ZooKeeper when select_sequential_consistency enabled */ + /** + * Synchronously go to ZooKeeper when select_sequential_consistency enabled + */ if (context.getSettingsRef().select_sequential_consistency) { max_added_blocks = getMaxAddedBlocks(); From 6289f44fd4c9e166acbff4f1188e3e1f3313c90e Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 5 Oct 2020 00:28:36 +0800 Subject: [PATCH 0017/1088] multiversion storage for StorageMemory --- src/Storages/StorageMemory.cpp | 60 +++++++++++++++------------------- src/Storages/StorageMemory.h | 9 +++-- 2 files changed, 30 insertions(+), 39 deletions(-) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 72b8fd78d65..d0a2d23bf2b 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -28,7 +28,7 @@ public: /// We don't use synchronisation here, because elements in range [first, last] won't be modified. MemorySource( Names column_names_, - BlocksList::iterator first_, + BlocksList::const_iterator first_, size_t num_blocks_, const StorageMemory & storage, const StorageMetadataPtr & metadata_snapshot) @@ -42,11 +42,7 @@ public: /// If called, will initialize the number of blocks at first read. /// It allows to read data which was inserted into memory table AFTER Storage::read was called. /// This hack is needed for global subqueries. - void delayInitialization(BlocksList * data_, std::mutex * mutex_) - { - data = data_; - mutex = mutex_; - } + void delayInitialization(std::shared_ptr data_) { data = data_; } String getName() const override { return "Memory"; } @@ -55,13 +51,11 @@ protected: { if (data) { - std::lock_guard guard(*mutex); current_it = data->begin(); num_blocks = data->size(); is_finished = num_blocks == 0; data = nullptr; - mutex = nullptr; } if (is_finished) @@ -90,13 +84,12 @@ protected: } private: Names column_names; - BlocksList::iterator current_it; + BlocksList::const_iterator current_it; size_t current_block_idx = 0; size_t num_blocks; bool is_finished = false; - BlocksList * data = nullptr; - std::mutex * mutex = nullptr; + std::shared_ptr data = nullptr; }; @@ -115,8 +108,9 @@ public: void write(const Block & block) override { metadata_snapshot->check(block, true); - std::lock_guard lock(storage.mutex); - storage.data.push_back(block); + auto new_data = std::make_unique(*(storage.data.get())); + new_data->push_back(block); + storage.data.set(std::move(new_data)); } private: StorageMemory & storage; @@ -125,7 +119,7 @@ private: StorageMemory::StorageMemory(const StorageID & table_id_, ColumnsDescription columns_description_, ConstraintsDescription constraints_) - : IStorage(table_id_) + : IStorage(table_id_), data(std::make_unique()) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(std::move(columns_description_)); @@ -145,7 +139,7 @@ Pipe StorageMemory::read( { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - std::lock_guard lock(mutex); + auto current_data = data.get(); if (delay_read_for_global_subqueries) { @@ -157,19 +151,19 @@ Pipe StorageMemory::read( /// set for IN or hash table for JOIN, which can't be done concurrently. /// Since no other manipulation with data is done, multiple sources shouldn't give any profit. - auto source = std::make_shared(column_names, data.begin(), data.size(), *this, metadata_snapshot); - source->delayInitialization(&data, &mutex); + auto source = std::make_shared(column_names, current_data->begin(), current_data->size(), *this, metadata_snapshot); + source->delayInitialization(current_data); return Pipe(std::move(source)); } - size_t size = data.size(); + size_t size = current_data->size(); if (num_streams > size) num_streams = size; Pipes pipes; - BlocksList::iterator it = data.begin(); + auto it = current_data->begin(); size_t offset = 0; for (size_t stream = 0; stream < num_streams; ++stream) @@ -200,11 +194,10 @@ BlockOutputStreamPtr StorageMemory::write(const ASTPtr & /*query*/, const Storag void StorageMemory::drop() { - std::lock_guard lock(mutex); - data.clear(); + data.set(std::make_unique()); } -static inline void columnUpdate(Block & old_block, const Block & new_block) +static inline void updateBlockData(Block & old_block, const Block & new_block) { for (const auto & it : new_block) { @@ -231,38 +224,37 @@ void StorageMemory::mutate(const MutationCommands & commands, const Context & co } in->readSuffix(); - std::lock_guard lock(mutex); - // all column affected if (interpreter->isAffectingAllColumns()) { - std::swap(data, out); + data.set(std::make_unique(out)); } else { - auto data_it = data.begin(); + auto new_data = std::make_unique(*(data.get())); + auto data_it = new_data->begin(); auto out_it = out.begin(); - while (data_it != data.end() && out_it != out.end()) + while (data_it != new_data->end() && out_it != out.end()) { - columnUpdate(*data_it, *out_it); + updateBlockData(*data_it, *out_it); ++data_it; ++out_it; } + data.set(std::move(new_data)); } } void StorageMemory::truncate( const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) { - std::lock_guard lock(mutex); - data.clear(); + data.set(std::make_unique()); } std::optional StorageMemory::totalRows() const { UInt64 rows = 0; - std::lock_guard lock(mutex); - for (const auto & buffer : data) + auto current_data = data.get(); + for (const auto & buffer : *current_data) rows += buffer.rows(); return rows; } @@ -270,8 +262,8 @@ std::optional StorageMemory::totalRows() const std::optional StorageMemory::totalBytes() const { UInt64 bytes = 0; - std::lock_guard lock(mutex); - for (const auto & buffer : data) + auto current_data = data.get(); + for (const auto & buffer : *current_data) bytes += buffer.allocatedBytes(); return bytes; } diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index f0d5d7aadec..a3cc00a88eb 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -8,6 +8,7 @@ #include #include +#include namespace DB { @@ -26,7 +27,7 @@ friend struct ext::shared_ptr_helper; public: String getName() const override { return "Memory"; } - size_t getSize() const { return data.size(); } + size_t getSize() const { return data.get()->size(); } Pipe read( const Names & column_names, @@ -88,10 +89,8 @@ public: void delayReadForGlobalSubqueries() { delay_read_for_global_subqueries = true; } private: - /// The data itself. `list` - so that when inserted to the end, the existing iterators are not invalidated. - BlocksList data; - - mutable std::mutex mutex; + /// MultiVersion data storage, so that we can copy the list of blocks to readers. + MultiVersion data; bool delay_read_for_global_subqueries = false; From 5400a412db9bdba5fcfce0763090b4b2991075da Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 5 Oct 2020 00:36:43 +0800 Subject: [PATCH 0018/1088] add test --- ...ltiversion_storage_for_storagememory.reference | 10 ++++++++++ ...507_multiversion_storage_for_storagememory.sql | 15 +++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/01507_multiversion_storage_for_storagememory.reference create mode 100644 tests/queries/0_stateless/01507_multiversion_storage_for_storagememory.sql diff --git a/tests/queries/0_stateless/01507_multiversion_storage_for_storagememory.reference b/tests/queries/0_stateless/01507_multiversion_storage_for_storagememory.reference new file mode 100644 index 00000000000..8b1acc12b63 --- /dev/null +++ b/tests/queries/0_stateless/01507_multiversion_storage_for_storagememory.reference @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/01507_multiversion_storage_for_storagememory.sql b/tests/queries/0_stateless/01507_multiversion_storage_for_storagememory.sql new file mode 100644 index 00000000000..fec9105cc6b --- /dev/null +++ b/tests/queries/0_stateless/01507_multiversion_storage_for_storagememory.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS defaults; +CREATE TABLE defaults +( + n Int32 +)ENGINE = Memory(); + +INSERT INTO defaults SELECT * FROM numbers(10); + +SELECT * FROM defaults; + +TRUNCATE defaults; + +SELECT * FROM defaults; + +DROP TABLE defaults; From 3bcb9b8db5dd4745f0c83940bbe11d21394754c1 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 8 Oct 2020 17:18:56 +0800 Subject: [PATCH 0019/1088] fix --- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMemory.h | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 4231e45c74d..ad67696adaa 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -159,7 +159,7 @@ Pipe StorageMemory::read( metadata_snapshot, /// This hack is needed for global subqueries. /// It allows to set up this Source for read AFTER Storage::read() has been called and just before actual reading - [this](BlocksList::const_iterator & current_it, size_t & num_blocks) { + [this, ¤t_data](BlocksList::const_iterator & current_it, size_t & num_blocks) { std::lock_guard guard(mutex); current_it = current_data->begin(); num_blocks = current_data->size(); diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index a443d73a459..a1e98dcdedf 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -94,6 +94,8 @@ private: /// MultiVersion data storage, so that we can copy the list of blocks to readers. MultiVersion data; + mutable std::mutex mutex; + bool delay_read_for_global_subqueries = false; std::atomic total_size_bytes = 0; From ff1a3146091ae1ec7c03a7cef4d71e48fde126e2 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 8 Oct 2020 19:15:11 +0800 Subject: [PATCH 0020/1088] fix --- src/Storages/StorageMemory.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index ad67696adaa..46d76fc1f1e 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -139,8 +139,6 @@ Pipe StorageMemory::read( { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - auto current_data = data.get(); - if (delay_read_for_global_subqueries) { /// Note: for global subquery we use single source. @@ -153,19 +151,21 @@ Pipe StorageMemory::read( return Pipe(std::make_shared( column_names, - current_data->end(), + data.get()->end(), 0, *this, metadata_snapshot, /// This hack is needed for global subqueries. /// It allows to set up this Source for read AFTER Storage::read() has been called and just before actual reading - [this, ¤t_data](BlocksList::const_iterator & current_it, size_t & num_blocks) { + [this](BlocksList::const_iterator & current_it, size_t & num_blocks) { std::lock_guard guard(mutex); - current_it = current_data->begin(); - num_blocks = current_data->size(); + current_it = data.get()->begin(); + num_blocks = data.get()->size(); })); } + auto current_data = data.get(); + size_t size = current_data->size(); if (num_streams > size) @@ -205,7 +205,7 @@ BlockOutputStreamPtr StorageMemory::write(const ASTPtr & /*query*/, const Storag void StorageMemory::drop() { std::lock_guard lock(mutex); - data.set(std::make_unique()); + data.set(std::make_unique()); total_size_bytes.store(0, std::memory_order_relaxed); total_size_rows.store(0, std::memory_order_relaxed); } From ae544940ce6284f6c8a0399e15c4cb04ac97bfc8 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 8 Oct 2020 19:16:53 +0800 Subject: [PATCH 0021/1088] format --- src/Storages/StorageMemory.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 46d76fc1f1e..a20d8370244 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -157,7 +157,8 @@ Pipe StorageMemory::read( metadata_snapshot, /// This hack is needed for global subqueries. /// It allows to set up this Source for read AFTER Storage::read() has been called and just before actual reading - [this](BlocksList::const_iterator & current_it, size_t & num_blocks) { + [this](BlocksList::const_iterator & current_it, size_t & num_blocks) + { std::lock_guard guard(mutex); current_it = data.get()->begin(); num_blocks = data.get()->size(); From 677787ff0e867a11b5ab312b883fd4162416b04c Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 8 Oct 2020 19:27:13 +0800 Subject: [PATCH 0022/1088] fix fix --- src/Storages/StorageMemory.cpp | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index a20d8370244..b6b6cb6285b 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes class MemorySource : public SourceWithProgress { - using InitializerFunc = std::function; + using InitializerFunc = std::function &)>; public: /// Blocks are stored in std::list which may be appended in another thread. /// We use pointer to the beginning of the list and its current size. @@ -35,11 +35,13 @@ public: size_t num_blocks_, const StorageMemory & storage, const StorageMetadataPtr & metadata_snapshot, - InitializerFunc initializer_func_ = [](BlocksList::const_iterator &, size_t &) {}) + std::shared_ptr data_, + InitializerFunc initializer_func_ = [](BlocksList::const_iterator &, size_t &, std::shared_ptr &) {}) : SourceWithProgress(metadata_snapshot->getSampleBlockForColumns(column_names_, storage.getVirtuals(), storage.getStorageID())) , column_names(std::move(column_names_)) , current_it(first_) , num_blocks(num_blocks_) + , data(data_) , initializer_func(std::move(initializer_func_)) { } @@ -51,7 +53,7 @@ protected: { if (!postponed_init_done) { - initializer_func(current_it, num_blocks); + initializer_func(current_it, num_blocks, data); postponed_init_done = true; } @@ -78,6 +80,7 @@ private: size_t num_blocks; size_t current_block_idx = 0; + std::shared_ptr data; bool postponed_init_done = false; InitializerFunc initializer_func; }; @@ -155,13 +158,12 @@ Pipe StorageMemory::read( 0, *this, metadata_snapshot, - /// This hack is needed for global subqueries. - /// It allows to set up this Source for read AFTER Storage::read() has been called and just before actual reading - [this](BlocksList::const_iterator & current_it, size_t & num_blocks) + data.get(), + [this](BlocksList::const_iterator & current_it, size_t & num_blocks, std::shared_ptr & current_data) { - std::lock_guard guard(mutex); - current_it = data.get()->begin(); - num_blocks = data.get()->size(); + current_data = data.get(); + current_it = current_data->begin(); + num_blocks = current_data->size(); })); } @@ -184,7 +186,7 @@ Pipe StorageMemory::read( assert(num_blocks > 0); - pipes.emplace_back(std::make_shared(column_names, it, num_blocks, *this, metadata_snapshot)); + pipes.emplace_back(std::make_shared(column_names, it, num_blocks, *this, metadata_snapshot, current_data)); while (offset < next_offset) { From 09862fb1baa54b58f23ca688eb264ec5f6e7db3a Mon Sep 17 00:00:00 2001 From: myrrc Date: Mon, 12 Oct 2020 14:17:35 +0300 Subject: [PATCH 0023/1088] typo --- tests/clickhouse-test | 37 ++++++++++++++++++++++++++----------- 1 file changed, 26 insertions(+), 11 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 2a9c95eb830..52a1d63737f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -112,7 +112,6 @@ def get_db_engine(args): return "" # Will use default engine def run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file): - # print(client_options) if args.database: @@ -149,10 +148,12 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std pattern = "{client} --send_logs_level={logs_level} --testmode --multiquery {options} < " + pattern command = pattern.format(**params) - #print(command) + + print(command) proc = Popen(command, shell=True, env=os.environ) start_time = datetime.now() + while (datetime.now() - start_time).total_seconds() < args.timeout and proc.poll() is None: sleep(0.01) @@ -314,6 +315,7 @@ def run_tests_array(all_tests_with_params): stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr' proc, stdout, stderr, total_time = run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file) + if proc.returncode is None: try: proc.kill() @@ -344,7 +346,7 @@ def run_tests_array(all_tests_with_params): if stderr: print(stderr.encode('utf-8')) - # Stop on fatal errors like segmentation fault. They are send to client via logs. + # Stop on fatal errors like segmentation fault. They are sent to client via logs. if ' ' in stderr: SERVER_DIED = True @@ -481,7 +483,7 @@ def collect_build_flags(client): elif '-fsanitize=memory' in stdout: result.append(BuildFlags.MEMORY) else: - raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) + raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) (stdout, stderr) = clickhouse_proc.communicate("SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'") @@ -492,7 +494,7 @@ def collect_build_flags(client): elif 'RelWithDebInfo' in stdout or 'Release' in stdout: result.append(BuildFlags.RELEASE) else: - raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) + raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) (stdout, stderr) = clickhouse_proc.communicate("SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'") @@ -501,7 +503,7 @@ def collect_build_flags(client): if 'ON' in stdout or '1' in stdout: result.append(BuildFlags.UNBUNDLED) else: - raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) + raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) (stdout, stderr) = clickhouse_proc.communicate("SELECT value FROM system.settings WHERE name = 'default_database_engine'") @@ -510,7 +512,7 @@ def collect_build_flags(client): if 'Ordinary' in stdout: result.append(BuildFlags.DATABASE_ORDINARY) else: - raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) + raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) return result @@ -530,8 +532,12 @@ def main(args): return stdout.startswith('1') if not check_server_started(args.client, args.server_check_retries): - raise Exception("clickhouse-server is not responding. Cannot execute 'SELECT 1' query.") + raise Exception( + "Server is not responding. Cannot execute 'SELECT 1' query. \ + Note: if you are using unbundled mode, you also have to specify -c option.") + build_flags = collect_build_flags(args.client) + if args.use_skip_list: tests_to_skip_from_list = collect_tests_to_skip(args.skip_list_path, build_flags) else: @@ -776,8 +782,13 @@ if __name__ == '__main__': parser=ArgumentParser(description='ClickHouse functional tests') parser.add_argument('-q', '--queries', help='Path to queries dir') parser.add_argument('--tmp', help='Path to tmp dir') - parser.add_argument('-b', '--binary', default='clickhouse', help='Path to clickhouse binary or name of binary in PATH') - parser.add_argument('-c', '--client', help='Client program') + + parser.add_argument('-b', '--binary', default='clickhouse', + help='Path to clickhouse (if bundled, clickhouse-server otherwise) binary or name of binary in PATH') + + parser.add_argument('-c', '--client', + help='Path to clickhouse-client (if unbundled, useless otherwise) binary of name of binary in PATH') + parser.add_argument('--extract_from_config', help='extract-from-config program') parser.add_argument('--configclient', help='Client config (if you use not default ports)') parser.add_argument('--configserver', default= '/etc/clickhouse-server/config.xml', help='Preprocessed server config') @@ -851,10 +862,14 @@ if __name__ == '__main__': if args.client is None: if find_binary(args.binary + '-client'): args.client = args.binary + '-client' + + print("Using " + args.client + " as client program (expecting unbundled mode)") elif find_binary(args.binary): args.client = args.binary + ' client' + + print("Using " + args.client + " as client program (expecting bundled mode)") else: - print("No 'clickhouse' binary found in PATH", file=sys.stderr) + print("No 'clickhouse' or 'clickhouse-client' client binary found", file=sys.stderr) parser.print_help() exit(1) From b50397ce38d87e599cf4fc1ede254bd6665242a6 Mon Sep 17 00:00:00 2001 From: myrrc Date: Mon, 12 Oct 2020 15:38:52 +0300 Subject: [PATCH 0024/1088] added binary modes output in cmakelists --- programs/CMakeLists.txt | 70 ++++++++++++++++++++++++++++++++++++++++- 1 file changed, 69 insertions(+), 1 deletion(-) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 3577ee3df31..6096df131b1 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -43,13 +43,81 @@ else () ${ENABLE_CLICKHOUSE_ALL}) endif () +message(STATUS "ClickHouse modes:") + +if (NOT ENABLE_CLICKHOUSE_SERVER) + message(WARNING "ClickHouse server mode is not going to be built.") +else() + message(STATUS "Server mode: ON") +endif() + +if (NOT ENABLE_CLICKHOUSE_CLIENT) + message(WARNING "ClickHouse client mode is not going to be built. You won't be able to connect to the server and run + tests") +else() + message(STATUS "Client mode: ON") +endif() + +if (ENABLE_CLICKHOUSE_LOCAL) + message(STATUS "Local mode: ON") +else() + message(STATUS "Local mode: OFF") +endif() + +if (ENABLE_CLICKHOUSE_BENCHMARK) + message(STATUS "Benchmark mode: ON") +else() + message(STATUS "Benchmark mode: OFF") +endif() + +if (ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG) + message(STATUS "Extract from config mode: ON") +else() + message(STATUS "Extract from config mode: OFF") +endif() + +if (ENABLE_CLICKHOUSE_COMPRESSOR) + message(STATUS "Compressor mode: ON") +else() + message(STATUS "Compressor mode: OFF") +endif() + +if (ENABLE_CLICKHOUSE_COPIER) + message(STATUS "Copier mode: ON") +else() + message(STATUS "Copier mode: OFF") +endif() + +if (ENABLE_CLICKHOUSE_FORMAT) + message(STATUS "Format mode: ON") +else() + message(STATUS "Format mode: OFF") +endif() + +if (ENABLE_CLICKHOUSE_OBFUSCATOR) + message(STATUS "Obfuscator mode: ON") +else() + message(STATUS "Obfuscator mode: OFF") +endif() + +if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) + message(STATUS "ODBC bridge mode: ON") +else() + message(STATUS "ODBC bridge mode: OFF") +endif() + +if (ENABLE_CLICKHOUSE_INSTALL) + message(STATUS "ClickHouse install: ON") +else() + message(STATUS "ClickHouse install: OFF") +endif() + if(NOT (MAKE_STATIC_LIBRARIES OR SPLIT_SHARED_LIBRARIES)) set(CLICKHOUSE_ONE_SHARED ON) endif() configure_file (config_tools.h.in ${ConfigIncludePath}/config_tools.h) - macro(clickhouse_target_link_split_lib target name) if(NOT CLICKHOUSE_ONE_SHARED) target_link_libraries(${target} PRIVATE clickhouse-${name}-lib) From 53471315ff18871cfba2e7a21bba75fd56781ca9 Mon Sep 17 00:00:00 2001 From: myrrc Date: Mon, 12 Oct 2020 17:42:19 +0300 Subject: [PATCH 0025/1088] updated docs -- removed the unbundled option for... Default values in Cmake in ClickHouse, as there could be some caveats (e.g. tester not working) --- docs/_includes/cmake_in_clickhouse_header.md | 2 +- .../sql-reference/aggregate-functions/reference/avgweighted.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/_includes/cmake_in_clickhouse_header.md b/docs/_includes/cmake_in_clickhouse_header.md index 10776e04c01..7dfda35e34a 100644 --- a/docs/_includes/cmake_in_clickhouse_header.md +++ b/docs/_includes/cmake_in_clickhouse_header.md @@ -13,9 +13,9 @@ cmake .. \ -DENABLE_CLICKHOUSE_SERVER=ON \ -DENABLE_CLICKHOUSE_CLIENT=ON \ -DUSE_STATIC_LIBRARIES=OFF \ - -DCLICKHOUSE_SPLIT_BINARY=ON \ -DSPLIT_SHARED_LIBRARIES=ON \ -DENABLE_LIBRARIES=OFF \ + -DUSE_UNWIND=ON \ -DENABLE_UTILS=OFF \ -DENABLE_TESTS=OFF ``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md index 20b7187a744..d0064c3007d 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md @@ -17,7 +17,7 @@ avgWeighted(x, weight) - `x` — Values. [Integer](../../../sql-reference/data-types/int-uint.md) or [floating-point](../../../sql-reference/data-types/float.md). - `weight` — Weights of the values. [Integer](../../../sql-reference/data-types/int-uint.md) or [floating-point](../../../sql-reference/data-types/float.md). -Type of `x` and `weight` must be the same. +Type of `x` and `weight` may be different. **Returned value** From 77e5a63648ef23cb182385a1059aa58d238536ae Mon Sep 17 00:00:00 2001 From: Kevin Chiang Date: Mon, 12 Oct 2020 12:58:38 -0700 Subject: [PATCH 0026/1088] Fix a markdown issue that causes a section of the note to appear outside of the Note box --- docs/en/sql-reference/statements/alter/index/index.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/index/index.md b/docs/en/sql-reference/statements/alter/index/index.md index 4660478551f..6b00696e07f 100644 --- a/docs/en/sql-reference/statements/alter/index/index.md +++ b/docs/en/sql-reference/statements/alter/index/index.md @@ -19,5 +19,4 @@ The first two commands areare lightweight in a sense that they only change metad Also, they are replicated, syncing indices metadata via ZooKeeper. !!! note "Note" - Index manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including -[replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants). + Index manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants). From 74bdc7e645df799d35fdb32fbe77c481c89a390a Mon Sep 17 00:00:00 2001 From: Kevin Chiang Date: Mon, 12 Oct 2020 12:59:18 -0700 Subject: [PATCH 0027/1088] MergeTrees can have data skipping indexes, removing section from architecture to avoid confusion. --- docs/en/development/architecture.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/development/architecture.md b/docs/en/development/architecture.md index da5b4470704..c86818c7a1a 100644 --- a/docs/en/development/architecture.md +++ b/docs/en/development/architecture.md @@ -177,8 +177,6 @@ When you `INSERT` a bunch of data into `MergeTree`, that bunch is sorted by prim `MergeTree` is not an LSM tree because it doesn’t contain “memtable” and “log”: inserted data is written directly to the filesystem. This makes it suitable only to INSERT data in batches, not by individual row and not very frequently – about once per second is ok, but a thousand times a second is not. We did it this way for simplicity’s sake, and because we are already inserting data in batches in our applications. -> MergeTree tables can only have one (primary) index: there aren’t any secondary indices. It would be nice to allow multiple physical representations under one logical table, for example, to store data in more than one physical order or even to allow representations with pre-aggregated data along with original data. - There are MergeTree engines that are doing additional work during background merges. Examples are `CollapsingMergeTree` and `AggregatingMergeTree`. This could be treated as special support for updates. Keep in mind that these are not real updates because users usually have no control over the time when background merges are executed, and data in a `MergeTree` table is almost always stored in more than one part, not in completely merged form. ## Replication {#replication} From 6d9b310f61ad4102421cc7a3d7a2bc725f66ee34 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 13 Oct 2020 17:51:08 +0300 Subject: [PATCH 0028/1088] Add optimize_skip_merged_partitions --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 7 +++++++ src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 1 + src/Storages/MergeTree/MergeTreeSettings.h | 1 + src/Storages/StorageMergeTree.cpp | 7 ++++++- src/Storages/StorageReplicatedMergeTree.cpp | 13 ++++++++++--- 5 files changed, 25 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index fb0a488700c..3c1a40342de 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -358,6 +358,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( const AllowedMergingPredicate & can_merge, const String & partition_id, bool final, + bool * is_single_merged_part, String * out_disable_reason) { MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id); @@ -372,6 +373,12 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( return false; } + if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0) + { + *is_single_merged_part = true; + return false; + } + auto it = parts.begin(); auto prev_it = it; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 0ad525d1901..4118c09fa89 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -96,6 +96,7 @@ public: const AllowedMergingPredicate & can_merge, const String & partition_id, bool final, + bool * is_single_merged_part, String * out_disable_reason = nullptr); /** Merge the parts. diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 97bc73caf5b..c074c6b35b4 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -48,6 +48,7 @@ struct Settings; M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Amount of bytes, accumulated in WAL to do fsync.", 0) \ M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Interval in milliseconds after which fsync for WAL is being done.", 0) \ M(Bool, in_memory_parts_insert_sync, false, "If true insert of part with in-memory format will wait for fsync of WAL", 0) \ + M(Bool, optimize_skip_merged_partitions, true, "Skip partitions with one part with level > 0 in optimize final", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 0f1afe1bd62..6826ab8f220 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -654,6 +654,7 @@ bool StorageMergeTree::merge( }; bool selected = false; + bool is_single_merged_part = false; if (partition_id.empty()) { @@ -682,7 +683,7 @@ bool StorageMergeTree::merge( { UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_part, disk_space, can_merge, partition_id, final, out_disable_reason); + future_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, out_disable_reason); /// If final - we will wait for currently processing merges to finish and continue. /// TODO Respect query settings for timeout @@ -709,6 +710,10 @@ bool StorageMergeTree::merge( if (!selected) { + if (final && is_single_merged_part) + { + return true; + } if (out_disable_reason) { if (!out_disable_reason->empty()) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2ac8ddb7846..f8122f59046 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3761,11 +3761,16 @@ bool StorageReplicatedMergeTree::optimize( ReplicatedMergeTreeMergePredicate can_merge = queue.getMergePredicate(zookeeper); FutureMergedMutatedPart future_merged_part; + bool is_single_merged_part = false; bool selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, true, nullptr); + future_merged_part, disk_space, can_merge, partition_id, true, &is_single_merged_part, nullptr); if (!selected) + { + if (is_single_merged_part) + return true; break; + } ReplicatedMergeTreeLogEntryData merge_entry; CreateMergeEntryResult create_result = createLogEntryToMergeParts( @@ -3798,6 +3803,7 @@ bool StorageReplicatedMergeTree::optimize( FutureMergedMutatedPart future_merged_part; String disable_reason; bool selected = false; + bool is_single_merged_part = false; if (!partition) { selected = merger_mutator.selectPartsToMerge( @@ -3805,15 +3811,16 @@ bool StorageReplicatedMergeTree::optimize( } else { - UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); String partition_id = getPartitionIDFromQuery(partition, query_context); selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, final, &disable_reason); + future_merged_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, &disable_reason); } if (!selected) { + if (final && is_single_merged_part) + return true; std::stringstream message; message << "Cannot select parts for optimization"; if (!disable_reason.empty()) From 1ccf10505acfc9d564af9e83670156855bab7c4c Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 13 Oct 2020 21:25:45 +0300 Subject: [PATCH 0029/1088] add tests and comments --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 +++++- .../MergeTree/MergeTreeDataMergerMutator.h | 4 +++- src/Storages/StorageMergeTree.cpp | 4 ++++ src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++++++- ...3_optimize_skip_merged_partitions.reference | 2 ++ .../01533_optimize_skip_merged_partitions.sql | 18 ++++++++++++++++++ 6 files changed, 41 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01533_optimize_skip_merged_partitions.reference create mode 100644 tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 3c1a40342de..ac08cd31dd2 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -358,7 +358,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( const AllowedMergingPredicate & can_merge, const String & partition_id, bool final, - bool * is_single_merged_part, + bool * is_single_merged_part, String * out_disable_reason) { MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id); @@ -373,6 +373,8 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( return false; } + /// If final, optimize_skip_merged_partitions is true and we have only one part in partition with level > 0 + /// than we don't select it to merge if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0) { *is_single_merged_part = true; @@ -637,6 +639,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor { static const String TMP_PREFIX = "tmp_merge_"; + + if (merges_blocker.isCancelled()) throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 4118c09fa89..481fa24637b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -88,7 +88,9 @@ public: String * out_disable_reason = nullptr); /** Select all the parts in the specified partition for merge, if possible. - * final - choose to merge even a single part - that is, allow to merge one part "with itself". + * final - choose to merge even a single part - that is, allow to merge one part "with itself", + * but if setting optimize_skip_merged_partitions is true (it's true as default) than single part with level > 0 + * won't be merged with itself. */ bool selectAllPartsToMergeWithinPartition( FutureMergedMutatedPart & future_part, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 6826ab8f220..574c65502f1 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -654,6 +654,9 @@ bool StorageMergeTree::merge( }; bool selected = false; + + /// This flag is true when there is only one part in partition, it's level > 0 + /// and setting optimize_skip_merged_partitions is true bool is_single_merged_part = false; if (partition_id.empty()) @@ -710,6 +713,7 @@ bool StorageMergeTree::merge( if (!selected) { + /// If is_single_merged_part is true we treat this part as already merged if (final && is_single_merged_part) { return true; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f8122f59046..19ce88ad0b8 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3761,12 +3761,16 @@ bool StorageReplicatedMergeTree::optimize( ReplicatedMergeTreeMergePredicate can_merge = queue.getMergePredicate(zookeeper); FutureMergedMutatedPart future_merged_part; + + /// This flag is true when there is only one part in partition, it's level > 0 + /// and setting optimize_skip_merged_partitions is true bool is_single_merged_part = false; bool selected = merger_mutator.selectAllPartsToMergeWithinPartition( future_merged_part, disk_space, can_merge, partition_id, true, &is_single_merged_part, nullptr); if (!selected) { + /// If is_single_merged_part is true we treat this part as already merged if (is_single_merged_part) return true; break; @@ -3789,7 +3793,7 @@ bool StorageReplicatedMergeTree::optimize( } if (try_no == max_retries) return handle_noop("Can't create merge queue node in ZooKeeper, because log was updated in every of " - + toString(max_retries) + " tries"); + + toString(max_retries) + " tries"); } } else @@ -3803,6 +3807,9 @@ bool StorageReplicatedMergeTree::optimize( FutureMergedMutatedPart future_merged_part; String disable_reason; bool selected = false; + + /// This flag is true when there is only one part in partition, it's level > 0 + /// and setting optimize_skip_merged_partitions is true bool is_single_merged_part = false; if (!partition) { @@ -3819,6 +3826,7 @@ bool StorageReplicatedMergeTree::optimize( if (!selected) { + /// If is_single_merged_part is true we treat this part as already merged if (final && is_single_merged_part) return true; std::stringstream message; diff --git a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.reference b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.reference new file mode 100644 index 00000000000..300e1103c0f --- /dev/null +++ b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.reference @@ -0,0 +1,2 @@ +optimize_final 200001 1 1 +optimize_final 202001 1 1 diff --git a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql new file mode 100644 index 00000000000..325aaa2d6bb --- /dev/null +++ b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS optimize_final; + +CREATE TABLE optimize_final(t DateTime, x Int32) ENGINE = MergeTree() PARTITION BY toYYYYMM(t) ORDER BY x; + +INSERT INTO optimize_final SELECT toDate('2000-01-01'), number FROM numbers(5); +INSERT INTO optimize_final SELECT toDate('2000-01-01'), number + 5 FROM numbers(5); + +OPTIMIZE TABLE optimize_final FINAL; + +INSERT INTO optimize_final SELECT toDate('2020-01-01'), number FROM numbers(5); +INSERT INTO optimize_final SELECT toDate('2020-01-01'), number + 5 FROM numbers(5); + +OPTIMIZE TABLE optimize_final FINAL; + +SELECT table, partition, active, level from system.parts where table = 'optimize_final' and active = 1; + +DROP TABLE optimize_final; + From 51807599e7e438d5c033db87f9ded0921dc7903b Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 13 Oct 2020 23:10:37 +0300 Subject: [PATCH 0030/1088] fix style --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index ac08cd31dd2..4b28dbf9949 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -639,8 +639,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor { static const String TMP_PREFIX = "tmp_merge_"; - - if (merges_blocker.isCancelled()) throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); From 8e7e232387fe19bc44ffaf39cb6057d3bb5e2283 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 15 Oct 2020 13:36:00 +0300 Subject: [PATCH 0031/1088] wip dealing with template magic --- .../reference/avgweighted.md | 17 +++ .../AggregateFunctionAvgWeighted.cpp | 48 ++++---- .../AggregateFunctionAvgWeighted.h | 21 ++-- src/DataTypes/IDataType.h | 107 ++++++++---------- tests/clickhouse-test | 2 +- .../queries/0_stateless/01035_avg_weighted.sh | 17 ++- 6 files changed, 121 insertions(+), 91 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md index d0064c3007d..8dd2ecf4616 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md @@ -42,3 +42,20 @@ Result: │ 8 │ └────────────────────────┘ ``` + +**Example** + +Query: + +``` sql +SELECT avgWeighted(x, w) +FROM values('x Int8, w Float64', (4, 1), (1, 0), (10, 2)) +``` + +Result: + +``` text +┌─avgWeighted(x, weight)─┐ +│ 8 │ +└────────────────────────┘ +``` diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp index 10698332a14..95ba80f1cb1 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp @@ -13,26 +13,37 @@ namespace ErrorCodes namespace { - -template -struct AvgWeighted +constexpr bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) { - using FieldType = std::conditional_t< - IsDecimalNumber, - std::conditional_t, - Decimal256, - Decimal128>, - NearestFieldType>; + const WhichDataType l_dt(left), r_dt(right); - using Function = AggregateFunctionAvgWeighted>; + constexpr auto allow = [](WhichDataType t) + { + return t.isInt() || t.isUInt() || t.isFloat() || t.isDecimal(); + }; + + return allow(l_dt) && allow(r_dt); +} + +template struct BiggerType + +template struct LargestType +{ + using Type = bool; }; -template -using AggregateFuncAvgWeighted = typename AvgWeighted::Function; -bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) +template using AvgData = AggregateFunctionAvgData< + typename LargestType::Type, + typename LargestType::Type>; + +template using Function = AggregateFunctionAvgWeighted< + U, V, typename LargestType::Type, AvgData>; + +template +static IAggregateFunction * create(const IDataType & first_type, const IDataType & second_type, TArgs && ... args) { - return (isInteger(left) || isFloat(left)) && (isInteger(right) || isFloat(right)); + } AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name, const DataTypes & argument_types, const Array & parameters) @@ -40,8 +51,6 @@ AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name assertNoParameters(name, parameters); assertBinary(name, argument_types); - AggregateFunctionPtr res; - const auto data_type = static_cast(argument_types[0]); const auto data_type_weight = static_cast(argument_types[1]); @@ -52,10 +61,8 @@ AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name " are non-conforming as arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (isDecimal(data_type)) - res.reset(createWithDecimalType(*data_type, *data_type, argument_types)); - else - res.reset(createWithNumericType(*data_type, argument_types)); + AggregateFunctionPtr res; + res.reset(create(*data_type, *data_type_weight, argument_types)); if (!res) throw Exception("Illegal type " + data_type->getName() + " of argument for aggregate function " + name, @@ -70,5 +77,4 @@ void registerAggregateFunctionAvgWeighted(AggregateFunctionFactory & factory) { factory.registerFunction("avgWeighted", createAggregateFunctionAvgWeighted, AggregateFunctionFactory::CaseSensitive); } - } diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index 53a0ae50fb4..7ffdf41cfd9 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -4,21 +4,28 @@ namespace DB { -template -class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase> + +template +class AggregateFunctionAvgWeighted final : + public AggregateFunctionAvgBase> { public: - using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; + using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; + template using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - const auto & values = static_cast(*columns[0]); - const auto & weights = static_cast(*columns[1]); + const auto & values = static_cast &>(*columns[0]); + const auto & weights = static_cast &>(*columns[1]); - this->data(place).numerator += static_cast(values.getData()[row_num]) * weights.getData()[row_num]; - this->data(place).denominator += weights.getData()[row_num]; + const auto value = values.getData()[row_num]; + const auto weight = weights.getData()[row_num]; + + this->data(place).numerator += static_cast(value) * weight; + this->data(place).denominator += weight; } String getName() const override { return "avgWeighted"; } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 5e25d47534e..797b05d2ef3 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -466,75 +466,64 @@ struct WhichDataType { TypeIndex idx; - WhichDataType(TypeIndex idx_ = TypeIndex::Nothing) - : idx(idx_) - {} + constexpr WhichDataType(TypeIndex idx_ = TypeIndex::Nothing) : idx(idx_) {} + constexpr WhichDataType(const IDataType & data_type) : idx(data_type.getTypeId()) {} + constexpr WhichDataType(const IDataType * data_type) : idx(data_type->getTypeId()) {} + constexpr WhichDataType(const DataTypePtr & data_type) : idx(data_type->getTypeId()) {} - WhichDataType(const IDataType & data_type) - : idx(data_type.getTypeId()) - {} + constexpr bool isUInt8() const { return idx == TypeIndex::UInt8; } + constexpr bool isUInt16() const { return idx == TypeIndex::UInt16; } + constexpr bool isUInt32() const { return idx == TypeIndex::UInt32; } + constexpr bool isUInt64() const { return idx == TypeIndex::UInt64; } + constexpr bool isUInt128() const { return idx == TypeIndex::UInt128; } + constexpr bool isUInt256() const { return idx == TypeIndex::UInt256; } + constexpr bool isUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64() || isUInt128() || isUInt256(); } + constexpr bool isNativeUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64(); } - WhichDataType(const IDataType * data_type) - : idx(data_type->getTypeId()) - {} + constexpr bool isInt8() const { return idx == TypeIndex::Int8; } + constexpr bool isInt16() const { return idx == TypeIndex::Int16; } + constexpr bool isInt32() const { return idx == TypeIndex::Int32; } + constexpr bool isInt64() const { return idx == TypeIndex::Int64; } + constexpr bool isInt128() const { return idx == TypeIndex::Int128; } + constexpr bool isInt256() const { return idx == TypeIndex::Int256; } + constexpr bool isInt() const { return isInt8() || isInt16() || isInt32() || isInt64() || isInt128() || isInt256(); } + constexpr bool isNativeInt() const { return isInt8() || isInt16() || isInt32() || isInt64(); } - WhichDataType(const DataTypePtr & data_type) - : idx(data_type->getTypeId()) - {} + constexpr bool isDecimal32() const { return idx == TypeIndex::Decimal32; } + constexpr bool isDecimal64() const { return idx == TypeIndex::Decimal64; } + constexpr bool isDecimal128() const { return idx == TypeIndex::Decimal128; } + constexpr bool isDecimal256() const { return idx == TypeIndex::Decimal256; } + constexpr bool isDecimal() const { return isDecimal32() || isDecimal64() || isDecimal128() || isDecimal256(); } - bool isUInt8() const { return idx == TypeIndex::UInt8; } - bool isUInt16() const { return idx == TypeIndex::UInt16; } - bool isUInt32() const { return idx == TypeIndex::UInt32; } - bool isUInt64() const { return idx == TypeIndex::UInt64; } - bool isUInt128() const { return idx == TypeIndex::UInt128; } - bool isUInt256() const { return idx == TypeIndex::UInt256; } - bool isUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64() || isUInt128() || isUInt256(); } - bool isNativeUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64(); } + constexpr bool isFloat32() const { return idx == TypeIndex::Float32; } + constexpr bool isFloat64() const { return idx == TypeIndex::Float64; } + constexpr bool isFloat() const { return isFloat32() || isFloat64(); } - bool isInt8() const { return idx == TypeIndex::Int8; } - bool isInt16() const { return idx == TypeIndex::Int16; } - bool isInt32() const { return idx == TypeIndex::Int32; } - bool isInt64() const { return idx == TypeIndex::Int64; } - bool isInt128() const { return idx == TypeIndex::Int128; } - bool isInt256() const { return idx == TypeIndex::Int256; } - bool isInt() const { return isInt8() || isInt16() || isInt32() || isInt64() || isInt128() || isInt256(); } - bool isNativeInt() const { return isInt8() || isInt16() || isInt32() || isInt64(); } + constexpr bool isEnum8() const { return idx == TypeIndex::Enum8; } + constexpr bool isEnum16() const { return idx == TypeIndex::Enum16; } + constexpr bool isEnum() const { return isEnum8() || isEnum16(); } - bool isDecimal32() const { return idx == TypeIndex::Decimal32; } - bool isDecimal64() const { return idx == TypeIndex::Decimal64; } - bool isDecimal128() const { return idx == TypeIndex::Decimal128; } - bool isDecimal256() const { return idx == TypeIndex::Decimal256; } - bool isDecimal() const { return isDecimal32() || isDecimal64() || isDecimal128() || isDecimal256(); } + constexpr bool isDate() const { return idx == TypeIndex::Date; } + constexpr bool isDateTime() const { return idx == TypeIndex::DateTime; } + constexpr bool isDateTime64() const { return idx == TypeIndex::DateTime64; } + constexpr bool isDateOrDateTime() const { return isDate() || isDateTime() || isDateTime64(); } - bool isFloat32() const { return idx == TypeIndex::Float32; } - bool isFloat64() const { return idx == TypeIndex::Float64; } - bool isFloat() const { return isFloat32() || isFloat64(); } + constexpr bool isString() const { return idx == TypeIndex::String; } + constexpr bool isFixedString() const { return idx == TypeIndex::FixedString; } + constexpr bool isStringOrFixedString() const { return isString() || isFixedString(); } - bool isEnum8() const { return idx == TypeIndex::Enum8; } - bool isEnum16() const { return idx == TypeIndex::Enum16; } - bool isEnum() const { return isEnum8() || isEnum16(); } + constexpr bool isUUID() const { return idx == TypeIndex::UUID; } + constexpr bool isArray() const { return idx == TypeIndex::Array; } + constexpr bool isTuple() const { return idx == TypeIndex::Tuple; } + constexpr bool isSet() const { return idx == TypeIndex::Set; } + constexpr bool isInterval() const { return idx == TypeIndex::Interval; } - bool isDate() const { return idx == TypeIndex::Date; } - bool isDateTime() const { return idx == TypeIndex::DateTime; } - bool isDateTime64() const { return idx == TypeIndex::DateTime64; } - bool isDateOrDateTime() const { return isDate() || isDateTime() || isDateTime64(); } + constexpr bool isNothing() const { return idx == TypeIndex::Nothing; } + constexpr bool isNullable() const { return idx == TypeIndex::Nullable; } + constexpr bool isFunction() const { return idx == TypeIndex::Function; } + constexpr bool isAggregateFunction() const { return idx == TypeIndex::AggregateFunction; } - bool isString() const { return idx == TypeIndex::String; } - bool isFixedString() const { return idx == TypeIndex::FixedString; } - bool isStringOrFixedString() const { return isString() || isFixedString(); } - - bool isUUID() const { return idx == TypeIndex::UUID; } - bool isArray() const { return idx == TypeIndex::Array; } - bool isTuple() const { return idx == TypeIndex::Tuple; } - bool isSet() const { return idx == TypeIndex::Set; } - bool isInterval() const { return idx == TypeIndex::Interval; } - - bool isNothing() const { return idx == TypeIndex::Nothing; } - bool isNullable() const { return idx == TypeIndex::Nullable; } - bool isFunction() const { return idx == TypeIndex::Function; } - bool isAggregateFunction() const { return idx == TypeIndex::AggregateFunction; } - - bool IsBigIntOrDeimal() const { return isInt128() || isInt256() || isUInt256() || isDecimal256(); } + constexpr bool IsBigIntOrDeimal() const { return isInt128() || isInt256() || isUInt256() || isDecimal256(); } }; /// IDataType helpers (alternative for IDataType virtual methods with single point of truth) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 2e0f6a70dca..3a2f9eee60a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -149,7 +149,7 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std command = pattern.format(**params) - print(command) + # print(command) proc = Popen(command, shell=True, env=os.environ) start_time = datetime.now() diff --git a/tests/queries/0_stateless/01035_avg_weighted.sh b/tests/queries/0_stateless/01035_avg_weighted.sh index 023ec50db2f..58ea962148b 100755 --- a/tests/queries/0_stateless/01035_avg_weighted.sh +++ b/tests/queries/0_stateless/01035_avg_weighted.sh @@ -3,9 +3,20 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 5), (2, 4), (3, 3), (4, 2), (5, 1)]) AS t));" ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 0), (2, 0), (3, 0), (4, 0), (5, 0)]) AS t));" -echo "$(${CLICKHOUSE_CLIENT} --server_logs_file=/dev/null --query="SELECT avgWeighted(toDecimal64(0, 0), toFloat64(0))" 2>&1)" \ - | grep -c 'Code: 43. DB::Exception: .* DB::Exception:.* Different types .* of arguments for aggregate function avgWeighted' +types=("Int8" "Int16" "Int32" "Int64" "UInt8" "UInt16" "UInt32" "UInt64" "Float32" "Float64") + +for left in "${types[@]}" +do + for right in "${types[@]}" + do + ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 1), (1, 0), (10, 2))" + ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 1), (1, 0))" + ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 0), (1, 0))" + done +done + +echo "$(${CLICKHOUSE_CLIENT} --server_logs_file=/dev/null --query="SELECT avgWeighted(['string'], toFloat64(0))" 2>&1)" \ + | grep -c 'Code: 43. DB::Exception: .* DB::Exception:.* Types .* of arguments are non-conforming as arguments for aggregate function avgWeighted' From 25c39689c37b3ebc71e2bb95576986c101da38c8 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 15 Oct 2020 15:15:02 +0300 Subject: [PATCH 0032/1088] Check expired ttl --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 8 ++++++-- src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 1 + src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 5 +++-- 4 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 4b28dbf9949..a5d0db47929 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -359,6 +359,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( const String & partition_id, bool final, bool * is_single_merged_part, + const StorageMetadataPtr & metadata_snapshot, String * out_disable_reason) { MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id); @@ -374,8 +375,9 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( } /// If final, optimize_skip_merged_partitions is true and we have only one part in partition with level > 0 - /// than we don't select it to merge - if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0) + /// than we don't select it to merge. But if there are some expired TTL then merge is needed + if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0 && + (!metadata_snapshot->hasAnyTTL() || parts[0]->checkAllTTLCalculated(metadata_snapshot))) { *is_single_merged_part = true; return false; @@ -639,6 +641,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor { static const String TMP_PREFIX = "tmp_merge_"; + + if (merges_blocker.isCancelled()) throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 481fa24637b..a3f240f86de 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -99,6 +99,7 @@ public: const String & partition_id, bool final, bool * is_single_merged_part, + const StorageMetadataPtr & metadata_snapshot, String * out_disable_reason = nullptr); /** Merge the parts. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 574c65502f1..69558b13d9b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -686,7 +686,7 @@ bool StorageMergeTree::merge( { UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, out_disable_reason); + future_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, metadata_snapshot, out_disable_reason); /// If final - we will wait for currently processing merges to finish and continue. /// TODO Respect query settings for timeout diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 19ce88ad0b8..09d36ddd4e4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3739,6 +3739,7 @@ bool StorageReplicatedMergeTree::optimize( }; const auto storage_settings_ptr = getSettings(); + auto metadata_snapshot = getInMemoryMetadataPtr(); if (!partition && final) { @@ -3766,7 +3767,7 @@ bool StorageReplicatedMergeTree::optimize( /// and setting optimize_skip_merged_partitions is true bool is_single_merged_part = false; bool selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, true, &is_single_merged_part, nullptr); + future_merged_part, disk_space, can_merge, partition_id, true, &is_single_merged_part, metadata_snapshot, nullptr); if (!selected) { @@ -3821,7 +3822,7 @@ bool StorageReplicatedMergeTree::optimize( UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); String partition_id = getPartitionIDFromQuery(partition, query_context); selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, &disable_reason); + future_merged_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, metadata_snapshot, &disable_reason); } if (!selected) From d6fc6d35fae91764da3d3e8d60405faeeaa6765b Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 15 Oct 2020 19:08:24 +0300 Subject: [PATCH 0033/1088] Fix style --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index a5d0db47929..624c5f156aa 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -641,8 +641,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor { static const String TMP_PREFIX = "tmp_merge_"; - - if (merges_blocker.isCancelled()) throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); From 6c1e0e3df8590784c29f97750ddbea5fe3901037 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 15 Oct 2020 19:40:20 +0300 Subject: [PATCH 0034/1088] Change default to false --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c074c6b35b4..91c9e7988bd 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -48,7 +48,7 @@ struct Settings; M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Amount of bytes, accumulated in WAL to do fsync.", 0) \ M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Interval in milliseconds after which fsync for WAL is being done.", 0) \ M(Bool, in_memory_parts_insert_sync, false, "If true insert of part with in-memory format will wait for fsync of WAL", 0) \ - M(Bool, optimize_skip_merged_partitions, true, "Skip partitions with one part with level > 0 in optimize final", 0) \ + M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ From 581bfa425331a0f29b8e7e90ba0a7db5e9b319a6 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 15 Oct 2020 22:15:40 +0300 Subject: [PATCH 0035/1088] Fix test --- .../0_stateless/01533_optimize_skip_merged_partitions.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql index 325aaa2d6bb..70f73d960fc 100644 --- a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql +++ b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS optimize_final; -CREATE TABLE optimize_final(t DateTime, x Int32) ENGINE = MergeTree() PARTITION BY toYYYYMM(t) ORDER BY x; +CREATE TABLE optimize_final(t DateTime, x Int32) ENGINE = MergeTree() PARTITION BY toYYYYMM(t) ORDER BY x SETTINGS optimize_skip_merged_partitions=1; INSERT INTO optimize_final SELECT toDate('2000-01-01'), number FROM numbers(5); INSERT INTO optimize_final SELECT toDate('2000-01-01'), number + 5 FROM numbers(5); From ffd3d2519eb51117f6c143122b2b13f32d1b2b8d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Oct 2020 03:13:17 +0300 Subject: [PATCH 0036/1088] Avoid stack overflow in materialized views, part 1 --- src/Storages/StorageMaterializedView.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 3e1df80ff42..8c7c6c7c9f1 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -21,6 +21,7 @@ #include #include +#include #include @@ -319,11 +320,13 @@ void StorageMaterializedView::shutdown() StoragePtr StorageMaterializedView::getTargetTable() const { + checkStackSize(); return DatabaseCatalog::instance().getTable(target_table_id, global_context); } StoragePtr StorageMaterializedView::tryGetTargetTable() const { + checkStackSize(); return DatabaseCatalog::instance().tryGetTable(target_table_id, global_context); } From 6c3aad34bec63250abce2b59def849ba18f31e1d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Oct 2020 03:19:45 +0300 Subject: [PATCH 0037/1088] Avoid stack overflow in materialized views, part 2: sanity check --- src/Storages/StorageMaterializedView.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 8c7c6c7c9f1..fd75807eb2e 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -30,6 +30,7 @@ namespace DB namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int NOT_IMPLEMENTED; extern const int INCORRECT_QUERY; extern const int QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW; @@ -72,7 +73,9 @@ StorageMaterializedView::StorageMaterializedView( setInMemoryMetadata(storage_metadata); if (!has_inner_table) + { target_table_id = query.to_table_id; + } else if (attach_) { /// If there is an ATTACH request, then the internal table must already be created. @@ -98,6 +101,9 @@ StorageMaterializedView::StorageMaterializedView( target_table_id = DatabaseCatalog::instance().getTable({manual_create_query->database, manual_create_query->table}, global_context)->getStorageID(); } + if (target_table_id == getStorageID()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Materialized view {} cannot point to itself", getStorageID().getFullTableName()); + if (!select.select_table_id.empty()) DatabaseCatalog::instance().addDependency(select.select_table_id, getStorageID()); } From cc3feb36a65a993c7a2a0838236e5035a011683c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Oct 2020 03:21:50 +0300 Subject: [PATCH 0038/1088] Avoid stack overflow in materialized views, part 3: more checks --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 3baa2b30c3f..1252dd7f4de 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -29,6 +30,8 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( , context(context_) , query_ptr(query_ptr_) { + checkStackSize(); + /** TODO This is a very important line. At any insertion into the table one of streams should own lock. * Although now any insertion into the table is done via PushingToViewsBlockOutputStream, * but it's clear that here is not the best place for this functionality. From 625d03d8c244d57dcd687a3dfdd177ce51dbe5c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Oct 2020 03:37:53 +0300 Subject: [PATCH 0039/1088] Avoid stack overflow in materialized views, part 4: correct checks --- src/Interpreters/StorageID.cpp | 9 +++++++++ src/Interpreters/StorageID.h | 1 + 2 files changed, 10 insertions(+) diff --git a/src/Interpreters/StorageID.cpp b/src/Interpreters/StorageID.cpp index 2d6a4900dd3..a7d02601dbf 100644 --- a/src/Interpreters/StorageID.cpp +++ b/src/Interpreters/StorageID.cpp @@ -79,6 +79,15 @@ bool StorageID::operator<(const StorageID & rhs) const return !hasUUID(); } +bool StorageID::operator==(const StorageID & rhs) const +{ + assertNotEmpty(); + if (!hasUUID() && !rhs.hasUUID()) + return std::tie(database_name, table_name) == std::tie(rhs.database_name, rhs.table_name); + else + return hasUUID() && rhs.hasUUID() && uuid == rhs.uuid; +} + String StorageID::getFullTableName() const { return backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name); diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index 9343f67fe7a..d42dfda06fd 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -68,6 +68,7 @@ struct StorageID return uuid != UUIDHelpers::Nil; } + bool operator==(const StorageID & rhs) const; bool operator<(const StorageID & rhs) const; void assertNotEmpty() const From 622cfafcb4a7dc1d9aa2b73624b385c2bd77468f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Oct 2020 03:45:05 +0300 Subject: [PATCH 0040/1088] Avoid stack overflow in materialized views, part 5: add a test --- ...materialized_view_stack_overflow.reference | 0 ...01527_materialized_view_stack_overflow.sql | 28 +++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 tests/queries/0_stateless/01527_materialized_view_stack_overflow.reference create mode 100644 tests/queries/0_stateless/01527_materialized_view_stack_overflow.sql diff --git a/tests/queries/0_stateless/01527_materialized_view_stack_overflow.reference b/tests/queries/0_stateless/01527_materialized_view_stack_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01527_materialized_view_stack_overflow.sql b/tests/queries/0_stateless/01527_materialized_view_stack_overflow.sql new file mode 100644 index 00000000000..4a67ef4b2d8 --- /dev/null +++ b/tests/queries/0_stateless/01527_materialized_view_stack_overflow.sql @@ -0,0 +1,28 @@ +DROP TABLE IF EXISTS t; +DROP TABLE IF EXISTS v; + +CREATE TABLE t (c String) ENGINE = Memory; + +CREATE MATERIALIZED VIEW v to v AS SELECT c FROM t; -- { serverError 36 } +CREATE MATERIALIZED VIEW v to t AS SELECT * FROM v; -- { serverError 60 } + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS v1; +DROP TABLE IF EXISTS v2; + +CREATE TABLE t1 (c String) ENGINE = Memory; +CREATE TABLE t2 (c String) ENGINE = Memory; + +CREATE MATERIALIZED VIEW v1 to t1 AS SELECT * FROM t2; +CREATE MATERIALIZED VIEW v2 to t2 AS SELECT * FROM t1; + +INSERT INTO t1 VALUES ('Hello'); -- { serverError 306 } +INSERT INTO t2 VALUES ('World'); -- { serverError 306 } + +DROP TABLE IF EXISTS t; +DROP TABLE IF EXISTS v; +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS v1; +DROP TABLE IF EXISTS v2; From 4bb700fc1b0031bf34186b2091265e84a555bff4 Mon Sep 17 00:00:00 2001 From: myrrc Date: Fri, 16 Oct 2020 23:15:38 +0300 Subject: [PATCH 0041/1088] cleaned up traits for AggregateFunction, added some dev docs --- .../AggregateFunctionAvg.cpp | 22 ++-- src/AggregateFunctions/AggregateFunctionAvg.h | 111 +++++++++++------- .../AggregateFunctionAvgWeighted.cpp | 20 ++-- .../AggregateFunctionAvgWeighted.h | 21 ++-- 4 files changed, 93 insertions(+), 81 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAvg.cpp b/src/AggregateFunctions/AggregateFunctionAvg.cpp index 3764fd67ff5..51d679ac47f 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvg.cpp @@ -13,19 +13,16 @@ namespace ErrorCodes namespace { +template +using AvgNumerator = std::conditional_t< + IsDecimalNumber, + std::conditional_t, + Decimal256, + Decimal128>, + NearestFieldType>; template -struct Avg -{ - using FieldType = std::conditional_t, - std::conditional_t, Decimal256, Decimal128>, - NearestFieldType>; - // using FieldType = std::conditional_t, Decimal128, NearestFieldType>; - using Function = AggregateFunctionAvg>; -}; - -template -using AggregateFuncAvg = typename Avg::Function; +using AggregateFuncAvg = AggregateFunctionAvg, UInt64>; AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const DataTypes & argument_types, const Array & parameters) { @@ -34,6 +31,7 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const AggregateFunctionPtr res; DataTypePtr data_type = argument_types[0]; + if (isDecimal(data_type)) res.reset(createWithDecimalType(*data_type, *data_type, argument_types)); else @@ -44,12 +42,10 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return res; } - } void registerAggregateFunctionAvg(AggregateFunctionFactory & factory) { factory.registerFunction("avg", createAggregateFunctionAvg, AggregateFunctionFactory::CaseInsensitive); } - } diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 944d9cbfaf5..0d58e7e53cc 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -2,73 +2,95 @@ #include #include - #include #include #include - #include namespace DB { -namespace ErrorCodes -{ -} +template +using DecimalOrVectorCol = std::conditional_t, ColumnDecimal, ColumnVector>; -template -struct AggregateFunctionAvgData +/// A type-fixed rational fraction represented by a pair of #Numerator and #Denominator. +template +struct RationalFraction { - using NumeratorType = T; + using NumeratorType = Numerator; using DenominatorType = Denominator; - T numerator{0}; + Numerator numerator{0}; Denominator denominator{0}; - template - ResultT NO_SANITIZE_UNDEFINED result() const + /// Calculate the fraction as a #Result. + template + Result NO_SANITIZE_UNDEFINED result() const { - if constexpr (std::is_floating_point_v) - if constexpr (std::numeric_limits::is_iec559) + if constexpr (std::is_floating_point_v) + if constexpr (std::numeric_limits::is_iec559) { if constexpr (is_big_int_v) - return static_cast(numerator) / static_cast(denominator); + return static_cast(numerator) / static_cast(denominator); else - return static_cast(numerator) / denominator; /// allow division by zero + return static_cast(numerator) / denominator; /// allow division by zero } if (denominator == static_cast(0)) - return static_cast(0); + return static_cast(0); - if constexpr (std::is_same_v) - return static_cast(numerator / static_cast(denominator)); + if constexpr (std::is_same_v) + return static_cast(numerator / static_cast(denominator)); else - return static_cast(numerator / denominator); + return static_cast(numerator / denominator); } }; -/// Calculates arithmetic mean of numbers. -template -class AggregateFunctionAvgBase : public IAggregateFunctionDataHelper +template +struct AvgTraits +{ + using ResultType = Float64; + using ResultDataType = DataTypeNumber; + using ResultVectorType = ColumnVector; +}; + +template +struct AvgTraits>> +{ + using ResultType = T; + using ResultDataType = DataTypeDecimal; + using ResultVectorType = ColumnDecimal; +}; + +/** + * @tparam DesiredResult The type that we want to be used for resulting column. "Desired" as the real type in most cases + * would be not DesiredResult, but Float64. + * @tparam Numerator The type that the initial numerator column would have (needed to cast the input IColumn to + * appropriate type). + * @tparam Denominator The type that the initial denominator column would have. + * + * @tparam Derived When deriving from this class, use the child class name as in CRTP, e.g. + * class Self : Agg. + */ +template +class AggregateFunctionAvgBase : public IAggregateFunctionDataHelper, Derived> { public: - using ResultType = std::conditional_t, T, Float64>; - using ResultDataType = std::conditional_t, DataTypeDecimal, DataTypeNumber>; - using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; - using ColVecResult = std::conditional_t, ColumnDecimal, ColumnVector>; + using Base = IAggregateFunctionDataHelper, Derived>; + using Traits = AvgTraits; /// ctor for native types - AggregateFunctionAvgBase(const DataTypes & argument_types_) : IAggregateFunctionDataHelper(argument_types_, {}), scale(0) {} + explicit AggregateFunctionAvgBase(const DataTypes & argument_types_): Base(argument_types_, {}), scale(0) {} /// ctor for Decimals AggregateFunctionAvgBase(const IDataType & data_type, const DataTypes & argument_types_) - : IAggregateFunctionDataHelper(argument_types_, {}), scale(getDecimalScale(data_type)) - { - } + : Base(argument_types_, {}), scale(getDecimalScale(data_type)) {} DataTypePtr getReturnType() const override { - if constexpr (IsDecimalNumber) + using ResultDataType = typename Traits::ResultDataType; + + if constexpr (IsDecimalNumber) return std::make_shared(ResultDataType::maxPrecision(), scale); else return std::make_shared(); @@ -84,7 +106,7 @@ public: { writeBinary(this->data(place).numerator, buf); - if constexpr (std::is_unsigned_v) + if constexpr (std::is_unsigned_v) writeVarUInt(this->data(place).denominator, buf); else /// Floating point denominator type can be used writeBinary(this->data(place).denominator, buf); @@ -94,7 +116,7 @@ public: { readBinary(this->data(place).numerator, buf); - if constexpr (std::is_unsigned_v) + if constexpr (std::is_unsigned_v) readVarUInt(this->data(place).denominator, buf); else /// Floating point denominator type can be used readBinary(this->data(place).denominator, buf); @@ -102,29 +124,32 @@ public: void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override { - auto & column = static_cast(to); - column.getData().push_back(this->data(place).template result()); + using ResultType = typename Traits::ResultType; + using ResultVectorType = typename Traits::ResultVectorType; + + static_cast(to).getData().push_back(this->data(place).template result()); } protected: UInt32 scale; }; -template -class AggregateFunctionAvg final : public AggregateFunctionAvgBase> +template +class AggregateFunctionAvg final : + public AggregateFunctionAvgBase> { public: - using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; + using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; - using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; - void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const final { - const auto & column = static_cast(*columns[0]); + const auto & column = static_cast &>(*columns[0]); this->data(place).numerator += column.getData()[row_num]; this->data(place).denominator += 1; } - String getName() const override { return "avg"; } + String getName() const final { return "avg"; } }; - } diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp index 95ba80f1cb1..0b62a786c84 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -25,20 +26,14 @@ constexpr bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) return allow(l_dt) && allow(r_dt); } -template struct BiggerType +// TODO signed to unsigned +template struct LargestType { using Type = V; }; -template struct LargestType -{ - using Type = bool; -}; +template +struct LargestType sizeof(V))>> { using Type = typename LargestType::Type; }; - -template using AvgData = AggregateFunctionAvgData< - typename LargestType::Type, - typename LargestType::Type>; - -template using Function = AggregateFunctionAvgWeighted< - U, V, typename LargestType::Type, AvgData>; +template using LargestTypeT = typename LargestType::Type; +template using Function = AggregateFunctionAvgWeighted, U, V>; template static IAggregateFunction * create(const IDataType & first_type, const IDataType & second_type, TArgs && ... args) @@ -70,7 +65,6 @@ AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name return res; } - } void registerAggregateFunctionAvgWeighted(AggregateFunctionFactory & factory) diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index 7ffdf41cfd9..0294bc9725b 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -5,30 +5,27 @@ namespace DB { -template +template class AggregateFunctionAvgWeighted final : - public AggregateFunctionAvgBase> + public AggregateFunctionAvgBase> { public: - using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; - - template - using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; + using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - const auto & values = static_cast &>(*columns[0]); - const auto & weights = static_cast &>(*columns[1]); + const auto & values = static_cast &>(*columns[0]); + const auto & weights = static_cast &>(*columns[1]); - const auto value = values.getData()[row_num]; + const Numerator value = static_cast(values.getData()[row_num]); const auto weight = weights.getData()[row_num]; - this->data(place).numerator += static_cast(value) * weight; + this->data(place).numerator += value * weight; this->data(place).denominator += weight; } String getName() const override { return "avgWeighted"; } }; - } From ee890e7689305c24118666a4f829ea3b4e6fe1fe Mon Sep 17 00:00:00 2001 From: annvsh Date: Sat, 17 Oct 2020 23:39:01 +0700 Subject: [PATCH 0042/1088] Added RawBLOB format description --- docs/en/interfaces/formats.md | 38 +++++++++++++++++++++++++++++++++++ docs/ru/interfaces/formats.md | 38 +++++++++++++++++++++++++++++++++++ 2 files changed, 76 insertions(+) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index d96d48bdca3..7b5e887e04b 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -57,6 +57,7 @@ The supported formats are: | [XML](#xml) | ✗ | ✔ | | [CapnProto](#capnproto) | ✔ | ✗ | | [LineAsString](#lineasstring) | ✔ | ✗ | +| [RawBLOB](#rawblob) | ✔ | ✔ | You can control some format processing parameters with the ClickHouse settings. For more information read the [Settings](../operations/settings/settings.md) section. @@ -1338,4 +1339,41 @@ Result: └───────────────────────────────────────────────────┘ ``` +## RawBLOB {#rawblob} + +This format slurps all input data into a single value. This format can only parse a table with a single field of type [String](../sql-reference/data-types/string.md) or similar. +When an empty value is passed to the input, ClickHouse generates an exception: + + ``` text +Code: 108. DB::Exception: No data to insert +``` + +The result is output in binary format without delimiters and escaping. If more than one value is output, the format is ambiguous, and it will be impossible to read the data back. + +**Example** + +``` bash +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -n --query " +DROP TABLE IF EXISTS t; +CREATE TABLE t (a LowCardinality(Nullable(String))) ENGINE = Memory; + +${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT RawBLOB" < ${BASH_SOURCE[0]} + +cat ${BASH_SOURCE[0]} | md5sum + +${CLICKHOUSE_CLIENT} -n --query "SELECT * FROM t FORMAT RawBLOB" | md5sum + +${CLICKHOUSE_CLIENT} --query " +DROP TABLE t; +``` + +Result: + +``` text +f9725a22f9191e064120d718e26862a9 - +``` + [Original article](https://clickhouse.tech/docs/en/interfaces/formats/) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 2745139998f..b0901e70f20 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -39,6 +39,7 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT | [XML](#xml) | ✗ | ✔ | | [CapnProto](#capnproto) | ✔ | ✗ | | [LineAsString](#lineasstring) | ✔ | ✗ | +| [RawBLOB](#rawblob) | ✔ | ✔ | Вы можете регулировать некоторые параметры работы с форматами с помощью настроек ClickHouse. За дополнительной информацией обращайтесь к разделу [Настройки](../operations/settings/settings.md). @@ -1143,4 +1144,41 @@ SELECT * FROM line_as_string; └───────────────────────────────────────────────────┘ ``` +## RawBLOB {#rawblob} + +Этот формат объединяет все входные данные в одно значение. Этот формат может парсить только таблицу с одним полем типа [String](../sql-reference/data-types/string.md) или подобным ему. +При передаче на вход пустого значения ClickHouse сгенерирует исключение: + + ``` text +Code: 108. DB::Exception: No data to insert +``` + +Результат выводится в двоичном формате без разделителей и экранирования. При выводе более одного значения формат неоднозначен и будет невозможно прочитать данные снова. + +**Пример** + +``` bash +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -n --query " +DROP TABLE IF EXISTS t; +CREATE TABLE t (a LowCardinality(Nullable(String))) ENGINE = Memory; + +${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT RawBLOB" < ${BASH_SOURCE[0]} + +cat ${BASH_SOURCE[0]} | md5sum + +${CLICKHOUSE_CLIENT} -n --query "SELECT * FROM t FORMAT RawBLOB" | md5sum + +${CLICKHOUSE_CLIENT} --query " +DROP TABLE t; +``` + +Результат: + +``` text +f9725a22f9191e064120d718e26862a9 - +``` + [Оригинальная статья](https://clickhouse.tech/docs/ru/interfaces/formats/) From a9532d611b34b4f427fd44dda719f86fc235a428 Mon Sep 17 00:00:00 2001 From: annvsh Date: Sat, 17 Oct 2020 23:50:19 +0700 Subject: [PATCH 0043/1088] Fixed --- docs/en/interfaces/formats.md | 1 + docs/ru/interfaces/formats.md | 2 ++ 2 files changed, 3 insertions(+) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 7b5e887e04b..1a96a2e4643 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1348,6 +1348,7 @@ When an empty value is passed to the input, ClickHouse generates an exception: Code: 108. DB::Exception: No data to insert ``` + The result is output in binary format without delimiters and escaping. If more than one value is output, the format is ambiguous, and it will be impossible to read the data back. **Example** diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index b0901e70f20..844c1a7fd59 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -1153,6 +1153,7 @@ SELECT * FROM line_as_string; Code: 108. DB::Exception: No data to insert ``` + Результат выводится в двоичном формате без разделителей и экранирования. При выводе более одного значения формат неоднозначен и будет невозможно прочитать данные снова. **Пример** @@ -1181,4 +1182,5 @@ DROP TABLE t; f9725a22f9191e064120d718e26862a9 - ``` + [Оригинальная статья](https://clickhouse.tech/docs/ru/interfaces/formats/) From 42047068b7c1473632cb9efd508e394c63d7a7ca Mon Sep 17 00:00:00 2001 From: annvsh Date: Sat, 17 Oct 2020 23:51:26 +0700 Subject: [PATCH 0044/1088] Fixed --- docs/ru/interfaces/formats.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 844c1a7fd59..ad326e2cfef 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -1182,5 +1182,4 @@ DROP TABLE t; f9725a22f9191e064120d718e26862a9 - ``` - [Оригинальная статья](https://clickhouse.tech/docs/ru/interfaces/formats/) From a6402336375563bbaa454bfc71316dedf231ea8d Mon Sep 17 00:00:00 2001 From: annvsh Date: Sat, 17 Oct 2020 23:58:53 +0700 Subject: [PATCH 0045/1088] Fixed --- docs/en/interfaces/formats.md | 2 +- docs/ru/interfaces/formats.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 1a96a2e4643..cccef0725b7 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1344,7 +1344,7 @@ Result: This format slurps all input data into a single value. This format can only parse a table with a single field of type [String](../sql-reference/data-types/string.md) or similar. When an empty value is passed to the input, ClickHouse generates an exception: - ``` text +``` text Code: 108. DB::Exception: No data to insert ``` diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index ad326e2cfef..11fb65fd0c0 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -1149,7 +1149,7 @@ SELECT * FROM line_as_string; Этот формат объединяет все входные данные в одно значение. Этот формат может парсить только таблицу с одним полем типа [String](../sql-reference/data-types/string.md) или подобным ему. При передаче на вход пустого значения ClickHouse сгенерирует исключение: - ``` text +``` text Code: 108. DB::Exception: No data to insert ``` From ada6f7eb2469fc32c12b65153f2b6789a6b25d89 Mon Sep 17 00:00:00 2001 From: annvsh Date: Sun, 18 Oct 2020 00:05:24 +0700 Subject: [PATCH 0046/1088] Fixed --- docs/en/interfaces/formats.md | 1 + docs/ru/interfaces/formats.md | 1 + 2 files changed, 2 insertions(+) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index cccef0725b7..46b29478fc8 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1369,6 +1369,7 @@ ${CLICKHOUSE_CLIENT} -n --query "SELECT * FROM t FORMAT RawBLOB" | md5sum ${CLICKHOUSE_CLIENT} --query " DROP TABLE t; +" ``` Result: diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 11fb65fd0c0..654ad3de9f4 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -1174,6 +1174,7 @@ ${CLICKHOUSE_CLIENT} -n --query "SELECT * FROM t FORMAT RawBLOB" | md5sum ${CLICKHOUSE_CLIENT} --query " DROP TABLE t; +" ``` Результат: From 30d477bc103166ede10e17af3ac42841896e9983 Mon Sep 17 00:00:00 2001 From: myrrc Date: Mon, 19 Oct 2020 18:23:35 +0300 Subject: [PATCH 0047/1088] added create() function fo avgWeighted --- .../reference/avgweighted.md | 18 +++++-- src/AggregateFunctions/AggregateFunctionAvg.h | 3 -- .../AggregateFunctionAvgWeighted.cpp | 53 ++++++++++++++++--- src/AggregateFunctions/Helpers.h | 1 + .../queries/0_stateless/01035_avg_weighted.sh | 9 ++-- 5 files changed, 65 insertions(+), 19 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md index 8dd2ecf4616..b4d698abe05 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md @@ -14,17 +14,25 @@ avgWeighted(x, weight) **Parameters** -- `x` — Values. [Integer](../../../sql-reference/data-types/int-uint.md) or [floating-point](../../../sql-reference/data-types/float.md). -- `weight` — Weights of the values. [Integer](../../../sql-reference/data-types/int-uint.md) or [floating-point](../../../sql-reference/data-types/float.md). +- `x` — Values. +- `weight` — Weights of the values. -Type of `x` and `weight` may be different. +`x` and `weight` must both be +[Integer](../../../sql-reference/data-types/int-uint.md), +[floating-point](../../../sql-reference/data-types/float.md), or +[Decimal](../../../sql-reference/data-types/decimal.md), +but may have different types. **Returned value** -- Weighted mean. - `NaN`. If all the weights are equal to 0. +- Weighted mean otherwise. -Type: [Float64](../../../sql-reference/data-types/float.md). +**Return type** + +- `Decimal` if both types are [Decimal](../../../sql-reference/data-types/decimal.md) (largest type taken). +(depending on the largest type). +- [Float64](../../../sql-reference/data-types/float.md) otherwise. **Example** diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 0d58e7e53cc..2d5d8a2b236 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -17,9 +17,6 @@ using DecimalOrVectorCol = std::conditional_t, ColumnDecimal< template struct RationalFraction { - using NumeratorType = Numerator; - using DenominatorType = Denominator; - Numerator numerator{0}; Denominator denominator{0}; diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp index 0b62a786c84..74e45c9b19a 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp @@ -26,19 +26,59 @@ constexpr bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) return allow(l_dt) && allow(r_dt); } -// TODO signed to unsigned -template struct LargestType { using Type = V; }; +template using Biggest = std::conditional_t<(sizeof(U) > sizeof(V)), U, V>; template -struct LargestType sizeof(V))>> { using Type = typename LargestType::Type; }; +struct LargestType +{ + using Biggest = Biggest; + static constexpr bool UDecimal = IsDecimalNumber; + static constexpr bool VDecimal = IsDecimalNumber; + + using TypeIfBothDecimal = std::conditional_t, + Decimal256, + Decimal128>; + + using Type = std::conditional_t; +}; template using LargestTypeT = typename LargestType::Type; template using Function = AggregateFunctionAvgWeighted, U, V>; -template + +#define AT_SWITCH(LINE) \ + switch (which.idx) \ + { \ + LINE(Int8); LINE(Int16); LINE(Int32); LINE(Int64); LINE(Int128); LINE(Int256); \ + LINE(UInt8); LINE(UInt16); LINE(UInt32); LINE(UInt64); LINE(UInt128); LINE(UInt256); \ + LINE(Decimal32); LINE(Decimal64); LINE(Decimal128); LINE(Decimal256); \ + LINE(Float32); LINE(Float64); \ + default: return nullptr; \ + } + +template +static IAggregateFunction * create(const IDataType & second_type, TArgs && ... args) +{ + const WhichDataType which(second_type); + +#define LINE(Type) \ + case TypeIndex::Type: return new Function(std::forward(args)...) + AT_SWITCH(LINE) +#undef LINE +} + +// Not using helper functions because there are no templates for binary decimal/numeric function. +template static IAggregateFunction * create(const IDataType & first_type, const IDataType & second_type, TArgs && ... args) { + const WhichDataType which(first_type); +#define LINE(Type) \ + case TypeIndex::Type: return create(second_type, std::forward(args)...) + AT_SWITCH(LINE) +#undef LINE } AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name, const DataTypes & argument_types, const Array & parameters) @@ -59,10 +99,7 @@ AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name AggregateFunctionPtr res; res.reset(create(*data_type, *data_type_weight, argument_types)); - if (!res) - throw Exception("Illegal type " + data_type->getName() + " of argument for aggregate function " + name, - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - + assert(res); // type checking should be done in allowTypes. return res; } } diff --git a/src/AggregateFunctions/Helpers.h b/src/AggregateFunctions/Helpers.h index 4d68c17eef4..f688dabeab6 100644 --- a/src/AggregateFunctions/Helpers.h +++ b/src/AggregateFunctions/Helpers.h @@ -15,6 +15,7 @@ M(Float32) \ M(Float64) +// No UInt128 here because of the name conflict #define FOR_NUMERIC_TYPES(M) \ M(UInt8) \ M(UInt16) \ diff --git a/tests/queries/0_stateless/01035_avg_weighted.sh b/tests/queries/0_stateless/01035_avg_weighted.sh index 58ea962148b..9b8d0bea14d 100755 --- a/tests/queries/0_stateless/01035_avg_weighted.sh +++ b/tests/queries/0_stateless/01035_avg_weighted.sh @@ -6,15 +6,18 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 5), (2, 4), (3, 3), (4, 2), (5, 1)]) AS t));" ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 0), (2, 0), (3, 0), (4, 0), (5, 0)]) AS t));" -types=("Int8" "Int16" "Int32" "Int64" "UInt8" "UInt16" "UInt32" "UInt64" "Float32" "Float64") +types=("Int8" "Int16" "Int32" "Int64" "Int128" "Int256" + "UInt8" "UInt16" "UInt32" "UInt64" "UInt128" "UInt256" + "Float32" "Float64" + "Decimal32" "Decimal64" "Decimal128" "Decimal256") for left in "${types[@]}" do for right in "${types[@]}" do ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 1), (1, 0), (10, 2))" - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 1), (1, 0))" - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 0), (1, 0))" + ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (8, 1), (122, 0))" + ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (0, 0), (1, 0))" done done From 293d2f06fac5d1daeca5e24b5c0c43910195a307 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 20 Oct 2020 15:38:56 +0800 Subject: [PATCH 0048/1088] Fix: throw error when column transformer use non-exsit column --- src/Parsers/ASTColumnsTransformers.cpp | 9 +++++++++ .../0_stateless/01470_columns_transformers.reference | 6 ------ tests/queries/0_stateless/01470_columns_transformers.sql | 4 ++-- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 43d54f07ab8..a204a409926 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -12,6 +12,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NO_SUCH_COLUMN_IN_TABLE; } void IASTColumnsTransformer::transform(const ASTPtr & transformer, ASTs & nodes) @@ -130,6 +131,7 @@ void ASTColumnsReplaceTransformer::transform(ASTs & nodes) const replace_map.emplace(replacement.name, replacement.expr); } + UInt8 replace_column_sucess = 0; for (auto & column : nodes) { if (const auto * id = column->as()) @@ -139,6 +141,7 @@ void ASTColumnsReplaceTransformer::transform(ASTs & nodes) const { column = replace_it->second; column->setAlias(replace_it->first); + ++replace_column_sucess; } } else if (auto * ast_with_alias = dynamic_cast(column.get())) @@ -151,9 +154,15 @@ void ASTColumnsReplaceTransformer::transform(ASTs & nodes) const replaceChildren(new_ast, column, replace_it->first); column = new_ast; column->setAlias(replace_it->first); + ++replace_column_sucess; } } } + + if (replace_column_sucess < replace_map.size()) + throw Exception( + "Expressions in columns transformer REPLACE should use same column name as original column", + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); } } diff --git a/tests/queries/0_stateless/01470_columns_transformers.reference b/tests/queries/0_stateless/01470_columns_transformers.reference index 2d8a1802289..397499f990f 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.reference +++ b/tests/queries/0_stateless/01470_columns_transformers.reference @@ -8,7 +8,6 @@ 1970-04-11 1970-01-11 1970-11-21 222 18 347 111 11 173.5 -1970-04-11 1970-01-11 1970-11-21 SELECT sum(i), sum(j), @@ -51,11 +50,6 @@ SELECT avg(j + 2 AS j), avg(k) FROM columns_transformers -SELECT - toDate(any(i)), - toDate(any(j)), - toDate(any(k)) -FROM columns_transformers AS a SELECT (i + 1) + 1 AS i, j, diff --git a/tests/queries/0_stateless/01470_columns_transformers.sql b/tests/queries/0_stateless/01470_columns_transformers.sql index f95cee51fb0..755978e82c4 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.sql +++ b/tests/queries/0_stateless/01470_columns_transformers.sql @@ -13,11 +13,12 @@ SELECT columns_transformers.* EXCEPT(j) APPLY(avg) from columns_transformers; -- EXCEPT after APPLY will not match anything SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a; +SELECT * REPLACE(i + 1 AS col) from columns_transformers; -- { serverError 16 } SELECT * REPLACE(i + 1 AS i) APPLY(sum) from columns_transformers; SELECT columns_transformers.* REPLACE(j + 2 AS j, i + 1 AS i) APPLY(avg) from columns_transformers; SELECT columns_transformers.* REPLACE(j + 1 AS j, j + 2 AS j) APPLY(avg) from columns_transformers; -- { serverError 43 } -- REPLACE after APPLY will not match anything -SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a; +SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a; -- { serverError 16 } EXPLAIN SYNTAX SELECT * APPLY(sum) from columns_transformers; EXPLAIN SYNTAX SELECT columns_transformers.* APPLY(avg) from columns_transformers; @@ -28,7 +29,6 @@ EXPLAIN SYNTAX SELECT columns_transformers.* EXCEPT(j) APPLY(avg) from columns_t EXPLAIN SYNTAX SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a; EXPLAIN SYNTAX SELECT * REPLACE(i + 1 AS i) APPLY(sum) from columns_transformers; EXPLAIN SYNTAX SELECT columns_transformers.* REPLACE(j + 2 AS j, i + 1 AS i) APPLY(avg) from columns_transformers; -EXPLAIN SYNTAX SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a; -- Multiple REPLACE in a row EXPLAIN SYNTAX SELECT * REPLACE(i + 1 AS i) REPLACE(i + 1 AS i) from columns_transformers; From 91b1dab75b841035e9b43b20f37205bfa82dc705 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Wed, 21 Oct 2020 15:54:13 +0800 Subject: [PATCH 0049/1088] Add EXCEPTSTRICT and REPLACESTRICT column transformers --- src/Parsers/ASTColumnsTransformers.cpp | 52 +++++++++++++++---- src/Parsers/ASTColumnsTransformers.h | 2 + src/Parsers/ExpressionElementParsers.cpp | 28 +++++++++- src/Parsers/ExpressionElementParsers.h | 3 ++ .../01470_columns_transformers.reference | 6 +++ .../01470_columns_transformers.sql | 7 ++- 6 files changed, 85 insertions(+), 13 deletions(-) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index a204a409926..3a9ecba9d3f 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -46,7 +46,9 @@ void ASTColumnsApplyTransformer::transform(ASTs & nodes) const void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXCEPT" << (settings.hilite ? hilite_none : "") << "("; + settings.ostr << (settings.hilite ? hilite_keyword : ""); + settings.ostr << (is_strict ? "EXCEPTSTRICT" : "EXCEPT"); + settings.ostr << (settings.hilite ? hilite_none : "") << "("; for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { @@ -62,23 +64,42 @@ void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, Fo void ASTColumnsExceptTransformer::transform(ASTs & nodes) const { + ASTs unexcepted_columns(children); nodes.erase( std::remove_if( nodes.begin(), nodes.end(), - [this](const ASTPtr & node_child) + [&](const ASTPtr & node_child) { if (const auto * id = node_child->as()) { - for (const auto & except_child : children) + for (size_t i = 0; i < children.size(); i++) { - if (except_child->as().name == id->shortName()) + if (children[i]->as().name == id->shortName()) + { + unexcepted_columns.erase(unexcepted_columns.begin() + i); return true; + } } } return false; }), nodes.end()); + + if (is_strict && !unexcepted_columns.empty()) + { + String unexisted_columns; + for (size_t i = 0; i < unexcepted_columns.size(); ++i) + { + if (i > 0) + unexisted_columns += ", "; + unexisted_columns += unexcepted_columns[i]->as().name; + } + + throw Exception( + "Columns transformer EXPCEPTSTRICT process unexist column : " + unexisted_columns, + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + } } void ASTColumnsReplaceTransformer::Replacement::formatImpl( @@ -90,7 +111,9 @@ void ASTColumnsReplaceTransformer::Replacement::formatImpl( void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "REPLACE" << (settings.hilite ? hilite_none : "") << "("; + settings.ostr << (settings.hilite ? hilite_keyword : ""); + settings.ostr << (is_strict ? "REPLACESTRICT" : "REPLACE"); + settings.ostr << (settings.hilite ? hilite_none : "") << "("; for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { @@ -131,7 +154,6 @@ void ASTColumnsReplaceTransformer::transform(ASTs & nodes) const replace_map.emplace(replacement.name, replacement.expr); } - UInt8 replace_column_sucess = 0; for (auto & column : nodes) { if (const auto * id = column->as()) @@ -141,7 +163,7 @@ void ASTColumnsReplaceTransformer::transform(ASTs & nodes) const { column = replace_it->second; column->setAlias(replace_it->first); - ++replace_column_sucess; + replace_map.erase(replace_it); } } else if (auto * ast_with_alias = dynamic_cast(column.get())) @@ -154,15 +176,25 @@ void ASTColumnsReplaceTransformer::transform(ASTs & nodes) const replaceChildren(new_ast, column, replace_it->first); column = new_ast; column->setAlias(replace_it->first); - ++replace_column_sucess; + replace_map.erase(replace_it); } } } - if (replace_column_sucess < replace_map.size()) + if (is_strict && !replace_map.empty()) + { + String unexisted_columns = ""; + for (auto it = replace_map.begin(); it != replace_map.end(); ++it) + { + if (unexisted_columns != "") + unexisted_columns += ", "; + unexisted_columns += it->first; + } throw Exception( - "Expressions in columns transformer REPLACE should use same column name as original column", + "Columns transformer REPLACESTRICT process unexist column : " + unexisted_columns, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + } + } } diff --git a/src/Parsers/ASTColumnsTransformers.h b/src/Parsers/ASTColumnsTransformers.h index 4b7a933647e..ac9d9958f9a 100644 --- a/src/Parsers/ASTColumnsTransformers.h +++ b/src/Parsers/ASTColumnsTransformers.h @@ -30,6 +30,7 @@ protected: class ASTColumnsExceptTransformer : public IASTColumnsTransformer { public: + bool is_strict = false; String getID(char) const override { return "ColumnsExceptTransformer"; } ASTPtr clone() const override { @@ -66,6 +67,7 @@ public: void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; }; + bool is_strict = false; String getID(char) const override { return "ColumnsReplaceTransformer"; } ASTPtr clone() const override { diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 1d861c6d78a..047acc170f8 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1236,6 +1236,10 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e ParserKeyword except("EXCEPT"); ParserKeyword replace("REPLACE"); ParserKeyword as("AS"); + ParserKeyword except_strict("EXCEPTSTRICT"); + ParserKeyword replace_strict("REPLACESTRICT"); + bool is_except = false; + bool is_replace = false; if (apply.ignore(pos, expected)) { @@ -1256,7 +1260,26 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e node = std::move(res); return true; } + else if (except_strict.ignore(pos, expected)) + { + is_except = true; + is_strict = true; + } else if (except.ignore(pos, expected)) + { + is_except = true; + } + else if (replace_strict.ignore(pos, expected)) + { + is_replace = true; + is_strict = true; + } + else if (replace.ignore(pos, expected)) + { + is_replace = true; + } + + if (is_except) { if (pos->type != TokenType::OpeningRoundBracket) return false; @@ -1282,11 +1305,13 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e auto res = std::make_shared(); res->children = std::move(identifiers); + res->is_strict = is_strict; node = std::move(res); return true; } - else if (replace.ignore(pos, expected)) + else if (is_replace) { + if (pos->type != TokenType::OpeningRoundBracket) return false; ++pos; @@ -1323,6 +1348,7 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e auto res = std::make_shared(); res->children = std::move(replacements); + res->is_strict = is_strict; node = std::move(res); return true; } diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 702d757761a..c920e4dc339 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -92,9 +92,12 @@ protected: */ class ParserColumnsTransformers : public IParserBase { +public: + ParserColumnsTransformers(bool is_strict_ = false): is_strict(is_strict_) {} protected: const char * getName() const override { return "COLUMNS transformers"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + bool is_strict; }; /** A function, for example, f(x, y + 1, g(z)). diff --git a/tests/queries/0_stateless/01470_columns_transformers.reference b/tests/queries/0_stateless/01470_columns_transformers.reference index 397499f990f..2d8a1802289 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.reference +++ b/tests/queries/0_stateless/01470_columns_transformers.reference @@ -8,6 +8,7 @@ 1970-04-11 1970-01-11 1970-11-21 222 18 347 111 11 173.5 +1970-04-11 1970-01-11 1970-11-21 SELECT sum(i), sum(j), @@ -50,6 +51,11 @@ SELECT avg(j + 2 AS j), avg(k) FROM columns_transformers +SELECT + toDate(any(i)), + toDate(any(j)), + toDate(any(k)) +FROM columns_transformers AS a SELECT (i + 1) + 1 AS i, j, diff --git a/tests/queries/0_stateless/01470_columns_transformers.sql b/tests/queries/0_stateless/01470_columns_transformers.sql index 755978e82c4..5aa68f91453 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.sql +++ b/tests/queries/0_stateless/01470_columns_transformers.sql @@ -13,12 +13,14 @@ SELECT columns_transformers.* EXCEPT(j) APPLY(avg) from columns_transformers; -- EXCEPT after APPLY will not match anything SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a; -SELECT * REPLACE(i + 1 AS col) from columns_transformers; -- { serverError 16 } +SELECT * EXCEPTSTRICT(i, j1) from columns_transformers; -- { serverError 16 } +SELECT * REPLACESTRICT(i + 1 AS col) from columns_transformers; -- { serverError 16 } SELECT * REPLACE(i + 1 AS i) APPLY(sum) from columns_transformers; SELECT columns_transformers.* REPLACE(j + 2 AS j, i + 1 AS i) APPLY(avg) from columns_transformers; SELECT columns_transformers.* REPLACE(j + 1 AS j, j + 2 AS j) APPLY(avg) from columns_transformers; -- { serverError 43 } -- REPLACE after APPLY will not match anything -SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a; -- { serverError 16 } +SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a; +SELECT a.* APPLY(toDate) REPLACESTRICT(i + 1 AS i) APPLY(any) from columns_transformers a; -- { serverError 16 } EXPLAIN SYNTAX SELECT * APPLY(sum) from columns_transformers; EXPLAIN SYNTAX SELECT columns_transformers.* APPLY(avg) from columns_transformers; @@ -29,6 +31,7 @@ EXPLAIN SYNTAX SELECT columns_transformers.* EXCEPT(j) APPLY(avg) from columns_t EXPLAIN SYNTAX SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a; EXPLAIN SYNTAX SELECT * REPLACE(i + 1 AS i) APPLY(sum) from columns_transformers; EXPLAIN SYNTAX SELECT columns_transformers.* REPLACE(j + 2 AS j, i + 1 AS i) APPLY(avg) from columns_transformers; +EXPLAIN SYNTAX SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a; -- Multiple REPLACE in a row EXPLAIN SYNTAX SELECT * REPLACE(i + 1 AS i) REPLACE(i + 1 AS i) from columns_transformers; From 8519f66555ea3bfb3060ea7c2bdcfe576702184e Mon Sep 17 00:00:00 2001 From: annvsh Date: Wed, 21 Oct 2020 19:46:31 +0700 Subject: [PATCH 0050/1088] x --- docs/ru/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 654ad3de9f4..3519b29c2d4 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -1146,7 +1146,7 @@ SELECT * FROM line_as_string; ## RawBLOB {#rawblob} -Этот формат объединяет все входные данные в одно значение. Этот формат может парсить только таблицу с одним полем типа [String](../sql-reference/data-types/string.md) или подобным ему. +Этот формат считывает все входные данные в одно значение. Этот формат может парсить только таблицу с одним полем типа [String](../sql-reference/data-types/string.md) или подобным ему. При передаче на вход пустого значения ClickHouse сгенерирует исключение: ``` text From ad2d2cf10dda2b36915e78c6b5ef93cf3e56fc66 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Thu, 22 Oct 2020 12:40:50 +0800 Subject: [PATCH 0051/1088] Modify varaible name and log info --- src/Parsers/ASTColumnsTransformers.cpp | 37 +++++++++---------- src/Parsers/ExpressionElementParsers.cpp | 30 ++++----------- .../01470_columns_transformers.sql | 6 +-- 3 files changed, 27 insertions(+), 46 deletions(-) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 3a9ecba9d3f..474c3262d78 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -46,9 +46,7 @@ void ASTColumnsApplyTransformer::transform(ASTs & nodes) const void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << (settings.hilite ? hilite_keyword : ""); - settings.ostr << (is_strict ? "EXCEPTSTRICT" : "EXCEPT"); - settings.ostr << (settings.hilite ? hilite_none : "") << "("; + settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXCEPT" << (is_strict ? " STRICT " : "") << (settings.hilite ? hilite_none : "") << "("; for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { @@ -64,7 +62,8 @@ void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, Fo void ASTColumnsExceptTransformer::transform(ASTs & nodes) const { - ASTs unexcepted_columns(children); + ASTs expected_columns(children); + nodes.erase( std::remove_if( nodes.begin(), @@ -73,11 +72,11 @@ void ASTColumnsExceptTransformer::transform(ASTs & nodes) const { if (const auto * id = node_child->as()) { - for (size_t i = 0; i < children.size(); i++) + for (int i = children.size() - 1; i >= 0; --i) { if (children[i]->as().name == id->shortName()) { - unexcepted_columns.erase(unexcepted_columns.begin() + i); + expected_columns.erase(expected_columns.begin() + i); return true; } } @@ -86,18 +85,18 @@ void ASTColumnsExceptTransformer::transform(ASTs & nodes) const }), nodes.end()); - if (is_strict && !unexcepted_columns.empty()) + if (is_strict && !expected_columns.empty()) { - String unexisted_columns; - for (size_t i = 0; i < unexcepted_columns.size(); ++i) + String expected_columns_str; + for (size_t i = 0; i < expected_columns.size(); ++i) { if (i > 0) - unexisted_columns += ", "; - unexisted_columns += unexcepted_columns[i]->as().name; + expected_columns_str += ", "; + expected_columns_str += expected_columns[i]->as().name; } throw Exception( - "Columns transformer EXPCEPTSTRICT process unexist column : " + unexisted_columns, + "Columns transformer EXCEPT expects following column(s) : " + expected_columns_str, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); } } @@ -111,9 +110,7 @@ void ASTColumnsReplaceTransformer::Replacement::formatImpl( void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << (settings.hilite ? hilite_keyword : ""); - settings.ostr << (is_strict ? "REPLACESTRICT" : "REPLACE"); - settings.ostr << (settings.hilite ? hilite_none : "") << "("; + settings.ostr << (settings.hilite ? hilite_keyword : "") << "REPLACE" << (is_strict ? " STRICT " : "") << (settings.hilite ? hilite_none : "") << "("; for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { @@ -183,15 +180,15 @@ void ASTColumnsReplaceTransformer::transform(ASTs & nodes) const if (is_strict && !replace_map.empty()) { - String unexisted_columns = ""; + String expected_columns = ""; for (auto it = replace_map.begin(); it != replace_map.end(); ++it) { - if (unexisted_columns != "") - unexisted_columns += ", "; - unexisted_columns += it->first; + if (expected_columns != "") + expected_columns += ", "; + expected_columns += it->first; } throw Exception( - "Columns transformer REPLACESTRICT process unexist column : " + unexisted_columns, + "Columns transformer REPLACE expects following column(s) : " + expected_columns, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 047acc170f8..dc8f39ae894 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1236,10 +1236,7 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e ParserKeyword except("EXCEPT"); ParserKeyword replace("REPLACE"); ParserKeyword as("AS"); - ParserKeyword except_strict("EXCEPTSTRICT"); - ParserKeyword replace_strict("REPLACESTRICT"); - bool is_except = false; - bool is_replace = false; + ParserKeyword strict("STRICT"); if (apply.ignore(pos, expected)) { @@ -1260,27 +1257,12 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e node = std::move(res); return true; } - else if (except_strict.ignore(pos, expected)) - { - is_except = true; - is_strict = true; - } else if (except.ignore(pos, expected)) { - is_except = true; - } - else if (replace_strict.ignore(pos, expected)) - { - is_replace = true; - is_strict = true; - } - else if (replace.ignore(pos, expected)) - { - is_replace = true; - } - if (is_except) - { + if (strict.ignore(pos, expected)) + is_strict = true; + if (pos->type != TokenType::OpeningRoundBracket) return false; ++pos; @@ -1309,8 +1291,10 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e node = std::move(res); return true; } - else if (is_replace) + else if (replace.ignore(pos, expected)) { + if (strict.ignore(pos, expected)) + is_strict = true; if (pos->type != TokenType::OpeningRoundBracket) return false; diff --git a/tests/queries/0_stateless/01470_columns_transformers.sql b/tests/queries/0_stateless/01470_columns_transformers.sql index 5aa68f91453..a3d103cd876 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.sql +++ b/tests/queries/0_stateless/01470_columns_transformers.sql @@ -13,14 +13,14 @@ SELECT columns_transformers.* EXCEPT(j) APPLY(avg) from columns_transformers; -- EXCEPT after APPLY will not match anything SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a; -SELECT * EXCEPTSTRICT(i, j1) from columns_transformers; -- { serverError 16 } -SELECT * REPLACESTRICT(i + 1 AS col) from columns_transformers; -- { serverError 16 } +SELECT * EXCEPT STRICT(i, j1) from columns_transformers; -- { serverError 16 } +SELECT * REPLACE STRICT(i + 1 AS col) from columns_transformers; -- { serverError 16 } SELECT * REPLACE(i + 1 AS i) APPLY(sum) from columns_transformers; SELECT columns_transformers.* REPLACE(j + 2 AS j, i + 1 AS i) APPLY(avg) from columns_transformers; SELECT columns_transformers.* REPLACE(j + 1 AS j, j + 2 AS j) APPLY(avg) from columns_transformers; -- { serverError 43 } -- REPLACE after APPLY will not match anything SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a; -SELECT a.* APPLY(toDate) REPLACESTRICT(i + 1 AS i) APPLY(any) from columns_transformers a; -- { serverError 16 } +SELECT a.* APPLY(toDate) REPLACE STRICT(i + 1 AS i) APPLY(any) from columns_transformers a; -- { serverError 16 } EXPLAIN SYNTAX SELECT * APPLY(sum) from columns_transformers; EXPLAIN SYNTAX SELECT columns_transformers.* APPLY(avg) from columns_transformers; From 3d3b49d009a9613984e865b9535bfa9418e135e4 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 22 Oct 2020 16:44:48 +0300 Subject: [PATCH 0052/1088] Try to parse DataType arguments as another nested one --- src/Parsers/ParserDataType.cpp | 28 +++++++++++++------ .../01532_tuple_with_name_type.reference | 4 +++ .../01532_tuple_with_name_type.sql | 16 +++++++++++ 3 files changed, 40 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/01532_tuple_with_name_type.reference create mode 100644 tests/queries/0_stateless/01532_tuple_with_name_type.sql diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index a0a4eb97efe..9b10111db06 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -1,10 +1,12 @@ #include -#include -#include + #include #include +#include +#include #include + namespace DB { @@ -78,14 +80,24 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ++pos; /// Parse optional parameters - ParserList args_parser(std::make_unique(), std::make_unique(TokenType::Comma)); ASTPtr expr_list_args; - if (!args_parser.parse(pos, expr_list_args, expected)) - return false; - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; + ParserList args_parser_nested(std::make_unique(), std::make_unique(TokenType::Comma), false); + if (args_parser_nested.parse(pos, expr_list_args, expected)) + { + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + } + else + { + ParserList args_parser_expr(std::make_unique(), std::make_unique(TokenType::Comma)); + if (!args_parser_expr.parse(pos, expr_list_args, expected)) + return false; + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + } function_node->arguments = expr_list_args; function_node->children.push_back(function_node->arguments); diff --git a/tests/queries/0_stateless/01532_tuple_with_name_type.reference b/tests/queries/0_stateless/01532_tuple_with_name_type.reference new file mode 100644 index 00000000000..065734bf705 --- /dev/null +++ b/tests/queries/0_stateless/01532_tuple_with_name_type.reference @@ -0,0 +1,4 @@ +a Tuple(key String, value String) +a Tuple(Tuple(key String, value String)) +a.key Array(String) +a.value Array(String) diff --git a/tests/queries/0_stateless/01532_tuple_with_name_type.sql b/tests/queries/0_stateless/01532_tuple_with_name_type.sql new file mode 100644 index 00000000000..86e0de330a2 --- /dev/null +++ b/tests/queries/0_stateless/01532_tuple_with_name_type.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS test_01532_1; +DROP TABLE IF EXISTS test_01532_2; +DROP TABLE IF EXISTS test_01532_3; + +CREATE TABLE test_01532_1 (a Tuple(key String, value String)) ENGINE Memory(); +DESCRIBE TABLE test_01532_1; + +CREATE TABLE test_01532_2 (a Tuple(Tuple(key String, value String))) ENGINE Memory(); +DESCRIBE TABLE test_01532_2; + +CREATE TABLE test_01532_3 (a Array(Tuple(key String, value String))) ENGINE Memory(); +DESCRIBE TABLE test_01532_3; + +DROP TABLE test_01532_1; +DROP TABLE test_01532_2; +DROP TABLE test_01532_3; From 338ecb6fe17a5ac69c14577814e133dcc5e8a34b Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 22 Oct 2020 17:29:32 +0300 Subject: [PATCH 0053/1088] possibly added the numerator/denominator determination algorithm in avgWeighted --- .../reference/avgweighted.md | 5 +- src/AggregateFunctions/AggregateFunctionAvg.h | 44 ++----- .../AggregateFunctionAvgWeighted.cpp | 24 +--- .../AggregateFunctionAvgWeighted.h | 108 ++++++++++++++++-- 4 files changed, 109 insertions(+), 72 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md index b4d698abe05..12ee76354d5 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md @@ -29,10 +29,7 @@ but may have different types. - Weighted mean otherwise. **Return type** - -- `Decimal` if both types are [Decimal](../../../sql-reference/data-types/decimal.md) (largest type taken). -(depending on the largest type). -- [Float64](../../../sql-reference/data-types/float.md) otherwise. +- [Float64](../../../sql-reference/data-types/float.md). **Example** diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 2d5d8a2b236..7d5d24ce750 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -43,25 +43,7 @@ struct RationalFraction } }; -template -struct AvgTraits -{ - using ResultType = Float64; - using ResultDataType = DataTypeNumber; - using ResultVectorType = ColumnVector; -}; - -template -struct AvgTraits>> -{ - using ResultType = T; - using ResultDataType = DataTypeDecimal; - using ResultVectorType = ColumnDecimal; -}; - /** - * @tparam DesiredResult The type that we want to be used for resulting column. "Desired" as the real type in most cases - * would be not DesiredResult, but Float64. * @tparam Numerator The type that the initial numerator column would have (needed to cast the input IColumn to * appropriate type). * @tparam Denominator The type that the initial denominator column would have. @@ -69,12 +51,11 @@ struct AvgTraits>> * @tparam Derived When deriving from this class, use the child class name as in CRTP, e.g. * class Self : Agg. */ -template +template class AggregateFunctionAvgBase : public IAggregateFunctionDataHelper, Derived> { public: using Base = IAggregateFunctionDataHelper, Derived>; - using Traits = AvgTraits; /// ctor for native types explicit AggregateFunctionAvgBase(const DataTypes & argument_types_): Base(argument_types_, {}), scale(0) {} @@ -85,12 +66,7 @@ public: DataTypePtr getReturnType() const override { - using ResultDataType = typename Traits::ResultDataType; - - if constexpr (IsDecimalNumber) - return std::make_shared(ResultDataType::maxPrecision(), scale); - else - return std::make_shared(); + return std::make_shared>(); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override @@ -121,28 +97,24 @@ public: void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override { - using ResultType = typename Traits::ResultType; - using ResultVectorType = typename Traits::ResultVectorType; - - static_cast(to).getData().push_back(this->data(place).template result()); + static_cast &>(to).getData().push_back(this->data(place).template result()); } protected: UInt32 scale; }; -template +template class AggregateFunctionAvg final : - public AggregateFunctionAvgBase> + public AggregateFunctionAvgBase> { public: - using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; + using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const final { - const auto & column = static_cast &>(*columns[0]); + const auto & column = static_cast &>(*columns[0]); this->data(place).numerator += column.getData()[row_num]; this->data(place).denominator += 1; } diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp index 74e45c9b19a..5b43aa19a5c 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp @@ -26,28 +26,6 @@ constexpr bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) return allow(l_dt) && allow(r_dt); } -template using Biggest = std::conditional_t<(sizeof(U) > sizeof(V)), U, V>; - -template -struct LargestType -{ - using Biggest = Biggest; - static constexpr bool UDecimal = IsDecimalNumber; - static constexpr bool VDecimal = IsDecimalNumber; - - using TypeIfBothDecimal = std::conditional_t, - Decimal256, - Decimal128>; - - using Type = std::conditional_t; -}; - -template using LargestTypeT = typename LargestType::Type; -template using Function = AggregateFunctionAvgWeighted, U, V>; - - #define AT_SWITCH(LINE) \ switch (which.idx) \ { \ @@ -64,7 +42,7 @@ static IAggregateFunction * create(const IDataType & second_type, TArgs && ... a const WhichDataType which(second_type); #define LINE(Type) \ - case TypeIndex::Type: return new Function(std::forward(args)...) + case TypeIndex::Type: return new AggregateFunctionAvgWeighted(std::forward(args)...) AT_SWITCH(LINE) #undef LINE } diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index 0294bc9725b..73827ac5a41 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -1,28 +1,118 @@ #pragma once +#include #include namespace DB { +template struct NextAvgType { using Type = T; }; +template <> struct NextAvgType { using Type = Int16; }; +template <> struct NextAvgType { using Type = Int32; }; +template <> struct NextAvgType { using Type = Int64; }; +template <> struct NextAvgType { using Type = Int128; }; +template <> struct NextAvgType { using Type = Int256; }; +template <> struct NextAvgType { using Type = Int256; }; -template +template <> struct NextAvgType { using Type = UInt16; }; +template <> struct NextAvgType { using Type = UInt32; }; +template <> struct NextAvgType { using Type = UInt64; }; +template <> struct NextAvgType { using Type = UInt128; }; +template <> struct NextAvgType { using Type = UInt256; }; +template <> struct NextAvgType { using Type = UInt256; }; + +template <> struct NextAvgType { using Type = Decimal128; }; +template <> struct NextAvgType { using Type = Decimal128; }; +template <> struct NextAvgType { using Type = Decimal256; }; +template <> struct NextAvgType { using Type = Decimal256; }; + +template <> struct NextAvgType { using Type = Float64; }; +template <> struct NextAvgType { using Type = Float64; }; + +template using NextAvgTypeT = typename NextAvgType::Type; +template using Largest = std::conditional_t<(sizeof(T) > sizeof(U)), T, U>; + +template +struct GetNumDenom +{ + static constexpr bool UDecimal = IsDecimalNumber; + static constexpr bool VDecimal = IsDecimalNumber; + static constexpr bool BothDecimal = UDecimal && VDecimal; + static constexpr bool NoneDecimal = !UDecimal && !VDecimal; + + template + static constexpr bool IsIntegral = std::is_integral_v + || std::is_same_v || std::is_same_v + || std::is_same_v || std::is_same_v; + + static constexpr bool BothOrNoneDecimal = BothDecimal || NoneDecimal; + + using Num = std::conditional>, + + std::conditional_t, + NextAvgTypeT, + Float64>, + /// When the denominator only is Decimal, we check the numerator (as the above case). + std::conditional_t, + NextAvgTypeT, + Float64>>>; + + /** + * When both types are Decimal, we can perform computations in the Decimals only. + * When none of the types is Decimal, the result is always correct, the numerator is the next largest type up to + * Float64. + * We use #V only as the denominator accumulates the sum of the weights. + * + * When the numerator only is Decimal, we set the denominator to next Largest type. + * - If the denominator was floating-point, the numerator would be Float64. + * - If not, the numerator would be Decimal (as the denominator is integral). + * + * When the denominator only is Decimal, the numerator is either integral (so we leave the Decimal), or Float64, + * so we set the denominator to Float64; + */ + using Denom = std::conditional, + Float64, + NextAvgTypeT>; +}; + +template using AvgWeightedNum = typename GetNumDenom::Num; +template using AvgWeightedDenom = typename GetNumDenom::Denom; + +template +using AggFuncAvgWeightedBase = AggregateFunctionAvgBase< + AvgWeightedNum, + AvgWeightedDenom, Derived>; + +/** + * @tparam Values The values column type. + * @tparam Weights The weights column type. + */ +template class AggregateFunctionAvgWeighted final : - public AggregateFunctionAvgBase> + AggFuncAvgWeightedBase> { public: - using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; + using AggFuncAvgWeightedBase> + ::AggFuncAvgWeightedBase; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - const auto & values = static_cast &>(*columns[0]); - const auto & weights = static_cast &>(*columns[1]); + const auto & values = static_cast &>(*columns[0]); + const auto & weights = static_cast &>(*columns[1]); - const Numerator value = static_cast(values.getData()[row_num]); + const auto value = values.getData()[row_num]; const auto weight = weights.getData()[row_num]; - this->data(place).numerator += value * weight; + using TargetNum = AvgWeightedNum; + + this->data(place).numerator += static_cast(value) * weight; this->data(place).denominator += weight; } From 3d479cdd8cfac510af3862fa3b15570241e85a63 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 22 Oct 2020 18:47:21 +0300 Subject: [PATCH 0054/1088] added resul type deduction structs --- .../reference/avgweighted.md | 5 +- .../AggregateFunctionAvg.cpp | 15 +- src/AggregateFunctions/AggregateFunctionAvg.h | 83 ++++++-- .../AggregateFunctionAvgWeighted.h | 182 ++++++++++-------- 4 files changed, 176 insertions(+), 109 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md index 12ee76354d5..a6fb5999fb8 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md @@ -29,7 +29,10 @@ but may have different types. - Weighted mean otherwise. **Return type** -- [Float64](../../../sql-reference/data-types/float.md). + +- `Decimal` if both types are [Decimal](../../../sql-reference/data-types/decimal.md) + or if one type is Decimal and other is Integer. +- [Float64](../../../sql-reference/data-types/float.md) otherwise. **Example** diff --git a/src/AggregateFunctions/AggregateFunctionAvg.cpp b/src/AggregateFunctions/AggregateFunctionAvg.cpp index 51d679ac47f..cf35e99dafb 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvg.cpp @@ -13,17 +13,6 @@ namespace ErrorCodes namespace { -template -using AvgNumerator = std::conditional_t< - IsDecimalNumber, - std::conditional_t, - Decimal256, - Decimal128>, - NearestFieldType>; - -template -using AggregateFuncAvg = AggregateFunctionAvg, UInt64>; - AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const DataTypes & argument_types, const Array & parameters) { assertNoParameters(name, parameters); @@ -33,9 +22,9 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const DataTypePtr data_type = argument_types[0]; if (isDecimal(data_type)) - res.reset(createWithDecimalType(*data_type, *data_type, argument_types)); + res.reset(createWithDecimalType(*data_type, *data_type, argument_types)); else - res.reset(createWithNumericType(*data_type, argument_types)); + res.reset(createWithNumericType(*data_type, argument_types)); if (!res) throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 7d5d24ce750..5dd6cf856ef 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -44,18 +44,57 @@ struct RationalFraction }; /** - * @tparam Numerator The type that the initial numerator column would have (needed to cast the input IColumn to + * Motivation: ClickHouse has added the Decimal data type, which basically represents a fraction that stores + * the precise (unlike floating-point) result with respect to some scale. + * + * These decimal types can't be divided by floating point data types, so functions like avg or avgWeighted + * can't return the Floa64 column as a result when of the input columns is Decimal (because that would, in case of + * avgWeighted, involve division numerator (Decimal) / denominator (Float64)). + * + * The rules for determining the output and intermediate storage types for these functions are different, so + * the struct representing the deduction guide is presented. + * + * Given the initial Columns types (e.g. values and weights for avgWeighted, values for avg), + * the struct calculated the output type and the intermediate storage type (that's used by the RationalFraction). + */ +template +struct AvgFunctionTypesDeductionTemplate +{ + using Numerator = int; + using Denominator = int; + using Fraction = RationalFraction; + + using ResultType = bool; + using ResultDataType = bool; + using ResultVectorType = bool; +}; + +/** + * @tparam InitialNumerator The type that the initial numerator column would have (needed to cast the input IColumn to * appropriate type). - * @tparam Denominator The type that the initial denominator column would have. + * @tparam InitialDenominator The type that the initial denominator column would have. + * + * @tparam Deduction Function template that, given the numerator and the denominator, finds the actual + * suitable storage and the resulting column type. * * @tparam Derived When deriving from this class, use the child class name as in CRTP, e.g. * class Self : Agg. */ -template -class AggregateFunctionAvgBase : public IAggregateFunctionDataHelper, Derived> +template class Deduction, class Derived> +class AggregateFunctionAvgBase : public + IAggregateFunctionDataHelper::Fraction, Derived> { public: - using Base = IAggregateFunctionDataHelper, Derived>; + using Deducted = Deduction; + + using ResultType = typename Deducted::ResultType; + using ResultDataType = typename Deducted::ResultDataType; + using ResultVectorType = typename Deducted::ResultVectorType; + + using Numerator = typename Deducted::Numerator; + using Denominator = typename Deducted::Denominator; + + using Base = IAggregateFunctionDataHelper; /// ctor for native types explicit AggregateFunctionAvgBase(const DataTypes & argument_types_): Base(argument_types_, {}), scale(0) {} @@ -66,7 +105,10 @@ public: DataTypePtr getReturnType() const override { - return std::make_shared>(); + if constexpr (IsDecimalNumber) + return std::make_shared(ResultDataType::maxPrecision(), scale); + else + return std::make_shared(); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override @@ -97,20 +139,39 @@ public: void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override { - static_cast &>(to).getData().push_back(this->data(place).template result()); + static_cast(to).getData().push_back(this->data(place).template result()); } protected: UInt32 scale; }; -template +template +struct AvgFunctionTypesDeduction +{ + using Numerator = std::conditional_t, + std::conditional_t, + Decimal256, + Decimal128>, + NearestFieldType>; + + using Denominator = V; + using Fraction = RationalFraction; + + using ResultType = std::conditional_t, T, Float64>; + using ResultDataType = std::conditional_t, DataTypeDecimal, DataTypeNumber>; + using ResultVectorType = std::conditional_t, ColumnDecimal, ColumnVector>; +}; + +template class AggregateFunctionAvg final : - public AggregateFunctionAvgBase> + public AggregateFunctionAvgBase> { public: - using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; + using Base = + AggregateFunctionAvgBase>; + + using Base::Base; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const final { diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index 73827ac5a41..88dd49e6147 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -5,102 +5,118 @@ namespace DB { -template struct NextAvgType { using Type = T; }; -template <> struct NextAvgType { using Type = Int16; }; -template <> struct NextAvgType { using Type = Int32; }; -template <> struct NextAvgType { using Type = Int64; }; -template <> struct NextAvgType { using Type = Int128; }; -template <> struct NextAvgType { using Type = Int256; }; -template <> struct NextAvgType { using Type = Int256; }; - -template <> struct NextAvgType { using Type = UInt16; }; -template <> struct NextAvgType { using Type = UInt32; }; -template <> struct NextAvgType { using Type = UInt64; }; -template <> struct NextAvgType { using Type = UInt128; }; -template <> struct NextAvgType { using Type = UInt256; }; -template <> struct NextAvgType { using Type = UInt256; }; - -template <> struct NextAvgType { using Type = Decimal128; }; -template <> struct NextAvgType { using Type = Decimal128; }; -template <> struct NextAvgType { using Type = Decimal256; }; -template <> struct NextAvgType { using Type = Decimal256; }; - -template <> struct NextAvgType { using Type = Float64; }; -template <> struct NextAvgType { using Type = Float64; }; - -template using NextAvgTypeT = typename NextAvgType::Type; -template using Largest = std::conditional_t<(sizeof(T) > sizeof(U)), T, U>; - -template -struct GetNumDenom +template +struct AvgWeightedFunctionTypesDeduction { - static constexpr bool UDecimal = IsDecimalNumber; - static constexpr bool VDecimal = IsDecimalNumber; - static constexpr bool BothDecimal = UDecimal && VDecimal; - static constexpr bool NoneDecimal = !UDecimal && !VDecimal; + template struct NextAvgType { using Type = T; }; + template <> struct NextAvgType { using Type = Int16; }; + template <> struct NextAvgType { using Type = Int32; }; + template <> struct NextAvgType { using Type = Int64; }; + template <> struct NextAvgType { using Type = Int128; }; + template <> struct NextAvgType { using Type = Int256; }; + template <> struct NextAvgType { using Type = Int256; }; - template - static constexpr bool IsIntegral = std::is_integral_v - || std::is_same_v || std::is_same_v - || std::is_same_v || std::is_same_v; + template <> struct NextAvgType { using Type = UInt16; }; + template <> struct NextAvgType { using Type = UInt32; }; + template <> struct NextAvgType { using Type = UInt64; }; + template <> struct NextAvgType { using Type = UInt128; }; + template <> struct NextAvgType { using Type = UInt256; }; + template <> struct NextAvgType { using Type = UInt256; }; - static constexpr bool BothOrNoneDecimal = BothDecimal || NoneDecimal; + template <> struct NextAvgType { using Type = Decimal128; }; + template <> struct NextAvgType { using Type = Decimal128; }; + template <> struct NextAvgType { using Type = Decimal256; }; + template <> struct NextAvgType { using Type = Decimal256; }; - using Num = std::conditional>, + template <> struct NextAvgType { using Type = Float64; }; + template <> struct NextAvgType { using Type = Float64; }; - std::conditional_t, - NextAvgTypeT, - Float64>, - /// When the denominator only is Decimal, we check the numerator (as the above case). - std::conditional_t, - NextAvgTypeT, - Float64>>>; + template using NextAvgTypeT = typename NextAvgType::Type; + template using Largest = std::conditional_t<(sizeof(T) > sizeof(U)), T, U>; - /** - * When both types are Decimal, we can perform computations in the Decimals only. - * When none of the types is Decimal, the result is always correct, the numerator is the next largest type up to - * Float64. - * We use #V only as the denominator accumulates the sum of the weights. - * - * When the numerator only is Decimal, we set the denominator to next Largest type. - * - If the denominator was floating-point, the numerator would be Float64. - * - If not, the numerator would be Decimal (as the denominator is integral). - * - * When the denominator only is Decimal, the numerator is either integral (so we leave the Decimal), or Float64, - * so we set the denominator to Float64; - */ - using Denom = std::conditional, - Float64, - NextAvgTypeT>; + struct GetNumDenom + { + using U = Values; + using V = Weights; + static constexpr bool UDecimal = IsDecimalNumber; + static constexpr bool VDecimal = IsDecimalNumber; + static constexpr bool BothDecimal = UDecimal && VDecimal; + static constexpr bool NoneDecimal = !UDecimal && !VDecimal; + + template + static constexpr bool IsIntegral = std::is_integral_v + || std::is_same_v || std::is_same_v + || std::is_same_v || std::is_same_v; + + static constexpr bool BothOrNoneDecimal = BothDecimal || NoneDecimal; + + using Num = std::conditional>, + + std::conditional_t, + NextAvgTypeT, + Float64>, + /// When the denominator only is Decimal, we check the numerator (as the above case). + std::conditional_t, + NextAvgTypeT, + Float64>>>; + + /** + * When both types are Decimal, we can perform computations in the Decimals only. + * When none of the types is Decimal, the result is always correct, the numerator is the next largest type up to + * Float64. + * We use #V only as the denominator accumulates the sum of the weights. + * + * When the numerator only is Decimal, we set the denominator to next Largest type. + * - If the denominator was floating-point, the numerator would be Float64. + * - If not, the numerator would be Decimal (as the denominator is integral). + * + * When the denominator only is Decimal, the numerator is either integral (so we leave the Decimal), or Float64, + * so we set the denominator to Float64; + */ + using Denom = std::conditional, + Float64, + NextAvgTypeT>; + }; + + using Numerator = typename GetNumDenom::Num; + using Denominator = typename GetNumDenom::Denom; + using Fraction = RationalFraction; + + /// If either Numerator or Denominator are Decimal, the result is also Decimal as everything was checked in + /// GetNumDenom. + using T = std::conditional_t && IsDecimalNumber, + Largest, + std::conditional_t, + Numerator, + std::conditional_t, + Denominator, + bool>>>; // both numerator and denominator are non-decimal. + + using ResultType = std::conditional_t, T, Float64>; + using ResultDataType = std::conditional_t, DataTypeDecimal, DataTypeNumber>; + using ResultVectorType = std::conditional_t, ColumnDecimal, ColumnVector>; }; -template using AvgWeightedNum = typename GetNumDenom::Num; -template using AvgWeightedDenom = typename GetNumDenom::Denom; - -template -using AggFuncAvgWeightedBase = AggregateFunctionAvgBase< - AvgWeightedNum, - AvgWeightedDenom, Derived>; - /** * @tparam Values The values column type. * @tparam Weights The weights column type. */ template -class AggregateFunctionAvgWeighted final : - AggFuncAvgWeightedBase> +class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase< + Values, Weights, AvgWeightedFunctionTypesDeduction, AggregateFunctionAvgWeighted> { public: - using AggFuncAvgWeightedBase> - ::AggFuncAvgWeightedBase; + using Base = AggregateFunctionAvgBase< + Values, Weights, AvgWeightedFunctionTypesDeduction, AggregateFunctionAvgWeighted>; + using Base::Base; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { @@ -110,9 +126,7 @@ public: const auto value = values.getData()[row_num]; const auto weight = weights.getData()[row_num]; - using TargetNum = AvgWeightedNum; - - this->data(place).numerator += static_cast(value) * weight; + this->data(place).numerator += static_cast(value) * weight; this->data(place).denominator += weight; } From e4f35e27e103171923623f88c4a0430aaad36f43 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 22 Oct 2020 18:53:45 +0300 Subject: [PATCH 0055/1088] RationalFraction initialization fix --- src/AggregateFunctions/AggregateFunctionAvg.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 5dd6cf856ef..a35e1a965a9 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -17,8 +17,10 @@ using DecimalOrVectorCol = std::conditional_t, ColumnDecimal< template struct RationalFraction { - Numerator numerator{0}; - Denominator denominator{0}; + /// {0} prohibited as excess elements. + /// = 0 prohibited as there are no suitable constructors. + Numerator numerator = static_cast(0); + Denominator denominator = static_cast(0); /// Calculate the fraction as a #Result. template From 2a589d7c155e291a4381a6d93783cfd0518337a3 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 22 Oct 2020 19:13:18 +0300 Subject: [PATCH 0056/1088] updated deduction rules to correct integral / Decimal division --- src/AggregateFunctions/AggregateFunctionAvg.h | 8 +++---- .../AggregateFunctionAvgWeighted.h | 22 +++++++++---------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index a35e1a965a9..0b77aa0c537 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -17,10 +17,10 @@ using DecimalOrVectorCol = std::conditional_t, ColumnDecimal< template struct RationalFraction { - /// {0} prohibited as excess elements. - /// = 0 prohibited as there are no suitable constructors. - Numerator numerator = static_cast(0); - Denominator denominator = static_cast(0); + constexpr RationalFraction(): numerator(0), denominator(0) {} + + Numerator numerator; + Denominator denominator; /// Calculate the fraction as a #Result. template diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index 88dd49e6147..8c245253710 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -44,13 +44,14 @@ struct AvgWeightedFunctionTypesDeduction static constexpr bool NoneDecimal = !UDecimal && !VDecimal; template - static constexpr bool IsIntegral = std::is_integral_v - || std::is_same_v || std::is_same_v - || std::is_same_v || std::is_same_v; + static constexpr bool IsIntegral = std::is_integral_v; + /// we do not include extended integral types here as they produce errors while diving on Decimals. + /// || std::is_same_v || std::is_same_v + /// || std::is_same_v || std::is_same_v; static constexpr bool BothOrNoneDecimal = BothDecimal || NoneDecimal; - using Num = std::conditional, NextAvgTypeT, Float64>, - /// When the denominator only is Decimal, we check the numerator (as the above case). - std::conditional_t, - NextAvgTypeT, - Float64>>>; + /// When the denominator only is Decimal, it would be converted to Float64 (as integral / Decimal + /// produces a compile error, vice versa allowed), so we just cast the numerator to Flaoat64; + Float64>>; /** * When both types are Decimal, we can perform computations in the Decimals only. @@ -78,10 +78,10 @@ struct AvgWeightedFunctionTypesDeduction * - If the denominator was floating-point, the numerator would be Float64. * - If not, the numerator would be Decimal (as the denominator is integral). * - * When the denominator only is Decimal, the numerator is either integral (so we leave the Decimal), or Float64, - * so we set the denominator to Float64; + * When the denominator only is Decimal, it will be casted to Float64 as integral / Decimal produces a compile + * time error. */ - using Denom = std::conditional, + using Denom = std::conditional_t>; }; From f18232739b904d7b46d39f4827ea8a0fef656315 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 22 Oct 2020 19:32:06 +0300 Subject: [PATCH 0057/1088] some other changes to deduction guides --- .../AggregateFunctionAvgWeighted.h | 38 +++++++++++-------- 1 file changed, 23 insertions(+), 15 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index 8c245253710..6b1c41d2748 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -8,24 +8,26 @@ namespace DB template struct AvgWeightedFunctionTypesDeduction { - template struct NextAvgType { using Type = T; }; + template struct NextAvgType { }; template <> struct NextAvgType { using Type = Int16; }; template <> struct NextAvgType { using Type = Int32; }; template <> struct NextAvgType { using Type = Int64; }; template <> struct NextAvgType { using Type = Int128; }; - template <> struct NextAvgType { using Type = Int256; }; - template <> struct NextAvgType { using Type = Int256; }; template <> struct NextAvgType { using Type = UInt16; }; template <> struct NextAvgType { using Type = UInt32; }; template <> struct NextAvgType { using Type = UInt64; }; template <> struct NextAvgType { using Type = UInt128; }; - template <> struct NextAvgType { using Type = UInt256; }; - template <> struct NextAvgType { using Type = UInt256; }; + + // Promoted to Float as these types don't go well when operating with above ones + template <> struct NextAvgType { using Type = Float64; }; + template <> struct NextAvgType { using Type = Float64; }; + template <> struct NextAvgType { using Type = Float64; }; + template <> struct NextAvgType { using Type = Float64; }; template <> struct NextAvgType { using Type = Decimal128; }; template <> struct NextAvgType { using Type = Decimal128; }; - template <> struct NextAvgType { using Type = Decimal256; }; + template <> struct NextAvgType { using Type = Decimal128; }; template <> struct NextAvgType { using Type = Decimal256; }; template <> struct NextAvgType { using Type = Float64; }; @@ -43,11 +45,11 @@ struct AvgWeightedFunctionTypesDeduction static constexpr bool BothDecimal = UDecimal && VDecimal; static constexpr bool NoneDecimal = !UDecimal && !VDecimal; - template - static constexpr bool IsIntegral = std::is_integral_v; - /// we do not include extended integral types here as they produce errors while diving on Decimals. - /// || std::is_same_v || std::is_same_v - /// || std::is_same_v || std::is_same_v; + /// we do not include extended integral types here as they produce errors while diving on Decimals. + template static constexpr bool IsIntegral = std::is_integral_v; + template static constexpr bool IsExtendedIntegral = + std::is_same_v || std::is_same_v + || std::is_same_v || std::is_same_v; static constexpr bool BothOrNoneDecimal = BothDecimal || NoneDecimal; @@ -80,8 +82,11 @@ struct AvgWeightedFunctionTypesDeduction * * When the denominator only is Decimal, it will be casted to Float64 as integral / Decimal produces a compile * time error. + * + * Extended integer types can't be multiplied by doubles (I don't know, why), so we also convert them to + * double. */ - using Denom = std::conditional_t, Float64, NextAvgTypeT>; }; @@ -123,10 +128,13 @@ public: const auto & values = static_cast &>(*columns[0]); const auto & weights = static_cast &>(*columns[1]); - const auto value = values.getData()[row_num]; - const auto weight = weights.getData()[row_num]; + using Numerator = typename Base::Numerator; + using Denominator = typename Base::Denominator; - this->data(place).numerator += static_cast(value) * weight; + const Numerator value = Numerator(values.getData()[row_num]); + const Denominator weight = Denominator(weights.getData()[row_num]); + + this->data(place).numerator += value * weight; this->data(place).denominator += weight; } From 6a3ceba9a33818de6fa5cfb4d9aaae8169f4da94 Mon Sep 17 00:00:00 2001 From: filipe Date: Fri, 23 Oct 2020 01:28:25 -0300 Subject: [PATCH 0058/1088] add countMatches sql function (issue #15413) --- .../functions/string-search-functions.md | 4 + src/Functions/countMatches.cpp | 13 ++ src/Functions/countMatchesImpl.h | 132 ++++++++++++++++++ src/Functions/registerFunctionsString.cpp | 2 + 4 files changed, 151 insertions(+) create mode 100644 src/Functions/countMatches.cpp create mode 100644 src/Functions/countMatchesImpl.h diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index a625af14505..5f97ff304b3 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -394,3 +394,7 @@ For case-insensitive search or/and in UTF-8 format use functions `ngramSearchCas For UTF-8 case we use 3-gram distance. All these are not perfectly fair n-gram distances. We use 2-byte hashes to hash n-grams and then calculate the (non-)symmetric difference between these hash tables – collisions may occur. With UTF-8 case-insensitive format we do not use fair `tolower` function – we zero the 5-th bit (starting from zero) of each codepoint byte and first bit of zeroth byte if bytes more than one – this works for Latin and mostly for all Cyrillic letters. [Original article](https://clickhouse.tech/docs/en/query_language/functions/string_search_functions/) + +## countMatches(haystack, pattern) {#countmatcheshaystack-pattern} + +Returns the number of regular expression matches for a `pattern` in a `haystack`. diff --git a/src/Functions/countMatches.cpp b/src/Functions/countMatches.cpp new file mode 100644 index 00000000000..935b9fb9904 --- /dev/null +++ b/src/Functions/countMatches.cpp @@ -0,0 +1,13 @@ +#include "FunctionFactory.h" +#include "countMatchesImpl.h" + + +namespace DB +{ + +void registerFunctionCountMatches(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/countMatchesImpl.h b/src/Functions/countMatchesImpl.h new file mode 100644 index 00000000000..86e25a252ce --- /dev/null +++ b/src/Functions/countMatchesImpl.h @@ -0,0 +1,132 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; +} + +using Pos = const char *; + +class FunctionCountMatches : public IFunction +{ +public: + static constexpr auto name = "countMatches"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[1])) + throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be String.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override + { + + const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[1]).column.get()); + + if (!col) + throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + + " of first argument of function " + getName() + ". Must be constant string.", + ErrorCodes::ILLEGAL_COLUMN); + + Regexps::Pool::Pointer re = Regexps::get(col->getValue()); + size_t capture = re->getNumberOfSubpatterns() > 0 ? 1 : 0; + OptimizedRegularExpression::MatchVec matches; + matches.resize(capture + 1); + + size_t array_argument_position = arguments[0]; + + const ColumnString * col_str = checkAndGetColumn(block.getByPosition(array_argument_position).column.get()); + const ColumnConst * col_const_str = + checkAndGetColumnConstStringOrFixedString(block.getByPosition(array_argument_position).column.get()); + + auto col_res = ColumnUInt64::create(); + ColumnUInt64::Container & vec_res = col_res->getData(); + + if (col_str) + { + const ColumnString::Chars & src_chars = col_str->getChars(); + const ColumnString::Offsets & src_offsets = col_str->getOffsets(); + + vec_res.resize(src_offsets.size()); + + size_t size = src_offsets.size(); + ColumnString::Offset current_src_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + Pos pos = reinterpret_cast(&src_chars[current_src_offset]); + current_src_offset = src_offsets[i]; + Pos end = reinterpret_cast(&src_chars[current_src_offset]) - 1; + + uint64_t match_count = 0; + while (true) + { + if (!pos || pos > end) + break; + if (!re->match(pos, end - pos, matches) || !matches[0].length) + break; + pos += matches[0].offset + matches[0].length; + match_count++; + } + vec_res[i] = match_count; + } + + block.getByPosition(result).column = std::move(col_res); + } + else if (col_const_str) + { + String src = col_const_str->getValue(); + + Pos pos = reinterpret_cast(src.data()); + Pos end = reinterpret_cast(src.data() + src.size()); + + uint64_t match_count = 0; + while (true) + { + if (!pos || pos > end) + break; + if (!re->match(pos, end - pos, matches) || !matches[0].length) + break; + pos += matches[0].offset + matches[0].length; + match_count++; + } + + block.getByPosition(result).column = DataTypeUInt64().createColumnConst(col_const_str->size(), match_count); + } + else + throw Exception("Illegal columns " + block.getByPosition(array_argument_position).column->getName() + + ", " + block.getByPosition(array_argument_position).column->getName() + + " of arguments of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +} diff --git a/src/Functions/registerFunctionsString.cpp b/src/Functions/registerFunctionsString.cpp index 5d4c165e1e3..647f63fe910 100644 --- a/src/Functions/registerFunctionsString.cpp +++ b/src/Functions/registerFunctionsString.cpp @@ -32,6 +32,7 @@ void registerFunctionTrim(FunctionFactory &); void registerFunctionRegexpQuoteMeta(FunctionFactory &); void registerFunctionNormalizeQuery(FunctionFactory &); void registerFunctionNormalizedQueryHash(FunctionFactory &); +void registerFunctionCountMatches(FunctionFactory &); #if USE_BASE64 void registerFunctionBase64Encode(FunctionFactory &); @@ -66,6 +67,7 @@ void registerFunctionsString(FunctionFactory & factory) registerFunctionRegexpQuoteMeta(factory); registerFunctionNormalizeQuery(factory); registerFunctionNormalizedQueryHash(factory); + registerFunctionCountMatches(factory); #if USE_BASE64 registerFunctionBase64Encode(factory); registerFunctionBase64Decode(factory); From 08009631355ad31eb630e4b5f153c270cbbcf811 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Fri, 23 Oct 2020 17:15:55 +0800 Subject: [PATCH 0059/1088] Update parser for column transformers --- src/Parsers/ASTColumnsTransformers.cpp | 18 +++-- src/Parsers/ExpressionElementParsers.cpp | 70 ++++++++++++------- .../01470_columns_transformers.reference | 5 ++ .../01470_columns_transformers.sql | 4 ++ 4 files changed, 67 insertions(+), 30 deletions(-) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 474c3262d78..719c818b82d 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -33,7 +33,7 @@ void IASTColumnsTransformer::transform(const ASTPtr & transformer, ASTs & nodes) void ASTColumnsApplyTransformer::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "APPLY" << (settings.hilite ? hilite_none : "") << "(" << func_name << ")"; + settings.ostr << (settings.hilite ? hilite_keyword : "") << "APPLY" << (settings.hilite ? hilite_none : "") << " " << func_name; } void ASTColumnsApplyTransformer::transform(ASTs & nodes) const @@ -46,7 +46,10 @@ void ASTColumnsApplyTransformer::transform(ASTs & nodes) const void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXCEPT" << (is_strict ? " STRICT " : "") << (settings.hilite ? hilite_none : "") << "("; + settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXCEPT" << (is_strict ? " STRICT " : "") << (settings.hilite ? hilite_none : ""); + + if (children.size() > 1) + settings.ostr << " ("; for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { @@ -57,7 +60,8 @@ void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, Fo (*it)->formatImpl(settings, state, frame); } - settings.ostr << ")"; + if (children.size() > 1) + settings.ostr << ")"; } void ASTColumnsExceptTransformer::transform(ASTs & nodes) const @@ -110,7 +114,10 @@ void ASTColumnsReplaceTransformer::Replacement::formatImpl( void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "REPLACE" << (is_strict ? " STRICT " : "") << (settings.hilite ? hilite_none : "") << "("; + settings.ostr << (settings.hilite ? hilite_keyword : "") << "REPLACE" << (is_strict ? " STRICT " : "") << (settings.hilite ? hilite_none : ""); + + if (children.size() > 1) + settings.ostr << " ("; for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { @@ -121,7 +128,8 @@ void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, F (*it)->formatImpl(settings, state, frame); } - settings.ostr << ")"; + if (children.size() > 1) + settings.ostr << ")"; } void ASTColumnsReplaceTransformer::replaceChildren(ASTPtr & node, const ASTPtr & replacement, const String & name) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index dc8f39ae894..501a9b00f3b 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1240,17 +1240,24 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e if (apply.ignore(pos, expected)) { - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; + bool with_open_round_bracket = false; + + if (pos->type == TokenType::OpeningRoundBracket) + { + ++pos; + with_open_round_bracket = true; + } String func_name; if (!parseIdentifierOrStringLiteral(pos, expected, func_name)) return false; - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; + if (with_open_round_bracket) + { + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + } auto res = std::make_shared(); res->func_name = func_name; @@ -1259,14 +1266,9 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e } else if (except.ignore(pos, expected)) { - if (strict.ignore(pos, expected)) is_strict = true; - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - ASTs identifiers; auto parse_id = [&identifiers, &pos, &expected] { @@ -1278,12 +1280,23 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e return true; }; - if (!ParserList::parseUtil(pos, expected, parse_id, false)) - return false; + if (pos->type == TokenType::OpeningRoundBracket) + { + // support one or more parameter + ++pos; + if (!ParserList::parseUtil(pos, expected, parse_id, false)) + return false; - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + } + else + { + // only one parameter + if (!parse_id()) + return false; + } auto res = std::make_shared(); res->children = std::move(identifiers); @@ -1296,10 +1309,6 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e if (strict.ignore(pos, expected)) is_strict = true; - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - ASTs replacements; ParserExpression element_p; ParserIdentifier ident_p; @@ -1323,12 +1332,23 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e return true; }; - if (!ParserList::parseUtil(pos, expected, parse_id, false)) - return false; + if (pos->type == TokenType::OpeningRoundBracket) + { + ++pos; - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; + if (!ParserList::parseUtil(pos, expected, parse_id, false)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + } + else + { + // only one parameter + if (!parse_id()) + return false; + } auto res = std::make_shared(); res->children = std::move(replacements); diff --git a/tests/queries/0_stateless/01470_columns_transformers.reference b/tests/queries/0_stateless/01470_columns_transformers.reference index 2d8a1802289..9a8a02c7f98 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.reference +++ b/tests/queries/0_stateless/01470_columns_transformers.reference @@ -1,4 +1,5 @@ 220 18 347 +220 18 347 110 9 173.5 1970-04-11 1970-01-11 1970-11-21 2 3 @@ -6,6 +7,10 @@ 18 347 110 173.5 1970-04-11 1970-01-11 1970-11-21 +10 324 +8 23 +101 10 324 +121 8 23 222 18 347 111 11 173.5 1970-04-11 1970-01-11 1970-11-21 diff --git a/tests/queries/0_stateless/01470_columns_transformers.sql b/tests/queries/0_stateless/01470_columns_transformers.sql index a3d103cd876..f7c30eef61b 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.sql +++ b/tests/queries/0_stateless/01470_columns_transformers.sql @@ -4,6 +4,7 @@ CREATE TABLE columns_transformers (i Int64, j Int16, k Int64) Engine=TinyLog; INSERT INTO columns_transformers VALUES (100, 10, 324), (120, 8, 23); SELECT * APPLY(sum) from columns_transformers; +SELECT * APPLY sum from columns_transformers; SELECT columns_transformers.* APPLY(avg) from columns_transformers; SELECT a.* APPLY(toDate) APPLY(any) from columns_transformers a; SELECT COLUMNS('[jk]') APPLY(toString) APPLY(length) from columns_transformers; @@ -13,7 +14,10 @@ SELECT columns_transformers.* EXCEPT(j) APPLY(avg) from columns_transformers; -- EXCEPT after APPLY will not match anything SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a; +SELECT * EXCEPT STRICT i from columns_transformers; +SELECT * EXCEPT STRICT i, j1 from columns_transformers; -- { serverError 47 } SELECT * EXCEPT STRICT(i, j1) from columns_transformers; -- { serverError 16 } +SELECT * REPLACE STRICT i + 1 AS i from columns_transformers; SELECT * REPLACE STRICT(i + 1 AS col) from columns_transformers; -- { serverError 16 } SELECT * REPLACE(i + 1 AS i) APPLY(sum) from columns_transformers; SELECT columns_transformers.* REPLACE(j + 2 AS j, i + 1 AS i) APPLY(avg) from columns_transformers; From 1399f8889e9c4ae703e0da7833106ac9a1b21414 Mon Sep 17 00:00:00 2001 From: annvsh Date: Fri, 23 Oct 2020 22:51:08 +0700 Subject: [PATCH 0060/1088] Fixed --- docs/en/interfaces/formats.md | 35 ++++++++++++++------------------- docs/ru/interfaces/formats.md | 37 +++++++++++++++-------------------- 2 files changed, 31 insertions(+), 41 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 46b29478fc8..bb96f6c62a9 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1342,34 +1342,29 @@ Result: ## RawBLOB {#rawblob} This format slurps all input data into a single value. This format can only parse a table with a single field of type [String](../sql-reference/data-types/string.md) or similar. -When an empty value is passed to the input, ClickHouse generates an exception: +The result is output in binary format without delimiters and escaping. If more than one value is output, the format is ambiguous, and it will be impossible to read the data back. + +The difference between `RawBLOB` and `TSVRaw`: +- data is output in binary format, no escaping; +- no delimiters between values; +- no newline at the end of each value. + +In `Raw BLOB` unlike `Raw Binary` strings are output without their length. + +When an empty value is passed to the `RawBLOB` input, ClickHouse generates an exception: ``` text Code: 108. DB::Exception: No data to insert ``` - -The result is output in binary format without delimiters and escaping. If more than one value is output, the format is ambiguous, and it will be impossible to read the data back. - **Example** ``` bash -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. "$CURDIR"/../shell_config.sh - -${CLICKHOUSE_CLIENT} -n --query " -DROP TABLE IF EXISTS t; -CREATE TABLE t (a LowCardinality(Nullable(String))) ENGINE = Memory; - -${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT RawBLOB" < ${BASH_SOURCE[0]} - -cat ${BASH_SOURCE[0]} | md5sum - -${CLICKHOUSE_CLIENT} -n --query "SELECT * FROM t FORMAT RawBLOB" | md5sum - -${CLICKHOUSE_CLIENT} --query " -DROP TABLE t; -" +$ clickhouse-client --query "DROP TABLE IF EXISTS {some_table};" +$ clickhouse-client --query "CREATE TABLE {some_table} (a String) ENGINE = Memory;" +$ cat {filename} | clickhouse-client --query="INSERT INTO {some_table} FORMAT RawBLOB" +$ clickhouse-client --query "SELECT * FROM {some_table} FORMAT RawBLOB" | md5sum +$ clickhouse-client --query "DROP TABLE {some_table};" ``` Result: diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 3519b29c2d4..e65de3d74e0 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -1146,35 +1146,30 @@ SELECT * FROM line_as_string; ## RawBLOB {#rawblob} -Этот формат считывает все входные данные в одно значение. Этот формат может парсить только таблицу с одним полем типа [String](../sql-reference/data-types/string.md) или подобным ему. -При передаче на вход пустого значения ClickHouse сгенерирует исключение: +Этот формат считывает все входные данные в одно значение. Формат может парсить только таблицу с одним полем типа [String](../sql-reference/data-types/string.md) или подобным ему. +Результат выводится в бинарном виде без разделителей и экранирования. При выводе более одного значения формат неоднозначен и будет невозможно прочитать данные снова. + +Отличия между `RawBLOB` и `TabSeparatedRaw`: +- данные выводятся в бинарном виде, без экранирования; +- нет разделителей между значениями; +- нет новой строки в конце каждого значения. + +В `RawBLOB`, в отличие от `RowBinary`, строки выводятся без их длины. + +При передаче на вход `RawBLOB` пустого значения ClickHouse сгенерирует исключение: ``` text Code: 108. DB::Exception: No data to insert ``` - -Результат выводится в двоичном формате без разделителей и экранирования. При выводе более одного значения формат неоднозначен и будет невозможно прочитать данные снова. - **Пример** ``` bash -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. "$CURDIR"/../shell_config.sh - -${CLICKHOUSE_CLIENT} -n --query " -DROP TABLE IF EXISTS t; -CREATE TABLE t (a LowCardinality(Nullable(String))) ENGINE = Memory; - -${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT RawBLOB" < ${BASH_SOURCE[0]} - -cat ${BASH_SOURCE[0]} | md5sum - -${CLICKHOUSE_CLIENT} -n --query "SELECT * FROM t FORMAT RawBLOB" | md5sum - -${CLICKHOUSE_CLIENT} --query " -DROP TABLE t; -" +$ clickhouse-client --query "DROP TABLE IF EXISTS {some_table};" +$ clickhouse-client --query "CREATE TABLE {some_table} (a String) ENGINE = Memory;" +$ cat {filename} | clickhouse-client --query="INSERT INTO {some_table} FORMAT RawBLOB" +$ clickhouse-client --query "SELECT * FROM {some_table} FORMAT RawBLOB" | md5sum +$ clickhouse-client --query "DROP TABLE {some_table};" ``` Результат: From 15e4e0346972779b99e6c4456f554e52cf5c4edb Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 24 Oct 2020 13:18:04 +0000 Subject: [PATCH 0061/1088] add union distinct and setting union_default_mode --- src/Common/ErrorCodes.cpp | 2 + src/Core/Settings.h | 3 +- src/Core/SettingsEnums.cpp | 6 +++ src/Core/SettingsEnums.h | 10 ++++- .../InterpreterSelectWithUnionQuery.cpp | 37 +++++++++++++++- .../InterpreterSelectWithUnionQuery.h | 4 +- src/Parsers/ASTSelectWithUnionQuery.cpp | 20 +++++++-- src/Parsers/ASTSelectWithUnionQuery.h | 9 ++++ src/Parsers/ParserSelectWithUnionQuery.cpp | 42 +++++++++++++++++-- 9 files changed, 122 insertions(+), 11 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b841368f662..499b02ca0d9 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -519,6 +519,8 @@ namespace ErrorCodes extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND = 2001; extern const int ILLEGAL_PROJECTION_MANIPULATOR = 2002; extern const int UNRECOGNIZED_ARGUMENTS = 2003; + extern const int UNKNOWN_UNION = 2004; + extern const int EXPECTED_ALL_OR_DISTINCT = 2005; } } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d73098ca6e0..f9431c133e3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -400,7 +400,8 @@ class IColumn; M(Bool, optimize_trivial_insert_select, true, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \ M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing. Will be removed after 2021-02-12", 0) \ M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ - M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) + M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) \ + M(UnionMode, union_default_mode, UnionMode::ALL, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS below. diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index c337cd5e3ce..2e1cf025256 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -12,6 +12,7 @@ namespace ErrorCodes extern const int UNKNOWN_JOIN; extern const int BAD_ARGUMENTS; extern const int UNKNOWN_MYSQL_DATATYPES_SUPPORT_LEVEL; + extern const int UNKNOWN_UNION; } @@ -96,4 +97,9 @@ IMPLEMENT_SETTING_MULTI_ENUM(MySQLDataTypesSupport, ErrorCodes::UNKNOWN_MYSQL_DA {{"decimal", MySQLDataTypesSupport::DECIMAL}, {"datetime64", MySQLDataTypesSupport::DATETIME64}}) +IMPLEMENT_SETTING_ENUM(UnionMode, ErrorCodes::UNKNOWN_UNION, + {{"", UnionMode::Unspecified}, + {"ALL", UnionMode::ALL}, + {"DISTINCT", UnionMode::DISTINCT}}) + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 80b9bf9adde..2546ebe3237 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -119,7 +119,6 @@ enum class DefaultDatabaseEngine DECLARE_SETTING_ENUM(DefaultDatabaseEngine) - enum class MySQLDataTypesSupport { DECIMAL, // convert MySQL's decimal and number to ClickHouse Decimal when applicable @@ -129,4 +128,13 @@ enum class MySQLDataTypesSupport DECLARE_SETTING_MULTI_ENUM(MySQLDataTypesSupport) +enum class UnionMode +{ + Unspecified = 0, // Query UNION without UnionMode will throw exception + ALL, // Query UNION without UnionMode -> SELECT ... UNION ALL SELECT ... + DISTINCT // Query UNION without UnionMode -> SELECT ... UNION DISTINCT SELECT ... +}; + +DECLARE_SETTING_ENUM(UnionMode) + } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index ba0ebfaaf27..b31723383f7 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB @@ -18,6 +19,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int UNION_ALL_RESULT_STRUCTURES_MISMATCH; + extern const int EXPECTED_ALL_OR_DISTINCT; } @@ -31,13 +33,35 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( context(std::make_shared(context_)), max_streams(context->getSettingsRef().max_threads) { - const auto & ast = query_ptr->as(); + auto & ast = query_ptr->as(); size_t num_selects = ast.list_of_selects->children.size(); if (!num_selects) throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR); + /// For SELECT ... UNION/UNION ALL/UNION DISTINCT SELECT ... query, + /// rewrite ast with settings.union_default_mode + if (num_selects > 1) + { + if (ast.mode == ASTSelectWithUnionQuery::Mode::Unspecified) + { + const Settings & settings = context->getSettingsRef(); + if (settings.union_default_mode == UnionMode::ALL) + ast.mode = ASTSelectWithUnionQuery::Mode::ALL; + else if (settings.union_default_mode == UnionMode::DISTINCT) + { + ast.mode = ASTSelectWithUnionQuery::Mode::DISTINCT; + } + else + throw Exception( + "Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty", + DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); + } + if (ast.mode == ASTSelectWithUnionQuery::Mode::DISTINCT) + distinct_union = true; + } + /// Initialize interpreters for each SELECT query. /// Note that we pass 'required_result_column_names' to first SELECT. /// And for the rest, we pass names at the corresponding positions of 'required_result_column_names' in the result of first SELECT, @@ -197,6 +221,17 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) auto union_step = std::make_unique(std::move(data_streams), result_header, max_threads); query_plan.unitePlans(std::move(union_step), std::move(plans)); + + /// Add distinct transform for UNION DISTINCT query + if (distinct_union) + { + const Settings & settings = context->getSettingsRef(); + SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); + + auto distinct_step = std::make_unique(query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false); + + query_plan.addStep(std::move(distinct_step)); + } } BlockIO InterpreterSelectWithUnionQuery::execute() diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index 5590066a4db..dceba011418 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -12,7 +12,7 @@ class Context; class InterpreterSelectQuery; class QueryPlan; -/** Interprets one or multiple SELECT queries inside UNION ALL chain. +/** Interprets one or multiple SELECT queries inside UNION/UNION ALL/UNION DISTINCT chain. */ class InterpreterSelectWithUnionQuery : public IInterpreter { @@ -54,6 +54,8 @@ private: size_t max_streams = 1; + bool distinct_union = false; + static Block getCommonHeaderForUnion(const Blocks & headers); }; diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 96cac839c58..f2b64b37089 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -23,13 +23,25 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F { std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + std::string mode_str; + switch (mode) + { + case Mode::Unspecified: + mode_str = ""; + break; + case Mode::ALL: + mode_str = "ALL"; + break; + case Mode::DISTINCT: + mode_str = "DISTINCT"; + break; + } + for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it) { if (it != list_of_selects->children.begin()) - settings.ostr - << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") - << "UNION ALL" << (settings.hilite ? hilite_none : "") - << settings.nl_or_ws; + settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION " << mode_str + << (settings.hilite ? hilite_none : "") << settings.nl_or_ws; (*it)->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index 41ec8bb1076..9c476d3674a 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -17,6 +17,15 @@ public: ASTPtr clone() const override; void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; + enum class Mode + { + Unspecified, + ALL, + DISTINCT + }; + + Mode mode = Mode::Unspecified; + ASTPtr list_of_selects; }; diff --git a/src/Parsers/ParserSelectWithUnionQuery.cpp b/src/Parsers/ParserSelectWithUnionQuery.cpp index cebe8ba876d..bf9c9cf552c 100644 --- a/src/Parsers/ParserSelectWithUnionQuery.cpp +++ b/src/Parsers/ParserSelectWithUnionQuery.cpp @@ -27,15 +27,51 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & { ASTPtr list_node; - ParserList parser(std::make_unique(), std::make_unique("UNION ALL"), false); - if (!parser.parse(pos, list_node, expected)) - return false; + ParserList parser_union(std::make_unique(), std::make_unique("UNION"), false); + ParserList parser_union_all(std::make_unique(), std::make_unique("UNION ALL"), false); + ParserList parser_union_distinct(std::make_unique(), std::make_unique("UNION DISTINCT"), false); + + auto begin = pos; + auto current_expected = expected; + ASTSelectWithUnionQuery::Mode union_mode = ASTSelectWithUnionQuery::Mode::ALL; + + /// Parser SELECT lists and UNION type, must have UNION + auto union_parser = [&](auto & parser, auto mode) { + if (!parser.parse(pos, list_node, expected)) + { + pos = begin; + expected = current_expected; + return false; + } + /// number of SELECT lists should not less than 2 + if (list_node->children.size() < 2) + { + pos = begin; + expected = current_expected; + return false; + } + union_mode = mode; + return true; + }; + + /// We first parse: SELECT ... UNION SELECT ... + /// SELECT ... UNION ALL SELECT ... + /// SELECT ... UNION DISTINCT SELECT ... + if (!union_parser(parser_union, ASTSelectWithUnionQuery::Mode::Unspecified) + && !union_parser(parser_union_all, ASTSelectWithUnionQuery::Mode::ALL) + && !union_parser(parser_union_distinct, ASTSelectWithUnionQuery::Mode::DISTINCT)) + { + /// If above parse failed, we back to parse SELECT without UNION + if (!parser_union.parse(pos, list_node, expected)) + return false; + } auto select_with_union_query = std::make_shared(); node = select_with_union_query; select_with_union_query->list_of_selects = std::make_shared(); select_with_union_query->children.push_back(select_with_union_query->list_of_selects); + select_with_union_query->mode = union_mode; // flatten inner union query for (auto & child : list_node->children) From 4d0c87816a6eb35e3aa5562f68122477f65d07a0 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 24 Oct 2020 13:22:25 +0000 Subject: [PATCH 0062/1088] add test --- ...t_and_setting_union_default_mode.reference | 140 ++++++++++++++++++ ...istinct_and_setting_union_default_mode.sql | 13 ++ 2 files changed, 153 insertions(+) create mode 100644 tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference create mode 100644 tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql diff --git a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference new file mode 100644 index 00000000000..6c4547333fe --- /dev/null +++ b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference @@ -0,0 +1,140 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql new file mode 100644 index 00000000000..1a6e582aebe --- /dev/null +++ b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql @@ -0,0 +1,13 @@ +SELECT * FROM numbers(10) UNION SELECT * FROM numbers(10); +SELECT * FROM numbers(10) UNION ALL SELECT * FROM numbers(10); +SELECT * FROM numbers(10) UNION DISTINCT SELECT * FROM numbers(10); + +SET union_default_mode='ALL'; +SELECT * FROM numbers(10) UNION SELECT * FROM numbers(10); +SELECT * FROM numbers(10) UNION ALL SELECT * FROM numbers(10); +SELECT * FROM numbers(10) UNION DISTINCT SELECT * FROM numbers(10); + +SET union_default_mode='DISTINCT'; +SELECT * FROM numbers(10) UNION SELECT * FROM numbers(10); +SELECT * FROM numbers(10) UNION ALL SELECT * FROM numbers(10); +SELECT * FROM numbers(10) UNION DISTINCT SELECT * FROM numbers(10); From b2a3ace6d1f2ac802a2acfccad9966eec37284fa Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 24 Oct 2020 13:41:20 +0000 Subject: [PATCH 0063/1088] fix --- src/Core/SettingsEnums.h | 1 + src/Interpreters/InterpreterSelectWithUnionQuery.cpp | 4 ++-- src/Parsers/ASTSelectWithUnionQuery.cpp | 7 +++++-- src/Parsers/ParserSelectWithUnionQuery.cpp | 7 ++++--- 4 files changed, 12 insertions(+), 7 deletions(-) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 2546ebe3237..c2ef08135eb 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -119,6 +119,7 @@ enum class DefaultDatabaseEngine DECLARE_SETTING_ENUM(DefaultDatabaseEngine) + enum class MySQLDataTypesSupport { DECIMAL, // convert MySQL's decimal and number to ClickHouse Decimal when applicable diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index b31723383f7..f4ef78dd183 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -47,17 +47,17 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( if (ast.mode == ASTSelectWithUnionQuery::Mode::Unspecified) { const Settings & settings = context->getSettingsRef(); + if (settings.union_default_mode == UnionMode::ALL) ast.mode = ASTSelectWithUnionQuery::Mode::ALL; else if (settings.union_default_mode == UnionMode::DISTINCT) - { ast.mode = ASTSelectWithUnionQuery::Mode::DISTINCT; - } else throw Exception( "Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty", DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); } + if (ast.mode == ASTSelectWithUnionQuery::Mode::DISTINCT) distinct_union = true; } diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index f2b64b37089..7a61b3b5da1 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -40,8 +40,11 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it) { if (it != list_of_selects->children.begin()) - settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION " << mode_str - << (settings.hilite ? hilite_none : "") << settings.nl_or_ws; + settings.ostr << settings.nl_or_ws << indent_str + << (settings.hilite ? hilite_keyword : "") + << "UNION " << mode_str + << (settings.hilite ? hilite_none : "") + << settings.nl_or_ws; (*it)->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ParserSelectWithUnionQuery.cpp b/src/Parsers/ParserSelectWithUnionQuery.cpp index bf9c9cf552c..5b98b083fd4 100644 --- a/src/Parsers/ParserSelectWithUnionQuery.cpp +++ b/src/Parsers/ParserSelectWithUnionQuery.cpp @@ -28,15 +28,16 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ASTPtr list_node; ParserList parser_union(std::make_unique(), std::make_unique("UNION"), false); - ParserList parser_union_all(std::make_unique(), std::make_unique("UNION ALL"), false); - ParserList parser_union_distinct(std::make_unique(), std::make_unique("UNION DISTINCT"), false); + ParserList parser_union_all(std::make_unique(), std::make_unique("UNION ALL"), false); + ParserList parser_union_distinct(std::make_unique(), std::make_unique("UNION DISTINCT"), false); auto begin = pos; auto current_expected = expected; ASTSelectWithUnionQuery::Mode union_mode = ASTSelectWithUnionQuery::Mode::ALL; /// Parser SELECT lists and UNION type, must have UNION - auto union_parser = [&](auto & parser, auto mode) { + auto union_parser = [&](auto & parser, auto mode) + { if (!parser.parse(pos, list_node, expected)) { pos = begin; From dcf4b0df1f20b391963db5ccda93b7a1b1075583 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 24 Oct 2020 14:03:15 +0000 Subject: [PATCH 0064/1088] fix --- src/Parsers/ASTSelectWithUnionQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 7a61b3b5da1..1ef47257eae 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -40,7 +40,7 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it) { if (it != list_of_selects->children.begin()) - settings.ostr << settings.nl_or_ws << indent_str + settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION " << mode_str << (settings.hilite ? hilite_none : "") From 5c192df20f974f26f24fcdaf4259468e10c545b7 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 24 Oct 2020 15:40:10 +0000 Subject: [PATCH 0065/1088] update documents --- docs/en/sql-reference/statements/select/index.md | 4 ++-- .../statements/select/{union-all.md => union.md} | 11 +++++++---- 2 files changed, 9 insertions(+), 6 deletions(-) rename docs/en/sql-reference/statements/select/{union-all.md => union.md} (64%) diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 07be8c2bf45..42d3e81f226 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -25,7 +25,7 @@ SELECT [DISTINCT] expr_list [ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [LIMIT [offset_value, ]n BY columns] [LIMIT [n, ]m] [WITH TIES] -[UNION ALL ...] +[UNION ...] [INTO OUTFILE filename] [FORMAT format] ``` @@ -46,7 +46,7 @@ Specifics of each optional clause are covered in separate sections, which are li - [SELECT clause](#select-clause) - [DISTINCT clause](../../../sql-reference/statements/select/distinct.md) - [LIMIT clause](../../../sql-reference/statements/select/limit.md) -- [UNION ALL clause](../../../sql-reference/statements/select/union-all.md) +- [UNION clause](../../../sql-reference/statements/select/union.md) - [INTO OUTFILE clause](../../../sql-reference/statements/select/into-outfile.md) - [FORMAT clause](../../../sql-reference/statements/select/format.md) diff --git a/docs/en/sql-reference/statements/select/union-all.md b/docs/en/sql-reference/statements/select/union.md similarity index 64% rename from docs/en/sql-reference/statements/select/union-all.md rename to docs/en/sql-reference/statements/select/union.md index 5230363609e..88211858707 100644 --- a/docs/en/sql-reference/statements/select/union-all.md +++ b/docs/en/sql-reference/statements/select/union.md @@ -1,5 +1,5 @@ --- -toc_title: UNION ALL +toc_title: UNION --- # UNION ALL Clause {#union-all-clause} @@ -25,10 +25,13 @@ Type casting is performed for unions. For example, if two queries being combined Queries that are parts of `UNION ALL` can’t be enclosed in round brackets. [ORDER BY](../../../sql-reference/statements/select/order-by.md) and [LIMIT](../../../sql-reference/statements/select/limit.md) are applied to separate queries, not to the final result. If you need to apply a conversion to the final result, you can put all the queries with `UNION ALL` in a subquery in the [FROM](../../../sql-reference/statements/select/from.md) clause. -## Limitations {#limitations} +# UNION DISTINCT Clause {#union-distinct-clause} +The difference between `UNION ALL` and `UNION DISTINCT` is that `UNION DISTINCT` will do a distinct transform for union result, it is equivalent to `SELECT DISTINCT` from a subquery containing `UNION ALL`. + +# UNION Clause {#union-clause} +By defaul, `UNION` has same react as `UNION ALL`, but you can specify union mode by setting `union_default_mode`, values can be 'ALL', 'DISTINCT' or empty string. However, if you use `UNION` with setting `union_default_mode` to empty string, it will throw an exception. -Only `UNION ALL` is supported. The regular `UNION` (`UNION DISTINCT`) is not supported. If you need `UNION DISTINCT`, you can write `SELECT DISTINCT` from a subquery containing `UNION ALL`. ## Implementation Details {#implementation-details} -Queries that are parts of `UNION ALL` can be run simultaneously, and their results can be mixed together. +Queries that are parts of `UNION/UNION ALL/UNION DISTINCT` can be run simultaneously, and their results can be mixed together. From d8370116c1cabcb9d46ad2f3a7cc290a0a180b7c Mon Sep 17 00:00:00 2001 From: myrrc Date: Sun, 25 Oct 2020 23:33:01 +0300 Subject: [PATCH 0066/1088] simplified the functions (agreement to cast to Float64) --- .../aggregate-functions/reference/avg.md | 3 +- .../reference/avgweighted.md | 6 +- .../AggregateFunctionAvg.cpp | 22 +-- src/AggregateFunctions/AggregateFunctionAvg.h | 111 ++++----------- .../AggregateFunctionAvgWeighted.cpp | 42 +----- .../AggregateFunctionAvgWeighted.h | 128 +----------------- .../AggregateFunctionFactory.h | 4 +- 7 files changed, 49 insertions(+), 267 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/avg.md b/docs/en/sql-reference/aggregate-functions/reference/avg.md index 4ebae95b79d..1741bbb744b 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/avg.md +++ b/docs/en/sql-reference/aggregate-functions/reference/avg.md @@ -4,4 +4,5 @@ toc_priority: 5 # avg {#agg_function-avg} -Calculates the average. Only works for numbers. The result is always Float64. +Calculates the average. Only works for numbers (Integral, floating-point, or Decimals). +The result is always Float64. diff --git a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md index a6fb5999fb8..22993f93e16 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md @@ -28,11 +28,7 @@ but may have different types. - `NaN`. If all the weights are equal to 0. - Weighted mean otherwise. -**Return type** - -- `Decimal` if both types are [Decimal](../../../sql-reference/data-types/decimal.md) - or if one type is Decimal and other is Integer. -- [Float64](../../../sql-reference/data-types/float.md) otherwise. +**Return type** is always [Float64](../../../sql-reference/data-types/float.md). **Example** diff --git a/src/AggregateFunctions/AggregateFunctionAvg.cpp b/src/AggregateFunctions/AggregateFunctionAvg.cpp index cf35e99dafb..4d1b01b25fc 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvg.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -13,23 +14,22 @@ namespace ErrorCodes namespace { +constexpr bool allowType(const DataTypePtr& type) noexcept +{ + const WhichDataType t(type); + return t.isInt() || t.isUInt() || t.isFloat() || t.isDecimal(); +} + AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const DataTypes & argument_types, const Array & parameters) { assertNoParameters(name, parameters); assertUnary(name, argument_types); - AggregateFunctionPtr res; - DataTypePtr data_type = argument_types[0]; - - if (isDecimal(data_type)) - res.reset(createWithDecimalType(*data_type, *data_type, argument_types)); - else - res.reset(createWithNumericType(*data_type, argument_types)); - - if (!res) + if (!allowType(argument_types[0])) throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return res; + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(argument_types); } } diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 0b77aa0c537..66ab20cec73 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -10,10 +10,7 @@ namespace DB { -template -using DecimalOrVectorCol = std::conditional_t, ColumnDecimal, ColumnVector>; - -/// A type-fixed rational fraction represented by a pair of #Numerator and #Denominator. +/// A type-fixed fraction represented by a pair of #Numerator and #Denominator. template struct RationalFraction { @@ -26,77 +23,42 @@ struct RationalFraction template Result NO_SANITIZE_UNDEFINED result() const { - if constexpr (std::is_floating_point_v) - if constexpr (std::numeric_limits::is_iec559) - { - if constexpr (is_big_int_v) - return static_cast(numerator) / static_cast(denominator); - else - return static_cast(numerator) / denominator; /// allow division by zero - } + if constexpr (std::is_floating_point_v && std::numeric_limits::is_iec559) + return static_cast(numerator) / denominator; /// allow division by zero if (denominator == static_cast(0)) return static_cast(0); - if constexpr (std::is_same_v) - return static_cast(numerator / static_cast(denominator)); - else - return static_cast(numerator / denominator); + return static_cast(numerator / denominator); } }; /** - * Motivation: ClickHouse has added the Decimal data type, which basically represents a fraction that stores - * the precise (unlike floating-point) result with respect to some scale. + * The discussion showed that the easiest (and simplest) way is to cast both the columns of numerator and denominator + * to Float64. Another way would be to write some template magic that figures out the appropriate numerator + * and denominator (and the resulting type) in favour of extended integral types (UInt128 e.g.) and Decimals ( + * which are a mess themselves). The second way is also a bit useless because now Decimals are not used in functions + * like avg. * - * These decimal types can't be divided by floating point data types, so functions like avg or avgWeighted - * can't return the Floa64 column as a result when of the input columns is Decimal (because that would, in case of - * avgWeighted, involve division numerator (Decimal) / denominator (Float64)). - * - * The rules for determining the output and intermediate storage types for these functions are different, so - * the struct representing the deduction guide is presented. - * - * Given the initial Columns types (e.g. values and weights for avgWeighted, values for avg), - * the struct calculated the output type and the intermediate storage type (that's used by the RationalFraction). - */ -template -struct AvgFunctionTypesDeductionTemplate -{ - using Numerator = int; - using Denominator = int; - using Fraction = RationalFraction; - - using ResultType = bool; - using ResultDataType = bool; - using ResultVectorType = bool; -}; - -/** - * @tparam InitialNumerator The type that the initial numerator column would have (needed to cast the input IColumn to - * appropriate type). - * @tparam InitialDenominator The type that the initial denominator column would have. - * - * @tparam Deduction Function template that, given the numerator and the denominator, finds the actual - * suitable storage and the resulting column type. + * The ability to explicitly specify the denominator is made for avg (it uses the integral value as the denominator is + * simply the length of the supplied list). * * @tparam Derived When deriving from this class, use the child class name as in CRTP, e.g. * class Self : Agg. */ -template class Deduction, class Derived> +template class AggregateFunctionAvgBase : public - IAggregateFunctionDataHelper::Fraction, Derived> + IAggregateFunctionDataHelper, Derived> { public: - using Deducted = Deduction; + using Numerator = Float64; + using Fraction = RationalFraction; - using ResultType = typename Deducted::ResultType; - using ResultDataType = typename Deducted::ResultDataType; - using ResultVectorType = typename Deducted::ResultVectorType; + using ResultType = Float64; + using ResultDataType = DataTypeNumber; + using ResultVectorType = ColumnVector; - using Numerator = typename Deducted::Numerator; - using Denominator = typename Deducted::Denominator; - - using Base = IAggregateFunctionDataHelper; + using Base = IAggregateFunctionDataHelper; /// ctor for native types explicit AggregateFunctionAvgBase(const DataTypes & argument_types_): Base(argument_types_, {}), scale(0) {} @@ -107,10 +69,7 @@ public: DataTypePtr getReturnType() const override { - if constexpr (IsDecimalNumber) - return std::make_shared(ResultDataType::maxPrecision(), scale); - else - return std::make_shared(); + return std::make_shared(); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override @@ -148,38 +107,16 @@ protected: UInt32 scale; }; -template -struct AvgFunctionTypesDeduction -{ - using Numerator = std::conditional_t, - std::conditional_t, - Decimal256, - Decimal128>, - NearestFieldType>; - - using Denominator = V; - using Fraction = RationalFraction; - - using ResultType = std::conditional_t, T, Float64>; - using ResultDataType = std::conditional_t, DataTypeDecimal, DataTypeNumber>; - using ResultVectorType = std::conditional_t, ColumnDecimal, ColumnVector>; -}; - -template -class AggregateFunctionAvg final : - public AggregateFunctionAvgBase> +class AggregateFunctionAvg final : public AggregateFunctionAvgBase { public: - using Base = - AggregateFunctionAvgBase>; - - using Base::Base; + using AggregateFunctionAvgBase::AggregateFunctionAvgBase; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const final { - const auto & column = static_cast &>(*columns[0]); + const auto & column = static_cast &>(*columns[0]); this->data(place).numerator += column.getData()[row_num]; - this->data(place).denominator += 1; + ++this->data(place).denominator; } String getName() const final { return "avg"; } diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp index 5b43aa19a5c..6b677414d87 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -14,7 +15,7 @@ namespace ErrorCodes namespace { -constexpr bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) +constexpr bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) noexcept { const WhichDataType l_dt(left), r_dt(right); @@ -26,39 +27,6 @@ constexpr bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) return allow(l_dt) && allow(r_dt); } -#define AT_SWITCH(LINE) \ - switch (which.idx) \ - { \ - LINE(Int8); LINE(Int16); LINE(Int32); LINE(Int64); LINE(Int128); LINE(Int256); \ - LINE(UInt8); LINE(UInt16); LINE(UInt32); LINE(UInt64); LINE(UInt128); LINE(UInt256); \ - LINE(Decimal32); LINE(Decimal64); LINE(Decimal128); LINE(Decimal256); \ - LINE(Float32); LINE(Float64); \ - default: return nullptr; \ - } - -template -static IAggregateFunction * create(const IDataType & second_type, TArgs && ... args) -{ - const WhichDataType which(second_type); - -#define LINE(Type) \ - case TypeIndex::Type: return new AggregateFunctionAvgWeighted(std::forward(args)...) - AT_SWITCH(LINE) -#undef LINE -} - -// Not using helper functions because there are no templates for binary decimal/numeric function. -template -static IAggregateFunction * create(const IDataType & first_type, const IDataType & second_type, TArgs && ... args) -{ - const WhichDataType which(first_type); - -#define LINE(Type) \ - case TypeIndex::Type: return create(second_type, std::forward(args)...) - AT_SWITCH(LINE) -#undef LINE -} - AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name, const DataTypes & argument_types, const Array & parameters) { assertNoParameters(name, parameters); @@ -74,11 +42,7 @@ AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name " are non-conforming as arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - AggregateFunctionPtr res; - res.reset(create(*data_type, *data_type_weight, argument_types)); - - assert(res); // type checking should be done in allowTypes. - return res; + return std::make_shared(argument_types); } } diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index 6b1c41d2748..c9b60cf9f50 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -5,134 +5,18 @@ namespace DB { -template -struct AvgWeightedFunctionTypesDeduction -{ - template struct NextAvgType { }; - template <> struct NextAvgType { using Type = Int16; }; - template <> struct NextAvgType { using Type = Int32; }; - template <> struct NextAvgType { using Type = Int64; }; - template <> struct NextAvgType { using Type = Int128; }; - - template <> struct NextAvgType { using Type = UInt16; }; - template <> struct NextAvgType { using Type = UInt32; }; - template <> struct NextAvgType { using Type = UInt64; }; - template <> struct NextAvgType { using Type = UInt128; }; - - // Promoted to Float as these types don't go well when operating with above ones - template <> struct NextAvgType { using Type = Float64; }; - template <> struct NextAvgType { using Type = Float64; }; - template <> struct NextAvgType { using Type = Float64; }; - template <> struct NextAvgType { using Type = Float64; }; - - template <> struct NextAvgType { using Type = Decimal128; }; - template <> struct NextAvgType { using Type = Decimal128; }; - template <> struct NextAvgType { using Type = Decimal128; }; - template <> struct NextAvgType { using Type = Decimal256; }; - - template <> struct NextAvgType { using Type = Float64; }; - template <> struct NextAvgType { using Type = Float64; }; - - template using NextAvgTypeT = typename NextAvgType::Type; - template using Largest = std::conditional_t<(sizeof(T) > sizeof(U)), T, U>; - - struct GetNumDenom - { - using U = Values; - using V = Weights; - static constexpr bool UDecimal = IsDecimalNumber; - static constexpr bool VDecimal = IsDecimalNumber; - static constexpr bool BothDecimal = UDecimal && VDecimal; - static constexpr bool NoneDecimal = !UDecimal && !VDecimal; - - /// we do not include extended integral types here as they produce errors while diving on Decimals. - template static constexpr bool IsIntegral = std::is_integral_v; - template static constexpr bool IsExtendedIntegral = - std::is_same_v || std::is_same_v - || std::is_same_v || std::is_same_v; - - static constexpr bool BothOrNoneDecimal = BothDecimal || NoneDecimal; - - using Num = std::conditional_t>, - - std::conditional_t, - NextAvgTypeT, - Float64>, - /// When the denominator only is Decimal, it would be converted to Float64 (as integral / Decimal - /// produces a compile error, vice versa allowed), so we just cast the numerator to Flaoat64; - Float64>>; - - /** - * When both types are Decimal, we can perform computations in the Decimals only. - * When none of the types is Decimal, the result is always correct, the numerator is the next largest type up to - * Float64. - * We use #V only as the denominator accumulates the sum of the weights. - * - * When the numerator only is Decimal, we set the denominator to next Largest type. - * - If the denominator was floating-point, the numerator would be Float64. - * - If not, the numerator would be Decimal (as the denominator is integral). - * - * When the denominator only is Decimal, it will be casted to Float64 as integral / Decimal produces a compile - * time error. - * - * Extended integer types can't be multiplied by doubles (I don't know, why), so we also convert them to - * double. - */ - using Denom = std::conditional_t<(VDecimal && !UDecimal) || IsExtendedIntegral, - Float64, - NextAvgTypeT>; - }; - - using Numerator = typename GetNumDenom::Num; - using Denominator = typename GetNumDenom::Denom; - using Fraction = RationalFraction; - - /// If either Numerator or Denominator are Decimal, the result is also Decimal as everything was checked in - /// GetNumDenom. - using T = std::conditional_t && IsDecimalNumber, - Largest, - std::conditional_t, - Numerator, - std::conditional_t, - Denominator, - bool>>>; // both numerator and denominator are non-decimal. - - using ResultType = std::conditional_t, T, Float64>; - using ResultDataType = std::conditional_t, DataTypeDecimal, DataTypeNumber>; - using ResultVectorType = std::conditional_t, ColumnDecimal, ColumnVector>; -}; - -/** - * @tparam Values The values column type. - * @tparam Weights The weights column type. - */ -template -class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase< - Values, Weights, AvgWeightedFunctionTypesDeduction, AggregateFunctionAvgWeighted> +class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase { public: - using Base = AggregateFunctionAvgBase< - Values, Weights, AvgWeightedFunctionTypesDeduction, AggregateFunctionAvgWeighted>; - using Base::Base; + using AggregateFunctionAvgBase::AggregateFunctionAvgBase; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - const auto & values = static_cast &>(*columns[0]); - const auto & weights = static_cast &>(*columns[1]); + const auto & values = static_cast &>(*columns[0]); + const auto & weights = static_cast &>(*columns[1]); - using Numerator = typename Base::Numerator; - using Denominator = typename Base::Denominator; - - const Numerator value = Numerator(values.getData()[row_num]); - const Denominator weight = Denominator(weights.getData()[row_num]); + const auto value = values.getData()[row_num]; + const auto weight = weights.getData()[row_num]; this->data(place).numerator += value * weight; this->data(place).denominator += weight; diff --git a/src/AggregateFunctions/AggregateFunctionFactory.h b/src/AggregateFunctions/AggregateFunctionFactory.h index 143e6562a30..07db76d8dd1 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.h +++ b/src/AggregateFunctions/AggregateFunctionFactory.h @@ -21,7 +21,8 @@ class IDataType; using DataTypePtr = std::shared_ptr; using DataTypes = std::vector; -/** Creator have arguments: name of aggregate function, types of arguments, values of parameters. +/** + * The invoker has arguments: name of aggregate function, types of arguments, values of parameters. * Parameters are for "parametric" aggregate functions. * For example, in quantileWeighted(0.9)(x, weight), 0.9 is "parameter" and x, weight are "arguments". */ @@ -89,7 +90,6 @@ private: std::optional tryGetPropertiesImpl(const String & name, int recursion_level) const; -private: using AggregateFunctions = std::unordered_map; AggregateFunctions aggregate_functions; From 824c04f87369232133d99f2f3359943c04e19d80 Mon Sep 17 00:00:00 2001 From: hchen9 Date: Sun, 25 Oct 2020 18:12:17 -0700 Subject: [PATCH 0067/1088] Fix conflict code --- src/Storages/StorageProxy.h | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 7b010476b22..22f6e8fa03a 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -146,8 +146,8 @@ public: void checkPartitionCanBeDropped(const ASTPtr & partition) override { getNested()->checkPartitionCanBeDropped(partition); } Strings getDataPaths() const override { return getNested()->getDataPaths(); } StoragePolicyPtr getStoragePolicy() const override { return getNested()->getStoragePolicy(); } - std::optional totalRows() const override { return getNested()->totalRows(); } - std::optional totalBytes() const override { return getNested()->totalBytes(); } + std::optional totalRows(const Context& context) const override { return getNested()->totalRows(context); } + std::optional totalBytes(const Context& context) const override { return getNested()->totalBytes(context); } std::optional lifetimeRows() const override { return getNested()->lifetimeRows(); } std::optional lifetimeBytes() const override { return getNested()->lifetimeBytes(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index cf807055d95..b76c841101d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3719,7 +3719,7 @@ std::optional StorageReplicatedMergeTree::totalRowsByPartitionPredicate( } if (is_valid) res += part->rows_count; - }); + }, context); return res; } From d9bc486c0ea904bf768ffc5df4e06aa22510b6a8 Mon Sep 17 00:00:00 2001 From: hchen9 Date: Sun, 25 Oct 2020 20:57:19 -0700 Subject: [PATCH 0068/1088] Fix StorageJoin and StorageSet --- src/Storages/StorageJoin.cpp | 4 ++-- src/Storages/StorageJoin.h | 4 ++-- src/Storages/StorageSet.cpp | 4 ++-- src/Storages/StorageSet.h | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index c6d85174e68..1aedcb4bddd 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -102,8 +102,8 @@ HashJoinPtr StorageJoin::getJoin(std::shared_ptr analyzed_join) const void StorageJoin::insertBlock(const Block & block) { join->addJoinedBlock(block, true); } size_t StorageJoin::getSize() const { return join->getTotalRowCount(); } -std::optional StorageJoin::totalRows() const { return join->getTotalRowCount(); } -std::optional StorageJoin::totalBytes() const { return join->getTotalByteCount(); } +std::optional StorageJoin::totalRows(const Context&) const { return join->getTotalRowCount(); } +std::optional StorageJoin::totalBytes(const Context&) const { return join->getTotalByteCount(); } void registerStorageJoin(StorageFactory & factory) diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 857f3646441..0567fb4221c 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -46,8 +46,8 @@ public: size_t max_block_size, unsigned num_streams) override; - std::optional totalRows() const override; - std::optional totalBytes() const override; + std::optional totalRows(const Context& context) const override; + std::optional totalBytes(const Context& context) const override; private: Block sample_block; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index b7779d2e550..98ccc783b3c 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -153,8 +153,8 @@ void StorageSet::insertBlock(const Block & block) { set->insertFromBlock(block); void StorageSet::finishInsert() { set->finishInsert(); } size_t StorageSet::getSize() const { return set->getTotalRowCount(); } -std::optional StorageSet::totalRows() const { return set->getTotalRowCount(); } -std::optional StorageSet::totalBytes() const { return set->getTotalByteCount(); } +std::optional StorageSet::totalRows(const Context&) const { return set->getTotalRowCount(); } +std::optional StorageSet::totalBytes(const Context&) const { return set->getTotalByteCount(); } void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) { diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index 98677dcfb15..157b2395260 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -72,8 +72,8 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; - std::optional totalRows() const override; - std::optional totalBytes() const override; + std::optional totalRows(const Context& context) const override; + std::optional totalBytes(const Context& context) const override; private: SetPtr set; From a9e4c7144c1d4b139bde4a0c6884eef7bd99e97a Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 26 Oct 2020 09:33:34 +0000 Subject: [PATCH 0069/1088] fix --- .../sql-reference/statements/select/index.md | 2 +- .../select/{union.md => union-all.md} | 2 +- src/Core/Settings.h | 2 +- .../InterpreterSelectWithUnionQuery.cpp | 124 +++++++++++++----- .../InterpreterSelectWithUnionQuery.h | 4 +- src/Parsers/ASTSelectWithUnionQuery.cpp | 26 ++-- src/Parsers/ASTSelectWithUnionQuery.h | 4 +- src/Parsers/ExpressionListParsers.cpp | 46 +++++++ src/Parsers/ExpressionListParsers.h | 48 +++++++ src/Parsers/ParserSelectWithUnionQuery.cpp | 48 ++----- 10 files changed, 215 insertions(+), 91 deletions(-) rename docs/en/sql-reference/statements/select/{union.md => union-all.md} (86%) diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 42d3e81f226..130000cf4b0 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -46,7 +46,7 @@ Specifics of each optional clause are covered in separate sections, which are li - [SELECT clause](#select-clause) - [DISTINCT clause](../../../sql-reference/statements/select/distinct.md) - [LIMIT clause](../../../sql-reference/statements/select/limit.md) -- [UNION clause](../../../sql-reference/statements/select/union.md) +- [UNION clause](../../../sql-reference/statements/select/union-all.md) - [INTO OUTFILE clause](../../../sql-reference/statements/select/into-outfile.md) - [FORMAT clause](../../../sql-reference/statements/select/format.md) diff --git a/docs/en/sql-reference/statements/select/union.md b/docs/en/sql-reference/statements/select/union-all.md similarity index 86% rename from docs/en/sql-reference/statements/select/union.md rename to docs/en/sql-reference/statements/select/union-all.md index 88211858707..1784da37c9a 100644 --- a/docs/en/sql-reference/statements/select/union.md +++ b/docs/en/sql-reference/statements/select/union-all.md @@ -29,7 +29,7 @@ Queries that are parts of `UNION ALL` can’t be enclosed in round brackets. [OR The difference between `UNION ALL` and `UNION DISTINCT` is that `UNION DISTINCT` will do a distinct transform for union result, it is equivalent to `SELECT DISTINCT` from a subquery containing `UNION ALL`. # UNION Clause {#union-clause} -By defaul, `UNION` has same react as `UNION ALL`, but you can specify union mode by setting `union_default_mode`, values can be 'ALL', 'DISTINCT' or empty string. However, if you use `UNION` with setting `union_default_mode` to empty string, it will throw an exception. +By defaul, `UNION` has same react as `UNION DISTINCT`, but you can specify union mode by setting `union_default_mode`, values can be 'ALL', 'DISTINCT' or empty string. However, if you use `UNION` with setting `union_default_mode` to empty string, it will throw an exception. ## Implementation Details {#implementation-details} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f9431c133e3..50832ecee5c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -401,7 +401,7 @@ class IColumn; M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing. Will be removed after 2021-02-12", 0) \ M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) \ - M(UnionMode, union_default_mode, UnionMode::ALL, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) + M(UnionMode, union_default_mode, UnionMode::DISTINCT, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS below. diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index f4ef78dd183..19f08250430 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -1,3 +1,5 @@ +#include +#include #include #include #include @@ -40,26 +42,38 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( if (!num_selects) throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR); - /// For SELECT ... UNION/UNION ALL/UNION DISTINCT SELECT ... query, - /// rewrite ast with settings.union_default_mode + /// Rewrite ast with settings.union_default_mode if (num_selects > 1) { - if (ast.mode == ASTSelectWithUnionQuery::Mode::Unspecified) + const Settings & settings = context->getSettingsRef(); + for (auto & mode : ast.union_modes) { - const Settings & settings = context->getSettingsRef(); + if (mode == ASTSelectWithUnionQuery::Mode::Unspecified) + { - if (settings.union_default_mode == UnionMode::ALL) - ast.mode = ASTSelectWithUnionQuery::Mode::ALL; - else if (settings.union_default_mode == UnionMode::DISTINCT) - ast.mode = ASTSelectWithUnionQuery::Mode::DISTINCT; - else - throw Exception( - "Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty", - DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); + if (settings.union_default_mode == UnionMode::ALL) + mode = ASTSelectWithUnionQuery::Mode::ALL; + else if (settings.union_default_mode == UnionMode::DISTINCT) + mode = ASTSelectWithUnionQuery::Mode::DISTINCT; + else + throw Exception( + "Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty", + DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); + } + } + /// Optimize: if there is UNION DISTINCT, all previous UNION DISTINCT can be rewritten to UNION ALL. + /// Therefore we have at most one UNION DISTINCT in a sequence. + for (auto rit = ast.union_modes.rbegin(); rit != ast.union_modes.rend(); ++rit) + { + if (*rit == ASTSelectWithUnionQuery::Mode::DISTINCT) + { + /// Number of streams need to do a DISTINCT transform after unite + union_distinct_num = ast.union_modes.rend() - rit + 1; + for (auto mode_to_modify = ++rit; mode_to_modify != ast.union_modes.rend(); ++mode_to_modify) + *mode_to_modify = ASTSelectWithUnionQuery::Mode::ALL; + break; + } } - - if (ast.mode == ASTSelectWithUnionQuery::Mode::DISTINCT) - distinct_union = true; } /// Initialize interpreters for each SELECT query. @@ -207,31 +221,79 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) return; } - std::vector> plans(num_plans); - DataStreams data_streams(num_plans); - - for (size_t i = 0; i < num_plans; ++i) + /// All UNION streams in the chain does not need to do DISTINCT transform + if (union_distinct_num == 0) { - plans[i] = std::make_unique(); - nested_interpreters[i]->buildQueryPlan(*plans[i]); - data_streams[i] = plans[i]->getCurrentDataStream(); + std::vector> plans(num_plans); + DataStreams data_streams(num_plans); + + for (size_t i = 0; i < num_plans; ++i) + { + plans[i] = std::make_unique(); + nested_interpreters[i]->buildQueryPlan(*plans[i]); + data_streams[i] = plans[i]->getCurrentDataStream(); + } + + auto max_threads = context->getSettingsRef().max_threads; + auto union_step = std::make_unique(std::move(data_streams), result_header, max_threads); + + query_plan.unitePlans(std::move(union_step), std::move(plans)); } - auto max_threads = context->getSettingsRef().max_threads; - auto union_step = std::make_unique(std::move(data_streams), result_header, max_threads); - - query_plan.unitePlans(std::move(union_step), std::move(plans)); - - /// Add distinct transform for UNION DISTINCT query - if (distinct_union) + /// The first union_distinct_num UNION streams need to do a DISTINCT transform after unite + else { + QueryPlan distinct_query_plan; + + std::vector> plans(union_distinct_num); + DataStreams data_streams(union_distinct_num); + + for (size_t i = 0; i < union_distinct_num; ++i) + { + plans[i] = std::make_unique(); + nested_interpreters[i]->buildQueryPlan(*plans[i]); + data_streams[i] = plans[i]->getCurrentDataStream(); + } + + auto max_threads = context->getSettingsRef().max_threads; + auto union_step = std::make_unique(std::move(data_streams), result_header, max_threads); + + distinct_query_plan.unitePlans(std::move(union_step), std::move(plans)); + + /// Add distinct transform const Settings & settings = context->getSettingsRef(); SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); - auto distinct_step = std::make_unique(query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false); + auto distinct_step + = std::make_unique(distinct_query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false); - query_plan.addStep(std::move(distinct_step)); + distinct_query_plan.addStep(std::move(distinct_step)); + + /// No other UNION streams after DISTINCT stream + if (num_plans == union_distinct_num) + { + query_plan = std::move(distinct_query_plan); + return; + } + + /// Build final UNION step + std::vector> final_plans(num_plans - union_distinct_num + 1); + DataStreams final_data_streams(num_plans - union_distinct_num + 1); + + final_plans[0] = std::make_unique(std::move(distinct_query_plan)); + final_data_streams[0] = final_plans[0]->getCurrentDataStream(); + + for (size_t i = 1; i < num_plans - union_distinct_num + 1; ++i) + { + final_plans[i] = std::make_unique(); + nested_interpreters[union_distinct_num + i - 1]->buildQueryPlan(*final_plans[i]); + final_data_streams[i] = final_plans[i]->getCurrentDataStream(); + } + + auto final_union_step = std::make_unique(std::move(final_data_streams), result_header, max_threads); + query_plan.unitePlans(std::move(final_union_step), std::move(final_plans)); } + } BlockIO InterpreterSelectWithUnionQuery::execute() diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index dceba011418..b41f966f336 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -54,7 +55,8 @@ private: size_t max_streams = 1; - bool distinct_union = false; + /// First union_distinct_num streams need to do a DISTINCT transform after unite + size_t union_distinct_num = 0; static Block getCommonHeaderForUnion(const Blocks & headers); }; diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 1ef47257eae..41b34e14571 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -23,27 +23,21 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F { std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); - std::string mode_str; - switch (mode) + auto mode_to_str = [&](auto mode) { - case Mode::Unspecified: - mode_str = ""; - break; - case Mode::ALL: - mode_str = "ALL"; - break; - case Mode::DISTINCT: - mode_str = "DISTINCT"; - break; - } + if (mode == Mode::Unspecified) + return ""; + else if (mode == Mode::ALL) + return "ALL"; + else + return "DISTINCT"; + }; for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it) { if (it != list_of_selects->children.begin()) - settings.ostr << settings.nl_or_ws << indent_str - << (settings.hilite ? hilite_keyword : "") - << "UNION " << mode_str - << (settings.hilite ? hilite_none : "") + settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION " + << mode_to_str(union_modes[it - list_of_selects->children.begin() - 1]) << (settings.hilite ? hilite_none : "") << settings.nl_or_ws; (*it)->formatImpl(settings, state, frame); diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index 9c476d3674a..776ff033724 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -24,7 +24,9 @@ public: DISTINCT }; - Mode mode = Mode::Unspecified; + using Modes = std::vector; + + Modes union_modes; ASTPtr list_of_selects; }; diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 26affe020b1..b9e5eb2a678 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -99,6 +100,51 @@ bool ParserList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return true; } +bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTs elements; + + auto parse_element = [&] + { + ASTPtr element; + if (!elem_parser->parse(pos, element, expected)) + return false; + + elements.push_back(element); + return true; + }; + + /// Parse UNION type + auto parse_separator = [&] + { + if (s_union_parser->ignore(pos, expected)) + { + // SELECT ... UNION ALL SELECT ... + if (s_all_parser->check(pos, expected)) + { + union_modes.push_back(ASTSelectWithUnionQuery::Mode::ALL); + } + // SELECT ... UNION DISTINCT SELECT ... + else if (s_distinct_parser->check(pos, expected)) + { + union_modes.push_back(ASTSelectWithUnionQuery::Mode::DISTINCT); + } + // SELECT ... UNION SELECT ... + else + union_modes.push_back(ASTSelectWithUnionQuery::Mode::Unspecified); + return true; + } + return false; + }; + + if (!parseUtil(pos, parse_element, parse_separator)) + return false; + + auto list = std::make_shared(result_separator); + list->children = std::move(elements); + node = list; + return true; +} static bool parseOperator(IParser::Pos & pos, const char * op, Expected & expected) { diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 93a47648a0b..798c127cde7 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -5,6 +5,7 @@ #include #include +#include namespace DB { @@ -73,6 +74,53 @@ private: char result_separator; }; +class ParserUnionList : public IParserBase +{ +public: + ParserUnionList(ParserPtr && elem_parser_, ParserPtr && s_union_parser_, ParserPtr && s_all_parser_, ParserPtr && s_distinct_parser_) + : elem_parser(std::move(elem_parser_)) + , s_union_parser(std::move(s_union_parser_)) + , s_all_parser(std::move(s_all_parser_)) + , s_distinct_parser(std::move(s_distinct_parser_)) + { + } + + template + static bool parseUtil(Pos & pos, const ElemFunc & parse_element, const SepFunc & parse_separator) + { + Pos begin = pos; + if (!parse_element()) + { + pos = begin; + return false; + } + + while (true) + { + begin = pos; + if (!parse_separator() || !parse_element()) + { + pos = begin; + return true; + } + } + + return false; + } + + auto getUnionModes() const { return union_modes; } + +protected: + const char * getName() const override { return "list of union elements"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +private: + ParserPtr elem_parser; + ParserPtr s_union_parser; + ParserPtr s_all_parser; + ParserPtr s_distinct_parser; + char result_separator = ','; + ASTSelectWithUnionQuery::Modes union_modes; +}; /** An expression with an infix binary left-associative operator. * For example, a + b - c + d. diff --git a/src/Parsers/ParserSelectWithUnionQuery.cpp b/src/Parsers/ParserSelectWithUnionQuery.cpp index 5b98b083fd4..c1e748fbbd6 100644 --- a/src/Parsers/ParserSelectWithUnionQuery.cpp +++ b/src/Parsers/ParserSelectWithUnionQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -27,52 +28,21 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & { ASTPtr list_node; - ParserList parser_union(std::make_unique(), std::make_unique("UNION"), false); - ParserList parser_union_all(std::make_unique(), std::make_unique("UNION ALL"), false); - ParserList parser_union_distinct(std::make_unique(), std::make_unique("UNION DISTINCT"), false); + ParserUnionList parser( + std::make_unique(), + std::make_unique("UNION"), + std::make_unique("ALL"), + std::make_unique("DISTINCT")); - auto begin = pos; - auto current_expected = expected; - ASTSelectWithUnionQuery::Mode union_mode = ASTSelectWithUnionQuery::Mode::ALL; - - /// Parser SELECT lists and UNION type, must have UNION - auto union_parser = [&](auto & parser, auto mode) - { - if (!parser.parse(pos, list_node, expected)) - { - pos = begin; - expected = current_expected; - return false; - } - /// number of SELECT lists should not less than 2 - if (list_node->children.size() < 2) - { - pos = begin; - expected = current_expected; - return false; - } - union_mode = mode; - return true; - }; - - /// We first parse: SELECT ... UNION SELECT ... - /// SELECT ... UNION ALL SELECT ... - /// SELECT ... UNION DISTINCT SELECT ... - if (!union_parser(parser_union, ASTSelectWithUnionQuery::Mode::Unspecified) - && !union_parser(parser_union_all, ASTSelectWithUnionQuery::Mode::ALL) - && !union_parser(parser_union_distinct, ASTSelectWithUnionQuery::Mode::DISTINCT)) - { - /// If above parse failed, we back to parse SELECT without UNION - if (!parser_union.parse(pos, list_node, expected)) - return false; - } + if (!parser.parse(pos, list_node, expected)) + return false; auto select_with_union_query = std::make_shared(); node = select_with_union_query; select_with_union_query->list_of_selects = std::make_shared(); select_with_union_query->children.push_back(select_with_union_query->list_of_selects); - select_with_union_query->mode = union_mode; + select_with_union_query->union_modes = parser.getUnionModes(); // flatten inner union query for (auto & child : list_node->children) From 5b4981b4667dcf202b7c138a363e6f2e985795bf Mon Sep 17 00:00:00 2001 From: myrrc Date: Mon, 26 Oct 2020 16:27:58 +0300 Subject: [PATCH 0070/1088] added getFloat64 to ColumnDecimal, updated tests --- src/AggregateFunctions/AggregateFunctionAvg.h | 5 +-- .../AggregateFunctionAvgWeighted.h | 7 +--- src/Columns/ColumnDecimal.h | 3 ++ .../0_stateless/01035_avg_weighted.reference | 3 -- .../queries/0_stateless/01035_avg_weighted.sh | 18 ++++++++- .../01035_avg_weighted_long.reference | 0 .../0_stateless/01035_avg_weighted_long.sh | 39 +++++++++++++++++++ 7 files changed, 62 insertions(+), 13 deletions(-) delete mode 100644 tests/queries/0_stateless/01035_avg_weighted.reference mode change 100755 => 100644 tests/queries/0_stateless/01035_avg_weighted.sh create mode 100644 tests/queries/0_stateless/01035_avg_weighted_long.reference create mode 100755 tests/queries/0_stateless/01035_avg_weighted_long.sh diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 66ab20cec73..16eb11143da 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -14,7 +14,7 @@ namespace DB template struct RationalFraction { - constexpr RationalFraction(): numerator(0), denominator(0) {} + constexpr RationalFraction(): numerator(0), denominator(0) {} Numerator numerator; Denominator denominator; @@ -114,8 +114,7 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const final { - const auto & column = static_cast &>(*columns[0]); - this->data(place).numerator += column.getData()[row_num]; + this->data(place).numerator += columns[0]->getFloat64(row_num); ++this->data(place).denominator; } diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index c9b60cf9f50..ca9f0757cba 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -12,11 +12,8 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - const auto & values = static_cast &>(*columns[0]); - const auto & weights = static_cast &>(*columns[1]); - - const auto value = values.getData()[row_num]; - const auto weight = weights.getData()[row_num]; + const auto value = columns[0]->getFloat64(row_num); + const auto weight = columns[1]->getFloat64(row_num); this->data(place).numerator += value * weight; this->data(place).denominator += weight; diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index c33ab34b541..1939d87e357 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -3,6 +3,7 @@ #include #include +#include "Core/DecimalFunctions.h" #include #include #include @@ -182,6 +183,8 @@ public: throw Exception("getDataAt() is not implemented for big integers", ErrorCodes::NOT_IMPLEMENTED); } + Float64 getFloat64(size_t n) const final { return DecimalUtils::convertTo(data[n], scale); } + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; const char * deserializeAndInsertFromArena(const char * pos) override; void updateHashWithValue(size_t n, SipHash & hash) const override; diff --git a/tests/queries/0_stateless/01035_avg_weighted.reference b/tests/queries/0_stateless/01035_avg_weighted.reference deleted file mode 100644 index 7ace086f33b..00000000000 --- a/tests/queries/0_stateless/01035_avg_weighted.reference +++ /dev/null @@ -1,3 +0,0 @@ -2.3333333333333335 -nan -1 diff --git a/tests/queries/0_stateless/01035_avg_weighted.sh b/tests/queries/0_stateless/01035_avg_weighted.sh old mode 100755 new mode 100644 index 9b8d0bea14d..bb3e97a1d31 --- a/tests/queries/0_stateless/01035_avg_weighted.sh +++ b/tests/queries/0_stateless/01035_avg_weighted.sh @@ -6,10 +6,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 5), (2, 4), (3, 3), (4, 2), (5, 1)]) AS t));" ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 0), (2, 0), (3, 0), (4, 0), (5, 0)]) AS t));" +# Decimals not tested in types' combinations as they +# 1) Require different initialization (precision + scale) +# 2) Won't be used in these functions often types=("Int8" "Int16" "Int32" "Int64" "Int128" "Int256" "UInt8" "UInt16" "UInt32" "UInt64" "UInt128" "UInt256" - "Float32" "Float64" - "Decimal32" "Decimal64" "Decimal128" "Decimal256") + "Float32" "Float64") for left in "${types[@]}" do @@ -21,5 +23,17 @@ do done done +# Decimal types +dtypes=("32" "64" "128" "256") + +for left in "${dtypes[@]}" +do + for right in "${dtypes[@]}" + do + ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(toDecimal${left}(2, 4), toDecimal${right}(1, 4))" + done +done + + echo "$(${CLICKHOUSE_CLIENT} --server_logs_file=/dev/null --query="SELECT avgWeighted(['string'], toFloat64(0))" 2>&1)" \ | grep -c 'Code: 43. DB::Exception: .* DB::Exception:.* Types .* of arguments are non-conforming as arguments for aggregate function avgWeighted' diff --git a/tests/queries/0_stateless/01035_avg_weighted_long.reference b/tests/queries/0_stateless/01035_avg_weighted_long.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01035_avg_weighted_long.sh b/tests/queries/0_stateless/01035_avg_weighted_long.sh new file mode 100755 index 00000000000..bb3e97a1d31 --- /dev/null +++ b/tests/queries/0_stateless/01035_avg_weighted_long.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 5), (2, 4), (3, 3), (4, 2), (5, 1)]) AS t));" +${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 0), (2, 0), (3, 0), (4, 0), (5, 0)]) AS t));" + +# Decimals not tested in types' combinations as they +# 1) Require different initialization (precision + scale) +# 2) Won't be used in these functions often +types=("Int8" "Int16" "Int32" "Int64" "Int128" "Int256" + "UInt8" "UInt16" "UInt32" "UInt64" "UInt128" "UInt256" + "Float32" "Float64") + +for left in "${types[@]}" +do + for right in "${types[@]}" + do + ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 1), (1, 0), (10, 2))" + ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (8, 1), (122, 0))" + ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (0, 0), (1, 0))" + done +done + +# Decimal types +dtypes=("32" "64" "128" "256") + +for left in "${dtypes[@]}" +do + for right in "${dtypes[@]}" + do + ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(toDecimal${left}(2, 4), toDecimal${right}(1, 4))" + done +done + + +echo "$(${CLICKHOUSE_CLIENT} --server_logs_file=/dev/null --query="SELECT avgWeighted(['string'], toFloat64(0))" 2>&1)" \ + | grep -c 'Code: 43. DB::Exception: .* DB::Exception:.* Types .* of arguments are non-conforming as arguments for aggregate function avgWeighted' From 4e3b9656fad71b2c9efe9bda0882f71c223189ec Mon Sep 17 00:00:00 2001 From: myrrc Date: Mon, 26 Oct 2020 17:43:09 +0300 Subject: [PATCH 0071/1088] fixed tests, updated master --- .../queries/0_stateless/01035_avg_weighted.sh | 39 --- .../01035_avg_weighted_long.reference | 228 ++++++++++++++++++ .../0_stateless/01035_avg_weighted_long.sh | 21 +- 3 files changed, 240 insertions(+), 48 deletions(-) delete mode 100644 tests/queries/0_stateless/01035_avg_weighted.sh diff --git a/tests/queries/0_stateless/01035_avg_weighted.sh b/tests/queries/0_stateless/01035_avg_weighted.sh deleted file mode 100644 index bb3e97a1d31..00000000000 --- a/tests/queries/0_stateless/01035_avg_weighted.sh +++ /dev/null @@ -1,39 +0,0 @@ -#!/usr/bin/env bash - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. "$CUR_DIR"/../shell_config.sh - -${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 5), (2, 4), (3, 3), (4, 2), (5, 1)]) AS t));" -${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 0), (2, 0), (3, 0), (4, 0), (5, 0)]) AS t));" - -# Decimals not tested in types' combinations as they -# 1) Require different initialization (precision + scale) -# 2) Won't be used in these functions often -types=("Int8" "Int16" "Int32" "Int64" "Int128" "Int256" - "UInt8" "UInt16" "UInt32" "UInt64" "UInt128" "UInt256" - "Float32" "Float64") - -for left in "${types[@]}" -do - for right in "${types[@]}" - do - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 1), (1, 0), (10, 2))" - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (8, 1), (122, 0))" - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (0, 0), (1, 0))" - done -done - -# Decimal types -dtypes=("32" "64" "128" "256") - -for left in "${dtypes[@]}" -do - for right in "${dtypes[@]}" - do - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(toDecimal${left}(2, 4), toDecimal${right}(1, 4))" - done -done - - -echo "$(${CLICKHOUSE_CLIENT} --server_logs_file=/dev/null --query="SELECT avgWeighted(['string'], toFloat64(0))" 2>&1)" \ - | grep -c 'Code: 43. DB::Exception: .* DB::Exception:.* Types .* of arguments are non-conforming as arguments for aggregate function avgWeighted' diff --git a/tests/queries/0_stateless/01035_avg_weighted_long.reference b/tests/queries/0_stateless/01035_avg_weighted_long.reference index e69de29bb2d..584f3184414 100644 --- a/tests/queries/0_stateless/01035_avg_weighted_long.reference +++ b/tests/queries/0_stateless/01035_avg_weighted_long.reference @@ -0,0 +1,228 @@ +2.3333333333333335 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +8 +nan +1 +1 +1 +1 +1 +1 +1 +1 +1 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +1 diff --git a/tests/queries/0_stateless/01035_avg_weighted_long.sh b/tests/queries/0_stateless/01035_avg_weighted_long.sh index bb3e97a1d31..2d764918026 100755 --- a/tests/queries/0_stateless/01035_avg_weighted_long.sh +++ b/tests/queries/0_stateless/01035_avg_weighted_long.sh @@ -6,23 +6,27 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 5), (2, 4), (3, 3), (4, 2), (5, 1)]) AS t));" ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 0), (2, 0), (3, 0), (4, 0), (5, 0)]) AS t));" -# Decimals not tested in types' combinations as they -# 1) Require different initialization (precision + scale) -# 2) Won't be used in these functions often -types=("Int8" "Int16" "Int32" "Int64" "Int128" "Int256" - "UInt8" "UInt16" "UInt32" "UInt64" "UInt128" "UInt256" - "Float32" "Float64") +types=("Int8" "Int16" "Int32" "Int64" "UInt8" "UInt16" "UInt32" "UInt64" "Float32" "Float64") for left in "${types[@]}" do for right in "${types[@]}" do ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 1), (1, 0), (10, 2))" - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (8, 1), (122, 0))" ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (0, 0), (1, 0))" done done +exttypes=("Int128" "Int256" "UInt256") + +for left in "${exttypes[@]}" +do + for right in "${exttypes[@]}" + do + ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(to${left}(1), to${right}(2))" + done +done + # Decimal types dtypes=("32" "64" "128" "256") @@ -34,6 +38,5 @@ do done done - echo "$(${CLICKHOUSE_CLIENT} --server_logs_file=/dev/null --query="SELECT avgWeighted(['string'], toFloat64(0))" 2>&1)" \ - | grep -c 'Code: 43. DB::Exception: .* DB::Exception:.* Types .* of arguments are non-conforming as arguments for aggregate function avgWeighted' + | grep -c 'Code: 43. DB::Exception: .* DB::Exception:.* Types .* are non-conforming as arguments for aggregate function avgWeighted' From 7bfd5d9e8deb1f9d4b35f3d6fa309b1f1f311cb6 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 28 Oct 2020 01:29:09 +0000 Subject: [PATCH 0072/1088] need fix --- .../InterpreterSelectWithUnionQuery.cpp | 367 +++++++++++------- .../InterpreterSelectWithUnionQuery.h | 29 +- src/Interpreters/executeQuery.cpp | 2 + src/Parsers/ASTSelectWithUnionQuery.cpp | 12 +- src/Parsers/ASTSelectWithUnionQuery.h | 8 +- src/Parsers/ExpressionListParsers.cpp | 2 +- src/Parsers/ExpressionListParsers.h | 1 - src/Parsers/IAST.cpp | 5 + src/Parsers/IAST.h | 2 + src/Parsers/ParserSelectWithUnionQuery.cpp | 22 +- ...istinct_and_setting_union_default_mode.sql | 15 +- 11 files changed, 290 insertions(+), 175 deletions(-) diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 19f08250430..92f88342241 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -1,9 +1,6 @@ -#include -#include #include #include #include -#include #include #include #include @@ -13,6 +10,7 @@ #include #include +#include namespace DB { @@ -24,6 +22,153 @@ namespace ErrorCodes extern const int EXPECTED_ALL_OR_DISTINCT; } +struct CustomizeUnionModeRewrite +{ + using TypeToVisit = ASTSelectWithUnionQuery; + + const UnionMode & default_union_mode; + + void visit(ASTSelectWithUnionQuery & union_select, ASTPtr &) + { + size_t num_selects = union_select.list_of_selects->children.size(); + if (!num_selects) + throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR); + if (num_selects > 1) + { + for (auto & mode : union_select.union_modes) + { + if (mode == ASTSelectWithUnionQuery::Mode::Unspecified) + { + if (default_union_mode == UnionMode::ALL) + mode = ASTSelectWithUnionQuery::Mode::ALL; + else if (default_union_mode == UnionMode::DISTINCT) + mode = ASTSelectWithUnionQuery::Mode::DISTINCT; + else + throw Exception( + "Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty", + DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); + } + } + /// Optimize: if there is UNION DISTINCT, all previous UNION DISTINCT can be rewritten to UNION ALL. + /// Therefore we have at most one UNION DISTINCT in a sequence. + for (auto rit = union_select.union_modes.rbegin(); rit != union_select.union_modes.rend(); ++rit) + { + if (*rit == ASTSelectWithUnionQuery::Mode::DISTINCT) + { + /// Number of streams need to do a DISTINCT transform after unite + for (auto mode_to_modify = ++rit; mode_to_modify != union_select.union_modes.rend(); ++mode_to_modify) + *mode_to_modify = ASTSelectWithUnionQuery::Mode::ALL; + break; + } + } + } + } +}; + +using CustomizeUnionQueryOptimizeVisitor = InDepthNodeVisitor, true>; + +QueryPlan NestedInterpreter::buildQueryPlan(const std::shared_ptr & context, const Block & header) +{ + QueryPlan res; + if (type == Type::LEAF) + { + if (interpreter) + { + interpreter->buildQueryPlan(res); + return res; + } + else + throw Exception("Interpreter is not initialized.", ErrorCodes::LOGICAL_ERROR); + } + + if (num_distinct_union == 0) + { + std::vector> plans(children.size()); + DataStreams data_streams(children.size()); + + for (size_t i = 0; i < children.size(); ++i) + { + plans[i] = std::make_unique(children[i]->buildQueryPlan(context, header)); + data_streams[i] = plans[i]->getCurrentDataStream(); + } + + auto max_threads = context->getSettingsRef().max_threads; + auto union_step = std::make_unique(std::move(data_streams), header, max_threads); + + res.unitePlans(std::move(union_step), std::move(plans)); + return res; + } + /// The first union_distinct_num UNION streams need to do a DISTINCT transform after unite + else + { + QueryPlan distinct_query_plan; + + std::vector> plans(num_distinct_union); + DataStreams data_streams(num_distinct_union); + + for (size_t i = 0; i < num_distinct_union; ++i) + { + plans[i] = std::make_unique(children[i]->buildQueryPlan(context, header)); + data_streams[i] = plans[i]->getCurrentDataStream(); + } + + auto max_threads = context->getSettingsRef().max_threads; + auto union_step = std::make_unique(std::move(data_streams), header, max_threads); + + distinct_query_plan.unitePlans(std::move(union_step), std::move(plans)); + + /// Add distinct transform + const Settings & settings = context->getSettingsRef(); + SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); + + auto distinct_step + = std::make_unique(distinct_query_plan.getCurrentDataStream(), limits, 0, header.getNames(), false); + + distinct_query_plan.addStep(std::move(distinct_step)); + + /// No other UNION streams after DISTINCT stream + if (num_distinct_union == children.size()) + { + return distinct_query_plan; + } + + /// Build final UNION step + std::vector> final_plans(children.size() - num_distinct_union + 1); + DataStreams final_data_streams(children.size() - num_distinct_union + 1); + + final_plans[0] = std::make_unique(std::move(distinct_query_plan)); + final_data_streams[0] = final_plans[0]->getCurrentDataStream(); + + for (size_t i = 1; i < children.size() - num_distinct_union + 1; ++i) + { + final_plans[i] = std::make_unique(children[num_distinct_union + i - 1]->buildQueryPlan(context, header)); + final_data_streams[i] = final_plans[i]->getCurrentDataStream(); + } + + auto final_union_step = std::make_unique(std::move(final_data_streams), header, max_threads); + res.unitePlans(std::move(final_union_step), std::move(final_plans)); + return res; + } +} + +void NestedInterpreter::ignoreWithTotals() +{ + if (type == Type::LEAF) + { + if (interpreter) + interpreter->ignoreWithTotals(); + else + { + throw Exception("Interpreter is not initialized.", ErrorCodes::LOGICAL_ERROR); + } + return; + } + for (auto & child : children) + { + child->ignoreWithTotals(); + } +} + InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( const ASTPtr & query_ptr_, @@ -35,71 +180,48 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( context(std::make_shared(context_)), max_streams(context->getSettingsRef().max_threads) { - auto & ast = query_ptr->as(); - - size_t num_selects = ast.list_of_selects->children.size(); - - if (!num_selects) + std::cout << "\n\n In InterpreterSelectWithUnionQuery\n\n"; + const auto & ast = query_ptr->as(); + std::cout << "\n\n before throw\n\n"; + if (!ast.flatten_nodes_list) + std::cout << "\n\n flatten_nodes_list is null\n\n"; + size_t total_num_selects = ast.flatten_nodes_list->children.size(); + std::cout << "\n\n after get num throw\n\n"; + if (!total_num_selects) throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR); + std::cout << "\n\n after throw\n\n"; /// Rewrite ast with settings.union_default_mode - if (num_selects > 1) - { - const Settings & settings = context->getSettingsRef(); - for (auto & mode : ast.union_modes) - { - if (mode == ASTSelectWithUnionQuery::Mode::Unspecified) - { + const auto & settings = context->getSettingsRef(); + CustomizeUnionQueryOptimizeVisitor::Data data_union_mode{settings.union_default_mode}; + CustomizeUnionQueryOptimizeVisitor(data_union_mode).visit(query_ptr); - if (settings.union_default_mode == UnionMode::ALL) - mode = ASTSelectWithUnionQuery::Mode::ALL; - else if (settings.union_default_mode == UnionMode::DISTINCT) - mode = ASTSelectWithUnionQuery::Mode::DISTINCT; - else - throw Exception( - "Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty", - DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); - } - } - /// Optimize: if there is UNION DISTINCT, all previous UNION DISTINCT can be rewritten to UNION ALL. - /// Therefore we have at most one UNION DISTINCT in a sequence. - for (auto rit = ast.union_modes.rbegin(); rit != ast.union_modes.rend(); ++rit) - { - if (*rit == ASTSelectWithUnionQuery::Mode::DISTINCT) - { - /// Number of streams need to do a DISTINCT transform after unite - union_distinct_num = ast.union_modes.rend() - rit + 1; - for (auto mode_to_modify = ++rit; mode_to_modify != ast.union_modes.rend(); ++mode_to_modify) - *mode_to_modify = ASTSelectWithUnionQuery::Mode::ALL; - break; - } - } - } - - /// Initialize interpreters for each SELECT query. + /// We first build nested interpreters for each select query, then using this nested interpreters to build Tree Structured nested interpreter. /// Note that we pass 'required_result_column_names' to first SELECT. /// And for the rest, we pass names at the corresponding positions of 'required_result_column_names' in the result of first SELECT, /// because names could be different. - nested_interpreters.reserve(num_selects); - - std::vector required_result_column_names_for_other_selects(num_selects); - if (!required_result_column_names.empty() && num_selects > 1) + std::vector> interpreters; + interpreters.reserve(total_num_selects); + std::vector required_result_column_names_for_other_selects(total_num_selects); + if (!required_result_column_names.empty() && total_num_selects > 1) { /// Result header if there are no filtering by 'required_result_column_names'. /// We use it to determine positions of 'required_result_column_names' in SELECT clause. - Block full_result_header = InterpreterSelectQuery( - ast.list_of_selects->children.at(0), *context, options.copy().analyze().noModify()).getSampleBlock(); + Block full_result_header + = InterpreterSelectQuery(ast.flatten_nodes_list->children.at(0), *context, options.copy().analyze().noModify()) + .getSampleBlock(); std::vector positions_of_required_result_columns(required_result_column_names.size()); for (size_t required_result_num = 0, size = required_result_column_names.size(); required_result_num < size; ++required_result_num) positions_of_required_result_columns[required_result_num] = full_result_header.getPositionByName(required_result_column_names[required_result_num]); - for (size_t query_num = 1; query_num < num_selects; ++query_num) + for (size_t query_num = 1; query_num < total_num_selects; ++query_num) { - Block full_result_header_for_current_select = InterpreterSelectQuery( - ast.list_of_selects->children.at(query_num), *context, options.copy().analyze().noModify()).getSampleBlock(); + Block full_result_header_for_current_select + = InterpreterSelectQuery(ast.flatten_nodes_list->children.at(query_num), *context, options.copy().analyze().noModify()) + .getSampleBlock(); if (full_result_header_for_current_select.columns() != full_result_header.columns()) throw Exception("Different number of columns in UNION ALL elements:\n" @@ -114,29 +236,26 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( } } - for (size_t query_num = 0; query_num < num_selects; ++query_num) + for (size_t query_num = 0; query_num < total_num_selects; ++query_num) { const Names & current_required_result_column_names = query_num == 0 ? required_result_column_names : required_result_column_names_for_other_selects[query_num]; - nested_interpreters.emplace_back(std::make_unique( - ast.list_of_selects->children.at(query_num), - *context, - options, - current_required_result_column_names)); + interpreters.emplace_back(std::make_shared( + ast.flatten_nodes_list->children.at(query_num), *context, options, current_required_result_column_names)); } /// Determine structure of the result. - if (num_selects == 1) + if (total_num_selects == 1) { - result_header = nested_interpreters.front()->getSampleBlock(); + result_header = interpreters.front()->getSampleBlock(); } else { - Blocks headers(num_selects); - for (size_t query_num = 0; query_num < num_selects; ++query_num) - headers[query_num] = nested_interpreters[query_num]->getSampleBlock(); + Blocks headers(total_num_selects); + for (size_t query_num = 0; query_num < total_num_selects; ++query_num) + headers[query_num] = interpreters[query_num]->getSampleBlock(); result_header = getCommonHeaderForUnion(headers); } @@ -144,7 +263,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( /// InterpreterSelectWithUnionQuery ignores limits if all nested interpreters ignore limits. bool all_nested_ignore_limits = true; bool all_nested_ignore_quota = true; - for (auto & interpreter : nested_interpreters) + for (auto & interpreter : interpreters) { if (!interpreter->ignoreLimits()) all_nested_ignore_limits = false; @@ -153,8 +272,46 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( } options.ignore_limits |= all_nested_ignore_limits; options.ignore_quota |= all_nested_ignore_quota; + + int index = 0; + buildNestedTreeInterpreter(query_ptr, nested_interpreter, interpreters, index); } +/// We build a Tree Structured nested interpreters to build QueryPlan later +/// The structure of build nested interpreters is same as AST Tree +void InterpreterSelectWithUnionQuery::buildNestedTreeInterpreter( + const ASTPtr & ast_ptr, + std::shared_ptr nested_interpreter_, + std::vector> & interpreters, + int & index) +{ + std::cout << "\n\n in build \n\n"; + if (auto inner_union = ast_ptr->as()) + { + auto internal_intepreter = std::make_shared(); + const auto & union_modes = inner_union->union_modes; + + for (auto rit = union_modes.rbegin(); rit != union_modes.rend(); ++rit) + { + if (*rit == ASTSelectWithUnionQuery::Mode::DISTINCT) + { + internal_intepreter->num_distinct_union = union_modes.rend() - rit + 1; + break; + } + } + + nested_interpreter_->children.push_back(internal_intepreter); + + for (auto & child : inner_union->list_of_selects->children) + buildNestedTreeInterpreter(child, internal_intepreter, interpreters, index); + return; + } + + auto leaf_interpreter = std::make_shared(); + leaf_interpreter->type = NestedInterpreter::Type::LEAF; + leaf_interpreter->interpreter = interpreters[index++]; + nested_interpreter_->children.push_back(leaf_interpreter); +} Block InterpreterSelectWithUnionQuery::getCommonHeaderForUnion(const Blocks & headers) { @@ -212,88 +369,7 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock( void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) { - size_t num_plans = nested_interpreters.size(); - - /// Skip union for single interpreter. - if (num_plans == 1) - { - nested_interpreters.front()->buildQueryPlan(query_plan); - return; - } - - /// All UNION streams in the chain does not need to do DISTINCT transform - if (union_distinct_num == 0) - { - std::vector> plans(num_plans); - DataStreams data_streams(num_plans); - - for (size_t i = 0; i < num_plans; ++i) - { - plans[i] = std::make_unique(); - nested_interpreters[i]->buildQueryPlan(*plans[i]); - data_streams[i] = plans[i]->getCurrentDataStream(); - } - - auto max_threads = context->getSettingsRef().max_threads; - auto union_step = std::make_unique(std::move(data_streams), result_header, max_threads); - - query_plan.unitePlans(std::move(union_step), std::move(plans)); - } - - /// The first union_distinct_num UNION streams need to do a DISTINCT transform after unite - else - { - QueryPlan distinct_query_plan; - - std::vector> plans(union_distinct_num); - DataStreams data_streams(union_distinct_num); - - for (size_t i = 0; i < union_distinct_num; ++i) - { - plans[i] = std::make_unique(); - nested_interpreters[i]->buildQueryPlan(*plans[i]); - data_streams[i] = plans[i]->getCurrentDataStream(); - } - - auto max_threads = context->getSettingsRef().max_threads; - auto union_step = std::make_unique(std::move(data_streams), result_header, max_threads); - - distinct_query_plan.unitePlans(std::move(union_step), std::move(plans)); - - /// Add distinct transform - const Settings & settings = context->getSettingsRef(); - SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); - - auto distinct_step - = std::make_unique(distinct_query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false); - - distinct_query_plan.addStep(std::move(distinct_step)); - - /// No other UNION streams after DISTINCT stream - if (num_plans == union_distinct_num) - { - query_plan = std::move(distinct_query_plan); - return; - } - - /// Build final UNION step - std::vector> final_plans(num_plans - union_distinct_num + 1); - DataStreams final_data_streams(num_plans - union_distinct_num + 1); - - final_plans[0] = std::make_unique(std::move(distinct_query_plan)); - final_data_streams[0] = final_plans[0]->getCurrentDataStream(); - - for (size_t i = 1; i < num_plans - union_distinct_num + 1; ++i) - { - final_plans[i] = std::make_unique(); - nested_interpreters[union_distinct_num + i - 1]->buildQueryPlan(*final_plans[i]); - final_data_streams[i] = final_plans[i]->getCurrentDataStream(); - } - - auto final_union_step = std::make_unique(std::move(final_data_streams), result_header, max_threads); - query_plan.unitePlans(std::move(final_union_step), std::move(final_plans)); - } - + query_plan = nested_interpreter->buildQueryPlan(context, result_header); } BlockIO InterpreterSelectWithUnionQuery::execute() @@ -314,8 +390,7 @@ BlockIO InterpreterSelectWithUnionQuery::execute() void InterpreterSelectWithUnionQuery::ignoreWithTotals() { - for (auto & interpreter : nested_interpreters) - interpreter->ignoreWithTotals(); + nested_interpreter->ignoreWithTotals(); } } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index b41f966f336..4af73b3c723 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -1,10 +1,10 @@ #pragma once -#include #include #include #include #include +#include namespace DB { @@ -13,6 +13,22 @@ class Context; class InterpreterSelectQuery; class QueryPlan; +struct NestedInterpreter +{ + ~NestedInterpreter() { } + enum class Type + { + LEAF, + INTERNAL + }; + Type type = Type::INTERNAL; + std::vector> children; + std::shared_ptr interpreter; + size_t num_distinct_union = 0; + QueryPlan buildQueryPlan(const std::shared_ptr & context, const Block & header); + void ignoreWithTotals(); +}; + /** Interprets one or multiple SELECT queries inside UNION/UNION ALL/UNION DISTINCT chain. */ class InterpreterSelectWithUnionQuery : public IInterpreter @@ -49,16 +65,19 @@ private: ASTPtr query_ptr; std::shared_ptr context; - std::vector> nested_interpreters; + std::shared_ptr nested_interpreter; Block result_header; size_t max_streams = 1; - /// First union_distinct_num streams need to do a DISTINCT transform after unite - size_t union_distinct_num = 0; - static Block getCommonHeaderForUnion(const Blocks & headers); + + static void buildNestedTreeInterpreter( + const ASTPtr & ast_ptr, + std::shared_ptr nested_interpreter_, + std::vector> & interpreters, + int & index); }; } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 57c557c5658..8b4b35785c1 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -70,10 +70,12 @@ namespace ErrorCodes static void checkASTSizeLimits(const IAST & ast, const Settings & settings) { + std::cout << "\n\n before check limits"; if (settings.max_ast_depth) ast.checkDepth(settings.max_ast_depth); if (settings.max_ast_elements) ast.checkSize(settings.max_ast_elements); + std::cout << "\n\n after check limits"; } diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 41b34e14571..5deae6f653f 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -14,6 +15,9 @@ ASTPtr ASTSelectWithUnionQuery::clone() const res->list_of_selects = list_of_selects->clone(); res->children.push_back(res->list_of_selects); + res->union_modes.insert(res->union_modes.begin(), union_modes.begin(), union_modes.end()); + res->flatten_nodes_list = flatten_nodes_list->clone(); + cloneOutputOptions(*res); return res; } @@ -21,8 +25,10 @@ ASTPtr ASTSelectWithUnionQuery::clone() const void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { + std::cout << "\n\nin format \n\n"; std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); +#if 0 auto mode_to_str = [&](auto mode) { if (mode == Mode::Unspecified) @@ -32,16 +38,18 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F else return "DISTINCT"; }; +#endif - for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it) + for (ASTs::const_iterator it = flatten_nodes_list->children.begin(); it != flatten_nodes_list->children.end(); ++it) { if (it != list_of_selects->children.begin()) settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION " - << mode_to_str(union_modes[it - list_of_selects->children.begin() - 1]) << (settings.hilite ? hilite_none : "") + // << mode_to_str(union_modes[it - list_of_selects->children.begin() - 1]) << (settings.hilite ? hilite_none : "") << settings.nl_or_ws; (*it)->formatImpl(settings, state, frame); } + std::cout << "\n\nafter format \n\n"; } } diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index 776ff033724..67ec21e246c 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -5,9 +5,8 @@ namespace DB { - -/** Single SELECT query or multiple SELECT queries with UNION ALL. - * Only UNION ALL is possible. No UNION DISTINCT or plain UNION. +/** Single SELECT query or multiple SELECT queries with UNION + * or UNION or UNION DISTINCT */ class ASTSelectWithUnionQuery : public ASTQueryWithOutput { @@ -29,6 +28,9 @@ public: Modes union_modes; ASTPtr list_of_selects; + + /// we need flatten_nodes to help build nested_interpreter + ASTPtr flatten_nodes_list; }; } diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 8b9a36422af..220d304751e 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -139,7 +139,7 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!parseUtil(pos, parse_element, parse_separator)) return false; - auto list = std::make_shared(result_separator); + auto list = std::make_shared(); list->children = std::move(elements); node = list; return true; diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index a6f1be712c4..d93952923a9 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -119,7 +119,6 @@ private: ParserPtr s_union_parser; ParserPtr s_all_parser; ParserPtr s_distinct_parser; - char result_separator = ','; ASTSelectWithUnionQuery::Modes union_modes; }; diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 8ee4154541b..d9f0b3562bc 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -76,13 +76,18 @@ void IAST::updateTreeHashImpl(SipHash & hash_state) const size_t IAST::checkDepthImpl(size_t max_depth, size_t level) const { + std::cout << "\n\n in check depth impl\n\n"; + std::cout << "\nchildren.size = " << children.size() << "\n\n"; size_t res = level + 1; for (const auto & child : children) { + std::cout << "\n in for\n\n"; if (level >= max_depth) throw Exception("AST is too deep. Maximum: " + toString(max_depth), ErrorCodes::TOO_DEEP_AST); res = std::max(res, child->checkDepthImpl(max_depth, level + 1)); + std::cout << "\n after for\n\n"; } + std::cout << "\n\n after impl\n\n"; return res; } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index c88c80021d6..d1fca853592 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -11,6 +11,7 @@ #include #include +#include class SipHash; @@ -91,6 +92,7 @@ public: */ size_t checkDepth(size_t max_depth) const { + std::cout << "\n in check depth\n\n"; return checkDepthImpl(max_depth, 0); } diff --git a/src/Parsers/ParserSelectWithUnionQuery.cpp b/src/Parsers/ParserSelectWithUnionQuery.cpp index c1e748fbbd6..ee03da753e4 100644 --- a/src/Parsers/ParserSelectWithUnionQuery.cpp +++ b/src/Parsers/ParserSelectWithUnionQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -9,13 +10,14 @@ namespace DB { - static void getSelectsFromUnionListNode(ASTPtr & ast_select, ASTs & selects) { if (auto * inner_union = ast_select->as()) { for (auto & child : inner_union->list_of_selects->children) + { getSelectsFromUnionListNode(child, selects); + } return; } @@ -23,9 +25,9 @@ static void getSelectsFromUnionListNode(ASTPtr & ast_select, ASTs & selects) selects.push_back(std::move(ast_select)); } - bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { + std::cout << "\n\n in ParserSelectWithUnionQuery\n\n"; ASTPtr list_node; ParserUnionList parser( @@ -42,11 +44,23 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & node = select_with_union_query; select_with_union_query->list_of_selects = std::make_shared(); select_with_union_query->children.push_back(select_with_union_query->list_of_selects); + select_with_union_query->list_of_selects->children.insert( + select_with_union_query->list_of_selects->children.begin(), list_node->children.begin(), list_node->children.end()); select_with_union_query->union_modes = parser.getUnionModes(); - // flatten inner union query + /// NOTE: We cann't simply flatten inner union query now, since we may have different union mode in query, + /// so flatten may change it's semantics. For example: + /// flatten `SELECT 1 UNION (SELECT 1 UNION ALL SELETC 1)` -> `SELECT 1 UNION SELECT 1 UNION ALL SELECT 1` + /// We can use a non-flatten AST to help build QueryPlan in InterpreterSelectWithUnionQuery + + select_with_union_query->flatten_nodes_list = std::make_shared(); + for (auto & child : list_node->children) - getSelectsFromUnionListNode(child, select_with_union_query->list_of_selects->children); + { + getSelectsFromUnionListNode(child, select_with_union_query->flatten_nodes_list->children); + } + std::cout << "\n\n after ParserSelectWithUnionQuery\n\n"; + std::cout << "\n\n flatten_nodes.size =" << select_with_union_query->flatten_nodes_list->children.size() << "\n\n"; return true; } diff --git a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql index 1a6e582aebe..12fe204591c 100644 --- a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql +++ b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql @@ -1,13 +1,2 @@ -SELECT * FROM numbers(10) UNION SELECT * FROM numbers(10); -SELECT * FROM numbers(10) UNION ALL SELECT * FROM numbers(10); -SELECT * FROM numbers(10) UNION DISTINCT SELECT * FROM numbers(10); - -SET union_default_mode='ALL'; -SELECT * FROM numbers(10) UNION SELECT * FROM numbers(10); -SELECT * FROM numbers(10) UNION ALL SELECT * FROM numbers(10); -SELECT * FROM numbers(10) UNION DISTINCT SELECT * FROM numbers(10); - -SET union_default_mode='DISTINCT'; -SELECT * FROM numbers(10) UNION SELECT * FROM numbers(10); -SELECT * FROM numbers(10) UNION ALL SELECT * FROM numbers(10); -SELECT * FROM numbers(10) UNION DISTINCT SELECT * FROM numbers(10); +SELECT 1; +(((((((SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1; From 5e2a3d12d762ee5222b2a70990e06c4255771454 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 29 Oct 2020 17:25:44 +0300 Subject: [PATCH 0073/1088] Split requests and responses part from zookeeper --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 623 +++++++++++++++++++++++ src/Common/ZooKeeper/ZooKeeperCommon.h | 326 ++++++++++++ src/Common/ZooKeeper/ZooKeeperImpl.cpp | 455 ----------------- src/Common/ZooKeeper/ZooKeeperImpl.h | 33 +- 4 files changed, 951 insertions(+), 486 deletions(-) create mode 100644 src/Common/ZooKeeper/ZooKeeperCommon.cpp create mode 100644 src/Common/ZooKeeper/ZooKeeperCommon.h diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp new file mode 100644 index 00000000000..eb04536ae00 --- /dev/null +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -0,0 +1,623 @@ +#include +#include +#include +#include +#include +#include + + +namespace Coordination +{ + +using namespace DB; + +/// ZooKeeper has 1 MB node size and serialization limit by default, +/// but it can be raised up, so we have a slightly larger limit on our side. +#define MAX_STRING_OR_ARRAY_SIZE (1 << 28) /// 256 MiB + +/// Assuming we are at little endian. + +static void write(int64_t x, WriteBuffer & out) +{ + x = __builtin_bswap64(x); + writeBinary(x, out); +} + +static void write(int32_t x, WriteBuffer & out) +{ + x = __builtin_bswap32(x); + writeBinary(x, out); +} + +static void write(bool x, WriteBuffer & out) +{ + writeBinary(x, out); +} + +static void write(const String & s, WriteBuffer & out) +{ + write(int32_t(s.size()), out); + out.write(s.data(), s.size()); +} + +template void write(std::array s, WriteBuffer & out) +{ + write(int32_t(N), out); + out.write(s.data(), N); +} + +template void write(const std::vector & arr, WriteBuffer & out) +{ + write(int32_t(arr.size()), out); + for (const auto & elem : arr) + write(elem, out); +} + +static void write(const ACL & acl, WriteBuffer & out) +{ + write(acl.permissions, out); + write(acl.scheme, out); + write(acl.id, out); +} + +static void write(const Stat & stat, WriteBuffer & out) +{ + write(stat.czxid, out); + write(stat.mzxid, out); + write(stat.ctime, out); + write(stat.mtime, out); + write(stat.version, out); + write(stat.cversion, out); + write(stat.aversion, out); + write(stat.ephemeralOwner, out); + write(stat.dataLength, out); + write(stat.numChildren, out); + write(stat.pzxid, out); +} + +static void write(const Error & x, WriteBuffer & out) +{ + write(static_cast(x), out); +} + +static void read(int64_t & x, ReadBuffer & in) +{ + readBinary(x, in); + x = __builtin_bswap64(x); +} + +static void read(int32_t & x, ReadBuffer & in) +{ + readBinary(x, in); + x = __builtin_bswap32(x); +} + +static void read(Error & x, ReadBuffer & in) +{ + int32_t code; + read(code, in); + x = Error(code); +} + +static void read(bool & x, ReadBuffer & in) +{ + readBinary(x, in); +} + +static void read(String & s, ReadBuffer & in) +{ + int32_t size = 0; + read(size, in); + + if (size == -1) + { + /// It means that zookeeper node has NULL value. We will treat it like empty string. + s.clear(); + return; + } + + if (size < 0) + throw Exception("Negative size while reading string from ZooKeeper", Error::ZMARSHALLINGERROR); + + if (size > MAX_STRING_OR_ARRAY_SIZE) + throw Exception("Too large string size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); + + s.resize(size); + in.read(s.data(), size); +} + +template void read(std::array & s, ReadBuffer & in) +{ + int32_t size = 0; + read(size, in); + if (size != N) + throw Exception("Unexpected array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); + in.read(s.data(), N); +} + +static void read(Stat & stat, ReadBuffer & in) +{ + read(stat.czxid, in); + read(stat.mzxid, in); + read(stat.ctime, in); + read(stat.mtime, in); + read(stat.version, in); + read(stat.cversion, in); + read(stat.aversion, in); + read(stat.ephemeralOwner, in); + read(stat.dataLength, in); + read(stat.numChildren, in); + read(stat.pzxid, in); +} + +template void read(std::vector & arr, ReadBuffer & in) +{ + int32_t size = 0; + read(size, in); + if (size < 0) + throw Exception("Negative size while reading array from ZooKeeper", Error::ZMARSHALLINGERROR); + if (size > MAX_STRING_OR_ARRAY_SIZE) + throw Exception("Too large array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); + arr.resize(size); + for (auto & elem : arr) + read(elem, in); +} + +static void read(ACL & acl, ReadBuffer & in) +{ + read(acl.permissions, in); + read(acl.scheme, in); + read(acl.id, in); +} + +void ZooKeeperRequest::write(WriteBuffer & out) const +{ + /// Excessive copy to calculate length. + WriteBufferFromOwnString buf; + Coordination::write(xid, buf); + Coordination::write(getOpNum(), buf); + writeImpl(buf); + Coordination::write(buf.str(), out); + out.next(); +} + +void ZooKeeperWatchResponse::readImpl(ReadBuffer & in) +{ + Coordination::read(type, in); + Coordination::read(state, in); + Coordination::read(path, in); +} + +void ZooKeeperWatchResponse::writeImpl(WriteBuffer & out) const +{ + Coordination::write(type, out); + Coordination::write(state, out); + Coordination::write(path, out); +} + +void ZooKeeperAuthRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(type, out); + Coordination::write(scheme, out); + Coordination::write(data, out); +} + +void ZooKeeperAuthRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(type, in); + Coordination::read(scheme, in); + Coordination::read(data, in); +} + +void ZooKeeperCreateRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path, out); + Coordination::write(data, out); + Coordination::write(acls, out); + + int32_t flags = 0; + + if (is_ephemeral) + flags |= 1; + if (is_sequential) + flags |= 2; + + Coordination::write(flags, out); +} + +void ZooKeeperCreateRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(path, in); + Coordination::read(data, in); + Coordination::read(acls, in); + + int32_t flags = 0; + Coordination::read(flags, in); + + if (flags & 1) + is_ephemeral = true; + if (flags & 2) + is_sequential = true; +} + +void ZooKeeperCreateResponse::readImpl(ReadBuffer & in) +{ + Coordination::read(path_created, in); +} + +void ZooKeeperCreateResponse::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path_created, out); +} + +void ZooKeeperRemoveRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path, out); + Coordination::write(version, out); +} + +void ZooKeeperRemoveRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(path, in); + Coordination::read(version, in); +} + +void ZooKeeperExistsRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path, out); + Coordination::write(has_watch, out); +} + +void ZooKeeperExistsRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(path, in); + Coordination::read(has_watch, in); +} + +void ZooKeeperExistsResponse::readImpl(ReadBuffer & in) +{ + Coordination::read(stat, in); +} + +void ZooKeeperExistsResponse::writeImpl(WriteBuffer & out) const +{ + Coordination::write(stat, out); +} + +void ZooKeeperGetRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path, out); + Coordination::write(has_watch, out); +} + +void ZooKeeperGetRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(path, in); + Coordination::read(has_watch, in); +} + +void ZooKeeperGetResponse::readImpl(ReadBuffer & in) +{ + Coordination::read(data, in); + Coordination::read(stat, in); +} + +void ZooKeeperGetResponse::writeImpl(WriteBuffer & out) const +{ + Coordination::write(data, out); + Coordination::write(stat, out); +} + +void ZooKeeperSetRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path, out); + Coordination::write(data, out); + Coordination::write(version, out); +} + +void ZooKeeperSetRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(path, in); + Coordination::read(data, in); + Coordination::read(version, in); +} + +void ZooKeeperSetResponse::readImpl(ReadBuffer & in) +{ + Coordination::read(stat, in); +} + +void ZooKeeperSetResponse::writeImpl(WriteBuffer & out) const +{ + Coordination::write(stat, out); +} + +void ZooKeeperListRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path, out); + Coordination::write(has_watch, out); +} + +void ZooKeeperListRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(path, in); + Coordination::read(has_watch, in); +} + +void ZooKeeperListResponse::readImpl(ReadBuffer & in) +{ + Coordination::read(names, in); + Coordination::read(stat, in); +} + +void ZooKeeperListResponse::writeImpl(WriteBuffer & out) const +{ + Coordination::write(names, out); + Coordination::write(stat, out); +} + +void ZooKeeperCheckRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path, out); + Coordination::write(version, out); +} + +void ZooKeeperCheckRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(path, in); + Coordination::read(version, in); +} + +void ZooKeeperErrorResponse::readImpl(ReadBuffer & in) +{ + Coordination::Error read_error; + Coordination::read(read_error, in); + + if (read_error != error) + throw Exception(fmt::format("Error code in ErrorResponse ({}) doesn't match error code in header ({})", read_error, error), + Error::ZMARSHALLINGERROR); +} + +void ZooKeeperErrorResponse::writeImpl(WriteBuffer & out) const +{ + Coordination::write(error, out); +} + +ZooKeeperMultiRequest::ZooKeeperMultiRequest(const Requests & generic_requests, const ACLs & default_acls) +{ + /// Convert nested Requests to ZooKeeperRequests. + /// Note that deep copy is required to avoid modifying path in presence of chroot prefix. + requests.reserve(generic_requests.size()); + + for (const auto & generic_request : generic_requests) + { + if (const auto * concrete_request_create = dynamic_cast(generic_request.get())) + { + auto create = std::make_shared(*concrete_request_create); + if (create->acls.empty()) + create->acls = default_acls; + requests.push_back(create); + } + else if (const auto * concrete_request_remove = dynamic_cast(generic_request.get())) + { + requests.push_back(std::make_shared(*concrete_request_remove)); + } + else if (const auto * concrete_request_set = dynamic_cast(generic_request.get())) + { + requests.push_back(std::make_shared(*concrete_request_set)); + } + else if (const auto * concrete_request_check = dynamic_cast(generic_request.get())) + { + requests.push_back(std::make_shared(*concrete_request_check)); + } + else + throw Exception("Illegal command as part of multi ZooKeeper request", Error::ZBADARGUMENTS); + } +} + +void ZooKeeperMultiRequest::writeImpl(WriteBuffer & out) const +{ + for (const auto & request : requests) + { + const auto & zk_request = dynamic_cast(*request); + + bool done = false; + int32_t error = -1; + + Coordination::write(zk_request.getOpNum(), out); + Coordination::write(done, out); + Coordination::write(error, out); + + zk_request.writeImpl(out); + } + + OpNum op_num = -1; + bool done = true; + int32_t error = -1; + + Coordination::write(op_num, out); + Coordination::write(done, out); + Coordination::write(error, out); +} + +void ZooKeeperMultiRequest::readImpl(ReadBuffer & in) +{ + + while (true) + { + OpNum op_num; + bool done; + int32_t error; + Coordination::read(op_num, in); + Coordination::read(done, in); + Coordination::read(error, in); + + if (done) + { + if (op_num != -1) + throw Exception("Unexpected op_num received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); + if (error != -1) + throw Exception("Unexpected error value received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); + break; + } + + ZooKeeperRequestPtr request = ZooKeeperRequestFactory::instance().get(op_num); + request->readImpl(in); + requests.push_back(request); + + if (in.eof()) + throw Exception("Not enough results received for multi transaction", Error::ZMARSHALLINGERROR); + } +} + +void ZooKeeperMultiResponse::readImpl(ReadBuffer & in) +{ + for (auto & response : responses) + { + OpNum op_num; + bool done; + Error op_error; + + Coordination::read(op_num, in); + Coordination::read(done, in); + Coordination::read(op_error, in); + + if (done) + throw Exception("Not enough results received for multi transaction", Error::ZMARSHALLINGERROR); + + /// op_num == -1 is special for multi transaction. + /// For unknown reason, error code is duplicated in header and in response body. + + if (op_num == -1) + response = std::make_shared(); + + if (op_error != Error::ZOK) + { + response->error = op_error; + + /// Set error for whole transaction. + /// If some operations fail, ZK send global error as zero and then send details about each operation. + /// It will set error code for first failed operation and it will set special "runtime inconsistency" code for other operations. + if (error == Error::ZOK && op_error != Error::ZRUNTIMEINCONSISTENCY) + error = op_error; + } + + if (op_error == Error::ZOK || op_num == -1) + dynamic_cast(*response).readImpl(in); + } + + /// Footer. + { + OpNum op_num; + bool done; + int32_t error_read; + + Coordination::read(op_num, in); + Coordination::read(done, in); + Coordination::read(error_read, in); + + if (!done) + throw Exception("Too many results received for multi transaction", Error::ZMARSHALLINGERROR); + if (op_num != -1) + throw Exception("Unexpected op_num received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); + if (error_read != -1) + throw Exception("Unexpected error value received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); + } +} + +void ZooKeeperMultiResponse::writeImpl(WriteBuffer & out) const +{ + for (auto & response : responses) + { + const ZooKeeperResponse & zk_response = dynamic_cast(*response); + OpNum op_num = zk_response.getOpNum(); + bool done = false; + Error op_error = zk_response.error; + + Coordination::write(op_num, out); + Coordination::write(done, out); + Coordination::write(op_error, out); + zk_response.writeImpl(out); + } + + /// Footer. + { + OpNum op_num = -1; + bool done = true; + int32_t error_read = - 1; + + Coordination::write(op_num, out); + Coordination::write(done, out); + Coordination::write(error_read, out); + } +} + +ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperCreateRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperListRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperCheckRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperMultiRequest::makeResponse() const { return std::make_shared(requests); } +ZooKeeperResponsePtr ZooKeeperCloseRequest::makeResponse() const { return std::make_shared(); } + +void ZooKeeperRequestFactory::registerRequest(OpNum op_num, Creator creator) +{ + if (!op_num_to_request.try_emplace(op_num, creator).second) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Request with op num {} already registered", op_num); +} + +std::shared_ptr ZooKeeperRequest::read(ReadBuffer & in) +{ + XID xid; + OpNum op_num; + + Coordination::read(xid, in); + Coordination::read(op_num, in); + + auto request = ZooKeeperRequestFactory::instance().get(op_num); + request->xid = xid; + request->readImpl(in); + return request; +} + +ZooKeeperRequestPtr ZooKeeperRequestFactory::get(OpNum op_num) const +{ + auto it = op_num_to_request.find(op_num); + if (it == op_num_to_request.end()) + throw Exception("Unknown operation type " + std::to_string(op_num), Error::ZBADARGUMENTS); + + return it->second(); +} + +ZooKeeperRequestFactory & ZooKeeperRequestFactory::instance() +{ + static ZooKeeperRequestFactory factory; + return factory; +} + +template +void registerZooKeeperRequest(ZooKeeperRequestFactory & factory) +{ + factory.registerRequest(num, [] { return std::make_shared(); }); +} + +ZooKeeperRequestFactory::ZooKeeperRequestFactory() +{ + registerZooKeeperRequest<11, ZooKeeperHeartbeatRequest>(*this); + registerZooKeeperRequest<100, ZooKeeperAuthRequest>(*this); + registerZooKeeperRequest<-11, ZooKeeperCloseRequest>(*this); + registerZooKeeperRequest<1, ZooKeeperCreateRequest>(*this); + registerZooKeeperRequest<2, ZooKeeperRemoveRequest>(*this); + registerZooKeeperRequest<3, ZooKeeperExistsRequest>(*this); + registerZooKeeperRequest<4, ZooKeeperGetRequest>(*this); + registerZooKeeperRequest<5, ZooKeeperSetRequest>(*this); + registerZooKeeperRequest<12, ZooKeeperListRequest>(*this); + registerZooKeeperRequest<13, ZooKeeperCheckRequest>(*this); + registerZooKeeperRequest<14, ZooKeeperMultiRequest>(*this); +} + +} diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h new file mode 100644 index 00000000000..0b19869dd5a --- /dev/null +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -0,0 +1,326 @@ +#pragma once + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + + +namespace Coordination +{ + +using XID = int32_t; +using OpNum = int32_t; + + +struct ZooKeeperResponse : virtual Response +{ + virtual ~ZooKeeperResponse() override = default; + virtual void readImpl(ReadBuffer &) = 0; + virtual void writeImpl(WriteBuffer &) const = 0; + virtual OpNum getOpNum() const = 0; +}; + +using ZooKeeperResponsePtr = std::shared_ptr; + +/// Exposed in header file for Yandex.Metrica code. +struct ZooKeeperRequest : virtual Request +{ + XID xid = 0; + bool has_watch = false; + /// If the request was not send and the error happens, we definitely sure, that it has not been processed by the server. + /// If the request was sent and we didn't get the response and the error happens, then we cannot be sure was it processed or not. + bool probably_sent = false; + + ZooKeeperRequest() = default; + ZooKeeperRequest(const ZooKeeperRequest &) = default; + virtual ~ZooKeeperRequest() override = default; + + virtual OpNum getOpNum() const = 0; + + /// Writes length, xid, op_num, then the rest. + void write(WriteBuffer & out) const; + + virtual void writeImpl(WriteBuffer &) const = 0; + virtual void readImpl(ReadBuffer &) = 0; + + static std::shared_ptr read(ReadBuffer & in); + + virtual ZooKeeperResponsePtr makeResponse() const = 0; +}; + +using ZooKeeperRequestPtr = std::shared_ptr; + +struct ZooKeeperHeartbeatRequest final : ZooKeeperRequest +{ + String getPath() const override { return {}; } + OpNum getOpNum() const override { return 11; } + void writeImpl(WriteBuffer &) const override {} + void readImpl(ReadBuffer &) override {} + ZooKeeperResponsePtr makeResponse() const override; +}; + +struct ZooKeeperHeartbeatResponse final : ZooKeeperResponse +{ + void readImpl(ReadBuffer &) override {} + void writeImpl(WriteBuffer &) const override {} + OpNum getOpNum() const override { return 11; } +}; + +struct ZooKeeperWatchResponse final : WatchResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer & in) override; + + void writeImpl(WriteBuffer & out) const override; + + /// TODO FIXME alesap + OpNum getOpNum() const override { return 0; } +}; + +struct ZooKeeperAuthRequest final : ZooKeeperRequest +{ + int32_t type = 0; /// ignored by the server + String scheme; + String data; + + String getPath() const override { return {}; } + OpNum getOpNum() const override { return 100; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + + ZooKeeperResponsePtr makeResponse() const override; +}; + +struct ZooKeeperAuthResponse final : ZooKeeperResponse +{ + void readImpl(ReadBuffer &) override {} + void writeImpl(WriteBuffer &) const override {} + + OpNum getOpNum() const override { return 100; } +}; + +struct ZooKeeperCloseRequest final : ZooKeeperRequest +{ + String getPath() const override { return {}; } + OpNum getOpNum() const override { return -11; } + void writeImpl(WriteBuffer &) const override {} + void readImpl(ReadBuffer &) override {} + + ZooKeeperResponsePtr makeResponse() const override; +}; + +struct ZooKeeperCloseResponse final : ZooKeeperResponse +{ + void readImpl(ReadBuffer &) override + { + throw Exception("Received response for close request", Error::ZRUNTIMEINCONSISTENCY); + } + + void writeImpl(WriteBuffer &) const override {} + + OpNum getOpNum() const override { return -11; } +}; + +struct ZooKeeperCreateRequest final : CreateRequest, ZooKeeperRequest +{ + ZooKeeperCreateRequest() = default; + explicit ZooKeeperCreateRequest(const CreateRequest & base) : CreateRequest(base) {} + + OpNum getOpNum() const override { return 1; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + + ZooKeeperResponsePtr makeResponse() const override; +}; + +struct ZooKeeperCreateResponse final : CreateResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer & in) override; + + void writeImpl(WriteBuffer & out) const override; + + OpNum getOpNum() const override { return 1; } +}; + +struct ZooKeeperRemoveRequest final : RemoveRequest, ZooKeeperRequest +{ + ZooKeeperRemoveRequest() = default; + explicit ZooKeeperRemoveRequest(const RemoveRequest & base) : RemoveRequest(base) {} + + OpNum getOpNum() const override { return 2; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + + ZooKeeperResponsePtr makeResponse() const override; +}; + +struct ZooKeeperRemoveResponse final : RemoveResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer &) override {} + void writeImpl(WriteBuffer &) const override {} + OpNum getOpNum() const override { return 2; } +}; + +struct ZooKeeperExistsRequest final : ExistsRequest, ZooKeeperRequest +{ + OpNum getOpNum() const override { return 3; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + + ZooKeeperResponsePtr makeResponse() const override; +}; + +struct ZooKeeperExistsResponse final : ExistsResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer & in) override; + void writeImpl(WriteBuffer & out) const override; + OpNum getOpNum() const override { return 3; } +}; + +struct ZooKeeperGetRequest final : GetRequest, ZooKeeperRequest +{ + OpNum getOpNum() const override { return 4; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + + ZooKeeperResponsePtr makeResponse() const override; +}; + +struct ZooKeeperGetResponse final : GetResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer & in) override; + void writeImpl(WriteBuffer & out) const override; + OpNum getOpNum() const override { return 4; } +}; + +struct ZooKeeperSetRequest final : SetRequest, ZooKeeperRequest +{ + ZooKeeperSetRequest() = default; + explicit ZooKeeperSetRequest(const SetRequest & base) : SetRequest(base) {} + + OpNum getOpNum() const override { return 5; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + ZooKeeperResponsePtr makeResponse() const override; +}; + +struct ZooKeeperSetResponse final : SetResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer & in) override; + void writeImpl(WriteBuffer & out) const override; + OpNum getOpNum() const override { return 5; } +}; + +struct ZooKeeperListRequest final : ListRequest, ZooKeeperRequest +{ + OpNum getOpNum() const override { return 12; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + ZooKeeperResponsePtr makeResponse() const override; +}; + +struct ZooKeeperListResponse final : ListResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer & in) override; + void writeImpl(WriteBuffer & out) const override; + OpNum getOpNum() const override { return 12; } +}; + +struct ZooKeeperCheckRequest final : CheckRequest, ZooKeeperRequest +{ + ZooKeeperCheckRequest() = default; + explicit ZooKeeperCheckRequest(const CheckRequest & base) : CheckRequest(base) {} + + OpNum getOpNum() const override { return 13; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + + ZooKeeperResponsePtr makeResponse() const override; +}; + +struct ZooKeeperCheckResponse final : CheckResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer &) override {} + void writeImpl(WriteBuffer &) const override {} + OpNum getOpNum() const override { return 13; } +}; + +/// This response may be received only as an element of responses in MultiResponse. +struct ZooKeeperErrorResponse final : ErrorResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer & in) override; + void writeImpl(WriteBuffer & out) const override; + + OpNum getOpNum() const override { return -1; } +}; + +struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest +{ + OpNum getOpNum() const override { return 14; } + ZooKeeperMultiRequest() = default; + + ZooKeeperMultiRequest(const Requests & generic_requests, const ACLs & default_acls); + + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + + ZooKeeperResponsePtr makeResponse() const override; +}; + +struct ZooKeeperMultiResponse final : MultiResponse, ZooKeeperResponse +{ + OpNum getOpNum() const override { return 14; } + + explicit ZooKeeperMultiResponse(const Requests & requests) + { + responses.reserve(requests.size()); + + for (const auto & request : requests) + responses.emplace_back(dynamic_cast(*request).makeResponse()); + } + + explicit ZooKeeperMultiResponse(const Responses & responses_) + { + responses = responses_; + } + + void readImpl(ReadBuffer & in) override; + + void writeImpl(WriteBuffer & out) const override; + +}; + +class ZooKeeperRequestFactory final : private boost::noncopyable +{ + +public: + using Creator = std::function; + using OpNumToRequest = std::unordered_map; + + static ZooKeeperRequestFactory & instance(); + + ZooKeeperRequestPtr get(OpNum op_num) const; + + void registerRequest(OpNum op_num, Creator creator); + +private: + OpNumToRequest op_num_to_request; + +private: + ZooKeeperRequestFactory(); +}; + +} diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index f5c57781eef..f3c8b537cf2 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -281,17 +281,6 @@ static void write(int32_t x, WriteBuffer & out) writeBinary(x, out); } -static void write(bool x, WriteBuffer & out) -{ - writeBinary(x, out); -} - -static void write(const String & s, WriteBuffer & out) -{ - write(int32_t(s.size()), out); - out.write(s.data(), s.size()); -} - template void write(std::array s, WriteBuffer & out) { write(int32_t(N), out); @@ -305,14 +294,6 @@ template void write(const std::vector & arr, WriteBuffer & out) write(elem, out); } -static void write(const ACL & acl, WriteBuffer & out) -{ - write(acl.permissions, out); - write(acl.scheme, out); - write(acl.id, out); -} - - static void read(int64_t & x, ReadBuffer & in) { readBinary(x, in); @@ -332,33 +313,6 @@ static void read(Error & x, ReadBuffer & in) x = Error(code); } -static void read(bool & x, ReadBuffer & in) -{ - readBinary(x, in); -} - -static void read(String & s, ReadBuffer & in) -{ - int32_t size = 0; - read(size, in); - - if (size == -1) - { - /// It means that zookeeper node has NULL value. We will treat it like empty string. - s.clear(); - return; - } - - if (size < 0) - throw Exception("Negative size while reading string from ZooKeeper", Error::ZMARSHALLINGERROR); - - if (size > MAX_STRING_OR_ARRAY_SIZE) - throw Exception("Too large string size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); - - s.resize(size); - in.read(s.data(), size); -} - template void read(std::array & s, ReadBuffer & in) { int32_t size = 0; @@ -368,21 +322,6 @@ template void read(std::array & s, ReadBuffer & in) in.read(s.data(), N); } -static void read(Stat & stat, ReadBuffer & in) -{ - read(stat.czxid, in); - read(stat.mzxid, in); - read(stat.ctime, in); - read(stat.mtime, in); - read(stat.version, in); - read(stat.cversion, in); - read(stat.aversion, in); - read(stat.ephemeralOwner, in); - read(stat.dataLength, in); - read(stat.numChildren, in); - read(stat.pzxid, in); -} - template void read(std::vector & arr, ReadBuffer & in) { int32_t size = 0; @@ -396,7 +335,6 @@ template void read(std::vector & arr, ReadBuffer & in) read(elem, in); } - template void ZooKeeper::write(const T & x) { @@ -409,19 +347,6 @@ void ZooKeeper::read(T & x) Coordination::read(x, *in); } - -void ZooKeeperRequest::write(WriteBuffer & out) const -{ - /// Excessive copy to calculate length. - WriteBufferFromOwnString buf; - Coordination::write(xid, buf); - Coordination::write(getOpNum(), buf); - writeImpl(buf); - Coordination::write(buf.str(), out); - out.next(); -} - - static void removeRootPath(String & path, const String & root_path) { if (root_path.empty()) @@ -433,385 +358,6 @@ static void removeRootPath(String & path, const String & root_path) path = path.substr(root_path.size()); } - -struct ZooKeeperResponse : virtual Response -{ - virtual ~ZooKeeperResponse() override = default; - virtual void readImpl(ReadBuffer &) = 0; -}; - - -struct ZooKeeperHeartbeatRequest final : ZooKeeperRequest -{ - String getPath() const override { return {}; } - ZooKeeper::OpNum getOpNum() const override { return 11; } - void writeImpl(WriteBuffer &) const override {} - ZooKeeperResponsePtr makeResponse() const override; -}; - -struct ZooKeeperHeartbeatResponse final : ZooKeeperResponse -{ - void readImpl(ReadBuffer &) override {} -}; - -struct ZooKeeperWatchResponse final : WatchResponse, ZooKeeperResponse -{ - void readImpl(ReadBuffer & in) override - { - Coordination::read(type, in); - Coordination::read(state, in); - Coordination::read(path, in); - } -}; - -struct ZooKeeperAuthRequest final : ZooKeeperRequest -{ - int32_t type = 0; /// ignored by the server - String scheme; - String data; - - String getPath() const override { return {}; } - ZooKeeper::OpNum getOpNum() const override { return 100; } - void writeImpl(WriteBuffer & out) const override - { - Coordination::write(type, out); - Coordination::write(scheme, out); - Coordination::write(data, out); - } - ZooKeeperResponsePtr makeResponse() const override; -}; - -struct ZooKeeperAuthResponse final : ZooKeeperResponse -{ - void readImpl(ReadBuffer &) override {} -}; - -struct ZooKeeperCloseRequest final : ZooKeeperRequest -{ - String getPath() const override { return {}; } - ZooKeeper::OpNum getOpNum() const override { return -11; } - void writeImpl(WriteBuffer &) const override {} - ZooKeeperResponsePtr makeResponse() const override; -}; - -struct ZooKeeperCloseResponse final : ZooKeeperResponse -{ - void readImpl(ReadBuffer &) override - { - throw Exception("Received response for close request", Error::ZRUNTIMEINCONSISTENCY); - } -}; - -struct ZooKeeperCreateRequest final : CreateRequest, ZooKeeperRequest -{ - ZooKeeperCreateRequest() = default; - explicit ZooKeeperCreateRequest(const CreateRequest & base) : CreateRequest(base) {} - - ZooKeeper::OpNum getOpNum() const override { return 1; } - void writeImpl(WriteBuffer & out) const override - { - Coordination::write(path, out); - Coordination::write(data, out); - Coordination::write(acls, out); - - int32_t flags = 0; - - if (is_ephemeral) - flags |= 1; - if (is_sequential) - flags |= 2; - - Coordination::write(flags, out); - } - ZooKeeperResponsePtr makeResponse() const override; -}; - -struct ZooKeeperCreateResponse final : CreateResponse, ZooKeeperResponse -{ - void readImpl(ReadBuffer & in) override - { - Coordination::read(path_created, in); - } -}; - -struct ZooKeeperRemoveRequest final : RemoveRequest, ZooKeeperRequest -{ - ZooKeeperRemoveRequest() = default; - explicit ZooKeeperRemoveRequest(const RemoveRequest & base) : RemoveRequest(base) {} - - ZooKeeper::OpNum getOpNum() const override { return 2; } - void writeImpl(WriteBuffer & out) const override - { - Coordination::write(path, out); - Coordination::write(version, out); - } - ZooKeeperResponsePtr makeResponse() const override; -}; - -struct ZooKeeperRemoveResponse final : RemoveResponse, ZooKeeperResponse -{ - void readImpl(ReadBuffer &) override {} -}; - -struct ZooKeeperExistsRequest final : ExistsRequest, ZooKeeperRequest -{ - ZooKeeper::OpNum getOpNum() const override { return 3; } - void writeImpl(WriteBuffer & out) const override - { - Coordination::write(path, out); - Coordination::write(has_watch, out); - } - ZooKeeperResponsePtr makeResponse() const override; -}; - -struct ZooKeeperExistsResponse final : ExistsResponse, ZooKeeperResponse -{ - void readImpl(ReadBuffer & in) override - { - Coordination::read(stat, in); - } -}; - -struct ZooKeeperGetRequest final : GetRequest, ZooKeeperRequest -{ - ZooKeeper::OpNum getOpNum() const override { return 4; } - void writeImpl(WriteBuffer & out) const override - { - Coordination::write(path, out); - Coordination::write(has_watch, out); - } - ZooKeeperResponsePtr makeResponse() const override; -}; - -struct ZooKeeperGetResponse final : GetResponse, ZooKeeperResponse -{ - void readImpl(ReadBuffer & in) override - { - Coordination::read(data, in); - Coordination::read(stat, in); - } -}; - -struct ZooKeeperSetRequest final : SetRequest, ZooKeeperRequest -{ - ZooKeeperSetRequest() = default; - explicit ZooKeeperSetRequest(const SetRequest & base) : SetRequest(base) {} - - ZooKeeper::OpNum getOpNum() const override { return 5; } - void writeImpl(WriteBuffer & out) const override - { - Coordination::write(path, out); - Coordination::write(data, out); - Coordination::write(version, out); - } - ZooKeeperResponsePtr makeResponse() const override; -}; - -struct ZooKeeperSetResponse final : SetResponse, ZooKeeperResponse -{ - void readImpl(ReadBuffer & in) override - { - Coordination::read(stat, in); - } -}; - -struct ZooKeeperListRequest final : ListRequest, ZooKeeperRequest -{ - ZooKeeper::OpNum getOpNum() const override { return 12; } - void writeImpl(WriteBuffer & out) const override - { - Coordination::write(path, out); - Coordination::write(has_watch, out); - } - ZooKeeperResponsePtr makeResponse() const override; -}; - -struct ZooKeeperListResponse final : ListResponse, ZooKeeperResponse -{ - void readImpl(ReadBuffer & in) override - { - Coordination::read(names, in); - Coordination::read(stat, in); - } -}; - -struct ZooKeeperCheckRequest final : CheckRequest, ZooKeeperRequest -{ - ZooKeeperCheckRequest() = default; - explicit ZooKeeperCheckRequest(const CheckRequest & base) : CheckRequest(base) {} - - ZooKeeper::OpNum getOpNum() const override { return 13; } - void writeImpl(WriteBuffer & out) const override - { - Coordination::write(path, out); - Coordination::write(version, out); - } - ZooKeeperResponsePtr makeResponse() const override; -}; - -struct ZooKeeperCheckResponse final : CheckResponse, ZooKeeperResponse -{ - void readImpl(ReadBuffer &) override {} -}; - -/// This response may be received only as an element of responses in MultiResponse. -struct ZooKeeperErrorResponse final : ErrorResponse, ZooKeeperResponse -{ - void readImpl(ReadBuffer & in) override - { - Coordination::Error read_error; - Coordination::read(read_error, in); - - if (read_error != error) - throw Exception(fmt::format("Error code in ErrorResponse ({}) doesn't match error code in header ({})", read_error, error), - Error::ZMARSHALLINGERROR); - } -}; - -struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest -{ - ZooKeeper::OpNum getOpNum() const override { return 14; } - - ZooKeeperMultiRequest(const Requests & generic_requests, const ACLs & default_acls) - { - /// Convert nested Requests to ZooKeeperRequests. - /// Note that deep copy is required to avoid modifying path in presence of chroot prefix. - requests.reserve(generic_requests.size()); - - for (const auto & generic_request : generic_requests) - { - if (const auto * concrete_request_create = dynamic_cast(generic_request.get())) - { - auto create = std::make_shared(*concrete_request_create); - if (create->acls.empty()) - create->acls = default_acls; - requests.push_back(create); - } - else if (const auto * concrete_request_remove = dynamic_cast(generic_request.get())) - { - requests.push_back(std::make_shared(*concrete_request_remove)); - } - else if (const auto * concrete_request_set = dynamic_cast(generic_request.get())) - { - requests.push_back(std::make_shared(*concrete_request_set)); - } - else if (const auto * concrete_request_check = dynamic_cast(generic_request.get())) - { - requests.push_back(std::make_shared(*concrete_request_check)); - } - else - throw Exception("Illegal command as part of multi ZooKeeper request", Error::ZBADARGUMENTS); - } - } - - void writeImpl(WriteBuffer & out) const override - { - for (const auto & request : requests) - { - const auto & zk_request = dynamic_cast(*request); - - bool done = false; - int32_t error = -1; - - Coordination::write(zk_request.getOpNum(), out); - Coordination::write(done, out); - Coordination::write(error, out); - - zk_request.writeImpl(out); - } - - ZooKeeper::OpNum op_num = -1; - bool done = true; - int32_t error = -1; - - Coordination::write(op_num, out); - Coordination::write(done, out); - Coordination::write(error, out); - } - - ZooKeeperResponsePtr makeResponse() const override; -}; - -struct ZooKeeperMultiResponse final : MultiResponse, ZooKeeperResponse -{ - explicit ZooKeeperMultiResponse(const Requests & requests) - { - responses.reserve(requests.size()); - - for (const auto & request : requests) - responses.emplace_back(dynamic_cast(*request).makeResponse()); - } - - void readImpl(ReadBuffer & in) override - { - for (auto & response : responses) - { - ZooKeeper::OpNum op_num; - bool done; - Error op_error; - - Coordination::read(op_num, in); - Coordination::read(done, in); - Coordination::read(op_error, in); - - if (done) - throw Exception("Not enough results received for multi transaction", Error::ZMARSHALLINGERROR); - - /// op_num == -1 is special for multi transaction. - /// For unknown reason, error code is duplicated in header and in response body. - - if (op_num == -1) - response = std::make_shared(); - - if (op_error != Error::ZOK) - { - response->error = op_error; - - /// Set error for whole transaction. - /// If some operations fail, ZK send global error as zero and then send details about each operation. - /// It will set error code for first failed operation and it will set special "runtime inconsistency" code for other operations. - if (error == Error::ZOK && op_error != Error::ZRUNTIMEINCONSISTENCY) - error = op_error; - } - - if (op_error == Error::ZOK || op_num == -1) - dynamic_cast(*response).readImpl(in); - } - - /// Footer. - { - ZooKeeper::OpNum op_num; - bool done; - int32_t error_read; - - Coordination::read(op_num, in); - Coordination::read(done, in); - Coordination::read(error_read, in); - - if (!done) - throw Exception("Too many results received for multi transaction", Error::ZMARSHALLINGERROR); - if (op_num != -1) - throw Exception("Unexpected op_num received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); - if (error_read != -1) - throw Exception("Unexpected error value received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); - } - } -}; - - -ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperCreateRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperListRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperCheckRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperMultiRequest::makeResponse() const { return std::make_shared(requests); } -ZooKeeperResponsePtr ZooKeeperCloseRequest::makeResponse() const { return std::make_shared(); } - - static constexpr int32_t protocol_version = 0; static constexpr ZooKeeper::XID watch_xid = -1; @@ -1688,5 +1234,4 @@ void ZooKeeper::close() ProfileEvents::increment(ProfileEvents::ZooKeeperClose); } - } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 085b0e9856a..c96d7d2f0cb 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -85,9 +86,6 @@ namespace Coordination using namespace DB; -struct ZooKeeperRequest; - - /** Usage scenario: look at the documentation for IKeeper class. */ class ZooKeeper : public IKeeper @@ -193,7 +191,7 @@ private: struct RequestInfo { - std::shared_ptr request; + ZooKeeperRequestPtr request; ResponseCallback callback; WatchCallback watch; clock::time_point time; @@ -246,31 +244,4 @@ private: CurrentMetrics::Increment active_session_metric_increment{CurrentMetrics::ZooKeeperSession}; }; -struct ZooKeeperResponse; -using ZooKeeperResponsePtr = std::shared_ptr; - -/// Exposed in header file for Yandex.Metrica code. -struct ZooKeeperRequest : virtual Request -{ - ZooKeeper::XID xid = 0; - bool has_watch = false; - /// If the request was not send and the error happens, we definitely sure, that is has not been processed by the server. - /// If the request was sent and we didn't get the response and the error happens, then we cannot be sure was it processed or not. - bool probably_sent = false; - - ZooKeeperRequest() = default; - ZooKeeperRequest(const ZooKeeperRequest &) = default; - virtual ~ZooKeeperRequest() override = default; - - virtual ZooKeeper::OpNum getOpNum() const = 0; - - /// Writes length, xid, op_num, then the rest. - void write(WriteBuffer & out) const; - - virtual void writeImpl(WriteBuffer &) const = 0; - - virtual ZooKeeperResponsePtr makeResponse() const = 0; -}; - - } From 18fbb3dc23212ed03d6788aff612c3a7eac405f6 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 30 Oct 2020 00:29:10 +0300 Subject: [PATCH 0074/1088] some drafts --- .../AggregateFunctionSumMap.h | 2 +- src/Core/Settings.h | 2 + src/DataTypes/DataTypeTuple.cpp | 126 ++++++++++--- src/Formats/FormatFactory.cpp | 2 + src/Formats/FormatSettings.h | 4 + src/Functions/tuple.cpp | 2 +- src/IO/HTTPCommon.cpp | 5 +- src/IO/ReadHelpers.cpp | 8 +- src/Parsers/ExpressionElementParsers.cpp | 1 + .../Impl/JSONEachRowRowOutputFormat.cpp | 67 ++++++- .../Formats/Impl/JSONEachRowRowOutputFormat.h | 9 +- ...JSONEachRowWithProgressRowOutputFormat.cpp | 14 +- .../Formats/Impl/JSONRowOutputFormat.cpp | 166 +++++++++++++----- .../Formats/Impl/JSONRowOutputFormat.h | 13 +- src/Storages/StorageFile.cpp | 13 ++ 15 files changed, 343 insertions(+), 91 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 456334ee9c3..6533903bbc1 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -105,7 +105,7 @@ public: types.emplace_back(std::make_shared(result_type)); } - return std::make_shared(types); + return std::make_shared(types, Strings{"keys", "values"}); } static const auto & getArgumentColumns(const IColumn**& columns) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index fdc1ba4b28a..791ed49b7c7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -439,6 +439,8 @@ class IColumn; M(Bool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.", 0) \ \ M(Bool, output_format_json_escape_forward_slashes, true, "Controls escaping forward slashes for string outputs in JSON output format. This is intended for compatibility with JavaScript. Don't confuse with backslashes that are always escaped.", 0) \ + M(Bool, output_format_json_write_metadata, true, "Write metadata in JSON output format.", 0) \ + M(Bool, output_format_json_named_tuple_as_object, false, "Serialize named tuple columns as JSON objects.", 0) \ \ M(UInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \ M(UInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \ diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 453cb7f37a3..f37e28dcef4 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -25,12 +25,13 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; - extern const int EMPTY_DATA_PASSED; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int DUPLICATE_COLUMN; extern const int BAD_ARGUMENTS; + extern const int DUPLICATE_COLUMN; + extern const int EMPTY_DATA_PASSED; + extern const int LOGICAL_ERROR; extern const int NOT_FOUND_COLUMN_IN_BLOCK; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH; } @@ -145,6 +146,20 @@ static void addElementSafe(const DataTypes & elems, IColumn & column, F && impl) try { impl(); + + // Check that all columns now have the same size. + size_t new_size = column.size(); + for (auto i : ext::range(1, ext::size(elems))) + { + const auto & element_column = extractElementColumn(column, i); + if (element_column.size() != new_size) + { + // This is not a logical error because it may work with + // user-supplied data. + throw Exception(ErrorCodes::SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH, + "Cannot read a tuple because not all elements are present"); + } + } } catch (...) { @@ -157,6 +172,7 @@ static void addElementSafe(const DataTypes & elems, IColumn & column, F && impl) throw; } + } @@ -213,37 +229,93 @@ void DataTypeTuple::deserializeText(IColumn & column, ReadBuffer & istr, const F void DataTypeTuple::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - writeChar('[', ostr); - for (const auto i : ext::range(0, ext::size(elems))) + if (settings.json.named_tuple_as_object + && have_explicit_names) { - if (i != 0) - writeChar(',', ostr); - elems[i]->serializeAsTextJSON(extractElementColumn(column, i), row_num, ostr, settings); + writeChar('{', ostr); + for (const auto i : ext::range(0, ext::size(elems))) + { + if (i != 0) + { + writeChar(',', ostr); + } + writeJSONString(names[i], ostr, settings); + writeChar(':', ostr); + elems[i]->serializeAsTextJSON(extractElementColumn(column, i), row_num, ostr, settings); + } + writeChar('}', ostr); + } + else + { + writeChar('[', ostr); + for (const auto i : ext::range(0, ext::size(elems))) + { + if (i != 0) + writeChar(',', ostr); + elems[i]->serializeAsTextJSON(extractElementColumn(column, i), row_num, ostr, settings); + } + writeChar(']', ostr); } - writeChar(']', ostr); } void DataTypeTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - const size_t size = elems.size(); - assertChar('[', istr); - - addElementSafe(elems, column, [&] + if (settings.json.named_tuple_as_object + && have_explicit_names) { - for (const auto i : ext::range(0, size)) - { - skipWhitespaceIfAny(istr); - if (i != 0) - { - assertChar(',', istr); - skipWhitespaceIfAny(istr); - } - elems[i]->deserializeAsTextJSON(extractElementColumn(column, i), istr, settings); - } - }); + skipWhitespaceIfAny(istr); + assertChar('{', istr); + skipWhitespaceIfAny(istr); - skipWhitespaceIfAny(istr); - assertChar(']', istr); + addElementSafe(elems, column, [&] + { + // Require all elements but in arbitrary order. + for (auto i : ext::range(0, ext::size(elems))) + { + if (i > 0) + { + skipWhitespaceIfAny(istr); + assertChar(',', istr); + skipWhitespaceIfAny(istr); + } + + std::string name; + readDoubleQuotedString(name, istr); + skipWhitespaceIfAny(istr); + assertChar(':', istr); + skipWhitespaceIfAny(istr); + + const size_t element_pos = getPositionByName(name); + auto & element_column = extractElementColumn(column, element_pos); + elems[element_pos]->deserializeAsTextJSON(element_column, istr, settings); + } + }); + + skipWhitespaceIfAny(istr); + assertChar('}', istr); + } + else + { + const size_t size = elems.size(); + assertChar('[', istr); + + addElementSafe(elems, column, [&] + { + for (const auto i : ext::range(0, size)) + { + skipWhitespaceIfAny(istr); + if (i != 0) + { + assertChar(',', istr); + skipWhitespaceIfAny(istr); + } + elems[i]->deserializeAsTextJSON(extractElementColumn(column, i), istr, settings); + } + }); + + skipWhitespaceIfAny(istr); + assertChar(']', istr); + } } void DataTypeTuple::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 4dc5b816420..d7601166136 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -99,6 +99,8 @@ static FormatSettings getOutputFormatSetting(const Settings & settings, const Co format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes; + format_settings.json.write_metadata = settings.output_format_json_write_metadata; + //format_settings.json.named_tuple_as_object = settings.output_format_json_named_tuple_as_object; format_settings.csv.delimiter = settings.format_csv_delimiter; format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes; format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 8d7c3cdb49f..102ae4a5a24 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -22,6 +22,10 @@ struct FormatSettings bool quote_64bit_integers = true; bool quote_denormals = true; bool escape_forward_slashes = true; + bool write_metadata = false; + bool named_tuple_as_object = true; + bool list_of_rows = false; + bool serialize_as_strings = false; }; JSON json; diff --git a/src/Functions/tuple.cpp b/src/Functions/tuple.cpp index 6d24391ed46..6522723dde4 100644 --- a/src/Functions/tuple.cpp +++ b/src/Functions/tuple.cpp @@ -68,7 +68,7 @@ public: /// Create named tuple if possible. if (DataTypeTuple::canBeCreatedWithNames(names)) - return std::make_shared(types, names, false); + return std::make_shared(types, names); return std::make_shared(types); } diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 6b7f30cd9b6..7c3242ba0c8 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -236,7 +236,10 @@ void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPR { auto status = response.getStatus(); - if (!(status == Poco::Net::HTTPResponse::HTTP_OK || (isRedirect(status) && allow_redirects))) + if (!(status == Poco::Net::HTTPResponse::HTTP_OK + || status == Poco::Net::HTTPResponse::HTTP_CREATED + || status == Poco::Net::HTTPResponse::HTTP_ACCEPTED + || (isRedirect(status) && allow_redirects))) { std::stringstream error_message; error_message << "Received error from remote server " << request.getURI() << ". HTTP status code: " << status << " " diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 900e9c7b535..3ff8158096e 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -493,8 +493,12 @@ template static void readAnyQuotedStringInto(Vector & s, ReadBuffer & buf) { if (buf.eof() || *buf.position() != quote) - throw Exception("Cannot parse quoted string: expected opening quote", - ErrorCodes::CANNOT_PARSE_QUOTED_STRING); + { + throw Exception(ErrorCodes::CANNOT_PARSE_QUOTED_STRING, + "Cannot parse quoted string: expected opening quote '{}', got '{}'", + std::string{quote}, buf.eof() ? "EOF" : std::string{*buf.position()}); + } + ++buf.position(); while (!buf.eof()) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 3c45bd005a9..8d25609a7a7 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1475,6 +1475,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp || ParserFunction().parse(pos, node, expected) || ParserQualifiedAsterisk().parse(pos, node, expected) || ParserAsterisk().parse(pos, node, expected) + //|| ParserTupleElementExpression().parse(pos, node, expected) || ParserCompoundIdentifier().parse(pos, node, expected) || ParserSubstitution().parse(pos, node, expected) || ParserMySQLGlobalVariable().parse(pos, node, expected); diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp index b3b83949642..2459c6f2451 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -12,9 +12,9 @@ JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat( WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, - const FormatSettings & settings_, - bool yield_strings_) - : IRowOutputFormat(header_, out_, params_), settings(settings_), yield_strings(yield_strings_) + const FormatSettings & settings_) + : IRowOutputFormat(header_, out_, params_), + settings(settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -33,7 +33,7 @@ void JSONEachRowRowOutputFormat::writeField(const IColumn & column, const IDataT writeString(fields[field_number], out); writeChar(':', out); - if (yield_strings) + if (settings.json.serialize_as_strings) { WriteBufferFromOwnString buf; @@ -61,29 +61,78 @@ void JSONEachRowRowOutputFormat::writeRowStartDelimiter() void JSONEachRowRowOutputFormat::writeRowEndDelimiter() { - writeCString("}\n", out); + writeCString("}", out); field_number = 0; } +void JSONEachRowRowOutputFormat::writeRowBetweenDelimiter() +{ + if (settings.json.list_of_rows) + { + writeCString(",\n", out); + } + else + { + writeCString("\n", out); + } +} + + +void JSONEachRowRowOutputFormat::writePrefix() +{ + if (settings.json.list_of_rows) + { + writeCString("[\n", out); + } +} + + +void JSONEachRowRowOutputFormat::writeSuffix() +{ + if (settings.json.list_of_rows) + { + writeCString("\n]\n", out); + } +} + + void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory) { factory.registerOutputFormatProcessor("JSONEachRow", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, - const FormatSettings & format_settings) + const FormatSettings & _format_settings) { - return std::make_shared(buf, sample, params, format_settings, false); + FormatSettings settings = _format_settings; + settings.json.serialize_as_strings = false; + return std::make_shared(buf, sample, params, + settings); }); factory.registerOutputFormatProcessor("JSONStringsEachRow", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, - const FormatSettings & format_settings) + const FormatSettings & _format_settings) { - return std::make_shared(buf, sample, params, format_settings, true); + FormatSettings settings = _format_settings; + settings.json.serialize_as_strings = true; + return std::make_shared(buf, sample, params, + settings); + }); + + factory.registerOutputFormatProcessor("JSONList", []( + WriteBuffer & buf, + const Block & sample, + const RowOutputFormatParams & params, + const FormatSettings & _format_settings) + { + FormatSettings settings = _format_settings; + settings.json.list_of_rows = true; + return std::make_shared(buf, sample, params, + settings); }); } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h index bd9cfff68c5..38760379056 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -19,8 +19,7 @@ public: WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, - const FormatSettings & settings_, - bool yield_strings_); + const FormatSettings & settings_); String getName() const override { return "JSONEachRowRowOutputFormat"; } @@ -28,6 +27,9 @@ public: void writeFieldDelimiter() override; void writeRowStartDelimiter() override; void writeRowEndDelimiter() override; + void writeRowBetweenDelimiter() override; + void writePrefix() override; + void writeSuffix() override; protected: /// No totals and extremes. @@ -40,9 +42,6 @@ private: Names fields; FormatSettings settings; - -protected: - bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp index 48c23abd680..4612ce99f05 100644 --- a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp @@ -34,18 +34,24 @@ void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factor WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, - const FormatSettings & format_settings) + const FormatSettings & _format_settings) { - return std::make_shared(buf, sample, params, format_settings, false); + FormatSettings settings = _format_settings; + settings.json.serialize_as_strings = false; + return std::make_shared(buf, + sample, params, settings); }); factory.registerOutputFormatProcessor("JSONStringsEachRowWithProgress", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, - const FormatSettings & format_settings) + const FormatSettings & _format_settings) { - return std::make_shared(buf, sample, params, format_settings, true); + FormatSettings settings = _format_settings; + settings.json.serialize_as_strings = true; + return std::make_shared(buf, + sample, params, settings); }); } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index 517f126060f..87eb88720c2 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -1,12 +1,45 @@ +#include + +#include +#include +#include +#include #include #include -#include #include namespace DB { +void JSONRowOutputFormat::addColumn(String name, DataTypePtr type, + bool & need_validate_utf8, std::string tabs) +{ + if (!type->textCanContainOnlyValidUTF8()) + need_validate_utf8 = true; + + WriteBufferFromOwnString buf; + writeJSONString(name, buf, settings); + + const auto * as_tuple = typeid_cast(type.get()); + const bool recurse = settings.json.named_tuple_as_object + && as_tuple && as_tuple->haveExplicitNames(); + + fields.emplace_back(FieldInfo{buf.str(), type, recurse, tabs}); + + if (recurse) + { + const auto & types = as_tuple->getElements(); + const auto & names = as_tuple->getElementNames(); + + assert(types.size() == names.size()); + for (size_t i = 0; i < types.size(); i++) + { + addColumn(names[i], types[i], need_validate_utf8, tabs + "\t"); + } + } +} + JSONRowOutputFormat::JSONRowOutputFormat( WriteBuffer & out_, const Block & header, @@ -17,20 +50,22 @@ JSONRowOutputFormat::JSONRowOutputFormat( { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); - fields.assign(columns.begin(), columns.end()); + fields.reserve(columns.size()); + + const std::string initial_tabs = settings.json.write_metadata ? "\t\t\t" : "\t\t"; bool need_validate_utf8 = false; - for (size_t i = 0; i < sample.columns(); ++i) + for (const auto & column : columns) { - if (!sample.getByPosition(i).type->textCanContainOnlyValidUTF8()) - need_validate_utf8 = true; - - WriteBufferFromOwnString buf; - writeJSONString(fields[i].name, buf, settings); - - fields[i].name = buf.str(); + addColumn(column.name, column.type, need_validate_utf8, initial_tabs); } +// for (size_t i = 0; i < fields.size(); i++) +// { +// fmt::print(stderr, "{}: '{}' '{}' '{}\n", +// i, fields[i].name, fields[i].type->getName(), fields[i].recurse); +// } + if (need_validate_utf8) { validating_ostr = std::make_unique(out); @@ -43,40 +78,76 @@ JSONRowOutputFormat::JSONRowOutputFormat( void JSONRowOutputFormat::writePrefix() { - writeCString("{\n", *ostr); - writeCString("\t\"meta\":\n", *ostr); - writeCString("\t[\n", *ostr); - - for (size_t i = 0; i < fields.size(); ++i) + if (settings.json.write_metadata) { - writeCString("\t\t{\n", *ostr); + writeCString("{\n", *ostr); + writeCString("\t\"meta\":\n", *ostr); + writeCString("\t[\n", *ostr); - writeCString("\t\t\t\"name\": ", *ostr); - writeString(fields[i].name, *ostr); - writeCString(",\n", *ostr); - writeCString("\t\t\t\"type\": ", *ostr); - writeJSONString(fields[i].type->getName(), *ostr, settings); - writeChar('\n', *ostr); + for (size_t i = 0; i < fields.size(); ++i) + { + writeCString("\t\t{\n", *ostr); - writeCString("\t\t}", *ostr); - if (i + 1 < fields.size()) - writeChar(',', *ostr); + writeCString("\t\t\t\"name\": ", *ostr); + writeString(fields[i].name, *ostr); + writeCString(",\n", *ostr); + writeCString("\t\t\t\"type\": ", *ostr); + writeJSONString(fields[i].type->getName(), *ostr, settings); + writeChar('\n', *ostr); + + writeCString("\t\t}", *ostr); + if (i + 1 < fields.size()) + writeChar(',', *ostr); + writeChar('\n', *ostr); + } + + writeCString("\t],\n", *ostr); writeChar('\n', *ostr); + writeCString("\t\"data\":\n", *ostr); + writeCString("\t", *ostr); } - - writeCString("\t],\n", *ostr); - writeChar('\n', *ostr); - writeCString("\t\"data\":\n", *ostr); - writeCString("\t[\n", *ostr); + writeCString("[\n", *ostr); } - void JSONRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) { - writeCString("\t\t\t", *ostr); +// fmt::print(stderr, "write field column '{}' type '{}'\n", +// column.getName(), type.getName()); + + writeString(fields[field_number].tabs, *ostr); writeString(fields[field_number].name, *ostr); writeCString(": ", *ostr); + // Sanity check: the input column type is the same as in header block. + // If I don't write out the raw pointer explicitly, for some reason clang + // complains about side effect in dereferencing the pointer: + // src/Processors/Formats/Impl/JSONRowOutputFormat.cpp:120:35: warning: expression with side effects will be evaluated despite being used as an operand to 'typeid' [-Wpotentially-evaluated-expression] + [[maybe_unused]] const IDataType * raw_ptr = fields[field_number].type.get(); + assert(typeid(type) == typeid(*raw_ptr)); + + if (fields[field_number].recurse) + { + const auto & tabs = fields[field_number].tabs; + ++field_number; + const auto & tuple_column = assert_cast(column); + const auto & nested_columns = tuple_column.getColumns(); + writeCString("{\n", *ostr); + for (size_t i = 0; i < nested_columns.size(); i++) + { + // field_number is incremented inside, and should match the nested + // columns. + writeField(*nested_columns[i], *fields[field_number].type, row_num); + if (i + 1 < nested_columns.size()) + { + writeCString(",", *ostr); + } + writeCString("\n", *ostr); + } + writeString(tabs, *ostr); + writeCString("}", *ostr); + return; + } + if (yield_strings) { WriteBufferFromOwnString buf; @@ -144,6 +215,12 @@ void JSONRowOutputFormat::writeSuffix() void JSONRowOutputFormat::writeBeforeTotals() { + if (!settings.json.write_metadata) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Cannot output totals in JSON format without metadata"); + } + writeCString(",\n", *ostr); writeChar('\n', *ostr); writeCString("\t\"totals\":\n", *ostr); @@ -172,6 +249,12 @@ void JSONRowOutputFormat::writeAfterTotals() void JSONRowOutputFormat::writeBeforeExtremes() { + if (!settings.json.write_metadata) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Cannot output extremes in JSON format without metadata"); + } + writeCString(",\n", *ostr); writeChar('\n', *ostr); writeCString("\t\"extremes\":\n", *ostr); @@ -217,17 +300,20 @@ void JSONRowOutputFormat::writeAfterExtremes() void JSONRowOutputFormat::writeLastSuffix() { - writeCString(",\n\n", *ostr); - writeCString("\t\"rows\": ", *ostr); - writeIntText(row_count, *ostr); + if (settings.json.write_metadata) + { + writeCString(",\n\n", *ostr); + writeCString("\t\"rows\": ", *ostr); + writeIntText(row_count, *ostr); - writeRowsBeforeLimitAtLeast(); + writeRowsBeforeLimitAtLeast(); - if (settings.write_statistics) - writeStatistics(); + if (settings.write_statistics) + writeStatistics(); - writeChar('\n', *ostr); - writeCString("}\n", *ostr); + writeChar('\n', *ostr); + writeCString("}\n", *ostr); + } ostr->next(); } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.h b/src/Processors/Formats/Impl/JSONRowOutputFormat.h index 88b74afbabd..a4593663aeb 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.h @@ -70,6 +70,8 @@ protected: void writeRowsBeforeLimitAtLeast(); void writeStatistics(); + void addColumn(String name, DataTypePtr type, bool & need_validate_utf8, + std::string tabs); std::unique_ptr validating_ostr; /// Validates UTF-8 sequences, replaces bad sequences with replacement character. WriteBuffer * ostr; @@ -78,7 +80,16 @@ protected: size_t row_count = 0; bool applied_limit = false; size_t rows_before_limit = 0; - NamesAndTypes fields; + + struct FieldInfo + { + String name; + DataTypePtr type; + bool recurse = false; + std::string tabs; + }; + + std::vector fields; Progress progress; Stopwatch watch; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 8c7cd7b63d7..956e88f6641 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -166,6 +166,19 @@ StorageFile::StorageFile(int table_fd_, CommonArguments args) StorageFile::StorageFile(const std::string & table_path_, const std::string & user_files_path, CommonArguments args) : StorageFile(args) { + fmt::print(stderr, "Create storage file from file at \n{}\n", StackTrace().toString()); + + const auto & changes = args.context.getSettings().changes(); + for (const auto & change : changes) + { + fmt::print(stderr, "Changed setting '{}' to '{}'\n", + change.name, toString(change.value)); + } + + fmt::print(stderr, "delimiter = {}\n", + toString(args.context.getSettings().get("format_csv_delimiter"))); + + is_db_table = false; paths = getPathsList(table_path_, user_files_path, args.context); From 1a3fe377a6ce6196e139d5ceb49d73b74f9501f4 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 30 Oct 2020 04:52:37 +0300 Subject: [PATCH 0075/1088] prepare for format settings in File engine --- src/Formats/FormatFactory.cpp | 85 +++++--- src/Formats/FormatFactory.h | 15 +- src/Formats/FormatSettings.h | 190 +++++++++--------- src/Processors/Formats/IRowInputFormat.cpp | 2 - src/Processors/Formats/IRowInputFormat.h | 3 - src/Processors/Formats/IRowOutputFormat.h | 8 - .../Formats/Impl/ProtobufRowOutputFormat.cpp | 32 +-- .../Formats/Impl/ProtobufRowOutputFormat.h | 4 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 2 - src/Storages/Kafka/KafkaBlockOutputStream.cpp | 17 +- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 17 +- src/Storages/StorageFile.cpp | 52 +++-- src/Storages/StorageFile.h | 6 +- 13 files changed, 232 insertions(+), 201 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index d7601166136..c7c3c6c595a 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -41,8 +41,10 @@ const FormatFactory::Creators & FormatFactory::getCreators(const String & name) } -static FormatSettings getInputFormatSetting(const Settings & settings, const Context & context) +FormatSettings getInputFormatSettings(const Context & context) { + const auto & settings = context.getSettingsRef(); + FormatSettings format_settings; format_settings.csv.delimiter = settings.format_csv_delimiter; format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes; @@ -92,8 +94,10 @@ static FormatSettings getInputFormatSetting(const Settings & settings, const Con return format_settings; } -static FormatSettings getOutputFormatSetting(const Settings & settings, const Context & context) +FormatSettings getOutputFormatSettings(const Context & context) { + const auto & settings = context.getSettingsRef(); + FormatSettings format_settings; format_settings.enable_streaming = settings.output_format_enable_streaming; format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; @@ -144,7 +148,7 @@ BlockInputStreamPtr FormatFactory::getInput( const Block & sample, const Context & context, UInt64 max_block_size, - ReadCallback callback) const + std::optional format_settings) const { if (name == "Native") return std::make_shared(buf, sample, 0); @@ -155,10 +159,12 @@ BlockInputStreamPtr FormatFactory::getInput( if (!input_getter) throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); - const Settings & settings = context.getSettingsRef(); - FormatSettings format_settings = getInputFormatSetting(settings, context); + if (!format_settings) + { + format_settings = getInputFormatSettings(context); + } - return input_getter(buf, sample, max_block_size, callback ? callback : ReadCallback(), format_settings); + return input_getter(buf, sample, max_block_size, {}, *format_settings); } const Settings & settings = context.getSettingsRef(); @@ -184,17 +190,21 @@ BlockInputStreamPtr FormatFactory::getInput( if (!input_getter) throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); - FormatSettings format_settings = getInputFormatSetting(settings, context); + if (!format_settings) + { + format_settings = getInputFormatSettings(context); + } RowInputFormatParams row_input_format_params; row_input_format_params.max_block_size = max_block_size; - row_input_format_params.allow_errors_num = format_settings.input_allow_errors_num; - row_input_format_params.allow_errors_ratio = format_settings.input_allow_errors_ratio; - row_input_format_params.callback = std::move(callback); + row_input_format_params.allow_errors_num = format_settings->input_allow_errors_num; + row_input_format_params.allow_errors_ratio = format_settings->input_allow_errors_ratio; row_input_format_params.max_execution_time = settings.max_execution_time; row_input_format_params.timeout_overflow_mode = settings.timeout_overflow_mode; - auto input_creator_params = ParallelParsingBlockInputStream::InputCreatorParams{sample, row_input_format_params, format_settings}; + auto input_creator_params = + ParallelParsingBlockInputStream::InputCreatorParams{sample, + row_input_format_params, *format_settings}; ParallelParsingBlockInputStream::Params params{buf, input_getter, input_creator_params, file_segmentation_engine, static_cast(settings.max_threads), @@ -202,13 +212,15 @@ BlockInputStreamPtr FormatFactory::getInput( return std::make_shared(params); } - auto format = getInputFormat(name, buf, sample, context, max_block_size, std::move(callback)); + auto format = getInputFormat(name, buf, sample, context, max_block_size, + format_settings); return std::make_shared(std::move(format)); } -BlockOutputStreamPtr FormatFactory::getOutput( - const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback, const bool ignore_no_row_delimiter) const +BlockOutputStreamPtr FormatFactory::getOutput(const String & name, + WriteBuffer & buf, const Block & sample, const Context & context, + WriteCallback callback, std::optional format_settings) const { if (!getCreators(name).output_processor_creator) { @@ -216,18 +228,23 @@ BlockOutputStreamPtr FormatFactory::getOutput( if (!output_getter) throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT); - const Settings & settings = context.getSettingsRef(); - FormatSettings format_settings = getOutputFormatSetting(settings, context); + if (!format_settings) + { + format_settings = getOutputFormatSettings(context); + } /** Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. */ return std::make_shared( - output_getter(buf, sample, std::move(callback), format_settings), sample); + output_getter(buf, sample, std::move(callback), *format_settings), + sample); } - auto format = getOutputFormat(name, buf, sample, context, std::move(callback), ignore_no_row_delimiter); - return std::make_shared(std::make_shared(format), sample); + auto format = getOutputFormat(name, buf, sample, context, std::move(callback), + format_settings); + return std::make_shared( + std::make_shared(format), sample); } @@ -237,24 +254,27 @@ InputFormatPtr FormatFactory::getInputFormat( const Block & sample, const Context & context, UInt64 max_block_size, - ReadCallback callback) const + std::optional format_settings) const { const auto & input_getter = getCreators(name).input_processor_creator; if (!input_getter) throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); const Settings & settings = context.getSettingsRef(); - FormatSettings format_settings = getInputFormatSetting(settings, context); + + if (!format_settings) + { + format_settings = getInputFormatSettings(context); + } RowInputFormatParams params; params.max_block_size = max_block_size; - params.allow_errors_num = format_settings.input_allow_errors_num; - params.allow_errors_ratio = format_settings.input_allow_errors_ratio; - params.callback = std::move(callback); + params.allow_errors_num = format_settings->input_allow_errors_num; + params.allow_errors_ratio = format_settings->input_allow_errors_ratio; params.max_execution_time = settings.max_execution_time; params.timeout_overflow_mode = settings.timeout_overflow_mode; - auto format = input_getter(buf, sample, params, format_settings); + auto format = input_getter(buf, sample, params, *format_settings); /// It's a kludge. Because I cannot remove context from values format. if (auto * values = typeid_cast(format.get())) @@ -265,26 +285,29 @@ InputFormatPtr FormatFactory::getInputFormat( OutputFormatPtr FormatFactory::getOutputFormat( - const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback, const bool ignore_no_row_delimiter) const + const String & name, WriteBuffer & buf, const Block & sample, + const Context & context, WriteCallback callback, + std::optional format_settings) const { const auto & output_getter = getCreators(name).output_processor_creator; if (!output_getter) throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT); - const Settings & settings = context.getSettingsRef(); - FormatSettings format_settings = getOutputFormatSetting(settings, context); + if (!format_settings) + { + format_settings = getOutputFormatSettings(context); + } RowOutputFormatParams params; - params.ignore_no_row_delimiter = ignore_no_row_delimiter; params.callback = std::move(callback); /** TODO: Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. */ - auto format = output_getter(buf, sample, params, format_settings); + auto format = output_getter(buf, sample, params, *format_settings); /// Enable auto-flush for streaming mode. Currently it is needed by INSERT WATCH query. - if (format_settings.enable_streaming) + if (format_settings->enable_streaming) format->setAutoFlush(); /// It's a kludge. Because I cannot remove context from MySQL format. diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index d49414e3944..78d4d61ea09 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -16,6 +17,7 @@ namespace DB class Block; class Context; struct FormatSettings; +struct Settings; class ReadBuffer; class WriteBuffer; @@ -32,6 +34,8 @@ struct RowOutputFormatParams; using InputFormatPtr = std::shared_ptr; using OutputFormatPtr = std::shared_ptr; +FormatSettings getInputFormatSettings(const Context & context); +FormatSettings getOutputFormatSettings(const Context & context); /** Allows to create an IBlockInputStream or IBlockOutputStream by the name of the format. * Note: format and compression are independent things. @@ -105,10 +109,11 @@ public: const Block & sample, const Context & context, UInt64 max_block_size, - ReadCallback callback = {}) const; + std::optional format_settings = std::nullopt) const; BlockOutputStreamPtr getOutput(const String & name, WriteBuffer & buf, - const Block & sample, const Context & context, WriteCallback callback = {}, const bool ignore_no_row_delimiter = false) const; + const Block & sample, const Context & context, WriteCallback callback = {}, + std::optional format_settings = std::nullopt) const; InputFormatPtr getInputFormat( const String & name, @@ -116,10 +121,12 @@ public: const Block & sample, const Context & context, UInt64 max_block_size, - ReadCallback callback = {}) const; + std::optional format_settings = std::nullopt) const; OutputFormatPtr getOutputFormat( - const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback = {}, const bool ignore_no_row_delimiter = false) const; + const String & name, WriteBuffer & buf, const Block & sample, + const Context & context, WriteCallback callback = {}, + std::optional format_settings = std::nullopt) const; /// Register format by its name. void registerInputFormat(const String & name, InputCreator input_creator); diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 102ae4a5a24..76dc7bc1729 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -17,80 +17,6 @@ struct FormatSettings /// Option means that each chunk of data need to be formatted independently. Also each chunk will be flushed at the end of processing. bool enable_streaming = false; - struct JSON - { - bool quote_64bit_integers = true; - bool quote_denormals = true; - bool escape_forward_slashes = true; - bool write_metadata = false; - bool named_tuple_as_object = true; - bool list_of_rows = false; - bool serialize_as_strings = false; - }; - - JSON json; - - struct CSV - { - char delimiter = ','; - bool allow_single_quotes = true; - bool allow_double_quotes = true; - bool unquoted_null_literal_as_null = false; - bool empty_as_default = false; - bool crlf_end_of_line = false; - bool input_format_enum_as_number = false; - }; - - CSV csv; - - struct Pretty - { - UInt64 max_rows = 10000; - UInt64 max_column_pad_width = 250; - UInt64 max_value_width = 10000; - bool color = true; - - bool output_format_pretty_row_numbers = false; - - enum class Charset - { - UTF8, - ASCII, - }; - - Charset charset = Charset::UTF8; - }; - - Pretty pretty; - - struct Values - { - bool interpret_expressions = true; - bool deduce_templates_of_expressions = true; - bool accurate_types_of_literals = true; - }; - - Values values; - - struct Template - { - String resultset_format; - String row_format; - String row_between_delimiter; - }; - - Template template_settings; - - struct TSV - { - bool empty_as_default = false; - bool crlf_end_of_line = false; - String null_representation = "\\N"; - bool input_format_enum_as_number = false; - }; - - TSV tsv; - bool skip_unknown_fields = false; bool with_names_use_header = false; bool write_statistics = true; @@ -117,24 +43,29 @@ struct FormatSettings UInt64 input_allow_errors_num = 0; Float32 input_allow_errors_ratio = 0; - struct Arrow + struct { UInt64 row_group_size = 1000000; } arrow; - struct Parquet + struct { - UInt64 row_group_size = 1000000; - } parquet; + String schema_registry_url; + String output_codec; + UInt64 output_sync_interval = 16 * 1024; + bool allow_missing_fields = false; + } avro; - struct Schema + struct CSV { - std::string format_schema; - std::string format_schema_path; - bool is_server = false; - }; - - Schema schema; + char delimiter = ','; + bool allow_single_quotes = true; + bool allow_double_quotes = true; + bool unquoted_null_literal_as_null = false; + bool empty_as_default = false; + bool crlf_end_of_line = false; + bool input_format_enum_as_number = false; + } csv; struct Custom { @@ -145,29 +76,92 @@ struct FormatSettings std::string row_between_delimiter; std::string field_delimiter; std::string escaping_rule; - }; + } custom; - Custom custom; - - struct Avro + struct { - String schema_registry_url; - String output_codec; - UInt64 output_sync_interval = 16 * 1024; - bool allow_missing_fields = false; - }; + bool quote_64bit_integers = true; + bool quote_denormals = true; + bool escape_forward_slashes = true; + bool write_metadata = false; + bool named_tuple_as_object = true; + bool list_of_rows = false; + bool serialize_as_strings = false; + } json; - Avro avro; + struct + { + UInt64 row_group_size = 1000000; + } parquet; - struct Regexp + struct Pretty + { + UInt64 max_rows = 10000; + UInt64 max_column_pad_width = 250; + UInt64 max_value_width = 10000; + bool color = true; + + bool output_format_pretty_row_numbers = false; + + enum class Charset + { + UTF8, + ASCII, + }; + + Charset charset = Charset::UTF8; + } pretty; + + struct + { + bool write_row_delimiters = true; + /** + * Some buffers (kafka / rabbit) split the rows internally using callback + * so we can push there formats without framing / delimiters (like + * ProtobufSingle). In other cases you can't write more than single row + * in unframed format. + * Not sure we need this parameter at all, it only serves as an additional + * safety check in ProtobufSingle format, but exporting constant-size + * records w/o delimiters might be generally useful, not only for Kafka. + */ + bool allow_many_rows_no_delimiters = false; + } protobuf; + + struct { std::string regexp; std::string escaping_rule; bool skip_unmatched = false; - }; + } regexp; - Regexp regexp; + struct + { + std::string format_schema; + std::string format_schema_path; + bool is_server = false; + } schema; + struct + { + String resultset_format; + String row_format; + String row_between_delimiter; + } template_settings; + + struct + { + bool empty_as_default = false; + bool crlf_end_of_line = false; + String null_representation = "\\N"; + bool input_format_enum_as_number = false; + } tsv; + + struct + { + bool interpret_expressions = true; + bool deduce_templates_of_expressions = true; + bool accurate_types_of_literals = true; + } values; }; } diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 12d4db1f4a8..48cfdb12d8b 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -63,8 +63,6 @@ Chunk IRowInputFormat::generate() info.read_columns.clear(); if (!readRow(columns, info)) break; - if (params.callback) - params.callback(); for (size_t column_idx = 0; column_idx < info.read_columns.size(); ++column_idx) { diff --git a/src/Processors/Formats/IRowInputFormat.h b/src/Processors/Formats/IRowInputFormat.h index 1931fba2a0d..14109f9c2be 100644 --- a/src/Processors/Formats/IRowInputFormat.h +++ b/src/Processors/Formats/IRowInputFormat.h @@ -27,9 +27,6 @@ struct RowInputFormatParams UInt64 allow_errors_num; Float64 allow_errors_ratio; - using ReadCallback = std::function; - ReadCallback callback; - Poco::Timespan max_execution_time = 0; OverflowMode timeout_overflow_mode = OverflowMode::THROW; }; diff --git a/src/Processors/Formats/IRowOutputFormat.h b/src/Processors/Formats/IRowOutputFormat.h index 4312691ea5e..4fb94f7b7f7 100644 --- a/src/Processors/Formats/IRowOutputFormat.h +++ b/src/Processors/Formats/IRowOutputFormat.h @@ -15,14 +15,6 @@ struct RowOutputFormatParams // Callback used to indicate that another row is written. WriteCallback callback; - - /** - * some buffers (kafka / rabbit) split the rows internally using callback - * so we can push there formats without framing / delimiters - * (like ProtobufSingle). In other cases you can't write more than single row - * in unframed format. - */ - bool ignore_no_row_delimiter = false; }; class WriteBuffer; diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index 930a83c52da..3c885e80e31 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -23,18 +23,22 @@ ProtobufRowOutputFormat::ProtobufRowOutputFormat( const Block & header, const RowOutputFormatParams & params_, const FormatSchemaInfo & format_schema, - const bool use_length_delimiters_) + const FormatSettings & settings) : IRowOutputFormat(header, out_, params_) , data_types(header.getDataTypes()) - , writer(out, ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), header.getNames(), use_length_delimiters_) - , throw_on_multiple_rows_undelimited(!use_length_delimiters_ && !params_.ignore_no_row_delimiter) + , writer(out, + ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), + header.getNames(), settings.protobuf.write_row_delimiters) + , allow_only_one_row( + !settings.protobuf.write_row_delimiters + && !settings.protobuf.allow_many_rows_no_delimiters) { value_indices.resize(header.columns()); } void ProtobufRowOutputFormat::write(const Columns & columns, size_t row_num) { - if (throw_on_multiple_rows_undelimited && !first_row) + if (allow_only_one_row && !first_row) { throw Exception("The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter.", ErrorCodes::NO_ROW_DELIMITER); } @@ -51,19 +55,23 @@ void ProtobufRowOutputFormat::write(const Columns & columns, size_t row_num) void registerOutputFormatProcessorProtobuf(FormatFactory & factory) { - for (bool use_length_delimiters : {false, true}) + for (bool write_row_delimiters : {false, true}) { factory.registerOutputFormatProcessor( - use_length_delimiters ? "Protobuf" : "ProtobufSingle", - [use_length_delimiters](WriteBuffer & buf, + write_row_delimiters ? "Protobuf" : "ProtobufSingle", + [write_row_delimiters](WriteBuffer & buf, const Block & header, const RowOutputFormatParams & params, - const FormatSettings & settings) + const FormatSettings & _settings) { - return std::make_shared(buf, header, params, - FormatSchemaInfo(settings.schema.format_schema, "Protobuf", true, - settings.schema.is_server, settings.schema.format_schema_path), - use_length_delimiters); + FormatSettings settings = _settings; + settings.protobuf.write_row_delimiters = write_row_delimiters; + return std::make_shared( + buf, header, params, + FormatSchemaInfo(settings.schema.format_schema, "Protobuf", + true, settings.schema.is_server, + settings.schema.format_schema_path), + settings); }); } } diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h index 740efcfa24c..847f7607ff5 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h @@ -41,7 +41,7 @@ public: const Block & header, const RowOutputFormatParams & params_, const FormatSchemaInfo & format_schema, - const bool use_length_delimiters_); + const FormatSettings & settings); String getName() const override { return "ProtobufRowOutputFormat"; } @@ -53,7 +53,7 @@ private: DataTypes data_types; ProtobufWriter writer; std::vector value_indices; - const bool throw_on_multiple_rows_undelimited; + const bool allow_only_one_row; }; } diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index de5a1b71580..c3b753e7261 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -54,8 +54,6 @@ Chunk ValuesBlockInputFormat::generate() if (buf.eof() || *buf.position() == ';') break; readRow(columns, rows_in_block); - if (params.callback) - params.callback(); } catch (Exception & e) { diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.cpp b/src/Storages/Kafka/KafkaBlockOutputStream.cpp index 9d7fe465d44..685ab59fdac 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockOutputStream.cpp @@ -32,13 +32,16 @@ void KafkaBlockOutputStream::writePrefix() if (!buffer) throw Exception("Failed to create Kafka producer!", ErrorCodes::CANNOT_CREATE_IO_BUFFER); - child = FormatFactory::instance().getOutput( - storage.getFormatName(), *buffer, getHeader(), *context, [this](const Columns & columns, size_t row) - { - buffer->countRow(columns, row); - }, - /* ignore_no_row_delimiter = */ true - ); + auto format_settings = getOutputFormatSettings(*context); + format_settings.protobuf.allow_many_rows_no_delimiters = true; + + child = FormatFactory::instance().getOutput(storage.getFormatName(), *buffer, + getHeader(), *context, + [this](const Columns & columns, size_t row) + { + buffer->countRow(columns, row); + }, + format_settings); } void KafkaBlockOutputStream::write(const Block & block) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 76129dee30d..c4b2f187a5a 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -42,13 +42,16 @@ void RabbitMQBlockOutputStream::writePrefix() buffer->activateWriting(); - child = FormatFactory::instance().getOutput( - storage.getFormatName(), *buffer, getHeader(), context, [this](const Columns & /* columns */, size_t /* rows */) - { - buffer->countRow(); - }, - /* ignore_no_row_delimiter = */ true - ); + auto format_settings = getOutputFormatSettings(context); + format_settings.protobuf.allow_many_rows_no_delimiters = true; + + child = FormatFactory::instance().getOutput(storage.getFormatName(), *buffer, + getHeader(), context, + [this](const Columns & /* columns */, size_t /* rows */) + { + buffer->countRow(); + }, + format_settings); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 956e88f6641..5a9fe7872e2 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -594,32 +594,40 @@ void StorageFile::truncate( void registerStorageFile(StorageFactory & factory) { + StorageFactory::StorageFeatures storage_features{ + .supports_settings = true, + .source_access_type = AccessType::FILE + }; + factory.registerStorage( "File", - [](const StorageFactory::Arguments & args) + [](const StorageFactory::Arguments & factory_args) { - ASTs & engine_args = args.engine_args; + StorageFile::CommonArguments storage_args{ + .table_id = factory_args.table_id, + .columns = factory_args.columns, + .constraints = factory_args.constraints, + .context = factory_args.context + }; - if (!(engine_args.size() >= 1 && engine_args.size() <= 3)) // NOLINT + ASTs & engine_args_ast = factory_args.engine_args; + + if (!(engine_args_ast.size() >= 1 && engine_args_ast.size() <= 3)) // NOLINT throw Exception( "Storage File requires from 1 to 3 arguments: name of used format, source and compression_method.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); - String format_name = engine_args[0]->as().value.safeGet(); + engine_args_ast[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args_ast[0], factory_args.local_context); + storage_args.format_name = engine_args_ast[0]->as().value.safeGet(); - String compression_method; - StorageFile::CommonArguments common_args{ - args.table_id, format_name, compression_method, args.columns, args.constraints, args.context}; - - if (engine_args.size() == 1) /// Table in database - return StorageFile::create(args.relative_data_path, common_args); + if (engine_args_ast.size() == 1) /// Table in database + return StorageFile::create(factory_args.relative_data_path, storage_args); /// Will use FD if engine_args[1] is int literal or identifier with std* name int source_fd = -1; String source_path; - if (auto opt_name = tryGetIdentifierName(engine_args[1])) + if (auto opt_name = tryGetIdentifierName(engine_args_ast[1])) { if (*opt_name == "stdin") source_fd = STDIN_FILENO; @@ -631,7 +639,7 @@ void registerStorageFile(StorageFactory & factory) throw Exception( "Unknown identifier '" + *opt_name + "' in second arg of File storage constructor", ErrorCodes::UNKNOWN_IDENTIFIER); } - else if (const auto * literal = engine_args[1]->as()) + else if (const auto * literal = engine_args_ast[1]->as()) { auto type = literal->value.getType(); if (type == Field::Types::Int64) @@ -644,23 +652,23 @@ void registerStorageFile(StorageFactory & factory) throw Exception("Second argument must be path or file descriptor", ErrorCodes::BAD_ARGUMENTS); } - if (engine_args.size() == 3) + if (engine_args_ast.size() == 3) { - engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context); - compression_method = engine_args[2]->as().value.safeGet(); + engine_args_ast[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args_ast[2], factory_args.local_context); + storage_args.compression_method = engine_args_ast[2]->as().value.safeGet(); } else - compression_method = "auto"; + storage_args.compression_method = "auto"; if (0 <= source_fd) /// File descriptor - return StorageFile::create(source_fd, common_args); + return StorageFile::create(source_fd, storage_args); else /// User's file - return StorageFile::create(source_path, args.context.getUserFilesPath(), common_args); + return StorageFile::create(source_path, factory_args.context.getUserFilesPath(), storage_args); }, - { - .source_access_type = AccessType::FILE, - }); + storage_features); } + + NamesAndTypesList StorageFile::getVirtuals() const { return NamesAndTypesList{ diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index f331538b4c7..cc8f6dc2b8a 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -50,9 +50,9 @@ public: struct CommonArguments { - const StorageID & table_id; - const std::string & format_name; - const std::string & compression_method; + StorageID table_id; + std::string format_name; + std::string compression_method; const ColumnsDescription & columns; const ConstraintsDescription & constraints; const Context & context; From 145ad495ce9322bec999d939af85179980984ea2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 30 Oct 2020 13:42:32 +0300 Subject: [PATCH 0076/1088] fixes --- src/Formats/tests/tab_separated_streams.cpp | 4 ++-- src/Functions/tuple.cpp | 8 ++++++-- .../Formats/Impl/JSONRowOutputFormat.cpp | 15 ++++++++++----- 3 files changed, 18 insertions(+), 9 deletions(-) diff --git a/src/Formats/tests/tab_separated_streams.cpp b/src/Formats/tests/tab_separated_streams.cpp index bb38d51cf98..bd733e4b9aa 100644 --- a/src/Formats/tests/tab_separated_streams.cpp +++ b/src/Formats/tests/tab_separated_streams.cpp @@ -38,8 +38,8 @@ try FormatSettings format_settings; - RowInputFormatParams in_params{DEFAULT_INSERT_BLOCK_SIZE, 0, 0, []{}}; - RowOutputFormatParams out_params{[](const Columns & /* columns */, size_t /* row */){},false}; + RowInputFormatParams in_params{DEFAULT_INSERT_BLOCK_SIZE, 0, 0}; + RowOutputFormatParams out_params{[](const Columns & /* columns */, size_t /* row */){}}; InputFormatPtr input_format = std::make_shared(sample, in_buf, in_params, false, false, format_settings); BlockInputStreamPtr block_input = std::make_shared(std::move(input_format)); diff --git a/src/Functions/tuple.cpp b/src/Functions/tuple.cpp index 6522723dde4..e4cbde6f9d3 100644 --- a/src/Functions/tuple.cpp +++ b/src/Functions/tuple.cpp @@ -66,9 +66,13 @@ public: names.emplace_back(argument.name); } - /// Create named tuple if possible. + /// Create named tuple if possible. We don't print tuple element names + /// because they are bad anyway -- aliases are not used, e.g. tuple(1 a) + /// will have element name '1' and not 'a'. If we ever change this, and + /// add the ability to access tuple elements by name, like tuple(1 a).a, + /// we should probably enable printing for better discoverability. if (DataTypeTuple::canBeCreatedWithNames(names)) - return std::make_shared(types, names); + return std::make_shared(types, names, false /*print names*/); return std::make_shared(types); } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index 87eb88720c2..b7514bcff89 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -12,6 +12,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + void JSONRowOutputFormat::addColumn(String name, DataTypePtr type, bool & need_validate_utf8, std::string tabs) { @@ -29,13 +34,13 @@ void JSONRowOutputFormat::addColumn(String name, DataTypePtr type, if (recurse) { - const auto & types = as_tuple->getElements(); + const auto & element_types = as_tuple->getElements(); const auto & names = as_tuple->getElementNames(); - assert(types.size() == names.size()); - for (size_t i = 0; i < types.size(); i++) + assert(element_types.size() == names.size()); + for (size_t i = 0; i < element_types.size(); i++) { - addColumn(names[i], types[i], need_validate_utf8, tabs + "\t"); + addColumn(names[i], element_types[i], need_validate_utf8, tabs + "\t"); } } } @@ -195,7 +200,7 @@ void JSONRowOutputFormat::writeRowStartDelimiter() void JSONRowOutputFormat::writeRowEndDelimiter() { writeChar('\n', *ostr); - writeCString("\t\t}", *ostr); + writeCString("\t\t}\n", *ostr); field_number = 0; ++row_count; } From e81140b5fadb7382ef2bdb91b15212acb555ec04 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 30 Oct 2020 15:13:53 +0300 Subject: [PATCH 0077/1088] format settings in File engine --- .../AggregateFunctionSumMap.h | 2 +- src/Formats/FormatFactory.cpp | 136 +++++++----------- src/Formats/FormatFactory.h | 3 +- src/Formats/FormatSettings.h | 2 +- .../Impl/JSONEachRowRowOutputFormat.cpp | 13 +- .../Formats/Impl/JSONRowOutputFormat.cpp | 2 +- src/Storages/Kafka/KafkaBlockOutputStream.cpp | 2 +- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 2 +- src/Storages/StorageFile.cpp | 59 ++++++-- src/Storages/StorageFile.h | 2 + src/TableFunctions/TableFunctionFile.cpp | 23 ++- 11 files changed, 131 insertions(+), 115 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 6533903bbc1..456334ee9c3 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -105,7 +105,7 @@ public: types.emplace_back(std::make_shared(result_type)); } - return std::make_shared(types, Strings{"keys", "values"}); + return std::make_shared(types); } static const auto & getArgumentColumns(const IColumn**& columns) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index c7c3c6c595a..814b75960fe 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -41,102 +41,76 @@ const FormatFactory::Creators & FormatFactory::getCreators(const String & name) } -FormatSettings getInputFormatSettings(const Context & context) +FormatSettings getFormatSettings(const Context & context) { const auto & settings = context.getSettingsRef(); FormatSettings format_settings; - format_settings.csv.delimiter = settings.format_csv_delimiter; - format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes; + + format_settings.avro.allow_missing_fields = settings.input_format_avro_allow_missing_fields; + format_settings.avro.output_codec = settings.output_format_avro_codec; + format_settings.avro.output_sync_interval = settings.output_format_avro_sync_interval; + format_settings.avro.schema_registry_url = settings.format_avro_schema_registry_url.toString(); format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes; - format_settings.csv.unquoted_null_literal_as_null = settings.input_format_csv_unquoted_null_literal_as_null; + format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes; + format_settings.csv.crlf_end_of_line = settings.output_format_csv_crlf_end_of_line; + format_settings.csv.delimiter = settings.format_csv_delimiter; format_settings.csv.empty_as_default = settings.input_format_defaults_for_omitted_fields; format_settings.csv.input_format_enum_as_number = settings.input_format_csv_enum_as_number; - format_settings.null_as_default = settings.input_format_null_as_default; - format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions; - format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions; - format_settings.values.accurate_types_of_literals = settings.input_format_values_accurate_types_of_literals; - format_settings.with_names_use_header = settings.input_format_with_names_use_header; - format_settings.skip_unknown_fields = settings.input_format_skip_unknown_fields; - format_settings.import_nested_json = settings.input_format_import_nested_json; + format_settings.csv.unquoted_null_literal_as_null = settings.input_format_csv_unquoted_null_literal_as_null; + format_settings.custom.escaping_rule = settings.format_custom_escaping_rule; + format_settings.custom.field_delimiter = settings.format_custom_field_delimiter; + format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter; + format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter; + format_settings.custom.result_before_delimiter = settings.format_custom_result_before_delimiter; + format_settings.custom.row_after_delimiter = settings.format_custom_row_after_delimiter; + format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter; + format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; format_settings.date_time_input_format = settings.date_time_input_format; + format_settings.date_time_output_format = settings.date_time_output_format; + format_settings.enable_streaming = settings.output_format_enable_streaming; + format_settings.import_nested_json = settings.input_format_import_nested_json; format_settings.input_allow_errors_num = settings.input_format_allow_errors_num; format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio; - format_settings.template_settings.resultset_format = settings.format_template_resultset; - format_settings.template_settings.row_format = settings.format_template_row; - format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter; - format_settings.tsv.empty_as_default = settings.input_format_tsv_empty_as_default; - format_settings.tsv.input_format_enum_as_number = settings.input_format_tsv_enum_as_number; + format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes; + format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; + format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; + format_settings.json.write_metadata = settings.output_format_json_write_metadata; + format_settings.null_as_default = settings.input_format_null_as_default; + format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; + format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; + format_settings.pretty.color = settings.output_format_pretty_color; + format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; + format_settings.pretty.max_rows = settings.output_format_pretty_max_rows; + format_settings.pretty.max_value_width = settings.output_format_pretty_max_value_width; + format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers; + format_settings.regexp.escaping_rule = settings.format_regexp_escaping_rule; + format_settings.regexp.regexp = settings.format_regexp; + format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched; format_settings.schema.format_schema = settings.format_schema; format_settings.schema.format_schema_path = context.getFormatSchemaPath(); format_settings.schema.is_server = context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER); - format_settings.custom.result_before_delimiter = settings.format_custom_result_before_delimiter; - format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter; - format_settings.custom.escaping_rule = settings.format_custom_escaping_rule; - format_settings.custom.field_delimiter = settings.format_custom_field_delimiter; - format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter; - format_settings.custom.row_after_delimiter = settings.format_custom_row_after_delimiter; - format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; - format_settings.regexp.regexp = settings.format_regexp; - format_settings.regexp.escaping_rule = settings.format_regexp_escaping_rule; - format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched; + format_settings.skip_unknown_fields = settings.input_format_skip_unknown_fields; + format_settings.template_settings.resultset_format = settings.format_template_resultset; + format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter; + format_settings.template_settings.row_format = settings.format_template_row; + format_settings.tsv.crlf_end_of_line = settings.output_format_tsv_crlf_end_of_line; + format_settings.tsv.empty_as_default = settings.input_format_tsv_empty_as_default; + format_settings.tsv.input_format_enum_as_number = settings.input_format_tsv_enum_as_number; + format_settings.tsv.null_representation = settings.output_format_tsv_null_representation; + format_settings.values.accurate_types_of_literals = settings.input_format_values_accurate_types_of_literals; + format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions; + format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions; + format_settings.with_names_use_header = settings.input_format_with_names_use_header; + format_settings.write_statistics = settings.output_format_write_statistics; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context - if (context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER)) + if (format_settings.schema.is_server) { const Poco::URI & avro_schema_registry_url = settings.format_avro_schema_registry_url; if (!avro_schema_registry_url.empty()) context.getRemoteHostFilter().checkURL(avro_schema_registry_url); } - format_settings.avro.schema_registry_url = settings.format_avro_schema_registry_url.toString(); - format_settings.avro.allow_missing_fields = settings.input_format_avro_allow_missing_fields; - - return format_settings; -} - -FormatSettings getOutputFormatSettings(const Context & context) -{ - const auto & settings = context.getSettingsRef(); - - FormatSettings format_settings; - format_settings.enable_streaming = settings.output_format_enable_streaming; - format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; - format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; - format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes; - format_settings.json.write_metadata = settings.output_format_json_write_metadata; - //format_settings.json.named_tuple_as_object = settings.output_format_json_named_tuple_as_object; - format_settings.csv.delimiter = settings.format_csv_delimiter; - format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes; - format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes; - format_settings.csv.crlf_end_of_line = settings.output_format_csv_crlf_end_of_line; - format_settings.pretty.max_rows = settings.output_format_pretty_max_rows; - format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; - format_settings.pretty.max_value_width = settings.output_format_pretty_max_value_width; - format_settings.pretty.color = settings.output_format_pretty_color; - format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? - FormatSettings::Pretty::Charset::ASCII : - FormatSettings::Pretty::Charset::UTF8; - format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers; - format_settings.template_settings.resultset_format = settings.format_template_resultset; - format_settings.template_settings.row_format = settings.format_template_row; - format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter; - format_settings.tsv.crlf_end_of_line = settings.output_format_tsv_crlf_end_of_line; - format_settings.tsv.null_representation = settings.output_format_tsv_null_representation; - format_settings.write_statistics = settings.output_format_write_statistics; - format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; - format_settings.schema.format_schema = settings.format_schema; - format_settings.schema.format_schema_path = context.getFormatSchemaPath(); - format_settings.schema.is_server = context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER); - format_settings.custom.result_before_delimiter = settings.format_custom_result_before_delimiter; - format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter; - format_settings.custom.escaping_rule = settings.format_custom_escaping_rule; - format_settings.custom.field_delimiter = settings.format_custom_field_delimiter; - format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter; - format_settings.custom.row_after_delimiter = settings.format_custom_row_after_delimiter; - format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; - format_settings.avro.output_codec = settings.output_format_avro_codec; - format_settings.avro.output_sync_interval = settings.output_format_avro_sync_interval; - format_settings.date_time_output_format = settings.date_time_output_format; return format_settings; } @@ -161,7 +135,7 @@ BlockInputStreamPtr FormatFactory::getInput( if (!format_settings) { - format_settings = getInputFormatSettings(context); + format_settings = getFormatSettings(context); } return input_getter(buf, sample, max_block_size, {}, *format_settings); @@ -192,7 +166,7 @@ BlockInputStreamPtr FormatFactory::getInput( if (!format_settings) { - format_settings = getInputFormatSettings(context); + format_settings = getFormatSettings(context); } RowInputFormatParams row_input_format_params; @@ -230,7 +204,7 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name, if (!format_settings) { - format_settings = getOutputFormatSettings(context); + format_settings = getFormatSettings(context); } /** Materialization is needed, because formats can use the functions `IDataType`, @@ -264,7 +238,7 @@ InputFormatPtr FormatFactory::getInputFormat( if (!format_settings) { - format_settings = getInputFormatSettings(context); + format_settings = getFormatSettings(context); } RowInputFormatParams params; @@ -295,7 +269,7 @@ OutputFormatPtr FormatFactory::getOutputFormat( if (!format_settings) { - format_settings = getOutputFormatSettings(context); + format_settings = getFormatSettings(context); } RowOutputFormatParams params; diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 78d4d61ea09..619acd10e0f 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -34,8 +34,7 @@ struct RowOutputFormatParams; using InputFormatPtr = std::shared_ptr; using OutputFormatPtr = std::shared_ptr; -FormatSettings getInputFormatSettings(const Context & context); -FormatSettings getOutputFormatSettings(const Context & context); +FormatSettings getFormatSettings(const Context & context); /** Allows to create an IBlockInputStream or IBlockOutputStream by the name of the format. * Note: format and compression are independent things. diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 76dc7bc1729..a4f2962c223 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -85,7 +85,7 @@ struct FormatSettings bool escape_forward_slashes = true; bool write_metadata = false; bool named_tuple_as_object = true; - bool list_of_rows = false; + bool array_of_rows = false; bool serialize_as_strings = false; } json; diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp index 2459c6f2451..79027b4afac 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -68,7 +68,7 @@ void JSONEachRowRowOutputFormat::writeRowEndDelimiter() void JSONEachRowRowOutputFormat::writeRowBetweenDelimiter() { - if (settings.json.list_of_rows) + if (settings.json.array_of_rows) { writeCString(",\n", out); } @@ -81,7 +81,7 @@ void JSONEachRowRowOutputFormat::writeRowBetweenDelimiter() void JSONEachRowRowOutputFormat::writePrefix() { - if (settings.json.list_of_rows) + if (settings.json.array_of_rows) { writeCString("[\n", out); } @@ -90,9 +90,10 @@ void JSONEachRowRowOutputFormat::writePrefix() void JSONEachRowRowOutputFormat::writeSuffix() { - if (settings.json.list_of_rows) + writeCString("\n"); + if (settings.json.array_of_rows) { - writeCString("\n]\n", out); + writeCString("]\n", out); } } @@ -123,14 +124,14 @@ void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory) settings); }); - factory.registerOutputFormatProcessor("JSONList", []( + factory.registerOutputFormatProcessor("JSONArray", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, const FormatSettings & _format_settings) { FormatSettings settings = _format_settings; - settings.json.list_of_rows = true; + settings.json.array_of_rows = true; return std::make_shared(buf, sample, params, settings); }); diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index b7514bcff89..babb217ea15 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -200,7 +200,7 @@ void JSONRowOutputFormat::writeRowStartDelimiter() void JSONRowOutputFormat::writeRowEndDelimiter() { writeChar('\n', *ostr); - writeCString("\t\t}\n", *ostr); + writeCString("\t\t}", *ostr); field_number = 0; ++row_count; } diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.cpp b/src/Storages/Kafka/KafkaBlockOutputStream.cpp index 685ab59fdac..e7bf562339f 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockOutputStream.cpp @@ -32,7 +32,7 @@ void KafkaBlockOutputStream::writePrefix() if (!buffer) throw Exception("Failed to create Kafka producer!", ErrorCodes::CANNOT_CREATE_IO_BUFFER); - auto format_settings = getOutputFormatSettings(*context); + auto format_settings = getFormatSettings(*context); format_settings.protobuf.allow_many_rows_no_delimiters = true; child = FormatFactory::instance().getOutput(storage.getFormatName(), *buffer, diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index c4b2f187a5a..b3bd57bdd0b 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -42,7 +42,7 @@ void RabbitMQBlockOutputStream::writePrefix() buffer->activateWriting(); - auto format_settings = getOutputFormatSettings(context); + auto format_settings = getFormatSettings(context); format_settings.protobuf.allow_many_rows_no_delimiters = true; child = FormatFactory::instance().getOutput(storage.getFormatName(), *buffer, diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 5a9fe7872e2..d175736e8f8 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include @@ -166,7 +167,8 @@ StorageFile::StorageFile(int table_fd_, CommonArguments args) StorageFile::StorageFile(const std::string & table_path_, const std::string & user_files_path, CommonArguments args) : StorageFile(args) { - fmt::print(stderr, "Create storage file from file at \n{}\n", StackTrace().toString()); + fmt::print(stderr, "Create storage File '{}' from file at \n{}\n", + args.table_id.getNameForLogs(), StackTrace().toString()); const auto & changes = args.context.getSettings().changes(); for (const auto & change : changes) @@ -175,7 +177,7 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us change.name, toString(change.value)); } - fmt::print(stderr, "delimiter = {}\n", + fmt::print(stderr, "delimiter = '{}'\n", toString(args.context.getSettings().get("format_csv_delimiter"))); @@ -202,6 +204,9 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArguments args) : StorageFile(args) { + fmt::print(stderr, "Create storage File '{}' from database at \n{}\n", + args.table_id.getNameForLogs(), StackTrace().toString()); + if (relative_table_dir_path.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); if (args.format_name == "Distributed") @@ -215,6 +220,7 @@ StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArgu StorageFile::StorageFile(CommonArguments args) : IStorage(args.table_id) , format_name(args.format_name) + , format_settings(args.format_settings) , compression_method(args.compression_method) , base_path(args.context.getPath()) { @@ -256,12 +262,12 @@ public: } StorageFileSource( - std::shared_ptr storage_, - const StorageMetadataPtr & metadata_snapshot_, - const Context & context_, - UInt64 max_block_size_, - FilesInfoPtr files_info_, - ColumnsDescription columns_description_) + std::shared_ptr storage_, + const StorageMetadataPtr & metadata_snapshot_, + const Context & context_, + UInt64 max_block_size_, + FilesInfoPtr files_info_, + ColumnsDescription columns_description_) : SourceWithProgress(getHeader(metadata_snapshot_, files_info_->need_path_column, files_info_->need_file_column)) , storage(std::move(storage_)) , metadata_snapshot(metadata_snapshot_) @@ -337,9 +343,11 @@ public: method = chooseCompressionMethod(current_path, storage->compression_method); } - read_buf = wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method); - reader = FormatFactory::instance().getInput( - storage->format_name, *read_buf, metadata_snapshot->getSampleBlock(), context, max_block_size); + read_buf = wrapReadBufferWithCompressionMethod( + std::move(nested_buffer), method); + reader = FormatFactory::instance().getInput(storage->format_name, + *read_buf, metadata_snapshot->getSampleBlock(), context, + max_block_size, storage->format_settings); if (columns_description.hasDefaults()) reader = std::make_shared(reader, columns_description, context); @@ -443,8 +451,11 @@ Pipe StorageFile::read( pipes.reserve(num_streams); for (size_t i = 0; i < num_streams; ++i) + { pipes.emplace_back(std::make_shared( - this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns())); + this_ptr, metadata_snapshot, context, max_block_size, files_info, + metadata_snapshot->getColumns())); + } return Pipe::unitePipes(std::move(pipes)); } @@ -457,7 +468,8 @@ public: StorageFile & storage_, const StorageMetadataPtr & metadata_snapshot_, const CompressionMethod compression_method, - const Context & context) + const Context & context, + const FormatSettings & format_settings) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , lock(storage.rwlock) @@ -485,7 +497,9 @@ public: write_buf = wrapWriteBufferWithCompressionMethod(std::move(naked_buffer), compression_method, 3); - writer = FormatFactory::instance().getOutput(storage.format_name, *write_buf, metadata_snapshot->getSampleBlock(), context); + writer = FormatFactory::instance().getOutput(storage.format_name, + *write_buf, metadata_snapshot->getSampleBlock(), context, + {}, format_settings); } Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } @@ -534,7 +548,8 @@ BlockOutputStreamPtr StorageFile::write( path = paths[0]; return std::make_shared(*this, metadata_snapshot, - chooseCompressionMethod(path, compression_method), context); + chooseCompressionMethod(path, compression_method), context, + format_settings); } Strings StorageFile::getDataPaths() const @@ -620,6 +635,20 @@ void registerStorageFile(StorageFactory & factory) engine_args_ast[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args_ast[0], factory_args.local_context); storage_args.format_name = engine_args_ast[0]->as().value.safeGet(); + if (factory_args.storage_def->settings) + { + Context local_context_copy = factory_args.local_context; + local_context_copy.applySettingsChanges( + factory_args.storage_def->settings->changes); + storage_args.format_settings = getFormatSettings( + local_context_copy); + } + else + { + storage_args.format_settings = getFormatSettings( + factory_args.local_context); + } + if (engine_args_ast.size() == 1) /// Table in database return StorageFile::create(factory_args.relative_data_path, storage_args); diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index cc8f6dc2b8a..695cd0d3912 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -52,6 +52,7 @@ public: { StorageID table_id; std::string format_name; + FormatSettings format_settings; std::string compression_method; const ColumnsDescription & columns; const ConstraintsDescription & constraints; @@ -79,6 +80,7 @@ private: explicit StorageFile(CommonArguments args); std::string format_name; + FormatSettings format_settings; int table_fd = -1; String compression_method; diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 39de6dce92c..5dbb338b91f 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -1,10 +1,12 @@ -#include -#include -#include -#include #include -#include + #include "registerTableFunctions.h" +#include +#include +#include +#include +#include +#include namespace DB { @@ -12,7 +14,16 @@ StoragePtr TableFunctionFile::getStorage( const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method_) const { - StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), format_, compression_method_, columns, ConstraintsDescription{}, global_context}; + StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), + format_, + getFormatSettings(global_context), + compression_method_, + columns, + ConstraintsDescription{}, + global_context}; + + fmt::print(stderr, "format settings delimiter = '{}'\n", + args.format_settings.csv.delimiter); return StorageFile::create(source, global_context.getUserFilesPath(), args); } From 417fa65cb487e5115281b3b12ba877f2de08cbd0 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 30 Oct 2020 16:34:59 +0300 Subject: [PATCH 0078/1088] fixup --- src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp index 79027b4afac..1c6ed0e843f 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -90,7 +90,7 @@ void JSONEachRowRowOutputFormat::writePrefix() void JSONEachRowRowOutputFormat::writeSuffix() { - writeCString("\n"); + writeCString("\n", out); if (settings.json.array_of_rows) { writeCString("]\n", out); From c2525ef211a5cc1b93914cc67a7a6a36a2f2df5e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 30 Oct 2020 17:16:47 +0300 Subject: [PATCH 0079/1088] Server and client pinging each other --- programs/server/Server.cpp | 16 + src/Common/ZooKeeper/TestKeeperStorage.cpp | 6 + src/Common/ZooKeeper/TestKeeperStorage.h | 77 +++++ src/Common/ZooKeeper/ZooKeeperCommon.h | 1 - src/Interpreters/Context.cpp | 12 + src/Interpreters/Context.h | 3 + src/Server/TCPHandlerFactory.h | 11 +- src/Server/TestKeeperTCPHandler.cpp | 339 +++++++++++++++++++++ src/Server/TestKeeperTCPHandler.h | 48 +++ 9 files changed, 510 insertions(+), 3 deletions(-) create mode 100644 src/Common/ZooKeeper/TestKeeperStorage.cpp create mode 100644 src/Common/ZooKeeper/TestKeeperStorage.h create mode 100644 src/Server/TestKeeperTCPHandler.cpp create mode 100644 src/Server/TestKeeperTCPHandler.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b85cb5e75f2..f855a76e362 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -949,6 +949,22 @@ int Server::main(const std::vector & /*args*/) LOG_INFO(log, "Listening for connections with native protocol (tcp): {}", address.toString()); }); + /// TCP TestKeeper + create_server("test_keeper_tcp_port", [&](UInt16 port) + { + Poco::Net::ServerSocket socket; + auto address = socket_bind_listen(socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + servers.emplace_back(std::make_unique( + new TCPHandlerFactory(*this, false, true), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + + LOG_INFO(log, "Listening for connections to fake zookeeper (tcp): {}", address.toString()); + }); + /// TCP with SSL create_server("tcp_port_secure", [&](UInt16 port) { diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Common/ZooKeeper/TestKeeperStorage.cpp new file mode 100644 index 00000000000..c238fa2620f --- /dev/null +++ b/src/Common/ZooKeeper/TestKeeperStorage.cpp @@ -0,0 +1,6 @@ +#include + +namespace DB +{ + +} diff --git a/src/Common/ZooKeeper/TestKeeperStorage.h b/src/Common/ZooKeeper/TestKeeperStorage.h new file mode 100644 index 00000000000..f0c8a942dff --- /dev/null +++ b/src/Common/ZooKeeper/TestKeeperStorage.h @@ -0,0 +1,77 @@ +#pragma once + +#include +#include +#include +#include + +namespace zkutil +{ + +using namespace DB; + +class TestKeeperStorage +{ + struct TestKeeperRequest; + using TestKeeperRequestPtr = std::shared_ptr; + + std::atomic session_id_counter{0}; + + struct Node + { + String data; + Coordination::ACLs acls; + bool is_ephemeral = false; + bool is_sequental = false; + Coordination::Stat stat{}; + int32_t seq_num = 0; + }; + + using Container = std::map; + + using WatchCallbacks = std::vector; + using Watches = std::map; + + Container container; + + String root_path; + + std::atomic zxid{0}; + + Watches watches; + Watches list_watches; /// Watches for 'list' request (watches on children). + + using clock = std::chrono::steady_clock; + + struct RequestInfo + { + TestKeeperRequestPtr request; + Coordination::ResponseCallback callback; + Coordination::WatchCallback watch; + clock::time_point time; + }; + using RequestsQueue = ConcurrentBoundedQueue; + RequestsQueue requests_queue{1}; + + void pushRequest(RequestInfo && request); + + void finalize(); + + ThreadFromGlobalPool processing_thread; + + void processingThread(); + +public: + void putRequest(const Coordination::ZooKeeperRequestPtr & request, std::shared_ptr response_out); + + int64_t getSessionID() + { + return session_id_counter.fetch_add(1); + } + int64_t getZXID() + { + return zxid.fetch_add(1); + } +}; + +} diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 0b19869dd5a..6293cbb09fe 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -25,7 +25,6 @@ namespace Coordination using XID = int32_t; using OpNum = int32_t; - struct ZooKeeperResponse : virtual Response { virtual ~ZooKeeperResponse() override = default; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b75e9ae9d58..863b179df90 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -300,6 +301,8 @@ struct ContextShared mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper. + mutable std::mutex test_keeper_storage_mutex; + mutable std::shared_ptr test_keeper_storage; mutable std::mutex auxiliary_zookeepers_mutex; mutable std::map auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients. @@ -1492,6 +1495,15 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const return shared->zookeeper; } +std::shared_ptr & Context::getTestKeeperStorage() const +{ + std::lock_guard lock(shared->test_keeper_storage_mutex); + if (!shared->test_keeper_storage) + shared->test_keeper_storage = std::make_shared(); + + return shared->test_keeper_storage; +} + zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const { std::lock_guard lock(shared->auxiliary_zookeepers_mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index fbf578494ed..22e90d8eb5a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -40,6 +40,7 @@ namespace Poco namespace zkutil { class ZooKeeper; + class TestKeeperStorage; } @@ -483,6 +484,8 @@ public: std::shared_ptr getZooKeeper() const; /// Same as above but return a zookeeper connection from auxiliary_zookeepers configuration entry. std::shared_ptr getAuxiliaryZooKeeper(const String & name) const; + + std::shared_ptr & getTestKeeperStorage() const; /// Has ready or expired ZooKeeper bool hasZooKeeper() const; /// Reset current zookeeper session. Do not create a new one. diff --git a/src/Server/TCPHandlerFactory.h b/src/Server/TCPHandlerFactory.h index 5ecd427bf8b..945a2350508 100644 --- a/src/Server/TCPHandlerFactory.h +++ b/src/Server/TCPHandlerFactory.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace Poco { class Logger; } @@ -16,6 +17,7 @@ class TCPHandlerFactory : public Poco::Net::TCPServerConnectionFactory private: IServer & server; Poco::Logger * log; + bool test_keeper; class DummyTCPHandler : public Poco::Net::TCPServerConnection { @@ -25,9 +27,10 @@ private: }; public: - explicit TCPHandlerFactory(IServer & server_, bool secure_ = false) + explicit TCPHandlerFactory(IServer & server_, bool secure_ = false, bool test_keeper_ = false) : server(server_) , log(&Poco::Logger::get(std::string("TCP") + (secure_ ? "S" : "") + "HandlerFactory")) + , test_keeper(test_keeper_) { } @@ -36,7 +39,11 @@ public: try { LOG_TRACE(log, "TCP Request. Address: {}", socket.peerAddress().toString()); - return new TCPHandler(server, socket); + + if (test_keeper) + return new TestKeeperTCPHandler(server, socket); + else + return new TCPHandler(server, socket); } catch (const Poco::Net::NetException &) { diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp new file mode 100644 index 00000000000..a0679554f64 --- /dev/null +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -0,0 +1,339 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + +} + +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wunused-function" +#endif + +/// ZooKeeper has 1 MB node size and serialization limit by default, +/// but it can be raised up, so we have a slightly larger limit on our side. +#define MAX_STRING_OR_ARRAY_SIZE (1 << 28) /// 256 MiB + +/// Assuming we are at little endian. + +static void write(int64_t x, WriteBuffer & out) +{ + x = __builtin_bswap64(x); + writeBinary(x, out); +} + +static void write(int32_t x, WriteBuffer & out) +{ + x = __builtin_bswap32(x); + writeBinary(x, out); +} + +static void write(bool x, WriteBuffer & out) +{ + writeBinary(x, out); +} + +static void write(const String & s, WriteBuffer & out) +{ + write(int32_t(s.size()), out); + out.write(s.data(), s.size()); +} + +template void write(std::array s, WriteBuffer & out) +{ + write(int32_t(N), out); + out.write(s.data(), N); +} + +template void write(const std::vector & arr, WriteBuffer & out) +{ + write(int32_t(arr.size()), out); + for (const auto & elem : arr) + write(elem, out); +} + +static void write(const Coordination::ACL & acl, WriteBuffer & out) +{ + write(acl.permissions, out); + write(acl.scheme, out); + write(acl.id, out); +} + +static void write(const Coordination::Stat & stat, WriteBuffer & out) +{ + write(stat.czxid, out); + write(stat.mzxid, out); + write(stat.ctime, out); + write(stat.mtime, out); + write(stat.version, out); + write(stat.cversion, out); + write(stat.aversion, out); + write(stat.ephemeralOwner, out); + write(stat.dataLength, out); + write(stat.numChildren, out); + write(stat.pzxid, out); +} + +static void write(const Coordination::Error & x, WriteBuffer & out) +{ + write(static_cast(x), out); +} + +static void read(int64_t & x, ReadBuffer & in) +{ + readBinary(x, in); + x = __builtin_bswap64(x); +} + +static void read(int32_t & x, ReadBuffer & in) +{ + readBinary(x, in); + x = __builtin_bswap32(x); +} + +static void read(Coordination::Error & x, ReadBuffer & in) +{ + int32_t code; + read(code, in); + x = Coordination::Error(code); +} + +static void read(bool & x, ReadBuffer & in) +{ + readBinary(x, in); +} + +static void read(String & s, ReadBuffer & in) +{ + int32_t size = 0; + read(size, in); + + if (size == -1) + { + /// It means that zookeeper node has NULL value. We will treat it like empty string. + s.clear(); + return; + } + + if (size < 0) + throw Exception("Negative size while reading string from ZooKeeper", ErrorCodes::LOGICAL_ERROR); + + if (size > MAX_STRING_OR_ARRAY_SIZE) + throw Exception("Too large string size while reading from ZooKeeper", ErrorCodes::LOGICAL_ERROR); + + s.resize(size); + in.read(s.data(), size); +} + +template void read(std::array & s, ReadBuffer & in) +{ + int32_t size = 0; + read(size, in); + if (size != N) + throw Exception("Unexpected array size while reading from ZooKeeper", ErrorCodes::LOGICAL_ERROR); + in.read(s.data(), N); +} + +static void read(Coordination::Stat & stat, ReadBuffer & in) +{ + read(stat.czxid, in); + read(stat.mzxid, in); + read(stat.ctime, in); + read(stat.mtime, in); + read(stat.version, in); + read(stat.cversion, in); + read(stat.aversion, in); + read(stat.ephemeralOwner, in); + read(stat.dataLength, in); + read(stat.numChildren, in); + read(stat.pzxid, in); +} + +template void read(std::vector & arr, ReadBuffer & in) +{ + int32_t size = 0; + read(size, in); + if (size < 0) + throw Exception("Negative size while reading array from ZooKeeper", ErrorCodes::LOGICAL_ERROR); + if (size > MAX_STRING_OR_ARRAY_SIZE) + throw Exception("Too large array size while reading from ZooKeeper", ErrorCodes::LOGICAL_ERROR); + arr.resize(size); + for (auto & elem : arr) + read(elem, in); +} + +static void read(Coordination::ACL & acl, ReadBuffer & in) +{ + read(acl.permissions, in); + read(acl.scheme, in); + read(acl.id, in); +} + +#ifdef __clang__ +#pragma clang diagnostic pop +#endif + +void TestKeeperTCPHandler::sendHandshake() +{ + static constexpr int32_t handshake_length = 36; + static constexpr int32_t protocol_version = 0; + static constexpr int32_t DEFAULT_SESSION_TIMEOUT = 30000; + + write(handshake_length, *out); + write(protocol_version, *out); + write(DEFAULT_SESSION_TIMEOUT, *out); + write(test_keeper_storage->getSessionID(), *out); + constexpr int32_t passwd_len = 16; + std::array passwd{}; + write(passwd, *out); + out->next(); +} + +void TestKeeperTCPHandler::run() +{ + runImpl(); +} + +void TestKeeperTCPHandler::receiveHandshake() +{ + int32_t handshake_length; + int32_t protocol_version; + int64_t last_zxid_seen; + int32_t timeout; + int64_t previous_session_id = 0; /// We don't support session restore. So previous session_id is always zero. + constexpr int32_t passwd_len = 16; + std::array passwd {}; + + read(handshake_length, *in); + if (handshake_length != 44) + throw Exception("Unexpected handshake length received: " + toString(handshake_length), ErrorCodes::LOGICAL_ERROR); + + read(protocol_version, *in); + + if (protocol_version != 0) + throw Exception("Unexpected protocol version: " + toString(protocol_version), ErrorCodes::LOGICAL_ERROR); + + read(last_zxid_seen, *in); + + if (last_zxid_seen != 0) + throw Exception("Non zero last_zxid_seen is not supported", ErrorCodes::LOGICAL_ERROR); + + read(timeout, *in); + read(previous_session_id, *in); + + if (previous_session_id != 0) + throw Exception("Non zero previous session id is not supported", ErrorCodes::LOGICAL_ERROR); + + read(passwd, *in); +} + + +void TestKeeperTCPHandler::runImpl() +{ + setThreadName("TstKprHandler"); + ThreadStatus thread_status; + auto global_receive_timeout = global_context.getSettingsRef().receive_timeout; + auto global_send_timeout = global_context.getSettingsRef().send_timeout; + + socket().setReceiveTimeout(global_receive_timeout); + socket().setSendTimeout(global_send_timeout); + socket().setNoDelay(true); + + in = std::make_shared(socket()); + out = std::make_shared(socket()); + + if (in->eof()) + { + LOG_WARNING(log, "Client has not sent any data."); + return; + } + + try + { + receiveHandshake(); + } + catch (const Exception & e) /// Typical for an incorrect username, password, or address. + { + LOG_DEBUG(log, "Cannot receive handshake {}", e.displayText()); + } + + sendHandshake(); + + while (true) + { + UInt64 max_wait = operation_timeout.totalMicroseconds(); + if (in->poll(max_wait)) + { + receiveHeartbeatRequest(); + sendHeartbeatResponse(); + } + } +} + + +void TestKeeperTCPHandler::receiveHeartbeatRequest() +{ + LOG_DEBUG(log, "Receiving heartbeat event"); + int32_t length; + read(length, *in); + int32_t total_count = in->count(); + LOG_DEBUG(log, "RECEIVED LENGTH {}", length); + int32_t xid; + LOG_DEBUG(log, "READING XID"); + read(xid, *in); + + LOG_DEBUG(log, "Received xid {}", xid); + + if (xid == -2) + { + int32_t opnum; + read(opnum, *in); + LOG_DEBUG(log, "RRECEIVED OP NUM {}", opnum); + auto request = std::make_shared(); + request->readImpl(*in); + int32_t readed = in->count() - total_count; + if (readed != length) + LOG_DEBUG(log, "EXPECTED TO READ {}, BUT GOT {}", length, readed); + } + else + { + LOG_INFO(log, "UNKNOWN EVENT xid:{}", xid); + } + + LOG_DEBUG(log, "Event received"); +} + + +void TestKeeperTCPHandler::sendHeartbeatResponse() +{ + LOG_DEBUG(log, "Sending heartbeat event"); + int32_t length = sizeof(int32_t) + sizeof(int64_t) + sizeof(Coordination::Error); + write(length, *out); + int64_t zxid = test_keeper_storage->getZXID(); + int32_t xid = -2; + write(xid, *out); + write(zxid, *out); + write(Coordination::Error::ZOK, *out); + auto response = std::make_shared(); + response->writeImpl(*out); + out->next(); +} + +} diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h new file mode 100644 index 00000000000..967ea1d29e7 --- /dev/null +++ b/src/Server/TestKeeperTCPHandler.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include "IServer.h" +#include +#include +#include +#include +#include + +namespace DB +{ + +class TestKeeperTCPHandler : public Poco::Net::TCPServerConnection +{ +public: + TestKeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_) + : Poco::Net::TCPServerConnection(socket_) + , server(server_) + , log(&Poco::Logger::get("TestKeeperTCPHandler")) + , global_context(server.context()) + , test_keeper_storage(global_context.getTestKeeperStorage()) + , operation_timeout(10000) + { + } + + void run() override; +private: + IServer & server; + Poco::Logger * log; + Context global_context; + std::shared_ptr test_keeper_storage; + Poco::Timespan operation_timeout; + + /// Streams for reading/writing from/to client connection socket. + std::shared_ptr in; + std::shared_ptr out; + + void runImpl(); + + void sendHandshake(); + void receiveHandshake(); + + void receiveHeartbeatRequest(); + void sendHeartbeatResponse(); +}; + +} From b3379e9cf05c88b3b5225e836171e9d637e69302 Mon Sep 17 00:00:00 2001 From: myrrc Date: Fri, 30 Oct 2020 19:17:57 +0300 Subject: [PATCH 0080/1088] fixed some wrong tests, updated docs --- .../aggregate-functions/reference/avg.md | 58 ++++++++++++++++++- .../reference/avgweighted.md | 36 +++++++++++- src/AggregateFunctions/AggregateFunctionAvg.h | 56 ++++++------------ .../AggregateFunctionAvgWeighted.h | 4 +- .../00700_decimal_empty_aggregates.reference | 3 - .../00700_decimal_empty_aggregates.sql | 4 -- tests/queries/0_stateless/01035_avg.reference | 2 + tests/queries/0_stateless/01035_avg.sql | 9 +++ 8 files changed, 122 insertions(+), 50 deletions(-) create mode 100644 tests/queries/0_stateless/01035_avg.reference create mode 100644 tests/queries/0_stateless/01035_avg.sql diff --git a/docs/en/sql-reference/aggregate-functions/reference/avg.md b/docs/en/sql-reference/aggregate-functions/reference/avg.md index 1741bbb744b..06117cb83cf 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/avg.md +++ b/docs/en/sql-reference/aggregate-functions/reference/avg.md @@ -4,5 +4,59 @@ toc_priority: 5 # avg {#agg_function-avg} -Calculates the average. Only works for numbers (Integral, floating-point, or Decimals). -The result is always Float64. +Calculates the arithmetic mean. + +**Syntax** + +``` sql +avgWeighted(x) +``` + +**Parameter** + +- `x` — Values. + +`x` must be +[Integer](../../../sql-reference/data-types/int-uint.md), +[floating-point](../../../sql-reference/data-types/float.md), or +[Decimal](../../../sql-reference/data-types/decimal.md). + +**Returned value** + +- `0` if the supplied parameter is empty. +- Mean otherwise. + +**Return type** is always [Float64](../../../sql-reference/data-types/float.md). + +**Example** + +Query: + +``` sql +SELECT avg(x) FROM values('x Int8', 0, 1, 2, 3, 4, 5) +``` + +Result: + +``` text +┌─avg(x)─┐ +│ 2.5 │ +└────────┘ +``` + +**Example** + +Query: + +``` sql +CREATE table test (t UInt8) ENGINE = Memory; +SELECT avg(t) FROM test +``` + +Result: + +``` text +┌─avg(x)─┐ +│ 0 │ +└────────┘ +``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md index 22993f93e16..7b9c0de2755 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md @@ -25,7 +25,7 @@ but may have different types. **Returned value** -- `NaN`. If all the weights are equal to 0. +- `NaN` if all the weights are equal to 0 or the supplied weights parameter is empty. - Weighted mean otherwise. **Return type** is always [Float64](../../../sql-reference/data-types/float.md). @@ -63,3 +63,37 @@ Result: │ 8 │ └────────────────────────┘ ``` + +**Example** + +Query: + +``` sql +SELECT avgWeighted(x, w) +FROM values('x Int8, w Int8', (0, 0), (1, 0), (10, 0)) +``` + +Result: + +``` text +┌─avgWeighted(x, weight)─┐ +│ nan │ +└────────────────────────┘ +``` + +**Example** + +Query: + +``` sql +CREATE table test (t UInt8) ENGINE = Memory; +SELECT avgWeighted(t) FROM test +``` + +Result: + +``` text +┌─avgWeighted(x, weight)─┐ +│ nan │ +└────────────────────────┘ +``` diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 16eb11143da..c28d235a8f4 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -10,26 +10,23 @@ namespace DB { -/// A type-fixed fraction represented by a pair of #Numerator and #Denominator. -template + +/// @tparam BothZeroMeansNaN If false, the pair 0 / 0 = 0, nan otherwise. +template struct RationalFraction { - constexpr RationalFraction(): numerator(0), denominator(0) {} + Float64 numerator{0}; + Denominator denominator{0}; - Numerator numerator; - Denominator denominator; - - /// Calculate the fraction as a #Result. - template - Result NO_SANITIZE_UNDEFINED result() const + Float64 NO_SANITIZE_UNDEFINED result() const { - if constexpr (std::is_floating_point_v && std::numeric_limits::is_iec559) - return static_cast(numerator) / denominator; /// allow division by zero + if constexpr (BothZeroMeansNaN && std::numeric_limits::is_iec559) + return static_cast(numerator) / denominator; /// allow division by zero if (denominator == static_cast(0)) - return static_cast(0); + return static_cast(0); - return static_cast(numerator / denominator); + return static_cast(numerator / denominator); } }; @@ -46,31 +43,17 @@ struct RationalFraction * @tparam Derived When deriving from this class, use the child class name as in CRTP, e.g. * class Self : Agg. */ -template +template class AggregateFunctionAvgBase : public - IAggregateFunctionDataHelper, Derived> + IAggregateFunctionDataHelper, Derived> { public: - using Numerator = Float64; - using Fraction = RationalFraction; - - using ResultType = Float64; - using ResultDataType = DataTypeNumber; - using ResultVectorType = ColumnVector; - + using Fraction = RationalFraction; using Base = IAggregateFunctionDataHelper; - /// ctor for native types - explicit AggregateFunctionAvgBase(const DataTypes & argument_types_): Base(argument_types_, {}), scale(0) {} + explicit AggregateFunctionAvgBase(const DataTypes & argument_types_): Base(argument_types_, {}) {} - /// ctor for Decimals - AggregateFunctionAvgBase(const IDataType & data_type, const DataTypes & argument_types_) - : Base(argument_types_, {}), scale(getDecimalScale(data_type)) {} - - DataTypePtr getReturnType() const override - { - return std::make_shared(); - } + DataTypePtr getReturnType() const override { return std::make_shared>(); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override { @@ -100,17 +83,14 @@ public: void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override { - static_cast(to).getData().push_back(this->data(place).template result()); + static_cast &>(to).getData().push_back(this->data(place).result()); } - -protected: - UInt32 scale; }; -class AggregateFunctionAvg final : public AggregateFunctionAvgBase +class AggregateFunctionAvg final : public AggregateFunctionAvgBase { public: - using AggregateFunctionAvgBase::AggregateFunctionAvgBase; + using AggregateFunctionAvgBase::AggregateFunctionAvgBase; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const final { diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index ca9f0757cba..ef9384e48ab 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -5,10 +5,10 @@ namespace DB { -class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase +class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase { public: - using AggregateFunctionAvgBase::AggregateFunctionAvgBase; + using AggregateFunctionAvgBase::AggregateFunctionAvgBase; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { diff --git a/tests/queries/0_stateless/00700_decimal_empty_aggregates.reference b/tests/queries/0_stateless/00700_decimal_empty_aggregates.reference index 580cf0e26b7..b079e91fddc 100644 --- a/tests/queries/0_stateless/00700_decimal_empty_aggregates.reference +++ b/tests/queries/0_stateless/00700_decimal_empty_aggregates.reference @@ -5,9 +5,6 @@ 0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000 -0.0000 0.0000000 0.00000000 Decimal(9, 4) Decimal(18, 7) Decimal(38, 8) -0.0000 0.0000000 0.00000000 Decimal(9, 4) Decimal(18, 7) Decimal(38, 8) -0.0000 0.0000000 0.00000000 Decimal(9, 4) Decimal(18, 7) Decimal(38, 8) (0,0,0) (0,0,0) (0,0,0) (0,0,0) (0,0,0) 0 0 0 0 0 0 diff --git a/tests/queries/0_stateless/00700_decimal_empty_aggregates.sql b/tests/queries/0_stateless/00700_decimal_empty_aggregates.sql index 2d14ffae49d..c77f605a4c2 100644 --- a/tests/queries/0_stateless/00700_decimal_empty_aggregates.sql +++ b/tests/queries/0_stateless/00700_decimal_empty_aggregates.sql @@ -16,10 +16,6 @@ SELECT sum(a), sum(b), sum(c), sumWithOverflow(a), sumWithOverflow(b), sumWithOv SELECT sum(a+1), sum(b+1), sum(c+1), sumWithOverflow(a+1), sumWithOverflow(b+1), sumWithOverflow(c+1) FROM decimal; SELECT sum(a-1), sum(b-1), sum(c-1), sumWithOverflow(a-1), sumWithOverflow(b-1), sumWithOverflow(c-1) FROM decimal; -SELECT avg(a) as aa, avg(b) as ab, avg(c) as ac, toTypeName(aa), toTypeName(ab),toTypeName(ac) FROM decimal; -SELECT avg(a) as aa, avg(b) as ab, avg(c) as ac, toTypeName(aa), toTypeName(ab),toTypeName(ac) FROM decimal WHERE a > 0; -SELECT avg(a) as aa, avg(b) as ab, avg(c) as ac, toTypeName(aa), toTypeName(ab),toTypeName(ac) FROM decimal WHERE a < 0; - SELECT (uniq(a), uniq(b), uniq(c)), (uniqCombined(a), uniqCombined(b), uniqCombined(c)), (uniqCombined(17)(a), uniqCombined(17)(b), uniqCombined(17)(c)), diff --git a/tests/queries/0_stateless/01035_avg.reference b/tests/queries/0_stateless/01035_avg.reference new file mode 100644 index 00000000000..d1644f95165 --- /dev/null +++ b/tests/queries/0_stateless/01035_avg.reference @@ -0,0 +1,2 @@ +0 +499.5 diff --git a/tests/queries/0_stateless/01035_avg.sql b/tests/queries/0_stateless/01035_avg.sql new file mode 100644 index 00000000000..ee58587736f --- /dev/null +++ b/tests/queries/0_stateless/01035_avg.sql @@ -0,0 +1,9 @@ +CREATE TABLE IF NOT EXISTS test_01035 ( + t UInt16 +) ENGINE = Memory; + +SELECT avg(t) FROM test_01035; +INSERT INTO test_01035 SELECT * FROM system.numbers LIMIT 1000; +SELECT avg(t) FROM test_01035; + +DROP TABLE IF EXISTS test_01035 From c24ffc72c76c3f95d5cba32b32fd2b02ffb9121e Mon Sep 17 00:00:00 2001 From: myrrc Date: Fri, 30 Oct 2020 19:35:55 +0300 Subject: [PATCH 0081/1088] fixed some other tests --- tests/queries/0_stateless/00700_decimal_aggregates.reference | 3 --- tests/queries/0_stateless/00700_decimal_aggregates.sql | 4 ---- .../00910_decimal_group_array_crash_3783.reference | 2 +- .../0_stateless/01018_empty_aggregation_filling.reference | 2 +- 4 files changed, 2 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/00700_decimal_aggregates.reference b/tests/queries/0_stateless/00700_decimal_aggregates.reference index f58e1900b80..251445675a2 100644 --- a/tests/queries/0_stateless/00700_decimal_aggregates.reference +++ b/tests/queries/0_stateless/00700_decimal_aggregates.reference @@ -5,9 +5,6 @@ -1275.0000 -424.99999983 -255.00000000 -1275.0000 -424.99999983 -255.00000000 101.0000 101.00000000 101.00000000 101.0000 101.00000000 101.00000000 -101.0000 -101.00000000 -101.00000000 -101.0000 -101.00000000 -101.00000000 -0.0000 0.00000000 0.00000000 -25.5000 8.49999999 5.10000000 --25.5000 -8.49999999 -5.10000000 (101,101,101) (101,101,101) (101,101,101) (101,101,101) (102,100,101) 5 5 5 10 10 10 diff --git a/tests/queries/0_stateless/00700_decimal_aggregates.sql b/tests/queries/0_stateless/00700_decimal_aggregates.sql index c9caa4fa9be..a1814fc866f 100644 --- a/tests/queries/0_stateless/00700_decimal_aggregates.sql +++ b/tests/queries/0_stateless/00700_decimal_aggregates.sql @@ -20,10 +20,6 @@ SELECT sum(a), sum(b), sum(c), sumWithOverflow(a), sumWithOverflow(b), sumWithOv SELECT sum(a+1), sum(b+1), sum(c+1), sumWithOverflow(a+1), sumWithOverflow(b+1), sumWithOverflow(c+1) FROM decimal; SELECT sum(a-1), sum(b-1), sum(c-1), sumWithOverflow(a-1), sumWithOverflow(b-1), sumWithOverflow(c-1) FROM decimal; -SELECT avg(a), avg(b), avg(c) FROM decimal; -SELECT avg(a), avg(b), avg(c) FROM decimal WHERE a > 0; -SELECT avg(a), avg(b), avg(c) FROM decimal WHERE a < 0; - SELECT (uniq(a), uniq(b), uniq(c)), (uniqCombined(a), uniqCombined(b), uniqCombined(c)), (uniqCombined(17)(a), uniqCombined(17)(b), uniqCombined(17)(c)), diff --git a/tests/queries/0_stateless/00910_decimal_group_array_crash_3783.reference b/tests/queries/0_stateless/00910_decimal_group_array_crash_3783.reference index 5bacac7ffea..232d9aa7974 100644 --- a/tests/queries/0_stateless/00910_decimal_group_array_crash_3783.reference +++ b/tests/queries/0_stateless/00910_decimal_group_array_crash_3783.reference @@ -4,6 +4,6 @@ [499500.00] [499500.00000] [499500.0000000000] -1545081300 [('ed87e57c-9331-462a-80b4-9f0c005e88c8',0.4400)] +1545081300 [('ed87e57c-9331-462a-80b4-9f0c005e88c8',0.44)] 4341757 5657967 2018-11-01 16:47:46 txt 321.380000000000 315.080000000000 0.000000000000 2018-11-02 00:00:00 4360430 5681495 2018-11-02 09:00:07 txt 274.350000000000 268.970000000000 0.000000000000 2018-11-02 00:00:00 diff --git a/tests/queries/0_stateless/01018_empty_aggregation_filling.reference b/tests/queries/0_stateless/01018_empty_aggregation_filling.reference index e16276600fb..4595c3b9112 100644 --- a/tests/queries/0_stateless/01018_empty_aggregation_filling.reference +++ b/tests/queries/0_stateless/01018_empty_aggregation_filling.reference @@ -41,7 +41,7 @@ nan \N \N \N -0.00 +0 \N 0 \N From 06ae95a10a9adb93f47b9df862716da2ba65dcca Mon Sep 17 00:00:00 2001 From: myrrc Date: Fri, 30 Oct 2020 21:08:33 +0300 Subject: [PATCH 0082/1088] fixing nan values for functions --- .../aggregate-functions/reference/avg.md | 4 +- src/AggregateFunctions/AggregateFunctionAvg.h | 15 +- .../AggregateFunctionAvgWeighted.h | 4 +- tests/queries/0_stateless/01035_avg.reference | 2 +- ...01518_nullable_aggregate_states2.reference | 3654 ++++++++--------- 5 files changed, 1839 insertions(+), 1840 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/avg.md b/docs/en/sql-reference/aggregate-functions/reference/avg.md index 06117cb83cf..e2e6aace734 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/avg.md +++ b/docs/en/sql-reference/aggregate-functions/reference/avg.md @@ -23,7 +23,7 @@ avgWeighted(x) **Returned value** -- `0` if the supplied parameter is empty. +- `NaN` if the supplied parameter is empty. - Mean otherwise. **Return type** is always [Float64](../../../sql-reference/data-types/float.md). @@ -57,6 +57,6 @@ Result: ``` text ┌─avg(x)─┐ -│ 0 │ +│ nan │ └────────┘ ``` diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index c28d235a8f4..2369ae835c3 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -11,8 +11,7 @@ namespace DB { -/// @tparam BothZeroMeansNaN If false, the pair 0 / 0 = 0, nan otherwise. -template +template struct RationalFraction { Float64 numerator{0}; @@ -20,7 +19,7 @@ struct RationalFraction Float64 NO_SANITIZE_UNDEFINED result() const { - if constexpr (BothZeroMeansNaN && std::numeric_limits::is_iec559) + if constexpr (std::numeric_limits::is_iec559) return static_cast(numerator) / denominator; /// allow division by zero if (denominator == static_cast(0)) @@ -43,12 +42,12 @@ struct RationalFraction * @tparam Derived When deriving from this class, use the child class name as in CRTP, e.g. * class Self : Agg. */ -template +template class AggregateFunctionAvgBase : public - IAggregateFunctionDataHelper, Derived> + IAggregateFunctionDataHelper, Derived> { public: - using Fraction = RationalFraction; + using Fraction = RationalFraction; using Base = IAggregateFunctionDataHelper; explicit AggregateFunctionAvgBase(const DataTypes & argument_types_): Base(argument_types_, {}) {} @@ -87,10 +86,10 @@ public: } }; -class AggregateFunctionAvg final : public AggregateFunctionAvgBase +class AggregateFunctionAvg final : public AggregateFunctionAvgBase { public: - using AggregateFunctionAvgBase::AggregateFunctionAvgBase; + using AggregateFunctionAvgBase::AggregateFunctionAvgBase; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const final { diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index ef9384e48ab..ca9f0757cba 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -5,10 +5,10 @@ namespace DB { -class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase +class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase { public: - using AggregateFunctionAvgBase::AggregateFunctionAvgBase; + using AggregateFunctionAvgBase::AggregateFunctionAvgBase; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { diff --git a/tests/queries/0_stateless/01035_avg.reference b/tests/queries/0_stateless/01035_avg.reference index d1644f95165..8e5b888b523 100644 --- a/tests/queries/0_stateless/01035_avg.reference +++ b/tests/queries/0_stateless/01035_avg.reference @@ -1,2 +1,2 @@ -0 +nan 499.5 diff --git a/tests/queries/0_stateless/01518_nullable_aggregate_states2.reference b/tests/queries/0_stateless/01518_nullable_aggregate_states2.reference index cb1a5a32ebf..e068a35dbd1 100644 --- a/tests/queries/0_stateless/01518_nullable_aggregate_states2.reference +++ b/tests/queries/0_stateless/01518_nullable_aggregate_states2.reference @@ -6,2010 +6,2010 @@ -3 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N -4 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N -5 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N -0 102 0 99900 0 300 150 15150 0 300 150 15150 0.00000 300.00000 150.00000 15150.00000 2020-01-01 2020-01-02 2020-01-01 00:00:00 2020-01-02 03:45:00 2020-01-01 00:00:00.000 2020-01-02 03:45:00.000 0 99900 49950 5044950 0 99900 49950 5044950 -32569 32366 4529.009900990099 457430 -127 124 -2.9504950495049505 -298 -1 102 1 9991 0.003 300.003 150.003 15150.3033 0.003 300.003 150.003 15150.30329 0.00300 300.00300 150.00300 15150.30300 2020-01-01 2020-01-02 2020-01-01 00:00:01 2020-01-02 03:45:01 2020-01-01 00:00:01.000 2020-01-02 03:45:01.000 1 99901 49951 5045051 1 99901 49951 5045051 -32568 32367 4530.009900990099 457531 -126 125 -1.9504950495049505 -197 -10 102 10 99910 0.03003 300.03003 150.03003 15153.03303 0.03003 300.03003 150.03002 15153.03296 0.03003 300.03003 150.03003 15153.03303 2020-01-01 2020-01-02 2020-01-01 00:00:10 2020-01-02 03:45:10 2020-01-01 00:00:10.000 2020-01-02 03:45:10.000 10 99910 49960 5045960 10 99910 49960 5045960 -32559 32376 4539.009900990099 458440 -128 127 -0.5544554455445545 -56 -100 101 100 99001 0.3003 297.3003 148.8003 14880.03003 0.3003 297.3003 148.80029 14880.02962 0.30030 297.30030 148.80030 14880.03000 2020-01-01 2020-01-02 2020-01-01 00:01:40 2020-01-02 03:30:01 2020-01-01 00:01:40.000 2020-01-02 03:30:01.000 100 99001 49550.5 4955050 100 99001 49550.5 4955050 -32469 32466 4986.02 498602 -127 124 -0.86 -86 -101 101 10091 99002 0.3033 297.3033 148.8033 14880.33033 0.3033 297.3033 148.8033 14880.33035 0.30330 297.30330 148.80330 14880.33000 2020-01-01 2020-01-02 2020-01-01 00:01:41 2020-01-02 03:30:02 2020-01-01 00:01:41.000 2020-01-02 03:30:02.000 101 99002 49551.5 4955150 101 99002 49551.5 4955150 -32468 32467 4987.02 498702 -126 125 0.14 14 -102 101 10092 99003 0.3063 297.3063 148.8063 14880.63063 0.3063 297.3063 148.8063 14880.6305 0.30630 297.30630 148.80630 14880.63000 2020-01-01 2020-01-02 2020-01-01 00:01:42 2020-01-02 03:30:03 2020-01-01 00:01:42.000 2020-01-02 03:30:03.000 102 99003 49552.5 4955250 102 99003 49552.5 4955250 -32467 32468 4988.02 498802 -125 126 1.14 114 -103 101 10093 99004 0.3093 297.3093 148.8093 14880.93093 0.3093 297.3093 148.8093 14880.93085 0.30930 297.30930 148.80930 14880.93000 2020-01-01 2020-01-02 2020-01-01 00:01:43 2020-01-02 03:30:04 2020-01-01 00:01:43.000 2020-01-02 03:30:04.000 103 99004 49553.5 4955350 103 99004 49553.5 4955350 -32466 32469 4989.02 498902 -124 127 2.14 214 -104 101 10094 99005 0.31231 297.31231 148.81231 14881.23123 0.31231 297.31232 148.81231 14881.23144 0.31231 297.31231 148.81231 14881.23100 2020-01-01 2020-01-02 2020-01-01 00:01:44 2020-01-02 03:30:05 2020-01-01 00:01:44.000 2020-01-02 03:30:05.000 104 99005 49554.5 4955450 104 99005 49554.5 4955450 -32465 32470 4990.02 499002 -128 127 0.58 58 -105 101 10095 99006 0.31531 297.31531 148.81531 14881.53153 0.31531 297.3153 148.81531 14881.53174 0.31531 297.31531 148.81531 14881.53100 2020-01-01 2020-01-02 2020-01-01 00:01:45 2020-01-02 03:30:06 2020-01-01 00:01:45.000 2020-01-02 03:30:06.000 105 99006 49555.5 4955550 105 99006 49555.5 4955550 -32464 32471 4991.02 499102 -128 123 -0.98 -98 -106 101 10096 99007 0.31831 297.31831 148.81831 14881.83183 0.31831 297.31833 148.81831 14881.83182 0.31831 297.31831 148.81831 14881.83100 2020-01-01 2020-01-02 2020-01-01 00:01:46 2020-01-02 03:30:07 2020-01-01 00:01:46.000 2020-01-02 03:30:07.000 106 99007 49556.5 4955650 106 99007 49556.5 4955650 -32463 32472 4992.02 499202 -127 124 0.02 2 -107 101 10097 99008 0.32132 297.32132 148.82132 14882.13213 0.32132 297.32132 148.82131 14882.13197 0.32132 297.32132 148.82132 14882.13200 2020-01-01 2020-01-02 2020-01-01 00:01:47 2020-01-02 03:30:08 2020-01-01 00:01:47.000 2020-01-02 03:30:08.000 107 99008 49557.5 4955750 107 99008 49557.5 4955750 -32462 32473 4993.02 499302 -126 125 1.02 102 -108 101 10098 99009 0.32432 297.32432 148.82432 14882.43243 0.32432 297.3243 148.82432 14882.43232 0.32432 297.32432 148.82432 14882.43200 2020-01-01 2020-01-02 2020-01-01 00:01:48 2020-01-02 03:30:09 2020-01-01 00:01:48.000 2020-01-02 03:30:09.000 108 99009 49558.5 4955850 108 99009 49558.5 4955850 -32461 32474 4994.02 499402 -125 126 2.02 202 -109 101 10099 99010 0.32732 297.32732 148.82732 14882.73273 0.32732 297.32733 148.82732 14882.7329 0.32732 297.32732 148.82732 14882.73200 2020-01-01 2020-01-02 2020-01-01 00:01:49 2020-01-02 03:30:10 2020-01-01 00:01:49.000 2020-01-02 03:30:10.000 109 99010 49559.5 4955950 109 99010 49559.5 4955950 -32460 32475 4995.02 499502 -124 127 3.02 302 +0 102 0 99900 0 300 150 15150 0 300 150 15150 0.00000 300.00000 150 15150.00000 2020-01-01 2020-01-02 2020-01-01 00:00:00 2020-01-02 03:45:00 2020-01-01 00:00:00.000 2020-01-02 03:45:00.000 0 99900 49950 5044950 0 99900 49950 5044950 -32569 32366 4529.009900990099 457430 -127 124 -2.9504950495049505 -298 +1 102 1 9991 0.003 300.003 150.003 15150.3033 0.003 300.003 150.003 15150.30329 0.00300 300.00300 150.00300000000004 15150.30300 2020-01-01 2020-01-02 2020-01-01 00:00:01 2020-01-02 03:45:01 2020-01-01 00:00:01.000 2020-01-02 03:45:01.000 1 99901 49951 5045051 1 99901 49951 5045051 -32568 32367 4530.009900990099 457531 -126 125 -1.9504950495049505 -197 +10 102 10 99910 0.03003 300.03003 150.03003 15153.03303 0.03003 300.03003 150.03002 15153.03296 0.03003 300.03003 150.03002999999993 15153.03303 2020-01-01 2020-01-02 2020-01-01 00:00:10 2020-01-02 03:45:10 2020-01-01 00:00:10.000 2020-01-02 03:45:10.000 10 99910 49960 5045960 10 99910 49960 5045960 -32559 32376 4539.009900990099 458440 -128 127 -0.5544554455445545 -56 +100 101 100 99001 0.3003 297.3003 148.8003 14880.03003 0.3003 297.3003 148.80029 14880.02962 0.30030 297.30030 148.80030000000005 14880.03000 2020-01-01 2020-01-02 2020-01-01 00:01:40 2020-01-02 03:30:01 2020-01-01 00:01:40.000 2020-01-02 03:30:01.000 100 99001 49550.5 4955050 100 99001 49550.5 4955050 -32469 32466 4986.02 498602 -127 124 -0.86 -86 +101 101 10091 99002 0.3033 297.3033 148.8033 14880.33033 0.3033 297.3033 148.8033 14880.33035 0.30330 297.30330 148.80329999999987 14880.33000 2020-01-01 2020-01-02 2020-01-01 00:01:41 2020-01-02 03:30:02 2020-01-01 00:01:41.000 2020-01-02 03:30:02.000 101 99002 49551.5 4955150 101 99002 49551.5 4955150 -32468 32467 4987.02 498702 -126 125 0.14 14 +102 101 10092 99003 0.3063 297.3063 148.8063 14880.63063 0.3063 297.3063 148.8063 14880.6305 0.30630 297.30630 148.80630000000008 14880.63000 2020-01-01 2020-01-02 2020-01-01 00:01:42 2020-01-02 03:30:03 2020-01-01 00:01:42.000 2020-01-02 03:30:03.000 102 99003 49552.5 4955250 102 99003 49552.5 4955250 -32467 32468 4988.02 498802 -125 126 1.14 114 +103 101 10093 99004 0.3093 297.3093 148.8093 14880.93093 0.3093 297.3093 148.8093 14880.93085 0.30930 297.30930 148.80930000000006 14880.93000 2020-01-01 2020-01-02 2020-01-01 00:01:43 2020-01-02 03:30:04 2020-01-01 00:01:43.000 2020-01-02 03:30:04.000 103 99004 49553.5 4955350 103 99004 49553.5 4955350 -32466 32469 4989.02 498902 -124 127 2.14 214 +104 101 10094 99005 0.31231 297.31231 148.81231 14881.23123 0.31231 297.31232 148.81231 14881.23144 0.31231 297.31231 148.81230999999997 14881.23100 2020-01-01 2020-01-02 2020-01-01 00:01:44 2020-01-02 03:30:05 2020-01-01 00:01:44.000 2020-01-02 03:30:05.000 104 99005 49554.5 4955450 104 99005 49554.5 4955450 -32465 32470 4990.02 499002 -128 127 0.58 58 +105 101 10095 99006 0.31531 297.31531 148.81531 14881.53153 0.31531 297.3153 148.81531 14881.53174 0.31531 297.31531 148.81530999999998 14881.53100 2020-01-01 2020-01-02 2020-01-01 00:01:45 2020-01-02 03:30:06 2020-01-01 00:01:45.000 2020-01-02 03:30:06.000 105 99006 49555.5 4955550 105 99006 49555.5 4955550 -32464 32471 4991.02 499102 -128 123 -0.98 -98 +106 101 10096 99007 0.31831 297.31831 148.81831 14881.83183 0.31831 297.31833 148.81831 14881.83182 0.31831 297.31831 148.81831000000003 14881.83100 2020-01-01 2020-01-02 2020-01-01 00:01:46 2020-01-02 03:30:07 2020-01-01 00:01:46.000 2020-01-02 03:30:07.000 106 99007 49556.5 4955650 106 99007 49556.5 4955650 -32463 32472 4992.02 499202 -127 124 0.02 2 +107 101 10097 99008 0.32132 297.32132 148.82132 14882.13213 0.32132 297.32132 148.82131 14882.13197 0.32132 297.32132 148.8213199999999 14882.13200 2020-01-01 2020-01-02 2020-01-01 00:01:47 2020-01-02 03:30:08 2020-01-01 00:01:47.000 2020-01-02 03:30:08.000 107 99008 49557.5 4955750 107 99008 49557.5 4955750 -32462 32473 4993.02 499302 -126 125 1.02 102 +108 101 10098 99009 0.32432 297.32432 148.82432 14882.43243 0.32432 297.3243 148.82432 14882.43232 0.32432 297.32432 148.82431999999994 14882.43200 2020-01-01 2020-01-02 2020-01-01 00:01:48 2020-01-02 03:30:09 2020-01-01 00:01:48.000 2020-01-02 03:30:09.000 108 99009 49558.5 4955850 108 99009 49558.5 4955850 -32461 32474 4994.02 499402 -125 126 2.02 202 +109 101 10099 99010 0.32732 297.32732 148.82732 14882.73273 0.32732 297.32733 148.82732 14882.7329 0.32732 297.32732 148.8273200000001 14882.73200 2020-01-01 2020-01-02 2020-01-01 00:01:49 2020-01-02 03:30:10 2020-01-01 00:01:49.000 2020-01-02 03:30:10.000 109 99010 49559.5 4955950 109 99010 49559.5 4955950 -32460 32475 4995.02 499502 -124 127 3.02 302 11 102 10001 99911 0.03303 300.03303 150.03303 15153.33633 0.03303 300.03302 150.03303 15153.33627 0.03303 300.03303 150.03303 15153.33603 2020-01-01 2020-01-02 2020-01-01 00:00:11 2020-01-02 03:45:11 2020-01-01 00:00:11.000 2020-01-02 03:45:11.000 11 99911 49961 5046061 11 99911 49961 5046061 -32558 32377 4540.009900990099 458541 -128 123 -2.089108910891089 -211 110 101 10100 99011 0.33033 297.33033 148.83033 14883.03303 0.33033 297.33032 148.83033 14883.03321 0.33033 297.33033 148.83033 14883.03300 2020-01-01 2020-01-02 2020-01-01 00:01:50 2020-01-02 03:30:11 2020-01-01 00:01:50.000 2020-01-02 03:30:11.000 110 99011 49560.5 4956050 110 99011 49560.5 4956050 -32459 32476 4996.02 499602 -128 127 1.46 146 111 101 10101 99012 0.33333 297.33333 148.83333 14883.33333 0.33333 297.33334 148.83333 14883.33329 0.33333 297.33333 148.83333 14883.33300 2020-01-01 2020-01-02 2020-01-01 00:01:51 2020-01-02 03:30:12 2020-01-01 00:01:51.000 2020-01-02 03:30:12.000 111 99012 49561.5 4956150 111 99012 49561.5 4956150 -32458 32477 4997.02 499702 -128 123 -0.1 -10 -112 101 10102 99013 0.33633 297.33633 148.83633 14883.63363 0.33633 297.33633 148.83633 14883.63348 0.33633 297.33633 148.83633 14883.63300 2020-01-01 2020-01-02 2020-01-01 00:01:52 2020-01-02 03:30:13 2020-01-01 00:01:52.000 2020-01-02 03:30:13.000 112 99013 49562.5 4956250 112 99013 49562.5 4956250 -32457 32478 4998.02 499802 -127 124 0.9 90 -113 101 10103 99014 0.33933 297.33933 148.83933 14883.93393 0.33933 297.33932 148.83933 14883.9338 0.33933 297.33933 148.83933 14883.93300 2020-01-01 2020-01-02 2020-01-01 00:01:53 2020-01-02 03:30:14 2020-01-01 00:01:53.000 2020-01-02 03:30:14.000 113 99014 49563.5 4956350 113 99014 49563.5 4956350 -32456 32479 4999.02 499902 -126 125 1.9 190 -114 101 10104 99015 0.34234 297.34234 148.84234 14884.23423 0.34234 297.34235 148.84234 14884.23437 0.34234 297.34234 148.84234 14884.23400 2020-01-01 2020-01-02 2020-01-01 00:01:54 2020-01-02 03:30:15 2020-01-01 00:01:54.000 2020-01-02 03:30:15.000 114 99015 49564.5 4956450 114 99015 49564.5 4956450 -32455 32480 5000.02 500002 -125 126 2.9 290 -115 101 10105 99016 0.34534 297.34534 148.84534 14884.53453 0.34534 297.34534 148.84534 14884.53468 0.34534 297.34534 148.84534 14884.53400 2020-01-01 2020-01-02 2020-01-01 00:01:55 2020-01-02 03:30:16 2020-01-01 00:01:55.000 2020-01-02 03:30:16.000 115 99016 49565.5 4956550 115 99016 49565.5 4956550 -32454 32481 5001.02 500102 -124 127 3.9 390 -116 101 10106 99017 0.34834 297.34834 148.84834 14884.83483 0.34834 297.34836 148.84834 14884.83476 0.34834 297.34834 148.84834 14884.83400 2020-01-01 2020-01-02 2020-01-01 00:01:56 2020-01-02 03:30:17 2020-01-01 00:01:56.000 2020-01-02 03:30:17.000 116 99017 49566.5 4956650 116 99017 49566.5 4956650 -32453 32482 5002.02 500202 -128 127 2.34 234 -117 101 10107 99018 0.35135 297.35135 148.85135 14885.13513 0.35135 297.35135 148.85134 14885.13495 0.35135 297.35135 148.85135 14885.13500 2020-01-01 2020-01-02 2020-01-01 00:01:57 2020-01-02 03:30:18 2020-01-01 00:01:57.000 2020-01-02 03:30:18.000 117 99018 49567.5 4956750 117 99018 49567.5 4956750 -32452 32483 5003.02 500302 -128 123 0.78 78 -118 101 10108 99019 0.35435 297.35435 148.85435 14885.43543 0.35435 297.35434 148.85435 14885.43526 0.35435 297.35435 148.85435 14885.43500 2020-01-01 2020-01-02 2020-01-01 00:01:58 2020-01-02 03:30:19 2020-01-01 00:01:58.000 2020-01-02 03:30:19.000 118 99019 49568.5 4956850 118 99019 49568.5 4956850 -32451 32484 5004.02 500402 -127 124 1.78 178 -119 101 10109 99020 0.35735 297.35735 148.85735 14885.73573 0.35735 297.35736 148.85736 14885.736 0.35735 297.35735 148.85735 14885.73500 2020-01-01 2020-01-02 2020-01-01 00:01:59 2020-01-02 03:30:20 2020-01-01 00:01:59.000 2020-01-02 03:30:20.000 119 99020 49569.5 4956950 119 99020 49569.5 4956950 -32450 32485 5005.02 500502 -126 125 2.78 278 +112 101 10102 99013 0.33633 297.33633 148.83633 14883.63363 0.33633 297.33633 148.83633 14883.63348 0.33633 297.33633 148.83633000000006 14883.63300 2020-01-01 2020-01-02 2020-01-01 00:01:52 2020-01-02 03:30:13 2020-01-01 00:01:52.000 2020-01-02 03:30:13.000 112 99013 49562.5 4956250 112 99013 49562.5 4956250 -32457 32478 4998.02 499802 -127 124 0.9 90 +113 101 10103 99014 0.33933 297.33933 148.83933 14883.93393 0.33933 297.33932 148.83933 14883.9338 0.33933 297.33933 148.83933000000002 14883.93300 2020-01-01 2020-01-02 2020-01-01 00:01:53 2020-01-02 03:30:14 2020-01-01 00:01:53.000 2020-01-02 03:30:14.000 113 99014 49563.5 4956350 113 99014 49563.5 4956350 -32456 32479 4999.02 499902 -126 125 1.9 190 +114 101 10104 99015 0.34234 297.34234 148.84234 14884.23423 0.34234 297.34235 148.84234 14884.23437 0.34234 297.34234 148.84233999999998 14884.23400 2020-01-01 2020-01-02 2020-01-01 00:01:54 2020-01-02 03:30:15 2020-01-01 00:01:54.000 2020-01-02 03:30:15.000 114 99015 49564.5 4956450 114 99015 49564.5 4956450 -32455 32480 5000.02 500002 -125 126 2.9 290 +115 101 10105 99016 0.34534 297.34534 148.84534 14884.53453 0.34534 297.34534 148.84534 14884.53468 0.34534 297.34534 148.84533999999994 14884.53400 2020-01-01 2020-01-02 2020-01-01 00:01:55 2020-01-02 03:30:16 2020-01-01 00:01:55.000 2020-01-02 03:30:16.000 115 99016 49565.5 4956550 115 99016 49565.5 4956550 -32454 32481 5001.02 500102 -124 127 3.9 390 +116 101 10106 99017 0.34834 297.34834 148.84834 14884.83483 0.34834 297.34836 148.84834 14884.83476 0.34834 297.34834 148.84834000000015 14884.83400 2020-01-01 2020-01-02 2020-01-01 00:01:56 2020-01-02 03:30:17 2020-01-01 00:01:56.000 2020-01-02 03:30:17.000 116 99017 49566.5 4956650 116 99017 49566.5 4956650 -32453 32482 5002.02 500202 -128 127 2.34 234 +117 101 10107 99018 0.35135 297.35135 148.85135 14885.13513 0.35135 297.35135 148.85134 14885.13495 0.35135 297.35135 148.85135000000005 14885.13500 2020-01-01 2020-01-02 2020-01-01 00:01:57 2020-01-02 03:30:18 2020-01-01 00:01:57.000 2020-01-02 03:30:18.000 117 99018 49567.5 4956750 117 99018 49567.5 4956750 -32452 32483 5003.02 500302 -128 123 0.78 78 +118 101 10108 99019 0.35435 297.35435 148.85435 14885.43543 0.35435 297.35434 148.85435 14885.43526 0.35435 297.35435 148.8543499999999 14885.43500 2020-01-01 2020-01-02 2020-01-01 00:01:58 2020-01-02 03:30:19 2020-01-01 00:01:58.000 2020-01-02 03:30:19.000 118 99019 49568.5 4956850 118 99019 49568.5 4956850 -32451 32484 5004.02 500402 -127 124 1.78 178 +119 101 10109 99020 0.35735 297.35735 148.85735 14885.73573 0.35735 297.35736 148.85736 14885.736 0.35735 297.35735 148.8573500000001 14885.73500 2020-01-01 2020-01-02 2020-01-01 00:01:59 2020-01-02 03:30:20 2020-01-01 00:01:59.000 2020-01-02 03:30:20.000 119 99020 49569.5 4956950 119 99020 49569.5 4956950 -32450 32485 5005.02 500502 -126 125 2.78 278 12 102 10002 99912 0.03603 300.03603 150.03603 15153.63963 0.03603 300.03604 150.03603 15153.6399 0.03603 300.03603 150.03603 15153.63903 2020-01-01 2020-01-02 2020-01-01 00:00:12 2020-01-02 03:45:12 2020-01-01 00:00:12.000 2020-01-02 03:45:12.000 12 99912 49962 5046162 12 99912 49962 5046162 -32557 32378 4541.009900990099 458642 -127 124 -1.0891089108910892 -110 -120 101 10110 99021 0.36036 297.36036 148.86036 14886.03603 0.36036 297.36035 148.86036 14886.03615 0.36036 297.36036 148.86036 14886.03600 2020-01-01 2020-01-02 2020-01-01 00:02:00 2020-01-02 03:30:21 2020-01-01 00:02:00.000 2020-01-02 03:30:21.000 120 99021 49570.5 4957050 120 99021 49570.5 4957050 -32449 32486 5006.02 500602 -125 126 3.78 378 -121 101 10111 99022 0.36336 297.36336 148.86336 14886.33633 0.36336 297.36337 148.86336 14886.33627 0.36336 297.36336 148.86336 14886.33600 2020-01-01 2020-01-02 2020-01-01 00:02:01 2020-01-02 03:30:22 2020-01-01 00:02:01.000 2020-01-02 03:30:22.000 121 99022 49571.5 4957150 121 99022 49571.5 4957150 -32448 32487 5007.02 500702 -124 127 4.78 478 +120 101 10110 99021 0.36036 297.36036 148.86036 14886.03603 0.36036 297.36035 148.86036 14886.03615 0.36036 297.36036 148.86036000000004 14886.03600 2020-01-01 2020-01-02 2020-01-01 00:02:00 2020-01-02 03:30:21 2020-01-01 00:02:00.000 2020-01-02 03:30:21.000 120 99021 49570.5 4957050 120 99021 49570.5 4957050 -32449 32486 5006.02 500602 -125 126 3.78 378 +121 101 10111 99022 0.36336 297.36336 148.86336 14886.33633 0.36336 297.36337 148.86336 14886.33627 0.36336 297.36336 148.86335999999994 14886.33600 2020-01-01 2020-01-02 2020-01-01 00:02:01 2020-01-02 03:30:22 2020-01-01 00:02:01.000 2020-01-02 03:30:22.000 121 99022 49571.5 4957150 121 99022 49571.5 4957150 -32448 32487 5007.02 500702 -124 127 4.78 478 122 101 10112 99023 0.36636 297.36636 148.86636 14886.63663 0.36636 297.36636 148.86636 14886.63642 0.36636 297.36636 148.86636 14886.63600 2020-01-01 2020-01-02 2020-01-01 00:02:02 2020-01-02 03:30:23 2020-01-01 00:02:02.000 2020-01-02 03:30:23.000 122 99023 49572.5 4957250 122 99023 49572.5 4957250 -32447 32488 5008.02 500802 -128 127 3.22 322 -123 101 10113 99024 0.36936 297.36936 148.86936 14886.93693 0.36936 297.36935 148.86936 14886.93673 0.36936 297.36936 148.86936 14886.93600 2020-01-01 2020-01-02 2020-01-01 00:02:03 2020-01-02 03:30:24 2020-01-01 00:02:03.000 2020-01-02 03:30:24.000 123 99024 49573.5 4957350 123 99024 49573.5 4957350 -32446 32489 5009.02 500902 -128 127 1.66 166 -124 101 10114 99025 0.37237 297.37237 148.87237 14887.23723 0.37237 297.37238 148.87237 14887.23746 0.37237 297.37237 148.87237 14887.23700 2020-01-01 2020-01-02 2020-01-01 00:02:04 2020-01-02 03:30:25 2020-01-01 00:02:04.000 2020-01-02 03:30:25.000 124 99025 49574.5 4957450 124 99025 49574.5 4957450 -32445 32490 5010.02 501002 -128 124 0.1 10 -125 101 10115 99026 0.37537 297.37537 148.87537 14887.53753 0.37537 297.37537 148.87537 14887.53762 0.37537 297.37537 148.87537 14887.53700 2020-01-01 2020-01-02 2020-01-01 00:02:05 2020-01-02 03:30:26 2020-01-01 00:02:05.000 2020-01-02 03:30:26.000 125 99026 49575.5 4957550 125 99026 49575.5 4957550 -32444 32491 5011.02 501102 -127 125 1.1 110 -126 101 10116 99027 0.37837 297.37837 148.87837 14887.83783 0.37837 297.3784 148.87837 14887.83774 0.37837 297.37837 148.87837 14887.83700 2020-01-01 2020-01-02 2020-01-01 00:02:06 2020-01-02 03:30:27 2020-01-01 00:02:06.000 2020-01-02 03:30:27.000 126 99027 49576.5 4957650 126 99027 49576.5 4957650 -32443 32492 5012.02 501202 -126 126 2.1 210 +123 101 10113 99024 0.36936 297.36936 148.86936 14886.93693 0.36936 297.36935 148.86936 14886.93673 0.36936 297.36936 148.86936000000003 14886.93600 2020-01-01 2020-01-02 2020-01-01 00:02:03 2020-01-02 03:30:24 2020-01-01 00:02:03.000 2020-01-02 03:30:24.000 123 99024 49573.5 4957350 123 99024 49573.5 4957350 -32446 32489 5009.02 500902 -128 127 1.66 166 +124 101 10114 99025 0.37237 297.37237 148.87237 14887.23723 0.37237 297.37238 148.87237 14887.23746 0.37237 297.37237 148.87236999999993 14887.23700 2020-01-01 2020-01-02 2020-01-01 00:02:04 2020-01-02 03:30:25 2020-01-01 00:02:04.000 2020-01-02 03:30:25.000 124 99025 49574.5 4957450 124 99025 49574.5 4957450 -32445 32490 5010.02 501002 -128 124 0.1 10 +125 101 10115 99026 0.37537 297.37537 148.87537 14887.53753 0.37537 297.37537 148.87537 14887.53762 0.37537 297.37537 148.87536999999992 14887.53700 2020-01-01 2020-01-02 2020-01-01 00:02:05 2020-01-02 03:30:26 2020-01-01 00:02:05.000 2020-01-02 03:30:26.000 125 99026 49575.5 4957550 125 99026 49575.5 4957550 -32444 32491 5011.02 501102 -127 125 1.1 110 +126 101 10116 99027 0.37837 297.37837 148.87837 14887.83783 0.37837 297.3784 148.87837 14887.83774 0.37837 297.37837 148.87837000000007 14887.83700 2020-01-01 2020-01-02 2020-01-01 00:02:06 2020-01-02 03:30:27 2020-01-01 00:02:06.000 2020-01-02 03:30:27.000 126 99027 49576.5 4957650 126 99027 49576.5 4957650 -32443 32492 5012.02 501202 -126 126 2.1 210 127 101 10117 99028 0.38138 297.38138 148.88138 14888.13813 0.38138 297.38138 148.88137 14888.13789 0.38138 297.38138 148.88138 14888.13800 2020-01-01 2020-01-02 2020-01-01 00:02:07 2020-01-02 03:30:28 2020-01-01 00:02:07.000 2020-01-02 03:30:28.000 127 99028 49577.5 4957750 127 99028 49577.5 4957750 -32442 32493 5013.02 501302 -125 127 3.1 310 -128 101 10118 99029 0.38438 297.38438 148.88438 14888.43843 0.38438 297.3844 148.88438 14888.43862 0.38438 297.38438 148.88438 14888.43800 2020-01-01 2020-01-02 2020-01-01 00:02:08 2020-01-02 03:30:29 2020-01-01 00:02:08.000 2020-01-02 03:30:29.000 128 99029 49578.5 4957850 128 99029 49578.5 4957850 -32441 32494 5014.02 501402 -128 127 1.54 154 -129 101 10119 99030 0.38738 297.38738 148.88738 14888.73873 0.38738 297.3874 148.88738 14888.73894 0.38738 297.38738 148.88738 14888.73800 2020-01-01 2020-01-02 2020-01-01 00:02:09 2020-01-02 03:30:30 2020-01-01 00:02:09.000 2020-01-02 03:30:30.000 129 99030 49579.5 4957950 129 99030 49579.5 4957950 -32440 32495 5015.02 501502 -128 127 -0.02 -2 -13 102 10003 99913 0.03903 300.03903 150.03903 15153.94294 0.03903 300.03903 150.03903 15153.94255 0.03903 300.03903 150.03903 15153.94203 2020-01-01 2020-01-02 2020-01-01 00:00:13 2020-01-02 03:45:13 2020-01-01 00:00:13.000 2020-01-02 03:45:13.000 13 99913 49963 5046263 13 99913 49963 5046263 -32556 32379 4542.009900990099 458743 -126 125 -0.0891089108910891 -9 +128 101 10118 99029 0.38438 297.38438 148.88438 14888.43843 0.38438 297.3844 148.88438 14888.43862 0.38438 297.38438 148.88438000000002 14888.43800 2020-01-01 2020-01-02 2020-01-01 00:02:08 2020-01-02 03:30:29 2020-01-01 00:02:08.000 2020-01-02 03:30:29.000 128 99029 49578.5 4957850 128 99029 49578.5 4957850 -32441 32494 5014.02 501402 -128 127 1.54 154 +129 101 10119 99030 0.38738 297.38738 148.88738 14888.73873 0.38738 297.3874 148.88738 14888.73894 0.38738 297.38738 148.88738000000006 14888.73800 2020-01-01 2020-01-02 2020-01-01 00:02:09 2020-01-02 03:30:30 2020-01-01 00:02:09.000 2020-01-02 03:30:30.000 129 99030 49579.5 4957950 129 99030 49579.5 4957950 -32440 32495 5015.02 501502 -128 127 -0.02 -2 +13 102 10003 99913 0.03903 300.03903 150.03903 15153.94294 0.03903 300.03903 150.03903 15153.94255 0.03903 300.03903 150.03902999999994 15153.94203 2020-01-01 2020-01-02 2020-01-01 00:00:13 2020-01-02 03:45:13 2020-01-01 00:00:13.000 2020-01-02 03:45:13.000 13 99913 49963 5046263 13 99913 49963 5046263 -32556 32379 4542.009900990099 458743 -126 125 -0.0891089108910891 -9 130 101 10120 99031 0.39039 297.39039 148.89039 14889.03903 0.39039 297.39038 148.89039 14889.03909 0.39039 297.39039 148.89039 14889.03900 2020-01-01 2020-01-02 2020-01-01 00:02:10 2020-01-02 03:30:31 2020-01-01 00:02:10.000 2020-01-02 03:30:31.000 130 99031 49580.5 4958050 130 99031 49580.5 4958050 -32439 32496 5016.02 501602 -128 123 -1.58 -158 -131 101 10121 99032 0.39339 297.39339 148.89339 14889.33933 0.39339 297.3934 148.89339 14889.33921 0.39339 297.39339 148.89339 14889.33900 2020-01-01 2020-01-02 2020-01-01 00:02:11 2020-01-02 03:30:32 2020-01-01 00:02:11.000 2020-01-02 03:30:32.000 131 99032 49581.5 4958150 131 99032 49581.5 4958150 -32438 32497 5017.02 501702 -127 124 -0.58 -58 -132 101 10122 99033 0.39639 297.39639 148.89639 14889.63963 0.39639 297.3964 148.89639 14889.63936 0.39639 297.39639 148.89639 14889.63900 2020-01-01 2020-01-02 2020-01-01 00:02:12 2020-01-02 03:30:33 2020-01-01 00:02:12.000 2020-01-02 03:30:33.000 132 99033 49582.5 4958250 132 99033 49582.5 4958250 -32437 32498 5018.02 501802 -126 125 0.42 42 -133 101 10123 99034 0.39939 297.39939 148.89939 14889.93993 0.39939 297.3994 148.8994 14889.94009 0.39939 297.39939 148.89939 14889.93900 2020-01-01 2020-01-02 2020-01-01 00:02:13 2020-01-02 03:30:34 2020-01-01 00:02:13.000 2020-01-02 03:30:34.000 133 99034 49583.5 4958350 133 99034 49583.5 4958350 -32436 32499 5019.02 501902 -125 126 1.42 142 -134 101 10124 99035 0.4024 297.4024 148.9024 14890.24024 0.4024 297.4024 148.9024 14890.24041 0.40240 297.40240 148.90240 14890.24000 2020-01-01 2020-01-02 2020-01-01 00:02:14 2020-01-02 03:30:35 2020-01-01 00:02:14.000 2020-01-02 03:30:35.000 134 99035 49584.5 4958450 134 99035 49584.5 4958450 -32435 32500 5020.02 502002 -124 127 2.42 242 -135 101 10125 99036 0.4054 297.4054 148.9054 14890.54054 0.4054 297.4054 148.9054 14890.54059 0.40540 297.40540 148.90540 14890.54000 2020-01-01 2020-01-02 2020-01-01 00:02:15 2020-01-02 03:30:36 2020-01-01 00:02:15.000 2020-01-02 03:30:36.000 135 99036 49585.5 4958550 135 99036 49585.5 4958550 -32434 32501 5021.02 502102 -128 127 0.86 86 -136 101 10126 99037 0.4084 297.4084 148.9084 14890.84084 0.4084 297.40842 148.9084 14890.84068 0.40840 297.40840 148.90840 14890.84000 2020-01-01 2020-01-02 2020-01-01 00:02:16 2020-01-02 03:30:37 2020-01-01 00:02:16.000 2020-01-02 03:30:37.000 136 99037 49586.5 4958650 136 99037 49586.5 4958650 -32433 32502 5022.02 502202 -128 123 -0.7 -70 -137 101 10127 99038 0.41141 297.41141 148.91141 14891.14114 0.41141 297.4114 148.9114 14891.14099 0.41141 297.41141 148.91141 14891.14100 2020-01-01 2020-01-02 2020-01-01 00:02:17 2020-01-02 03:30:38 2020-01-01 00:02:17.000 2020-01-02 03:30:38.000 137 99038 49587.5 4958750 137 99038 49587.5 4958750 -32432 32503 5023.02 502302 -127 124 0.3 30 -138 101 10128 99039 0.41441 297.41441 148.91441 14891.44144 0.41441 297.41443 148.91441 14891.44157 0.41441 297.41441 148.91441 14891.44100 2020-01-01 2020-01-02 2020-01-01 00:02:18 2020-01-02 03:30:39 2020-01-01 00:02:18.000 2020-01-02 03:30:39.000 138 99039 49588.5 4958850 138 99039 49588.5 4958850 -32431 32504 5024.02 502402 -126 125 1.3 130 +131 101 10121 99032 0.39339 297.39339 148.89339 14889.33933 0.39339 297.3934 148.89339 14889.33921 0.39339 297.39339 148.89338999999998 14889.33900 2020-01-01 2020-01-02 2020-01-01 00:02:11 2020-01-02 03:30:32 2020-01-01 00:02:11.000 2020-01-02 03:30:32.000 131 99032 49581.5 4958150 131 99032 49581.5 4958150 -32438 32497 5017.02 501702 -127 124 -0.58 -58 +132 101 10122 99033 0.39639 297.39639 148.89639 14889.63963 0.39639 297.3964 148.89639 14889.63936 0.39639 297.39639 148.8963899999999 14889.63900 2020-01-01 2020-01-02 2020-01-01 00:02:12 2020-01-02 03:30:33 2020-01-01 00:02:12.000 2020-01-02 03:30:33.000 132 99033 49582.5 4958250 132 99033 49582.5 4958250 -32437 32498 5018.02 501802 -126 125 0.42 42 +133 101 10123 99034 0.39939 297.39939 148.89939 14889.93993 0.39939 297.3994 148.8994 14889.94009 0.39939 297.39939 148.89938999999998 14889.93900 2020-01-01 2020-01-02 2020-01-01 00:02:13 2020-01-02 03:30:34 2020-01-01 00:02:13.000 2020-01-02 03:30:34.000 133 99034 49583.5 4958350 133 99034 49583.5 4958350 -32436 32499 5019.02 501902 -125 126 1.42 142 +134 101 10124 99035 0.4024 297.4024 148.9024 14890.24024 0.4024 297.4024 148.9024 14890.24041 0.40240 297.40240 148.90240000000009 14890.24000 2020-01-01 2020-01-02 2020-01-01 00:02:14 2020-01-02 03:30:35 2020-01-01 00:02:14.000 2020-01-02 03:30:35.000 134 99035 49584.5 4958450 134 99035 49584.5 4958450 -32435 32500 5020.02 502002 -124 127 2.42 242 +135 101 10125 99036 0.4054 297.4054 148.9054 14890.54054 0.4054 297.4054 148.9054 14890.54059 0.40540 297.40540 148.90539999999993 14890.54000 2020-01-01 2020-01-02 2020-01-01 00:02:15 2020-01-02 03:30:36 2020-01-01 00:02:15.000 2020-01-02 03:30:36.000 135 99036 49585.5 4958550 135 99036 49585.5 4958550 -32434 32501 5021.02 502102 -128 127 0.86 86 +136 101 10126 99037 0.4084 297.4084 148.9084 14890.84084 0.4084 297.40842 148.9084 14890.84068 0.40840 297.40840 148.90840000000003 14890.84000 2020-01-01 2020-01-02 2020-01-01 00:02:16 2020-01-02 03:30:37 2020-01-01 00:02:16.000 2020-01-02 03:30:37.000 136 99037 49586.5 4958650 136 99037 49586.5 4958650 -32433 32502 5022.02 502202 -128 123 -0.7 -70 +137 101 10127 99038 0.41141 297.41141 148.91141 14891.14114 0.41141 297.4114 148.9114 14891.14099 0.41141 297.41141 148.91141000000005 14891.14100 2020-01-01 2020-01-02 2020-01-01 00:02:17 2020-01-02 03:30:38 2020-01-01 00:02:17.000 2020-01-02 03:30:38.000 137 99038 49587.5 4958750 137 99038 49587.5 4958750 -32432 32503 5023.02 502302 -127 124 0.3 30 +138 101 10128 99039 0.41441 297.41441 148.91441 14891.44144 0.41441 297.41443 148.91441 14891.44157 0.41441 297.41441 148.91440999999995 14891.44100 2020-01-01 2020-01-02 2020-01-01 00:02:18 2020-01-02 03:30:39 2020-01-01 00:02:18.000 2020-01-02 03:30:39.000 138 99039 49588.5 4958850 138 99039 49588.5 4958850 -32431 32504 5024.02 502402 -126 125 1.3 130 139 101 10129 99040 0.41741 297.41741 148.91741 14891.74174 0.41741 297.41742 148.91741 14891.74188 0.41741 297.41741 148.91741 14891.74100 2020-01-01 2020-01-02 2020-01-01 00:02:19 2020-01-02 03:30:40 2020-01-01 00:02:19.000 2020-01-02 03:30:40.000 139 99040 49589.5 4958950 139 99040 49589.5 4958950 -32430 32505 5025.02 502502 -125 126 2.3 230 -14 102 10004 99914 0.04204 300.04204 150.04204 15154.24624 0.04204 300.04205 150.04204 15154.2463 0.04204 300.04204 150.04204 15154.24604 2020-01-01 2020-01-02 2020-01-01 00:00:14 2020-01-02 03:45:14 2020-01-01 00:00:14.000 2020-01-02 03:45:14.000 14 99914 49964 5046364 14 99914 49964 5046364 -32555 32380 4543.009900990099 458844 -125 126 0.9108910891089109 92 -140 101 10130 99041 0.42042 297.42042 148.92042 14892.04204 0.42042 297.4204 148.92042 14892.04206 0.42042 297.42042 148.92042 14892.04200 2020-01-01 2020-01-02 2020-01-01 00:02:20 2020-01-02 03:30:41 2020-01-01 00:02:20.000 2020-01-02 03:30:41.000 140 99041 49590.5 4959050 140 99041 49590.5 4959050 -32429 32506 5026.02 502602 -124 127 3.3 330 -141 101 10131 99042 0.42342 297.42342 148.92342 14892.34234 0.42342 297.42343 148.92342 14892.34215 0.42342 297.42342 148.92342 14892.34200 2020-01-01 2020-01-02 2020-01-01 00:02:21 2020-01-02 03:30:42 2020-01-01 00:02:21.000 2020-01-02 03:30:42.000 141 99042 49591.5 4959150 141 99042 49591.5 4959150 -32428 32507 5027.02 502702 -128 127 1.74 174 -142 101 10132 99043 0.42642 297.42642 148.92642 14892.64264 0.42642 297.42642 148.92642 14892.64246 0.42642 297.42642 148.92642 14892.64200 2020-01-01 2020-01-02 2020-01-01 00:02:22 2020-01-02 03:30:43 2020-01-01 00:02:22.000 2020-01-02 03:30:43.000 142 99043 49592.5 4959250 142 99043 49592.5 4959250 -32427 32508 5028.02 502802 -128 123 0.18 18 -143 101 10133 99044 0.42942 297.42942 148.92942 14892.94294 0.42942 297.42944 148.92943 14892.94304 0.42942 297.42942 148.92942 14892.94200 2020-01-01 2020-01-02 2020-01-01 00:02:23 2020-01-02 03:30:44 2020-01-01 00:02:23.000 2020-01-02 03:30:44.000 143 99044 49593.5 4959350 143 99044 49593.5 4959350 -32426 32509 5029.02 502902 -127 124 1.18 118 +14 102 10004 99914 0.04204 300.04204 150.04204 15154.24624 0.04204 300.04205 150.04204 15154.2463 0.04204 300.04204 150.04204000000004 15154.24604 2020-01-01 2020-01-02 2020-01-01 00:00:14 2020-01-02 03:45:14 2020-01-01 00:00:14.000 2020-01-02 03:45:14.000 14 99914 49964 5046364 14 99914 49964 5046364 -32555 32380 4543.009900990099 458844 -125 126 0.9108910891089109 92 +140 101 10130 99041 0.42042 297.42042 148.92042 14892.04204 0.42042 297.4204 148.92042 14892.04206 0.42042 297.42042 148.9204200000001 14892.04200 2020-01-01 2020-01-02 2020-01-01 00:02:20 2020-01-02 03:30:41 2020-01-01 00:02:20.000 2020-01-02 03:30:41.000 140 99041 49590.5 4959050 140 99041 49590.5 4959050 -32429 32506 5026.02 502602 -124 127 3.3 330 +141 101 10131 99042 0.42342 297.42342 148.92342 14892.34234 0.42342 297.42343 148.92342 14892.34215 0.42342 297.42342 148.92341999999994 14892.34200 2020-01-01 2020-01-02 2020-01-01 00:02:21 2020-01-02 03:30:42 2020-01-01 00:02:21.000 2020-01-02 03:30:42.000 141 99042 49591.5 4959150 141 99042 49591.5 4959150 -32428 32507 5027.02 502702 -128 127 1.74 174 +142 101 10132 99043 0.42642 297.42642 148.92642 14892.64264 0.42642 297.42642 148.92642 14892.64246 0.42642 297.42642 148.9264199999999 14892.64200 2020-01-01 2020-01-02 2020-01-01 00:02:22 2020-01-02 03:30:43 2020-01-01 00:02:22.000 2020-01-02 03:30:43.000 142 99043 49592.5 4959250 142 99043 49592.5 4959250 -32427 32508 5028.02 502802 -128 123 0.18 18 +143 101 10133 99044 0.42942 297.42942 148.92942 14892.94294 0.42942 297.42944 148.92943 14892.94304 0.42942 297.42942 148.9294200000001 14892.94200 2020-01-01 2020-01-02 2020-01-01 00:02:23 2020-01-02 03:30:44 2020-01-01 00:02:23.000 2020-01-02 03:30:44.000 143 99044 49593.5 4959350 143 99044 49593.5 4959350 -32426 32509 5029.02 502902 -127 124 1.18 118 144 101 10134 99045 0.43243 297.43243 148.93243 14893.24324 0.43243 297.43243 148.93243 14893.24338 0.43243 297.43243 148.93243 14893.24300 2020-01-01 2020-01-02 2020-01-01 00:02:24 2020-01-02 03:30:45 2020-01-01 00:02:24.000 2020-01-02 03:30:45.000 144 99045 49594.5 4959450 144 99045 49594.5 4959450 -32425 32510 5030.02 503002 -126 125 2.18 218 -145 101 10135 99046 0.43543 297.43543 148.93543 14893.54354 0.43543 297.43542 148.93543 14893.54354 0.43543 297.43543 148.93543 14893.54300 2020-01-01 2020-01-02 2020-01-01 00:02:25 2020-01-02 03:30:46 2020-01-01 00:02:25.000 2020-01-02 03:30:46.000 145 99046 49595.5 4959550 145 99046 49595.5 4959550 -32424 32511 5031.02 503102 -125 126 3.18 318 -146 101 10136 99047 0.43843 297.43843 148.93843 14893.84384 0.43843 297.43845 148.93844 14893.84427 0.43843 297.43843 148.93843 14893.84300 2020-01-01 2020-01-02 2020-01-01 00:02:26 2020-01-02 03:30:47 2020-01-01 00:02:26.000 2020-01-02 03:30:47.000 146 99047 49596.5 4959650 146 99047 49596.5 4959650 -32423 32512 5032.02 503202 -124 127 4.18 418 +145 101 10135 99046 0.43543 297.43543 148.93543 14893.54354 0.43543 297.43542 148.93543 14893.54354 0.43543 297.43543 148.93542999999983 14893.54300 2020-01-01 2020-01-02 2020-01-01 00:02:25 2020-01-02 03:30:46 2020-01-01 00:02:25.000 2020-01-02 03:30:46.000 145 99046 49595.5 4959550 145 99046 49595.5 4959550 -32424 32511 5031.02 503102 -125 126 3.18 318 +146 101 10136 99047 0.43843 297.43843 148.93843 14893.84384 0.43843 297.43845 148.93844 14893.84427 0.43843 297.43843 148.93843000000007 14893.84300 2020-01-01 2020-01-02 2020-01-01 00:02:26 2020-01-02 03:30:47 2020-01-01 00:02:26.000 2020-01-02 03:30:47.000 146 99047 49596.5 4959650 146 99047 49596.5 4959650 -32423 32512 5032.02 503202 -124 127 4.18 418 147 101 10137 99048 0.44144 297.44144 148.94144 14894.14414 0.44144 297.44144 148.94143 14894.14392 0.44144 297.44144 148.94144 14894.14400 2020-01-01 2020-01-02 2020-01-01 00:02:27 2020-01-02 03:30:48 2020-01-01 00:02:27.000 2020-01-02 03:30:48.000 147 99048 49597.5 4959750 147 99048 49597.5 4959750 -32422 32513 5033.02 503302 -128 127 2.62 262 148 101 10138 99049 0.44444 297.44444 148.94444 14894.44444 0.44444 297.44446 148.94444 14894.4445 0.44444 297.44444 148.94444 14894.44400 2020-01-01 2020-01-02 2020-01-01 00:02:28 2020-01-02 03:30:49 2020-01-01 00:02:28.000 2020-01-02 03:30:49.000 148 99049 49598.5 4959850 148 99049 49598.5 4959850 -32421 32514 5034.02 503402 -128 127 1.06 106 -149 101 10139 99050 0.44744 297.44744 148.94744 14894.74474 0.44744 297.44745 148.94744 14894.74485 0.44744 297.44744 148.94744 14894.74400 2020-01-01 2020-01-02 2020-01-01 00:02:29 2020-01-02 03:30:50 2020-01-01 00:02:29.000 2020-01-02 03:30:50.000 149 99050 49599.5 4959950 149 99050 49599.5 4959950 -32420 32515 5035.02 503502 -128 124 -0.5 -50 -15 102 10005 99915 0.04504 300.04504 150.04504 15154.54954 0.04504 300.04504 150.04504 15154.54945 0.04504 300.04504 150.04504 15154.54904 2020-01-01 2020-01-02 2020-01-01 00:00:15 2020-01-02 03:45:15 2020-01-01 00:00:15.000 2020-01-02 03:45:15.000 15 99915 49965 5046465 15 99915 49965 5046465 -32554 32381 4544.009900990099 458945 -124 127 1.9108910891089108 193 -150 101 10140 99051 0.45045 297.45045 148.95045 14895.04504 0.45045 297.45044 148.95045 14895.04501 0.45045 297.45045 148.95045 14895.04500 2020-01-01 2020-01-02 2020-01-01 00:02:30 2020-01-02 03:30:51 2020-01-01 00:02:30.000 2020-01-02 03:30:51.000 150 99051 49600.5 4960050 150 99051 49600.5 4960050 -32419 32516 5036.02 503602 -127 125 0.5 50 -151 101 10141 99052 0.45345 297.45345 148.95345 14895.34534 0.45345 297.45346 148.95345 14895.34574 0.45345 297.45345 148.95345 14895.34500 2020-01-01 2020-01-02 2020-01-01 00:02:31 2020-01-02 03:30:52 2020-01-01 00:02:31.000 2020-01-02 03:30:52.000 151 99052 49601.5 4960150 151 99052 49601.5 4960150 -32418 32517 5037.02 503702 -126 126 1.5 150 -152 101 10142 99053 0.45645 297.45645 148.95645 14895.64564 0.45645 297.45645 148.95645 14895.6454 0.45645 297.45645 148.95645 14895.64500 2020-01-01 2020-01-02 2020-01-01 00:02:32 2020-01-02 03:30:53 2020-01-01 00:02:32.000 2020-01-02 03:30:53.000 152 99053 49602.5 4960250 152 99053 49602.5 4960250 -32417 32518 5038.02 503802 -125 127 2.5 250 -153 101 10143 99054 0.45945 297.45945 148.95945 14895.94594 0.45945 297.45947 148.95946 14895.94601 0.45945 297.45945 148.95945 14895.94500 2020-01-01 2020-01-02 2020-01-01 00:02:33 2020-01-02 03:30:54 2020-01-01 00:02:33.000 2020-01-02 03:30:54.000 153 99054 49603.5 4960350 153 99054 49603.5 4960350 -32416 32519 5039.02 503902 -128 127 0.94 94 -154 101 10144 99055 0.46246 297.46246 148.96246 14896.24624 0.46246 297.46246 148.96246 14896.24633 0.46246 297.46246 148.96246 14896.24600 2020-01-01 2020-01-02 2020-01-01 00:02:34 2020-01-02 03:30:55 2020-01-01 00:02:34.000 2020-01-02 03:30:55.000 154 99055 49604.5 4960450 154 99055 49604.5 4960450 -32415 32520 5040.02 504002 -128 127 -0.62 -62 -155 101 10145 99056 0.46546 297.46546 148.96546 14896.54654 0.46546 297.46545 148.96546 14896.54647 0.46546 297.46546 148.96546 14896.54600 2020-01-01 2020-01-02 2020-01-01 00:02:35 2020-01-02 03:30:56 2020-01-01 00:02:35.000 2020-01-02 03:30:56.000 155 99056 49605.5 4960550 155 99056 49605.5 4960550 -32414 32521 5041.02 504102 -128 123 -2.18 -218 -156 101 10146 99057 0.46846 297.46846 148.96846 14896.84684 0.46846 297.46848 148.96847 14896.84721 0.46846 297.46846 148.96846 14896.84600 2020-01-01 2020-01-02 2020-01-01 00:02:36 2020-01-02 03:30:57 2020-01-01 00:02:36.000 2020-01-02 03:30:57.000 156 99057 49606.5 4960650 156 99057 49606.5 4960650 -32413 32522 5042.02 504202 -127 124 -1.18 -118 -157 101 10147 99058 0.47147 297.47147 148.97147 14897.14714 0.47147 297.47147 148.97146 14897.14687 0.47147 297.47147 148.97147 14897.14700 2020-01-01 2020-01-02 2020-01-01 00:02:37 2020-01-02 03:30:58 2020-01-01 00:02:37.000 2020-01-02 03:30:58.000 157 99058 49607.5 4960750 157 99058 49607.5 4960750 -32412 32523 5043.02 504302 -126 125 -0.18 -18 -158 101 10148 99059 0.47447 297.47447 148.97447 14897.44744 0.47447 297.4745 148.97447 14897.44748 0.47447 297.47447 148.97447 14897.44700 2020-01-01 2020-01-02 2020-01-01 00:02:38 2020-01-02 03:30:59 2020-01-01 00:02:38.000 2020-01-02 03:30:59.000 158 99059 49608.5 4960850 158 99059 49608.5 4960850 -32411 32524 5044.02 504402 -125 126 0.82 82 -159 101 10149 99060 0.47747 297.47747 148.97747 14897.74774 0.47747 297.47748 148.97747 14897.74779 0.47747 297.47747 148.97747 14897.74700 2020-01-01 2020-01-02 2020-01-01 00:02:39 2020-01-02 03:31:00 2020-01-01 00:02:39.000 2020-01-02 03:31:00.000 159 99060 49609.5 4960950 159 99060 49609.5 4960950 -32410 32525 5045.02 504502 -124 127 1.82 182 -16 102 10006 99916 0.04804 300.04804 150.04804 15154.85285 0.04804 300.04803 150.04804 15154.85279 0.04804 300.04804 150.04804 15154.85204 2020-01-01 2020-01-02 2020-01-01 00:00:16 2020-01-02 03:45:16 2020-01-01 00:00:16.000 2020-01-02 03:45:16.000 16 99916 49966 5046566 16 99916 49966 5046566 -32553 32382 4545.009900990099 459046 -128 127 0.37623762376237624 38 -160 101 10150 99061 0.48048 297.48048 148.98048 14898.04804 0.48048 297.48047 148.98048 14898.0481 0.48048 297.48048 148.98048 14898.04800 2020-01-01 2020-01-02 2020-01-01 00:02:40 2020-01-02 03:31:01 2020-01-01 00:02:40.000 2020-01-02 03:31:01.000 160 99061 49610.5 4961050 160 99061 49610.5 4961050 -32409 32526 5046.02 504602 -128 127 0.26 26 -161 101 10151 99062 0.48348 297.48348 148.98348 14898.34834 0.48348 297.4835 148.98348 14898.34868 0.48348 297.48348 148.98348 14898.34800 2020-01-01 2020-01-02 2020-01-01 00:02:41 2020-01-02 03:31:02 2020-01-01 00:02:41.000 2020-01-02 03:31:02.000 161 99062 49611.5 4961150 161 99062 49611.5 4961150 -32408 32527 5047.02 504702 -128 123 -1.3 -130 -162 101 10152 99063 0.48648 297.48648 148.98648 14898.64864 0.48648 297.48648 148.98648 14898.64837 0.48648 297.48648 148.98648 14898.64800 2020-01-01 2020-01-02 2020-01-01 00:02:42 2020-01-02 03:31:03 2020-01-01 00:02:42.000 2020-01-02 03:31:03.000 162 99063 49612.5 4961250 162 99063 49612.5 4961250 -32407 32528 5048.02 504802 -127 124 -0.3 -30 -163 101 10153 99064 0.48948 297.48948 148.98948 14898.94894 0.48948 297.4895 148.98948 14898.94895 0.48948 297.48948 148.98948 14898.94800 2020-01-01 2020-01-02 2020-01-01 00:02:43 2020-01-02 03:31:04 2020-01-01 00:02:43.000 2020-01-02 03:31:04.000 163 99064 49613.5 4961350 163 99064 49613.5 4961350 -32406 32529 5049.02 504902 -126 125 0.7 70 +149 101 10139 99050 0.44744 297.44744 148.94744 14894.74474 0.44744 297.44745 148.94744 14894.74485 0.44744 297.44744 148.94743999999994 14894.74400 2020-01-01 2020-01-02 2020-01-01 00:02:29 2020-01-02 03:30:50 2020-01-01 00:02:29.000 2020-01-02 03:30:50.000 149 99050 49599.5 4959950 149 99050 49599.5 4959950 -32420 32515 5035.02 503502 -128 124 -0.5 -50 +15 102 10005 99915 0.04504 300.04504 150.04504 15154.54954 0.04504 300.04504 150.04504 15154.54945 0.04504 300.04504 150.04504000000006 15154.54904 2020-01-01 2020-01-02 2020-01-01 00:00:15 2020-01-02 03:45:15 2020-01-01 00:00:15.000 2020-01-02 03:45:15.000 15 99915 49965 5046465 15 99915 49965 5046465 -32554 32381 4544.009900990099 458945 -124 127 1.9108910891089108 193 +150 101 10140 99051 0.45045 297.45045 148.95045 14895.04504 0.45045 297.45044 148.95045 14895.04501 0.45045 297.45045 148.95045000000005 14895.04500 2020-01-01 2020-01-02 2020-01-01 00:02:30 2020-01-02 03:30:51 2020-01-01 00:02:30.000 2020-01-02 03:30:51.000 150 99051 49600.5 4960050 150 99051 49600.5 4960050 -32419 32516 5036.02 503602 -127 125 0.5 50 +151 101 10141 99052 0.45345 297.45345 148.95345 14895.34534 0.45345 297.45346 148.95345 14895.34574 0.45345 297.45345 148.9534500000001 14895.34500 2020-01-01 2020-01-02 2020-01-01 00:02:31 2020-01-02 03:30:52 2020-01-01 00:02:31.000 2020-01-02 03:30:52.000 151 99052 49601.5 4960150 151 99052 49601.5 4960150 -32418 32517 5037.02 503702 -126 126 1.5 150 +152 101 10142 99053 0.45645 297.45645 148.95645 14895.64564 0.45645 297.45645 148.95645 14895.6454 0.45645 297.45645 148.9564499999999 14895.64500 2020-01-01 2020-01-02 2020-01-01 00:02:32 2020-01-02 03:30:53 2020-01-01 00:02:32.000 2020-01-02 03:30:53.000 152 99053 49602.5 4960250 152 99053 49602.5 4960250 -32417 32518 5038.02 503802 -125 127 2.5 250 +153 101 10143 99054 0.45945 297.45945 148.95945 14895.94594 0.45945 297.45947 148.95946 14895.94601 0.45945 297.45945 148.95945000000006 14895.94500 2020-01-01 2020-01-02 2020-01-01 00:02:33 2020-01-02 03:30:54 2020-01-01 00:02:33.000 2020-01-02 03:30:54.000 153 99054 49603.5 4960350 153 99054 49603.5 4960350 -32416 32519 5039.02 503902 -128 127 0.94 94 +154 101 10144 99055 0.46246 297.46246 148.96246 14896.24624 0.46246 297.46246 148.96246 14896.24633 0.46246 297.46246 148.96246000000008 14896.24600 2020-01-01 2020-01-02 2020-01-01 00:02:34 2020-01-02 03:30:55 2020-01-01 00:02:34.000 2020-01-02 03:30:55.000 154 99055 49604.5 4960450 154 99055 49604.5 4960450 -32415 32520 5040.02 504002 -128 127 -0.62 -62 +155 101 10145 99056 0.46546 297.46546 148.96546 14896.54654 0.46546 297.46545 148.96546 14896.54647 0.46546 297.46546 148.96545999999998 14896.54600 2020-01-01 2020-01-02 2020-01-01 00:02:35 2020-01-02 03:30:56 2020-01-01 00:02:35.000 2020-01-02 03:30:56.000 155 99056 49605.5 4960550 155 99056 49605.5 4960550 -32414 32521 5041.02 504102 -128 123 -2.18 -218 +156 101 10146 99057 0.46846 297.46846 148.96846 14896.84684 0.46846 297.46848 148.96847 14896.84721 0.46846 297.46846 148.96846000000002 14896.84600 2020-01-01 2020-01-02 2020-01-01 00:02:36 2020-01-02 03:30:57 2020-01-01 00:02:36.000 2020-01-02 03:30:57.000 156 99057 49606.5 4960650 156 99057 49606.5 4960650 -32413 32522 5042.02 504202 -127 124 -1.18 -118 +157 101 10147 99058 0.47147 297.47147 148.97147 14897.14714 0.47147 297.47147 148.97146 14897.14687 0.47147 297.47147 148.97147000000012 14897.14700 2020-01-01 2020-01-02 2020-01-01 00:02:37 2020-01-02 03:30:58 2020-01-01 00:02:37.000 2020-01-02 03:30:58.000 157 99058 49607.5 4960750 157 99058 49607.5 4960750 -32412 32523 5043.02 504302 -126 125 -0.18 -18 +158 101 10148 99059 0.47447 297.47447 148.97447 14897.44744 0.47447 297.4745 148.97447 14897.44748 0.47447 297.47447 148.97446999999994 14897.44700 2020-01-01 2020-01-02 2020-01-01 00:02:38 2020-01-02 03:30:59 2020-01-01 00:02:38.000 2020-01-02 03:30:59.000 158 99059 49608.5 4960850 158 99059 49608.5 4960850 -32411 32524 5044.02 504402 -125 126 0.82 82 +159 101 10149 99060 0.47747 297.47747 148.97747 14897.74774 0.47747 297.47748 148.97747 14897.74779 0.47747 297.47747 148.97746999999993 14897.74700 2020-01-01 2020-01-02 2020-01-01 00:02:39 2020-01-02 03:31:00 2020-01-01 00:02:39.000 2020-01-02 03:31:00.000 159 99060 49609.5 4960950 159 99060 49609.5 4960950 -32410 32525 5045.02 504502 -124 127 1.82 182 +16 102 10006 99916 0.04804 300.04804 150.04804 15154.85285 0.04804 300.04803 150.04804 15154.85279 0.04804 300.04804 150.0480399999999 15154.85204 2020-01-01 2020-01-02 2020-01-01 00:00:16 2020-01-02 03:45:16 2020-01-01 00:00:16.000 2020-01-02 03:45:16.000 16 99916 49966 5046566 16 99916 49966 5046566 -32553 32382 4545.009900990099 459046 -128 127 0.37623762376237624 38 +160 101 10150 99061 0.48048 297.48048 148.98048 14898.04804 0.48048 297.48047 148.98048 14898.0481 0.48048 297.48048 148.98048000000009 14898.04800 2020-01-01 2020-01-02 2020-01-01 00:02:40 2020-01-02 03:31:01 2020-01-01 00:02:40.000 2020-01-02 03:31:01.000 160 99061 49610.5 4961050 160 99061 49610.5 4961050 -32409 32526 5046.02 504602 -128 127 0.26 26 +161 101 10151 99062 0.48348 297.48348 148.98348 14898.34834 0.48348 297.4835 148.98348 14898.34868 0.48348 297.48348 148.98348000000004 14898.34800 2020-01-01 2020-01-02 2020-01-01 00:02:41 2020-01-02 03:31:02 2020-01-01 00:02:41.000 2020-01-02 03:31:02.000 161 99062 49611.5 4961150 161 99062 49611.5 4961150 -32408 32527 5047.02 504702 -128 123 -1.3 -130 +162 101 10152 99063 0.48648 297.48648 148.98648 14898.64864 0.48648 297.48648 148.98648 14898.64837 0.48648 297.48648 148.98647999999986 14898.64800 2020-01-01 2020-01-02 2020-01-01 00:02:42 2020-01-02 03:31:03 2020-01-01 00:02:42.000 2020-01-02 03:31:03.000 162 99063 49612.5 4961250 162 99063 49612.5 4961250 -32407 32528 5048.02 504802 -127 124 -0.3 -30 +163 101 10153 99064 0.48948 297.48948 148.98948 14898.94894 0.48948 297.4895 148.98948 14898.94895 0.48948 297.48948 148.98948000000007 14898.94800 2020-01-01 2020-01-02 2020-01-01 00:02:43 2020-01-02 03:31:04 2020-01-01 00:02:43.000 2020-01-02 03:31:04.000 163 99064 49613.5 4961350 163 99064 49613.5 4961350 -32406 32529 5049.02 504902 -126 125 0.7 70 164 101 10154 99065 0.49249 297.49249 148.99249 14899.24924 0.49249 297.4925 148.99249 14899.24926 0.49249 297.49249 148.99249 14899.24900 2020-01-01 2020-01-02 2020-01-01 00:02:44 2020-01-02 03:31:05 2020-01-01 00:02:44.000 2020-01-02 03:31:05.000 164 99065 49614.5 4961450 164 99065 49614.5 4961450 -32405 32530 5050.02 505002 -125 126 1.7 170 -165 101 10155 99066 0.49549 297.49549 148.99549 14899.54954 0.49549 297.49548 148.99549 14899.54957 0.49549 297.49549 148.99549 14899.54900 2020-01-01 2020-01-02 2020-01-01 00:02:45 2020-01-02 03:31:06 2020-01-01 00:02:45.000 2020-01-02 03:31:06.000 165 99066 49615.5 4961550 165 99066 49615.5 4961550 -32404 32531 5051.02 505102 -124 127 2.7 270 -166 101 10156 99067 0.49849 297.49849 148.99849 14899.84984 0.49849 297.4985 148.9985 14899.85015 0.49849 297.49849 148.99849 14899.84900 2020-01-01 2020-01-02 2020-01-01 00:02:46 2020-01-02 03:31:07 2020-01-01 00:02:46.000 2020-01-02 03:31:07.000 166 99067 49616.5 4961650 166 99067 49616.5 4961650 -32403 32532 5052.02 505202 -128 127 1.14 114 -167 101 10157 99068 0.5015 297.5015 149.0015 14900.15015 0.5015 297.5015 149.00149 14900.14984 0.50150 297.50150 149.00150 14900.15000 2020-01-01 2020-01-02 2020-01-01 00:02:47 2020-01-02 03:31:08 2020-01-01 00:02:47.000 2020-01-02 03:31:08.000 167 99068 49617.5 4961750 167 99068 49617.5 4961750 -32402 32533 5053.02 505302 -128 123 -0.42 -42 -168 101 10158 99069 0.5045 297.5045 149.0045 14900.45045 0.5045 297.50452 149.0045 14900.45042 0.50450 297.50450 149.00450 14900.45000 2020-01-01 2020-01-02 2020-01-01 00:02:48 2020-01-02 03:31:09 2020-01-01 00:02:48.000 2020-01-02 03:31:09.000 168 99069 49618.5 4961850 168 99069 49618.5 4961850 -32401 32534 5054.02 505402 -127 124 0.58 58 -169 101 10159 99070 0.5075 297.5075 149.0075 14900.75075 0.5075 297.5075 149.0075 14900.75073 0.50750 297.50750 149.00750 14900.75000 2020-01-01 2020-01-02 2020-01-01 00:02:49 2020-01-02 03:31:10 2020-01-01 00:02:49.000 2020-01-02 03:31:10.000 169 99070 49619.5 4961950 169 99070 49619.5 4961950 -32400 32535 5055.02 505502 -126 125 1.58 158 +165 101 10155 99066 0.49549 297.49549 148.99549 14899.54954 0.49549 297.49548 148.99549 14899.54957 0.49549 297.49549 148.99548999999996 14899.54900 2020-01-01 2020-01-02 2020-01-01 00:02:45 2020-01-02 03:31:06 2020-01-01 00:02:45.000 2020-01-02 03:31:06.000 165 99066 49615.5 4961550 165 99066 49615.5 4961550 -32404 32531 5051.02 505102 -124 127 2.7 270 +166 101 10156 99067 0.49849 297.49849 148.99849 14899.84984 0.49849 297.4985 148.9985 14899.85015 0.49849 297.49849 148.99848999999998 14899.84900 2020-01-01 2020-01-02 2020-01-01 00:02:46 2020-01-02 03:31:07 2020-01-01 00:02:46.000 2020-01-02 03:31:07.000 166 99067 49616.5 4961650 166 99067 49616.5 4961650 -32403 32532 5052.02 505202 -128 127 1.14 114 +167 101 10157 99068 0.5015 297.5015 149.0015 14900.15015 0.5015 297.5015 149.00149 14900.14984 0.50150 297.50150 149.00150000000008 14900.15000 2020-01-01 2020-01-02 2020-01-01 00:02:47 2020-01-02 03:31:08 2020-01-01 00:02:47.000 2020-01-02 03:31:08.000 167 99068 49617.5 4961750 167 99068 49617.5 4961750 -32402 32533 5053.02 505302 -128 123 -0.42 -42 +168 101 10158 99069 0.5045 297.5045 149.0045 14900.45045 0.5045 297.50452 149.0045 14900.45042 0.50450 297.50450 149.0044999999999 14900.45000 2020-01-01 2020-01-02 2020-01-01 00:02:48 2020-01-02 03:31:09 2020-01-01 00:02:48.000 2020-01-02 03:31:09.000 168 99069 49618.5 4961850 168 99069 49618.5 4961850 -32401 32534 5054.02 505402 -127 124 0.58 58 +169 101 10159 99070 0.5075 297.5075 149.0075 14900.75075 0.5075 297.5075 149.0075 14900.75073 0.50750 297.50750 149.00749999999994 14900.75000 2020-01-01 2020-01-02 2020-01-01 00:02:49 2020-01-02 03:31:10 2020-01-01 00:02:49.000 2020-01-02 03:31:10.000 169 99070 49619.5 4961950 169 99070 49619.5 4961950 -32400 32535 5055.02 505502 -126 125 1.58 158 17 102 10007 99917 0.05105 300.05105 150.05105 15155.15615 0.05105 300.05106 150.05105 15155.15638 0.05105 300.05105 150.05105 15155.15605 2020-01-01 2020-01-02 2020-01-01 00:00:17 2020-01-02 03:45:17 2020-01-01 00:00:17.000 2020-01-02 03:45:17.000 17 99917 49967 5046667 17 99917 49967 5046667 -32552 32383 4546.009900990099 459147 -128 127 -1.1584158415841583 -117 -170 101 10160 99071 0.51051 297.51051 149.01051 14901.05105 0.51051 297.5105 149.01051 14901.05104 0.51051 297.51051 149.01051 14901.05100 2020-01-01 2020-01-02 2020-01-01 00:02:50 2020-01-02 03:31:11 2020-01-01 00:02:50.000 2020-01-02 03:31:11.000 170 99071 49620.5 4962050 170 99071 49620.5 4962050 -32399 32536 5056.02 505602 -125 126 2.58 258 +170 101 10160 99071 0.51051 297.51051 149.01051 14901.05105 0.51051 297.5105 149.01051 14901.05104 0.51051 297.51051 149.01051000000004 14901.05100 2020-01-01 2020-01-02 2020-01-01 00:02:50 2020-01-02 03:31:11 2020-01-01 00:02:50.000 2020-01-02 03:31:11.000 170 99071 49620.5 4962050 170 99071 49620.5 4962050 -32399 32536 5056.02 505602 -125 126 2.58 258 171 101 10161 99072 0.51351 297.51351 149.01351 14901.35135 0.51351 297.51352 149.01351 14901.35162 0.51351 297.51351 149.01351 14901.35100 2020-01-01 2020-01-02 2020-01-01 00:02:51 2020-01-02 03:31:12 2020-01-01 00:02:51.000 2020-01-02 03:31:12.000 171 99072 49621.5 4962150 171 99072 49621.5 4962150 -32398 32537 5057.02 505702 -124 127 3.58 358 -172 101 10162 99073 0.51651 297.51651 149.01651 14901.65165 0.51651 297.5165 149.01651 14901.65131 0.51651 297.51651 149.01651 14901.65100 2020-01-01 2020-01-02 2020-01-01 00:02:52 2020-01-02 03:31:13 2020-01-01 00:02:52.000 2020-01-02 03:31:13.000 172 99073 49622.5 4962250 172 99073 49622.5 4962250 -32397 32538 5058.02 505802 -128 127 2.02 202 -173 101 10163 99074 0.51951 297.51951 149.01951 14901.95195 0.51951 297.51953 149.01951 14901.95189 0.51951 297.51951 149.01951 14901.95100 2020-01-01 2020-01-02 2020-01-01 00:02:53 2020-01-02 03:31:14 2020-01-01 00:02:53.000 2020-01-02 03:31:14.000 173 99074 49623.5 4962350 173 99074 49623.5 4962350 -32396 32539 5059.02 505902 -128 127 0.46 46 -174 101 10164 99075 0.52252 297.52252 149.02252 14902.25225 0.52252 297.52252 149.02252 14902.2522 0.52252 297.52252 149.02252 14902.25200 2020-01-01 2020-01-02 2020-01-01 00:02:54 2020-01-02 03:31:15 2020-01-01 00:02:54.000 2020-01-02 03:31:15.000 174 99075 49624.5 4962450 174 99075 49624.5 4962450 -32395 32540 5060.02 506002 -128 124 -1.1 -110 -175 101 10165 99076 0.52552 297.52552 149.02552 14902.55255 0.52552 297.5255 149.02552 14902.55251 0.52552 297.52552 149.02552 14902.55200 2020-01-01 2020-01-02 2020-01-01 00:02:55 2020-01-02 03:31:16 2020-01-01 00:02:55.000 2020-01-02 03:31:16.000 175 99076 49625.5 4962550 175 99076 49625.5 4962550 -32394 32541 5061.02 506102 -127 125 -0.1 -10 -176 101 10166 99077 0.52852 297.52852 149.02852 14902.85285 0.52852 297.52853 149.02853 14902.85312 0.52852 297.52852 149.02852 14902.85200 2020-01-01 2020-01-02 2020-01-01 00:02:56 2020-01-02 03:31:17 2020-01-01 00:02:56.000 2020-01-02 03:31:17.000 176 99077 49626.5 4962650 176 99077 49626.5 4962650 -32393 32542 5062.02 506202 -126 126 0.9 90 -177 101 10167 99078 0.53153 297.53153 149.03153 14903.15315 0.53153 297.53152 149.03152 14903.15278 0.53153 297.53153 149.03153 14903.15300 2020-01-01 2020-01-02 2020-01-01 00:02:57 2020-01-02 03:31:18 2020-01-01 00:02:57.000 2020-01-02 03:31:18.000 177 99078 49627.5 4962750 177 99078 49627.5 4962750 -32392 32543 5063.02 506302 -125 127 1.9 190 -178 101 10168 99079 0.53453 297.53453 149.03453 14903.45345 0.53453 297.53455 149.03453 14903.45352 0.53453 297.53453 149.03453 14903.45300 2020-01-01 2020-01-02 2020-01-01 00:02:58 2020-01-02 03:31:19 2020-01-01 00:02:58.000 2020-01-02 03:31:19.000 178 99079 49628.5 4962850 178 99079 49628.5 4962850 -32391 32544 5064.02 506402 -128 127 0.34 34 -179 101 10169 99080 0.53753 297.53753 149.03753 14903.75375 0.53753 297.53754 149.03753 14903.75366 0.53753 297.53753 149.03753 14903.75300 2020-01-01 2020-01-02 2020-01-01 00:02:59 2020-01-02 03:31:20 2020-01-01 00:02:59.000 2020-01-02 03:31:20.000 179 99080 49629.5 4962950 179 99080 49629.5 4962950 -32390 32545 5065.02 506502 -128 127 -1.22 -122 -18 102 10008 99918 0.05405 300.05405 150.05405 15155.45945 0.05405 300.05405 150.05404 15155.45903 0.05405 300.05405 150.05405 15155.45905 2020-01-01 2020-01-02 2020-01-01 00:00:18 2020-01-02 03:45:18 2020-01-01 00:00:18.000 2020-01-02 03:45:18.000 18 99918 49968 5046768 18 99918 49968 5046768 -32551 32384 4547.009900990099 459248 -128 124 -2.6930693069306932 -272 +172 101 10162 99073 0.51651 297.51651 149.01651 14901.65165 0.51651 297.5165 149.01651 14901.65131 0.51651 297.51651 149.01650999999998 14901.65100 2020-01-01 2020-01-02 2020-01-01 00:02:52 2020-01-02 03:31:13 2020-01-01 00:02:52.000 2020-01-02 03:31:13.000 172 99073 49622.5 4962250 172 99073 49622.5 4962250 -32397 32538 5058.02 505802 -128 127 2.02 202 +173 101 10163 99074 0.51951 297.51951 149.01951 14901.95195 0.51951 297.51953 149.01951 14901.95189 0.51951 297.51951 149.01951000000005 14901.95100 2020-01-01 2020-01-02 2020-01-01 00:02:53 2020-01-02 03:31:14 2020-01-01 00:02:53.000 2020-01-02 03:31:14.000 173 99074 49623.5 4962350 173 99074 49623.5 4962350 -32396 32539 5059.02 505902 -128 127 0.46 46 +174 101 10164 99075 0.52252 297.52252 149.02252 14902.25225 0.52252 297.52252 149.02252 14902.2522 0.52252 297.52252 149.02252000000016 14902.25200 2020-01-01 2020-01-02 2020-01-01 00:02:54 2020-01-02 03:31:15 2020-01-01 00:02:54.000 2020-01-02 03:31:15.000 174 99075 49624.5 4962450 174 99075 49624.5 4962450 -32395 32540 5060.02 506002 -128 124 -1.1 -110 +175 101 10165 99076 0.52552 297.52552 149.02552 14902.55255 0.52552 297.5255 149.02552 14902.55251 0.52552 297.52552 149.02551999999997 14902.55200 2020-01-01 2020-01-02 2020-01-01 00:02:55 2020-01-02 03:31:16 2020-01-01 00:02:55.000 2020-01-02 03:31:16.000 175 99076 49625.5 4962550 175 99076 49625.5 4962550 -32394 32541 5061.02 506102 -127 125 -0.1 -10 +176 101 10166 99077 0.52852 297.52852 149.02852 14902.85285 0.52852 297.52853 149.02853 14902.85312 0.52852 297.52852 149.02851999999993 14902.85200 2020-01-01 2020-01-02 2020-01-01 00:02:56 2020-01-02 03:31:17 2020-01-01 00:02:56.000 2020-01-02 03:31:17.000 176 99077 49626.5 4962650 176 99077 49626.5 4962650 -32393 32542 5062.02 506202 -126 126 0.9 90 +177 101 10167 99078 0.53153 297.53153 149.03153 14903.15315 0.53153 297.53152 149.03152 14903.15278 0.53153 297.53153 149.0315300000001 14903.15300 2020-01-01 2020-01-02 2020-01-01 00:02:57 2020-01-02 03:31:18 2020-01-01 00:02:57.000 2020-01-02 03:31:18.000 177 99078 49627.5 4962750 177 99078 49627.5 4962750 -32392 32543 5063.02 506302 -125 127 1.9 190 +178 101 10168 99079 0.53453 297.53453 149.03453 14903.45345 0.53453 297.53455 149.03453 14903.45352 0.53453 297.53453 149.03453000000005 14903.45300 2020-01-01 2020-01-02 2020-01-01 00:02:58 2020-01-02 03:31:19 2020-01-01 00:02:58.000 2020-01-02 03:31:19.000 178 99079 49628.5 4962850 178 99079 49628.5 4962850 -32391 32544 5064.02 506402 -128 127 0.34 34 +179 101 10169 99080 0.53753 297.53753 149.03753 14903.75375 0.53753 297.53754 149.03753 14903.75366 0.53753 297.53753 149.0375299999999 14903.75300 2020-01-01 2020-01-02 2020-01-01 00:02:59 2020-01-02 03:31:20 2020-01-01 00:02:59.000 2020-01-02 03:31:20.000 179 99080 49629.5 4962950 179 99080 49629.5 4962950 -32390 32545 5065.02 506502 -128 127 -1.22 -122 +18 102 10008 99918 0.05405 300.05405 150.05405 15155.45945 0.05405 300.05405 150.05404 15155.45903 0.05405 300.05405 150.05405000000007 15155.45905 2020-01-01 2020-01-02 2020-01-01 00:00:18 2020-01-02 03:45:18 2020-01-01 00:00:18.000 2020-01-02 03:45:18.000 18 99918 49968 5046768 18 99918 49968 5046768 -32551 32384 4547.009900990099 459248 -128 124 -2.6930693069306932 -272 180 101 10170 99081 0.54054 297.54054 149.04054 14904.05405 0.54054 297.54053 149.04053 14904.05398 0.54054 297.54054 149.04054 14904.05400 2020-01-01 2020-01-02 2020-01-01 00:03:00 2020-01-02 03:31:21 2020-01-01 00:03:00.000 2020-01-02 03:31:21.000 180 99081 49630.5 4963050 180 99081 49630.5 4963050 -32389 32546 5066.02 506602 -128 123 -2.78 -278 -181 101 10171 99082 0.54354 297.54354 149.04354 14904.35435 0.54354 297.54355 149.04354 14904.35459 0.54354 297.54354 149.04354 14904.35400 2020-01-01 2020-01-02 2020-01-01 00:03:01 2020-01-02 03:31:22 2020-01-01 00:03:01.000 2020-01-02 03:31:22.000 181 99082 49631.5 4963150 181 99082 49631.5 4963150 -32388 32547 5067.02 506702 -127 124 -1.78 -178 -182 101 10172 99083 0.54654 297.54654 149.04654 14904.65465 0.54654 297.54654 149.04654 14904.65425 0.54654 297.54654 149.04654 14904.65400 2020-01-01 2020-01-02 2020-01-01 00:03:02 2020-01-02 03:31:23 2020-01-01 00:03:02.000 2020-01-02 03:31:23.000 182 99083 49632.5 4963250 182 99083 49632.5 4963250 -32387 32548 5068.02 506802 -126 125 -0.78 -78 +181 101 10171 99082 0.54354 297.54354 149.04354 14904.35435 0.54354 297.54355 149.04354 14904.35459 0.54354 297.54354 149.04354000000004 14904.35400 2020-01-01 2020-01-02 2020-01-01 00:03:01 2020-01-02 03:31:22 2020-01-01 00:03:01.000 2020-01-02 03:31:22.000 181 99082 49631.5 4963150 181 99082 49631.5 4963150 -32388 32547 5067.02 506702 -127 124 -1.78 -178 +182 101 10172 99083 0.54654 297.54654 149.04654 14904.65465 0.54654 297.54654 149.04654 14904.65425 0.54654 297.54654 149.04653999999994 14904.65400 2020-01-01 2020-01-02 2020-01-01 00:03:02 2020-01-02 03:31:23 2020-01-01 00:03:02.000 2020-01-02 03:31:23.000 182 99083 49632.5 4963250 182 99083 49632.5 4963250 -32387 32548 5068.02 506802 -126 125 -0.78 -78 183 101 10173 99084 0.54954 297.54954 149.04954 14904.95495 0.54954 297.54956 149.04954 14904.95498 0.54954 297.54954 149.04954 14904.95400 2020-01-01 2020-01-02 2020-01-01 00:03:03 2020-01-02 03:31:24 2020-01-01 00:03:03.000 2020-01-02 03:31:24.000 183 99084 49633.5 4963350 183 99084 49633.5 4963350 -32386 32549 5069.02 506902 -125 126 0.22 22 -184 101 10174 99085 0.55255 297.55255 149.05255 14905.25525 0.55255 297.55255 149.05255 14905.25514 0.55255 297.55255 149.05255 14905.25500 2020-01-01 2020-01-02 2020-01-01 00:03:04 2020-01-02 03:31:25 2020-01-01 00:03:04.000 2020-01-02 03:31:25.000 184 99085 49634.5 4963450 184 99085 49634.5 4963450 -32385 32550 5070.02 507002 -124 127 1.22 122 -185 101 10175 99086 0.55555 297.55555 149.05555 14905.55555 0.55555 297.55554 149.05555 14905.55549 0.55555 297.55555 149.05555 14905.55500 2020-01-01 2020-01-02 2020-01-01 00:03:05 2020-01-02 03:31:26 2020-01-01 00:03:05.000 2020-01-02 03:31:26.000 185 99086 49635.5 4963550 185 99086 49635.5 4963550 -32384 32551 5071.02 507102 -128 127 -0.34 -34 -186 101 10176 99087 0.55855 297.55855 149.05855 14905.85585 0.55855 297.55856 149.05856 14905.85607 0.55855 297.55855 149.05855 14905.85500 2020-01-01 2020-01-02 2020-01-01 00:03:06 2020-01-02 03:31:27 2020-01-01 00:03:06.000 2020-01-02 03:31:27.000 186 99087 49636.5 4963650 186 99087 49636.5 4963650 -32383 32552 5072.02 507202 -128 123 -1.9 -190 -187 101 10177 99088 0.56156 297.56156 149.06156 14906.15615 0.56156 297.56155 149.06155 14906.15572 0.56156 297.56156 149.06156 14906.15600 2020-01-01 2020-01-02 2020-01-01 00:03:07 2020-01-02 03:31:28 2020-01-01 00:03:07.000 2020-01-02 03:31:28.000 187 99088 49637.5 4963750 187 99088 49637.5 4963750 -32382 32553 5073.02 507302 -127 124 -0.9 -90 +184 101 10174 99085 0.55255 297.55255 149.05255 14905.25525 0.55255 297.55255 149.05255 14905.25514 0.55255 297.55255 149.05255000000005 14905.25500 2020-01-01 2020-01-02 2020-01-01 00:03:04 2020-01-02 03:31:25 2020-01-01 00:03:04.000 2020-01-02 03:31:25.000 184 99085 49634.5 4963450 184 99085 49634.5 4963450 -32385 32550 5070.02 507002 -124 127 1.22 122 +185 101 10175 99086 0.55555 297.55555 149.05555 14905.55555 0.55555 297.55554 149.05555 14905.55549 0.55555 297.55555 149.05554999999993 14905.55500 2020-01-01 2020-01-02 2020-01-01 00:03:05 2020-01-02 03:31:26 2020-01-01 00:03:05.000 2020-01-02 03:31:26.000 185 99086 49635.5 4963550 185 99086 49635.5 4963550 -32384 32551 5071.02 507102 -128 127 -0.34 -34 +186 101 10176 99087 0.55855 297.55855 149.05855 14905.85585 0.55855 297.55856 149.05856 14905.85607 0.55855 297.55855 149.05854999999997 14905.85500 2020-01-01 2020-01-02 2020-01-01 00:03:06 2020-01-02 03:31:27 2020-01-01 00:03:06.000 2020-01-02 03:31:27.000 186 99087 49636.5 4963650 186 99087 49636.5 4963650 -32383 32552 5072.02 507202 -128 123 -1.9 -190 +187 101 10177 99088 0.56156 297.56156 149.06156 14906.15615 0.56156 297.56155 149.06155 14906.15572 0.56156 297.56156 149.06156000000007 14906.15600 2020-01-01 2020-01-02 2020-01-01 00:03:07 2020-01-02 03:31:28 2020-01-01 00:03:07.000 2020-01-02 03:31:28.000 187 99088 49637.5 4963750 187 99088 49637.5 4963750 -32382 32553 5073.02 507302 -127 124 -0.9 -90 188 101 10178 99089 0.56456 297.56456 149.06456 14906.45645 0.56456 297.56458 149.06456 14906.45645 0.56456 297.56456 149.06456 14906.45600 2020-01-01 2020-01-02 2020-01-01 00:03:08 2020-01-02 03:31:29 2020-01-01 00:03:08.000 2020-01-02 03:31:29.000 188 99089 49638.5 4963850 188 99089 49638.5 4963850 -32381 32554 5074.02 507402 -126 125 0.1 10 -189 101 10179 99090 0.56756 297.56756 149.06756 14906.75675 0.56756 297.56757 149.06756 14906.75661 0.56756 297.56756 149.06756 14906.75600 2020-01-01 2020-01-02 2020-01-01 00:03:09 2020-01-02 03:31:30 2020-01-01 00:03:09.000 2020-01-02 03:31:30.000 189 99090 49639.5 4963950 189 99090 49639.5 4963950 -32380 32555 5075.02 507502 -125 126 1.1 110 -19 102 10009 99919 0.05705 300.05705 150.05705 15155.76276 0.05705 300.05707 150.05705 15155.76279 0.05705 300.05705 150.05705 15155.76205 2020-01-01 2020-01-02 2020-01-01 00:00:19 2020-01-02 03:45:19 2020-01-01 00:00:19.000 2020-01-02 03:45:19.000 19 99919 49969 5046869 19 99919 49969 5046869 -32550 32385 4548.009900990099 459349 -127 125 -1.693069306930693 -171 -190 101 10180 99091 0.57057 297.57057 149.07057 14907.05705 0.57057 297.57056 149.07056 14907.05695 0.57057 297.57057 149.07057 14907.05700 2020-01-01 2020-01-02 2020-01-01 00:03:10 2020-01-02 03:31:31 2020-01-01 00:03:10.000 2020-01-02 03:31:31.000 190 99091 49640.5 4964050 190 99091 49640.5 4964050 -32379 32556 5076.02 507602 -124 127 2.1 210 +189 101 10179 99090 0.56756 297.56756 149.06756 14906.75675 0.56756 297.56757 149.06756 14906.75661 0.56756 297.56756 149.06756000000001 14906.75600 2020-01-01 2020-01-02 2020-01-01 00:03:09 2020-01-02 03:31:30 2020-01-01 00:03:09.000 2020-01-02 03:31:30.000 189 99090 49639.5 4963950 189 99090 49639.5 4963950 -32380 32555 5075.02 507502 -125 126 1.1 110 +19 102 10009 99919 0.05705 300.05705 150.05705 15155.76276 0.05705 300.05707 150.05705 15155.76279 0.05705 300.05705 150.05704999999998 15155.76205 2020-01-01 2020-01-02 2020-01-01 00:00:19 2020-01-02 03:45:19 2020-01-01 00:00:19.000 2020-01-02 03:45:19.000 19 99919 49969 5046869 19 99919 49969 5046869 -32550 32385 4548.009900990099 459349 -127 125 -1.693069306930693 -171 +190 101 10180 99091 0.57057 297.57057 149.07057 14907.05705 0.57057 297.57056 149.07056 14907.05695 0.57057 297.57057 149.07056999999995 14907.05700 2020-01-01 2020-01-02 2020-01-01 00:03:10 2020-01-02 03:31:31 2020-01-01 00:03:10.000 2020-01-02 03:31:31.000 190 99091 49640.5 4964050 190 99091 49640.5 4964050 -32379 32556 5076.02 507602 -124 127 2.1 210 191 101 10181 99092 0.57357 297.57357 149.07357 14907.35735 0.57357 297.57358 149.07357 14907.35753 0.57357 297.57357 149.07357 14907.35700 2020-01-01 2020-01-02 2020-01-01 00:03:11 2020-01-02 03:31:32 2020-01-01 00:03:11.000 2020-01-02 03:31:32.000 191 99092 49641.5 4964150 191 99092 49641.5 4964150 -32378 32557 5077.02 507702 -128 127 0.54 54 -192 101 10182 99093 0.57657 297.57657 149.07657 14907.65765 0.57657 297.57657 149.07657 14907.65784 0.57657 297.57657 149.07657 14907.65700 2020-01-01 2020-01-02 2020-01-01 00:03:12 2020-01-02 03:31:33 2020-01-01 00:03:12.000 2020-01-02 03:31:33.000 192 99093 49642.5 4964250 192 99093 49642.5 4964250 -32377 32558 5078.02 507802 -128 123 -1.02 -102 -193 101 10183 99094 0.57957 297.57957 149.07957 14907.95795 0.57957 297.5796 149.07957 14907.95793 0.57957 297.57957 149.07957 14907.95700 2020-01-01 2020-01-02 2020-01-01 00:03:13 2020-01-02 03:31:34 2020-01-01 00:03:13.000 2020-01-02 03:31:34.000 193 99094 49643.5 4964350 193 99094 49643.5 4964350 -32376 32559 5079.02 507902 -127 124 -0.02 -2 -194 101 10184 99095 0.58258 297.58258 149.08258 14908.25825 0.58258 297.58258 149.08258 14908.25811 0.58258 297.58258 149.08258 14908.25800 2020-01-01 2020-01-02 2020-01-01 00:03:14 2020-01-02 03:31:35 2020-01-01 00:03:14.000 2020-01-02 03:31:35.000 194 99095 49644.5 4964450 194 99095 49644.5 4964450 -32375 32560 5080.02 508002 -126 125 0.98 98 -195 101 10185 99096 0.58558 297.58558 149.08558 14908.55855 0.58558 297.58557 149.08558 14908.55842 0.58558 297.58558 149.08558 14908.55800 2020-01-01 2020-01-02 2020-01-01 00:03:15 2020-01-02 03:31:36 2020-01-01 00:03:15.000 2020-01-02 03:31:36.000 195 99096 49645.5 4964550 195 99096 49645.5 4964550 -32374 32561 5081.02 508102 -125 126 1.98 198 -196 101 10186 99097 0.58858 297.58858 149.08858 14908.85885 0.58858 297.5886 149.08859 14908.859 0.58858 297.58858 149.08858 14908.85800 2020-01-01 2020-01-02 2020-01-01 00:03:16 2020-01-02 03:31:37 2020-01-01 00:03:16.000 2020-01-02 03:31:37.000 196 99097 49646.5 4964650 196 99097 49646.5 4964650 -32373 32562 5082.02 508202 -124 127 2.98 298 +192 101 10182 99093 0.57657 297.57657 149.07657 14907.65765 0.57657 297.57657 149.07657 14907.65784 0.57657 297.57657 149.07656999999998 14907.65700 2020-01-01 2020-01-02 2020-01-01 00:03:12 2020-01-02 03:31:33 2020-01-01 00:03:12.000 2020-01-02 03:31:33.000 192 99093 49642.5 4964250 192 99093 49642.5 4964250 -32377 32558 5078.02 507802 -128 123 -1.02 -102 +193 101 10183 99094 0.57957 297.57957 149.07957 14907.95795 0.57957 297.5796 149.07957 14907.95793 0.57957 297.57957 149.07956999999993 14907.95700 2020-01-01 2020-01-02 2020-01-01 00:03:13 2020-01-02 03:31:34 2020-01-01 00:03:13.000 2020-01-02 03:31:34.000 193 99094 49643.5 4964350 193 99094 49643.5 4964350 -32376 32559 5079.02 507902 -127 124 -0.02 -2 +194 101 10184 99095 0.58258 297.58258 149.08258 14908.25825 0.58258 297.58258 149.08258 14908.25811 0.58258 297.58258 149.0825800000001 14908.25800 2020-01-01 2020-01-02 2020-01-01 00:03:14 2020-01-02 03:31:35 2020-01-01 00:03:14.000 2020-01-02 03:31:35.000 194 99095 49644.5 4964450 194 99095 49644.5 4964450 -32375 32560 5080.02 508002 -126 125 0.98 98 +195 101 10185 99096 0.58558 297.58558 149.08558 14908.55855 0.58558 297.58557 149.08558 14908.55842 0.58558 297.58558 149.08558000000008 14908.55800 2020-01-01 2020-01-02 2020-01-01 00:03:15 2020-01-02 03:31:36 2020-01-01 00:03:15.000 2020-01-02 03:31:36.000 195 99096 49645.5 4964550 195 99096 49645.5 4964550 -32374 32561 5081.02 508102 -125 126 1.98 198 +196 101 10186 99097 0.58858 297.58858 149.08858 14908.85885 0.58858 297.5886 149.08859 14908.859 0.58858 297.58858 149.08857999999992 14908.85800 2020-01-01 2020-01-02 2020-01-01 00:03:16 2020-01-02 03:31:37 2020-01-01 00:03:16.000 2020-01-02 03:31:37.000 196 99097 49646.5 4964650 196 99097 49646.5 4964650 -32373 32562 5082.02 508202 -124 127 2.98 298 197 101 10187 99098 0.59159 297.59159 149.09159 14909.15915 0.59159 297.59158 149.09159 14909.15931 0.59159 297.59159 149.09159 14909.15900 2020-01-01 2020-01-02 2020-01-01 00:03:17 2020-01-02 03:31:38 2020-01-01 00:03:17.000 2020-01-02 03:31:38.000 197 99098 49647.5 4964750 197 99098 49647.5 4964750 -32372 32563 5083.02 508302 -128 127 1.42 142 -198 101 10188 99099 0.59459 297.59459 149.09459 14909.45945 0.59459 297.5946 149.09459 14909.4594 0.59459 297.59459 149.09459 14909.45900 2020-01-01 2020-01-02 2020-01-01 00:03:18 2020-01-02 03:31:39 2020-01-01 00:03:18.000 2020-01-02 03:31:39.000 198 99099 49648.5 4964850 198 99099 49648.5 4964850 -32371 32564 5084.02 508402 -128 127 -0.14 -14 -199 101 10189 99100 0.59759 297.59759 149.09759 14909.75975 0.59759 297.5976 149.09759 14909.75958 0.59759 297.59759 149.09759 14909.75900 2020-01-01 2020-01-02 2020-01-01 00:03:19 2020-01-02 03:31:40 2020-01-01 00:03:19.000 2020-01-02 03:31:40.000 199 99100 49649.5 4964950 199 99100 49649.5 4964950 -32370 32565 5085.02 508502 -128 124 -1.7 -170 -2 102 1001 9992 0.006 300.006 150.006 15150.6066 0.006 300.006 150.006 15150.6069 0.00600 300.00600 150.00600 15150.60600 2020-01-01 2020-01-02 2020-01-01 00:00:02 2020-01-02 03:45:02 2020-01-01 00:00:02.000 2020-01-02 03:45:02.000 2 99902 49952 5045152 2 99902 49952 5045152 -32567 32368 4531.009900990099 457632 -125 126 -0.9504950495049505 -96 -20 102 10010 99920 0.06006 300.06006 150.06006 15156.06606 0.06006 300.06006 150.06005 15156.06593 0.06006 300.06006 150.06006 15156.06606 2020-01-01 2020-01-02 2020-01-01 00:00:20 2020-01-02 03:45:20 2020-01-01 00:00:20.000 2020-01-02 03:45:20.000 20 99920 49970 5046970 20 99920 49970 5046970 -32549 32386 4549.009900990099 459450 -126 126 -0.693069306930693 -70 -200 101 10190 99101 0.6006 297.6006 149.1006 14910.06006 0.6006 297.6006 149.10059 14910.0599 0.60060 297.60060 149.10060 14910.06000 2020-01-01 2020-01-02 2020-01-01 00:03:20 2020-01-02 03:31:41 2020-01-01 00:03:20.000 2020-01-02 03:31:41.000 200 99101 49650.5 4965050 200 99101 49650.5 4965050 -32369 32566 5086.02 508602 -127 125 -0.7 -70 -201 101 10191 99102 0.6036 297.6036 149.1036 14910.36036 0.6036 297.6036 149.1036 14910.36063 0.60360 297.60360 149.10360 14910.36000 2020-01-01 2020-01-02 2020-01-01 00:03:21 2020-01-02 03:31:42 2020-01-01 00:03:21.000 2020-01-02 03:31:42.000 201 99102 49651.5 4965150 201 99102 49651.5 4965150 -32368 32567 5087.02 508702 -126 126 0.3 30 -202 101 10192 99103 0.6066 297.6066 149.1066 14910.66066 0.6066 297.6066 149.1066 14910.66078 0.60660 297.60660 149.10660 14910.66000 2020-01-01 2020-01-02 2020-01-01 00:03:22 2020-01-02 03:31:43 2020-01-01 00:03:22.000 2020-01-02 03:31:43.000 202 99103 49652.5 4965250 202 99103 49652.5 4965250 -32367 32568 5088.02 508802 -125 127 1.3 130 -203 101 10193 99104 0.6096 297.6096 149.1096 14910.96096 0.6096 297.60962 149.1096 14910.9609 0.60960 297.60960 149.10960 14910.96000 2020-01-01 2020-01-02 2020-01-01 00:03:23 2020-01-02 03:31:44 2020-01-01 00:03:23.000 2020-01-02 03:31:44.000 203 99104 49653.5 4965350 203 99104 49653.5 4965350 -32366 32569 5089.02 508902 -128 127 -0.26 -26 -204 101 10194 99105 0.61261 297.61261 149.11261 14911.26126 0.61261 297.6126 149.11261 14911.26105 0.61261 297.61261 149.11261 14911.26100 2020-01-01 2020-01-02 2020-01-01 00:03:24 2020-01-02 03:31:45 2020-01-01 00:03:24.000 2020-01-02 03:31:45.000 204 99105 49654.5 4965450 204 99105 49654.5 4965450 -32365 32570 5090.02 509002 -128 127 -1.82 -182 -205 101 10195 99106 0.61561 297.61561 149.11561 14911.56156 0.61561 297.6156 149.11561 14911.56137 0.61561 297.61561 149.11561 14911.56100 2020-01-01 2020-01-02 2020-01-01 00:03:25 2020-01-02 03:31:46 2020-01-01 00:03:25.000 2020-01-02 03:31:46.000 205 99106 49655.5 4965550 205 99106 49655.5 4965550 -32364 32571 5091.02 509102 -128 123 -3.38 -338 -206 101 10196 99107 0.61861 297.61861 149.11861 14911.86186 0.61861 297.61862 149.11862 14911.8621 0.61861 297.61861 149.11861 14911.86100 2020-01-01 2020-01-02 2020-01-01 00:03:26 2020-01-02 03:31:47 2020-01-01 00:03:26.000 2020-01-02 03:31:47.000 206 99107 49656.5 4965650 206 99107 49656.5 4965650 -32363 32572 5092.02 509202 -127 124 -2.38 -238 -207 101 10197 99108 0.62162 297.62162 149.12162 14912.16216 0.62162 297.6216 149.12162 14912.16225 0.62162 297.62162 149.12162 14912.16200 2020-01-01 2020-01-02 2020-01-01 00:03:27 2020-01-02 03:31:48 2020-01-01 00:03:27.000 2020-01-02 03:31:48.000 207 99108 49657.5 4965750 207 99108 49657.5 4965750 -32362 32573 5093.02 509302 -126 125 -1.38 -138 +198 101 10188 99099 0.59459 297.59459 149.09459 14909.45945 0.59459 297.5946 149.09459 14909.4594 0.59459 297.59459 149.09459000000004 14909.45900 2020-01-01 2020-01-02 2020-01-01 00:03:18 2020-01-02 03:31:39 2020-01-01 00:03:18.000 2020-01-02 03:31:39.000 198 99099 49648.5 4964850 198 99099 49648.5 4964850 -32371 32564 5084.02 508402 -128 127 -0.14 -14 +199 101 10189 99100 0.59759 297.59759 149.09759 14909.75975 0.59759 297.5976 149.09759 14909.75958 0.59759 297.59759 149.09758999999997 14909.75900 2020-01-01 2020-01-02 2020-01-01 00:03:19 2020-01-02 03:31:40 2020-01-01 00:03:19.000 2020-01-02 03:31:40.000 199 99100 49649.5 4964950 199 99100 49649.5 4964950 -32370 32565 5085.02 508502 -128 124 -1.7 -170 +2 102 1001 9992 0.006 300.006 150.006 15150.6066 0.006 300.006 150.006 15150.6069 0.00600 300.00600 150.00599999999997 15150.60600 2020-01-01 2020-01-02 2020-01-01 00:00:02 2020-01-02 03:45:02 2020-01-01 00:00:02.000 2020-01-02 03:45:02.000 2 99902 49952 5045152 2 99902 49952 5045152 -32567 32368 4531.009900990099 457632 -125 126 -0.9504950495049505 -96 +20 102 10010 99920 0.06006 300.06006 150.06006 15156.06606 0.06006 300.06006 150.06005 15156.06593 0.06006 300.06006 150.0600599999999 15156.06606 2020-01-01 2020-01-02 2020-01-01 00:00:20 2020-01-02 03:45:20 2020-01-01 00:00:20.000 2020-01-02 03:45:20.000 20 99920 49970 5046970 20 99920 49970 5046970 -32549 32386 4549.009900990099 459450 -126 126 -0.693069306930693 -70 +200 101 10190 99101 0.6006 297.6006 149.1006 14910.06006 0.6006 297.6006 149.10059 14910.0599 0.60060 297.60060 149.1005999999999 14910.06000 2020-01-01 2020-01-02 2020-01-01 00:03:20 2020-01-02 03:31:41 2020-01-01 00:03:20.000 2020-01-02 03:31:41.000 200 99101 49650.5 4965050 200 99101 49650.5 4965050 -32369 32566 5086.02 508602 -127 125 -0.7 -70 +201 101 10191 99102 0.6036 297.6036 149.1036 14910.36036 0.6036 297.6036 149.1036 14910.36063 0.60360 297.60360 149.10360000000009 14910.36000 2020-01-01 2020-01-02 2020-01-01 00:03:21 2020-01-02 03:31:42 2020-01-01 00:03:21.000 2020-01-02 03:31:42.000 201 99102 49651.5 4965150 201 99102 49651.5 4965150 -32368 32567 5087.02 508702 -126 126 0.3 30 +202 101 10192 99103 0.6066 297.6066 149.1066 14910.66066 0.6066 297.6066 149.1066 14910.66078 0.60660 297.60660 149.10659999999993 14910.66000 2020-01-01 2020-01-02 2020-01-01 00:03:22 2020-01-02 03:31:43 2020-01-01 00:03:22.000 2020-01-02 03:31:43.000 202 99103 49652.5 4965250 202 99103 49652.5 4965250 -32367 32568 5088.02 508802 -125 127 1.3 130 +203 101 10193 99104 0.6096 297.6096 149.1096 14910.96096 0.6096 297.60962 149.1096 14910.9609 0.60960 297.60960 149.1095999999999 14910.96000 2020-01-01 2020-01-02 2020-01-01 00:03:23 2020-01-02 03:31:44 2020-01-01 00:03:23.000 2020-01-02 03:31:44.000 203 99104 49653.5 4965350 203 99104 49653.5 4965350 -32366 32569 5089.02 508902 -128 127 -0.26 -26 +204 101 10194 99105 0.61261 297.61261 149.11261 14911.26126 0.61261 297.6126 149.11261 14911.26105 0.61261 297.61261 149.11261000000005 14911.26100 2020-01-01 2020-01-02 2020-01-01 00:03:24 2020-01-02 03:31:45 2020-01-01 00:03:24.000 2020-01-02 03:31:45.000 204 99105 49654.5 4965450 204 99105 49654.5 4965450 -32365 32570 5090.02 509002 -128 127 -1.82 -182 +205 101 10195 99106 0.61561 297.61561 149.11561 14911.56156 0.61561 297.6156 149.11561 14911.56137 0.61561 297.61561 149.11561000000003 14911.56100 2020-01-01 2020-01-02 2020-01-01 00:03:25 2020-01-02 03:31:46 2020-01-01 00:03:25.000 2020-01-02 03:31:46.000 205 99106 49655.5 4965550 205 99106 49655.5 4965550 -32364 32571 5091.02 509102 -128 123 -3.38 -338 +206 101 10196 99107 0.61861 297.61861 149.11861 14911.86186 0.61861 297.61862 149.11862 14911.8621 0.61861 297.61861 149.11860999999985 14911.86100 2020-01-01 2020-01-02 2020-01-01 00:03:26 2020-01-02 03:31:47 2020-01-01 00:03:26.000 2020-01-02 03:31:47.000 206 99107 49656.5 4965650 206 99107 49656.5 4965650 -32363 32572 5092.02 509202 -127 124 -2.38 -238 +207 101 10197 99108 0.62162 297.62162 149.12162 14912.16216 0.62162 297.6216 149.12162 14912.16225 0.62162 297.62162 149.12161999999992 14912.16200 2020-01-01 2020-01-02 2020-01-01 00:03:27 2020-01-02 03:31:48 2020-01-01 00:03:27.000 2020-01-02 03:31:48.000 207 99108 49657.5 4965750 207 99108 49657.5 4965750 -32362 32573 5093.02 509302 -126 125 -1.38 -138 208 101 10198 99109 0.62462 297.62462 149.12462 14912.46246 0.62462 297.62463 149.12462 14912.46237 0.62462 297.62462 149.12462 14912.46200 2020-01-01 2020-01-02 2020-01-01 00:03:28 2020-01-02 03:31:49 2020-01-01 00:03:28.000 2020-01-02 03:31:49.000 208 99109 49658.5 4965850 208 99109 49658.5 4965850 -32361 32574 5094.02 509402 -125 126 -0.38 -38 -209 101 10199 99110 0.62762 297.62762 149.12762 14912.76276 0.62762 297.62762 149.12762 14912.76253 0.62762 297.62762 149.12762 14912.76200 2020-01-01 2020-01-02 2020-01-01 00:03:29 2020-01-02 03:31:50 2020-01-01 00:03:29.000 2020-01-02 03:31:50.000 209 99110 49659.5 4965950 209 99110 49659.5 4965950 -32360 32575 5095.02 509502 -124 127 0.62 62 -21 102 10011 99921 0.06306 300.06306 150.06306 15156.36936 0.06306 300.06305 150.06306 15156.36927 0.06306 300.06306 150.06306 15156.36906 2020-01-01 2020-01-02 2020-01-01 00:00:21 2020-01-02 03:45:21 2020-01-01 00:00:21.000 2020-01-02 03:45:21.000 21 99921 49971 5047071 21 99921 49971 5047071 -32548 32387 4550.009900990099 459551 -125 127 0.3069306930693069 31 -210 101 10200 99111 0.63063 297.63063 149.13063 14913.06306 0.63063 297.63065 149.13063 14913.06326 0.63063 297.63063 149.13063 14913.06300 2020-01-01 2020-01-02 2020-01-01 00:03:30 2020-01-02 03:31:51 2020-01-01 00:03:30.000 2020-01-02 03:31:51.000 210 99111 49660.5 4966050 210 99111 49660.5 4966050 -32359 32576 5096.02 509602 -128 127 -0.94 -94 -211 101 10201 99112 0.63363 297.63363 149.13363 14913.36336 0.63363 297.63364 149.13363 14913.36357 0.63363 297.63363 149.13363 14913.36300 2020-01-01 2020-01-02 2020-01-01 00:03:31 2020-01-02 03:31:52 2020-01-01 00:03:31.000 2020-01-02 03:31:52.000 211 99112 49661.5 4966150 211 99112 49661.5 4966150 -32358 32577 5097.02 509702 -128 123 -2.5 -250 -212 101 10202 99113 0.63663 297.63663 149.13663 14913.66366 0.63663 297.63663 149.13663 14913.66372 0.63663 297.63663 149.13663 14913.66300 2020-01-01 2020-01-02 2020-01-01 00:03:32 2020-01-02 03:31:53 2020-01-01 00:03:32.000 2020-01-02 03:31:53.000 212 99113 49662.5 4966250 212 99113 49662.5 4966250 -32357 32578 5098.02 509802 -127 124 -1.5 -150 -213 101 10203 99114 0.63963 297.63963 149.13963 14913.96396 0.63963 297.63965 149.13963 14913.96384 0.63963 297.63963 149.13963 14913.96300 2020-01-01 2020-01-02 2020-01-01 00:03:33 2020-01-02 03:31:54 2020-01-01 00:03:33.000 2020-01-02 03:31:54.000 213 99114 49663.5 4966350 213 99114 49663.5 4966350 -32356 32579 5099.02 509902 -126 125 -0.5 -50 +209 101 10199 99110 0.62762 297.62762 149.12762 14912.76276 0.62762 297.62762 149.12762 14912.76253 0.62762 297.62762 149.12761999999998 14912.76200 2020-01-01 2020-01-02 2020-01-01 00:03:29 2020-01-02 03:31:50 2020-01-01 00:03:29.000 2020-01-02 03:31:50.000 209 99110 49659.5 4965950 209 99110 49659.5 4965950 -32360 32575 5095.02 509502 -124 127 0.62 62 +21 102 10011 99921 0.06306 300.06306 150.06306 15156.36936 0.06306 300.06305 150.06306 15156.36927 0.06306 300.06306 150.06306000000012 15156.36906 2020-01-01 2020-01-02 2020-01-01 00:00:21 2020-01-02 03:45:21 2020-01-01 00:00:21.000 2020-01-02 03:45:21.000 21 99921 49971 5047071 21 99921 49971 5047071 -32548 32387 4550.009900990099 459551 -125 127 0.3069306930693069 31 +210 101 10200 99111 0.63063 297.63063 149.13063 14913.06306 0.63063 297.63065 149.13063 14913.06326 0.63063 297.63063 149.13062999999994 14913.06300 2020-01-01 2020-01-02 2020-01-01 00:03:30 2020-01-02 03:31:51 2020-01-01 00:03:30.000 2020-01-02 03:31:51.000 210 99111 49660.5 4966050 210 99111 49660.5 4966050 -32359 32576 5096.02 509602 -128 127 -0.94 -94 +211 101 10201 99112 0.63363 297.63363 149.13363 14913.36336 0.63363 297.63364 149.13363 14913.36357 0.63363 297.63363 149.13363000000004 14913.36300 2020-01-01 2020-01-02 2020-01-01 00:03:31 2020-01-02 03:31:52 2020-01-01 00:03:31.000 2020-01-02 03:31:52.000 211 99112 49661.5 4966150 211 99112 49661.5 4966150 -32358 32577 5097.02 509702 -128 123 -2.5 -250 +212 101 10202 99113 0.63663 297.63663 149.13663 14913.66366 0.63663 297.63663 149.13663 14913.66372 0.63663 297.63663 149.13663000000008 14913.66300 2020-01-01 2020-01-02 2020-01-01 00:03:32 2020-01-02 03:31:53 2020-01-01 00:03:32.000 2020-01-02 03:31:53.000 212 99113 49662.5 4966250 212 99113 49662.5 4966250 -32357 32578 5098.02 509802 -127 124 -1.5 -150 +213 101 10203 99114 0.63963 297.63963 149.13963 14913.96396 0.63963 297.63965 149.13963 14913.96384 0.63963 297.63963 149.1396299999999 14913.96300 2020-01-01 2020-01-02 2020-01-01 00:03:33 2020-01-02 03:31:54 2020-01-01 00:03:33.000 2020-01-02 03:31:54.000 213 99114 49663.5 4966350 213 99114 49663.5 4966350 -32356 32579 5099.02 509902 -126 125 -0.5 -50 214 101 10204 99115 0.64264 297.64264 149.14264 14914.26426 0.64264 297.64264 149.14263 14914.26399 0.64264 297.64264 149.14264 14914.26400 2020-01-01 2020-01-02 2020-01-01 00:03:34 2020-01-02 03:31:55 2020-01-01 00:03:34.000 2020-01-02 03:31:55.000 214 99115 49664.5 4966450 214 99115 49664.5 4966450 -32355 32580 5100.02 510002 -125 126 0.5 50 -215 101 10205 99116 0.64564 297.64564 149.14564 14914.56456 0.64564 297.64566 149.14564 14914.56473 0.64564 297.64564 149.14564 14914.56400 2020-01-01 2020-01-02 2020-01-01 00:03:35 2020-01-02 03:31:56 2020-01-01 00:03:35.000 2020-01-02 03:31:56.000 215 99116 49665.5 4966550 215 99116 49665.5 4966550 -32354 32581 5101.02 510102 -124 127 1.5 150 -216 101 10206 99117 0.64864 297.64864 149.14864 14914.86486 0.64864 297.64865 149.14865 14914.86504 0.64864 297.64864 149.14864 14914.86400 2020-01-01 2020-01-02 2020-01-01 00:03:36 2020-01-02 03:31:57 2020-01-01 00:03:36.000 2020-01-02 03:31:57.000 216 99117 49666.5 4966650 216 99117 49666.5 4966650 -32353 32582 5102.02 510202 -128 127 -0.06 -6 -217 101 10207 99118 0.65165 297.65165 149.15165 14915.16516 0.65165 297.65164 149.15165 14915.16523 0.65165 297.65165 149.15165 14915.16500 2020-01-01 2020-01-02 2020-01-01 00:03:37 2020-01-02 03:31:58 2020-01-01 00:03:37.000 2020-01-02 03:31:58.000 217 99118 49667.5 4966750 217 99118 49667.5 4966750 -32352 32583 5103.02 510302 -128 123 -1.62 -162 -218 101 10208 99119 0.65465 297.65465 149.15465 14915.46546 0.65465 297.65466 149.15465 14915.46531 0.65465 297.65465 149.15465 14915.46500 2020-01-01 2020-01-02 2020-01-01 00:03:38 2020-01-02 03:31:59 2020-01-01 00:03:38.000 2020-01-02 03:31:59.000 218 99119 49668.5 4966850 218 99119 49668.5 4966850 -32351 32584 5104.02 510402 -127 124 -0.62 -62 -219 101 10209 99120 0.65765 297.65765 149.15765 14915.76576 0.65765 297.65765 149.15765 14915.76562 0.65765 297.65765 149.15765 14915.76500 2020-01-01 2020-01-02 2020-01-01 00:03:39 2020-01-02 03:32:00 2020-01-01 00:03:39.000 2020-01-02 03:32:00.000 219 99120 49669.5 4966950 219 99120 49669.5 4966950 -32350 32585 5105.02 510502 -126 125 0.38 38 -22 102 10012 99922 0.06606 300.06606 150.06606 15156.67267 0.06606 300.06607 150.06606 15156.67287 0.06606 300.06606 150.06606 15156.67206 2020-01-01 2020-01-02 2020-01-01 00:00:22 2020-01-02 03:45:22 2020-01-01 00:00:22.000 2020-01-02 03:45:22.000 22 99922 49972 5047172 22 99922 49972 5047172 -32547 32388 4551.009900990099 459652 -128 127 -1.2277227722772277 -124 -220 101 10210 99121 0.66066 297.66066 149.16066 14916.06606 0.66066 297.66068 149.16066 14916.06619 0.66066 297.66066 149.16066 14916.06600 2020-01-01 2020-01-02 2020-01-01 00:03:40 2020-01-02 03:32:01 2020-01-01 00:03:40.000 2020-01-02 03:32:01.000 220 99121 49670.5 4967050 220 99121 49670.5 4967050 -32349 32586 5106.02 510602 -125 126 1.38 138 -221 101 10211 99122 0.66366 297.66366 149.16366 14916.36636 0.66366 297.66367 149.16366 14916.36651 0.66366 297.66366 149.16366 14916.36600 2020-01-01 2020-01-02 2020-01-01 00:03:41 2020-01-02 03:32:02 2020-01-01 00:03:41.000 2020-01-02 03:32:02.000 221 99122 49671.5 4967150 221 99122 49671.5 4967150 -32348 32587 5107.02 510702 -124 127 2.38 238 -222 101 10212 99123 0.66666 297.66666 149.16666 14916.66666 0.66666 297.66666 149.16666 14916.6667 0.66666 297.66666 149.16666 14916.66600 2020-01-01 2020-01-02 2020-01-01 00:03:42 2020-01-02 03:32:03 2020-01-01 00:03:42.000 2020-01-02 03:32:03.000 222 99123 49672.5 4967250 222 99123 49672.5 4967250 -32347 32588 5108.02 510802 -128 127 0.82 82 -223 101 10213 99124 0.66966 297.66966 149.16966 14916.96696 0.66966 297.66968 149.16966 14916.96678 0.66966 297.66966 149.16966 14916.96600 2020-01-01 2020-01-02 2020-01-01 00:03:43 2020-01-02 03:32:04 2020-01-01 00:03:43.000 2020-01-02 03:32:04.000 223 99124 49673.5 4967350 223 99124 49673.5 4967350 -32346 32589 5109.02 510902 -128 127 -0.74 -74 -224 101 10214 99125 0.67267 297.67267 149.17267 14917.26726 0.67267 297.67267 149.17267 14917.26709 0.67267 297.67267 149.17267 14917.26700 2020-01-01 2020-01-02 2020-01-01 00:03:44 2020-01-02 03:32:05 2020-01-01 00:03:44.000 2020-01-02 03:32:05.000 224 99125 49674.5 4967450 224 99125 49674.5 4967450 -32345 32590 5110.02 511002 -128 124 -2.3 -230 +215 101 10205 99116 0.64564 297.64564 149.14564 14914.56456 0.64564 297.64566 149.14564 14914.56473 0.64564 297.64564 149.14564000000007 14914.56400 2020-01-01 2020-01-02 2020-01-01 00:03:35 2020-01-02 03:31:56 2020-01-01 00:03:35.000 2020-01-02 03:31:56.000 215 99116 49665.5 4966550 215 99116 49665.5 4966550 -32354 32581 5101.02 510102 -124 127 1.5 150 +216 101 10206 99117 0.64864 297.64864 149.14864 14914.86486 0.64864 297.64865 149.14865 14914.86504 0.64864 297.64864 149.14863999999997 14914.86400 2020-01-01 2020-01-02 2020-01-01 00:03:36 2020-01-02 03:31:57 2020-01-01 00:03:36.000 2020-01-02 03:31:57.000 216 99117 49666.5 4966650 216 99117 49666.5 4966650 -32353 32582 5102.02 510202 -128 127 -0.06 -6 +217 101 10207 99118 0.65165 297.65165 149.15165 14915.16516 0.65165 297.65164 149.15165 14915.16523 0.65165 297.65165 149.1516499999999 14915.16500 2020-01-01 2020-01-02 2020-01-01 00:03:37 2020-01-02 03:31:58 2020-01-01 00:03:37.000 2020-01-02 03:31:58.000 217 99118 49667.5 4966750 217 99118 49667.5 4966750 -32352 32583 5103.02 510302 -128 123 -1.62 -162 +218 101 10208 99119 0.65465 297.65465 149.15465 14915.46546 0.65465 297.65466 149.15465 14915.46531 0.65465 297.65465 149.15465000000012 14915.46500 2020-01-01 2020-01-02 2020-01-01 00:03:38 2020-01-02 03:31:59 2020-01-01 00:03:38.000 2020-01-02 03:31:59.000 218 99119 49668.5 4966850 218 99119 49668.5 4966850 -32351 32584 5104.02 510402 -127 124 -0.62 -62 +219 101 10209 99120 0.65765 297.65765 149.15765 14915.76576 0.65765 297.65765 149.15765 14915.76562 0.65765 297.65765 149.15764999999993 14915.76500 2020-01-01 2020-01-02 2020-01-01 00:03:39 2020-01-02 03:32:00 2020-01-01 00:03:39.000 2020-01-02 03:32:00.000 219 99120 49669.5 4966950 219 99120 49669.5 4966950 -32350 32585 5105.02 510502 -126 125 0.38 38 +22 102 10012 99922 0.06606 300.06606 150.06606 15156.67267 0.06606 300.06607 150.06606 15156.67287 0.06606 300.06606 150.06605999999996 15156.67206 2020-01-01 2020-01-02 2020-01-01 00:00:22 2020-01-02 03:45:22 2020-01-01 00:00:22.000 2020-01-02 03:45:22.000 22 99922 49972 5047172 22 99922 49972 5047172 -32547 32388 4551.009900990099 459652 -128 127 -1.2277227722772277 -124 +220 101 10210 99121 0.66066 297.66066 149.16066 14916.06606 0.66066 297.66068 149.16066 14916.06619 0.66066 297.66066 149.16065999999984 14916.06600 2020-01-01 2020-01-02 2020-01-01 00:03:40 2020-01-02 03:32:01 2020-01-01 00:03:40.000 2020-01-02 03:32:01.000 220 99121 49670.5 4967050 220 99121 49670.5 4967050 -32349 32586 5106.02 510602 -125 126 1.38 138 +221 101 10211 99122 0.66366 297.66366 149.16366 14916.36636 0.66366 297.66367 149.16366 14916.36651 0.66366 297.66366 149.16366000000008 14916.36600 2020-01-01 2020-01-02 2020-01-01 00:03:41 2020-01-02 03:32:02 2020-01-01 00:03:41.000 2020-01-02 03:32:02.000 221 99122 49671.5 4967150 221 99122 49671.5 4967150 -32348 32587 5107.02 510702 -124 127 2.38 238 +222 101 10212 99123 0.66666 297.66666 149.16666 14916.66666 0.66666 297.66666 149.16666 14916.6667 0.66666 297.66666 149.16666000000004 14916.66600 2020-01-01 2020-01-02 2020-01-01 00:03:42 2020-01-02 03:32:03 2020-01-01 00:03:42.000 2020-01-02 03:32:03.000 222 99123 49672.5 4967250 222 99123 49672.5 4967250 -32347 32588 5108.02 510802 -128 127 0.82 82 +223 101 10213 99124 0.66966 297.66966 149.16966 14916.96696 0.66966 297.66968 149.16966 14916.96678 0.66966 297.66966 149.16965999999985 14916.96600 2020-01-01 2020-01-02 2020-01-01 00:03:43 2020-01-02 03:32:04 2020-01-01 00:03:43.000 2020-01-02 03:32:04.000 223 99124 49673.5 4967350 223 99124 49673.5 4967350 -32346 32589 5109.02 510902 -128 127 -0.74 -74 +224 101 10214 99125 0.67267 297.67267 149.17267 14917.26726 0.67267 297.67267 149.17267 14917.26709 0.67267 297.67267 149.17266999999995 14917.26700 2020-01-01 2020-01-02 2020-01-01 00:03:44 2020-01-02 03:32:05 2020-01-01 00:03:44.000 2020-01-02 03:32:05.000 224 99125 49674.5 4967450 224 99125 49674.5 4967450 -32345 32590 5110.02 511002 -128 124 -2.3 -230 225 101 10215 99126 0.67567 297.67567 149.17567 14917.56756 0.67567 297.6757 149.17567 14917.56767 0.67567 297.67567 149.17567 14917.56700 2020-01-01 2020-01-02 2020-01-01 00:03:45 2020-01-02 03:32:06 2020-01-01 00:03:45.000 2020-01-02 03:32:06.000 225 99126 49675.5 4967550 225 99126 49675.5 4967550 -32344 32591 5111.02 511102 -127 125 -1.3 -130 -226 101 10216 99127 0.67867 297.67867 149.17867 14917.86786 0.67867 297.67868 149.17868 14917.86802 0.67867 297.67867 149.17867 14917.86700 2020-01-01 2020-01-02 2020-01-01 00:03:46 2020-01-02 03:32:07 2020-01-01 00:03:46.000 2020-01-02 03:32:07.000 226 99127 49676.5 4967650 226 99127 49676.5 4967650 -32343 32592 5112.02 511202 -126 126 -0.3 -30 -227 101 10217 99128 0.68168 297.68168 149.18168 14918.16816 0.68168 297.68167 149.18168 14918.16817 0.68168 297.68168 149.18168 14918.16800 2020-01-01 2020-01-02 2020-01-01 00:03:47 2020-01-02 03:32:08 2020-01-01 00:03:47.000 2020-01-02 03:32:08.000 227 99128 49677.5 4967750 227 99128 49677.5 4967750 -32342 32593 5113.02 511302 -125 127 0.7 70 -228 101 10218 99129 0.68468 297.68468 149.18468 14918.46846 0.68468 297.6847 149.18468 14918.46825 0.68468 297.68468 149.18468 14918.46800 2020-01-01 2020-01-02 2020-01-01 00:03:48 2020-01-02 03:32:09 2020-01-01 00:03:48.000 2020-01-02 03:32:09.000 228 99129 49678.5 4967850 228 99129 49678.5 4967850 -32341 32594 5114.02 511402 -128 127 -0.86 -86 -229 101 10219 99130 0.68768 297.68768 149.18768 14918.76876 0.68768 297.68768 149.18768 14918.76855 0.68768 297.68768 149.18768 14918.76800 2020-01-01 2020-01-02 2020-01-01 00:03:49 2020-01-02 03:32:10 2020-01-01 00:03:49.000 2020-01-02 03:32:10.000 229 99130 49679.5 4967950 229 99130 49679.5 4967950 -32340 32595 5115.02 511502 -128 127 -2.42 -242 -23 102 10013 99923 0.06906 300.06906 150.06906 15156.97597 0.06906 300.06906 150.06907 15156.97617 0.06906 300.06906 150.06906 15156.97506 2020-01-01 2020-01-02 2020-01-01 00:00:23 2020-01-02 03:45:23 2020-01-01 00:00:23.000 2020-01-02 03:45:23.000 23 99923 49973 5047273 23 99923 49973 5047273 -32546 32389 4552.009900990099 459753 -128 127 -2.762376237623762 -279 +226 101 10216 99127 0.67867 297.67867 149.17867 14917.86786 0.67867 297.67868 149.17868 14917.86802 0.67867 297.67867 149.17866999999998 14917.86700 2020-01-01 2020-01-02 2020-01-01 00:03:46 2020-01-02 03:32:07 2020-01-01 00:03:46.000 2020-01-02 03:32:07.000 226 99127 49676.5 4967650 226 99127 49676.5 4967650 -32343 32592 5112.02 511202 -126 126 -0.3 -30 +227 101 10217 99128 0.68168 297.68168 149.18168 14918.16816 0.68168 297.68167 149.18168 14918.16817 0.68168 297.68168 149.18167999999991 14918.16800 2020-01-01 2020-01-02 2020-01-01 00:03:47 2020-01-02 03:32:08 2020-01-01 00:03:47.000 2020-01-02 03:32:08.000 227 99128 49677.5 4967750 227 99128 49677.5 4967750 -32342 32593 5113.02 511302 -125 127 0.7 70 +228 101 10218 99129 0.68468 297.68468 149.18468 14918.46846 0.68468 297.6847 149.18468 14918.46825 0.68468 297.68468 149.18468000000007 14918.46800 2020-01-01 2020-01-02 2020-01-01 00:03:48 2020-01-02 03:32:09 2020-01-01 00:03:48.000 2020-01-02 03:32:09.000 228 99129 49678.5 4967850 228 99129 49678.5 4967850 -32341 32594 5114.02 511402 -128 127 -0.86 -86 +229 101 10219 99130 0.68768 297.68768 149.18768 14918.76876 0.68768 297.68768 149.18768 14918.76855 0.68768 297.68768 149.1876799999999 14918.76800 2020-01-01 2020-01-02 2020-01-01 00:03:49 2020-01-02 03:32:10 2020-01-01 00:03:49.000 2020-01-02 03:32:10.000 229 99130 49679.5 4967950 229 99130 49679.5 4967950 -32340 32595 5115.02 511502 -128 127 -2.42 -242 +23 102 10013 99923 0.06906 300.06906 150.06906 15156.97597 0.06906 300.06906 150.06907 15156.97617 0.06906 300.06906 150.06905999999992 15156.97506 2020-01-01 2020-01-02 2020-01-01 00:00:23 2020-01-02 03:45:23 2020-01-01 00:00:23.000 2020-01-02 03:45:23.000 23 99923 49973 5047273 23 99923 49973 5047273 -32546 32389 4552.009900990099 459753 -128 127 -2.762376237623762 -279 230 101 10220 99131 0.69069 297.69069 149.19069 14919.06906 0.69069 297.6907 149.19069 14919.06914 0.69069 297.69069 149.19069 14919.06900 2020-01-01 2020-01-02 2020-01-01 00:03:50 2020-01-02 03:32:11 2020-01-01 00:03:50.000 2020-01-02 03:32:11.000 230 99131 49680.5 4968050 230 99131 49680.5 4968050 -32339 32596 5116.02 511602 -128 123 -3.98 -398 -231 101 10221 99132 0.69369 297.69369 149.19369 14919.36936 0.69369 297.6937 149.19369 14919.36949 0.69369 297.69369 149.19369 14919.36900 2020-01-01 2020-01-02 2020-01-01 00:03:51 2020-01-02 03:32:12 2020-01-01 00:03:51.000 2020-01-02 03:32:12.000 231 99132 49681.5 4968150 231 99132 49681.5 4968150 -32338 32597 5117.02 511702 -127 124 -2.98 -298 -232 101 10222 99133 0.69669 297.69669 149.19669 14919.66966 0.69669 297.6967 149.19669 14919.66964 0.69669 297.69669 149.19669 14919.66900 2020-01-01 2020-01-02 2020-01-01 00:03:52 2020-01-02 03:32:13 2020-01-01 00:03:52.000 2020-01-02 03:32:13.000 232 99133 49682.5 4968250 232 99133 49682.5 4968250 -32337 32598 5118.02 511802 -126 125 -1.98 -198 -233 101 10223 99134 0.69969 297.69969 149.19969 14919.96996 0.69969 297.6997 149.1997 14919.97037 0.69969 297.69969 149.19969 14919.96900 2020-01-01 2020-01-02 2020-01-01 00:03:53 2020-01-02 03:32:14 2020-01-01 00:03:53.000 2020-01-02 03:32:14.000 233 99134 49683.5 4968350 233 99134 49683.5 4968350 -32336 32599 5119.02 511902 -125 126 -0.98 -98 -234 101 10224 99135 0.7027 297.7027 149.2027 14920.27027 0.7027 297.7027 149.2027 14920.27003 0.70270 297.70270 149.20270 14920.27000 2020-01-01 2020-01-02 2020-01-01 00:03:54 2020-01-02 03:32:15 2020-01-01 00:03:54.000 2020-01-02 03:32:15.000 234 99135 49684.5 4968450 234 99135 49684.5 4968450 -32335 32600 5120.02 512002 -124 127 0.02 2 -235 101 10225 99136 0.7057 297.7057 149.2057 14920.57057 0.7057 297.70572 149.2057 14920.57065 0.70570 297.70570 149.20570 14920.57000 2020-01-01 2020-01-02 2020-01-01 00:03:55 2020-01-02 03:32:16 2020-01-01 00:03:55.000 2020-01-02 03:32:16.000 235 99136 49685.5 4968550 235 99136 49685.5 4968550 -32334 32601 5121.02 512102 -128 127 -1.54 -154 -236 101 10226 99137 0.7087 297.7087 149.2087 14920.87087 0.7087 297.7087 149.2087 14920.87095 0.70870 297.70870 149.20870 14920.87000 2020-01-01 2020-01-02 2020-01-01 00:03:56 2020-01-02 03:32:17 2020-01-01 00:03:56.000 2020-01-02 03:32:17.000 236 99137 49686.5 4968650 236 99137 49686.5 4968650 -32333 32602 5122.02 512202 -128 123 -3.1 -310 -237 101 10227 99138 0.71171 297.71171 149.21171 14921.17117 0.71171 297.7117 149.21171 14921.17111 0.71171 297.71171 149.21171 14921.17100 2020-01-01 2020-01-02 2020-01-01 00:03:57 2020-01-02 03:32:18 2020-01-01 00:03:57.000 2020-01-02 03:32:18.000 237 99138 49687.5 4968750 237 99138 49687.5 4968750 -32332 32603 5123.02 512302 -127 124 -2.1 -210 -238 101 10228 99139 0.71471 297.71471 149.21471 14921.47147 0.71471 297.71472 149.21471 14921.47184 0.71471 297.71471 149.21471 14921.47100 2020-01-01 2020-01-02 2020-01-01 00:03:58 2020-01-02 03:32:19 2020-01-01 00:03:58.000 2020-01-02 03:32:19.000 238 99139 49688.5 4968850 238 99139 49688.5 4968850 -32331 32604 5124.02 512402 -126 125 -1.1 -110 -239 101 10229 99140 0.71771 297.71771 149.21771 14921.77177 0.71771 297.7177 149.21771 14921.7715 0.71771 297.71771 149.21771 14921.77100 2020-01-01 2020-01-02 2020-01-01 00:03:59 2020-01-02 03:32:20 2020-01-01 00:03:59.000 2020-01-02 03:32:20.000 239 99140 49689.5 4968950 239 99140 49689.5 4968950 -32330 32605 5125.02 512502 -125 126 -0.1 -10 -24 102 10014 99924 0.07207 300.07207 150.07207 15157.27927 0.07207 300.07208 150.07207 15157.27928 0.07207 300.07207 150.07207 15157.27907 2020-01-01 2020-01-02 2020-01-01 00:00:24 2020-01-02 03:45:24 2020-01-01 00:00:24.000 2020-01-02 03:45:24.000 24 99924 49974 5047374 24 99924 49974 5047374 -32545 32390 4553.009900990099 459854 -128 123 -4.297029702970297 -434 -240 101 10230 99141 0.72072 297.72072 149.22072 14922.07207 0.72072 297.72073 149.22072 14922.07211 0.72072 297.72072 149.22072 14922.07200 2020-01-01 2020-01-02 2020-01-01 00:04:00 2020-01-02 03:32:21 2020-01-01 00:04:00.000 2020-01-02 03:32:21.000 240 99141 49690.5 4969050 240 99141 49690.5 4969050 -32329 32606 5126.02 512602 -124 127 0.9 90 +231 101 10221 99132 0.69369 297.69369 149.19369 14919.36936 0.69369 297.6937 149.19369 14919.36949 0.69369 297.69369 149.19369000000003 14919.36900 2020-01-01 2020-01-02 2020-01-01 00:03:51 2020-01-02 03:32:12 2020-01-01 00:03:51.000 2020-01-02 03:32:12.000 231 99132 49681.5 4968150 231 99132 49681.5 4968150 -32338 32597 5117.02 511702 -127 124 -2.98 -298 +232 101 10222 99133 0.69669 297.69669 149.19669 14919.66966 0.69669 297.6967 149.19669 14919.66964 0.69669 297.69669 149.19669000000002 14919.66900 2020-01-01 2020-01-02 2020-01-01 00:03:52 2020-01-02 03:32:13 2020-01-01 00:03:52.000 2020-01-02 03:32:13.000 232 99133 49682.5 4968250 232 99133 49682.5 4968250 -32337 32598 5118.02 511802 -126 125 -1.98 -198 +233 101 10223 99134 0.69969 297.69969 149.19969 14919.96996 0.69969 297.6997 149.1997 14919.97037 0.69969 297.69969 149.19968999999998 14919.96900 2020-01-01 2020-01-02 2020-01-01 00:03:53 2020-01-02 03:32:14 2020-01-01 00:03:53.000 2020-01-02 03:32:14.000 233 99134 49683.5 4968350 233 99134 49683.5 4968350 -32336 32599 5119.02 511902 -125 126 -0.98 -98 +234 101 10224 99135 0.7027 297.7027 149.2027 14920.27027 0.7027 297.7027 149.2027 14920.27003 0.70270 297.70270 149.2026999999999 14920.27000 2020-01-01 2020-01-02 2020-01-01 00:03:54 2020-01-02 03:32:15 2020-01-01 00:03:54.000 2020-01-02 03:32:15.000 234 99135 49684.5 4968450 234 99135 49684.5 4968450 -32335 32600 5120.02 512002 -124 127 0.02 2 +235 101 10225 99136 0.7057 297.7057 149.2057 14920.57057 0.7057 297.70572 149.2057 14920.57065 0.70570 297.70570 149.20570000000015 14920.57000 2020-01-01 2020-01-02 2020-01-01 00:03:55 2020-01-02 03:32:16 2020-01-01 00:03:55.000 2020-01-02 03:32:16.000 235 99136 49685.5 4968550 235 99136 49685.5 4968550 -32334 32601 5121.02 512102 -128 127 -1.54 -154 +236 101 10226 99137 0.7087 297.7087 149.2087 14920.87087 0.7087 297.7087 149.2087 14920.87095 0.70870 297.70870 149.20869999999996 14920.87000 2020-01-01 2020-01-02 2020-01-01 00:03:56 2020-01-02 03:32:17 2020-01-01 00:03:56.000 2020-01-02 03:32:17.000 236 99137 49686.5 4968650 236 99137 49686.5 4968650 -32333 32602 5122.02 512202 -128 123 -3.1 -310 +237 101 10227 99138 0.71171 297.71171 149.21171 14921.17117 0.71171 297.7117 149.21171 14921.17111 0.71171 297.71171 149.21170999999987 14921.17100 2020-01-01 2020-01-02 2020-01-01 00:03:57 2020-01-02 03:32:18 2020-01-01 00:03:57.000 2020-01-02 03:32:18.000 237 99138 49687.5 4968750 237 99138 49687.5 4968750 -32332 32603 5123.02 512302 -127 124 -2.1 -210 +238 101 10228 99139 0.71471 297.71471 149.21471 14921.47147 0.71471 297.71472 149.21471 14921.47184 0.71471 297.71471 149.21471000000008 14921.47100 2020-01-01 2020-01-02 2020-01-01 00:03:58 2020-01-02 03:32:19 2020-01-01 00:03:58.000 2020-01-02 03:32:19.000 238 99139 49688.5 4968850 238 99139 49688.5 4968850 -32331 32604 5124.02 512402 -126 125 -1.1 -110 +239 101 10229 99140 0.71771 297.71771 149.21771 14921.77177 0.71771 297.7177 149.21771 14921.7715 0.71771 297.71771 149.21771000000007 14921.77100 2020-01-01 2020-01-02 2020-01-01 00:03:59 2020-01-02 03:32:20 2020-01-01 00:03:59.000 2020-01-02 03:32:20.000 239 99140 49689.5 4968950 239 99140 49689.5 4968950 -32330 32605 5125.02 512502 -125 126 -0.1 -10 +24 102 10014 99924 0.07207 300.07207 150.07207 15157.27927 0.07207 300.07208 150.07207 15157.27928 0.07207 300.07207 150.07207000000008 15157.27907 2020-01-01 2020-01-02 2020-01-01 00:00:24 2020-01-02 03:45:24 2020-01-01 00:00:24.000 2020-01-02 03:45:24.000 24 99924 49974 5047374 24 99924 49974 5047374 -32545 32390 4553.009900990099 459854 -128 123 -4.297029702970297 -434 +240 101 10230 99141 0.72072 297.72072 149.22072 14922.07207 0.72072 297.72073 149.22072 14922.07211 0.72072 297.72072 149.22071999999994 14922.07200 2020-01-01 2020-01-02 2020-01-01 00:04:00 2020-01-02 03:32:21 2020-01-01 00:04:00.000 2020-01-02 03:32:21.000 240 99141 49690.5 4969050 240 99141 49690.5 4969050 -32329 32606 5126.02 512602 -124 127 0.9 90 241 101 10231 99142 0.72372 297.72372 149.22372 14922.37237 0.72372 297.72372 149.22372 14922.37243 0.72372 297.72372 149.22372 14922.37200 2020-01-01 2020-01-02 2020-01-01 00:04:01 2020-01-02 03:32:22 2020-01-01 00:04:01.000 2020-01-02 03:32:22.000 241 99142 49691.5 4969150 241 99142 49691.5 4969150 -32328 32607 5127.02 512702 -128 127 -0.66 -66 -242 101 10232 99143 0.72672 297.72672 149.22672 14922.67267 0.72672 297.7267 149.22672 14922.67273 0.72672 297.72672 149.22672 14922.67200 2020-01-01 2020-01-02 2020-01-01 00:04:02 2020-01-02 03:32:23 2020-01-01 00:04:02.000 2020-01-02 03:32:23.000 242 99143 49692.5 4969250 242 99143 49692.5 4969250 -32327 32608 5128.02 512802 -128 123 -2.22 -222 -243 101 10233 99144 0.72972 297.72972 149.22972 14922.97297 0.72972 297.72974 149.22973 14922.97332 0.72972 297.72972 149.22972 14922.97200 2020-01-01 2020-01-02 2020-01-01 00:04:03 2020-01-02 03:32:24 2020-01-01 00:04:03.000 2020-01-02 03:32:24.000 243 99144 49693.5 4969350 243 99144 49693.5 4969350 -32326 32609 5129.02 512902 -127 124 -1.22 -122 -244 101 10234 99145 0.73273 297.73273 149.23273 14923.27327 0.73273 297.73273 149.23272 14923.27297 0.73273 297.73273 149.23273 14923.27300 2020-01-01 2020-01-02 2020-01-01 00:04:04 2020-01-02 03:32:25 2020-01-01 00:04:04.000 2020-01-02 03:32:25.000 244 99145 49694.5 4969450 244 99145 49694.5 4969450 -32325 32610 5130.02 513002 -126 125 -0.22 -22 -245 101 10235 99146 0.73573 297.73573 149.23573 14923.57357 0.73573 297.73575 149.23573 14923.57358 0.73573 297.73573 149.23573 14923.57300 2020-01-01 2020-01-02 2020-01-01 00:04:05 2020-01-02 03:32:26 2020-01-01 00:04:05.000 2020-01-02 03:32:26.000 245 99146 49695.5 4969550 245 99146 49695.5 4969550 -32324 32611 5131.02 513102 -125 126 0.78 78 -246 101 10236 99147 0.73873 297.73873 149.23873 14923.87387 0.73873 297.73874 149.23873 14923.8739 0.73873 297.73873 149.23873 14923.87300 2020-01-01 2020-01-02 2020-01-01 00:04:06 2020-01-02 03:32:27 2020-01-01 00:04:06.000 2020-01-02 03:32:27.000 246 99147 49696.5 4969650 246 99147 49696.5 4969650 -32323 32612 5132.02 513202 -124 127 1.78 178 +242 101 10232 99143 0.72672 297.72672 149.22672 14922.67267 0.72672 297.7267 149.22672 14922.67273 0.72672 297.72672 149.22672000000003 14922.67200 2020-01-01 2020-01-02 2020-01-01 00:04:02 2020-01-02 03:32:23 2020-01-01 00:04:02.000 2020-01-02 03:32:23.000 242 99143 49692.5 4969250 242 99143 49692.5 4969250 -32327 32608 5128.02 512802 -128 123 -2.22 -222 +243 101 10233 99144 0.72972 297.72972 149.22972 14922.97297 0.72972 297.72974 149.22973 14922.97332 0.72972 297.72972 149.22971999999993 14922.97200 2020-01-01 2020-01-02 2020-01-01 00:04:03 2020-01-02 03:32:24 2020-01-01 00:04:03.000 2020-01-02 03:32:24.000 243 99144 49693.5 4969350 243 99144 49693.5 4969350 -32326 32609 5129.02 512902 -127 124 -1.22 -122 +244 101 10234 99145 0.73273 297.73273 149.23273 14923.27327 0.73273 297.73273 149.23272 14923.27297 0.73273 297.73273 149.23272999999995 14923.27300 2020-01-01 2020-01-02 2020-01-01 00:04:04 2020-01-02 03:32:25 2020-01-01 00:04:04.000 2020-01-02 03:32:25.000 244 99145 49694.5 4969450 244 99145 49694.5 4969450 -32325 32610 5130.02 513002 -126 125 -0.22 -22 +245 101 10235 99146 0.73573 297.73573 149.23573 14923.57357 0.73573 297.73575 149.23573 14923.57358 0.73573 297.73573 149.2357300000001 14923.57300 2020-01-01 2020-01-02 2020-01-01 00:04:05 2020-01-02 03:32:26 2020-01-01 00:04:05.000 2020-01-02 03:32:26.000 245 99146 49695.5 4969550 245 99146 49695.5 4969550 -32324 32611 5131.02 513102 -125 126 0.78 78 +246 101 10236 99147 0.73873 297.73873 149.23873 14923.87387 0.73873 297.73874 149.23873 14923.8739 0.73873 297.73873 149.23872999999992 14923.87300 2020-01-01 2020-01-02 2020-01-01 00:04:06 2020-01-02 03:32:27 2020-01-01 00:04:06.000 2020-01-02 03:32:27.000 246 99147 49696.5 4969650 246 99147 49696.5 4969650 -32323 32612 5132.02 513202 -124 127 1.78 178 247 101 10237 99148 0.74174 297.74174 149.24174 14924.17417 0.74174 297.74173 149.24174 14924.1742 0.74174 297.74174 149.24174 14924.17400 2020-01-01 2020-01-02 2020-01-01 00:04:07 2020-01-02 03:32:28 2020-01-01 00:04:07.000 2020-01-02 03:32:28.000 247 99148 49697.5 4969750 247 99148 49697.5 4969750 -32322 32613 5133.02 513302 -128 127 0.22 22 -248 101 10238 99149 0.74474 297.74474 149.24474 14924.47447 0.74474 297.74475 149.24474 14924.47478 0.74474 297.74474 149.24474 14924.47400 2020-01-01 2020-01-02 2020-01-01 00:04:08 2020-01-02 03:32:29 2020-01-01 00:04:08.000 2020-01-02 03:32:29.000 248 99149 49698.5 4969850 248 99149 49698.5 4969850 -32321 32614 5134.02 513402 -128 127 -1.34 -134 -249 101 10239 99150 0.74774 297.74774 149.24774 14924.77477 0.74774 297.74774 149.24774 14924.77447 0.74774 297.74774 149.24774 14924.77400 2020-01-01 2020-01-02 2020-01-01 00:04:09 2020-01-02 03:32:30 2020-01-01 00:04:09.000 2020-01-02 03:32:30.000 249 99150 49699.5 4969950 249 99150 49699.5 4969950 -32320 32615 5135.02 513502 -128 124 -2.9 -290 -25 102 10015 99925 0.07507 300.07507 150.07507 15157.58258 0.07507 300.07507 150.07507 15157.58241 0.07507 300.07507 150.07507 15157.58207 2020-01-01 2020-01-02 2020-01-01 00:00:25 2020-01-02 03:45:25 2020-01-01 00:00:25.000 2020-01-02 03:45:25.000 25 99925 49975 5047475 25 99925 49975 5047475 -32544 32391 4554.009900990099 459955 -127 124 -3.297029702970297 -333 -250 101 10240 99151 0.75075 297.75075 149.25075 14925.07507 0.75075 297.75076 149.25075 14925.07506 0.75075 297.75075 149.25075 14925.07500 2020-01-01 2020-01-02 2020-01-01 00:04:10 2020-01-02 03:32:31 2020-01-01 00:04:10.000 2020-01-02 03:32:31.000 250 99151 49700.5 4970050 250 99151 49700.5 4970050 -32319 32616 5136.02 513602 -127 125 -1.9 -190 -251 101 10241 99152 0.75375 297.75375 149.25375 14925.37537 0.75375 297.75375 149.25375 14925.37536 0.75375 297.75375 149.25375 14925.37500 2020-01-01 2020-01-02 2020-01-01 00:04:11 2020-01-02 03:32:32 2020-01-01 00:04:11.000 2020-01-02 03:32:32.000 251 99152 49701.5 4970150 251 99152 49701.5 4970150 -32318 32617 5137.02 513702 -126 126 -0.9 -90 -252 101 10242 99153 0.75675 297.75675 149.25675 14925.67567 0.75675 297.75674 149.25675 14925.67567 0.75675 297.75675 149.25675 14925.67500 2020-01-01 2020-01-02 2020-01-01 00:04:12 2020-01-02 03:32:33 2020-01-01 00:04:12.000 2020-01-02 03:32:33.000 252 99153 49702.5 4970250 252 99153 49702.5 4970250 -32317 32618 5138.02 513802 -125 127 0.1 10 -253 101 10243 99154 0.75975 297.75975 149.25975 14925.97597 0.75975 297.75977 149.25976 14925.97625 0.75975 297.75975 149.25975 14925.97500 2020-01-01 2020-01-02 2020-01-01 00:04:13 2020-01-02 03:32:34 2020-01-01 00:04:13.000 2020-01-02 03:32:34.000 253 99154 49703.5 4970350 253 99154 49703.5 4970350 -32316 32619 5139.02 513902 -128 127 -1.46 -146 -254 101 10244 99155 0.76276 297.76276 149.26276 14926.27627 0.76276 297.76276 149.26275 14926.27594 0.76276 297.76276 149.26276 14926.27600 2020-01-01 2020-01-02 2020-01-01 00:04:14 2020-01-02 03:32:35 2020-01-01 00:04:14.000 2020-01-02 03:32:35.000 254 99155 49704.5 4970450 254 99155 49704.5 4970450 -32315 32620 5140.02 514002 -128 127 -3.02 -302 -255 101 10245 99156 0.76576 297.76576 149.26576 14926.57657 0.76576 297.76578 149.26576 14926.57652 0.76576 297.76576 149.26576 14926.57600 2020-01-01 2020-01-02 2020-01-01 00:04:15 2020-01-02 03:32:36 2020-01-01 00:04:15.000 2020-01-02 03:32:36.000 255 99156 49705.5 4970550 255 99156 49705.5 4970550 -32314 32621 5141.02 514102 -128 123 -4.58 -458 -256 101 10246 99157 0.76876 297.76876 149.26876 14926.87687 0.76876 297.76877 149.26876 14926.87683 0.76876 297.76876 149.26876 14926.87600 2020-01-01 2020-01-02 2020-01-01 00:04:16 2020-01-02 03:32:37 2020-01-01 00:04:16.000 2020-01-02 03:32:37.000 256 99157 49706.5 4970650 256 99157 49706.5 4970650 -32313 32622 5142.02 514202 -127 124 -3.58 -358 -257 101 10247 99158 0.77177 297.77177 149.27177 14927.17717 0.77177 297.77176 149.27177 14927.17714 0.77177 297.77177 149.27177 14927.17700 2020-01-01 2020-01-02 2020-01-01 00:04:17 2020-01-02 03:32:38 2020-01-01 00:04:17.000 2020-01-02 03:32:38.000 257 99158 49707.5 4970750 257 99158 49707.5 4970750 -32312 32623 5143.02 514302 -126 125 -2.58 -258 +248 101 10238 99149 0.74474 297.74474 149.24474 14924.47447 0.74474 297.74475 149.24474 14924.47478 0.74474 297.74474 149.24474000000006 14924.47400 2020-01-01 2020-01-02 2020-01-01 00:04:08 2020-01-02 03:32:29 2020-01-01 00:04:08.000 2020-01-02 03:32:29.000 248 99149 49698.5 4969850 248 99149 49698.5 4969850 -32321 32614 5134.02 513402 -128 127 -1.34 -134 +249 101 10239 99150 0.74774 297.74774 149.24774 14924.77477 0.74774 297.74774 149.24774 14924.77447 0.74774 297.74774 149.24774000000002 14924.77400 2020-01-01 2020-01-02 2020-01-01 00:04:09 2020-01-02 03:32:30 2020-01-01 00:04:09.000 2020-01-02 03:32:30.000 249 99150 49699.5 4969950 249 99150 49699.5 4969950 -32320 32615 5135.02 513502 -128 124 -2.9 -290 +25 102 10015 99925 0.07507 300.07507 150.07507 15157.58258 0.07507 300.07507 150.07507 15157.58241 0.07507 300.07507 150.07507000000004 15157.58207 2020-01-01 2020-01-02 2020-01-01 00:00:25 2020-01-02 03:45:25 2020-01-01 00:00:25.000 2020-01-02 03:45:25.000 25 99925 49975 5047475 25 99925 49975 5047475 -32544 32391 4554.009900990099 459955 -127 124 -3.297029702970297 -333 +250 101 10240 99151 0.75075 297.75075 149.25075 14925.07507 0.75075 297.75076 149.25075 14925.07506 0.75075 297.75075 149.25074999999998 14925.07500 2020-01-01 2020-01-02 2020-01-01 00:04:10 2020-01-02 03:32:31 2020-01-01 00:04:10.000 2020-01-02 03:32:31.000 250 99151 49700.5 4970050 250 99151 49700.5 4970050 -32319 32616 5136.02 513602 -127 125 -1.9 -190 +251 101 10241 99152 0.75375 297.75375 149.25375 14925.37537 0.75375 297.75375 149.25375 14925.37536 0.75375 297.75375 149.25374999999994 14925.37500 2020-01-01 2020-01-02 2020-01-01 00:04:11 2020-01-02 03:32:32 2020-01-01 00:04:11.000 2020-01-02 03:32:32.000 251 99152 49701.5 4970150 251 99152 49701.5 4970150 -32318 32617 5137.02 513702 -126 126 -0.9 -90 +252 101 10242 99153 0.75675 297.75675 149.25675 14925.67567 0.75675 297.75674 149.25675 14925.67567 0.75675 297.75675 149.25675000000015 14925.67500 2020-01-01 2020-01-02 2020-01-01 00:04:12 2020-01-02 03:32:33 2020-01-01 00:04:12.000 2020-01-02 03:32:33.000 252 99153 49702.5 4970250 252 99153 49702.5 4970250 -32317 32618 5138.02 513802 -125 127 0.1 10 +253 101 10243 99154 0.75975 297.75975 149.25975 14925.97597 0.75975 297.75977 149.25976 14925.97625 0.75975 297.75975 149.25974999999997 14925.97500 2020-01-01 2020-01-02 2020-01-01 00:04:13 2020-01-02 03:32:34 2020-01-01 00:04:13.000 2020-01-02 03:32:34.000 253 99154 49703.5 4970350 253 99154 49703.5 4970350 -32316 32619 5139.02 513902 -128 127 -1.46 -146 +254 101 10244 99155 0.76276 297.76276 149.26276 14926.27627 0.76276 297.76276 149.26275 14926.27594 0.76276 297.76276 149.2627599999999 14926.27600 2020-01-01 2020-01-02 2020-01-01 00:04:14 2020-01-02 03:32:35 2020-01-01 00:04:14.000 2020-01-02 03:32:35.000 254 99155 49704.5 4970450 254 99155 49704.5 4970450 -32315 32620 5140.02 514002 -128 127 -3.02 -302 +255 101 10245 99156 0.76576 297.76576 149.26576 14926.57657 0.76576 297.76578 149.26576 14926.57652 0.76576 297.76576 149.2657600000001 14926.57600 2020-01-01 2020-01-02 2020-01-01 00:04:15 2020-01-02 03:32:36 2020-01-01 00:04:15.000 2020-01-02 03:32:36.000 255 99156 49705.5 4970550 255 99156 49705.5 4970550 -32314 32621 5141.02 514102 -128 123 -4.58 -458 +256 101 10246 99157 0.76876 297.76876 149.26876 14926.87687 0.76876 297.76877 149.26876 14926.87683 0.76876 297.76876 149.26876000000007 14926.87600 2020-01-01 2020-01-02 2020-01-01 00:04:16 2020-01-02 03:32:37 2020-01-01 00:04:16.000 2020-01-02 03:32:37.000 256 99157 49706.5 4970650 256 99157 49706.5 4970650 -32313 32622 5142.02 514202 -127 124 -3.58 -358 +257 101 10247 99158 0.77177 297.77177 149.27177 14927.17717 0.77177 297.77176 149.27177 14927.17714 0.77177 297.77177 149.27176999999995 14927.17700 2020-01-01 2020-01-02 2020-01-01 00:04:17 2020-01-02 03:32:38 2020-01-01 00:04:17.000 2020-01-02 03:32:38.000 257 99158 49707.5 4970750 257 99158 49707.5 4970750 -32312 32623 5143.02 514302 -126 125 -2.58 -258 258 101 10248 99159 0.77477 297.77477 149.27477 14927.47747 0.77477 297.77478 149.27477 14927.47776 0.77477 297.77477 149.27477 14927.47700 2020-01-01 2020-01-02 2020-01-01 00:04:18 2020-01-02 03:32:39 2020-01-01 00:04:18.000 2020-01-02 03:32:39.000 258 99159 49708.5 4970850 258 99159 49708.5 4970850 -32311 32624 5144.02 514402 -125 126 -1.58 -158 -259 101 10249 99160 0.77777 297.77777 149.27777 14927.77777 0.77777 297.77777 149.27777 14927.77742 0.77777 297.77777 149.27777 14927.77700 2020-01-01 2020-01-02 2020-01-01 00:04:19 2020-01-02 03:32:40 2020-01-01 00:04:19.000 2020-01-02 03:32:40.000 259 99160 49709.5 4970950 259 99160 49709.5 4970950 -32310 32625 5145.02 514502 -124 127 -0.58 -58 -26 102 10016 99926 0.07807 300.07807 150.07807 15157.88588 0.07807 300.07806 150.07807 15157.88575 0.07807 300.07807 150.07807 15157.88507 2020-01-01 2020-01-02 2020-01-01 00:00:26 2020-01-02 03:45:26 2020-01-01 00:00:26.000 2020-01-02 03:45:26.000 26 99926 49976 5047576 26 99926 49976 5047576 -32543 32392 4555.009900990099 460056 -126 125 -2.297029702970297 -232 -260 101 10250 99161 0.78078 297.78078 149.28078 14928.07807 0.78078 297.7808 149.28077 14928.07799 0.78078 297.78078 149.28078 14928.07800 2020-01-01 2020-01-02 2020-01-01 00:04:20 2020-01-02 03:32:41 2020-01-01 00:04:20.000 2020-01-02 03:32:41.000 260 99161 49710.5 4971050 260 99161 49710.5 4971050 -32309 32626 5146.02 514602 -128 127 -2.14 -214 -261 101 10251 99162 0.78378 297.78378 149.28378 14928.37837 0.78378 297.78378 149.28378 14928.3783 0.78378 297.78378 149.28378 14928.37800 2020-01-01 2020-01-02 2020-01-01 00:04:21 2020-01-02 03:32:42 2020-01-01 00:04:21.000 2020-01-02 03:32:42.000 261 99162 49711.5 4971150 261 99162 49711.5 4971150 -32308 32627 5147.02 514702 -128 123 -3.7 -370 -262 101 10252 99163 0.78678 297.78678 149.28678 14928.67867 0.78678 297.78677 149.28678 14928.67861 0.78678 297.78678 149.28678 14928.67800 2020-01-01 2020-01-02 2020-01-01 00:04:22 2020-01-02 03:32:43 2020-01-01 00:04:22.000 2020-01-02 03:32:43.000 262 99163 49712.5 4971250 262 99163 49712.5 4971250 -32307 32628 5148.02 514802 -127 124 -2.7 -270 -263 101 10253 99164 0.78978 297.78978 149.28978 14928.97897 0.78978 297.7898 149.28979 14928.97923 0.78978 297.78978 149.28978 14928.97800 2020-01-01 2020-01-02 2020-01-01 00:04:23 2020-01-02 03:32:44 2020-01-01 00:04:23.000 2020-01-02 03:32:44.000 263 99164 49713.5 4971350 263 99164 49713.5 4971350 -32306 32629 5149.02 514902 -126 125 -1.7 -170 -264 101 10254 99165 0.79279 297.79279 149.29279 14929.27927 0.79279 297.7928 149.29278 14929.27888 0.79279 297.79279 149.29279 14929.27900 2020-01-01 2020-01-02 2020-01-01 00:04:24 2020-01-02 03:32:45 2020-01-01 00:04:24.000 2020-01-02 03:32:45.000 264 99165 49714.5 4971450 264 99165 49714.5 4971450 -32305 32630 5150.02 515002 -125 126 -0.7 -70 -265 101 10255 99166 0.79579 297.79579 149.29579 14929.57957 0.79579 297.7958 149.29579 14929.57962 0.79579 297.79579 149.29579 14929.57900 2020-01-01 2020-01-02 2020-01-01 00:04:25 2020-01-02 03:32:46 2020-01-01 00:04:25.000 2020-01-02 03:32:46.000 265 99166 49715.5 4971550 265 99166 49715.5 4971550 -32304 32631 5151.02 515102 -124 127 0.3 30 -266 101 10256 99167 0.79879 297.79879 149.29879 14929.87987 0.79879 297.7988 149.29879 14929.87977 0.79879 297.79879 149.29879 14929.87900 2020-01-01 2020-01-02 2020-01-01 00:04:26 2020-01-02 03:32:47 2020-01-01 00:04:26.000 2020-01-02 03:32:47.000 266 99167 49716.5 4971650 266 99167 49716.5 4971650 -32303 32632 5152.02 515202 -128 127 -1.26 -126 -267 101 10257 99168 0.8018 297.8018 149.3018 14930.18018 0.8018 297.8018 149.3018 14930.18012 0.80180 297.80180 149.30180 14930.18000 2020-01-01 2020-01-02 2020-01-01 00:04:27 2020-01-02 03:32:48 2020-01-01 00:04:27.000 2020-01-02 03:32:48.000 267 99168 49717.5 4971750 267 99168 49717.5 4971750 -32302 32633 5153.02 515302 -128 123 -2.82 -282 -268 101 10258 99169 0.8048 297.8048 149.3048 14930.48048 0.8048 297.8048 149.3048 14930.4807 0.80480 297.80480 149.30480 14930.48000 2020-01-01 2020-01-02 2020-01-01 00:04:28 2020-01-02 03:32:49 2020-01-01 00:04:28.000 2020-01-02 03:32:49.000 268 99169 49718.5 4971850 268 99169 49718.5 4971850 -32301 32634 5154.02 515402 -127 124 -1.82 -182 -269 101 10259 99170 0.8078 297.8078 149.3078 14930.78078 0.8078 297.8078 149.3078 14930.78035 0.80780 297.80780 149.30780 14930.78000 2020-01-01 2020-01-02 2020-01-01 00:04:29 2020-01-02 03:32:50 2020-01-01 00:04:29.000 2020-01-02 03:32:50.000 269 99170 49719.5 4971950 269 99170 49719.5 4971950 -32300 32635 5155.02 515502 -126 125 -0.82 -82 -27 102 10017 99927 0.08108 300.08108 150.08108 15158.18918 0.08108 300.0811 150.08108 15158.18936 0.08108 300.08108 150.08108 15158.18908 2020-01-01 2020-01-02 2020-01-01 00:00:27 2020-01-02 03:45:27 2020-01-01 00:00:27.000 2020-01-02 03:45:27.000 27 99927 49977 5047677 27 99927 49977 5047677 -32542 32393 4556.009900990099 460157 -125 126 -1.297029702970297 -131 -270 101 10260 99171 0.81081 297.81081 149.31081 14931.08108 0.81081 297.81082 149.31081 14931.08109 0.81081 297.81081 149.31081 14931.08100 2020-01-01 2020-01-02 2020-01-01 00:04:30 2020-01-02 03:32:51 2020-01-01 00:04:30.000 2020-01-02 03:32:51.000 270 99171 49720.5 4972050 270 99171 49720.5 4972050 -32299 32636 5156.02 515602 -125 126 0.18 18 -271 101 10261 99172 0.81381 297.81381 149.31381 14931.38138 0.81381 297.8138 149.31381 14931.38124 0.81381 297.81381 149.31381 14931.38100 2020-01-01 2020-01-02 2020-01-01 00:04:31 2020-01-02 03:32:52 2020-01-01 00:04:31.000 2020-01-02 03:32:52.000 271 99172 49721.5 4972150 271 99172 49721.5 4972150 -32298 32637 5157.02 515702 -124 127 1.18 118 -272 101 10262 99173 0.81681 297.81681 149.31681 14931.68168 0.81681 297.8168 149.31681 14931.68159 0.81681 297.81681 149.31681 14931.68100 2020-01-01 2020-01-02 2020-01-01 00:04:32 2020-01-02 03:32:53 2020-01-01 00:04:32.000 2020-01-02 03:32:53.000 272 99173 49722.5 4972250 272 99173 49722.5 4972250 -32297 32638 5158.02 515802 -128 127 -0.38 -38 -273 101 10263 99174 0.81981 297.81981 149.31981 14931.98198 0.81981 297.81982 149.31982 14931.98217 0.81981 297.81981 149.31981 14931.98100 2020-01-01 2020-01-02 2020-01-01 00:04:33 2020-01-02 03:32:54 2020-01-01 00:04:33.000 2020-01-02 03:32:54.000 273 99174 49723.5 4972350 273 99174 49723.5 4972350 -32296 32639 5159.02 515902 -128 127 -1.94 -194 -274 101 10264 99175 0.82282 297.82282 149.32282 14932.28228 0.82282 297.8228 149.32282 14932.28247 0.82282 297.82282 149.32282 14932.28200 2020-01-01 2020-01-02 2020-01-01 00:04:34 2020-01-02 03:32:55 2020-01-01 00:04:34.000 2020-01-02 03:32:55.000 274 99175 49724.5 4972450 274 99175 49724.5 4972450 -32295 32640 5160.02 516002 -128 124 -3.5 -350 -275 101 10265 99176 0.82582 297.82582 149.32582 14932.58258 0.82582 297.82584 149.32582 14932.58256 0.82582 297.82582 149.32582 14932.58200 2020-01-01 2020-01-02 2020-01-01 00:04:35 2020-01-02 03:32:56 2020-01-01 00:04:35.000 2020-01-02 03:32:56.000 275 99176 49725.5 4972550 275 99176 49725.5 4972550 -32294 32641 5161.02 516102 -127 125 -2.5 -250 -276 101 10266 99177 0.82882 297.82882 149.32882 14932.88288 0.82882 297.82883 149.32882 14932.88275 0.82882 297.82882 149.32882 14932.88200 2020-01-01 2020-01-02 2020-01-01 00:04:36 2020-01-02 03:32:57 2020-01-01 00:04:36.000 2020-01-02 03:32:57.000 276 99177 49726.5 4972650 276 99177 49726.5 4972650 -32293 32642 5162.02 516202 -126 126 -1.5 -150 -277 101 10267 99178 0.83183 297.83183 149.33183 14933.18318 0.83183 297.83182 149.33183 14933.18305 0.83183 297.83183 149.33183 14933.18300 2020-01-01 2020-01-02 2020-01-01 00:04:37 2020-01-02 03:32:58 2020-01-01 00:04:37.000 2020-01-02 03:32:58.000 277 99178 49727.5 4972750 277 99178 49727.5 4972750 -32292 32643 5163.02 516302 -125 127 -0.5 -50 -278 101 10268 99179 0.83483 297.83483 149.33483 14933.48348 0.83483 297.83484 149.33483 14933.48364 0.83483 297.83483 149.33483 14933.48300 2020-01-01 2020-01-02 2020-01-01 00:04:38 2020-01-02 03:32:59 2020-01-01 00:04:38.000 2020-01-02 03:32:59.000 278 99179 49728.5 4972850 278 99179 49728.5 4972850 -32291 32644 5164.02 516402 -128 127 -2.06 -206 -279 101 10269 99180 0.83783 297.83783 149.33783 14933.78378 0.83783 297.83783 149.33783 14933.78394 0.83783 297.83783 149.33783 14933.78300 2020-01-01 2020-01-02 2020-01-01 00:04:39 2020-01-02 03:33:00 2020-01-01 00:04:39.000 2020-01-02 03:33:00.000 279 99180 49729.5 4972950 279 99180 49729.5 4972950 -32290 32645 5165.02 516502 -128 127 -3.62 -362 -28 102 10018 99928 0.08408 300.08408 150.08408 15158.49249 0.08408 300.08408 150.08408 15158.49265 0.08408 300.08408 150.08408 15158.49208 2020-01-01 2020-01-02 2020-01-01 00:00:28 2020-01-02 03:45:28 2020-01-01 00:00:28.000 2020-01-02 03:45:28.000 28 99928 49978 5047778 28 99928 49978 5047778 -32541 32394 4557.009900990099 460258 -124 127 -0.297029702970297 -30 -280 101 10270 99181 0.84084 297.84084 149.34084 14934.08408 0.84084 297.84085 149.34084 14934.08403 0.84084 297.84084 149.34084 14934.08400 2020-01-01 2020-01-02 2020-01-01 00:04:40 2020-01-02 03:33:01 2020-01-01 00:04:40.000 2020-01-02 03:33:01.000 280 99181 49730.5 4973050 280 99181 49730.5 4973050 -32289 32646 5166.02 516602 -128 123 -5.18 -518 -281 101 10271 99182 0.84384 297.84384 149.34384 14934.38438 0.84384 297.84384 149.34384 14934.38421 0.84384 297.84384 149.34384 14934.38400 2020-01-01 2020-01-02 2020-01-01 00:04:41 2020-01-02 03:33:02 2020-01-01 00:04:41.000 2020-01-02 03:33:02.000 281 99182 49731.5 4973150 281 99182 49731.5 4973150 -32288 32647 5167.02 516702 -127 124 -4.18 -418 -282 101 10272 99183 0.84684 297.84684 149.34684 14934.68468 0.84684 297.84683 149.34684 14934.68453 0.84684 297.84684 149.34684 14934.68400 2020-01-01 2020-01-02 2020-01-01 00:04:42 2020-01-02 03:33:03 2020-01-01 00:04:42.000 2020-01-02 03:33:03.000 282 99183 49732.5 4973250 282 99183 49732.5 4973250 -32287 32648 5168.02 516802 -126 125 -3.18 -318 -283 101 10273 99184 0.84984 297.84984 149.34984 14934.98498 0.84984 297.84985 149.34985 14934.98526 0.84984 297.84984 149.34984 14934.98400 2020-01-01 2020-01-02 2020-01-01 00:04:43 2020-01-02 03:33:04 2020-01-01 00:04:43.000 2020-01-02 03:33:04.000 283 99184 49733.5 4973350 283 99184 49733.5 4973350 -32286 32649 5169.02 516902 -125 126 -2.18 -218 -284 101 10274 99185 0.85285 297.85285 149.35285 14935.28528 0.85285 297.85284 149.35285 14935.28542 0.85285 297.85285 149.35285 14935.28500 2020-01-01 2020-01-02 2020-01-01 00:04:44 2020-01-02 03:33:05 2020-01-01 00:04:44.000 2020-01-02 03:33:05.000 284 99185 49734.5 4973450 284 99185 49734.5 4973450 -32285 32650 5170.02 517002 -124 127 -1.18 -118 -285 101 10275 99186 0.85585 297.85585 149.35585 14935.58558 0.85585 297.85587 149.35585 14935.5855 0.85585 297.85585 149.35585 14935.58500 2020-01-01 2020-01-02 2020-01-01 00:04:45 2020-01-02 03:33:06 2020-01-01 00:04:45.000 2020-01-02 03:33:06.000 285 99186 49735.5 4973550 285 99186 49735.5 4973550 -32284 32651 5171.02 517102 -128 127 -2.74 -274 -286 101 10276 99187 0.85885 297.85885 149.35885 14935.88588 0.85885 297.85886 149.35885 14935.88568 0.85885 297.85885 149.35885 14935.88500 2020-01-01 2020-01-02 2020-01-01 00:04:46 2020-01-02 03:33:07 2020-01-01 00:04:46.000 2020-01-02 03:33:07.000 286 99187 49736.5 4973650 286 99187 49736.5 4973650 -32283 32652 5172.02 517202 -128 123 -4.3 -430 -287 101 10277 99188 0.86186 297.86186 149.36186 14936.18618 0.86186 297.86185 149.36186 14936.186 0.86186 297.86186 149.36186 14936.18600 2020-01-01 2020-01-02 2020-01-01 00:04:47 2020-01-02 03:33:08 2020-01-01 00:04:47.000 2020-01-02 03:33:08.000 287 99188 49737.5 4973750 287 99188 49737.5 4973750 -32282 32653 5173.02 517302 -127 124 -3.3 -330 -288 101 10278 99189 0.86486 297.86486 149.36486 14936.48648 0.86486 297.86487 149.36486 14936.48673 0.86486 297.86486 149.36486 14936.48600 2020-01-01 2020-01-02 2020-01-01 00:04:48 2020-01-02 03:33:09 2020-01-01 00:04:48.000 2020-01-02 03:33:09.000 288 99189 49738.5 4973850 288 99189 49738.5 4973850 -32281 32654 5174.02 517402 -126 125 -2.3 -230 -289 101 10279 99190 0.86786 297.86786 149.36786 14936.78678 0.86786 297.86786 149.36786 14936.78688 0.86786 297.86786 149.36786 14936.78600 2020-01-01 2020-01-02 2020-01-01 00:04:49 2020-01-02 03:33:10 2020-01-01 00:04:49.000 2020-01-02 03:33:10.000 289 99190 49739.5 4973950 289 99190 49739.5 4973950 -32280 32655 5175.02 517502 -125 126 -1.3 -130 -29 102 10019 99929 0.08708 300.08708 150.08708 15158.79579 0.08708 300.0871 150.08708 15158.79576 0.08708 300.08708 150.08708 15158.79508 2020-01-01 2020-01-02 2020-01-01 00:00:29 2020-01-02 03:45:29 2020-01-01 00:00:29.000 2020-01-02 03:45:29.000 29 99929 49979 5047879 29 99929 49979 5047879 -32540 32395 4558.009900990099 460359 -128 127 -1.8316831683168318 -185 -290 101 10280 99191 0.87087 297.87087 149.37087 14937.08708 0.87087 297.87088 149.37087 14937.087 0.87087 297.87087 149.37087 14937.08700 2020-01-01 2020-01-02 2020-01-01 00:04:50 2020-01-02 03:33:11 2020-01-01 00:04:50.000 2020-01-02 03:33:11.000 290 99191 49740.5 4974050 290 99191 49740.5 4974050 -32279 32656 5176.02 517602 -124 127 -0.3 -30 -291 101 10281 99192 0.87387 297.87387 149.37387 14937.38738 0.87387 297.87387 149.37387 14937.38716 0.87387 297.87387 149.37387 14937.38700 2020-01-01 2020-01-02 2020-01-01 00:04:51 2020-01-02 03:33:12 2020-01-01 00:04:51.000 2020-01-02 03:33:12.000 291 99192 49741.5 4974150 291 99192 49741.5 4974150 -32278 32657 5177.02 517702 -128 127 -1.86 -186 -292 101 10282 99193 0.87687 297.87687 149.37687 14937.68768 0.87687 297.8769 149.37687 14937.68789 0.87687 297.87687 149.37687 14937.68700 2020-01-01 2020-01-02 2020-01-01 00:04:52 2020-01-02 03:33:13 2020-01-01 00:04:52.000 2020-01-02 03:33:13.000 292 99193 49742.5 4974250 292 99193 49742.5 4974250 -32277 32658 5178.02 517802 -128 123 -3.42 -342 -293 101 10283 99194 0.87987 297.87987 149.37987 14937.98798 0.87987 297.87988 149.37988 14937.9882 0.87987 297.87987 149.37987 14937.98700 2020-01-01 2020-01-02 2020-01-01 00:04:53 2020-01-02 03:33:14 2020-01-01 00:04:53.000 2020-01-02 03:33:14.000 293 99194 49743.5 4974350 293 99194 49743.5 4974350 -32276 32659 5179.02 517902 -127 124 -2.42 -242 -294 101 10284 99195 0.88288 297.88288 149.38288 14938.28828 0.88288 297.88287 149.38288 14938.28835 0.88288 297.88288 149.38288 14938.28800 2020-01-01 2020-01-02 2020-01-01 00:04:54 2020-01-02 03:33:15 2020-01-01 00:04:54.000 2020-01-02 03:33:15.000 294 99195 49744.5 4974450 294 99195 49744.5 4974450 -32275 32660 5180.02 518002 -126 125 -1.42 -142 -295 101 10285 99196 0.88588 297.88588 149.38588 14938.58858 0.88588 297.8859 149.38588 14938.58847 0.88588 297.88588 149.38588 14938.58800 2020-01-01 2020-01-02 2020-01-01 00:04:55 2020-01-02 03:33:16 2020-01-01 00:04:55.000 2020-01-02 03:33:16.000 295 99196 49745.5 4974550 295 99196 49745.5 4974550 -32274 32661 5181.02 518102 -125 126 -0.42 -42 -296 101 10286 99197 0.88888 297.88888 149.38888 14938.88888 0.88888 297.8889 149.38888 14938.88863 0.88888 297.88888 149.38888 14938.88800 2020-01-01 2020-01-02 2020-01-01 00:04:56 2020-01-02 03:33:17 2020-01-01 00:04:56.000 2020-01-02 03:33:17.000 296 99197 49746.5 4974650 296 99197 49746.5 4974650 -32273 32662 5182.02 518202 -124 127 0.58 58 -297 101 10287 99198 0.89189 297.89189 149.39189 14939.18918 0.89189 297.8919 149.39189 14939.18936 0.89189 297.89189 149.39189 14939.18900 2020-01-01 2020-01-02 2020-01-01 00:04:57 2020-01-02 03:33:18 2020-01-01 00:04:57.000 2020-01-02 03:33:18.000 297 99198 49747.5 4974750 297 99198 49747.5 4974750 -32272 32663 5183.02 518302 -128 127 -0.98 -98 -298 101 10288 99199 0.89489 297.89489 149.39489 14939.48948 0.89489 297.8949 149.39489 14939.48967 0.89489 297.89489 149.39489 14939.48900 2020-01-01 2020-01-02 2020-01-01 00:04:58 2020-01-02 03:33:19 2020-01-01 00:04:58.000 2020-01-02 03:33:19.000 298 99199 49748.5 4974850 298 99199 49748.5 4974850 -32271 32664 5184.02 518402 -128 127 -2.54 -254 -299 101 10289 99200 0.89789 297.89789 149.39789 14939.78978 0.89789 297.8979 149.39789 14939.78986 0.89789 297.89789 149.39789 14939.78900 2020-01-01 2020-01-02 2020-01-01 00:04:59 2020-01-02 03:33:20 2020-01-01 00:04:59.000 2020-01-02 03:33:20.000 299 99200 49749.5 4974950 299 99200 49749.5 4974950 -32270 32665 5185.02 518502 -128 124 -4.1 -410 -3 102 1002 9993 0.009 300.009 150.009 15150.9099 0.009 300.009 150.009 15150.90958 0.00900 300.00900 150.00900 15150.90900 2020-01-01 2020-01-02 2020-01-01 00:00:03 2020-01-02 03:45:03 2020-01-01 00:00:03.000 2020-01-02 03:45:03.000 3 99903 49953 5045253 3 99903 49953 5045253 -32566 32369 4532.009900990099 457733 -124 127 0.04950495049504951 5 -30 102 10020 99930 0.09009 300.09009 150.09009 15159.09909 0.09009 300.0901 150.09008 15159.09894 0.09009 300.09009 150.09009 15159.09909 2020-01-01 2020-01-02 2020-01-01 00:00:30 2020-01-02 03:45:30 2020-01-01 00:00:30.000 2020-01-02 03:45:30.000 30 99930 49980 5047980 30 99930 49980 5047980 -32539 32396 4559.009900990099 460460 -128 123 -3.366336633663366 -340 -300 101 10290 99201 0.9009 297.9009 149.4009 14940.09009 0.9009 297.9009 149.40089 14940.08995 0.90090 297.90090 149.40090 14940.09000 2020-01-01 2020-01-02 2020-01-01 00:05:00 2020-01-02 03:33:21 2020-01-01 00:05:00.000 2020-01-02 03:33:21.000 300 99201 49750.5 4975050 300 99201 49750.5 4975050 -32269 32666 5186.02 518602 -127 125 -3.1 -310 -301 101 10291 99202 0.9039 297.9039 149.4039 14940.39039 0.9039 297.9039 149.4039 14940.39009 0.90390 297.90390 149.40390 14940.39000 2020-01-01 2020-01-02 2020-01-01 00:05:01 2020-01-02 03:33:22 2020-01-01 00:05:01.000 2020-01-02 03:33:22.000 301 99202 49751.5 4975150 301 99202 49751.5 4975150 -32268 32667 5187.02 518702 -126 126 -2.1 -210 -302 101 10292 99203 0.9069 297.9069 149.4069 14940.69069 0.9069 297.90692 149.4069 14940.69083 0.90690 297.90690 149.40690 14940.69000 2020-01-01 2020-01-02 2020-01-01 00:05:02 2020-01-02 03:33:23 2020-01-01 00:05:02.000 2020-01-02 03:33:23.000 302 99203 49752.5 4975250 302 99203 49752.5 4975250 -32267 32668 5188.02 518802 -125 127 -1.1 -110 -303 101 10293 99204 0.9099 297.9099 149.4099 14940.99099 0.9099 297.9099 149.40991 14940.99114 0.90990 297.90990 149.40990 14940.99000 2020-01-01 2020-01-02 2020-01-01 00:05:03 2020-01-02 03:33:24 2020-01-01 00:05:03.000 2020-01-02 03:33:24.000 303 99204 49753.5 4975350 303 99204 49753.5 4975350 -32266 32669 5189.02 518902 -128 127 -2.66 -266 -304 101 10294 99205 0.91291 297.91291 149.41291 14941.29129 0.91291 297.9129 149.41291 14941.29133 0.91291 297.91291 149.41291 14941.29100 2020-01-01 2020-01-02 2020-01-01 00:05:04 2020-01-02 03:33:25 2020-01-01 00:05:04.000 2020-01-02 03:33:25.000 304 99205 49754.5 4975450 304 99205 49754.5 4975450 -32265 32670 5190.02 519002 -128 127 -4.22 -422 -305 101 10295 99206 0.91591 297.91591 149.41591 14941.59159 0.91591 297.91592 149.41591 14941.59141 0.91591 297.91591 149.41591 14941.59100 2020-01-01 2020-01-02 2020-01-01 00:05:05 2020-01-02 03:33:26 2020-01-01 00:05:05.000 2020-01-02 03:33:26.000 305 99206 49755.5 4975550 305 99206 49755.5 4975550 -32264 32671 5191.02 519102 -128 123 -5.78 -578 -306 101 10296 99207 0.91891 297.91891 149.41891 14941.89189 0.91891 297.9189 149.41891 14941.89172 0.91891 297.91891 149.41891 14941.89100 2020-01-01 2020-01-02 2020-01-01 00:05:06 2020-01-02 03:33:27 2020-01-01 00:05:06.000 2020-01-02 03:33:27.000 306 99207 49756.5 4975650 306 99207 49756.5 4975650 -32263 32672 5192.02 519202 -127 124 -4.78 -478 -307 101 10297 99208 0.92192 297.92192 149.42192 14942.19219 0.92192 297.92194 149.42192 14942.1923 0.92192 297.92192 149.42192 14942.19200 2020-01-01 2020-01-02 2020-01-01 00:05:07 2020-01-02 03:33:28 2020-01-01 00:05:07.000 2020-01-02 03:33:28.000 307 99208 49757.5 4975750 307 99208 49757.5 4975750 -32262 32673 5193.02 519302 -126 125 -3.78 -378 -308 101 10298 99209 0.92492 297.92492 149.42492 14942.49249 0.92492 297.92493 149.42492 14942.49265 0.92492 297.92492 149.42492 14942.49200 2020-01-01 2020-01-02 2020-01-01 00:05:08 2020-01-02 03:33:29 2020-01-01 00:05:08.000 2020-01-02 03:33:29.000 308 99209 49758.5 4975850 308 99209 49758.5 4975850 -32261 32674 5194.02 519402 -125 126 -2.78 -278 -309 101 10299 99210 0.92792 297.92792 149.42792 14942.79279 0.92792 297.92792 149.42792 14942.7928 0.92792 297.92792 149.42792 14942.79200 2020-01-01 2020-01-02 2020-01-01 00:05:09 2020-01-02 03:33:30 2020-01-01 00:05:09.000 2020-01-02 03:33:30.000 309 99210 49759.5 4975950 309 99210 49759.5 4975950 -32260 32675 5195.02 519502 -124 127 -1.78 -178 -31 102 10021 99931 0.09309 300.09309 150.09309 15159.4024 0.09309 300.09308 150.09309 15159.40224 0.09309 300.09309 150.09309 15159.40209 2020-01-01 2020-01-02 2020-01-01 00:00:31 2020-01-02 03:45:31 2020-01-01 00:00:31.000 2020-01-02 03:45:31.000 31 99931 49981 5048081 31 99931 49981 5048081 -32538 32397 4560.009900990099 460561 -127 124 -2.366336633663366 -239 -310 101 10300 99211 0.93093 297.93093 149.43093 14943.09309 0.93093 297.93094 149.43092 14943.09288 0.93093 297.93093 149.43093 14943.09300 2020-01-01 2020-01-02 2020-01-01 00:05:10 2020-01-02 03:33:31 2020-01-01 00:05:10.000 2020-01-02 03:33:31.000 310 99211 49760.5 4976050 310 99211 49760.5 4976050 -32259 32676 5196.02 519602 -128 127 -3.34 -334 -311 101 10301 99212 0.93393 297.93393 149.43393 14943.39339 0.93393 297.93393 149.43393 14943.39319 0.93393 297.93393 149.43393 14943.39300 2020-01-01 2020-01-02 2020-01-01 00:05:11 2020-01-02 03:33:32 2020-01-01 00:05:11.000 2020-01-02 03:33:32.000 311 99212 49761.5 4976150 311 99212 49761.5 4976150 -32258 32677 5197.02 519702 -128 123 -4.9 -490 -312 101 10302 99213 0.93693 297.93693 149.43693 14943.69369 0.93693 297.93695 149.43693 14943.69377 0.93693 297.93693 149.43693 14943.69300 2020-01-01 2020-01-02 2020-01-01 00:05:12 2020-01-02 03:33:33 2020-01-01 00:05:12.000 2020-01-02 03:33:33.000 312 99213 49762.5 4976250 312 99213 49762.5 4976250 -32257 32678 5198.02 519802 -127 124 -3.9 -390 -313 101 10303 99214 0.93993 297.93993 149.43993 14943.99399 0.93993 297.93994 149.43994 14943.99412 0.93993 297.93993 149.43993 14943.99300 2020-01-01 2020-01-02 2020-01-01 00:05:13 2020-01-02 03:33:34 2020-01-01 00:05:13.000 2020-01-02 03:33:34.000 313 99214 49763.5 4976350 313 99214 49763.5 4976350 -32256 32679 5199.02 519902 -126 125 -2.9 -290 -314 101 10304 99215 0.94294 297.94294 149.44294 14944.29429 0.94294 297.94293 149.44294 14944.29427 0.94294 297.94294 149.44294 14944.29400 2020-01-01 2020-01-02 2020-01-01 00:05:14 2020-01-02 03:33:35 2020-01-01 00:05:14.000 2020-01-02 03:33:35.000 314 99215 49764.5 4976450 314 99215 49764.5 4976450 -32255 32680 5200.02 520002 -125 126 -1.9 -190 -315 101 10305 99216 0.94594 297.94594 149.44594 14944.59459 0.94594 297.94595 149.44595 14944.595 0.94594 297.94594 149.44594 14944.59400 2020-01-01 2020-01-02 2020-01-01 00:05:15 2020-01-02 03:33:36 2020-01-01 00:05:15.000 2020-01-02 03:33:36.000 315 99216 49765.5 4976550 315 99216 49765.5 4976550 -32254 32681 5201.02 520102 -124 127 -0.9 -90 -316 101 10306 99217 0.94894 297.94894 149.44894 14944.89489 0.94894 297.94894 149.44894 14944.89466 0.94894 297.94894 149.44894 14944.89400 2020-01-01 2020-01-02 2020-01-01 00:05:16 2020-01-02 03:33:37 2020-01-01 00:05:16.000 2020-01-02 03:33:37.000 316 99217 49766.5 4976650 316 99217 49766.5 4976650 -32253 32682 5202.02 520202 -128 127 -2.46 -246 -317 101 10307 99218 0.95195 297.95195 149.45195 14945.19519 0.95195 297.95197 149.45195 14945.19524 0.95195 297.95195 149.45195 14945.19500 2020-01-01 2020-01-02 2020-01-01 00:05:17 2020-01-02 03:33:38 2020-01-01 00:05:17.000 2020-01-02 03:33:38.000 317 99218 49767.5 4976750 317 99218 49767.5 4976750 -32252 32683 5203.02 520302 -128 123 -4.02 -402 -318 101 10308 99219 0.95495 297.95495 149.45495 14945.49549 0.95495 297.95496 149.45495 14945.49558 0.95495 297.95495 149.45495 14945.49500 2020-01-01 2020-01-02 2020-01-01 00:05:18 2020-01-02 03:33:39 2020-01-01 00:05:18.000 2020-01-02 03:33:39.000 318 99219 49768.5 4976850 318 99219 49768.5 4976850 -32251 32684 5204.02 520402 -127 124 -3.02 -302 -319 101 10309 99220 0.95795 297.95795 149.45795 14945.79579 0.95795 297.95795 149.45795 14945.79574 0.95795 297.95795 149.45795 14945.79500 2020-01-01 2020-01-02 2020-01-01 00:05:19 2020-01-02 03:33:40 2020-01-01 00:05:19.000 2020-01-02 03:33:40.000 319 99220 49769.5 4976950 319 99220 49769.5 4976950 -32250 32685 5205.02 520502 -126 125 -2.02 -202 -32 102 10022 99932 0.09609 300.09609 150.09609 15159.7057 0.09609 300.0961 150.09609 15159.706 0.09609 300.09609 150.09609 15159.70509 2020-01-01 2020-01-02 2020-01-01 00:00:32 2020-01-02 03:45:32 2020-01-01 00:00:32.000 2020-01-02 03:45:32.000 32 99932 49982 5048182 32 99932 49982 5048182 -32537 32398 4561.009900990099 460662 -126 125 -1.3663366336633664 -138 -320 101 10310 99221 0.96096 297.96096 149.46096 14946.09609 0.96096 297.96097 149.46096 14946.09647 0.96096 297.96096 149.46096 14946.09600 2020-01-01 2020-01-02 2020-01-01 00:05:20 2020-01-02 03:33:41 2020-01-01 00:05:20.000 2020-01-02 03:33:41.000 320 99221 49770.5 4977050 320 99221 49770.5 4977050 -32249 32686 5206.02 520602 -125 126 -1.02 -102 -321 101 10311 99222 0.96396 297.96396 149.46396 14946.39639 0.96396 297.96396 149.46396 14946.39613 0.96396 297.96396 149.46396 14946.39600 2020-01-01 2020-01-02 2020-01-01 00:05:21 2020-01-02 03:33:42 2020-01-01 00:05:21.000 2020-01-02 03:33:42.000 321 99222 49771.5 4977150 321 99222 49771.5 4977150 -32248 32687 5207.02 520702 -124 127 -0.02 -2 -322 101 10312 99223 0.96696 297.96696 149.46696 14946.69669 0.96696 297.96698 149.46696 14946.69674 0.96696 297.96696 149.46696 14946.69600 2020-01-01 2020-01-02 2020-01-01 00:05:22 2020-01-02 03:33:43 2020-01-01 00:05:22.000 2020-01-02 03:33:43.000 322 99223 49772.5 4977250 322 99223 49772.5 4977250 -32247 32688 5208.02 520802 -128 127 -1.58 -158 -323 101 10313 99224 0.96996 297.96996 149.46996 14946.99699 0.96997 297.96997 149.46997 14946.99706 0.96996 297.96996 149.46996 14946.99600 2020-01-01 2020-01-02 2020-01-01 00:05:23 2020-01-02 03:33:44 2020-01-01 00:05:23.000 2020-01-02 03:33:44.000 323 99224 49773.5 4977350 323 99224 49773.5 4977350 -32246 32689 5209.02 520902 -128 123 -3.14 -314 -324 101 10314 99225 0.97297 297.97297 149.47297 14947.29729 0.97297 297.97296 149.47297 14947.29737 0.97297 297.97297 149.47297 14947.29700 2020-01-01 2020-01-02 2020-01-01 00:05:24 2020-01-02 03:33:45 2020-01-01 00:05:24.000 2020-01-02 03:33:45.000 324 99225 49774.5 4977450 324 99225 49774.5 4977450 -32245 32690 5210.02 521002 -127 124 -2.14 -214 +259 101 10249 99160 0.77777 297.77777 149.27777 14927.77777 0.77777 297.77777 149.27777 14927.77742 0.77777 297.77777 149.27777000000003 14927.77700 2020-01-01 2020-01-02 2020-01-01 00:04:19 2020-01-02 03:32:40 2020-01-01 00:04:19.000 2020-01-02 03:32:40.000 259 99160 49709.5 4970950 259 99160 49709.5 4970950 -32310 32625 5145.02 514502 -124 127 -0.58 -58 +26 102 10016 99926 0.07807 300.07807 150.07807 15157.88588 0.07807 300.07806 150.07807 15157.88575 0.07807 300.07807 150.07806999999985 15157.88507 2020-01-01 2020-01-02 2020-01-01 00:00:26 2020-01-02 03:45:26 2020-01-01 00:00:26.000 2020-01-02 03:45:26.000 26 99926 49976 5047576 26 99926 49976 5047576 -32543 32392 4555.009900990099 460056 -126 125 -2.297029702970297 -232 +260 101 10250 99161 0.78078 297.78078 149.28078 14928.07807 0.78078 297.7808 149.28077 14928.07799 0.78078 297.78078 149.2807799999999 14928.07800 2020-01-01 2020-01-02 2020-01-01 00:04:20 2020-01-02 03:32:41 2020-01-01 00:04:20.000 2020-01-02 03:32:41.000 260 99161 49710.5 4971050 260 99161 49710.5 4971050 -32309 32626 5146.02 514602 -128 127 -2.14 -214 +261 101 10251 99162 0.78378 297.78378 149.28378 14928.37837 0.78378 297.78378 149.28378 14928.3783 0.78378 297.78378 149.28377999999992 14928.37800 2020-01-01 2020-01-02 2020-01-01 00:04:21 2020-01-02 03:32:42 2020-01-01 00:04:21.000 2020-01-02 03:32:42.000 261 99162 49711.5 4971150 261 99162 49711.5 4971150 -32308 32627 5147.02 514702 -128 123 -3.7 -370 +262 101 10252 99163 0.78678 297.78678 149.28678 14928.67867 0.78678 297.78677 149.28678 14928.67861 0.78678 297.78678 149.28678000000008 14928.67800 2020-01-01 2020-01-02 2020-01-01 00:04:22 2020-01-02 03:32:43 2020-01-01 00:04:22.000 2020-01-02 03:32:43.000 262 99163 49712.5 4971250 262 99163 49712.5 4971250 -32307 32628 5148.02 514802 -127 124 -2.7 -270 +263 101 10253 99164 0.78978 297.78978 149.28978 14928.97897 0.78978 297.7898 149.28979 14928.97923 0.78978 297.78978 149.28977999999992 14928.97800 2020-01-01 2020-01-02 2020-01-01 00:04:23 2020-01-02 03:32:44 2020-01-01 00:04:23.000 2020-01-02 03:32:44.000 263 99164 49713.5 4971350 263 99164 49713.5 4971350 -32306 32629 5149.02 514902 -126 125 -1.7 -170 +264 101 10254 99165 0.79279 297.79279 149.29279 14929.27927 0.79279 297.7928 149.29278 14929.27888 0.79279 297.79279 149.29279000000002 14929.27900 2020-01-01 2020-01-02 2020-01-01 00:04:24 2020-01-02 03:32:45 2020-01-01 00:04:24.000 2020-01-02 03:32:45.000 264 99165 49714.5 4971450 264 99165 49714.5 4971450 -32305 32630 5150.02 515002 -125 126 -0.7 -70 +265 101 10255 99166 0.79579 297.79579 149.29579 14929.57957 0.79579 297.7958 149.29579 14929.57962 0.79579 297.79579 149.29579000000007 14929.57900 2020-01-01 2020-01-02 2020-01-01 00:04:25 2020-01-02 03:32:46 2020-01-01 00:04:25.000 2020-01-02 03:32:46.000 265 99166 49715.5 4971550 265 99166 49715.5 4971550 -32304 32631 5151.02 515102 -124 127 0.3 30 +266 101 10256 99167 0.79879 297.79879 149.29879 14929.87987 0.79879 297.7988 149.29879 14929.87977 0.79879 297.79879 149.29879000000003 14929.87900 2020-01-01 2020-01-02 2020-01-01 00:04:26 2020-01-02 03:32:47 2020-01-01 00:04:26.000 2020-01-02 03:32:47.000 266 99167 49716.5 4971650 266 99167 49716.5 4971650 -32303 32632 5152.02 515202 -128 127 -1.26 -126 +267 101 10257 99168 0.8018 297.8018 149.3018 14930.18018 0.8018 297.8018 149.3018 14930.18012 0.80180 297.80180 149.3018 14930.18000 2020-01-01 2020-01-02 2020-01-01 00:04:27 2020-01-02 03:32:48 2020-01-01 00:04:27.000 2020-01-02 03:32:48.000 267 99168 49717.5 4971750 267 99168 49717.5 4971750 -32302 32633 5153.02 515302 -128 123 -2.82 -282 +268 101 10258 99169 0.8048 297.8048 149.3048 14930.48048 0.8048 297.8048 149.3048 14930.4807 0.80480 297.80480 149.30479999999994 14930.48000 2020-01-01 2020-01-02 2020-01-01 00:04:28 2020-01-02 03:32:49 2020-01-01 00:04:28.000 2020-01-02 03:32:49.000 268 99169 49718.5 4971850 268 99169 49718.5 4971850 -32301 32634 5154.02 515402 -127 124 -1.82 -182 +269 101 10259 99170 0.8078 297.8078 149.3078 14930.78078 0.8078 297.8078 149.3078 14930.78035 0.80780 297.80780 149.30780000000001 14930.78000 2020-01-01 2020-01-02 2020-01-01 00:04:29 2020-01-02 03:32:50 2020-01-01 00:04:29.000 2020-01-02 03:32:50.000 269 99170 49719.5 4971950 269 99170 49719.5 4971950 -32300 32635 5155.02 515502 -126 125 -0.82 -82 +27 102 10017 99927 0.08108 300.08108 150.08108 15158.18918 0.08108 300.0811 150.08108 15158.18936 0.08108 300.08108 150.08107999999996 15158.18908 2020-01-01 2020-01-02 2020-01-01 00:00:27 2020-01-02 03:45:27 2020-01-01 00:00:27.000 2020-01-02 03:45:27.000 27 99927 49977 5047677 27 99927 49977 5047677 -32542 32393 4556.009900990099 460157 -125 126 -1.297029702970297 -131 +270 101 10260 99171 0.81081 297.81081 149.31081 14931.08108 0.81081 297.81082 149.31081 14931.08109 0.81081 297.81081 149.31081000000023 14931.08100 2020-01-01 2020-01-02 2020-01-01 00:04:30 2020-01-02 03:32:51 2020-01-01 00:04:30.000 2020-01-02 03:32:51.000 270 99171 49720.5 4972050 270 99171 49720.5 4972050 -32299 32636 5156.02 515602 -125 126 0.18 18 +271 101 10261 99172 0.81381 297.81381 149.31381 14931.38138 0.81381 297.8138 149.31381 14931.38124 0.81381 297.81381 149.3138099999998 14931.38100 2020-01-01 2020-01-02 2020-01-01 00:04:31 2020-01-02 03:32:52 2020-01-01 00:04:31.000 2020-01-02 03:32:52.000 271 99172 49721.5 4972150 271 99172 49721.5 4972150 -32298 32637 5157.02 515702 -124 127 1.18 118 +272 101 10262 99173 0.81681 297.81681 149.31681 14931.68168 0.81681 297.8168 149.31681 14931.68159 0.81681 297.81681 149.31681000000015 14931.68100 2020-01-01 2020-01-02 2020-01-01 00:04:32 2020-01-02 03:32:53 2020-01-01 00:04:32.000 2020-01-02 03:32:53.000 272 99173 49722.5 4972250 272 99173 49722.5 4972250 -32297 32638 5158.02 515802 -128 127 -0.38 -38 +273 101 10263 99174 0.81981 297.81981 149.31981 14931.98198 0.81981 297.81982 149.31982 14931.98217 0.81981 297.81981 149.31981000000025 14931.98100 2020-01-01 2020-01-02 2020-01-01 00:04:33 2020-01-02 03:32:54 2020-01-01 00:04:33.000 2020-01-02 03:32:54.000 273 99174 49723.5 4972350 273 99174 49723.5 4972350 -32296 32639 5159.02 515902 -128 127 -1.94 -194 +274 101 10264 99175 0.82282 297.82282 149.32282 14932.28228 0.82282 297.8228 149.32282 14932.28247 0.82282 297.82282 149.3228199999999 14932.28200 2020-01-01 2020-01-02 2020-01-01 00:04:34 2020-01-02 03:32:55 2020-01-01 00:04:34.000 2020-01-02 03:32:55.000 274 99175 49724.5 4972450 274 99175 49724.5 4972450 -32295 32640 5160.02 516002 -128 124 -3.5 -350 +275 101 10265 99176 0.82582 297.82582 149.32582 14932.58258 0.82582 297.82584 149.32582 14932.58256 0.82582 297.82582 149.32582000000002 14932.58200 2020-01-01 2020-01-02 2020-01-01 00:04:35 2020-01-02 03:32:56 2020-01-01 00:04:35.000 2020-01-02 03:32:56.000 275 99176 49725.5 4972550 275 99176 49725.5 4972550 -32294 32641 5161.02 516102 -127 125 -2.5 -250 +276 101 10266 99177 0.82882 297.82882 149.32882 14932.88288 0.82882 297.82883 149.32882 14932.88275 0.82882 297.82882 149.32882000000015 14932.88200 2020-01-01 2020-01-02 2020-01-01 00:04:36 2020-01-02 03:32:57 2020-01-01 00:04:36.000 2020-01-02 03:32:57.000 276 99177 49726.5 4972650 276 99177 49726.5 4972650 -32293 32642 5162.02 516202 -126 126 -1.5 -150 +277 101 10267 99178 0.83183 297.83183 149.33183 14933.18318 0.83183 297.83182 149.33183 14933.18305 0.83183 297.83183 149.3318299999998 14933.18300 2020-01-01 2020-01-02 2020-01-01 00:04:37 2020-01-02 03:32:58 2020-01-01 00:04:37.000 2020-01-02 03:32:58.000 277 99178 49727.5 4972750 277 99178 49727.5 4972750 -32292 32643 5163.02 516302 -125 127 -0.5 -50 +278 101 10268 99179 0.83483 297.83483 149.33483 14933.48348 0.83483 297.83484 149.33483 14933.48364 0.83483 297.83483 149.3348299999998 14933.48300 2020-01-01 2020-01-02 2020-01-01 00:04:38 2020-01-02 03:32:59 2020-01-01 00:04:38.000 2020-01-02 03:32:59.000 278 99179 49728.5 4972850 278 99179 49728.5 4972850 -32291 32644 5164.02 516402 -128 127 -2.06 -206 +279 101 10269 99180 0.83783 297.83783 149.33783 14933.78378 0.83783 297.83783 149.33783 14933.78394 0.83783 297.83783 149.33783000000025 14933.78300 2020-01-01 2020-01-02 2020-01-01 00:04:39 2020-01-02 03:33:00 2020-01-01 00:04:39.000 2020-01-02 03:33:00.000 279 99180 49729.5 4972950 279 99180 49729.5 4972950 -32290 32645 5165.02 516502 -128 127 -3.62 -362 +28 102 10018 99928 0.08408 300.08408 150.08408 15158.49249 0.08408 300.08408 150.08408 15158.49265 0.08408 300.08408 150.08408000000003 15158.49208 2020-01-01 2020-01-02 2020-01-01 00:00:28 2020-01-02 03:45:28 2020-01-01 00:00:28.000 2020-01-02 03:45:28.000 28 99928 49978 5047778 28 99928 49978 5047778 -32541 32394 4557.009900990099 460258 -124 127 -0.297029702970297 -30 +280 101 10270 99181 0.84084 297.84084 149.34084 14934.08408 0.84084 297.84085 149.34084 14934.08403 0.84084 297.84084 149.34084000000016 14934.08400 2020-01-01 2020-01-02 2020-01-01 00:04:40 2020-01-02 03:33:01 2020-01-01 00:04:40.000 2020-01-02 03:33:01.000 280 99181 49730.5 4973050 280 99181 49730.5 4973050 -32289 32646 5166.02 516602 -128 123 -5.18 -518 +281 101 10271 99182 0.84384 297.84384 149.34384 14934.38438 0.84384 297.84384 149.34384 14934.38421 0.84384 297.84384 149.3438399999997 14934.38400 2020-01-01 2020-01-02 2020-01-01 00:04:41 2020-01-02 03:33:02 2020-01-01 00:04:41.000 2020-01-02 03:33:02.000 281 99182 49731.5 4973150 281 99182 49731.5 4973150 -32288 32647 5167.02 516702 -127 124 -4.18 -418 +282 101 10272 99183 0.84684 297.84684 149.34684 14934.68468 0.84684 297.84683 149.34684 14934.68453 0.84684 297.84684 149.34684000000013 14934.68400 2020-01-01 2020-01-02 2020-01-01 00:04:42 2020-01-02 03:33:03 2020-01-01 00:04:42.000 2020-01-02 03:33:03.000 282 99183 49732.5 4973250 282 99183 49732.5 4973250 -32287 32648 5168.02 516802 -126 125 -3.18 -318 +283 101 10273 99184 0.84984 297.84984 149.34984 14934.98498 0.84984 297.84985 149.34985 14934.98526 0.84984 297.84984 149.34984000000014 14934.98400 2020-01-01 2020-01-02 2020-01-01 00:04:43 2020-01-02 03:33:04 2020-01-01 00:04:43.000 2020-01-02 03:33:04.000 283 99184 49733.5 4973350 283 99184 49733.5 4973350 -32286 32649 5169.02 516902 -125 126 -2.18 -218 +284 101 10274 99185 0.85285 297.85285 149.35285 14935.28528 0.85285 297.85284 149.35285 14935.28542 0.85285 297.85285 149.3528499999999 14935.28500 2020-01-01 2020-01-02 2020-01-01 00:04:44 2020-01-02 03:33:05 2020-01-01 00:04:44.000 2020-01-02 03:33:05.000 284 99185 49734.5 4973450 284 99185 49734.5 4973450 -32285 32650 5170.02 517002 -124 127 -1.18 -118 +285 101 10275 99186 0.85585 297.85585 149.35585 14935.58558 0.85585 297.85587 149.35585 14935.5855 0.85585 297.85585 149.35584999999995 14935.58500 2020-01-01 2020-01-02 2020-01-01 00:04:45 2020-01-02 03:33:06 2020-01-01 00:04:45.000 2020-01-02 03:33:06.000 285 99186 49735.5 4973550 285 99186 49735.5 4973550 -32284 32651 5171.02 517102 -128 127 -2.74 -274 +286 101 10276 99187 0.85885 297.85885 149.35885 14935.88588 0.85885 297.85886 149.35885 14935.88568 0.85885 297.85885 149.35885000000005 14935.88500 2020-01-01 2020-01-02 2020-01-01 00:04:46 2020-01-02 03:33:07 2020-01-01 00:04:46.000 2020-01-02 03:33:07.000 286 99187 49736.5 4973650 286 99187 49736.5 4973650 -32283 32652 5172.02 517202 -128 123 -4.3 -430 +287 101 10277 99188 0.86186 297.86186 149.36186 14936.18618 0.86186 297.86185 149.36186 14936.186 0.86186 297.86186 149.36186000000026 14936.18600 2020-01-01 2020-01-02 2020-01-01 00:04:47 2020-01-02 03:33:08 2020-01-01 00:04:47.000 2020-01-02 03:33:08.000 287 99188 49737.5 4973750 287 99188 49737.5 4973750 -32282 32653 5173.02 517302 -127 124 -3.3 -330 +288 101 10278 99189 0.86486 297.86486 149.36486 14936.48648 0.86486 297.86487 149.36486 14936.48673 0.86486 297.86486 149.36485999999982 14936.48600 2020-01-01 2020-01-02 2020-01-01 00:04:48 2020-01-02 03:33:09 2020-01-01 00:04:48.000 2020-01-02 03:33:09.000 288 99189 49738.5 4973850 288 99189 49738.5 4973850 -32281 32654 5174.02 517402 -126 125 -2.3 -230 +289 101 10279 99190 0.86786 297.86786 149.36786 14936.78678 0.86786 297.86786 149.36786 14936.78688 0.86786 297.86786 149.36786000000018 14936.78600 2020-01-01 2020-01-02 2020-01-01 00:04:49 2020-01-02 03:33:10 2020-01-01 00:04:49.000 2020-01-02 03:33:10.000 289 99190 49739.5 4973950 289 99190 49739.5 4973950 -32280 32655 5175.02 517502 -125 126 -1.3 -130 +29 102 10019 99929 0.08708 300.08708 150.08708 15158.79579 0.08708 300.0871 150.08708 15158.79576 0.08708 300.08708 150.08707999999987 15158.79508 2020-01-01 2020-01-02 2020-01-01 00:00:29 2020-01-02 03:45:29 2020-01-01 00:00:29.000 2020-01-02 03:45:29.000 29 99929 49979 5047879 29 99929 49979 5047879 -32540 32395 4558.009900990099 460359 -128 127 -1.8316831683168318 -185 +290 101 10280 99191 0.87087 297.87087 149.37087 14937.08708 0.87087 297.87088 149.37087 14937.087 0.87087 297.87087 149.37087000000017 14937.08700 2020-01-01 2020-01-02 2020-01-01 00:04:50 2020-01-02 03:33:11 2020-01-01 00:04:50.000 2020-01-02 03:33:11.000 290 99191 49740.5 4974050 290 99191 49740.5 4974050 -32279 32656 5176.02 517602 -124 127 -0.3 -30 +291 101 10281 99192 0.87387 297.87387 149.37387 14937.38738 0.87387 297.87387 149.37387 14937.38716 0.87387 297.87387 149.37386999999993 14937.38700 2020-01-01 2020-01-02 2020-01-01 00:04:51 2020-01-02 03:33:12 2020-01-01 00:04:51.000 2020-01-02 03:33:12.000 291 99192 49741.5 4974150 291 99192 49741.5 4974150 -32278 32657 5177.02 517702 -128 127 -1.86 -186 +292 101 10282 99193 0.87687 297.87687 149.37687 14937.68768 0.87687 297.8769 149.37687 14937.68789 0.87687 297.87687 149.37687000000003 14937.68700 2020-01-01 2020-01-02 2020-01-01 00:04:52 2020-01-02 03:33:13 2020-01-01 00:04:52.000 2020-01-02 03:33:13.000 292 99193 49742.5 4974250 292 99193 49742.5 4974250 -32277 32658 5178.02 517802 -128 123 -3.42 -342 +293 101 10283 99194 0.87987 297.87987 149.37987 14937.98798 0.87987 297.87988 149.37988 14937.9882 0.87987 297.87987 149.37987000000012 14937.98700 2020-01-01 2020-01-02 2020-01-01 00:04:53 2020-01-02 03:33:14 2020-01-01 00:04:53.000 2020-01-02 03:33:14.000 293 99194 49743.5 4974350 293 99194 49743.5 4974350 -32276 32659 5179.02 517902 -127 124 -2.42 -242 +294 101 10284 99195 0.88288 297.88288 149.38288 14938.28828 0.88288 297.88287 149.38288 14938.28835 0.88288 297.88288 149.38287999999983 14938.28800 2020-01-01 2020-01-02 2020-01-01 00:04:54 2020-01-02 03:33:15 2020-01-01 00:04:54.000 2020-01-02 03:33:15.000 294 99195 49744.5 4974450 294 99195 49744.5 4974450 -32275 32660 5180.02 518002 -126 125 -1.42 -142 +295 101 10285 99196 0.88588 297.88588 149.38588 14938.58858 0.88588 297.8859 149.38588 14938.58847 0.88588 297.88588 149.38587999999984 14938.58800 2020-01-01 2020-01-02 2020-01-01 00:04:55 2020-01-02 03:33:16 2020-01-01 00:04:55.000 2020-01-02 03:33:16.000 295 99196 49745.5 4974550 295 99196 49745.5 4974550 -32274 32661 5181.02 518102 -125 126 -0.42 -42 +296 101 10286 99197 0.88888 297.88888 149.38888 14938.88888 0.88888 297.8889 149.38888 14938.88863 0.88888 297.88888 149.38888000000028 14938.88800 2020-01-01 2020-01-02 2020-01-01 00:04:56 2020-01-02 03:33:17 2020-01-01 00:04:56.000 2020-01-02 03:33:17.000 296 99197 49746.5 4974650 296 99197 49746.5 4974650 -32273 32662 5182.02 518202 -124 127 0.58 58 +297 101 10287 99198 0.89189 297.89189 149.39189 14939.18918 0.89189 297.8919 149.39189 14939.18936 0.89189 297.89189 149.39189000000016 14939.18900 2020-01-01 2020-01-02 2020-01-01 00:04:57 2020-01-02 03:33:18 2020-01-01 00:04:57.000 2020-01-02 03:33:18.000 297 99198 49747.5 4974750 297 99198 49747.5 4974750 -32272 32663 5183.02 518302 -128 127 -0.98 -98 +298 101 10288 99199 0.89489 297.89489 149.39489 14939.48948 0.89489 297.8949 149.39489 14939.48967 0.89489 297.89489 149.39488999999972 14939.48900 2020-01-01 2020-01-02 2020-01-01 00:04:58 2020-01-02 03:33:19 2020-01-01 00:04:58.000 2020-01-02 03:33:19.000 298 99199 49748.5 4974850 298 99199 49748.5 4974850 -32271 32664 5184.02 518402 -128 127 -2.54 -254 +299 101 10289 99200 0.89789 297.89789 149.39789 14939.78978 0.89789 297.8979 149.39789 14939.78986 0.89789 297.89789 149.39789000000016 14939.78900 2020-01-01 2020-01-02 2020-01-01 00:04:59 2020-01-02 03:33:20 2020-01-01 00:04:59.000 2020-01-02 03:33:20.000 299 99200 49749.5 4974950 299 99200 49749.5 4974950 -32270 32665 5185.02 518502 -128 124 -4.1 -410 +3 102 1002 9993 0.009 300.009 150.009 15150.9099 0.009 300.009 150.009 15150.90958 0.00900 300.00900 150.00900000000001 15150.90900 2020-01-01 2020-01-02 2020-01-01 00:00:03 2020-01-02 03:45:03 2020-01-01 00:00:03.000 2020-01-02 03:45:03.000 3 99903 49953 5045253 3 99903 49953 5045253 -32566 32369 4532.009900990099 457733 -124 127 0.04950495049504951 5 +30 102 10020 99930 0.09009 300.09009 150.09009 15159.09909 0.09009 300.0901 150.09008 15159.09894 0.09009 300.09009 150.0900899999998 15159.09909 2020-01-01 2020-01-02 2020-01-01 00:00:30 2020-01-02 03:45:30 2020-01-01 00:00:30.000 2020-01-02 03:45:30.000 30 99930 49980 5047980 30 99930 49980 5047980 -32539 32396 4559.009900990099 460460 -128 123 -3.366336633663366 -340 +300 101 10290 99201 0.9009 297.9009 149.4009 14940.09009 0.9009 297.9009 149.40089 14940.08995 0.90090 297.90090 149.40090000000006 14940.09000 2020-01-01 2020-01-02 2020-01-01 00:05:00 2020-01-02 03:33:21 2020-01-01 00:05:00.000 2020-01-02 03:33:21.000 300 99201 49750.5 4975050 300 99201 49750.5 4975050 -32269 32666 5186.02 518602 -127 125 -3.1 -310 +301 101 10291 99202 0.9039 297.9039 149.4039 14940.39039 0.9039 297.9039 149.4039 14940.39009 0.90390 297.90390 149.40389999999988 14940.39000 2020-01-01 2020-01-02 2020-01-01 00:05:01 2020-01-02 03:33:22 2020-01-01 00:05:01.000 2020-01-02 03:33:22.000 301 99202 49751.5 4975150 301 99202 49751.5 4975150 -32268 32667 5187.02 518702 -126 126 -2.1 -210 +302 101 10292 99203 0.9069 297.9069 149.4069 14940.69069 0.9069 297.90692 149.4069 14940.69083 0.90690 297.90690 149.40689999999998 14940.69000 2020-01-01 2020-01-02 2020-01-01 00:05:02 2020-01-02 03:33:23 2020-01-01 00:05:02.000 2020-01-02 03:33:23.000 302 99203 49752.5 4975250 302 99203 49752.5 4975250 -32267 32668 5188.02 518802 -125 127 -1.1 -110 +303 101 10293 99204 0.9099 297.9099 149.4099 14940.99099 0.9099 297.9099 149.40991 14940.99114 0.90990 297.90990 149.40990000000008 14940.99000 2020-01-01 2020-01-02 2020-01-01 00:05:03 2020-01-02 03:33:24 2020-01-01 00:05:03.000 2020-01-02 03:33:24.000 303 99204 49753.5 4975350 303 99204 49753.5 4975350 -32266 32669 5189.02 518902 -128 127 -2.66 -266 +304 101 10294 99205 0.91291 297.91291 149.41291 14941.29129 0.91291 297.9129 149.41291 14941.29133 0.91291 297.91291 149.41290999999973 14941.29100 2020-01-01 2020-01-02 2020-01-01 00:05:04 2020-01-02 03:33:25 2020-01-01 00:05:04.000 2020-01-02 03:33:25.000 304 99205 49754.5 4975450 304 99205 49754.5 4975450 -32265 32670 5190.02 519002 -128 127 -4.22 -422 +305 101 10295 99206 0.91591 297.91591 149.41591 14941.59159 0.91591 297.91592 149.41591 14941.59141 0.91591 297.91591 149.41590999999983 14941.59100 2020-01-01 2020-01-02 2020-01-01 00:05:05 2020-01-02 03:33:26 2020-01-01 00:05:05.000 2020-01-02 03:33:26.000 305 99206 49755.5 4975550 305 99206 49755.5 4975550 -32264 32671 5191.02 519102 -128 123 -5.78 -578 +306 101 10296 99207 0.91891 297.91891 149.41891 14941.89189 0.91891 297.9189 149.41891 14941.89172 0.91891 297.91891 149.41891000000018 14941.89100 2020-01-01 2020-01-02 2020-01-01 00:05:06 2020-01-02 03:33:27 2020-01-01 00:05:06.000 2020-01-02 03:33:27.000 306 99207 49756.5 4975650 306 99207 49756.5 4975650 -32263 32672 5192.02 519202 -127 124 -4.78 -478 +307 101 10297 99208 0.92192 297.92192 149.42192 14942.19219 0.92192 297.92194 149.42192 14942.1923 0.92192 297.92192 149.42192000000009 14942.19200 2020-01-01 2020-01-02 2020-01-01 00:05:07 2020-01-02 03:33:28 2020-01-01 00:05:07.000 2020-01-02 03:33:28.000 307 99208 49757.5 4975750 307 99208 49757.5 4975750 -32262 32673 5193.02 519302 -126 125 -3.78 -378 +308 101 10298 99209 0.92492 297.92492 149.42492 14942.49249 0.92492 297.92493 149.42492 14942.49265 0.92492 297.92492 149.42491999999996 14942.49200 2020-01-01 2020-01-02 2020-01-01 00:05:08 2020-01-02 03:33:29 2020-01-01 00:05:08.000 2020-01-02 03:33:29.000 308 99209 49758.5 4975850 308 99209 49758.5 4975850 -32261 32674 5194.02 519402 -125 126 -2.78 -278 +309 101 10299 99210 0.92792 297.92792 149.42792 14942.79279 0.92792 297.92792 149.42792 14942.7928 0.92792 297.92792 149.42792000000006 14942.79200 2020-01-01 2020-01-02 2020-01-01 00:05:09 2020-01-02 03:33:30 2020-01-01 00:05:09.000 2020-01-02 03:33:30.000 309 99210 49759.5 4975950 309 99210 49759.5 4975950 -32260 32675 5195.02 519502 -124 127 -1.78 -178 +31 102 10021 99931 0.09309 300.09309 150.09309 15159.4024 0.09309 300.09308 150.09309 15159.40224 0.09309 300.09309 150.0930900000002 15159.40209 2020-01-01 2020-01-02 2020-01-01 00:00:31 2020-01-02 03:45:31 2020-01-01 00:00:31.000 2020-01-02 03:45:31.000 31 99931 49981 5048081 31 99931 49981 5048081 -32538 32397 4560.009900990099 460561 -127 124 -2.366336633663366 -239 +310 101 10300 99211 0.93093 297.93093 149.43093 14943.09309 0.93093 297.93094 149.43092 14943.09288 0.93093 297.93093 149.4309300000003 14943.09300 2020-01-01 2020-01-02 2020-01-01 00:05:10 2020-01-02 03:33:31 2020-01-01 00:05:10.000 2020-01-02 03:33:31.000 310 99211 49760.5 4976050 310 99211 49760.5 4976050 -32259 32676 5196.02 519602 -128 127 -3.34 -334 +311 101 10301 99212 0.93393 297.93393 149.43393 14943.39339 0.93393 297.93393 149.43393 14943.39319 0.93393 297.93393 149.43392999999983 14943.39300 2020-01-01 2020-01-02 2020-01-01 00:05:11 2020-01-02 03:33:32 2020-01-01 00:05:11.000 2020-01-02 03:33:32.000 311 99212 49761.5 4976150 311 99212 49761.5 4976150 -32258 32677 5197.02 519702 -128 123 -4.9 -490 +312 101 10302 99213 0.93693 297.93693 149.43693 14943.69369 0.93693 297.93695 149.43693 14943.69377 0.93693 297.93693 149.43692999999988 14943.69300 2020-01-01 2020-01-02 2020-01-01 00:05:12 2020-01-02 03:33:33 2020-01-01 00:05:12.000 2020-01-02 03:33:33.000 312 99213 49762.5 4976250 312 99213 49762.5 4976250 -32257 32678 5198.02 519802 -127 124 -3.9 -390 +313 101 10303 99214 0.93993 297.93993 149.43993 14943.99399 0.93993 297.93994 149.43994 14943.99412 0.93993 297.93993 149.4399300000003 14943.99300 2020-01-01 2020-01-02 2020-01-01 00:05:13 2020-01-02 03:33:34 2020-01-01 00:05:13.000 2020-01-02 03:33:34.000 313 99214 49763.5 4976350 313 99214 49763.5 4976350 -32256 32679 5199.02 519902 -126 125 -2.9 -290 +314 101 10304 99215 0.94294 297.94294 149.44294 14944.29429 0.94294 297.94293 149.44294 14944.29427 0.94294 297.94294 149.4429400000002 14944.29400 2020-01-01 2020-01-02 2020-01-01 00:05:14 2020-01-02 03:33:35 2020-01-01 00:05:14.000 2020-01-02 03:33:35.000 314 99215 49764.5 4976450 314 99215 49764.5 4976450 -32255 32680 5200.02 520002 -125 126 -1.9 -190 +315 101 10305 99216 0.94594 297.94594 149.44594 14944.59459 0.94594 297.94595 149.44595 14944.595 0.94594 297.94594 149.44593999999975 14944.59400 2020-01-01 2020-01-02 2020-01-01 00:05:15 2020-01-02 03:33:36 2020-01-01 00:05:15.000 2020-01-02 03:33:36.000 315 99216 49765.5 4976550 315 99216 49765.5 4976550 -32254 32681 5201.02 520102 -124 127 -0.9 -90 +316 101 10306 99217 0.94894 297.94894 149.44894 14944.89489 0.94894 297.94894 149.44894 14944.89466 0.94894 297.94894 149.4489400000002 14944.89400 2020-01-01 2020-01-02 2020-01-01 00:05:16 2020-01-02 03:33:37 2020-01-01 00:05:16.000 2020-01-02 03:33:37.000 316 99217 49766.5 4976650 316 99217 49766.5 4976650 -32253 32682 5202.02 520202 -128 127 -2.46 -246 +317 101 10307 99218 0.95195 297.95195 149.45195 14945.19519 0.95195 297.95197 149.45195 14945.19524 0.95195 297.95195 149.4519500000001 14945.19500 2020-01-01 2020-01-02 2020-01-01 00:05:17 2020-01-02 03:33:38 2020-01-01 00:05:17.000 2020-01-02 03:33:38.000 317 99218 49767.5 4976750 317 99218 49767.5 4976750 -32252 32683 5203.02 520302 -128 123 -4.02 -402 +318 101 10308 99219 0.95495 297.95495 149.45495 14945.49549 0.95495 297.95496 149.45495 14945.49558 0.95495 297.95495 149.45494999999985 14945.49500 2020-01-01 2020-01-02 2020-01-01 00:05:18 2020-01-02 03:33:39 2020-01-01 00:05:18.000 2020-01-02 03:33:39.000 318 99219 49768.5 4976850 318 99219 49768.5 4976850 -32251 32684 5204.02 520402 -127 124 -3.02 -302 +319 101 10309 99220 0.95795 297.95795 149.45795 14945.79579 0.95795 297.95795 149.45795 14945.79574 0.95795 297.95795 149.45794999999998 14945.79500 2020-01-01 2020-01-02 2020-01-01 00:05:19 2020-01-02 03:33:40 2020-01-01 00:05:19.000 2020-01-02 03:33:40.000 319 99220 49769.5 4976950 319 99220 49769.5 4976950 -32250 32685 5205.02 520502 -126 125 -2.02 -202 +32 102 10022 99932 0.09609 300.09609 150.09609 15159.7057 0.09609 300.0961 150.09609 15159.706 0.09609 300.09609 150.09608999999972 15159.70509 2020-01-01 2020-01-02 2020-01-01 00:00:32 2020-01-02 03:45:32 2020-01-01 00:00:32.000 2020-01-02 03:45:32.000 32 99932 49982 5048182 32 99932 49982 5048182 -32537 32398 4561.009900990099 460662 -126 125 -1.3663366336633664 -138 +320 101 10310 99221 0.96096 297.96096 149.46096 14946.09609 0.96096 297.96097 149.46096 14946.09647 0.96096 297.96096 149.4609600000002 14946.09600 2020-01-01 2020-01-02 2020-01-01 00:05:20 2020-01-02 03:33:41 2020-01-01 00:05:20.000 2020-01-02 03:33:41.000 320 99221 49770.5 4977050 320 99221 49770.5 4977050 -32249 32686 5206.02 520602 -125 126 -1.02 -102 +321 101 10311 99222 0.96396 297.96396 149.46396 14946.39639 0.96396 297.96396 149.46396 14946.39613 0.96396 297.96396 149.46395999999976 14946.39600 2020-01-01 2020-01-02 2020-01-01 00:05:21 2020-01-02 03:33:42 2020-01-01 00:05:21.000 2020-01-02 03:33:42.000 321 99222 49771.5 4977150 321 99222 49771.5 4977150 -32248 32687 5207.02 520702 -124 127 -0.02 -2 +322 101 10312 99223 0.96696 297.96696 149.46696 14946.69669 0.96696 297.96698 149.46696 14946.69674 0.96696 297.96696 149.46695999999986 14946.69600 2020-01-01 2020-01-02 2020-01-01 00:05:22 2020-01-02 03:33:43 2020-01-01 00:05:22.000 2020-01-02 03:33:43.000 322 99223 49772.5 4977250 322 99223 49772.5 4977250 -32247 32688 5208.02 520802 -128 127 -1.58 -158 +323 101 10313 99224 0.96996 297.96996 149.46996 14946.99699 0.96997 297.96997 149.46997 14946.99706 0.96996 297.96996 149.4699600000002 14946.99600 2020-01-01 2020-01-02 2020-01-01 00:05:23 2020-01-02 03:33:44 2020-01-01 00:05:23.000 2020-01-02 03:33:44.000 323 99224 49773.5 4977350 323 99224 49773.5 4977350 -32246 32689 5209.02 520902 -128 123 -3.14 -314 +324 101 10314 99225 0.97297 297.97297 149.47297 14947.29729 0.97297 297.97296 149.47297 14947.29737 0.97297 297.97297 149.47297000000012 14947.29700 2020-01-01 2020-01-02 2020-01-01 00:05:24 2020-01-02 03:33:45 2020-01-01 00:05:24.000 2020-01-02 03:33:45.000 324 99225 49774.5 4977450 324 99225 49774.5 4977450 -32245 32690 5210.02 521002 -127 124 -2.14 -214 325 101 10315 99226 0.97597 297.97597 149.47597 14947.59759 0.97597 297.97598 149.47597 14947.59794 0.97597 297.97597 149.47597 14947.59700 2020-01-01 2020-01-02 2020-01-01 00:05:25 2020-01-02 03:33:46 2020-01-01 00:05:25.000 2020-01-02 03:33:46.000 325 99226 49775.5 4977550 325 99226 49775.5 4977550 -32244 32691 5211.02 521102 -126 125 -1.14 -114 -326 101 10316 99227 0.97897 297.97897 149.47897 14947.89789 0.97897 297.97897 149.47897 14947.8976 0.97897 297.97897 149.47897 14947.89700 2020-01-01 2020-01-02 2020-01-01 00:05:26 2020-01-02 03:33:47 2020-01-01 00:05:26.000 2020-01-02 03:33:47.000 326 99227 49776.5 4977650 326 99227 49776.5 4977650 -32243 32692 5212.02 521202 -125 126 -0.14 -14 +326 101 10316 99227 0.97897 297.97897 149.47897 14947.89789 0.97897 297.97897 149.47897 14947.8976 0.97897 297.97897 149.4789700000001 14947.89700 2020-01-01 2020-01-02 2020-01-01 00:05:26 2020-01-02 03:33:47 2020-01-01 00:05:26.000 2020-01-02 03:33:47.000 326 99227 49776.5 4977650 326 99227 49776.5 4977650 -32243 32692 5212.02 521202 -125 126 -0.14 -14 327 101 10317 99228 0.98198 297.98198 149.48198 14948.19819 0.98198 297.982 149.48198 14948.19821 0.98198 297.98198 149.48198 14948.19800 2020-01-01 2020-01-02 2020-01-01 00:05:27 2020-01-02 03:33:48 2020-01-01 00:05:27.000 2020-01-02 03:33:48.000 327 99228 49777.5 4977750 327 99228 49777.5 4977750 -32242 32693 5213.02 521302 -124 127 0.86 86 -328 101 10318 99229 0.98498 297.98498 149.48498 14948.49849 0.98498 297.985 149.48498 14948.49853 0.98498 297.98498 149.48498 14948.49800 2020-01-01 2020-01-02 2020-01-01 00:05:28 2020-01-02 03:33:49 2020-01-01 00:05:28.000 2020-01-02 03:33:49.000 328 99229 49778.5 4977850 328 99229 49778.5 4977850 -32241 32694 5214.02 521402 -128 127 -0.7 -70 -329 101 10319 99230 0.98798 297.98798 149.48798 14948.79879 0.98798 297.98798 149.48798 14948.79883 0.98798 297.98798 149.48798 14948.79800 2020-01-01 2020-01-02 2020-01-01 00:05:29 2020-01-02 03:33:50 2020-01-01 00:05:29.000 2020-01-02 03:33:50.000 329 99230 49779.5 4977950 329 99230 49779.5 4977950 -32240 32695 5215.02 521502 -128 127 -2.26 -226 -33 102 10023 99933 0.09909 300.09909 150.09909 15160.009 0.09909 300.0991 150.0991 15160.00913 0.09909 300.09909 150.09909 15160.00809 2020-01-01 2020-01-02 2020-01-01 00:00:33 2020-01-02 03:45:33 2020-01-01 00:00:33.000 2020-01-02 03:45:33.000 33 99933 49983 5048283 33 99933 49983 5048283 -32536 32399 4562.009900990099 460763 -125 126 -0.36633663366336633 -37 -330 101 10320 99231 0.99099 297.99099 149.49099 14949.09909 0.99099 297.991 149.49099 14949.09941 0.99099 297.99099 149.49099 14949.09900 2020-01-01 2020-01-02 2020-01-01 00:05:30 2020-01-02 03:33:51 2020-01-01 00:05:30.000 2020-01-02 03:33:51.000 330 99231 49780.5 4978050 330 99231 49780.5 4978050 -32239 32696 5216.02 521602 -128 123 -3.82 -382 -331 101 10321 99232 0.99399 297.99399 149.49399 14949.39939 0.99399 297.994 149.49399 14949.39911 0.99399 297.99399 149.49399 14949.39900 2020-01-01 2020-01-02 2020-01-01 00:05:31 2020-01-02 03:33:52 2020-01-01 00:05:31.000 2020-01-02 03:33:52.000 331 99232 49781.5 4978150 331 99232 49781.5 4978150 -32238 32697 5217.02 521702 -127 124 -2.82 -282 -332 101 10322 99233 0.99699 297.99699 149.49699 14949.69969 0.99699 297.997 149.49699 14949.69969 0.99699 297.99699 149.49699 14949.69900 2020-01-01 2020-01-02 2020-01-01 00:05:32 2020-01-02 03:33:53 2020-01-01 00:05:32.000 2020-01-02 03:33:53.000 332 99233 49782.5 4978250 332 99233 49782.5 4978250 -32237 32698 5218.02 521802 -126 125 -1.82 -182 -333 101 10323 99234 1 298 149.5 14950 1 298 149.5 14950 1.00000 298.00000 149.50000 14950.00000 2020-01-01 2020-01-02 2020-01-01 00:05:33 2020-01-02 03:33:54 2020-01-01 00:05:33.000 2020-01-02 03:33:54.000 333 99234 49783.5 4978350 333 99234 49783.5 4978350 -32236 32699 5219.02 521902 -125 126 -0.82 -82 -334 101 10324 99235 1.003 298.003 149.503 14950.3003 1.003 298.003 149.503 14950.30029 1.00300 298.00300 149.50300 14950.30000 2020-01-01 2020-01-02 2020-01-01 00:05:34 2020-01-02 03:33:55 2020-01-01 00:05:34.000 2020-01-02 03:33:55.000 334 99235 49784.5 4978450 334 99235 49784.5 4978450 -32235 32700 5220.02 522002 -124 127 0.18 18 -335 101 10325 99236 1.006 298.006 149.506 14950.6006 1.006 298.006 149.506 14950.60088 1.00600 298.00600 149.50600 14950.60000 2020-01-01 2020-01-02 2020-01-01 00:05:35 2020-01-02 03:33:56 2020-01-01 00:05:35.000 2020-01-02 03:33:56.000 335 99236 49785.5 4978550 335 99236 49785.5 4978550 -32234 32701 5221.02 522102 -128 127 -1.38 -138 -336 101 10326 99237 1.009 298.009 149.509 14950.9009 1.009 298.009 149.509 14950.90057 1.00900 298.00900 149.50900 14950.90000 2020-01-01 2020-01-02 2020-01-01 00:05:36 2020-01-02 03:33:57 2020-01-01 00:05:36.000 2020-01-02 03:33:57.000 336 99237 49786.5 4978650 336 99237 49786.5 4978650 -32233 32702 5222.02 522202 -128 123 -2.94 -294 -337 101 10327 99238 1.01201 298.01201 149.51201 14951.2012 1.01201 298.01202 149.51201 14951.20117 1.01201 298.01201 149.51201 14951.20100 2020-01-01 2020-01-02 2020-01-01 00:05:37 2020-01-02 03:33:58 2020-01-01 00:05:37.000 2020-01-02 03:33:58.000 337 99238 49787.5 4978750 337 99238 49787.5 4978750 -32232 32703 5223.02 522302 -127 124 -1.94 -194 -338 101 10328 99239 1.01501 298.01501 149.51501 14951.5015 1.01501 298.015 149.51501 14951.50146 1.01501 298.01501 149.51501 14951.50100 2020-01-01 2020-01-02 2020-01-01 00:05:38 2020-01-02 03:33:59 2020-01-01 00:05:38.000 2020-01-02 03:33:59.000 338 99239 49788.5 4978850 338 99239 49788.5 4978850 -32231 32704 5224.02 522402 -126 125 -0.94 -94 -339 101 10329 99240 1.01801 298.01801 149.51801 14951.8018 1.01801 298.018 149.51801 14951.80177 1.01801 298.01801 149.51801 14951.80100 2020-01-01 2020-01-02 2020-01-01 00:05:39 2020-01-02 03:34:00 2020-01-01 00:05:39.000 2020-01-02 03:34:00.000 339 99240 49789.5 4978950 339 99240 49789.5 4978950 -32230 32705 5225.02 522502 -125 126 0.06 6 -34 102 10024 99934 0.1021 300.1021 150.1021 15160.31231 0.1021 300.1021 150.1021 15160.31224 0.10210 300.10210 150.10210 15160.31210 2020-01-01 2020-01-02 2020-01-01 00:00:34 2020-01-02 03:45:34 2020-01-01 00:00:34.000 2020-01-02 03:45:34.000 34 99934 49984 5048384 34 99934 49984 5048384 -32535 32400 4563.009900990099 460864 -124 127 0.6336633663366337 64 -340 101 10330 99241 1.02102 298.02102 149.52102 14952.1021 1.02102 298.02103 149.52102 14952.10239 1.02102 298.02102 149.52102 14952.10200 2020-01-01 2020-01-02 2020-01-01 00:05:40 2020-01-02 03:34:01 2020-01-01 00:05:40.000 2020-01-02 03:34:01.000 340 99241 49790.5 4979050 340 99241 49790.5 4979050 -32229 32706 5226.02 522602 -124 127 1.06 106 -341 101 10331 99242 1.02402 298.02402 149.52402 14952.4024 1.02402 298.02402 149.52402 14952.40205 1.02402 298.02402 149.52402 14952.40200 2020-01-01 2020-01-02 2020-01-01 00:05:41 2020-01-02 03:34:02 2020-01-01 00:05:41.000 2020-01-02 03:34:02.000 341 99242 49791.5 4979150 341 99242 49791.5 4979150 -32228 32707 5227.02 522702 -128 127 -0.5 -50 -342 101 10332 99243 1.02702 298.02702 149.52702 14952.7027 1.02702 298.02704 149.52702 14952.70264 1.02702 298.02702 149.52702 14952.70200 2020-01-01 2020-01-02 2020-01-01 00:05:42 2020-01-02 03:34:03 2020-01-01 00:05:42.000 2020-01-02 03:34:03.000 342 99243 49792.5 4979250 342 99243 49792.5 4979250 -32227 32708 5228.02 522802 -128 123 -2.06 -206 -343 101 10333 99244 1.03003 298.03003 149.53003 14953.003 1.03003 298.03003 149.53002 14953.00293 1.03003 298.03003 149.53003 14953.00300 2020-01-01 2020-01-02 2020-01-01 00:05:43 2020-01-02 03:34:04 2020-01-01 00:05:43.000 2020-01-02 03:34:04.000 343 99244 49793.5 4979350 343 99244 49793.5 4979350 -32226 32709 5229.02 522902 -127 124 -1.06 -106 -344 101 10334 99245 1.03303 298.03303 149.53303 14953.3033 1.03303 298.03302 149.53303 14953.30323 1.03303 298.03303 149.53303 14953.30300 2020-01-01 2020-01-02 2020-01-01 00:05:44 2020-01-02 03:34:05 2020-01-01 00:05:44.000 2020-01-02 03:34:05.000 344 99245 49794.5 4979450 344 99245 49794.5 4979450 -32225 32710 5230.02 523002 -126 125 -0.06 -6 -345 101 10335 99246 1.03603 298.03603 149.53603 14953.6036 1.03603 298.03604 149.53603 14953.60386 1.03603 298.03603 149.53603 14953.60300 2020-01-01 2020-01-02 2020-01-01 00:05:45 2020-01-02 03:34:06 2020-01-01 00:05:45.000 2020-01-02 03:34:06.000 345 99246 49795.5 4979550 345 99246 49795.5 4979550 -32224 32711 5231.02 523102 -125 126 0.94 94 -346 101 10336 99247 1.03903 298.03903 149.53903 14953.9039 1.03903 298.03903 149.53903 14953.90352 1.03903 298.03903 149.53903 14953.90300 2020-01-01 2020-01-02 2020-01-01 00:05:46 2020-01-02 03:34:07 2020-01-01 00:05:46.000 2020-01-02 03:34:07.000 346 99247 49796.5 4979650 346 99247 49796.5 4979650 -32223 32712 5232.02 523202 -124 127 1.94 194 -347 101 10337 99248 1.04204 298.04204 149.54204 14954.2042 1.04204 298.04205 149.54204 14954.20427 1.04204 298.04204 149.54204 14954.20400 2020-01-01 2020-01-02 2020-01-01 00:05:47 2020-01-02 03:34:08 2020-01-01 00:05:47.000 2020-01-02 03:34:08.000 347 99248 49797.5 4979750 347 99248 49797.5 4979750 -32222 32713 5233.02 523302 -128 127 0.38 38 -348 101 10338 99249 1.04504 298.04504 149.54504 14954.5045 1.04504 298.04504 149.54504 14954.50441 1.04504 298.04504 149.54504 14954.50400 2020-01-01 2020-01-02 2020-01-01 00:05:48 2020-01-02 03:34:09 2020-01-01 00:05:48.000 2020-01-02 03:34:09.000 348 99249 49798.5 4979850 348 99249 49798.5 4979850 -32221 32714 5234.02 523402 -128 123 -1.18 -118 -349 101 10339 99250 1.04804 298.04804 149.54804 14954.8048 1.04804 298.04803 149.54804 14954.80474 1.04804 298.04804 149.54804 14954.80400 2020-01-01 2020-01-02 2020-01-01 00:05:49 2020-01-02 03:34:10 2020-01-01 00:05:49.000 2020-01-02 03:34:10.000 349 99250 49799.5 4979950 349 99250 49799.5 4979950 -32220 32715 5235.02 523502 -127 124 -0.18 -18 -35 102 10025 99935 0.1051 300.1051 150.1051 15160.61561 0.1051 300.1051 150.1051 15160.61542 0.10510 300.10510 150.10510 15160.61510 2020-01-01 2020-01-02 2020-01-01 00:00:35 2020-01-02 03:45:35 2020-01-01 00:00:35.000 2020-01-02 03:45:35.000 35 99935 49985 5048485 35 99935 49985 5048485 -32534 32401 4564.009900990099 460965 -128 127 -0.900990099009901 -91 +328 101 10318 99229 0.98498 297.98498 149.48498 14948.49849 0.98498 297.985 149.48498 14948.49853 0.98498 297.98498 149.48497999999987 14948.49800 2020-01-01 2020-01-02 2020-01-01 00:05:28 2020-01-02 03:33:49 2020-01-01 00:05:28.000 2020-01-02 03:33:49.000 328 99229 49778.5 4977850 328 99229 49778.5 4977850 -32241 32694 5214.02 521402 -128 127 -0.7 -70 +329 101 10319 99230 0.98798 297.98798 149.48798 14948.79879 0.98798 297.98798 149.48798 14948.79883 0.98798 297.98798 149.4879799999999 14948.79800 2020-01-01 2020-01-02 2020-01-01 00:05:29 2020-01-02 03:33:50 2020-01-01 00:05:29.000 2020-01-02 03:33:50.000 329 99230 49779.5 4977950 329 99230 49779.5 4977950 -32240 32695 5215.02 521502 -128 127 -2.26 -226 +33 102 10023 99933 0.09909 300.09909 150.09909 15160.009 0.09909 300.0991 150.0991 15160.00913 0.09909 300.09909 150.09908999999982 15160.00809 2020-01-01 2020-01-02 2020-01-01 00:00:33 2020-01-02 03:45:33 2020-01-01 00:00:33.000 2020-01-02 03:45:33.000 33 99933 49983 5048283 33 99933 49983 5048283 -32536 32399 4562.009900990099 460763 -125 126 -0.36633663366336633 -37 +330 101 10320 99231 0.99099 297.99099 149.49099 14949.09909 0.99099 297.991 149.49099 14949.09941 0.99099 297.99099 149.49099000000012 14949.09900 2020-01-01 2020-01-02 2020-01-01 00:05:30 2020-01-02 03:33:51 2020-01-01 00:05:30.000 2020-01-02 03:33:51.000 330 99231 49780.5 4978050 330 99231 49780.5 4978050 -32239 32696 5216.02 521602 -128 123 -3.82 -382 +331 101 10321 99232 0.99399 297.99399 149.49399 14949.39939 0.99399 297.994 149.49399 14949.39911 0.99399 297.99399 149.49399000000022 14949.39900 2020-01-01 2020-01-02 2020-01-01 00:05:31 2020-01-02 03:33:52 2020-01-01 00:05:31.000 2020-01-02 03:33:52.000 331 99232 49781.5 4978150 331 99232 49781.5 4978150 -32238 32697 5217.02 521702 -127 124 -2.82 -282 +332 101 10322 99233 0.99699 297.99699 149.49699 14949.69969 0.99699 297.997 149.49699 14949.69969 0.99699 297.99699 149.49698999999978 14949.69900 2020-01-01 2020-01-02 2020-01-01 00:05:32 2020-01-02 03:33:53 2020-01-01 00:05:32.000 2020-01-02 03:33:53.000 332 99233 49782.5 4978250 332 99233 49782.5 4978250 -32237 32698 5218.02 521802 -126 125 -1.82 -182 +333 101 10323 99234 1 298 149.5 14950 1 298 149.5 14950 1.00000 298.00000 149.5 14950.00000 2020-01-01 2020-01-02 2020-01-01 00:05:33 2020-01-02 03:33:54 2020-01-01 00:05:33.000 2020-01-02 03:33:54.000 333 99234 49783.5 4978350 333 99234 49783.5 4978350 -32236 32699 5219.02 521902 -125 126 -0.82 -82 +334 101 10324 99235 1.003 298.003 149.503 14950.3003 1.003 298.003 149.503 14950.30029 1.00300 298.00300 149.50300000000013 14950.30000 2020-01-01 2020-01-02 2020-01-01 00:05:34 2020-01-02 03:33:55 2020-01-01 00:05:34.000 2020-01-02 03:33:55.000 334 99235 49784.5 4978450 334 99235 49784.5 4978450 -32235 32700 5220.02 522002 -124 127 0.18 18 +335 101 10325 99236 1.006 298.006 149.506 14950.6006 1.006 298.006 149.506 14950.60088 1.00600 298.00600 149.5059999999999 14950.60000 2020-01-01 2020-01-02 2020-01-01 00:05:35 2020-01-02 03:33:56 2020-01-01 00:05:35.000 2020-01-02 03:33:56.000 335 99236 49785.5 4978550 335 99236 49785.5 4978550 -32234 32701 5221.02 522102 -128 127 -1.38 -138 +336 101 10326 99237 1.009 298.009 149.509 14950.9009 1.009 298.009 149.509 14950.90057 1.00900 298.00900 149.50900000000001 14950.90000 2020-01-01 2020-01-02 2020-01-01 00:05:36 2020-01-02 03:33:57 2020-01-01 00:05:36.000 2020-01-02 03:33:57.000 336 99237 49786.5 4978650 336 99237 49786.5 4978650 -32233 32702 5222.02 522202 -128 123 -2.94 -294 +337 101 10327 99238 1.01201 298.01201 149.51201 14951.2012 1.01201 298.01202 149.51201 14951.20117 1.01201 298.01201 149.51201000000023 14951.20100 2020-01-01 2020-01-02 2020-01-01 00:05:37 2020-01-02 03:33:58 2020-01-01 00:05:37.000 2020-01-02 03:33:58.000 337 99238 49787.5 4978750 337 99238 49787.5 4978750 -32232 32703 5223.02 522302 -127 124 -1.94 -194 +338 101 10328 99239 1.01501 298.01501 149.51501 14951.5015 1.01501 298.015 149.51501 14951.50146 1.01501 298.01501 149.5150099999998 14951.50100 2020-01-01 2020-01-02 2020-01-01 00:05:38 2020-01-02 03:33:59 2020-01-01 00:05:38.000 2020-01-02 03:33:59.000 338 99239 49788.5 4978850 338 99239 49788.5 4978850 -32231 32704 5224.02 522402 -126 125 -0.94 -94 +339 101 10329 99240 1.01801 298.01801 149.51801 14951.8018 1.01801 298.018 149.51801 14951.80177 1.01801 298.01801 149.5180099999998 14951.80100 2020-01-01 2020-01-02 2020-01-01 00:05:39 2020-01-02 03:34:00 2020-01-01 00:05:39.000 2020-01-02 03:34:00.000 339 99240 49789.5 4978950 339 99240 49789.5 4978950 -32230 32705 5225.02 522502 -125 126 0.06 6 +34 102 10024 99934 0.1021 300.1021 150.1021 15160.31231 0.1021 300.1021 150.1021 15160.31224 0.10210 300.10210 150.10210000000004 15160.31210 2020-01-01 2020-01-02 2020-01-01 00:00:34 2020-01-02 03:45:34 2020-01-01 00:00:34.000 2020-01-02 03:45:34.000 34 99934 49984 5048384 34 99934 49984 5048384 -32535 32400 4563.009900990099 460864 -124 127 0.6336633663366337 64 +340 101 10330 99241 1.02102 298.02102 149.52102 14952.1021 1.02102 298.02103 149.52102 14952.10239 1.02102 298.02102 149.5210200000001 14952.10200 2020-01-01 2020-01-02 2020-01-01 00:05:40 2020-01-02 03:34:01 2020-01-01 00:05:40.000 2020-01-02 03:34:01.000 340 99241 49790.5 4979050 340 99241 49790.5 4979050 -32229 32706 5226.02 522602 -124 127 1.06 106 +341 101 10331 99242 1.02402 298.02402 149.52402 14952.4024 1.02402 298.02402 149.52402 14952.40205 1.02402 298.02402 149.52402000000015 14952.40200 2020-01-01 2020-01-02 2020-01-01 00:05:41 2020-01-02 03:34:02 2020-01-01 00:05:41.000 2020-01-02 03:34:02.000 341 99242 49791.5 4979150 341 99242 49791.5 4979150 -32228 32707 5227.02 522702 -128 127 -0.5 -50 +342 101 10332 99243 1.02702 298.02702 149.52702 14952.7027 1.02702 298.02704 149.52702 14952.70264 1.02702 298.02702 149.5270199999997 14952.70200 2020-01-01 2020-01-02 2020-01-01 00:05:42 2020-01-02 03:34:03 2020-01-01 00:05:42.000 2020-01-02 03:34:03.000 342 99243 49792.5 4979250 342 99243 49792.5 4979250 -32227 32708 5228.02 522802 -128 123 -2.06 -206 +343 101 10333 99244 1.03003 298.03003 149.53003 14953.003 1.03003 298.03003 149.53002 14953.00293 1.03003 298.03003 149.53002999999993 14953.00300 2020-01-01 2020-01-02 2020-01-01 00:05:43 2020-01-02 03:34:04 2020-01-01 00:05:43.000 2020-01-02 03:34:04.000 343 99244 49793.5 4979350 343 99244 49793.5 4979350 -32226 32709 5229.02 522902 -127 124 -1.06 -106 +344 101 10334 99245 1.03303 298.03303 149.53303 14953.3033 1.03303 298.03302 149.53303 14953.30323 1.03303 298.03303 149.53303000000002 14953.30300 2020-01-01 2020-01-02 2020-01-01 00:05:44 2020-01-02 03:34:05 2020-01-01 00:05:44.000 2020-01-02 03:34:05.000 344 99245 49794.5 4979450 344 99245 49794.5 4979450 -32225 32710 5230.02 523002 -126 125 -0.06 -6 +345 101 10335 99246 1.03603 298.03603 149.53603 14953.6036 1.03603 298.03604 149.53603 14953.60386 1.03603 298.03603 149.5360299999999 14953.60300 2020-01-01 2020-01-02 2020-01-01 00:05:45 2020-01-02 03:34:06 2020-01-01 00:05:45.000 2020-01-02 03:34:06.000 345 99246 49795.5 4979550 345 99246 49795.5 4979550 -32224 32711 5231.02 523102 -125 126 0.94 94 +346 101 10336 99247 1.03903 298.03903 149.53903 14953.9039 1.03903 298.03903 149.53903 14953.90352 1.03903 298.03903 149.53902999999994 14953.90300 2020-01-01 2020-01-02 2020-01-01 00:05:46 2020-01-02 03:34:07 2020-01-01 00:05:46.000 2020-01-02 03:34:07.000 346 99247 49796.5 4979650 346 99247 49796.5 4979650 -32223 32712 5232.02 523202 -124 127 1.94 194 +347 101 10337 99248 1.04204 298.04204 149.54204 14954.2042 1.04204 298.04205 149.54204 14954.20427 1.04204 298.04204 149.54204000000016 14954.20400 2020-01-01 2020-01-02 2020-01-01 00:05:47 2020-01-02 03:34:08 2020-01-01 00:05:47.000 2020-01-02 03:34:08.000 347 99248 49797.5 4979750 347 99248 49797.5 4979750 -32222 32713 5233.02 523302 -128 127 0.38 38 +348 101 10338 99249 1.04504 298.04504 149.54504 14954.5045 1.04504 298.04504 149.54504 14954.50441 1.04504 298.04504 149.54504000000026 14954.50400 2020-01-01 2020-01-02 2020-01-01 00:05:48 2020-01-02 03:34:09 2020-01-01 00:05:48.000 2020-01-02 03:34:09.000 348 99249 49798.5 4979850 348 99249 49798.5 4979850 -32221 32714 5234.02 523402 -128 123 -1.18 -118 +349 101 10339 99250 1.04804 298.04804 149.54804 14954.8048 1.04804 298.04803 149.54804 14954.80474 1.04804 298.04804 149.54803999999982 14954.80400 2020-01-01 2020-01-02 2020-01-01 00:05:49 2020-01-02 03:34:10 2020-01-01 00:05:49.000 2020-01-02 03:34:10.000 349 99250 49799.5 4979950 349 99250 49799.5 4979950 -32220 32715 5235.02 523502 -127 124 -0.18 -18 +35 102 10025 99935 0.1051 300.1051 150.1051 15160.61561 0.1051 300.1051 150.1051 15160.61542 0.10510 300.10510 150.10510000000014 15160.61510 2020-01-01 2020-01-02 2020-01-01 00:00:35 2020-01-02 03:45:35 2020-01-01 00:00:35.000 2020-01-02 03:45:35.000 35 99935 49985 5048485 35 99935 49985 5048485 -32534 32401 4564.009900990099 460965 -128 127 -0.900990099009901 -91 350 101 10340 99251 1.05105 298.05105 149.55105 14955.1051 1.05105 298.05106 149.55105 14955.10532 1.05105 298.05105 149.55105 14955.10500 2020-01-01 2020-01-02 2020-01-01 00:05:50 2020-01-02 03:34:11 2020-01-01 00:05:50.000 2020-01-02 03:34:11.000 350 99251 49800.5 4980050 350 99251 49800.5 4980050 -32219 32716 5236.02 523602 -126 125 0.82 82 -351 101 10341 99252 1.05405 298.05405 149.55405 14955.4054 1.05405 298.05405 149.55404 14955.40499 1.05405 298.05405 149.55405 14955.40500 2020-01-01 2020-01-02 2020-01-01 00:05:51 2020-01-02 03:34:12 2020-01-01 00:05:51.000 2020-01-02 03:34:12.000 351 99252 49801.5 4980150 351 99252 49801.5 4980150 -32218 32717 5237.02 523702 -125 126 1.82 182 -352 101 10342 99253 1.05705 298.05705 149.55705 14955.7057 1.05705 298.05707 149.55705 14955.70574 1.05705 298.05705 149.55705 14955.70500 2020-01-01 2020-01-02 2020-01-01 00:05:52 2020-01-02 03:34:13 2020-01-01 00:05:52.000 2020-01-02 03:34:13.000 352 99253 49802.5 4980250 352 99253 49802.5 4980250 -32217 32718 5238.02 523802 -124 127 2.82 282 -353 101 10343 99254 1.06006 298.06006 149.56006 14956.006 1.06006 298.06006 149.56005 14956.00587 1.06006 298.06006 149.56006 14956.00600 2020-01-01 2020-01-02 2020-01-01 00:05:53 2020-01-02 03:34:14 2020-01-01 00:05:53.000 2020-01-02 03:34:14.000 353 99254 49803.5 4980350 353 99254 49803.5 4980350 -32216 32719 5239.02 523902 -128 127 1.26 126 -354 101 10344 99255 1.06306 298.06306 149.56306 14956.3063 1.06306 298.06305 149.56306 14956.3062 1.06306 298.06306 149.56306 14956.30600 2020-01-01 2020-01-02 2020-01-01 00:05:54 2020-01-02 03:34:15 2020-01-01 00:05:54.000 2020-01-02 03:34:15.000 354 99255 49804.5 4980450 354 99255 49804.5 4980450 -32215 32720 5240.02 524002 -128 127 -0.3 -30 -355 101 10345 99256 1.06606 298.06606 149.56606 14956.6066 1.06606 298.06607 149.56606 14956.6068 1.06606 298.06606 149.56606 14956.60600 2020-01-01 2020-01-02 2020-01-01 00:05:55 2020-01-02 03:34:16 2020-01-01 00:05:55.000 2020-01-02 03:34:16.000 355 99256 49805.5 4980550 355 99256 49805.5 4980550 -32214 32721 5241.02 524102 -128 123 -1.86 -186 -356 101 10346 99257 1.06906 298.06906 149.56906 14956.9069 1.06906 298.06906 149.56907 14956.90709 1.06906 298.06906 149.56906 14956.90600 2020-01-01 2020-01-02 2020-01-01 00:05:56 2020-01-02 03:34:17 2020-01-01 00:05:56.000 2020-01-02 03:34:17.000 356 99257 49806.5 4980650 356 99257 49806.5 4980650 -32213 32722 5242.02 524202 -127 124 -0.86 -86 -357 101 10347 99258 1.07207 298.07207 149.57207 14957.2072 1.07207 298.07208 149.57207 14957.20721 1.07207 298.07207 149.57207 14957.20700 2020-01-01 2020-01-02 2020-01-01 00:05:57 2020-01-02 03:34:18 2020-01-01 00:05:57.000 2020-01-02 03:34:18.000 357 99258 49807.5 4980750 357 99258 49807.5 4980750 -32212 32723 5243.02 524302 -126 125 0.14 14 -358 101 10348 99259 1.07507 298.07507 149.57507 14957.5075 1.07507 298.07507 149.57507 14957.50734 1.07507 298.07507 149.57507 14957.50700 2020-01-01 2020-01-02 2020-01-01 00:05:58 2020-01-02 03:34:19 2020-01-01 00:05:58.000 2020-01-02 03:34:19.000 358 99259 49808.5 4980850 358 99259 49808.5 4980850 -32211 32724 5244.02 524402 -125 126 1.14 114 -359 101 10349 99260 1.07807 298.07807 149.57807 14957.8078 1.07807 298.07806 149.57807 14957.80767 1.07807 298.07807 149.57807 14957.80700 2020-01-01 2020-01-02 2020-01-01 00:05:59 2020-01-02 03:34:20 2020-01-01 00:05:59.000 2020-01-02 03:34:20.000 359 99260 49809.5 4980950 359 99260 49809.5 4980950 -32210 32725 5245.02 524502 -124 127 2.14 214 -36 102 10026 99936 0.1081 300.1081 150.1081 15160.91891 0.1081 300.1081 150.1081 15160.91873 0.10810 300.10810 150.10810 15160.91810 2020-01-01 2020-01-02 2020-01-01 00:00:36 2020-01-02 03:45:36 2020-01-01 00:00:36.000 2020-01-02 03:45:36.000 36 99936 49986 5048586 36 99936 49986 5048586 -32533 32402 4565.009900990099 461066 -128 123 -2.4356435643564356 -246 -360 101 10350 99261 1.08108 298.08108 149.58108 14958.1081 1.08108 298.0811 149.58108 14958.10827 1.08108 298.08108 149.58108 14958.10800 2020-01-01 2020-01-02 2020-01-01 00:06:00 2020-01-02 03:34:21 2020-01-01 00:06:00.000 2020-01-02 03:34:21.000 360 99261 49810.5 4981050 360 99261 49810.5 4981050 -32209 32726 5246.02 524602 -128 127 0.58 58 -361 101 10351 99262 1.08408 298.08408 149.58408 14958.4084 1.08408 298.08408 149.58408 14958.40856 1.08408 298.08408 149.58408 14958.40800 2020-01-01 2020-01-02 2020-01-01 00:06:01 2020-01-02 03:34:22 2020-01-01 00:06:01.000 2020-01-02 03:34:22.000 361 99262 49811.5 4981150 361 99262 49811.5 4981150 -32208 32727 5247.02 524702 -128 123 -0.98 -98 -362 101 10352 99263 1.08708 298.08708 149.58708 14958.7087 1.08708 298.0871 149.58708 14958.70868 1.08708 298.08708 149.58708 14958.70800 2020-01-01 2020-01-02 2020-01-01 00:06:02 2020-01-02 03:34:23 2020-01-01 00:06:02.000 2020-01-02 03:34:23.000 362 99263 49812.5 4981250 362 99263 49812.5 4981250 -32207 32728 5248.02 524802 -127 124 0.02 2 -363 101 10353 99264 1.09009 298.09009 149.59009 14959.009 1.09009 298.0901 149.59008 14959.00884 1.09009 298.09009 149.59009 14959.00900 2020-01-01 2020-01-02 2020-01-01 00:06:03 2020-01-02 03:34:24 2020-01-01 00:06:03.000 2020-01-02 03:34:24.000 363 99264 49813.5 4981350 363 99264 49813.5 4981350 -32206 32729 5249.02 524902 -126 125 1.02 102 -364 101 10354 99265 1.09309 298.09309 149.59309 14959.3093 1.09309 298.09308 149.59309 14959.30915 1.09309 298.09309 149.59309 14959.30900 2020-01-01 2020-01-02 2020-01-01 00:06:04 2020-01-02 03:34:25 2020-01-01 00:06:04.000 2020-01-02 03:34:25.000 364 99265 49814.5 4981450 364 99265 49814.5 4981450 -32205 32730 5250.02 525002 -125 126 2.02 202 -365 101 10355 99266 1.09609 298.09609 149.59609 14959.6096 1.09609 298.0961 149.59609 14959.6099 1.09609 298.09609 149.59609 14959.60900 2020-01-01 2020-01-02 2020-01-01 00:06:05 2020-01-02 03:34:26 2020-01-01 00:06:05.000 2020-01-02 03:34:26.000 365 99266 49815.5 4981550 365 99266 49815.5 4981550 -32204 32731 5251.02 525102 -124 127 3.02 302 -366 101 10356 99267 1.09909 298.09909 149.59909 14959.9099 1.09909 298.0991 149.5991 14959.91003 1.09909 298.09909 149.59909 14959.90900 2020-01-01 2020-01-02 2020-01-01 00:06:06 2020-01-02 03:34:27 2020-01-01 00:06:06.000 2020-01-02 03:34:27.000 366 99267 49816.5 4981650 366 99267 49816.5 4981650 -32203 32732 5252.02 525202 -128 127 1.46 146 -367 101 10357 99268 1.1021 298.1021 149.6021 14960.21021 1.1021 298.1021 149.6021 14960.21015 1.10210 298.10210 149.60210 14960.21000 2020-01-01 2020-01-02 2020-01-01 00:06:07 2020-01-02 03:34:28 2020-01-01 00:06:07.000 2020-01-02 03:34:28.000 367 99268 49817.5 4981750 367 99268 49817.5 4981750 -32202 32733 5253.02 525302 -128 123 -0.1 -10 -368 101 10358 99269 1.1051 298.1051 149.6051 14960.51051 1.1051 298.1051 149.6051 14960.51031 1.10510 298.10510 149.60510 14960.51000 2020-01-01 2020-01-02 2020-01-01 00:06:08 2020-01-02 03:34:29 2020-01-01 00:06:08.000 2020-01-02 03:34:29.000 368 99269 49818.5 4981850 368 99269 49818.5 4981850 -32201 32734 5254.02 525402 -127 124 0.9 90 -369 101 10359 99270 1.1081 298.1081 149.6081 14960.81081 1.1081 298.1081 149.6081 14960.81062 1.10810 298.10810 149.60810 14960.81000 2020-01-01 2020-01-02 2020-01-01 00:06:09 2020-01-02 03:34:30 2020-01-01 00:06:09.000 2020-01-02 03:34:30.000 369 99270 49819.5 4981950 369 99270 49819.5 4981950 -32200 32735 5255.02 525502 -126 125 1.9 190 -37 102 10027 99937 0.11111 300.11111 150.11111 15161.22222 0.11111 300.1111 150.11111 15161.22248 0.11111 300.11111 150.11111 15161.22211 2020-01-01 2020-01-02 2020-01-01 00:00:37 2020-01-02 03:45:37 2020-01-01 00:00:37.000 2020-01-02 03:45:37.000 37 99937 49987 5048687 37 99937 49987 5048687 -32532 32403 4566.009900990099 461167 -127 124 -1.4356435643564356 -145 -370 101 10360 99271 1.11111 298.11111 149.61111 14961.11111 1.11111 298.1111 149.61111 14961.11137 1.11111 298.11111 149.61111 14961.11100 2020-01-01 2020-01-02 2020-01-01 00:06:10 2020-01-02 03:34:31 2020-01-01 00:06:10.000 2020-01-02 03:34:31.000 370 99271 49820.5 4982050 370 99271 49820.5 4982050 -32199 32736 5256.02 525602 -125 126 2.9 290 -371 101 10361 99272 1.11411 298.11411 149.61411 14961.41141 1.11411 298.1141 149.61411 14961.4115 1.11411 298.11411 149.61411 14961.41100 2020-01-01 2020-01-02 2020-01-01 00:06:11 2020-01-02 03:34:32 2020-01-01 00:06:11.000 2020-01-02 03:34:32.000 371 99272 49821.5 4982150 371 99272 49821.5 4982150 -32198 32737 5257.02 525702 -124 127 3.9 390 -372 101 10362 99273 1.11711 298.11711 149.61711 14961.71171 1.11711 298.11713 149.61711 14961.71165 1.11711 298.11711 149.61711 14961.71100 2020-01-01 2020-01-02 2020-01-01 00:06:12 2020-01-02 03:34:33 2020-01-01 00:06:12.000 2020-01-02 03:34:33.000 372 99273 49822.5 4982250 372 99273 49822.5 4982250 -32197 32738 5258.02 525802 -128 127 2.34 234 -373 101 10363 99274 1.12012 298.12012 149.62012 14962.01201 1.12012 298.12012 149.62011 14962.01179 1.12012 298.12012 149.62012 14962.01200 2020-01-01 2020-01-02 2020-01-01 00:06:13 2020-01-02 03:34:34 2020-01-01 00:06:13.000 2020-01-02 03:34:34.000 373 99274 49823.5 4982350 373 99274 49823.5 4982350 -32196 32739 5259.02 525902 -128 123 0.78 78 -374 101 10364 99275 1.12312 298.12312 149.62312 14962.31231 1.12312 298.1231 149.62312 14962.31208 1.12312 298.12312 149.62312 14962.31200 2020-01-01 2020-01-02 2020-01-01 00:06:14 2020-01-02 03:34:35 2020-01-01 00:06:14.000 2020-01-02 03:34:35.000 374 99275 49824.5 4982450 374 99275 49824.5 4982450 -32195 32740 5260.02 526002 -127 124 1.78 178 -375 101 10365 99276 1.12612 298.12612 149.62612 14962.61261 1.12612 298.12613 149.62612 14962.61283 1.12612 298.12612 149.62612 14962.61200 2020-01-01 2020-01-02 2020-01-01 00:06:15 2020-01-02 03:34:36 2020-01-01 00:06:15.000 2020-01-02 03:34:36.000 375 99276 49825.5 4982550 375 99276 49825.5 4982550 -32194 32741 5261.02 526102 -126 125 2.78 278 -376 101 10366 99277 1.12912 298.12912 149.62912 14962.91291 1.12912 298.12912 149.62912 14962.91297 1.12912 298.12912 149.62912 14962.91200 2020-01-01 2020-01-02 2020-01-01 00:06:16 2020-01-02 03:34:37 2020-01-01 00:06:16.000 2020-01-02 03:34:37.000 376 99277 49826.5 4982650 376 99277 49826.5 4982650 -32193 32742 5262.02 526202 -125 126 3.78 378 +351 101 10341 99252 1.05405 298.05405 149.55405 14955.4054 1.05405 298.05405 149.55404 14955.40499 1.05405 298.05405 149.55405000000016 14955.40500 2020-01-01 2020-01-02 2020-01-01 00:05:51 2020-01-02 03:34:12 2020-01-01 00:05:51.000 2020-01-02 03:34:12.000 351 99252 49801.5 4980150 351 99252 49801.5 4980150 -32218 32717 5237.02 523702 -125 126 1.82 182 +352 101 10342 99253 1.05705 298.05705 149.55705 14955.7057 1.05705 298.05707 149.55705 14955.70574 1.05705 298.05705 149.55704999999992 14955.70500 2020-01-01 2020-01-02 2020-01-01 00:05:52 2020-01-02 03:34:13 2020-01-01 00:05:52.000 2020-01-02 03:34:13.000 352 99253 49802.5 4980250 352 99253 49802.5 4980250 -32217 32718 5238.02 523802 -124 127 2.82 282 +353 101 10343 99254 1.06006 298.06006 149.56006 14956.006 1.06006 298.06006 149.56005 14956.00587 1.06006 298.06006 149.56005999999982 14956.00600 2020-01-01 2020-01-02 2020-01-01 00:05:53 2020-01-02 03:34:14 2020-01-01 00:05:53.000 2020-01-02 03:34:14.000 353 99254 49803.5 4980350 353 99254 49803.5 4980350 -32216 32719 5239.02 523902 -128 127 1.26 126 +354 101 10344 99255 1.06306 298.06306 149.56306 14956.3063 1.06306 298.06305 149.56306 14956.3062 1.06306 298.06306 149.56306000000026 14956.30600 2020-01-01 2020-01-02 2020-01-01 00:05:54 2020-01-02 03:34:15 2020-01-01 00:05:54.000 2020-01-02 03:34:15.000 354 99255 49804.5 4980450 354 99255 49804.5 4980450 -32215 32720 5240.02 524002 -128 127 -0.3 -30 +355 101 10345 99256 1.06606 298.06606 149.56606 14956.6066 1.06606 298.06607 149.56606 14956.6068 1.06606 298.06606 149.56605999999982 14956.60600 2020-01-01 2020-01-02 2020-01-01 00:05:55 2020-01-02 03:34:16 2020-01-01 00:05:55.000 2020-01-02 03:34:16.000 355 99256 49805.5 4980550 355 99256 49805.5 4980550 -32214 32721 5241.02 524102 -128 123 -1.86 -186 +356 101 10346 99257 1.06906 298.06906 149.56906 14956.9069 1.06906 298.06906 149.56907 14956.90709 1.06906 298.06906 149.56905999999984 14956.90600 2020-01-01 2020-01-02 2020-01-01 00:05:56 2020-01-02 03:34:17 2020-01-01 00:05:56.000 2020-01-02 03:34:17.000 356 99257 49806.5 4980650 356 99257 49806.5 4980650 -32213 32722 5242.02 524202 -127 124 -0.86 -86 +357 101 10347 99258 1.07207 298.07207 149.57207 14957.2072 1.07207 298.07208 149.57207 14957.20721 1.07207 298.07207 149.57207000000014 14957.20700 2020-01-01 2020-01-02 2020-01-01 00:05:57 2020-01-02 03:34:18 2020-01-01 00:05:57.000 2020-01-02 03:34:18.000 357 99258 49807.5 4980750 357 99258 49807.5 4980750 -32212 32723 5243.02 524302 -126 125 0.14 14 +358 101 10348 99259 1.07507 298.07507 149.57507 14957.5075 1.07507 298.07507 149.57507 14957.50734 1.07507 298.07507 149.57507000000015 14957.50700 2020-01-01 2020-01-02 2020-01-01 00:05:58 2020-01-02 03:34:19 2020-01-01 00:05:58.000 2020-01-02 03:34:19.000 358 99259 49808.5 4980850 358 99259 49808.5 4980850 -32211 32724 5244.02 524402 -125 126 1.14 114 +359 101 10349 99260 1.07807 298.07807 149.57807 14957.8078 1.07807 298.07806 149.57807 14957.80767 1.07807 298.07807 149.5780699999997 14957.80700 2020-01-01 2020-01-02 2020-01-01 00:05:59 2020-01-02 03:34:20 2020-01-01 00:05:59.000 2020-01-02 03:34:20.000 359 99260 49809.5 4980950 359 99260 49809.5 4980950 -32210 32725 5245.02 524502 -124 127 2.14 214 +36 102 10026 99936 0.1081 300.1081 150.1081 15160.91891 0.1081 300.1081 150.1081 15160.91873 0.10810 300.10810 150.10809999999998 15160.91810 2020-01-01 2020-01-02 2020-01-01 00:00:36 2020-01-02 03:45:36 2020-01-01 00:00:36.000 2020-01-02 03:45:36.000 36 99936 49986 5048586 36 99936 49986 5048586 -32533 32402 4565.009900990099 461066 -128 123 -2.4356435643564356 -246 +360 101 10350 99261 1.08108 298.08108 149.58108 14958.1081 1.08108 298.0811 149.58108 14958.10827 1.08108 298.08108 149.58107999999996 14958.10800 2020-01-01 2020-01-02 2020-01-01 00:06:00 2020-01-02 03:34:21 2020-01-01 00:06:00.000 2020-01-02 03:34:21.000 360 99261 49810.5 4981050 360 99261 49810.5 4981050 -32209 32726 5246.02 524602 -128 127 0.58 58 +361 101 10351 99262 1.08408 298.08408 149.58408 14958.4084 1.08408 298.08408 149.58408 14958.40856 1.08408 298.08408 149.58408000000006 14958.40800 2020-01-01 2020-01-02 2020-01-01 00:06:01 2020-01-02 03:34:22 2020-01-01 00:06:01.000 2020-01-02 03:34:22.000 361 99262 49811.5 4981150 361 99262 49811.5 4981150 -32208 32727 5247.02 524702 -128 123 -0.98 -98 +362 101 10352 99263 1.08708 298.08708 149.58708 14958.7087 1.08708 298.0871 149.58708 14958.70868 1.08708 298.08708 149.58707999999987 14958.70800 2020-01-01 2020-01-02 2020-01-01 00:06:02 2020-01-02 03:34:23 2020-01-01 00:06:02.000 2020-01-02 03:34:23.000 362 99263 49812.5 4981250 362 99263 49812.5 4981250 -32207 32728 5248.02 524802 -127 124 0.02 2 +363 101 10353 99264 1.09009 298.09009 149.59009 14959.009 1.09009 298.0901 149.59008 14959.00884 1.09009 298.09009 149.5900899999998 14959.00900 2020-01-01 2020-01-02 2020-01-01 00:06:03 2020-01-02 03:34:24 2020-01-01 00:06:03.000 2020-01-02 03:34:24.000 363 99264 49813.5 4981350 363 99264 49813.5 4981350 -32206 32729 5249.02 524902 -126 125 1.02 102 +364 101 10354 99265 1.09309 298.09309 149.59309 14959.3093 1.09309 298.09308 149.59309 14959.30915 1.09309 298.09309 149.5930900000002 14959.30900 2020-01-01 2020-01-02 2020-01-01 00:06:04 2020-01-02 03:34:25 2020-01-01 00:06:04.000 2020-01-02 03:34:25.000 364 99265 49814.5 4981450 364 99265 49814.5 4981450 -32205 32730 5250.02 525002 -125 126 2.02 202 +365 101 10355 99266 1.09609 298.09609 149.59609 14959.6096 1.09609 298.0961 149.59609 14959.6099 1.09609 298.09609 149.59608999999972 14959.60900 2020-01-01 2020-01-02 2020-01-01 00:06:05 2020-01-02 03:34:26 2020-01-01 00:06:05.000 2020-01-02 03:34:26.000 365 99266 49815.5 4981550 365 99266 49815.5 4981550 -32204 32731 5251.02 525102 -124 127 3.02 302 +366 101 10356 99267 1.09909 298.09909 149.59909 14959.9099 1.09909 298.0991 149.5991 14959.91003 1.09909 298.09909 149.59908999999982 14959.90900 2020-01-01 2020-01-02 2020-01-01 00:06:06 2020-01-02 03:34:27 2020-01-01 00:06:06.000 2020-01-02 03:34:27.000 366 99267 49816.5 4981650 366 99267 49816.5 4981650 -32203 32732 5252.02 525202 -128 127 1.46 146 +367 101 10357 99268 1.1021 298.1021 149.6021 14960.21021 1.1021 298.1021 149.6021 14960.21015 1.10210 298.10210 149.60210000000004 14960.21000 2020-01-01 2020-01-02 2020-01-01 00:06:07 2020-01-02 03:34:28 2020-01-01 00:06:07.000 2020-01-02 03:34:28.000 367 99268 49817.5 4981750 367 99268 49817.5 4981750 -32202 32733 5253.02 525302 -128 123 -0.1 -10 +368 101 10358 99269 1.1051 298.1051 149.6051 14960.51051 1.1051 298.1051 149.6051 14960.51031 1.10510 298.10510 149.60510000000014 14960.51000 2020-01-01 2020-01-02 2020-01-01 00:06:08 2020-01-02 03:34:29 2020-01-01 00:06:08.000 2020-01-02 03:34:29.000 368 99269 49818.5 4981850 368 99269 49818.5 4981850 -32201 32734 5254.02 525402 -127 124 0.9 90 +369 101 10359 99270 1.1081 298.1081 149.6081 14960.81081 1.1081 298.1081 149.6081 14960.81062 1.10810 298.10810 149.60809999999995 14960.81000 2020-01-01 2020-01-02 2020-01-01 00:06:09 2020-01-02 03:34:30 2020-01-01 00:06:09.000 2020-01-02 03:34:30.000 369 99270 49819.5 4981950 369 99270 49819.5 4981950 -32200 32735 5255.02 525502 -126 125 1.9 190 +37 102 10027 99937 0.11111 300.11111 150.11111 15161.22222 0.11111 300.1111 150.11111 15161.22248 0.11111 300.11111 150.11110999999985 15161.22211 2020-01-01 2020-01-02 2020-01-01 00:00:37 2020-01-02 03:45:37 2020-01-01 00:00:37.000 2020-01-02 03:45:37.000 37 99937 49987 5048687 37 99937 49987 5048687 -32532 32403 4566.009900990099 461167 -127 124 -1.4356435643564356 -145 +370 101 10360 99271 1.11111 298.11111 149.61111 14961.11111 1.11111 298.1111 149.61111 14961.11137 1.11111 298.11111 149.61110999999985 14961.11100 2020-01-01 2020-01-02 2020-01-01 00:06:10 2020-01-02 03:34:31 2020-01-01 00:06:10.000 2020-01-02 03:34:31.000 370 99271 49820.5 4982050 370 99271 49820.5 4982050 -32199 32736 5256.02 525602 -125 126 2.9 290 +371 101 10361 99272 1.11411 298.11411 149.61411 14961.41141 1.11411 298.1141 149.61411 14961.4115 1.11411 298.11411 149.6141100000003 14961.41100 2020-01-01 2020-01-02 2020-01-01 00:06:11 2020-01-02 03:34:32 2020-01-01 00:06:11.000 2020-01-02 03:34:32.000 371 99272 49821.5 4982150 371 99272 49821.5 4982150 -32198 32737 5257.02 525702 -124 127 3.9 390 +372 101 10362 99273 1.11711 298.11711 149.61711 14961.71171 1.11711 298.11713 149.61711 14961.71165 1.11711 298.11711 149.61710999999985 14961.71100 2020-01-01 2020-01-02 2020-01-01 00:06:12 2020-01-02 03:34:33 2020-01-01 00:06:12.000 2020-01-02 03:34:33.000 372 99273 49822.5 4982250 372 99273 49822.5 4982250 -32197 32738 5258.02 525802 -128 127 2.34 234 +373 101 10363 99274 1.12012 298.12012 149.62012 14962.01201 1.12012 298.12012 149.62011 14962.01179 1.12012 298.12012 149.62011999999973 14962.01200 2020-01-01 2020-01-02 2020-01-01 00:06:13 2020-01-02 03:34:34 2020-01-01 00:06:13.000 2020-01-02 03:34:34.000 373 99274 49823.5 4982350 373 99274 49823.5 4982350 -32196 32739 5259.02 525902 -128 123 0.78 78 +374 101 10364 99275 1.12312 298.12312 149.62312 14962.31231 1.12312 298.1231 149.62312 14962.31208 1.12312 298.12312 149.62312000000017 14962.31200 2020-01-01 2020-01-02 2020-01-01 00:06:14 2020-01-02 03:34:35 2020-01-01 00:06:14.000 2020-01-02 03:34:35.000 374 99275 49824.5 4982450 374 99275 49824.5 4982450 -32195 32740 5260.02 526002 -127 124 1.78 178 +375 101 10365 99276 1.12612 298.12612 149.62612 14962.61261 1.12612 298.12613 149.62612 14962.61283 1.12612 298.12612 149.62612000000018 14962.61200 2020-01-01 2020-01-02 2020-01-01 00:06:15 2020-01-02 03:34:36 2020-01-01 00:06:15.000 2020-01-02 03:34:36.000 375 99276 49825.5 4982550 375 99276 49825.5 4982550 -32194 32741 5261.02 526102 -126 125 2.78 278 +376 101 10366 99277 1.12912 298.12912 149.62912 14962.91291 1.12912 298.12912 149.62912 14962.91297 1.12912 298.12912 149.62911999999974 14962.91200 2020-01-01 2020-01-02 2020-01-01 00:06:16 2020-01-02 03:34:37 2020-01-01 00:06:16.000 2020-01-02 03:34:37.000 376 99277 49826.5 4982650 376 99277 49826.5 4982650 -32193 32742 5262.02 526202 -125 126 3.78 378 377 101 10367 99278 1.13213 298.13213 149.63213 14963.21321 1.13213 298.13214 149.63213 14963.21312 1.13213 298.13213 149.63213 14963.21300 2020-01-01 2020-01-02 2020-01-01 00:06:17 2020-01-02 03:34:38 2020-01-01 00:06:17.000 2020-01-02 03:34:38.000 377 99278 49827.5 4982750 377 99278 49827.5 4982750 -32192 32743 5263.02 526302 -124 127 4.78 478 -378 101 10368 99279 1.13513 298.13513 149.63513 14963.51351 1.13513 298.13513 149.63513 14963.51326 1.13513 298.13513 149.63513 14963.51300 2020-01-01 2020-01-02 2020-01-01 00:06:18 2020-01-02 03:34:39 2020-01-01 00:06:18.000 2020-01-02 03:34:39.000 378 99279 49828.5 4982850 378 99279 49828.5 4982850 -32191 32744 5264.02 526402 -128 127 3.22 322 -379 101 10369 99280 1.13813 298.13813 149.63813 14963.81381 1.13813 298.13815 149.63814 14963.81401 1.13813 298.13813 149.63813 14963.81300 2020-01-01 2020-01-02 2020-01-01 00:06:19 2020-01-02 03:34:40 2020-01-01 00:06:19.000 2020-01-02 03:34:40.000 379 99280 49829.5 4982950 379 99280 49829.5 4982950 -32190 32745 5265.02 526502 -128 127 1.66 166 -38 102 10028 99938 0.11411 300.11411 150.11411 15161.52552 0.11411 300.1141 150.11411 15161.52562 0.11411 300.11411 150.11411 15161.52511 2020-01-01 2020-01-02 2020-01-01 00:00:38 2020-01-02 03:45:38 2020-01-01 00:00:38.000 2020-01-02 03:45:38.000 38 99938 49988 5048788 38 99938 49988 5048788 -32531 32404 4567.009900990099 461268 -126 125 -0.43564356435643564 -44 -380 101 10370 99281 1.14114 298.14114 149.64114 14964.11411 1.14114 298.14114 149.64114 14964.11431 1.14114 298.14114 149.64114 14964.11400 2020-01-01 2020-01-02 2020-01-01 00:06:20 2020-01-02 03:34:41 2020-01-01 00:06:20.000 2020-01-02 03:34:41.000 380 99281 49830.5 4983050 380 99281 49830.5 4983050 -32189 32746 5266.02 526602 -128 124 0.1 10 -381 101 10371 99282 1.14414 298.14414 149.64414 14964.41441 1.14414 298.14413 149.64414 14964.41448 1.14414 298.14414 149.64414 14964.41400 2020-01-01 2020-01-02 2020-01-01 00:06:21 2020-01-02 03:34:42 2020-01-01 00:06:21.000 2020-01-02 03:34:42.000 381 99282 49831.5 4983150 381 99282 49831.5 4983150 -32188 32747 5267.02 526702 -127 125 1.1 110 -382 101 10372 99283 1.14714 298.14714 149.64714 14964.71471 1.14714 298.14716 149.64714 14964.71459 1.14714 298.14714 149.64714 14964.71400 2020-01-01 2020-01-02 2020-01-01 00:06:22 2020-01-02 03:34:43 2020-01-01 00:06:22.000 2020-01-02 03:34:43.000 382 99283 49832.5 4983250 382 99283 49832.5 4983250 -32187 32748 5268.02 526802 -126 126 2.1 210 -383 101 10373 99284 1.15015 298.15015 149.65015 14965.01501 1.15015 298.15015 149.65014 14965.01472 1.15015 298.15015 149.65015 14965.01500 2020-01-01 2020-01-02 2020-01-01 00:06:23 2020-01-02 03:34:44 2020-01-01 00:06:23.000 2020-01-02 03:34:44.000 383 99284 49833.5 4983350 383 99284 49833.5 4983350 -32186 32749 5269.02 526902 -125 127 3.1 310 -384 101 10374 99285 1.15315 298.15315 149.65315 14965.31531 1.15315 298.15317 149.65315 14965.31547 1.15315 298.15315 149.65315 14965.31500 2020-01-01 2020-01-02 2020-01-01 00:06:24 2020-01-02 03:34:45 2020-01-01 00:06:24.000 2020-01-02 03:34:45.000 384 99285 49834.5 4983450 384 99285 49834.5 4983450 -32185 32750 5270.02 527002 -128 127 1.54 154 -385 101 10375 99286 1.15615 298.15615 149.65615 14965.61561 1.15615 298.15616 149.65615 14965.61578 1.15615 298.15615 149.65615 14965.61500 2020-01-01 2020-01-02 2020-01-01 00:06:25 2020-01-02 03:34:46 2020-01-01 00:06:25.000 2020-01-02 03:34:46.000 385 99286 49835.5 4983550 385 99286 49835.5 4983550 -32184 32751 5271.02 527102 -128 127 -0.02 -2 -386 101 10376 99287 1.15915 298.15915 149.65915 14965.91591 1.15915 298.15915 149.65915 14965.91594 1.15915 298.15915 149.65915 14965.91500 2020-01-01 2020-01-02 2020-01-01 00:06:26 2020-01-02 03:34:47 2020-01-01 00:06:26.000 2020-01-02 03:34:47.000 386 99287 49836.5 4983650 386 99287 49836.5 4983650 -32183 32752 5272.02 527202 -128 123 -1.58 -158 -387 101 10377 99288 1.16216 298.16216 149.66216 14966.21621 1.16216 298.16217 149.66216 14966.21606 1.16216 298.16216 149.66216 14966.21600 2020-01-01 2020-01-02 2020-01-01 00:06:27 2020-01-02 03:34:48 2020-01-01 00:06:27.000 2020-01-02 03:34:48.000 387 99288 49837.5 4983750 387 99288 49837.5 4983750 -32182 32753 5273.02 527302 -127 124 -0.58 -58 +378 101 10368 99279 1.13513 298.13513 149.63513 14963.51351 1.13513 298.13513 149.63513 14963.51326 1.13513 298.13513 149.6351300000001 14963.51300 2020-01-01 2020-01-02 2020-01-01 00:06:18 2020-01-02 03:34:39 2020-01-01 00:06:18.000 2020-01-02 03:34:39.000 378 99279 49828.5 4982850 378 99279 49828.5 4982850 -32191 32744 5264.02 526402 -128 127 3.22 322 +379 101 10369 99280 1.13813 298.13813 149.63813 14963.81381 1.13813 298.13815 149.63814 14963.81401 1.13813 298.13813 149.63812999999985 14963.81300 2020-01-01 2020-01-02 2020-01-01 00:06:19 2020-01-02 03:34:40 2020-01-01 00:06:19.000 2020-01-02 03:34:40.000 379 99280 49829.5 4982950 379 99280 49829.5 4982950 -32190 32745 5265.02 526502 -128 127 1.66 166 +38 102 10028 99938 0.11411 300.11411 150.11411 15161.52552 0.11411 300.1141 150.11411 15161.52562 0.11411 300.11411 150.11411000000032 15161.52511 2020-01-01 2020-01-02 2020-01-01 00:00:38 2020-01-02 03:45:38 2020-01-01 00:00:38.000 2020-01-02 03:45:38.000 38 99938 49988 5048788 38 99938 49988 5048788 -32531 32404 4567.009900990099 461268 -126 125 -0.43564356435643564 -44 +380 101 10370 99281 1.14114 298.14114 149.64114 14964.11411 1.14114 298.14114 149.64114 14964.11431 1.14114 298.14114 149.64113999999984 14964.11400 2020-01-01 2020-01-02 2020-01-01 00:06:20 2020-01-02 03:34:41 2020-01-01 00:06:20.000 2020-01-02 03:34:41.000 380 99281 49830.5 4983050 380 99281 49830.5 4983050 -32189 32746 5266.02 526602 -128 124 0.1 10 +381 101 10371 99282 1.14414 298.14414 149.64414 14964.41441 1.14414 298.14413 149.64414 14964.41448 1.14414 298.14414 149.6441400000002 14964.41400 2020-01-01 2020-01-02 2020-01-01 00:06:21 2020-01-02 03:34:42 2020-01-01 00:06:21.000 2020-01-02 03:34:42.000 381 99282 49831.5 4983150 381 99282 49831.5 4983150 -32188 32747 5267.02 526702 -127 125 1.1 110 +382 101 10372 99283 1.14714 298.14714 149.64714 14964.71471 1.14714 298.14716 149.64714 14964.71459 1.14714 298.14714 149.64713999999975 14964.71400 2020-01-01 2020-01-02 2020-01-01 00:06:22 2020-01-02 03:34:43 2020-01-01 00:06:22.000 2020-01-02 03:34:43.000 382 99283 49832.5 4983250 382 99283 49832.5 4983250 -32187 32748 5268.02 526802 -126 126 2.1 210 +383 101 10373 99284 1.15015 298.15015 149.65015 14965.01501 1.15015 298.15015 149.65014 14965.01472 1.15015 298.15015 149.65014999999997 14965.01500 2020-01-01 2020-01-02 2020-01-01 00:06:23 2020-01-02 03:34:44 2020-01-01 00:06:23.000 2020-01-02 03:34:44.000 383 99284 49833.5 4983350 383 99284 49833.5 4983350 -32186 32749 5269.02 526902 -125 127 3.1 310 +384 101 10374 99285 1.15315 298.15315 149.65315 14965.31531 1.15315 298.15317 149.65315 14965.31547 1.15315 298.15315 149.65315000000007 14965.31500 2020-01-01 2020-01-02 2020-01-01 00:06:24 2020-01-02 03:34:45 2020-01-01 00:06:24.000 2020-01-02 03:34:45.000 384 99285 49834.5 4983450 384 99285 49834.5 4983450 -32185 32750 5270.02 527002 -128 127 1.54 154 +385 101 10375 99286 1.15615 298.15615 149.65615 14965.61561 1.15615 298.15616 149.65615 14965.61578 1.15615 298.15615 149.6561500000001 14965.61500 2020-01-01 2020-01-02 2020-01-01 00:06:25 2020-01-02 03:34:46 2020-01-01 00:06:25.000 2020-01-02 03:34:46.000 385 99286 49835.5 4983550 385 99286 49835.5 4983550 -32184 32751 5271.02 527102 -128 127 -0.02 -2 +386 101 10376 99287 1.15915 298.15915 149.65915 14965.91591 1.15915 298.15915 149.65915 14965.91594 1.15915 298.15915 149.65914999999998 14965.91500 2020-01-01 2020-01-02 2020-01-01 00:06:26 2020-01-02 03:34:47 2020-01-01 00:06:26.000 2020-01-02 03:34:47.000 386 99287 49836.5 4983650 386 99287 49836.5 4983650 -32183 32752 5272.02 527202 -128 123 -1.58 -158 +387 101 10377 99288 1.16216 298.16216 149.66216 14966.21621 1.16216 298.16217 149.66216 14966.21606 1.16216 298.16216 149.6621599999999 14966.21600 2020-01-01 2020-01-02 2020-01-01 00:06:27 2020-01-02 03:34:48 2020-01-01 00:06:27.000 2020-01-02 03:34:48.000 387 99288 49837.5 4983750 387 99288 49837.5 4983750 -32182 32753 5273.02 527302 -127 124 -0.58 -58 388 101 10378 99289 1.16516 298.16516 149.66516 14966.51651 1.16516 298.16516 149.66516 14966.51636 1.16516 298.16516 149.66516 14966.51600 2020-01-01 2020-01-02 2020-01-01 00:06:28 2020-01-02 03:34:49 2020-01-01 00:06:28.000 2020-01-02 03:34:49.000 388 99289 49838.5 4983850 388 99289 49838.5 4983850 -32181 32754 5274.02 527402 -126 125 0.42 42 -389 101 10379 99290 1.16816 298.16816 149.66816 14966.81681 1.16816 298.16818 149.66816 14966.81695 1.16816 298.16816 149.66816 14966.81600 2020-01-01 2020-01-02 2020-01-01 00:06:29 2020-01-02 03:34:50 2020-01-01 00:06:29.000 2020-01-02 03:34:50.000 389 99290 49839.5 4983950 389 99290 49839.5 4983950 -32180 32755 5275.02 527502 -125 126 1.42 142 -39 102 10029 99939 0.11711 300.11711 150.11711 15161.82882 0.11711 300.11713 150.11711 15161.82876 0.11711 300.11711 150.11711 15161.82811 2020-01-01 2020-01-02 2020-01-01 00:00:39 2020-01-02 03:45:39 2020-01-01 00:00:39.000 2020-01-02 03:45:39.000 39 99939 49989 5048889 39 99939 49989 5048889 -32530 32405 4568.009900990099 461369 -125 126 0.5643564356435643 57 -390 101 10380 99291 1.17117 298.17117 149.67117 14967.11711 1.17117 298.17117 149.67117 14967.11725 1.17117 298.17117 149.67117 14967.11700 2020-01-01 2020-01-02 2020-01-01 00:06:30 2020-01-02 03:34:51 2020-01-01 00:06:30.000 2020-01-02 03:34:51.000 390 99291 49840.5 4984050 390 99291 49840.5 4984050 -32179 32756 5276.02 527602 -124 127 2.42 242 -391 101 10381 99292 1.17417 298.17417 149.67417 14967.41741 1.17417 298.17416 149.67417 14967.41741 1.17417 298.17417 149.67417 14967.41700 2020-01-01 2020-01-02 2020-01-01 00:06:31 2020-01-02 03:34:52 2020-01-01 00:06:31.000 2020-01-02 03:34:52.000 391 99292 49841.5 4984150 391 99292 49841.5 4984150 -32178 32757 5277.02 527702 -128 127 0.86 86 -392 101 10382 99293 1.17717 298.17717 149.67717 14967.71771 1.17717 298.1772 149.67717 14967.71753 1.17717 298.17717 149.67717 14967.71700 2020-01-01 2020-01-02 2020-01-01 00:06:32 2020-01-02 03:34:53 2020-01-01 00:06:32.000 2020-01-02 03:34:53.000 392 99293 49842.5 4984250 392 99293 49842.5 4984250 -32177 32758 5278.02 527802 -128 123 -0.7 -70 -393 101 10383 99294 1.18018 298.18018 149.68018 14968.01801 1.18018 298.18018 149.68017 14968.01782 1.18018 298.18018 149.68018 14968.01800 2020-01-01 2020-01-02 2020-01-01 00:06:33 2020-01-02 03:34:54 2020-01-01 00:06:33.000 2020-01-02 03:34:54.000 393 99294 49843.5 4984350 393 99294 49843.5 4984350 -32176 32759 5279.02 527902 -127 124 0.3 30 +389 101 10379 99290 1.16816 298.16816 149.66816 14966.81681 1.16816 298.16818 149.66816 14966.81695 1.16816 298.16816 149.66815999999986 14966.81600 2020-01-01 2020-01-02 2020-01-01 00:06:29 2020-01-02 03:34:50 2020-01-01 00:06:29.000 2020-01-02 03:34:50.000 389 99290 49839.5 4983950 389 99290 49839.5 4983950 -32180 32755 5275.02 527502 -125 126 1.42 142 +39 102 10029 99939 0.11711 300.11711 150.11711 15161.82882 0.11711 300.11713 150.11711 15161.82876 0.11711 300.11711 150.11710999999983 15161.82811 2020-01-01 2020-01-02 2020-01-01 00:00:39 2020-01-02 03:45:39 2020-01-01 00:00:39.000 2020-01-02 03:45:39.000 39 99939 49989 5048889 39 99939 49989 5048889 -32530 32405 4568.009900990099 461369 -125 126 0.5643564356435643 57 +390 101 10380 99291 1.17117 298.17117 149.67117 14967.11711 1.17117 298.17117 149.67117 14967.11725 1.17117 298.17117 149.67116999999976 14967.11700 2020-01-01 2020-01-02 2020-01-01 00:06:30 2020-01-02 03:34:51 2020-01-01 00:06:30.000 2020-01-02 03:34:51.000 390 99291 49840.5 4984050 390 99291 49840.5 4984050 -32179 32756 5276.02 527602 -124 127 2.42 242 +391 101 10381 99292 1.17417 298.17417 149.67417 14967.41741 1.17417 298.17416 149.67417 14967.41741 1.17417 298.17417 149.67417000000012 14967.41700 2020-01-01 2020-01-02 2020-01-01 00:06:31 2020-01-02 03:34:52 2020-01-01 00:06:31.000 2020-01-02 03:34:52.000 391 99292 49841.5 4984150 391 99292 49841.5 4984150 -32178 32757 5277.02 527702 -128 127 0.86 86 +392 101 10382 99293 1.17717 298.17717 149.67717 14967.71771 1.17717 298.1772 149.67717 14967.71753 1.17717 298.17717 149.67717000000022 14967.71700 2020-01-01 2020-01-02 2020-01-01 00:06:32 2020-01-02 03:34:53 2020-01-01 00:06:32.000 2020-01-02 03:34:53.000 392 99293 49842.5 4984250 392 99293 49842.5 4984250 -32177 32758 5278.02 527802 -128 123 -0.7 -70 +393 101 10383 99294 1.18018 298.18018 149.68018 14968.01801 1.18018 298.18018 149.68017 14968.01782 1.18018 298.18018 149.68017999999987 14968.01800 2020-01-01 2020-01-02 2020-01-01 00:06:33 2020-01-02 03:34:54 2020-01-01 00:06:33.000 2020-01-02 03:34:54.000 393 99294 49843.5 4984350 393 99294 49843.5 4984350 -32176 32759 5279.02 527902 -127 124 0.3 30 394 101 10384 99295 1.18318 298.18318 149.68318 14968.31831 1.18318 298.1832 149.68318 14968.31842 1.18318 298.18318 149.68318 14968.31800 2020-01-01 2020-01-02 2020-01-01 00:06:34 2020-01-02 03:34:55 2020-01-01 00:06:34.000 2020-01-02 03:34:55.000 394 99295 49844.5 4984450 394 99295 49844.5 4984450 -32175 32760 5280.02 528002 -126 125 1.3 130 -395 101 10385 99296 1.18618 298.18618 149.68618 14968.61861 1.18618 298.1862 149.68618 14968.61875 1.18618 298.18618 149.68618 14968.61800 2020-01-01 2020-01-02 2020-01-01 00:06:35 2020-01-02 03:34:56 2020-01-01 00:06:35.000 2020-01-02 03:34:56.000 395 99296 49845.5 4984550 395 99296 49845.5 4984550 -32174 32761 5281.02 528102 -125 126 2.3 230 -396 101 10386 99297 1.18918 298.18918 149.68918 14968.91891 1.18918 298.18918 149.68918 14968.91889 1.18918 298.18918 149.68918 14968.91800 2020-01-01 2020-01-02 2020-01-01 00:06:36 2020-01-02 03:34:57 2020-01-01 00:06:36.000 2020-01-02 03:34:57.000 396 99297 49846.5 4984650 396 99297 49846.5 4984650 -32173 32762 5282.02 528202 -124 127 3.3 330 -397 101 10387 99298 1.19219 298.19219 149.69219 14969.21921 1.19219 298.1922 149.69219 14969.21964 1.19219 298.19219 149.69219 14969.21900 2020-01-01 2020-01-02 2020-01-01 00:06:37 2020-01-02 03:34:58 2020-01-01 00:06:37.000 2020-01-02 03:34:58.000 397 99298 49847.5 4984750 397 99298 49847.5 4984750 -32172 32763 5283.02 528302 -128 127 1.74 174 -398 101 10388 99299 1.19519 298.19519 149.69519 14969.51951 1.19519 298.1952 149.69519 14969.51929 1.19519 298.19519 149.69519 14969.51900 2020-01-01 2020-01-02 2020-01-01 00:06:38 2020-01-02 03:34:59 2020-01-01 00:06:38.000 2020-01-02 03:34:59.000 398 99299 49848.5 4984850 398 99299 49848.5 4984850 -32171 32764 5284.02 528402 -128 123 0.18 18 -399 101 10389 99300 1.19819 298.19819 149.69819 14969.81981 1.19819 298.1982 149.69819 14969.81989 1.19819 298.19819 149.69819 14969.81900 2020-01-01 2020-01-02 2020-01-01 00:06:39 2020-01-02 03:35:00 2020-01-01 00:06:39.000 2020-01-02 03:35:00.000 399 99300 49849.5 4984950 399 99300 49849.5 4984950 -32170 32765 5285.02 528502 -127 124 1.18 118 -4 102 1003 9994 0.01201 300.01201 150.01201 15151.21321 0.01201 300.01202 150.01201 15151.21318 0.01201 300.01201 150.01201 15151.21301 2020-01-01 2020-01-02 2020-01-01 00:00:04 2020-01-02 03:45:04 2020-01-01 00:00:04.000 2020-01-02 03:45:04.000 4 99904 49954 5045354 4 99904 49954 5045354 -32565 32370 4533.009900990099 457834 -128 127 -1.4851485148514851 -150 -40 102 10030 99940 0.12012 300.12012 150.12012 15162.13213 0.12012 300.12012 150.12011 15162.13191 0.12012 300.12012 150.12012 15162.13212 2020-01-01 2020-01-02 2020-01-01 00:00:40 2020-01-02 03:45:40 2020-01-01 00:00:40.000 2020-01-02 03:45:40.000 40 99940 49990 5048990 40 99940 49990 5048990 -32529 32406 4569.009900990099 461470 -124 127 1.5643564356435644 158 -400 101 10390 99301 1.2012 298.2012 149.7012 14970.12012 1.2012 298.2012 149.7012 14970.12022 1.20120 298.20120 149.70120 14970.12000 2020-01-01 2020-01-02 2020-01-01 00:06:40 2020-01-02 03:35:01 2020-01-01 00:06:40.000 2020-01-02 03:35:01.000 400 99301 49850.5 4985050 400 99301 49850.5 4985050 -32169 32766 5286.02 528602 -126 125 2.18 218 -401 101 10391 99302 1.2042 298.2042 149.7042 14970.42042 1.2042 298.2042 149.7042 14970.42035 1.20420 298.20420 149.70420 14970.42000 2020-01-01 2020-01-02 2020-01-01 00:06:41 2020-01-02 03:35:02 2020-01-01 00:06:41.000 2020-01-02 03:35:02.000 401 99302 49851.5 4985150 401 99302 49851.5 4985150 -32168 32767 5287.02 528702 -125 126 3.18 318 -402 101 10392 99303 1.2072 298.2072 149.7072 14970.72072 1.2072 298.2072 149.70721 14970.72111 1.20720 298.20720 149.70720 14970.72000 2020-01-01 2020-01-02 2020-01-01 00:06:42 2020-01-02 03:35:03 2020-01-01 00:06:42.000 2020-01-02 03:35:03.000 402 99303 49852.5 4985250 402 99303 49852.5 4985250 -32768 32370 4632.66 463266 -124 127 4.18 418 -403 101 10393 99304 1.21021 298.21021 149.71021 14971.02102 1.21021 298.2102 149.7102 14971.02077 1.21021 298.21021 149.71021 14971.02100 2020-01-01 2020-01-02 2020-01-01 00:06:43 2020-01-02 03:35:04 2020-01-01 00:06:43.000 2020-01-02 03:35:04.000 403 99304 49853.5 4985350 403 99304 49853.5 4985350 -32767 32371 4633.66 463366 -128 127 2.62 262 -404 101 10394 99305 1.21321 298.21321 149.71321 14971.32132 1.21321 298.21323 149.71321 14971.32139 1.21321 298.21321 149.71321 14971.32100 2020-01-01 2020-01-02 2020-01-01 00:06:44 2020-01-02 03:35:05 2020-01-01 00:06:44.000 2020-01-02 03:35:05.000 404 99305 49854.5 4985450 404 99305 49854.5 4985450 -32766 32372 4634.66 463466 -128 127 1.06 106 -405 101 10395 99306 1.21621 298.21621 149.71621 14971.62162 1.21621 298.21622 149.71621 14971.62169 1.21621 298.21621 149.71621 14971.62100 2020-01-01 2020-01-02 2020-01-01 00:06:45 2020-01-02 03:35:06 2020-01-01 00:06:45.000 2020-01-02 03:35:06.000 405 99306 49855.5 4985550 405 99306 49855.5 4985550 -32765 32373 4635.66 463566 -128 124 -0.5 -50 -406 101 10396 99307 1.21921 298.21921 149.71921 14971.92192 1.21921 298.2192 149.71921 14971.92199 1.21921 298.21921 149.71921 14971.92100 2020-01-01 2020-01-02 2020-01-01 00:06:46 2020-01-02 03:35:07 2020-01-01 00:06:46.000 2020-01-02 03:35:07.000 406 99307 49856.5 4985650 406 99307 49856.5 4985650 -32764 32374 4636.66 463666 -127 125 0.5 50 -407 101 10397 99308 1.22222 298.22222 149.72222 14972.22222 1.22222 298.22223 149.72222 14972.22257 1.22222 298.22222 149.72222 14972.22200 2020-01-01 2020-01-02 2020-01-01 00:06:47 2020-01-02 03:35:08 2020-01-01 00:06:47.000 2020-01-02 03:35:08.000 407 99308 49857.5 4985750 407 99308 49857.5 4985750 -32763 32375 4637.66 463766 -126 126 1.5 150 -408 101 10398 99309 1.22522 298.22522 149.72522 14972.52252 1.22522 298.22522 149.72522 14972.52224 1.22522 298.22522 149.72522 14972.52200 2020-01-01 2020-01-02 2020-01-01 00:06:48 2020-01-02 03:35:09 2020-01-01 00:06:48.000 2020-01-02 03:35:09.000 408 99309 49858.5 4985850 408 99309 49858.5 4985850 -32762 32376 4638.66 463866 -125 127 2.5 250 -409 101 10399 99310 1.22822 298.22822 149.72822 14972.82282 1.22822 298.22824 149.72822 14972.82286 1.22822 298.22822 149.72822 14972.82200 2020-01-01 2020-01-02 2020-01-01 00:06:49 2020-01-02 03:35:10 2020-01-01 00:06:49.000 2020-01-02 03:35:10.000 409 99310 49859.5 4985950 409 99310 49859.5 4985950 -32761 32377 4639.66 463966 -128 127 0.94 94 -41 102 10031 99941 0.12312 300.12312 150.12312 15162.43543 0.12312 300.1231 150.12312 15162.43521 0.12312 300.12312 150.12312 15162.43512 2020-01-01 2020-01-02 2020-01-01 00:00:41 2020-01-02 03:45:41 2020-01-01 00:00:41.000 2020-01-02 03:45:41.000 41 99941 49991 5049091 41 99941 49991 5049091 -32528 32407 4570.009900990099 461571 -128 127 0.0297029702970297 3 -410 101 10400 99311 1.23123 298.23123 149.73123 14973.12312 1.23123 298.23123 149.73123 14973.12316 1.23123 298.23123 149.73123 14973.12300 2020-01-01 2020-01-02 2020-01-01 00:06:50 2020-01-02 03:35:11 2020-01-01 00:06:50.000 2020-01-02 03:35:11.000 410 99311 49860.5 4986050 410 99311 49860.5 4986050 -32760 32378 4640.66 464066 -128 127 -0.62 -62 -411 101 10401 99312 1.23423 298.23423 149.73423 14973.42342 1.23423 298.23422 149.73423 14973.42345 1.23423 298.23423 149.73423 14973.42300 2020-01-01 2020-01-02 2020-01-01 00:06:51 2020-01-02 03:35:12 2020-01-01 00:06:51.000 2020-01-02 03:35:12.000 411 99312 49861.5 4986150 411 99312 49861.5 4986150 -32759 32379 4641.66 464166 -128 123 -2.18 -218 -412 101 10402 99313 1.23723 298.23723 149.73723 14973.72372 1.23723 298.23724 149.73724 14973.72405 1.23723 298.23723 149.73723 14973.72300 2020-01-01 2020-01-02 2020-01-01 00:06:52 2020-01-02 03:35:13 2020-01-01 00:06:52.000 2020-01-02 03:35:13.000 412 99313 49862.5 4986250 412 99313 49862.5 4986250 -32758 32380 4642.66 464266 -127 124 -1.18 -118 -413 101 10403 99314 1.24024 298.24024 149.74024 14974.02402 1.24024 298.24023 149.74023 14974.02374 1.24024 298.24024 149.74024 14974.02400 2020-01-01 2020-01-02 2020-01-01 00:06:53 2020-01-02 03:35:14 2020-01-01 00:06:53.000 2020-01-02 03:35:14.000 413 99314 49863.5 4986350 413 99314 49863.5 4986350 -32757 32381 4643.66 464366 -126 125 -0.18 -18 -414 101 10404 99315 1.24324 298.24324 149.74324 14974.32432 1.24324 298.24326 149.74324 14974.32433 1.24324 298.24324 149.74324 14974.32400 2020-01-01 2020-01-02 2020-01-01 00:06:54 2020-01-02 03:35:15 2020-01-01 00:06:54.000 2020-01-02 03:35:15.000 414 99315 49864.5 4986450 414 99315 49864.5 4986450 -32756 32382 4644.66 464466 -125 126 0.82 82 -415 101 10405 99316 1.24624 298.24624 149.74624 14974.62462 1.24624 298.24625 149.74624 14974.62463 1.24624 298.24624 149.74624 14974.62400 2020-01-01 2020-01-02 2020-01-01 00:06:55 2020-01-02 03:35:16 2020-01-01 00:06:55.000 2020-01-02 03:35:16.000 415 99316 49865.5 4986550 415 99316 49865.5 4986550 -32755 32383 4645.66 464566 -124 127 1.82 182 -416 101 10406 99317 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924 14974.92400 2020-01-01 2020-01-02 2020-01-01 00:06:56 2020-01-02 03:35:17 2020-01-01 00:06:56.000 2020-01-02 03:35:17.000 416 99317 49866.5 4986650 416 99317 49866.5 4986650 -32754 32384 4646.66 464666 -128 127 0.26 26 -417 101 10407 99318 1.25225 298.25225 149.75225 14975.22522 1.25225 298.25226 149.75225 14975.22552 1.25225 298.25225 149.75225 14975.22500 2020-01-01 2020-01-02 2020-01-01 00:06:57 2020-01-02 03:35:18 2020-01-01 00:06:57.000 2020-01-02 03:35:18.000 417 99318 49867.5 4986750 417 99318 49867.5 4986750 -32753 32385 4647.66 464766 -128 123 -1.3 -130 -418 101 10408 99319 1.25525 298.25525 149.75525 14975.52552 1.25525 298.25525 149.75525 14975.52521 1.25525 298.25525 149.75525 14975.52500 2020-01-01 2020-01-02 2020-01-01 00:06:58 2020-01-02 03:35:19 2020-01-01 00:06:58.000 2020-01-02 03:35:19.000 418 99319 49868.5 4986850 418 99319 49868.5 4986850 -32752 32386 4648.66 464866 -127 124 -0.3 -30 -419 101 10409 99320 1.25825 298.25825 149.75825 14975.82582 1.25825 298.25827 149.75825 14975.8258 1.25825 298.25825 149.75825 14975.82500 2020-01-01 2020-01-02 2020-01-01 00:06:59 2020-01-02 03:35:20 2020-01-01 00:06:59.000 2020-01-02 03:35:20.000 419 99320 49869.5 4986950 419 99320 49869.5 4986950 -32751 32387 4649.66 464966 -126 125 0.7 70 -42 102 10032 99942 0.12612 300.12612 150.12612 15162.73873 0.12612 300.12613 150.12612 15162.73896 0.12612 300.12612 150.12612 15162.73812 2020-01-01 2020-01-02 2020-01-01 00:00:42 2020-01-02 03:45:42 2020-01-01 00:00:42.000 2020-01-02 03:45:42.000 42 99942 49992 5049192 42 99942 49992 5049192 -32527 32408 4571.009900990099 461672 -128 127 -1.504950495049505 -152 -420 101 10410 99321 1.26126 298.26126 149.76126 14976.12612 1.26126 298.26126 149.76126 14976.12609 1.26126 298.26126 149.76126 14976.12600 2020-01-01 2020-01-02 2020-01-01 00:07:00 2020-01-02 03:35:21 2020-01-01 00:07:00.000 2020-01-02 03:35:21.000 420 99321 49870.5 4987050 420 99321 49870.5 4987050 -32750 32388 4650.66 465066 -125 126 1.7 170 -421 101 10411 99322 1.26426 298.26426 149.76426 14976.42642 1.26426 298.26425 149.76426 14976.4264 1.26426 298.26426 149.76426 14976.42600 2020-01-01 2020-01-02 2020-01-01 00:07:01 2020-01-02 03:35:22 2020-01-01 00:07:01.000 2020-01-02 03:35:22.000 421 99322 49871.5 4987150 421 99322 49871.5 4987150 -32749 32389 4651.66 465166 -124 127 2.7 270 -422 101 10412 99323 1.26726 298.26726 149.76726 14976.72672 1.26726 298.26727 149.76727 14976.72702 1.26726 298.26726 149.76726 14976.72600 2020-01-01 2020-01-02 2020-01-01 00:07:02 2020-01-02 03:35:23 2020-01-01 00:07:02.000 2020-01-02 03:35:23.000 422 99323 49872.5 4987250 422 99323 49872.5 4987250 -32748 32390 4652.66 465266 -128 127 1.14 114 -423 101 10413 99324 1.27027 298.27027 149.77027 14977.02702 1.27027 298.27026 149.77026 14977.02667 1.27027 298.27027 149.77027 14977.02700 2020-01-01 2020-01-02 2020-01-01 00:07:03 2020-01-02 03:35:24 2020-01-01 00:07:03.000 2020-01-02 03:35:24.000 423 99324 49873.5 4987350 423 99324 49873.5 4987350 -32747 32391 4653.66 465366 -128 123 -0.42 -42 -424 101 10414 99325 1.27327 298.27327 149.77327 14977.32732 1.27327 298.2733 149.77327 14977.32727 1.27327 298.27327 149.77327 14977.32700 2020-01-01 2020-01-02 2020-01-01 00:07:04 2020-01-02 03:35:25 2020-01-01 00:07:04.000 2020-01-02 03:35:25.000 424 99325 49874.5 4987450 424 99325 49874.5 4987450 -32746 32392 4654.66 465466 -127 124 0.58 58 -425 101 10415 99326 1.27627 298.27627 149.77627 14977.62762 1.27627 298.27628 149.77627 14977.62756 1.27627 298.27627 149.77627 14977.62700 2020-01-01 2020-01-02 2020-01-01 00:07:05 2020-01-02 03:35:26 2020-01-01 00:07:05.000 2020-01-02 03:35:26.000 425 99326 49875.5 4987550 425 99326 49875.5 4987550 -32745 32393 4655.66 465566 -126 125 1.58 158 -426 101 10416 99327 1.27927 298.27927 149.77927 14977.92792 1.27927 298.27927 149.77927 14977.92787 1.27927 298.27927 149.77927 14977.92700 2020-01-01 2020-01-02 2020-01-01 00:07:06 2020-01-02 03:35:27 2020-01-01 00:07:06.000 2020-01-02 03:35:27.000 426 99327 49876.5 4987650 426 99327 49876.5 4987650 -32744 32394 4656.66 465666 -125 126 2.58 258 -427 101 10417 99328 1.28228 298.28228 149.78228 14978.22822 1.28228 298.2823 149.78228 14978.22849 1.28228 298.28228 149.78228 14978.22800 2020-01-01 2020-01-02 2020-01-01 00:07:07 2020-01-02 03:35:28 2020-01-01 00:07:07.000 2020-01-02 03:35:28.000 427 99328 49877.5 4987750 427 99328 49877.5 4987750 -32743 32395 4657.66 465766 -124 127 3.58 358 -428 101 10418 99329 1.28528 298.28528 149.78528 14978.52852 1.28528 298.28528 149.78528 14978.52815 1.28528 298.28528 149.78528 14978.52800 2020-01-01 2020-01-02 2020-01-01 00:07:08 2020-01-02 03:35:29 2020-01-01 00:07:08.000 2020-01-02 03:35:29.000 428 99329 49878.5 4987850 428 99329 49878.5 4987850 -32742 32396 4658.66 465866 -128 127 2.02 202 -429 101 10419 99330 1.28828 298.28828 149.78828 14978.82882 1.28828 298.2883 149.78828 14978.8289 1.28828 298.28828 149.78828 14978.82800 2020-01-01 2020-01-02 2020-01-01 00:07:09 2020-01-02 03:35:30 2020-01-01 00:07:09.000 2020-01-02 03:35:30.000 429 99330 49879.5 4987950 429 99330 49879.5 4987950 -32741 32397 4659.66 465966 -128 127 0.46 46 -43 102 10033 99943 0.12912 300.12912 150.12912 15163.04204 0.12912 300.12912 150.12912 15163.04211 0.12912 300.12912 150.12912 15163.04112 2020-01-01 2020-01-02 2020-01-01 00:00:43 2020-01-02 03:45:43 2020-01-01 00:00:43.000 2020-01-02 03:45:43.000 43 99943 49993 5049293 43 99943 49993 5049293 -32526 32409 4572.009900990099 461773 -128 124 -3.0396039603960396 -307 -430 101 10420 99331 1.29129 298.29129 149.79129 14979.12912 1.29129 298.2913 149.79129 14979.12904 1.29129 298.29129 149.79129 14979.12900 2020-01-01 2020-01-02 2020-01-01 00:07:10 2020-01-02 03:35:31 2020-01-01 00:07:10.000 2020-01-02 03:35:31.000 430 99331 49880.5 4988050 430 99331 49880.5 4988050 -32740 32398 4660.66 466066 -128 124 -1.1 -110 -431 101 10421 99332 1.29429 298.29429 149.79429 14979.42942 1.29429 298.29428 149.79429 14979.42933 1.29429 298.29429 149.79429 14979.42900 2020-01-01 2020-01-02 2020-01-01 00:07:11 2020-01-02 03:35:32 2020-01-01 00:07:11.000 2020-01-02 03:35:32.000 431 99332 49881.5 4988150 431 99332 49881.5 4988150 -32739 32399 4661.66 466166 -127 125 -0.1 -10 -432 101 10422 99333 1.29729 298.29729 149.79729 14979.72972 1.29729 298.2973 149.79729 14979.72996 1.29729 298.29729 149.79729 14979.72900 2020-01-01 2020-01-02 2020-01-01 00:07:12 2020-01-02 03:35:33 2020-01-01 00:07:12.000 2020-01-02 03:35:33.000 432 99333 49882.5 4988250 432 99333 49882.5 4988250 -32738 32400 4662.66 466266 -126 126 0.9 90 -433 101 10423 99334 1.3003 298.3003 149.8003 14980.03003 1.3003 298.3003 149.80029 14980.02962 1.30030 298.30030 149.80030 14980.03000 2020-01-01 2020-01-02 2020-01-01 00:07:13 2020-01-02 03:35:34 2020-01-01 00:07:13.000 2020-01-02 03:35:34.000 433 99334 49883.5 4988350 433 99334 49883.5 4988350 -32737 32401 4663.66 466366 -125 127 1.9 190 -434 101 10424 99335 1.3033 298.3033 149.8033 14980.33033 1.3033 298.3033 149.8033 14980.33037 1.30330 298.30330 149.80330 14980.33000 2020-01-01 2020-01-02 2020-01-01 00:07:14 2020-01-02 03:35:35 2020-01-01 00:07:14.000 2020-01-02 03:35:35.000 434 99335 49884.5 4988450 434 99335 49884.5 4988450 -32736 32402 4664.66 466466 -128 127 0.34 34 -435 101 10425 99336 1.3063 298.3063 149.8063 14980.63063 1.3063 298.3063 149.8063 14980.63051 1.30630 298.30630 149.80630 14980.63000 2020-01-01 2020-01-02 2020-01-01 00:07:15 2020-01-02 03:35:36 2020-01-01 00:07:15.000 2020-01-02 03:35:36.000 435 99336 49885.5 4988550 435 99336 49885.5 4988550 -32735 32403 4665.66 466566 -128 127 -1.22 -122 -436 101 10426 99337 1.3093 298.3093 149.8093 14980.93093 1.3093 298.3093 149.8093 14980.93084 1.30930 298.30930 149.80930 14980.93000 2020-01-01 2020-01-02 2020-01-01 00:07:16 2020-01-02 03:35:37 2020-01-01 00:07:16.000 2020-01-02 03:35:37.000 436 99337 49886.5 4988650 436 99337 49886.5 4988650 -32734 32404 4666.66 466666 -128 123 -2.78 -278 -437 101 10427 99338 1.31231 298.31231 149.81231 14981.23123 1.31231 298.31232 149.81231 14981.23143 1.31231 298.31231 149.81231 14981.23100 2020-01-01 2020-01-02 2020-01-01 00:07:17 2020-01-02 03:35:38 2020-01-01 00:07:17.000 2020-01-02 03:35:38.000 437 99338 49887.5 4988750 437 99338 49887.5 4988750 -32733 32405 4667.66 466766 -127 124 -1.78 -178 -438 101 10428 99339 1.31531 298.31531 149.81531 14981.53153 1.31531 298.3153 149.81531 14981.53173 1.31531 298.31531 149.81531 14981.53100 2020-01-01 2020-01-02 2020-01-01 00:07:18 2020-01-02 03:35:39 2020-01-01 00:07:18.000 2020-01-02 03:35:39.000 438 99339 49888.5 4988850 438 99339 49888.5 4988850 -32732 32406 4668.66 466866 -126 125 -0.78 -78 -439 101 10429 99340 1.31831 298.31831 149.81831 14981.83183 1.31831 298.31833 149.81831 14981.83184 1.31831 298.31831 149.81831 14981.83100 2020-01-01 2020-01-02 2020-01-01 00:07:19 2020-01-02 03:35:40 2020-01-01 00:07:19.000 2020-01-02 03:35:40.000 439 99340 49889.5 4988950 439 99340 49889.5 4988950 -32731 32407 4669.66 466966 -125 126 0.22 22 +395 101 10385 99296 1.18618 298.18618 149.68618 14968.61861 1.18618 298.1862 149.68618 14968.61875 1.18618 298.18618 149.68618000000012 14968.61800 2020-01-01 2020-01-02 2020-01-01 00:06:35 2020-01-02 03:34:56 2020-01-01 00:06:35.000 2020-01-02 03:34:56.000 395 99296 49845.5 4984550 395 99296 49845.5 4984550 -32174 32761 5281.02 528102 -125 126 2.3 230 +396 101 10386 99297 1.18918 298.18918 149.68918 14968.91891 1.18918 298.18918 149.68918 14968.91889 1.18918 298.18918 149.68917999999988 14968.91800 2020-01-01 2020-01-02 2020-01-01 00:06:36 2020-01-02 03:34:57 2020-01-01 00:06:36.000 2020-01-02 03:34:57.000 396 99297 49846.5 4984650 396 99297 49846.5 4984650 -32173 32762 5282.02 528202 -124 127 3.3 330 +397 101 10387 99298 1.19219 298.19219 149.69219 14969.21921 1.19219 298.1922 149.69219 14969.21964 1.19219 298.19219 149.69218999999987 14969.21900 2020-01-01 2020-01-02 2020-01-01 00:06:37 2020-01-02 03:34:58 2020-01-01 00:06:37.000 2020-01-02 03:34:58.000 397 99298 49847.5 4984750 397 99298 49847.5 4984750 -32172 32763 5283.02 528302 -128 127 1.74 174 +398 101 10388 99299 1.19519 298.19519 149.69519 14969.51951 1.19519 298.1952 149.69519 14969.51929 1.19519 298.19519 149.69519000000022 14969.51900 2020-01-01 2020-01-02 2020-01-01 00:06:38 2020-01-02 03:34:59 2020-01-01 00:06:38.000 2020-01-02 03:34:59.000 398 99299 49848.5 4984850 398 99299 49848.5 4984850 -32171 32764 5284.02 528402 -128 123 0.18 18 +399 101 10389 99300 1.19819 298.19819 149.69819 14969.81981 1.19819 298.1982 149.69819 14969.81989 1.19819 298.19819 149.69818999999978 14969.81900 2020-01-01 2020-01-02 2020-01-01 00:06:39 2020-01-02 03:35:00 2020-01-01 00:06:39.000 2020-01-02 03:35:00.000 399 99300 49849.5 4984950 399 99300 49849.5 4984950 -32170 32765 5285.02 528502 -127 124 1.18 118 +4 102 1003 9994 0.01201 300.01201 150.01201 15151.21321 0.01201 300.01202 150.01201 15151.21318 0.01201 300.01201 150.01201000000026 15151.21301 2020-01-01 2020-01-02 2020-01-01 00:00:04 2020-01-02 03:45:04 2020-01-01 00:00:04.000 2020-01-02 03:45:04.000 4 99904 49954 5045354 4 99904 49954 5045354 -32565 32370 4533.009900990099 457834 -128 127 -1.4851485148514851 -150 +40 102 10030 99940 0.12012 300.12012 150.12012 15162.13213 0.12012 300.12012 150.12011 15162.13191 0.12012 300.12012 150.1201199999997 15162.13212 2020-01-01 2020-01-02 2020-01-01 00:00:40 2020-01-02 03:45:40 2020-01-01 00:00:40.000 2020-01-02 03:45:40.000 40 99940 49990 5048990 40 99940 49990 5048990 -32529 32406 4569.009900990099 461470 -124 127 1.5643564356435644 158 +400 101 10390 99301 1.2012 298.2012 149.7012 14970.12012 1.2012 298.2012 149.7012 14970.12022 1.20120 298.20120 149.7012 14970.12000 2020-01-01 2020-01-02 2020-01-01 00:06:40 2020-01-02 03:35:01 2020-01-01 00:06:40.000 2020-01-02 03:35:01.000 400 99301 49850.5 4985050 400 99301 49850.5 4985050 -32169 32766 5286.02 528602 -126 125 2.18 218 +401 101 10391 99302 1.2042 298.2042 149.7042 14970.42042 1.2042 298.2042 149.7042 14970.42035 1.20420 298.20420 149.7042000000001 14970.42000 2020-01-01 2020-01-02 2020-01-01 00:06:41 2020-01-02 03:35:02 2020-01-01 00:06:41.000 2020-01-02 03:35:02.000 401 99302 49851.5 4985150 401 99302 49851.5 4985150 -32168 32767 5287.02 528702 -125 126 3.18 318 +402 101 10392 99303 1.2072 298.2072 149.7072 14970.72072 1.2072 298.2072 149.70721 14970.72111 1.20720 298.20720 149.70720000000014 14970.72000 2020-01-01 2020-01-02 2020-01-01 00:06:42 2020-01-02 03:35:03 2020-01-01 00:06:42.000 2020-01-02 03:35:03.000 402 99303 49852.5 4985250 402 99303 49852.5 4985250 -32768 32370 4632.66 463266 -124 127 4.18 418 +403 101 10393 99304 1.21021 298.21021 149.71021 14971.02102 1.21021 298.2102 149.7102 14971.02077 1.21021 298.21021 149.71020999999988 14971.02100 2020-01-01 2020-01-02 2020-01-01 00:06:43 2020-01-02 03:35:04 2020-01-01 00:06:43.000 2020-01-02 03:35:04.000 403 99304 49853.5 4985350 403 99304 49853.5 4985350 -32767 32371 4633.66 463366 -128 127 2.62 262 +404 101 10394 99305 1.21321 298.21321 149.71321 14971.32132 1.21321 298.21323 149.71321 14971.32139 1.21321 298.21321 149.71320999999992 14971.32100 2020-01-01 2020-01-02 2020-01-01 00:06:44 2020-01-02 03:35:05 2020-01-01 00:06:44.000 2020-01-02 03:35:05.000 404 99305 49854.5 4985450 404 99305 49854.5 4985450 -32766 32372 4634.66 463466 -128 127 1.06 106 +405 101 10395 99306 1.21621 298.21621 149.71621 14971.62162 1.21621 298.21622 149.71621 14971.62169 1.21621 298.21621 149.71621000000002 14971.62100 2020-01-01 2020-01-02 2020-01-01 00:06:45 2020-01-02 03:35:06 2020-01-01 00:06:45.000 2020-01-02 03:35:06.000 405 99306 49855.5 4985550 405 99306 49855.5 4985550 -32765 32373 4635.66 463566 -128 124 -0.5 -50 +406 101 10396 99307 1.21921 298.21921 149.71921 14971.92192 1.21921 298.2192 149.71921 14971.92199 1.21921 298.21921 149.7192099999999 14971.92100 2020-01-01 2020-01-02 2020-01-01 00:06:46 2020-01-02 03:35:07 2020-01-01 00:06:46.000 2020-01-02 03:35:07.000 406 99307 49856.5 4985650 406 99307 49856.5 4985650 -32764 32374 4636.66 463666 -127 125 0.5 50 +407 101 10397 99308 1.22222 298.22222 149.72222 14972.22222 1.22222 298.22223 149.72222 14972.22257 1.22222 298.22222 149.7222199999998 14972.22200 2020-01-01 2020-01-02 2020-01-01 00:06:47 2020-01-02 03:35:08 2020-01-01 00:06:47.000 2020-01-02 03:35:08.000 407 99308 49857.5 4985750 407 99308 49857.5 4985750 -32763 32375 4637.66 463766 -126 126 1.5 150 +408 101 10398 99309 1.22522 298.22522 149.72522 14972.52252 1.22522 298.22522 149.72522 14972.52224 1.22522 298.22522 149.72522000000015 14972.52200 2020-01-01 2020-01-02 2020-01-01 00:06:48 2020-01-02 03:35:09 2020-01-01 00:06:48.000 2020-01-02 03:35:09.000 408 99309 49858.5 4985850 408 99309 49858.5 4985850 -32762 32376 4638.66 463866 -125 127 2.5 250 +409 101 10399 99310 1.22822 298.22822 149.72822 14972.82282 1.22822 298.22824 149.72822 14972.82286 1.22822 298.22822 149.72822000000025 14972.82200 2020-01-01 2020-01-02 2020-01-01 00:06:49 2020-01-02 03:35:10 2020-01-01 00:06:49.000 2020-01-02 03:35:10.000 409 99310 49859.5 4985950 409 99310 49859.5 4985950 -32761 32377 4639.66 463966 -128 127 0.94 94 +41 102 10031 99941 0.12312 300.12312 150.12312 15162.43543 0.12312 300.1231 150.12312 15162.43521 0.12312 300.12312 150.12312000000017 15162.43512 2020-01-01 2020-01-02 2020-01-01 00:00:41 2020-01-02 03:45:41 2020-01-01 00:00:41.000 2020-01-02 03:45:41.000 41 99941 49991 5049091 41 99941 49991 5049091 -32528 32407 4570.009900990099 461571 -128 127 0.0297029702970297 3 +410 101 10400 99311 1.23123 298.23123 149.73123 14973.12312 1.23123 298.23123 149.73123 14973.12316 1.23123 298.23123 149.7312299999999 14973.12300 2020-01-01 2020-01-02 2020-01-01 00:06:50 2020-01-02 03:35:11 2020-01-01 00:06:50.000 2020-01-02 03:35:11.000 410 99311 49860.5 4986050 410 99311 49860.5 4986050 -32760 32378 4640.66 464066 -128 127 -0.62 -62 +411 101 10401 99312 1.23423 298.23423 149.73423 14973.42342 1.23423 298.23422 149.73423 14973.42345 1.23423 298.23423 149.73423000000003 14973.42300 2020-01-01 2020-01-02 2020-01-01 00:06:51 2020-01-02 03:35:12 2020-01-01 00:06:51.000 2020-01-02 03:35:12.000 411 99312 49861.5 4986150 411 99312 49861.5 4986150 -32759 32379 4641.66 464166 -128 123 -2.18 -218 +412 101 10402 99313 1.23723 298.23723 149.73723 14973.72372 1.23723 298.23724 149.73724 14973.72405 1.23723 298.23723 149.73723000000015 14973.72300 2020-01-01 2020-01-02 2020-01-01 00:06:52 2020-01-02 03:35:13 2020-01-01 00:06:52.000 2020-01-02 03:35:13.000 412 99313 49862.5 4986250 412 99313 49862.5 4986250 -32758 32380 4642.66 464266 -127 124 -1.18 -118 +413 101 10403 99314 1.24024 298.24024 149.74024 14974.02402 1.24024 298.24023 149.74023 14974.02374 1.24024 298.24024 149.7402399999998 14974.02400 2020-01-01 2020-01-02 2020-01-01 00:06:53 2020-01-02 03:35:14 2020-01-01 00:06:53.000 2020-01-02 03:35:14.000 413 99314 49863.5 4986350 413 99314 49863.5 4986350 -32757 32381 4643.66 464366 -126 125 -0.18 -18 +414 101 10404 99315 1.24324 298.24324 149.74324 14974.32432 1.24324 298.24326 149.74324 14974.32433 1.24324 298.24324 149.74323999999982 14974.32400 2020-01-01 2020-01-02 2020-01-01 00:06:54 2020-01-02 03:35:15 2020-01-01 00:06:54.000 2020-01-02 03:35:15.000 414 99315 49864.5 4986450 414 99315 49864.5 4986450 -32756 32382 4644.66 464466 -125 126 0.82 82 +415 101 10405 99316 1.24624 298.24624 149.74624 14974.62462 1.24624 298.24625 149.74624 14974.62463 1.24624 298.24624 149.74624000000026 14974.62400 2020-01-01 2020-01-02 2020-01-01 00:06:55 2020-01-02 03:35:16 2020-01-01 00:06:55.000 2020-01-02 03:35:16.000 415 99316 49865.5 4986550 415 99316 49865.5 4986550 -32755 32383 4645.66 464566 -124 127 1.82 182 +416 101 10406 99317 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74923999999982 14974.92400 2020-01-01 2020-01-02 2020-01-01 00:06:56 2020-01-02 03:35:17 2020-01-01 00:06:56.000 2020-01-02 03:35:17.000 416 99317 49866.5 4986650 416 99317 49866.5 4986650 -32754 32384 4646.66 464666 -128 127 0.26 26 +417 101 10407 99318 1.25225 298.25225 149.75225 14975.22522 1.25225 298.25226 149.75225 14975.22552 1.25225 298.25225 149.7522499999997 14975.22500 2020-01-01 2020-01-02 2020-01-01 00:06:57 2020-01-02 03:35:18 2020-01-01 00:06:57.000 2020-01-02 03:35:18.000 417 99318 49867.5 4986750 417 99318 49867.5 4986750 -32753 32385 4647.66 464766 -128 123 -1.3 -130 +418 101 10408 99319 1.25525 298.25525 149.75525 14975.52552 1.25525 298.25525 149.75525 14975.52521 1.25525 298.25525 149.75525000000013 14975.52500 2020-01-01 2020-01-02 2020-01-01 00:06:58 2020-01-02 03:35:19 2020-01-01 00:06:58.000 2020-01-02 03:35:19.000 418 99319 49868.5 4986850 418 99319 49868.5 4986850 -32752 32386 4648.66 464866 -127 124 -0.3 -30 +419 101 10409 99320 1.25825 298.25825 149.75825 14975.82582 1.25825 298.25827 149.75825 14975.8258 1.25825 298.25825 149.75825000000015 14975.82500 2020-01-01 2020-01-02 2020-01-01 00:06:59 2020-01-02 03:35:20 2020-01-01 00:06:59.000 2020-01-02 03:35:20.000 419 99320 49869.5 4986950 419 99320 49869.5 4986950 -32751 32387 4649.66 464966 -126 125 0.7 70 +42 102 10032 99942 0.12612 300.12612 150.12612 15162.73873 0.12612 300.12613 150.12612 15162.73896 0.12612 300.12612 150.1261200000002 15162.73812 2020-01-01 2020-01-02 2020-01-01 00:00:42 2020-01-02 03:45:42 2020-01-01 00:00:42.000 2020-01-02 03:45:42.000 42 99942 49992 5049192 42 99942 49992 5049192 -32527 32408 4571.009900990099 461672 -128 127 -1.504950495049505 -152 +420 101 10410 99321 1.26126 298.26126 149.76126 14976.12612 1.26126 298.26126 149.76126 14976.12609 1.26126 298.26126 149.7612599999999 14976.12600 2020-01-01 2020-01-02 2020-01-01 00:07:00 2020-01-02 03:35:21 2020-01-01 00:07:00.000 2020-01-02 03:35:21.000 420 99321 49870.5 4987050 420 99321 49870.5 4987050 -32750 32388 4650.66 465066 -125 126 1.7 170 +421 101 10411 99322 1.26426 298.26426 149.76426 14976.42642 1.26426 298.26425 149.76426 14976.4264 1.26426 298.26426 149.76425999999995 14976.42600 2020-01-01 2020-01-02 2020-01-01 00:07:01 2020-01-02 03:35:22 2020-01-01 00:07:01.000 2020-01-02 03:35:22.000 421 99322 49871.5 4987150 421 99322 49871.5 4987150 -32749 32389 4651.66 465166 -124 127 2.7 270 +422 101 10412 99323 1.26726 298.26726 149.76726 14976.72672 1.26726 298.26727 149.76727 14976.72702 1.26726 298.26726 149.76726000000005 14976.72600 2020-01-01 2020-01-02 2020-01-01 00:07:02 2020-01-02 03:35:23 2020-01-01 00:07:02.000 2020-01-02 03:35:23.000 422 99323 49872.5 4987250 422 99323 49872.5 4987250 -32748 32390 4652.66 465266 -128 127 1.14 114 +423 101 10413 99324 1.27027 298.27027 149.77027 14977.02702 1.27027 298.27026 149.77026 14977.02667 1.27027 298.27027 149.77027000000027 14977.02700 2020-01-01 2020-01-02 2020-01-01 00:07:03 2020-01-02 03:35:24 2020-01-01 00:07:03.000 2020-01-02 03:35:24.000 423 99324 49873.5 4987350 423 99324 49873.5 4987350 -32747 32391 4653.66 465366 -128 123 -0.42 -42 +424 101 10414 99325 1.27327 298.27327 149.77327 14977.32732 1.27327 298.2733 149.77327 14977.32727 1.27327 298.27327 149.77326999999983 14977.32700 2020-01-01 2020-01-02 2020-01-01 00:07:04 2020-01-02 03:35:25 2020-01-01 00:07:04.000 2020-01-02 03:35:25.000 424 99325 49874.5 4987450 424 99325 49874.5 4987450 -32746 32392 4654.66 465466 -127 124 0.58 58 +425 101 10415 99326 1.27627 298.27627 149.77627 14977.62762 1.27627 298.27628 149.77627 14977.62756 1.27627 298.27627 149.77627000000018 14977.62700 2020-01-01 2020-01-02 2020-01-01 00:07:05 2020-01-02 03:35:26 2020-01-01 00:07:05.000 2020-01-02 03:35:26.000 425 99326 49875.5 4987550 425 99326 49875.5 4987550 -32745 32393 4655.66 465566 -126 125 1.58 158 +426 101 10416 99327 1.27927 298.27927 149.77927 14977.92792 1.27927 298.27927 149.77927 14977.92787 1.27927 298.27927 149.7792699999997 14977.92700 2020-01-01 2020-01-02 2020-01-01 00:07:06 2020-01-02 03:35:27 2020-01-01 00:07:06.000 2020-01-02 03:35:27.000 426 99327 49876.5 4987650 426 99327 49876.5 4987650 -32744 32394 4656.66 465666 -125 126 2.58 258 +427 101 10417 99328 1.28228 298.28228 149.78228 14978.22822 1.28228 298.2823 149.78228 14978.22849 1.28228 298.28228 149.78227999999993 14978.22800 2020-01-01 2020-01-02 2020-01-01 00:07:07 2020-01-02 03:35:28 2020-01-01 00:07:07.000 2020-01-02 03:35:28.000 427 99328 49877.5 4987750 427 99328 49877.5 4987750 -32743 32395 4657.66 465766 -124 127 3.58 358 +428 101 10418 99329 1.28528 298.28528 149.78528 14978.52852 1.28528 298.28528 149.78528 14978.52815 1.28528 298.28528 149.78528000000003 14978.52800 2020-01-01 2020-01-02 2020-01-01 00:07:08 2020-01-02 03:35:29 2020-01-01 00:07:08.000 2020-01-02 03:35:29.000 428 99329 49878.5 4987850 428 99329 49878.5 4987850 -32742 32396 4658.66 465866 -128 127 2.02 202 +429 101 10419 99330 1.28828 298.28828 149.78828 14978.82882 1.28828 298.2883 149.78828 14978.8289 1.28828 298.28828 149.78828000000013 14978.82800 2020-01-01 2020-01-02 2020-01-01 00:07:09 2020-01-02 03:35:30 2020-01-01 00:07:09.000 2020-01-02 03:35:30.000 429 99330 49879.5 4987950 429 99330 49879.5 4987950 -32741 32397 4659.66 465966 -128 127 0.46 46 +43 102 10033 99943 0.12912 300.12912 150.12912 15163.04204 0.12912 300.12912 150.12912 15163.04211 0.12912 300.12912 150.12911999999972 15163.04112 2020-01-01 2020-01-02 2020-01-01 00:00:43 2020-01-02 03:45:43 2020-01-01 00:00:43.000 2020-01-02 03:45:43.000 43 99943 49993 5049293 43 99943 49993 5049293 -32526 32409 4572.009900990099 461773 -128 124 -3.0396039603960396 -307 +430 101 10420 99331 1.29129 298.29129 149.79129 14979.12912 1.29129 298.2913 149.79129 14979.12904 1.29129 298.29129 149.79128999999983 14979.12900 2020-01-01 2020-01-02 2020-01-01 00:07:10 2020-01-02 03:35:31 2020-01-01 00:07:10.000 2020-01-02 03:35:31.000 430 99331 49880.5 4988050 430 99331 49880.5 4988050 -32740 32398 4660.66 466066 -128 124 -1.1 -110 +431 101 10421 99332 1.29429 298.29429 149.79429 14979.42942 1.29429 298.29428 149.79429 14979.42933 1.29429 298.29429 149.79428999999985 14979.42900 2020-01-01 2020-01-02 2020-01-01 00:07:11 2020-01-02 03:35:32 2020-01-01 00:07:11.000 2020-01-02 03:35:32.000 431 99332 49881.5 4988150 431 99332 49881.5 4988150 -32739 32399 4661.66 466166 -127 125 -0.1 -10 +432 101 10422 99333 1.29729 298.29729 149.79729 14979.72972 1.29729 298.2973 149.79729 14979.72996 1.29729 298.29729 149.7972900000003 14979.72900 2020-01-01 2020-01-02 2020-01-01 00:07:12 2020-01-02 03:35:33 2020-01-01 00:07:12.000 2020-01-02 03:35:33.000 432 99333 49882.5 4988250 432 99333 49882.5 4988250 -32738 32400 4662.66 466266 -126 126 0.9 90 +433 101 10423 99334 1.3003 298.3003 149.8003 14980.03003 1.3003 298.3003 149.80029 14980.02962 1.30030 298.30030 149.80030000000016 14980.03000 2020-01-01 2020-01-02 2020-01-01 00:07:13 2020-01-02 03:35:34 2020-01-01 00:07:13.000 2020-01-02 03:35:34.000 433 99334 49883.5 4988350 433 99334 49883.5 4988350 -32737 32401 4663.66 466366 -125 127 1.9 190 +434 101 10424 99335 1.3033 298.3033 149.8033 14980.33033 1.3033 298.3033 149.8033 14980.33037 1.30330 298.30330 149.80329999999972 14980.33000 2020-01-01 2020-01-02 2020-01-01 00:07:14 2020-01-02 03:35:35 2020-01-01 00:07:14.000 2020-01-02 03:35:35.000 434 99335 49884.5 4988450 434 99335 49884.5 4988450 -32736 32402 4664.66 466466 -128 127 0.34 34 +435 101 10425 99336 1.3063 298.3063 149.8063 14980.63063 1.3063 298.3063 149.8063 14980.63051 1.30630 298.30630 149.80630000000016 14980.63000 2020-01-01 2020-01-02 2020-01-01 00:07:15 2020-01-02 03:35:36 2020-01-01 00:07:15.000 2020-01-02 03:35:36.000 435 99336 49885.5 4988550 435 99336 49885.5 4988550 -32735 32403 4665.66 466566 -128 127 -1.22 -122 +436 101 10426 99337 1.3093 298.3093 149.8093 14980.93093 1.3093 298.3093 149.8093 14980.93084 1.30930 298.30930 149.80930000000018 14980.93000 2020-01-01 2020-01-02 2020-01-01 00:07:16 2020-01-02 03:35:37 2020-01-01 00:07:16.000 2020-01-02 03:35:37.000 436 99337 49886.5 4988650 436 99337 49886.5 4988650 -32734 32404 4666.66 466666 -128 123 -2.78 -278 +437 101 10427 99338 1.31231 298.31231 149.81231 14981.23123 1.31231 298.31232 149.81231 14981.23143 1.31231 298.31231 149.81231000000008 14981.23100 2020-01-01 2020-01-02 2020-01-01 00:07:17 2020-01-02 03:35:38 2020-01-01 00:07:17.000 2020-01-02 03:35:38.000 437 99338 49887.5 4988750 437 99338 49887.5 4988750 -32733 32405 4667.66 466766 -127 124 -1.78 -178 +438 101 10428 99339 1.31531 298.31531 149.81531 14981.53153 1.31531 298.3153 149.81531 14981.53173 1.31531 298.31531 149.81530999999998 14981.53100 2020-01-01 2020-01-02 2020-01-01 00:07:18 2020-01-02 03:35:39 2020-01-01 00:07:18.000 2020-01-02 03:35:39.000 438 99339 49888.5 4988850 438 99339 49888.5 4988850 -32732 32406 4668.66 466866 -126 125 -0.78 -78 +439 101 10429 99340 1.31831 298.31831 149.81831 14981.83183 1.31831 298.31833 149.81831 14981.83184 1.31831 298.31831 149.81831000000005 14981.83100 2020-01-01 2020-01-02 2020-01-01 00:07:19 2020-01-02 03:35:40 2020-01-01 00:07:19.000 2020-01-02 03:35:40.000 439 99340 49889.5 4988950 439 99340 49889.5 4988950 -32731 32407 4669.66 466966 -125 126 0.22 22 44 102 10034 99944 0.13213 300.13213 150.13213 15163.34534 0.13213 300.13214 150.13213 15163.34525 0.13213 300.13213 150.13213 15163.34513 2020-01-01 2020-01-02 2020-01-01 00:00:44 2020-01-02 03:45:44 2020-01-01 00:00:44.000 2020-01-02 03:45:44.000 44 99944 49994 5049394 44 99944 49994 5049394 -32525 32410 4573.009900990099 461874 -127 125 -2.0396039603960396 -206 -440 101 10430 99341 1.32132 298.32132 149.82132 14982.13213 1.32132 298.32132 149.82131 14982.13197 1.32132 298.32132 149.82132 14982.13200 2020-01-01 2020-01-02 2020-01-01 00:07:20 2020-01-02 03:35:41 2020-01-01 00:07:20.000 2020-01-02 03:35:41.000 440 99341 49890.5 4989050 440 99341 49890.5 4989050 -32730 32408 4670.66 467066 -124 127 1.22 122 -441 101 10431 99342 1.32432 298.32432 149.82432 14982.43243 1.32432 298.3243 149.82432 14982.4323 1.32432 298.32432 149.82432 14982.43200 2020-01-01 2020-01-02 2020-01-01 00:07:21 2020-01-02 03:35:42 2020-01-01 00:07:21.000 2020-01-02 03:35:42.000 441 99342 49891.5 4989150 441 99342 49891.5 4989150 -32729 32409 4671.66 467166 -128 127 -0.34 -34 -442 101 10432 99343 1.32732 298.32732 149.82732 14982.73273 1.32732 298.32733 149.82732 14982.7329 1.32732 298.32732 149.82732 14982.73200 2020-01-01 2020-01-02 2020-01-01 00:07:22 2020-01-02 03:35:43 2020-01-01 00:07:22.000 2020-01-02 03:35:43.000 442 99343 49892.5 4989250 442 99343 49892.5 4989250 -32728 32410 4672.66 467266 -128 123 -1.9 -190 +440 101 10430 99341 1.32132 298.32132 149.82132 14982.13213 1.32132 298.32132 149.82131 14982.13197 1.32132 298.32132 149.82131999999987 14982.13200 2020-01-01 2020-01-02 2020-01-01 00:07:20 2020-01-02 03:35:41 2020-01-01 00:07:20.000 2020-01-02 03:35:41.000 440 99341 49890.5 4989050 440 99341 49890.5 4989050 -32730 32408 4670.66 467066 -124 127 1.22 122 +441 101 10431 99342 1.32432 298.32432 149.82432 14982.43243 1.32432 298.3243 149.82432 14982.4323 1.32432 298.32432 149.82431999999994 14982.43200 2020-01-01 2020-01-02 2020-01-01 00:07:21 2020-01-02 03:35:42 2020-01-01 00:07:21.000 2020-01-02 03:35:42.000 441 99342 49891.5 4989150 441 99342 49891.5 4989150 -32729 32409 4671.66 467166 -128 127 -0.34 -34 +442 101 10432 99343 1.32732 298.32732 149.82732 14982.73273 1.32732 298.32733 149.82732 14982.7329 1.32732 298.32732 149.8273200000001 14982.73200 2020-01-01 2020-01-02 2020-01-01 00:07:22 2020-01-02 03:35:43 2020-01-01 00:07:22.000 2020-01-02 03:35:43.000 442 99343 49892.5 4989250 442 99343 49892.5 4989250 -32728 32410 4672.66 467266 -128 123 -1.9 -190 443 101 10433 99344 1.33033 298.33033 149.83033 14983.03303 1.33033 298.33032 149.83033 14983.03319 1.33033 298.33033 149.83033 14983.03300 2020-01-01 2020-01-02 2020-01-01 00:07:23 2020-01-02 03:35:44 2020-01-01 00:07:23.000 2020-01-02 03:35:44.000 443 99344 49893.5 4989350 443 99344 49893.5 4989350 -32727 32411 4673.66 467366 -127 124 -0.9 -90 444 101 10434 99345 1.33333 298.33333 149.83333 14983.33333 1.33333 298.33334 149.83333 14983.33331 1.33333 298.33333 149.83333 14983.33300 2020-01-01 2020-01-02 2020-01-01 00:07:24 2020-01-02 03:35:45 2020-01-01 00:07:24.000 2020-01-02 03:35:45.000 444 99345 49894.5 4989450 444 99345 49894.5 4989450 -32726 32412 4674.66 467466 -126 125 0.1 10 -445 101 10435 99346 1.33633 298.33633 149.83633 14983.63363 1.33633 298.33633 149.83633 14983.63348 1.33633 298.33633 149.83633 14983.63300 2020-01-01 2020-01-02 2020-01-01 00:07:25 2020-01-02 03:35:46 2020-01-01 00:07:25.000 2020-01-02 03:35:46.000 445 99346 49895.5 4989550 445 99346 49895.5 4989550 -32725 32413 4675.66 467566 -125 126 1.1 110 -446 101 10436 99347 1.33933 298.33933 149.83933 14983.93393 1.33933 298.33932 149.83933 14983.93378 1.33933 298.33933 149.83933 14983.93300 2020-01-01 2020-01-02 2020-01-01 00:07:26 2020-01-02 03:35:47 2020-01-01 00:07:26.000 2020-01-02 03:35:47.000 446 99347 49896.5 4989650 446 99347 49896.5 4989650 -32724 32414 4676.66 467666 -124 127 2.1 210 -447 101 10437 99348 1.34234 298.34234 149.84234 14984.23423 1.34234 298.34235 149.84234 14984.23437 1.34234 298.34234 149.84234 14984.23400 2020-01-01 2020-01-02 2020-01-01 00:07:27 2020-01-02 03:35:48 2020-01-01 00:07:27.000 2020-01-02 03:35:48.000 447 99348 49897.5 4989750 447 99348 49897.5 4989750 -32723 32415 4677.66 467766 -128 127 0.54 54 -448 101 10438 99349 1.34534 298.34534 149.84534 14984.53453 1.34534 298.34534 149.84534 14984.53466 1.34534 298.34534 149.84534 14984.53400 2020-01-01 2020-01-02 2020-01-01 00:07:28 2020-01-02 03:35:49 2020-01-01 00:07:28.000 2020-01-02 03:35:49.000 448 99349 49898.5 4989850 448 99349 49898.5 4989850 -32722 32416 4678.66 467866 -128 123 -1.02 -102 -449 101 10439 99350 1.34834 298.34834 149.84834 14984.83483 1.34834 298.34836 149.84834 14984.83478 1.34834 298.34834 149.84834 14984.83400 2020-01-01 2020-01-02 2020-01-01 00:07:29 2020-01-02 03:35:50 2020-01-01 00:07:29.000 2020-01-02 03:35:50.000 449 99350 49899.5 4989950 449 99350 49899.5 4989950 -32721 32417 4679.66 467966 -127 124 -0.02 -2 -45 102 10035 99945 0.13513 300.13513 150.13513 15163.64864 0.13513 300.13513 150.13513 15163.64839 0.13513 300.13513 150.13513 15163.64813 2020-01-01 2020-01-02 2020-01-01 00:00:45 2020-01-02 03:45:45 2020-01-01 00:00:45.000 2020-01-02 03:45:45.000 45 99945 49995 5049495 45 99945 49995 5049495 -32524 32411 4574.009900990099 461975 -126 126 -1.0396039603960396 -105 -450 101 10440 99351 1.35135 298.35135 149.85135 14985.13513 1.35135 298.35135 149.85134 14985.13495 1.35135 298.35135 149.85135 14985.13500 2020-01-01 2020-01-02 2020-01-01 00:07:30 2020-01-02 03:35:51 2020-01-01 00:07:30.000 2020-01-02 03:35:51.000 450 99351 49900.5 4990050 450 99351 49900.5 4990050 -32720 32418 4680.66 468066 -126 125 0.98 98 -451 101 10441 99352 1.35435 298.35435 149.85435 14985.43543 1.35435 298.35434 149.85435 14985.43525 1.35435 298.35435 149.85435 14985.43500 2020-01-01 2020-01-02 2020-01-01 00:07:31 2020-01-02 03:35:52 2020-01-01 00:07:31.000 2020-01-02 03:35:52.000 451 99352 49901.5 4990150 451 99352 49901.5 4990150 -32719 32419 4681.66 468166 -125 126 1.98 198 -452 101 10442 99353 1.35735 298.35735 149.85735 14985.73573 1.35735 298.35736 149.85736 14985.736 1.35735 298.35735 149.85735 14985.73500 2020-01-01 2020-01-02 2020-01-01 00:07:32 2020-01-02 03:35:53 2020-01-01 00:07:32.000 2020-01-02 03:35:53.000 452 99353 49902.5 4990250 452 99353 49902.5 4990250 -32718 32420 4682.66 468266 -124 127 2.98 298 -453 101 10443 99354 1.36036 298.36036 149.86036 14986.03603 1.36036 298.36035 149.86036 14986.03614 1.36036 298.36036 149.86036 14986.03600 2020-01-01 2020-01-02 2020-01-01 00:07:33 2020-01-02 03:35:54 2020-01-01 00:07:33.000 2020-01-02 03:35:54.000 453 99354 49903.5 4990350 453 99354 49903.5 4990350 -32717 32421 4683.66 468366 -128 127 1.42 142 -454 101 10444 99355 1.36336 298.36336 149.86336 14986.33633 1.36336 298.36337 149.86336 14986.33629 1.36336 298.36336 149.86336 14986.33600 2020-01-01 2020-01-02 2020-01-01 00:07:34 2020-01-02 03:35:55 2020-01-01 00:07:34.000 2020-01-02 03:35:55.000 454 99355 49904.5 4990450 454 99355 49904.5 4990450 -32716 32422 4684.66 468466 -128 127 -0.14 -14 +445 101 10435 99346 1.33633 298.33633 149.83633 14983.63363 1.33633 298.33633 149.83633 14983.63348 1.33633 298.33633 149.83633000000006 14983.63300 2020-01-01 2020-01-02 2020-01-01 00:07:25 2020-01-02 03:35:46 2020-01-01 00:07:25.000 2020-01-02 03:35:46.000 445 99346 49895.5 4989550 445 99346 49895.5 4989550 -32725 32413 4675.66 467566 -125 126 1.1 110 +446 101 10436 99347 1.33933 298.33933 149.83933 14983.93393 1.33933 298.33932 149.83933 14983.93378 1.33933 298.33933 149.83933000000002 14983.93300 2020-01-01 2020-01-02 2020-01-01 00:07:26 2020-01-02 03:35:47 2020-01-01 00:07:26.000 2020-01-02 03:35:47.000 446 99347 49896.5 4989650 446 99347 49896.5 4989650 -32724 32414 4676.66 467666 -124 127 2.1 210 +447 101 10437 99348 1.34234 298.34234 149.84234 14984.23423 1.34234 298.34235 149.84234 14984.23437 1.34234 298.34234 149.84233999999995 14984.23400 2020-01-01 2020-01-02 2020-01-01 00:07:27 2020-01-02 03:35:48 2020-01-01 00:07:27.000 2020-01-02 03:35:48.000 447 99348 49897.5 4989750 447 99348 49897.5 4989750 -32723 32415 4677.66 467766 -128 127 0.54 54 +448 101 10438 99349 1.34534 298.34534 149.84534 14984.53453 1.34534 298.34534 149.84534 14984.53466 1.34534 298.34534 149.84533999999994 14984.53400 2020-01-01 2020-01-02 2020-01-01 00:07:28 2020-01-02 03:35:49 2020-01-01 00:07:28.000 2020-01-02 03:35:49.000 448 99349 49898.5 4989850 448 99349 49898.5 4989850 -32722 32416 4678.66 467866 -128 123 -1.02 -102 +449 101 10439 99350 1.34834 298.34834 149.84834 14984.83483 1.34834 298.34836 149.84834 14984.83478 1.34834 298.34834 149.84834000000015 14984.83400 2020-01-01 2020-01-02 2020-01-01 00:07:29 2020-01-02 03:35:50 2020-01-01 00:07:29.000 2020-01-02 03:35:50.000 449 99350 49899.5 4989950 449 99350 49899.5 4989950 -32721 32417 4679.66 467966 -127 124 -0.02 -2 +45 102 10035 99945 0.13513 300.13513 150.13513 15163.64864 0.13513 300.13513 150.13513 15163.64839 0.13513 300.13513 150.13513000000003 15163.64813 2020-01-01 2020-01-02 2020-01-01 00:00:45 2020-01-02 03:45:45 2020-01-01 00:00:45.000 2020-01-02 03:45:45.000 45 99945 49995 5049495 45 99945 49995 5049495 -32524 32411 4574.009900990099 461975 -126 126 -1.0396039603960396 -105 +450 101 10440 99351 1.35135 298.35135 149.85135 14985.13513 1.35135 298.35135 149.85134 14985.13495 1.35135 298.35135 149.85135000000008 14985.13500 2020-01-01 2020-01-02 2020-01-01 00:07:30 2020-01-02 03:35:51 2020-01-01 00:07:30.000 2020-01-02 03:35:51.000 450 99351 49900.5 4990050 450 99351 49900.5 4990050 -32720 32418 4680.66 468066 -126 125 0.98 98 +451 101 10441 99352 1.35435 298.35435 149.85435 14985.43543 1.35435 298.35434 149.85435 14985.43525 1.35435 298.35435 149.8543499999999 14985.43500 2020-01-01 2020-01-02 2020-01-01 00:07:31 2020-01-02 03:35:52 2020-01-01 00:07:31.000 2020-01-02 03:35:52.000 451 99352 49901.5 4990150 451 99352 49901.5 4990150 -32719 32419 4681.66 468166 -125 126 1.98 198 +452 101 10442 99353 1.35735 298.35735 149.85735 14985.73573 1.35735 298.35736 149.85736 14985.736 1.35735 298.35735 149.8573500000001 14985.73500 2020-01-01 2020-01-02 2020-01-01 00:07:32 2020-01-02 03:35:53 2020-01-01 00:07:32.000 2020-01-02 03:35:53.000 452 99353 49902.5 4990250 452 99353 49902.5 4990250 -32718 32420 4682.66 468266 -124 127 2.98 298 +453 101 10443 99354 1.36036 298.36036 149.86036 14986.03603 1.36036 298.36035 149.86036 14986.03614 1.36036 298.36036 149.86036000000004 14986.03600 2020-01-01 2020-01-02 2020-01-01 00:07:33 2020-01-02 03:35:54 2020-01-01 00:07:33.000 2020-01-02 03:35:54.000 453 99354 49903.5 4990350 453 99354 49903.5 4990350 -32717 32421 4683.66 468366 -128 127 1.42 142 +454 101 10444 99355 1.36336 298.36336 149.86336 14986.33633 1.36336 298.36337 149.86336 14986.33629 1.36336 298.36336 149.86335999999994 14986.33600 2020-01-01 2020-01-02 2020-01-01 00:07:34 2020-01-02 03:35:55 2020-01-01 00:07:34.000 2020-01-02 03:35:55.000 454 99355 49904.5 4990450 454 99355 49904.5 4990450 -32716 32422 4684.66 468466 -128 127 -0.14 -14 455 101 10445 99356 1.36636 298.36636 149.86636 14986.63663 1.36636 298.36636 149.86636 14986.63641 1.36636 298.36636 149.86636 14986.63600 2020-01-01 2020-01-02 2020-01-01 00:07:35 2020-01-02 03:35:56 2020-01-01 00:07:35.000 2020-01-02 03:35:56.000 455 99356 49905.5 4990550 455 99356 49905.5 4990550 -32715 32423 4685.66 468566 -128 124 -1.7 -170 -456 101 10446 99357 1.36936 298.36936 149.86936 14986.93693 1.36936 298.36935 149.86936 14986.93672 1.36936 298.36936 149.86936 14986.93600 2020-01-01 2020-01-02 2020-01-01 00:07:36 2020-01-02 03:35:57 2020-01-01 00:07:36.000 2020-01-02 03:35:57.000 456 99357 49906.5 4990650 456 99357 49906.5 4990650 -32714 32424 4686.66 468666 -127 125 -0.7 -70 -457 101 10447 99358 1.37237 298.37237 149.87237 14987.23723 1.37237 298.37238 149.87237 14987.23747 1.37237 298.37237 149.87237 14987.23700 2020-01-01 2020-01-02 2020-01-01 00:07:37 2020-01-02 03:35:58 2020-01-01 00:07:37.000 2020-01-02 03:35:58.000 457 99358 49907.5 4990750 457 99358 49907.5 4990750 -32713 32425 4687.66 468766 -126 126 0.3 30 -458 101 10448 99359 1.37537 298.37537 149.87537 14987.53753 1.37537 298.37537 149.87537 14987.5376 1.37537 298.37537 149.87537 14987.53700 2020-01-01 2020-01-02 2020-01-01 00:07:38 2020-01-02 03:35:59 2020-01-01 00:07:38.000 2020-01-02 03:35:59.000 458 99359 49908.5 4990850 458 99359 49908.5 4990850 -32712 32426 4688.66 468866 -125 127 1.3 130 -459 101 10449 99360 1.37837 298.37837 149.87837 14987.83783 1.37837 298.3784 149.87837 14987.83775 1.37837 298.37837 149.87837 14987.83700 2020-01-01 2020-01-02 2020-01-01 00:07:39 2020-01-02 03:36:00 2020-01-01 00:07:39.000 2020-01-02 03:36:00.000 459 99360 49909.5 4990950 459 99360 49909.5 4990950 -32711 32427 4689.66 468966 -128 127 -0.26 -26 -46 102 10036 99946 0.13813 300.13813 150.13813 15163.95195 0.13813 300.13815 150.13814 15163.95214 0.13813 300.13813 150.13813 15163.95113 2020-01-01 2020-01-02 2020-01-01 00:00:46 2020-01-02 03:45:46 2020-01-01 00:00:46.000 2020-01-02 03:45:46.000 46 99946 49996 5049596 46 99946 49996 5049596 -32523 32412 4575.009900990099 462076 -125 127 -0.039603960396039604 -4 -460 101 10450 99361 1.38138 298.38138 149.88138 14988.13813 1.38138 298.38138 149.88137 14988.13789 1.38138 298.38138 149.88138 14988.13800 2020-01-01 2020-01-02 2020-01-01 00:07:40 2020-01-02 03:36:01 2020-01-01 00:07:40.000 2020-01-02 03:36:01.000 460 99361 49910.5 4991050 460 99361 49910.5 4991050 -32710 32428 4690.66 469066 -128 127 -1.82 -182 -461 101 10451 99362 1.38438 298.38438 149.88438 14988.43843 1.38438 298.3844 149.88438 14988.43864 1.38438 298.38438 149.88438 14988.43800 2020-01-01 2020-01-02 2020-01-01 00:07:41 2020-01-02 03:36:02 2020-01-01 00:07:41.000 2020-01-02 03:36:02.000 461 99362 49911.5 4991150 461 99362 49911.5 4991150 -32709 32429 4691.66 469166 -128 123 -3.38 -338 -462 101 10452 99363 1.38738 298.38738 149.88738 14988.73873 1.38738 298.3874 149.88738 14988.73894 1.38738 298.38738 149.88738 14988.73800 2020-01-01 2020-01-02 2020-01-01 00:07:42 2020-01-02 03:36:03 2020-01-01 00:07:42.000 2020-01-02 03:36:03.000 462 99363 49912.5 4991250 462 99363 49912.5 4991250 -32708 32430 4692.66 469266 -127 124 -2.38 -238 +456 101 10446 99357 1.36936 298.36936 149.86936 14986.93693 1.36936 298.36935 149.86936 14986.93672 1.36936 298.36936 149.86936000000003 14986.93600 2020-01-01 2020-01-02 2020-01-01 00:07:36 2020-01-02 03:35:57 2020-01-01 00:07:36.000 2020-01-02 03:35:57.000 456 99357 49906.5 4990650 456 99357 49906.5 4990650 -32714 32424 4686.66 468666 -127 125 -0.7 -70 +457 101 10447 99358 1.37237 298.37237 149.87237 14987.23723 1.37237 298.37238 149.87237 14987.23747 1.37237 298.37237 149.8723699999999 14987.23700 2020-01-01 2020-01-02 2020-01-01 00:07:37 2020-01-02 03:35:58 2020-01-01 00:07:37.000 2020-01-02 03:35:58.000 457 99358 49907.5 4990750 457 99358 49907.5 4990750 -32713 32425 4687.66 468766 -126 126 0.3 30 +458 101 10448 99359 1.37537 298.37537 149.87537 14987.53753 1.37537 298.37537 149.87537 14987.5376 1.37537 298.37537 149.87536999999992 14987.53700 2020-01-01 2020-01-02 2020-01-01 00:07:38 2020-01-02 03:35:59 2020-01-01 00:07:38.000 2020-01-02 03:35:59.000 458 99359 49908.5 4990850 458 99359 49908.5 4990850 -32712 32426 4688.66 468866 -125 127 1.3 130 +459 101 10449 99360 1.37837 298.37837 149.87837 14987.83783 1.37837 298.3784 149.87837 14987.83775 1.37837 298.37837 149.87837000000007 14987.83700 2020-01-01 2020-01-02 2020-01-01 00:07:39 2020-01-02 03:36:00 2020-01-01 00:07:39.000 2020-01-02 03:36:00.000 459 99360 49909.5 4990950 459 99360 49909.5 4990950 -32711 32427 4689.66 468966 -128 127 -0.26 -26 +46 102 10036 99946 0.13813 300.13813 150.13813 15163.95195 0.13813 300.13815 150.13814 15163.95214 0.13813 300.13813 150.13812999999993 15163.95113 2020-01-01 2020-01-02 2020-01-01 00:00:46 2020-01-02 03:45:46 2020-01-01 00:00:46.000 2020-01-02 03:45:46.000 46 99946 49996 5049596 46 99946 49996 5049596 -32523 32412 4575.009900990099 462076 -125 127 -0.039603960396039604 -4 +460 101 10450 99361 1.38138 298.38138 149.88138 14988.13813 1.38138 298.38138 149.88137 14988.13789 1.38138 298.38138 149.88138000000004 14988.13800 2020-01-01 2020-01-02 2020-01-01 00:07:40 2020-01-02 03:36:01 2020-01-01 00:07:40.000 2020-01-02 03:36:01.000 460 99361 49910.5 4991050 460 99361 49910.5 4991050 -32710 32428 4690.66 469066 -128 127 -1.82 -182 +461 101 10451 99362 1.38438 298.38438 149.88438 14988.43843 1.38438 298.3844 149.88438 14988.43864 1.38438 298.38438 149.88438000000002 14988.43800 2020-01-01 2020-01-02 2020-01-01 00:07:41 2020-01-02 03:36:02 2020-01-01 00:07:41.000 2020-01-02 03:36:02.000 461 99362 49911.5 4991150 461 99362 49911.5 4991150 -32709 32429 4691.66 469166 -128 123 -3.38 -338 +462 101 10452 99363 1.38738 298.38738 149.88738 14988.73873 1.38738 298.3874 149.88738 14988.73894 1.38738 298.38738 149.88738000000006 14988.73800 2020-01-01 2020-01-02 2020-01-01 00:07:42 2020-01-02 03:36:03 2020-01-01 00:07:42.000 2020-01-02 03:36:03.000 462 99363 49912.5 4991250 462 99363 49912.5 4991250 -32708 32430 4692.66 469266 -127 124 -2.38 -238 463 101 10453 99364 1.39039 298.39039 149.89039 14989.03903 1.39039 298.39038 149.89039 14989.03907 1.39039 298.39039 149.89039 14989.03900 2020-01-01 2020-01-02 2020-01-01 00:07:43 2020-01-02 03:36:04 2020-01-01 00:07:43.000 2020-01-02 03:36:04.000 463 99364 49913.5 4991350 463 99364 49913.5 4991350 -32707 32431 4693.66 469366 -126 125 -1.38 -138 -464 101 10454 99365 1.39339 298.39339 149.89339 14989.33933 1.39339 298.3934 149.89339 14989.33922 1.39339 298.39339 149.89339 14989.33900 2020-01-01 2020-01-02 2020-01-01 00:07:44 2020-01-02 03:36:05 2020-01-01 00:07:44.000 2020-01-02 03:36:05.000 464 99365 49914.5 4991450 464 99365 49914.5 4991450 -32706 32432 4694.66 469466 -125 126 -0.38 -38 -465 101 10455 99366 1.39639 298.39639 149.89639 14989.63963 1.39639 298.3964 149.89639 14989.63936 1.39639 298.39639 149.89639 14989.63900 2020-01-01 2020-01-02 2020-01-01 00:07:45 2020-01-02 03:36:06 2020-01-01 00:07:45.000 2020-01-02 03:36:06.000 465 99366 49915.5 4991550 465 99366 49915.5 4991550 -32705 32433 4695.66 469566 -124 127 0.62 62 -466 101 10456 99367 1.39939 298.39939 149.89939 14989.93993 1.39939 298.3994 149.8994 14989.94011 1.39939 298.39939 149.89939 14989.93900 2020-01-01 2020-01-02 2020-01-01 00:07:46 2020-01-02 03:36:07 2020-01-01 00:07:46.000 2020-01-02 03:36:07.000 466 99367 49916.5 4991650 466 99367 49916.5 4991650 -32704 32434 4696.66 469666 -128 127 -0.94 -94 -467 101 10457 99368 1.4024 298.4024 149.9024 14990.24024 1.4024 298.4024 149.9024 14990.24041 1.40240 298.40240 149.90240 14990.24000 2020-01-01 2020-01-02 2020-01-01 00:07:47 2020-01-02 03:36:08 2020-01-01 00:07:47.000 2020-01-02 03:36:08.000 467 99368 49917.5 4991750 467 99368 49917.5 4991750 -32703 32435 4697.66 469766 -128 123 -2.5 -250 -468 101 10458 99369 1.4054 298.4054 149.9054 14990.54054 1.4054 298.4054 149.9054 14990.54058 1.40540 298.40540 149.90540 14990.54000 2020-01-01 2020-01-02 2020-01-01 00:07:48 2020-01-02 03:36:09 2020-01-01 00:07:48.000 2020-01-02 03:36:09.000 468 99369 49918.5 4991850 468 99369 49918.5 4991850 -32702 32436 4698.66 469866 -127 124 -1.5 -150 -469 101 10459 99370 1.4084 298.4084 149.9084 14990.84084 1.4084 298.40842 149.9084 14990.8407 1.40840 298.40840 149.90840 14990.84000 2020-01-01 2020-01-02 2020-01-01 00:07:49 2020-01-02 03:36:10 2020-01-01 00:07:49.000 2020-01-02 03:36:10.000 469 99370 49919.5 4991950 469 99370 49919.5 4991950 -32701 32437 4699.66 469966 -126 125 -0.5 -50 -47 102 10037 99947 0.14114 300.14114 150.14114 15164.25525 0.14114 300.14114 150.14114 15164.25545 0.14114 300.14114 150.14114 15164.25514 2020-01-01 2020-01-02 2020-01-01 00:00:47 2020-01-02 03:45:47 2020-01-01 00:00:47.000 2020-01-02 03:45:47.000 47 99947 49997 5049697 47 99947 49997 5049697 -32522 32413 4576.009900990099 462177 -128 127 -1.5742574257425743 -159 -470 101 10460 99371 1.41141 298.41141 149.91141 14991.14114 1.41141 298.4114 149.9114 14991.14099 1.41141 298.41141 149.91141 14991.14100 2020-01-01 2020-01-02 2020-01-01 00:07:50 2020-01-02 03:36:11 2020-01-01 00:07:50.000 2020-01-02 03:36:11.000 470 99371 49920.5 4992050 470 99371 49920.5 4992050 -32700 32438 4700.66 470066 -125 126 0.5 50 -471 101 10461 99372 1.41441 298.41441 149.91441 14991.44144 1.41441 298.41443 149.91441 14991.44159 1.41441 298.41441 149.91441 14991.44100 2020-01-01 2020-01-02 2020-01-01 00:07:51 2020-01-02 03:36:12 2020-01-01 00:07:51.000 2020-01-02 03:36:12.000 471 99372 49921.5 4992150 471 99372 49921.5 4992150 -32699 32439 4701.66 470166 -124 127 1.5 150 +464 101 10454 99365 1.39339 298.39339 149.89339 14989.33933 1.39339 298.3934 149.89339 14989.33922 1.39339 298.39339 149.89338999999998 14989.33900 2020-01-01 2020-01-02 2020-01-01 00:07:44 2020-01-02 03:36:05 2020-01-01 00:07:44.000 2020-01-02 03:36:05.000 464 99365 49914.5 4991450 464 99365 49914.5 4991450 -32706 32432 4694.66 469466 -125 126 -0.38 -38 +465 101 10455 99366 1.39639 298.39639 149.89639 14989.63963 1.39639 298.3964 149.89639 14989.63936 1.39639 298.39639 149.8963899999999 14989.63900 2020-01-01 2020-01-02 2020-01-01 00:07:45 2020-01-02 03:36:06 2020-01-01 00:07:45.000 2020-01-02 03:36:06.000 465 99366 49915.5 4991550 465 99366 49915.5 4991550 -32705 32433 4695.66 469566 -124 127 0.62 62 +466 101 10456 99367 1.39939 298.39939 149.89939 14989.93993 1.39939 298.3994 149.8994 14989.94011 1.39939 298.39939 149.89938999999998 14989.93900 2020-01-01 2020-01-02 2020-01-01 00:07:46 2020-01-02 03:36:07 2020-01-01 00:07:46.000 2020-01-02 03:36:07.000 466 99367 49916.5 4991650 466 99367 49916.5 4991650 -32704 32434 4696.66 469666 -128 127 -0.94 -94 +467 101 10457 99368 1.4024 298.4024 149.9024 14990.24024 1.4024 298.4024 149.9024 14990.24041 1.40240 298.40240 149.90240000000009 14990.24000 2020-01-01 2020-01-02 2020-01-01 00:07:47 2020-01-02 03:36:08 2020-01-01 00:07:47.000 2020-01-02 03:36:08.000 467 99368 49917.5 4991750 467 99368 49917.5 4991750 -32703 32435 4697.66 469766 -128 123 -2.5 -250 +468 101 10458 99369 1.4054 298.4054 149.9054 14990.54054 1.4054 298.4054 149.9054 14990.54058 1.40540 298.40540 149.90539999999993 14990.54000 2020-01-01 2020-01-02 2020-01-01 00:07:48 2020-01-02 03:36:09 2020-01-01 00:07:48.000 2020-01-02 03:36:09.000 468 99369 49918.5 4991850 468 99369 49918.5 4991850 -32702 32436 4698.66 469866 -127 124 -1.5 -150 +469 101 10459 99370 1.4084 298.4084 149.9084 14990.84084 1.4084 298.40842 149.9084 14990.8407 1.40840 298.40840 149.90840000000003 14990.84000 2020-01-01 2020-01-02 2020-01-01 00:07:49 2020-01-02 03:36:10 2020-01-01 00:07:49.000 2020-01-02 03:36:10.000 469 99370 49919.5 4991950 469 99370 49919.5 4991950 -32701 32437 4699.66 469966 -126 125 -0.5 -50 +47 102 10037 99947 0.14114 300.14114 150.14114 15164.25525 0.14114 300.14114 150.14114 15164.25545 0.14114 300.14114 150.14113999999995 15164.25514 2020-01-01 2020-01-02 2020-01-01 00:00:47 2020-01-02 03:45:47 2020-01-01 00:00:47.000 2020-01-02 03:45:47.000 47 99947 49997 5049697 47 99947 49997 5049697 -32522 32413 4576.009900990099 462177 -128 127 -1.5742574257425743 -159 +470 101 10460 99371 1.41141 298.41141 149.91141 14991.14114 1.41141 298.4114 149.9114 14991.14099 1.41141 298.41141 149.91141000000005 14991.14100 2020-01-01 2020-01-02 2020-01-01 00:07:50 2020-01-02 03:36:11 2020-01-01 00:07:50.000 2020-01-02 03:36:11.000 470 99371 49920.5 4992050 470 99371 49920.5 4992050 -32700 32438 4700.66 470066 -125 126 0.5 50 +471 101 10461 99372 1.41441 298.41441 149.91441 14991.44144 1.41441 298.41443 149.91441 14991.44159 1.41441 298.41441 149.91440999999995 14991.44100 2020-01-01 2020-01-02 2020-01-01 00:07:51 2020-01-02 03:36:12 2020-01-01 00:07:51.000 2020-01-02 03:36:12.000 471 99372 49921.5 4992150 471 99372 49921.5 4992150 -32699 32439 4701.66 470166 -124 127 1.5 150 472 101 10462 99373 1.41741 298.41741 149.91741 14991.74174 1.41741 298.41742 149.91741 14991.74188 1.41741 298.41741 149.91741 14991.74100 2020-01-01 2020-01-02 2020-01-01 00:07:52 2020-01-02 03:36:13 2020-01-01 00:07:52.000 2020-01-02 03:36:13.000 472 99373 49922.5 4992250 472 99373 49922.5 4992250 -32698 32440 4702.66 470266 -128 127 -0.06 -6 -473 101 10463 99374 1.42042 298.42042 149.92042 14992.04204 1.42042 298.4204 149.92042 14992.04204 1.42042 298.42042 149.92042 14992.04200 2020-01-01 2020-01-02 2020-01-01 00:07:53 2020-01-02 03:36:14 2020-01-01 00:07:53.000 2020-01-02 03:36:14.000 473 99374 49923.5 4992350 473 99374 49923.5 4992350 -32697 32441 4703.66 470366 -128 123 -1.62 -162 -474 101 10464 99375 1.42342 298.42342 149.92342 14992.34234 1.42342 298.42343 149.92342 14992.34216 1.42342 298.42342 149.92342 14992.34200 2020-01-01 2020-01-02 2020-01-01 00:07:54 2020-01-02 03:36:15 2020-01-01 00:07:54.000 2020-01-02 03:36:15.000 474 99375 49924.5 4992450 474 99375 49924.5 4992450 -32696 32442 4704.66 470466 -127 124 -0.62 -62 -475 101 10465 99376 1.42642 298.42642 149.92642 14992.64264 1.42642 298.42642 149.92642 14992.64246 1.42642 298.42642 149.92642 14992.64200 2020-01-01 2020-01-02 2020-01-01 00:07:55 2020-01-02 03:36:16 2020-01-01 00:07:55.000 2020-01-02 03:36:16.000 475 99376 49925.5 4992550 475 99376 49925.5 4992550 -32695 32443 4705.66 470566 -126 125 0.38 38 -476 101 10466 99377 1.42942 298.42942 149.92942 14992.94294 1.42942 298.42944 149.92943 14992.94305 1.42942 298.42942 149.92942 14992.94200 2020-01-01 2020-01-02 2020-01-01 00:07:56 2020-01-02 03:36:17 2020-01-01 00:07:56.000 2020-01-02 03:36:17.000 476 99377 49926.5 4992650 476 99377 49926.5 4992650 -32694 32444 4706.66 470666 -125 126 1.38 138 -477 101 10467 99378 1.43243 298.43243 149.93243 14993.24324 1.43243 298.43243 149.93243 14993.24338 1.43243 298.43243 149.93243 14993.24300 2020-01-01 2020-01-02 2020-01-01 00:07:57 2020-01-02 03:36:18 2020-01-01 00:07:57.000 2020-01-02 03:36:18.000 477 99378 49927.5 4992750 477 99378 49927.5 4992750 -32693 32445 4707.66 470766 -124 127 2.38 238 -478 101 10468 99379 1.43543 298.43543 149.93543 14993.54354 1.43543 298.43542 149.93543 14993.54352 1.43543 298.43543 149.93543 14993.54300 2020-01-01 2020-01-02 2020-01-01 00:07:58 2020-01-02 03:36:19 2020-01-01 00:07:58.000 2020-01-02 03:36:19.000 478 99379 49928.5 4992850 478 99379 49928.5 4992850 -32692 32446 4708.66 470866 -128 127 0.82 82 -479 101 10469 99380 1.43843 298.43843 149.93843 14993.84384 1.43843 298.43845 149.93844 14993.84427 1.43843 298.43843 149.93843 14993.84300 2020-01-01 2020-01-02 2020-01-01 00:07:59 2020-01-02 03:36:20 2020-01-01 00:07:59.000 2020-01-02 03:36:20.000 479 99380 49929.5 4992950 479 99380 49929.5 4992950 -32691 32447 4709.66 470966 -128 127 -0.74 -74 -48 102 10038 99948 0.14414 300.14414 150.14414 15164.55855 0.14414 300.14413 150.14414 15164.55863 0.14414 300.14414 150.14414 15164.55814 2020-01-01 2020-01-02 2020-01-01 00:00:48 2020-01-02 03:45:48 2020-01-01 00:00:48.000 2020-01-02 03:45:48.000 48 99948 49998 5049798 48 99948 49998 5049798 -32521 32414 4577.009900990099 462278 -128 127 -3.108910891089109 -314 +473 101 10463 99374 1.42042 298.42042 149.92042 14992.04204 1.42042 298.4204 149.92042 14992.04204 1.42042 298.42042 149.9204200000001 14992.04200 2020-01-01 2020-01-02 2020-01-01 00:07:53 2020-01-02 03:36:14 2020-01-01 00:07:53.000 2020-01-02 03:36:14.000 473 99374 49923.5 4992350 473 99374 49923.5 4992350 -32697 32441 4703.66 470366 -128 123 -1.62 -162 +474 101 10464 99375 1.42342 298.42342 149.92342 14992.34234 1.42342 298.42343 149.92342 14992.34216 1.42342 298.42342 149.9234199999999 14992.34200 2020-01-01 2020-01-02 2020-01-01 00:07:54 2020-01-02 03:36:15 2020-01-01 00:07:54.000 2020-01-02 03:36:15.000 474 99375 49924.5 4992450 474 99375 49924.5 4992450 -32696 32442 4704.66 470466 -127 124 -0.62 -62 +475 101 10465 99376 1.42642 298.42642 149.92642 14992.64264 1.42642 298.42642 149.92642 14992.64246 1.42642 298.42642 149.9264199999999 14992.64200 2020-01-01 2020-01-02 2020-01-01 00:07:55 2020-01-02 03:36:16 2020-01-01 00:07:55.000 2020-01-02 03:36:16.000 475 99376 49925.5 4992550 475 99376 49925.5 4992550 -32695 32443 4705.66 470566 -126 125 0.38 38 +476 101 10466 99377 1.42942 298.42942 149.92942 14992.94294 1.42942 298.42944 149.92943 14992.94305 1.42942 298.42942 149.9294200000001 14992.94200 2020-01-01 2020-01-02 2020-01-01 00:07:56 2020-01-02 03:36:17 2020-01-01 00:07:56.000 2020-01-02 03:36:17.000 476 99377 49926.5 4992650 476 99377 49926.5 4992650 -32694 32444 4706.66 470666 -125 126 1.38 138 +477 101 10467 99378 1.43243 298.43243 149.93243 14993.24324 1.43243 298.43243 149.93243 14993.24338 1.43243 298.43243 149.93243000000004 14993.24300 2020-01-01 2020-01-02 2020-01-01 00:07:57 2020-01-02 03:36:18 2020-01-01 00:07:57.000 2020-01-02 03:36:18.000 477 99378 49927.5 4992750 477 99378 49927.5 4992750 -32693 32445 4707.66 470766 -124 127 2.38 238 +478 101 10468 99379 1.43543 298.43543 149.93543 14993.54354 1.43543 298.43542 149.93543 14993.54352 1.43543 298.43543 149.93542999999983 14993.54300 2020-01-01 2020-01-02 2020-01-01 00:07:58 2020-01-02 03:36:19 2020-01-01 00:07:58.000 2020-01-02 03:36:19.000 478 99379 49928.5 4992850 478 99379 49928.5 4992850 -32692 32446 4708.66 470866 -128 127 0.82 82 +479 101 10469 99380 1.43843 298.43843 149.93843 14993.84384 1.43843 298.43845 149.93844 14993.84427 1.43843 298.43843 149.93843000000007 14993.84300 2020-01-01 2020-01-02 2020-01-01 00:07:59 2020-01-02 03:36:20 2020-01-01 00:07:59.000 2020-01-02 03:36:20.000 479 99380 49929.5 4992950 479 99380 49929.5 4992950 -32691 32447 4709.66 470966 -128 127 -0.74 -74 +48 102 10038 99948 0.14414 300.14414 150.14414 15164.55855 0.14414 300.14413 150.14414 15164.55863 0.14414 300.14414 150.14414000000005 15164.55814 2020-01-01 2020-01-02 2020-01-01 00:00:48 2020-01-02 03:45:48 2020-01-01 00:00:48.000 2020-01-02 03:45:48.000 48 99948 49998 5049798 48 99948 49998 5049798 -32521 32414 4577.009900990099 462278 -128 127 -3.108910891089109 -314 480 101 10470 99381 1.44144 298.44144 149.94144 14994.14414 1.44144 298.44144 149.94143 14994.14392 1.44144 298.44144 149.94144 14994.14400 2020-01-01 2020-01-02 2020-01-01 00:08:00 2020-01-02 03:36:21 2020-01-01 00:08:00.000 2020-01-02 03:36:21.000 480 99381 49930.5 4993050 480 99381 49930.5 4993050 -32690 32448 4710.66 471066 -128 124 -2.3 -230 481 101 10471 99382 1.44444 298.44444 149.94444 14994.44444 1.44444 298.44446 149.94444 14994.44452 1.44444 298.44444 149.94444 14994.44400 2020-01-01 2020-01-02 2020-01-01 00:08:01 2020-01-02 03:36:22 2020-01-01 00:08:01.000 2020-01-02 03:36:22.000 481 99382 49931.5 4993150 481 99382 49931.5 4993150 -32689 32449 4711.66 471166 -127 125 -1.3 -130 -482 101 10472 99383 1.44744 298.44744 149.94744 14994.74474 1.44744 298.44745 149.94744 14994.74485 1.44744 298.44744 149.94744 14994.74400 2020-01-01 2020-01-02 2020-01-01 00:08:02 2020-01-02 03:36:23 2020-01-01 00:08:02.000 2020-01-02 03:36:23.000 482 99383 49932.5 4993250 482 99383 49932.5 4993250 -32688 32450 4712.66 471266 -126 126 -0.3 -30 -483 101 10473 99384 1.45045 298.45045 149.95045 14995.04504 1.45045 298.45044 149.95044 14995.04499 1.45045 298.45045 149.95045 14995.04500 2020-01-01 2020-01-02 2020-01-01 00:08:03 2020-01-02 03:36:24 2020-01-01 00:08:03.000 2020-01-02 03:36:24.000 483 99384 49933.5 4993350 483 99384 49933.5 4993350 -32687 32451 4713.66 471366 -125 127 0.7 70 -484 101 10474 99385 1.45345 298.45345 149.95345 14995.34534 1.45345 298.45346 149.95345 14995.34574 1.45345 298.45345 149.95345 14995.34500 2020-01-01 2020-01-02 2020-01-01 00:08:04 2020-01-02 03:36:25 2020-01-01 00:08:04.000 2020-01-02 03:36:25.000 484 99385 49934.5 4993450 484 99385 49934.5 4993450 -32686 32452 4714.66 471466 -128 127 -0.86 -86 -485 101 10475 99386 1.45645 298.45645 149.95645 14995.64564 1.45645 298.45645 149.95645 14995.6454 1.45645 298.45645 149.95645 14995.64500 2020-01-01 2020-01-02 2020-01-01 00:08:05 2020-01-02 03:36:26 2020-01-01 00:08:05.000 2020-01-02 03:36:26.000 485 99386 49935.5 4993550 485 99386 49935.5 4993550 -32685 32453 4715.66 471566 -128 127 -2.42 -242 -486 101 10476 99387 1.45945 298.45945 149.95945 14995.94594 1.45945 298.45947 149.95946 14995.94602 1.45945 298.45945 149.95945 14995.94500 2020-01-01 2020-01-02 2020-01-01 00:08:06 2020-01-02 03:36:27 2020-01-01 00:08:06.000 2020-01-02 03:36:27.000 486 99387 49936.5 4993650 486 99387 49936.5 4993650 -32684 32454 4716.66 471666 -128 123 -3.98 -398 -487 101 10477 99388 1.46246 298.46246 149.96246 14996.24624 1.46246 298.46246 149.96246 14996.24633 1.46246 298.46246 149.96246 14996.24600 2020-01-01 2020-01-02 2020-01-01 00:08:07 2020-01-02 03:36:28 2020-01-01 00:08:07.000 2020-01-02 03:36:28.000 487 99388 49937.5 4993750 487 99388 49937.5 4993750 -32683 32455 4717.66 471766 -127 124 -2.98 -298 -488 101 10478 99389 1.46546 298.46546 149.96546 14996.54654 1.46546 298.46545 149.96546 14996.54645 1.46546 298.46546 149.96546 14996.54600 2020-01-01 2020-01-02 2020-01-01 00:08:08 2020-01-02 03:36:29 2020-01-01 00:08:08.000 2020-01-02 03:36:29.000 488 99389 49938.5 4993850 488 99389 49938.5 4993850 -32682 32456 4718.66 471866 -126 125 -1.98 -198 -489 101 10479 99390 1.46846 298.46846 149.96846 14996.84684 1.46846 298.46848 149.96847 14996.84721 1.46846 298.46846 149.96846 14996.84600 2020-01-01 2020-01-02 2020-01-01 00:08:09 2020-01-02 03:36:30 2020-01-01 00:08:09.000 2020-01-02 03:36:30.000 489 99390 49939.5 4993950 489 99390 49939.5 4993950 -32681 32457 4719.66 471966 -125 126 -0.98 -98 -49 102 10039 99949 0.14714 300.14714 150.14714 15164.86186 0.14714 300.14716 150.14714 15164.86173 0.14714 300.14714 150.14714 15164.86114 2020-01-01 2020-01-02 2020-01-01 00:00:49 2020-01-02 03:45:49 2020-01-01 00:00:49.000 2020-01-02 03:45:49.000 49 99949 49999 5049899 49 99949 49999 5049899 -32520 32415 4578.009900990099 462379 -128 123 -4.643564356435643 -469 -490 101 10480 99391 1.47147 298.47147 149.97147 14997.14714 1.47147 298.47147 149.97146 14997.14687 1.47147 298.47147 149.97147 14997.14700 2020-01-01 2020-01-02 2020-01-01 00:08:10 2020-01-02 03:36:31 2020-01-01 00:08:10.000 2020-01-02 03:36:31.000 490 99391 49940.5 4994050 490 99391 49940.5 4994050 -32680 32458 4720.66 472066 -124 127 0.02 2 -491 101 10481 99392 1.47447 298.47447 149.97447 14997.44744 1.47447 298.4745 149.97447 14997.44749 1.47447 298.47447 149.97447 14997.44700 2020-01-01 2020-01-02 2020-01-01 00:08:11 2020-01-02 03:36:32 2020-01-01 00:08:11.000 2020-01-02 03:36:32.000 491 99392 49941.5 4994150 491 99392 49941.5 4994150 -32679 32459 4721.66 472166 -128 127 -1.54 -154 -492 101 10482 99393 1.47747 298.47747 149.97747 14997.74774 1.47747 298.47748 149.97747 14997.74779 1.47747 298.47747 149.97747 14997.74700 2020-01-01 2020-01-02 2020-01-01 00:08:12 2020-01-02 03:36:33 2020-01-01 00:08:12.000 2020-01-02 03:36:33.000 492 99393 49942.5 4994250 492 99393 49942.5 4994250 -32678 32460 4722.66 472266 -128 123 -3.1 -310 -493 101 10483 99394 1.48048 298.48048 149.98048 14998.04804 1.48048 298.48047 149.98048 14998.04809 1.48048 298.48048 149.98048 14998.04800 2020-01-01 2020-01-02 2020-01-01 00:08:13 2020-01-02 03:36:34 2020-01-01 00:08:13.000 2020-01-02 03:36:34.000 493 99394 49943.5 4994350 493 99394 49943.5 4994350 -32677 32461 4723.66 472366 -127 124 -2.1 -210 -494 101 10484 99395 1.48348 298.48348 149.98348 14998.34834 1.48348 298.4835 149.98348 14998.34868 1.48348 298.48348 149.98348 14998.34800 2020-01-01 2020-01-02 2020-01-01 00:08:14 2020-01-02 03:36:35 2020-01-01 00:08:14.000 2020-01-02 03:36:35.000 494 99395 49944.5 4994450 494 99395 49944.5 4994450 -32676 32462 4724.66 472466 -126 125 -1.1 -110 -495 100 10485 99396 1.48648 298.48648 149.98648 14998.64864 1.48648 298.48648 149.98648 14998.64837 1.48648 298.48648 149.98648 14998.64800 2020-01-01 2020-01-02 2020-01-01 00:08:15 2020-01-02 03:36:36 2020-01-01 00:08:15.000 2020-01-02 03:36:36.000 495 99396 49945.5 4994550 495 99396 49945.5 4994550 -32675 32463 4725.66 472566 -125 126 -0.1 -10 -496 100 10486 99397 1.48948 298.48948 149.98948 14998.94894 1.48948 298.4895 149.98948 14998.94896 1.48948 298.48948 149.98948 14998.94800 2020-01-01 2020-01-02 2020-01-01 00:08:16 2020-01-02 03:36:37 2020-01-01 00:08:16.000 2020-01-02 03:36:37.000 496 99397 49946.5 4994650 496 99397 49946.5 4994650 -32674 32464 4726.66 472666 -124 127 0.9 90 +482 101 10472 99383 1.44744 298.44744 149.94744 14994.74474 1.44744 298.44745 149.94744 14994.74485 1.44744 298.44744 149.94743999999994 14994.74400 2020-01-01 2020-01-02 2020-01-01 00:08:02 2020-01-02 03:36:23 2020-01-01 00:08:02.000 2020-01-02 03:36:23.000 482 99383 49932.5 4993250 482 99383 49932.5 4993250 -32688 32450 4712.66 471266 -126 126 -0.3 -30 +483 101 10473 99384 1.45045 298.45045 149.95045 14995.04504 1.45045 298.45044 149.95044 14995.04499 1.45045 298.45045 149.95045000000005 14995.04500 2020-01-01 2020-01-02 2020-01-01 00:08:03 2020-01-02 03:36:24 2020-01-01 00:08:03.000 2020-01-02 03:36:24.000 483 99384 49933.5 4993350 483 99384 49933.5 4993350 -32687 32451 4713.66 471366 -125 127 0.7 70 +484 101 10474 99385 1.45345 298.45345 149.95345 14995.34534 1.45345 298.45346 149.95345 14995.34574 1.45345 298.45345 149.95345000000012 14995.34500 2020-01-01 2020-01-02 2020-01-01 00:08:04 2020-01-02 03:36:25 2020-01-01 00:08:04.000 2020-01-02 03:36:25.000 484 99385 49934.5 4993450 484 99385 49934.5 4993450 -32686 32452 4714.66 471466 -128 127 -0.86 -86 +485 101 10475 99386 1.45645 298.45645 149.95645 14995.64564 1.45645 298.45645 149.95645 14995.6454 1.45645 298.45645 149.9564499999999 14995.64500 2020-01-01 2020-01-02 2020-01-01 00:08:05 2020-01-02 03:36:26 2020-01-01 00:08:05.000 2020-01-02 03:36:26.000 485 99386 49935.5 4993550 485 99386 49935.5 4993550 -32685 32453 4715.66 471566 -128 127 -2.42 -242 +486 101 10476 99387 1.45945 298.45945 149.95945 14995.94594 1.45945 298.45947 149.95946 14995.94602 1.45945 298.45945 149.95945000000006 14995.94500 2020-01-01 2020-01-02 2020-01-01 00:08:06 2020-01-02 03:36:27 2020-01-01 00:08:06.000 2020-01-02 03:36:27.000 486 99387 49936.5 4993650 486 99387 49936.5 4993650 -32684 32454 4716.66 471666 -128 123 -3.98 -398 +487 101 10477 99388 1.46246 298.46246 149.96246 14996.24624 1.46246 298.46246 149.96246 14996.24633 1.46246 298.46246 149.96246000000008 14996.24600 2020-01-01 2020-01-02 2020-01-01 00:08:07 2020-01-02 03:36:28 2020-01-01 00:08:07.000 2020-01-02 03:36:28.000 487 99388 49937.5 4993750 487 99388 49937.5 4993750 -32683 32455 4717.66 471766 -127 124 -2.98 -298 +488 101 10478 99389 1.46546 298.46546 149.96546 14996.54654 1.46546 298.46545 149.96546 14996.54645 1.46546 298.46546 149.96545999999998 14996.54600 2020-01-01 2020-01-02 2020-01-01 00:08:08 2020-01-02 03:36:29 2020-01-01 00:08:08.000 2020-01-02 03:36:29.000 488 99389 49938.5 4993850 488 99389 49938.5 4993850 -32682 32456 4718.66 471866 -126 125 -1.98 -198 +489 101 10479 99390 1.46846 298.46846 149.96846 14996.84684 1.46846 298.46848 149.96847 14996.84721 1.46846 298.46846 149.96846000000002 14996.84600 2020-01-01 2020-01-02 2020-01-01 00:08:09 2020-01-02 03:36:30 2020-01-01 00:08:09.000 2020-01-02 03:36:30.000 489 99390 49939.5 4993950 489 99390 49939.5 4993950 -32681 32457 4719.66 471966 -125 126 -0.98 -98 +49 102 10039 99949 0.14714 300.14714 150.14714 15164.86186 0.14714 300.14716 150.14714 15164.86173 0.14714 300.14714 150.14713999999992 15164.86114 2020-01-01 2020-01-02 2020-01-01 00:00:49 2020-01-02 03:45:49 2020-01-01 00:00:49.000 2020-01-02 03:45:49.000 49 99949 49999 5049899 49 99949 49999 5049899 -32520 32415 4578.009900990099 462379 -128 123 -4.643564356435643 -469 +490 101 10480 99391 1.47147 298.47147 149.97147 14997.14714 1.47147 298.47147 149.97146 14997.14687 1.47147 298.47147 149.97147000000012 14997.14700 2020-01-01 2020-01-02 2020-01-01 00:08:10 2020-01-02 03:36:31 2020-01-01 00:08:10.000 2020-01-02 03:36:31.000 490 99391 49940.5 4994050 490 99391 49940.5 4994050 -32680 32458 4720.66 472066 -124 127 0.02 2 +491 101 10481 99392 1.47447 298.47447 149.97447 14997.44744 1.47447 298.4745 149.97447 14997.44749 1.47447 298.47447 149.97446999999997 14997.44700 2020-01-01 2020-01-02 2020-01-01 00:08:11 2020-01-02 03:36:32 2020-01-01 00:08:11.000 2020-01-02 03:36:32.000 491 99392 49941.5 4994150 491 99392 49941.5 4994150 -32679 32459 4721.66 472166 -128 127 -1.54 -154 +492 101 10482 99393 1.47747 298.47747 149.97747 14997.74774 1.47747 298.47748 149.97747 14997.74779 1.47747 298.47747 149.9774699999999 14997.74700 2020-01-01 2020-01-02 2020-01-01 00:08:12 2020-01-02 03:36:33 2020-01-01 00:08:12.000 2020-01-02 03:36:33.000 492 99393 49942.5 4994250 492 99393 49942.5 4994250 -32678 32460 4722.66 472266 -128 123 -3.1 -310 +493 101 10483 99394 1.48048 298.48048 149.98048 14998.04804 1.48048 298.48047 149.98048 14998.04809 1.48048 298.48048 149.98048000000009 14998.04800 2020-01-01 2020-01-02 2020-01-01 00:08:13 2020-01-02 03:36:34 2020-01-01 00:08:13.000 2020-01-02 03:36:34.000 493 99394 49943.5 4994350 493 99394 49943.5 4994350 -32677 32461 4723.66 472366 -127 124 -2.1 -210 +494 101 10484 99395 1.48348 298.48348 149.98348 14998.34834 1.48348 298.4835 149.98348 14998.34868 1.48348 298.48348 149.98348000000004 14998.34800 2020-01-01 2020-01-02 2020-01-01 00:08:14 2020-01-02 03:36:35 2020-01-01 00:08:14.000 2020-01-02 03:36:35.000 494 99395 49944.5 4994450 494 99395 49944.5 4994450 -32676 32462 4724.66 472466 -126 125 -1.1 -110 +495 100 10485 99396 1.48648 298.48648 149.98648 14998.64864 1.48648 298.48648 149.98648 14998.64837 1.48648 298.48648 149.98647999999986 14998.64800 2020-01-01 2020-01-02 2020-01-01 00:08:15 2020-01-02 03:36:36 2020-01-01 00:08:15.000 2020-01-02 03:36:36.000 495 99396 49945.5 4994550 495 99396 49945.5 4994550 -32675 32463 4725.66 472566 -125 126 -0.1 -10 +496 100 10486 99397 1.48948 298.48948 149.98948 14998.94894 1.48948 298.4895 149.98948 14998.94896 1.48948 298.48948 149.9894800000001 14998.94800 2020-01-01 2020-01-02 2020-01-01 00:08:16 2020-01-02 03:36:37 2020-01-01 00:08:16.000 2020-01-02 03:36:37.000 496 99397 49946.5 4994650 496 99397 49946.5 4994650 -32674 32464 4726.66 472666 -124 127 0.9 90 497 100 10487 99398 1.49249 298.49249 149.99249 14999.24924 1.49249 298.4925 149.99249 14999.24926 1.49249 298.49249 149.99249 14999.24900 2020-01-01 2020-01-02 2020-01-01 00:08:17 2020-01-02 03:36:38 2020-01-01 00:08:17.000 2020-01-02 03:36:38.000 497 99398 49947.5 4994750 497 99398 49947.5 4994750 -32673 32465 4727.66 472766 -128 127 -0.66 -66 -498 100 10488 99399 1.49549 298.49549 149.99549 14999.54954 1.49549 298.49548 149.99549 14999.54956 1.49549 298.49549 149.99549 14999.54900 2020-01-01 2020-01-02 2020-01-01 00:08:18 2020-01-02 03:36:39 2020-01-01 00:08:18.000 2020-01-02 03:36:39.000 498 99399 49948.5 4994850 498 99399 49948.5 4994850 -32672 32466 4728.66 472866 -128 123 -2.22 -222 -499 100 10489 99400 1.49849 298.49849 149.99849 14999.84984 1.49849 298.4985 149.9985 14999.85015 1.49849 298.49849 149.99849 14999.84900 2020-01-01 2020-01-02 2020-01-01 00:08:19 2020-01-02 03:36:40 2020-01-01 00:08:19.000 2020-01-02 03:36:40.000 499 99400 49949.5 4994950 499 99400 49949.5 4994950 -32671 32467 4729.66 472966 -127 124 -1.22 -122 -5 102 1004 9995 0.01501 300.01501 150.01501 15151.51651 0.01501 300.015 150.01501 15151.51648 0.01501 300.01501 150.01501 15151.51601 2020-01-01 2020-01-02 2020-01-01 00:00:05 2020-01-02 03:45:05 2020-01-01 00:00:05.000 2020-01-02 03:45:05.000 5 99905 49955 5045455 5 99905 49955 5045455 -32564 32371 4534.009900990099 457935 -128 123 -3.01980198019802 -305 +498 100 10488 99399 1.49549 298.49549 149.99549 14999.54954 1.49549 298.49548 149.99549 14999.54956 1.49549 298.49549 149.99548999999996 14999.54900 2020-01-01 2020-01-02 2020-01-01 00:08:18 2020-01-02 03:36:39 2020-01-01 00:08:18.000 2020-01-02 03:36:39.000 498 99399 49948.5 4994850 498 99399 49948.5 4994850 -32672 32466 4728.66 472866 -128 123 -2.22 -222 +499 100 10489 99400 1.49849 298.49849 149.99849 14999.84984 1.49849 298.4985 149.9985 14999.85015 1.49849 298.49849 149.99848999999998 14999.84900 2020-01-01 2020-01-02 2020-01-01 00:08:19 2020-01-02 03:36:40 2020-01-01 00:08:19.000 2020-01-02 03:36:40.000 499 99400 49949.5 4994950 499 99400 49949.5 4994950 -32671 32467 4729.66 472966 -127 124 -1.22 -122 +5 102 1004 9995 0.01501 300.01501 150.01501 15151.51651 0.01501 300.015 150.01501 15151.51648 0.01501 300.01501 150.01500999999993 15151.51601 2020-01-01 2020-01-02 2020-01-01 00:00:05 2020-01-02 03:45:05 2020-01-01 00:00:05.000 2020-01-02 03:45:05.000 5 99905 49955 5045455 5 99905 49955 5045455 -32564 32371 4534.009900990099 457935 -128 123 -3.01980198019802 -305 50 102 10040 99950 0.15015 300.15015 150.15015 15165.16516 0.15015 300.15015 150.15014 15165.16487 0.15015 300.15015 150.15015 15165.16515 2020-01-01 2020-01-02 2020-01-01 00:00:50 2020-01-02 03:45:50 2020-01-01 00:00:50.000 2020-01-02 03:45:50.000 50 99950 50000 5050000 50 99950 50000 5050000 -32519 32416 4579.009900990099 462480 -127 124 -3.6435643564356437 -368 -500 100 10490 99401 1.5015 298.5015 150.0015 15000.15015 1.5015 298.5015 150.00149 15000.14984 1.50150 298.50150 150.00150 15000.15000 2020-01-01 2020-01-02 2020-01-01 00:08:20 2020-01-02 03:36:41 2020-01-01 00:08:20.000 2020-01-02 03:36:41.000 500 99401 49950.5 4995050 500 99401 49950.5 4995050 -32670 32468 4730.66 473066 -126 125 -0.22 -22 -501 100 10491 99402 1.5045 298.5045 150.0045 15000.45045 1.5045 298.50452 150.0045 15000.45043 1.50450 298.50450 150.00450 15000.45000 2020-01-01 2020-01-02 2020-01-01 00:08:21 2020-01-02 03:36:42 2020-01-01 00:08:21.000 2020-01-02 03:36:42.000 501 99402 49951.5 4995150 501 99402 49951.5 4995150 -32669 32469 4731.66 473166 -125 126 0.78 78 -502 100 10492 99403 1.5075 298.5075 150.0075 15000.75075 1.5075 298.5075 150.0075 15000.75073 1.50750 298.50750 150.00750 15000.75000 2020-01-01 2020-01-02 2020-01-01 00:08:22 2020-01-02 03:36:43 2020-01-01 00:08:22.000 2020-01-02 03:36:43.000 502 99403 49952.5 4995250 502 99403 49952.5 4995250 -32668 32470 4732.66 473266 -124 127 1.78 178 -503 100 10493 99404 1.51051 298.51051 150.01051 15001.05105 1.51051 298.5105 150.01051 15001.05103 1.51051 298.51051 150.01051 15001.05100 2020-01-01 2020-01-02 2020-01-01 00:08:23 2020-01-02 03:36:44 2020-01-01 00:08:23.000 2020-01-02 03:36:44.000 503 99404 49953.5 4995350 503 99404 49953.5 4995350 -32667 32471 4733.66 473366 -128 127 0.22 22 +500 100 10490 99401 1.5015 298.5015 150.0015 15000.15015 1.5015 298.5015 150.00149 15000.14984 1.50150 298.50150 150.00150000000008 15000.15000 2020-01-01 2020-01-02 2020-01-01 00:08:20 2020-01-02 03:36:41 2020-01-01 00:08:20.000 2020-01-02 03:36:41.000 500 99401 49950.5 4995050 500 99401 49950.5 4995050 -32670 32468 4730.66 473066 -126 125 -0.22 -22 +501 100 10491 99402 1.5045 298.5045 150.0045 15000.45045 1.5045 298.50452 150.0045 15000.45043 1.50450 298.50450 150.0044999999999 15000.45000 2020-01-01 2020-01-02 2020-01-01 00:08:21 2020-01-02 03:36:42 2020-01-01 00:08:21.000 2020-01-02 03:36:42.000 501 99402 49951.5 4995150 501 99402 49951.5 4995150 -32669 32469 4731.66 473166 -125 126 0.78 78 +502 100 10492 99403 1.5075 298.5075 150.0075 15000.75075 1.5075 298.5075 150.0075 15000.75073 1.50750 298.50750 150.00749999999994 15000.75000 2020-01-01 2020-01-02 2020-01-01 00:08:22 2020-01-02 03:36:43 2020-01-01 00:08:22.000 2020-01-02 03:36:43.000 502 99403 49952.5 4995250 502 99403 49952.5 4995250 -32668 32470 4732.66 473266 -124 127 1.78 178 +503 100 10493 99404 1.51051 298.51051 150.01051 15001.05105 1.51051 298.5105 150.01051 15001.05103 1.51051 298.51051 150.01051000000004 15001.05100 2020-01-01 2020-01-02 2020-01-01 00:08:23 2020-01-02 03:36:44 2020-01-01 00:08:23.000 2020-01-02 03:36:44.000 503 99404 49953.5 4995350 503 99404 49953.5 4995350 -32667 32471 4733.66 473366 -128 127 0.22 22 504 100 10494 99405 1.51351 298.51351 150.01351 15001.35135 1.51351 298.51352 150.01351 15001.35162 1.51351 298.51351 150.01351 15001.35100 2020-01-01 2020-01-02 2020-01-01 00:08:24 2020-01-02 03:36:45 2020-01-01 00:08:24.000 2020-01-02 03:36:45.000 504 99405 49954.5 4995450 504 99405 49954.5 4995450 -32666 32472 4734.66 473466 -128 127 -1.34 -134 505 100 10495 99406 1.51651 298.51651 150.01651 15001.65165 1.51651 298.5165 150.01651 15001.65131 1.51651 298.51651 150.01651 15001.65100 2020-01-01 2020-01-02 2020-01-01 00:08:25 2020-01-02 03:36:46 2020-01-01 00:08:25.000 2020-01-02 03:36:46.000 505 99406 49955.5 4995550 505 99406 49955.5 4995550 -32665 32473 4735.66 473566 -128 124 -2.9 -290 -506 100 10496 99407 1.51951 298.51951 150.01951 15001.95195 1.51951 298.51953 150.01951 15001.9519 1.51951 298.51951 150.01951 15001.95100 2020-01-01 2020-01-02 2020-01-01 00:08:26 2020-01-02 03:36:47 2020-01-01 00:08:26.000 2020-01-02 03:36:47.000 506 99407 49956.5 4995650 506 99407 49956.5 4995650 -32664 32474 4736.66 473666 -127 125 -1.9 -190 -507 100 10497 99408 1.52252 298.52252 150.02252 15002.25225 1.52252 298.52252 150.02252 15002.2522 1.52252 298.52252 150.02252 15002.25200 2020-01-01 2020-01-02 2020-01-01 00:08:27 2020-01-02 03:36:48 2020-01-01 00:08:27.000 2020-01-02 03:36:48.000 507 99408 49957.5 4995750 507 99408 49957.5 4995750 -32663 32475 4737.66 473766 -126 126 -0.9 -90 -508 100 10498 99409 1.52552 298.52552 150.02552 15002.55255 1.52552 298.5255 150.02552 15002.5525 1.52552 298.52552 150.02552 15002.55200 2020-01-01 2020-01-02 2020-01-01 00:08:28 2020-01-02 03:36:49 2020-01-01 00:08:28.000 2020-01-02 03:36:49.000 508 99409 49958.5 4995850 508 99409 49958.5 4995850 -32662 32476 4738.66 473866 -125 127 0.1 10 -509 100 10499 99410 1.52852 298.52852 150.02852 15002.85285 1.52852 298.52853 150.02853 15002.85312 1.52852 298.52852 150.02852 15002.85200 2020-01-01 2020-01-02 2020-01-01 00:08:29 2020-01-02 03:36:50 2020-01-01 00:08:29.000 2020-01-02 03:36:50.000 509 99410 49959.5 4995950 509 99410 49959.5 4995950 -32661 32477 4739.66 473966 -128 127 -1.46 -146 -51 102 10041 99951 0.15315 300.15315 150.15315 15165.46846 0.15315 300.15317 150.15315 15165.46863 0.15315 300.15315 150.15315 15165.46815 2020-01-01 2020-01-02 2020-01-01 00:00:51 2020-01-02 03:45:51 2020-01-01 00:00:51.000 2020-01-02 03:45:51.000 51 99951 50001 5050101 51 99951 50001 5050101 -32518 32417 4580.009900990099 462581 -126 125 -2.6435643564356437 -267 -510 100 10500 99411 1.53153 298.53153 150.03153 15003.15315 1.53153 298.53152 150.03152 15003.15278 1.53153 298.53153 150.03153 15003.15300 2020-01-01 2020-01-02 2020-01-01 00:08:30 2020-01-02 03:36:51 2020-01-01 00:08:30.000 2020-01-02 03:36:51.000 510 99411 49960.5 4996050 510 99411 49960.5 4996050 -32660 32478 4740.66 474066 -128 127 -3.02 -302 -511 100 10501 99412 1.53453 298.53453 150.03453 15003.45345 1.53453 298.53455 150.03453 15003.45354 1.53453 298.53453 150.03453 15003.45300 2020-01-01 2020-01-02 2020-01-01 00:08:31 2020-01-02 03:36:52 2020-01-01 00:08:31.000 2020-01-02 03:36:52.000 511 99412 49961.5 4996150 511 99412 49961.5 4996150 -32659 32479 4741.66 474166 -128 123 -4.58 -458 -512 100 10502 99413 1.53753 298.53753 150.03753 15003.75375 1.53753 298.53754 150.03753 15003.75366 1.53753 298.53753 150.03753 15003.75300 2020-01-01 2020-01-02 2020-01-01 00:08:32 2020-01-02 03:36:53 2020-01-01 00:08:32.000 2020-01-02 03:36:53.000 512 99413 49962.5 4996250 512 99413 49962.5 4996250 -32658 32480 4742.66 474266 -127 124 -3.58 -358 +506 100 10496 99407 1.51951 298.51951 150.01951 15001.95195 1.51951 298.51953 150.01951 15001.9519 1.51951 298.51951 150.01951000000005 15001.95100 2020-01-01 2020-01-02 2020-01-01 00:08:26 2020-01-02 03:36:47 2020-01-01 00:08:26.000 2020-01-02 03:36:47.000 506 99407 49956.5 4995650 506 99407 49956.5 4995650 -32664 32474 4736.66 473666 -127 125 -1.9 -190 +507 100 10497 99408 1.52252 298.52252 150.02252 15002.25225 1.52252 298.52252 150.02252 15002.2522 1.52252 298.52252 150.02252000000016 15002.25200 2020-01-01 2020-01-02 2020-01-01 00:08:27 2020-01-02 03:36:48 2020-01-01 00:08:27.000 2020-01-02 03:36:48.000 507 99408 49957.5 4995750 507 99408 49957.5 4995750 -32663 32475 4737.66 473766 -126 126 -0.9 -90 +508 100 10498 99409 1.52552 298.52552 150.02552 15002.55255 1.52552 298.5255 150.02552 15002.5525 1.52552 298.52552 150.02551999999994 15002.55200 2020-01-01 2020-01-02 2020-01-01 00:08:28 2020-01-02 03:36:49 2020-01-01 00:08:28.000 2020-01-02 03:36:49.000 508 99409 49958.5 4995850 508 99409 49958.5 4995850 -32662 32476 4738.66 473866 -125 127 0.1 10 +509 100 10499 99410 1.52852 298.52852 150.02852 15002.85285 1.52852 298.52853 150.02853 15002.85312 1.52852 298.52852 150.02851999999993 15002.85200 2020-01-01 2020-01-02 2020-01-01 00:08:29 2020-01-02 03:36:50 2020-01-01 00:08:29.000 2020-01-02 03:36:50.000 509 99410 49959.5 4995950 509 99410 49959.5 4995950 -32661 32477 4739.66 473966 -128 127 -1.46 -146 +51 102 10041 99951 0.15315 300.15315 150.15315 15165.46846 0.15315 300.15317 150.15315 15165.46863 0.15315 300.15315 150.15315000000007 15165.46815 2020-01-01 2020-01-02 2020-01-01 00:00:51 2020-01-02 03:45:51 2020-01-01 00:00:51.000 2020-01-02 03:45:51.000 51 99951 50001 5050101 51 99951 50001 5050101 -32518 32417 4580.009900990099 462581 -126 125 -2.6435643564356437 -267 +510 100 10500 99411 1.53153 298.53153 150.03153 15003.15315 1.53153 298.53152 150.03152 15003.15278 1.53153 298.53153 150.0315300000001 15003.15300 2020-01-01 2020-01-02 2020-01-01 00:08:30 2020-01-02 03:36:51 2020-01-01 00:08:30.000 2020-01-02 03:36:51.000 510 99411 49960.5 4996050 510 99411 49960.5 4996050 -32660 32478 4740.66 474066 -128 127 -3.02 -302 +511 100 10501 99412 1.53453 298.53453 150.03453 15003.45345 1.53453 298.53455 150.03453 15003.45354 1.53453 298.53453 150.03453000000005 15003.45300 2020-01-01 2020-01-02 2020-01-01 00:08:31 2020-01-02 03:36:52 2020-01-01 00:08:31.000 2020-01-02 03:36:52.000 511 99412 49961.5 4996150 511 99412 49961.5 4996150 -32659 32479 4741.66 474166 -128 123 -4.58 -458 +512 100 10502 99413 1.53753 298.53753 150.03753 15003.75375 1.53753 298.53754 150.03753 15003.75366 1.53753 298.53753 150.0375299999999 15003.75300 2020-01-01 2020-01-02 2020-01-01 00:08:32 2020-01-02 03:36:53 2020-01-01 00:08:32.000 2020-01-02 03:36:53.000 512 99413 49962.5 4996250 512 99413 49962.5 4996250 -32658 32480 4742.66 474266 -127 124 -3.58 -358 513 100 10503 99414 1.54054 298.54054 150.04054 15004.05405 1.54054 298.54053 150.04053 15004.05397 1.54054 298.54054 150.04054 15004.05400 2020-01-01 2020-01-02 2020-01-01 00:08:33 2020-01-02 03:36:54 2020-01-01 00:08:33.000 2020-01-02 03:36:54.000 513 99414 49963.5 4996350 513 99414 49963.5 4996350 -32657 32481 4743.66 474366 -126 125 -2.58 -258 514 100 10504 99415 1.54354 298.54354 150.04354 15004.35435 1.54354 298.54355 150.04354 15004.35459 1.54354 298.54354 150.04354 15004.35400 2020-01-01 2020-01-02 2020-01-01 00:08:34 2020-01-02 03:36:55 2020-01-01 00:08:34.000 2020-01-02 03:36:55.000 514 99415 49964.5 4996450 514 99415 49964.5 4996450 -32656 32482 4744.66 474466 -125 126 -1.58 -158 -515 100 10505 99416 1.54654 298.54654 150.04654 15004.65465 1.54654 298.54654 150.04654 15004.65425 1.54654 298.54654 150.04654 15004.65400 2020-01-01 2020-01-02 2020-01-01 00:08:35 2020-01-02 03:36:56 2020-01-01 00:08:35.000 2020-01-02 03:36:56.000 515 99416 49965.5 4996550 515 99416 49965.5 4996550 -32655 32483 4745.66 474566 -124 127 -0.58 -58 +515 100 10505 99416 1.54654 298.54654 150.04654 15004.65465 1.54654 298.54654 150.04654 15004.65425 1.54654 298.54654 150.04653999999994 15004.65400 2020-01-01 2020-01-02 2020-01-01 00:08:35 2020-01-02 03:36:56 2020-01-01 00:08:35.000 2020-01-02 03:36:56.000 515 99416 49965.5 4996550 515 99416 49965.5 4996550 -32655 32483 4745.66 474566 -124 127 -0.58 -58 516 100 10506 99417 1.54954 298.54954 150.04954 15004.95495 1.54954 298.54956 150.04955 15004.955 1.54954 298.54954 150.04954 15004.95400 2020-01-01 2020-01-02 2020-01-01 00:08:36 2020-01-02 03:36:57 2020-01-01 00:08:36.000 2020-01-02 03:36:57.000 516 99417 49966.5 4996650 516 99417 49966.5 4996650 -32654 32484 4746.66 474666 -128 127 -2.14 -214 -517 100 10507 99418 1.55255 298.55255 150.05255 15005.25525 1.55255 298.55255 150.05255 15005.25514 1.55255 298.55255 150.05255 15005.25500 2020-01-01 2020-01-02 2020-01-01 00:08:37 2020-01-02 03:36:58 2020-01-01 00:08:37.000 2020-01-02 03:36:58.000 517 99418 49967.5 4996750 517 99418 49967.5 4996750 -32653 32485 4747.66 474766 -128 123 -3.7 -370 -518 100 10508 99419 1.55555 298.55555 150.05555 15005.55555 1.55555 298.55554 150.05555 15005.55547 1.55555 298.55555 150.05555 15005.55500 2020-01-01 2020-01-02 2020-01-01 00:08:38 2020-01-02 03:36:59 2020-01-01 00:08:38.000 2020-01-02 03:36:59.000 518 99419 49968.5 4996850 518 99419 49968.5 4996850 -32652 32486 4748.66 474866 -127 124 -2.7 -270 -519 100 10509 99420 1.55855 298.55855 150.05855 15005.85585 1.55855 298.55856 150.05856 15005.85607 1.55855 298.55855 150.05855 15005.85500 2020-01-01 2020-01-02 2020-01-01 00:08:39 2020-01-02 03:37:00 2020-01-01 00:08:39.000 2020-01-02 03:37:00.000 519 99420 49969.5 4996950 519 99420 49969.5 4996950 -32651 32487 4749.66 474966 -126 125 -1.7 -170 +517 100 10507 99418 1.55255 298.55255 150.05255 15005.25525 1.55255 298.55255 150.05255 15005.25514 1.55255 298.55255 150.05255000000005 15005.25500 2020-01-01 2020-01-02 2020-01-01 00:08:37 2020-01-02 03:36:58 2020-01-01 00:08:37.000 2020-01-02 03:36:58.000 517 99418 49967.5 4996750 517 99418 49967.5 4996750 -32653 32485 4747.66 474766 -128 123 -3.7 -370 +518 100 10508 99419 1.55555 298.55555 150.05555 15005.55555 1.55555 298.55554 150.05555 15005.55547 1.55555 298.55555 150.05554999999993 15005.55500 2020-01-01 2020-01-02 2020-01-01 00:08:38 2020-01-02 03:36:59 2020-01-01 00:08:38.000 2020-01-02 03:36:59.000 518 99419 49968.5 4996850 518 99419 49968.5 4996850 -32652 32486 4748.66 474866 -127 124 -2.7 -270 +519 100 10509 99420 1.55855 298.55855 150.05855 15005.85585 1.55855 298.55856 150.05856 15005.85607 1.55855 298.55855 150.05854999999997 15005.85500 2020-01-01 2020-01-02 2020-01-01 00:08:39 2020-01-02 03:37:00 2020-01-01 00:08:39.000 2020-01-02 03:37:00.000 519 99420 49969.5 4996950 519 99420 49969.5 4996950 -32651 32487 4749.66 474966 -126 125 -1.7 -170 52 102 10042 99952 0.15615 300.15615 150.15615 15165.77177 0.15615 300.15616 150.15615 15165.77193 0.15615 300.15615 150.15615 15165.77115 2020-01-01 2020-01-02 2020-01-01 00:00:52 2020-01-02 03:45:52 2020-01-01 00:00:52.000 2020-01-02 03:45:52.000 52 99952 50002 5050202 52 99952 50002 5050202 -32517 32418 4581.009900990099 462682 -125 126 -1.6435643564356435 -166 -520 100 10510 99421 1.56156 298.56156 150.06156 15006.15615 1.56156 298.56155 150.06155 15006.15572 1.56156 298.56156 150.06156 15006.15600 2020-01-01 2020-01-02 2020-01-01 00:08:40 2020-01-02 03:37:01 2020-01-01 00:08:40.000 2020-01-02 03:37:01.000 520 99421 49970.5 4997050 520 99421 49970.5 4997050 -32650 32488 4750.66 475066 -125 126 -0.7 -70 -521 100 10511 99422 1.56456 298.56456 150.06456 15006.45645 1.56456 298.56458 150.06456 15006.45647 1.56456 298.56456 150.06456 15006.45600 2020-01-01 2020-01-02 2020-01-01 00:08:41 2020-01-02 03:37:02 2020-01-01 00:08:41.000 2020-01-02 03:37:02.000 521 99422 49971.5 4997150 521 99422 49971.5 4997150 -32649 32489 4751.66 475166 -124 127 0.3 30 +520 100 10510 99421 1.56156 298.56156 150.06156 15006.15615 1.56156 298.56155 150.06155 15006.15572 1.56156 298.56156 150.06156000000007 15006.15600 2020-01-01 2020-01-02 2020-01-01 00:08:40 2020-01-02 03:37:01 2020-01-01 00:08:40.000 2020-01-02 03:37:01.000 520 99421 49970.5 4997050 520 99421 49970.5 4997050 -32650 32488 4750.66 475066 -125 126 -0.7 -70 +521 100 10511 99422 1.56456 298.56456 150.06456 15006.45645 1.56456 298.56458 150.06456 15006.45647 1.56456 298.56456 150.06456000000003 15006.45600 2020-01-01 2020-01-02 2020-01-01 00:08:41 2020-01-02 03:37:02 2020-01-01 00:08:41.000 2020-01-02 03:37:02.000 521 99422 49971.5 4997150 521 99422 49971.5 4997150 -32649 32489 4751.66 475166 -124 127 0.3 30 522 100 10512 99423 1.56756 298.56756 150.06756 15006.75675 1.56756 298.56757 150.06756 15006.75661 1.56756 298.56756 150.06756 15006.75600 2020-01-01 2020-01-02 2020-01-01 00:08:42 2020-01-02 03:37:03 2020-01-01 00:08:42.000 2020-01-02 03:37:03.000 522 99423 49972.5 4997250 522 99423 49972.5 4997250 -32648 32490 4752.66 475266 -128 127 -1.26 -126 -523 100 10513 99424 1.57057 298.57057 150.07057 15007.05705 1.57057 298.57056 150.07056 15007.05694 1.57057 298.57057 150.07057 15007.05700 2020-01-01 2020-01-02 2020-01-01 00:08:43 2020-01-02 03:37:04 2020-01-01 00:08:43.000 2020-01-02 03:37:04.000 523 99424 49973.5 4997350 523 99424 49973.5 4997350 -32647 32491 4753.66 475366 -128 123 -2.82 -282 +523 100 10513 99424 1.57057 298.57057 150.07057 15007.05705 1.57057 298.57056 150.07056 15007.05694 1.57057 298.57057 150.0705699999999 15007.05700 2020-01-01 2020-01-02 2020-01-01 00:08:43 2020-01-02 03:37:04 2020-01-01 00:08:43.000 2020-01-02 03:37:04.000 523 99424 49973.5 4997350 523 99424 49973.5 4997350 -32647 32491 4753.66 475366 -128 123 -2.82 -282 524 100 10514 99425 1.57357 298.57357 150.07357 15007.35735 1.57357 298.57358 150.07357 15007.35753 1.57357 298.57357 150.07357 15007.35700 2020-01-01 2020-01-02 2020-01-01 00:08:44 2020-01-02 03:37:05 2020-01-01 00:08:44.000 2020-01-02 03:37:05.000 524 99425 49974.5 4997450 524 99425 49974.5 4997450 -32646 32492 4754.66 475466 -127 124 -1.82 -182 -525 100 10515 99426 1.57657 298.57657 150.07657 15007.65765 1.57657 298.57657 150.07657 15007.65783 1.57657 298.57657 150.07657 15007.65700 2020-01-01 2020-01-02 2020-01-01 00:08:45 2020-01-02 03:37:06 2020-01-01 00:08:45.000 2020-01-02 03:37:06.000 525 99426 49975.5 4997550 525 99426 49975.5 4997550 -32645 32493 4755.66 475566 -126 125 -0.82 -82 -526 100 10516 99427 1.57957 298.57957 150.07957 15007.95795 1.57957 298.5796 150.07957 15007.95795 1.57957 298.57957 150.07957 15007.95700 2020-01-01 2020-01-02 2020-01-01 00:08:46 2020-01-02 03:37:07 2020-01-01 00:08:46.000 2020-01-02 03:37:07.000 526 99427 49976.5 4997650 526 99427 49976.5 4997650 -32644 32494 4756.66 475666 -125 126 0.18 18 -527 100 10517 99428 1.58258 298.58258 150.08258 15008.25825 1.58258 298.58258 150.08258 15008.25811 1.58258 298.58258 150.08258 15008.25800 2020-01-01 2020-01-02 2020-01-01 00:08:47 2020-01-02 03:37:08 2020-01-01 00:08:47.000 2020-01-02 03:37:08.000 527 99428 49977.5 4997750 527 99428 49977.5 4997750 -32643 32495 4757.66 475766 -124 127 1.18 118 -528 100 10518 99429 1.58558 298.58558 150.08558 15008.55855 1.58558 298.58557 150.08558 15008.5584 1.58558 298.58558 150.08558 15008.55800 2020-01-01 2020-01-02 2020-01-01 00:08:48 2020-01-02 03:37:09 2020-01-01 00:08:48.000 2020-01-02 03:37:09.000 528 99429 49978.5 4997850 528 99429 49978.5 4997850 -32642 32496 4758.66 475866 -128 127 -0.38 -38 -529 100 10519 99430 1.58858 298.58858 150.08858 15008.85885 1.58858 298.5886 150.08859 15008.859 1.58858 298.58858 150.08858 15008.85800 2020-01-01 2020-01-02 2020-01-01 00:08:49 2020-01-02 03:37:10 2020-01-01 00:08:49.000 2020-01-02 03:37:10.000 529 99430 49979.5 4997950 529 99430 49979.5 4997950 -32641 32497 4759.66 475966 -128 127 -1.94 -194 +525 100 10515 99426 1.57657 298.57657 150.07657 15007.65765 1.57657 298.57657 150.07657 15007.65783 1.57657 298.57657 150.07656999999986 15007.65700 2020-01-01 2020-01-02 2020-01-01 00:08:45 2020-01-02 03:37:06 2020-01-01 00:08:45.000 2020-01-02 03:37:06.000 525 99426 49975.5 4997550 525 99426 49975.5 4997550 -32645 32493 4755.66 475566 -126 125 -0.82 -82 +526 100 10516 99427 1.57957 298.57957 150.07957 15007.95795 1.57957 298.5796 150.07957 15007.95795 1.57957 298.57957 150.0795699999999 15007.95700 2020-01-01 2020-01-02 2020-01-01 00:08:46 2020-01-02 03:37:07 2020-01-01 00:08:46.000 2020-01-02 03:37:07.000 526 99427 49976.5 4997650 526 99427 49976.5 4997650 -32644 32494 4756.66 475666 -125 126 0.18 18 +527 100 10517 99428 1.58258 298.58258 150.08258 15008.25825 1.58258 298.58258 150.08258 15008.25811 1.58258 298.58258 150.08258000000015 15008.25800 2020-01-01 2020-01-02 2020-01-01 00:08:47 2020-01-02 03:37:08 2020-01-01 00:08:47.000 2020-01-02 03:37:08.000 527 99428 49977.5 4997750 527 99428 49977.5 4997750 -32643 32495 4757.66 475766 -124 127 1.18 118 +528 100 10518 99429 1.58558 298.58558 150.08558 15008.55855 1.58558 298.58557 150.08558 15008.5584 1.58558 298.58558 150.08558000000025 15008.55800 2020-01-01 2020-01-02 2020-01-01 00:08:48 2020-01-02 03:37:09 2020-01-01 00:08:48.000 2020-01-02 03:37:09.000 528 99429 49978.5 4997850 528 99429 49978.5 4997850 -32642 32496 4758.66 475866 -128 127 -0.38 -38 +529 100 10519 99430 1.58858 298.58858 150.08858 15008.85885 1.58858 298.5886 150.08859 15008.859 1.58858 298.58858 150.08857999999975 15008.85800 2020-01-01 2020-01-02 2020-01-01 00:08:49 2020-01-02 03:37:10 2020-01-01 00:08:49.000 2020-01-02 03:37:10.000 529 99430 49979.5 4997950 529 99430 49979.5 4997950 -32641 32497 4759.66 475966 -128 127 -1.94 -194 53 102 10043 99953 0.15915 300.15915 150.15915 15166.07507 0.15915 300.15915 150.15915 15166.07511 0.15915 300.15915 150.15915 15166.07415 2020-01-01 2020-01-02 2020-01-01 00:00:53 2020-01-02 03:45:53 2020-01-01 00:00:53.000 2020-01-02 03:45:53.000 53 99953 50003 5050303 53 99953 50003 5050303 -32516 32419 4582.009900990099 462783 -124 127 -0.6435643564356436 -65 530 100 10520 99431 1.59159 298.59159 150.09159 15009.15915 1.59159 298.59158 150.09159 15009.15929 1.59159 298.59159 150.09159 15009.15900 2020-01-01 2020-01-02 2020-01-01 00:08:50 2020-01-02 03:37:11 2020-01-01 00:08:50.000 2020-01-02 03:37:11.000 530 99431 49980.5 4998050 530 99431 49980.5 4998050 -32640 32498 4760.66 476066 -128 124 -3.5 -350 -531 100 10521 99432 1.59459 298.59459 150.09459 15009.45945 1.59459 298.5946 150.09459 15009.45941 1.59459 298.59459 150.09459 15009.45900 2020-01-01 2020-01-02 2020-01-01 00:08:51 2020-01-02 03:37:12 2020-01-01 00:08:51.000 2020-01-02 03:37:12.000 531 99432 49981.5 4998150 531 99432 49981.5 4998150 -32639 32499 4761.66 476166 -127 125 -2.5 -250 -532 100 10522 99433 1.59759 298.59759 150.09759 15009.75975 1.59759 298.5976 150.09759 15009.75958 1.59759 298.59759 150.09759 15009.75900 2020-01-01 2020-01-02 2020-01-01 00:08:52 2020-01-02 03:37:13 2020-01-01 00:08:52.000 2020-01-02 03:37:13.000 532 99433 49982.5 4998250 532 99433 49982.5 4998250 -32638 32500 4762.66 476266 -126 126 -1.5 -150 -533 100 10523 99434 1.6006 298.6006 150.1006 15010.06006 1.6006 298.6006 150.10059 15010.05988 1.60060 298.60060 150.10060 15010.06000 2020-01-01 2020-01-02 2020-01-01 00:08:53 2020-01-02 03:37:14 2020-01-01 00:08:53.000 2020-01-02 03:37:14.000 533 99434 49983.5 4998350 533 99434 49983.5 4998350 -32637 32501 4763.66 476366 -125 127 -0.5 -50 -534 100 10524 99435 1.6036 298.6036 150.1036 15010.36036 1.6036 298.6036 150.1036 15010.36063 1.60360 298.60360 150.10360 15010.36000 2020-01-01 2020-01-02 2020-01-01 00:08:54 2020-01-02 03:37:15 2020-01-01 00:08:54.000 2020-01-02 03:37:15.000 534 99435 49984.5 4998450 534 99435 49984.5 4998450 -32636 32502 4764.66 476466 -128 127 -2.06 -206 -535 100 10525 99436 1.6066 298.6066 150.1066 15010.66066 1.6066 298.6066 150.1066 15010.66077 1.60660 298.60660 150.10660 15010.66000 2020-01-01 2020-01-02 2020-01-01 00:08:55 2020-01-02 03:37:16 2020-01-01 00:08:55.000 2020-01-02 03:37:16.000 535 99436 49985.5 4998550 535 99436 49985.5 4998550 -32635 32503 4765.66 476566 -128 127 -3.62 -362 -536 100 10526 99437 1.6096 298.6096 150.1096 15010.96096 1.6096 298.60962 150.1096 15010.96092 1.60960 298.60960 150.10960 15010.96000 2020-01-01 2020-01-02 2020-01-01 00:08:56 2020-01-02 03:37:17 2020-01-01 00:08:56.000 2020-01-02 03:37:17.000 536 99437 49986.5 4998650 536 99437 49986.5 4998650 -32634 32504 4766.66 476666 -128 123 -5.18 -518 -537 100 10527 99438 1.61261 298.61261 150.11261 15011.26126 1.61261 298.6126 150.11261 15011.26105 1.61261 298.61261 150.11261 15011.26100 2020-01-01 2020-01-02 2020-01-01 00:08:57 2020-01-02 03:37:18 2020-01-01 00:08:57.000 2020-01-02 03:37:18.000 537 99438 49987.5 4998750 537 99438 49987.5 4998750 -32633 32505 4767.66 476766 -127 124 -4.18 -418 -538 100 10528 99439 1.61561 298.61561 150.11561 15011.56156 1.61561 298.6156 150.11561 15011.56135 1.61561 298.61561 150.11561 15011.56100 2020-01-01 2020-01-02 2020-01-01 00:08:58 2020-01-02 03:37:19 2020-01-01 00:08:58.000 2020-01-02 03:37:19.000 538 99439 49988.5 4998850 538 99439 49988.5 4998850 -32632 32506 4768.66 476866 -126 125 -3.18 -318 -539 100 10529 99440 1.61861 298.61861 150.11861 15011.86186 1.61861 298.61862 150.11862 15011.8621 1.61861 298.61861 150.11861 15011.86100 2020-01-01 2020-01-02 2020-01-01 00:08:59 2020-01-02 03:37:20 2020-01-01 00:08:59.000 2020-01-02 03:37:20.000 539 99440 49989.5 4998950 539 99440 49989.5 4998950 -32631 32507 4769.66 476966 -125 126 -2.18 -218 -54 102 10044 99954 0.16216 300.16216 150.16216 15166.37837 0.16216 300.16217 150.16216 15166.37822 0.16216 300.16216 150.16216 15166.37816 2020-01-01 2020-01-02 2020-01-01 00:00:54 2020-01-02 03:45:54 2020-01-01 00:00:54.000 2020-01-02 03:45:54.000 54 99954 50004 5050404 54 99954 50004 5050404 -32515 32420 4583.009900990099 462884 -128 127 -2.1782178217821784 -220 -540 100 10530 99441 1.62162 298.62162 150.12162 15012.16216 1.62162 298.6216 150.12162 15012.16224 1.62162 298.62162 150.12162 15012.16200 2020-01-01 2020-01-02 2020-01-01 00:09:00 2020-01-02 03:37:21 2020-01-01 00:09:00.000 2020-01-02 03:37:21.000 540 99441 49990.5 4999050 540 99441 49990.5 4999050 -32630 32508 4770.66 477066 -124 127 -1.18 -118 +531 100 10521 99432 1.59459 298.59459 150.09459 15009.45945 1.59459 298.5946 150.09459 15009.45941 1.59459 298.59459 150.0945900000001 15009.45900 2020-01-01 2020-01-02 2020-01-01 00:08:51 2020-01-02 03:37:12 2020-01-01 00:08:51.000 2020-01-02 03:37:12.000 531 99432 49981.5 4998150 531 99432 49981.5 4998150 -32639 32499 4761.66 476166 -127 125 -2.5 -250 +532 100 10522 99433 1.59759 298.59759 150.09759 15009.75975 1.59759 298.5976 150.09759 15009.75958 1.59759 298.59759 150.0975899999999 15009.75900 2020-01-01 2020-01-02 2020-01-01 00:08:52 2020-01-02 03:37:13 2020-01-01 00:08:52.000 2020-01-02 03:37:13.000 532 99433 49982.5 4998250 532 99433 49982.5 4998250 -32638 32500 4762.66 476266 -126 126 -1.5 -150 +533 100 10523 99434 1.6006 298.6006 150.1006 15010.06006 1.6006 298.6006 150.10059 15010.05988 1.60060 298.60060 150.10059999999984 15010.06000 2020-01-01 2020-01-02 2020-01-01 00:08:53 2020-01-02 03:37:14 2020-01-01 00:08:53.000 2020-01-02 03:37:14.000 533 99434 49983.5 4998350 533 99434 49983.5 4998350 -32637 32501 4763.66 476366 -125 127 -0.5 -50 +534 100 10524 99435 1.6036 298.6036 150.1036 15010.36036 1.6036 298.6036 150.1036 15010.36063 1.60360 298.60360 150.10360000000026 15010.36000 2020-01-01 2020-01-02 2020-01-01 00:08:54 2020-01-02 03:37:15 2020-01-01 00:08:54.000 2020-01-02 03:37:15.000 534 99435 49984.5 4998450 534 99435 49984.5 4998450 -32636 32502 4764.66 476466 -128 127 -2.06 -206 +535 100 10525 99436 1.6066 298.6066 150.1066 15010.66066 1.6066 298.6066 150.1066 15010.66077 1.60660 298.60660 150.10659999999976 15010.66000 2020-01-01 2020-01-02 2020-01-01 00:08:55 2020-01-02 03:37:16 2020-01-01 00:08:55.000 2020-01-02 03:37:16.000 535 99436 49985.5 4998550 535 99436 49985.5 4998550 -32635 32503 4765.66 476566 -128 127 -3.62 -362 +536 100 10526 99437 1.6096 298.6096 150.1096 15010.96096 1.6096 298.60962 150.1096 15010.96092 1.60960 298.60960 150.1095999999998 15010.96000 2020-01-01 2020-01-02 2020-01-01 00:08:56 2020-01-02 03:37:17 2020-01-01 00:08:56.000 2020-01-02 03:37:17.000 536 99437 49986.5 4998650 536 99437 49986.5 4998650 -32634 32504 4766.66 476666 -128 123 -5.18 -518 +537 100 10527 99438 1.61261 298.61261 150.11261 15011.26126 1.61261 298.6126 150.11261 15011.26105 1.61261 298.61261 150.1126100000001 15011.26100 2020-01-01 2020-01-02 2020-01-01 00:08:57 2020-01-02 03:37:18 2020-01-01 00:08:57.000 2020-01-02 03:37:18.000 537 99438 49987.5 4998750 537 99438 49987.5 4998750 -32633 32505 4767.66 476766 -127 124 -4.18 -418 +538 100 10528 99439 1.61561 298.61561 150.11561 15011.56156 1.61561 298.6156 150.11561 15011.56135 1.61561 298.61561 150.11561000000015 15011.56100 2020-01-01 2020-01-02 2020-01-01 00:08:58 2020-01-02 03:37:19 2020-01-01 00:08:58.000 2020-01-02 03:37:19.000 538 99439 49988.5 4998850 538 99439 49988.5 4998850 -32632 32506 4768.66 476866 -126 125 -3.18 -318 +539 100 10529 99440 1.61861 298.61861 150.11861 15011.86186 1.61861 298.61862 150.11862 15011.8621 1.61861 298.61861 150.11860999999993 15011.86100 2020-01-01 2020-01-02 2020-01-01 00:08:59 2020-01-02 03:37:20 2020-01-01 00:08:59.000 2020-01-02 03:37:20.000 539 99440 49989.5 4998950 539 99440 49989.5 4998950 -32631 32507 4769.66 476966 -125 126 -2.18 -218 +54 102 10044 99954 0.16216 300.16216 150.16216 15166.37837 0.16216 300.16217 150.16216 15166.37822 0.16216 300.16216 150.1621599999999 15166.37816 2020-01-01 2020-01-02 2020-01-01 00:00:54 2020-01-02 03:45:54 2020-01-01 00:00:54.000 2020-01-02 03:45:54.000 54 99954 50004 5050404 54 99954 50004 5050404 -32515 32420 4583.009900990099 462884 -128 127 -2.1782178217821784 -220 +540 100 10530 99441 1.62162 298.62162 150.12162 15012.16216 1.62162 298.6216 150.12162 15012.16224 1.62162 298.62162 150.12161999999992 15012.16200 2020-01-01 2020-01-02 2020-01-01 00:09:00 2020-01-02 03:37:21 2020-01-01 00:09:00.000 2020-01-02 03:37:21.000 540 99441 49990.5 4999050 540 99441 49990.5 4999050 -32630 32508 4770.66 477066 -124 127 -1.18 -118 541 100 10531 99442 1.62462 298.62462 150.12462 15012.46246 1.62462 298.62463 150.12462 15012.46239 1.62462 298.62462 150.12462 15012.46200 2020-01-01 2020-01-02 2020-01-01 00:09:01 2020-01-02 03:37:22 2020-01-01 00:09:01.000 2020-01-02 03:37:22.000 541 99442 49991.5 4999150 541 99442 49991.5 4999150 -32629 32509 4771.66 477166 -128 127 -2.74 -274 -542 100 10532 99443 1.62762 298.62762 150.12762 15012.76276 1.62762 298.62762 150.12762 15012.76252 1.62762 298.62762 150.12762 15012.76200 2020-01-01 2020-01-02 2020-01-01 00:09:02 2020-01-02 03:37:23 2020-01-01 00:09:02.000 2020-01-02 03:37:23.000 542 99443 49992.5 4999250 542 99443 49992.5 4999250 -32628 32510 4772.66 477266 -128 123 -4.3 -430 -543 100 10533 99444 1.63063 298.63063 150.13063 15013.06306 1.63063 298.63065 150.13063 15013.06327 1.63063 298.63063 150.13063 15013.06300 2020-01-01 2020-01-02 2020-01-01 00:09:03 2020-01-02 03:37:24 2020-01-01 00:09:03.000 2020-01-02 03:37:24.000 543 99444 49993.5 4999350 543 99444 49993.5 4999350 -32627 32511 4773.66 477366 -127 124 -3.3 -330 -544 100 10534 99445 1.63363 298.63363 150.13363 15013.36336 1.63363 298.63364 150.13363 15013.36358 1.63363 298.63363 150.13363 15013.36300 2020-01-01 2020-01-02 2020-01-01 00:09:04 2020-01-02 03:37:25 2020-01-01 00:09:04.000 2020-01-02 03:37:25.000 544 99445 49994.5 4999450 544 99445 49994.5 4999450 -32626 32512 4774.66 477466 -126 125 -2.3 -230 -545 100 10535 99446 1.63663 298.63663 150.13663 15013.66366 1.63663 298.63663 150.13663 15013.6637 1.63663 298.63663 150.13663 15013.66300 2020-01-01 2020-01-02 2020-01-01 00:09:05 2020-01-02 03:37:26 2020-01-01 00:09:05.000 2020-01-02 03:37:26.000 545 99446 49995.5 4999550 545 99446 49995.5 4999550 -32625 32513 4775.66 477566 -125 126 -1.3 -130 -546 100 10536 99447 1.63963 298.63963 150.13963 15013.96396 1.63963 298.63965 150.13963 15013.96385 1.63963 298.63963 150.13963 15013.96300 2020-01-01 2020-01-02 2020-01-01 00:09:06 2020-01-02 03:37:27 2020-01-01 00:09:06.000 2020-01-02 03:37:27.000 546 99447 49996.5 4999650 546 99447 49996.5 4999650 -32624 32514 4776.66 477666 -124 127 -0.3 -30 +542 100 10532 99443 1.62762 298.62762 150.12762 15012.76276 1.62762 298.62762 150.12762 15012.76252 1.62762 298.62762 150.1276199999999 15012.76200 2020-01-01 2020-01-02 2020-01-01 00:09:02 2020-01-02 03:37:23 2020-01-01 00:09:02.000 2020-01-02 03:37:23.000 542 99443 49992.5 4999250 542 99443 49992.5 4999250 -32628 32510 4772.66 477266 -128 123 -4.3 -430 +543 100 10533 99444 1.63063 298.63063 150.13063 15013.06306 1.63063 298.63065 150.13063 15013.06327 1.63063 298.63063 150.13062999999977 15013.06300 2020-01-01 2020-01-02 2020-01-01 00:09:03 2020-01-02 03:37:24 2020-01-01 00:09:03.000 2020-01-02 03:37:24.000 543 99444 49993.5 4999350 543 99444 49993.5 4999350 -32627 32511 4773.66 477366 -127 124 -3.3 -330 +544 100 10534 99445 1.63363 298.63363 150.13363 15013.36336 1.63363 298.63364 150.13363 15013.36358 1.63363 298.63363 150.13363000000015 15013.36300 2020-01-01 2020-01-02 2020-01-01 00:09:04 2020-01-02 03:37:25 2020-01-01 00:09:04.000 2020-01-02 03:37:25.000 544 99445 49994.5 4999450 544 99445 49994.5 4999450 -32626 32512 4774.66 477466 -126 125 -2.3 -230 +545 100 10535 99446 1.63663 298.63663 150.13663 15013.66366 1.63663 298.63663 150.13663 15013.6637 1.63663 298.63663 150.13663000000028 15013.66300 2020-01-01 2020-01-02 2020-01-01 00:09:05 2020-01-02 03:37:26 2020-01-01 00:09:05.000 2020-01-02 03:37:26.000 545 99446 49995.5 4999550 545 99446 49995.5 4999550 -32625 32513 4775.66 477566 -125 126 -1.3 -130 +546 100 10536 99447 1.63963 298.63963 150.13963 15013.96396 1.63963 298.63965 150.13963 15013.96385 1.63963 298.63963 150.13962999999978 15013.96300 2020-01-01 2020-01-02 2020-01-01 00:09:06 2020-01-02 03:37:27 2020-01-01 00:09:06.000 2020-01-02 03:37:27.000 546 99447 49996.5 4999650 546 99447 49996.5 4999650 -32624 32514 4776.66 477666 -124 127 -0.3 -30 547 100 10537 99448 1.64264 298.64264 150.14264 15014.26426 1.64264 298.64264 150.14263 15014.26399 1.64264 298.64264 150.14264 15014.26400 2020-01-01 2020-01-02 2020-01-01 00:09:07 2020-01-02 03:37:28 2020-01-01 00:09:07.000 2020-01-02 03:37:28.000 547 99448 49997.5 4999750 547 99448 49997.5 4999750 -32623 32515 4777.66 477766 -128 127 -1.86 -186 -548 100 10538 99449 1.64564 298.64564 150.14564 15014.56456 1.64564 298.64566 150.14564 15014.56474 1.64564 298.64564 150.14564 15014.56400 2020-01-01 2020-01-02 2020-01-01 00:09:08 2020-01-02 03:37:29 2020-01-01 00:09:08.000 2020-01-02 03:37:29.000 548 99449 49998.5 4999850 548 99449 49998.5 4999850 -32622 32516 4778.66 477866 -128 123 -3.42 -342 -549 100 10539 99450 1.64864 298.64864 150.14864 15014.86486 1.64864 298.64865 150.14865 15014.86504 1.64864 298.64864 150.14864 15014.86400 2020-01-01 2020-01-02 2020-01-01 00:09:09 2020-01-02 03:37:30 2020-01-01 00:09:09.000 2020-01-02 03:37:30.000 549 99450 49999.5 4999950 549 99450 49999.5 4999950 -32621 32517 4779.66 477966 -127 124 -2.42 -242 -55 102 10045 99955 0.16516 300.16516 150.16516 15166.68168 0.16516 300.16516 150.16516 15166.68151 0.16516 300.16516 150.16516 15166.68116 2020-01-01 2020-01-02 2020-01-01 00:00:55 2020-01-02 03:45:55 2020-01-01 00:00:55.000 2020-01-02 03:45:55.000 55 99955 50005 5050505 55 99955 50005 5050505 -32514 32421 4584.009900990099 462985 -128 123 -3.712871287128713 -375 -550 100 10540 99451 1.65165 298.65165 150.15165 15015.16516 1.65165 298.65164 150.15165 15015.16521 1.65165 298.65165 150.15165 15015.16500 2020-01-01 2020-01-02 2020-01-01 00:09:10 2020-01-02 03:37:31 2020-01-01 00:09:10.000 2020-01-02 03:37:31.000 550 99451 50000.5 5000050 550 99451 50000.5 5000050 -32620 32518 4780.66 478066 -126 125 -1.42 -142 -551 100 10541 99452 1.65465 298.65465 150.15465 15015.46546 1.65465 298.65466 150.15465 15015.46533 1.65465 298.65465 150.15465 15015.46500 2020-01-01 2020-01-02 2020-01-01 00:09:11 2020-01-02 03:37:32 2020-01-01 00:09:11.000 2020-01-02 03:37:32.000 551 99452 50001.5 5000150 551 99452 50001.5 5000150 -32619 32519 4781.66 478166 -125 126 -0.42 -42 -552 100 10542 99453 1.65765 298.65765 150.15765 15015.76576 1.65765 298.65765 150.15765 15015.76562 1.65765 298.65765 150.15765 15015.76500 2020-01-01 2020-01-02 2020-01-01 00:09:12 2020-01-02 03:37:33 2020-01-01 00:09:12.000 2020-01-02 03:37:33.000 552 99453 50002.5 5000250 552 99453 50002.5 5000250 -32618 32520 4782.66 478266 -124 127 0.58 58 -553 100 10543 99454 1.66066 298.66066 150.16066 15016.06606 1.66066 298.66068 150.16066 15016.06621 1.66066 298.66066 150.16066 15016.06600 2020-01-01 2020-01-02 2020-01-01 00:09:13 2020-01-02 03:37:34 2020-01-01 00:09:13.000 2020-01-02 03:37:34.000 553 99454 50003.5 5000350 553 99454 50003.5 5000350 -32617 32521 4783.66 478366 -128 127 -0.98 -98 -554 100 10544 99455 1.66366 298.66366 150.16366 15016.36636 1.66366 298.66367 150.16366 15016.36651 1.66366 298.66366 150.16366 15016.36600 2020-01-01 2020-01-02 2020-01-01 00:09:14 2020-01-02 03:37:35 2020-01-01 00:09:14.000 2020-01-02 03:37:35.000 554 99455 50004.5 5000450 554 99455 50004.5 5000450 -32616 32522 4784.66 478466 -128 127 -2.54 -254 -555 100 10545 99456 1.66666 298.66666 150.16666 15016.66666 1.66666 298.66666 150.16666 15016.66668 1.66666 298.66666 150.16666 15016.66600 2020-01-01 2020-01-02 2020-01-01 00:09:15 2020-01-02 03:37:36 2020-01-01 00:09:15.000 2020-01-02 03:37:36.000 555 99456 50005.5 5000550 555 99456 50005.5 5000550 -32615 32523 4785.66 478566 -128 124 -4.1 -410 -556 100 10546 99457 1.66966 298.66966 150.16966 15016.96696 1.66966 298.66968 150.16966 15016.9668 1.66966 298.66966 150.16966 15016.96600 2020-01-01 2020-01-02 2020-01-01 00:09:16 2020-01-02 03:37:37 2020-01-01 00:09:16.000 2020-01-02 03:37:37.000 556 99457 50006.5 5000650 556 99457 50006.5 5000650 -32614 32524 4786.66 478666 -127 125 -3.1 -310 -557 100 10547 99458 1.67267 298.67267 150.17267 15017.26726 1.67267 298.67267 150.17267 15017.26709 1.67267 298.67267 150.17267 15017.26700 2020-01-01 2020-01-02 2020-01-01 00:09:17 2020-01-02 03:37:38 2020-01-01 00:09:17.000 2020-01-02 03:37:38.000 557 99458 50007.5 5000750 557 99458 50007.5 5000750 -32613 32525 4787.66 478766 -126 126 -2.1 -210 -558 100 10548 99459 1.67567 298.67567 150.17567 15017.56756 1.67567 298.6757 150.17567 15017.56769 1.67567 298.67567 150.17567 15017.56700 2020-01-01 2020-01-02 2020-01-01 00:09:18 2020-01-02 03:37:39 2020-01-01 00:09:18.000 2020-01-02 03:37:39.000 558 99459 50008.5 5000850 558 99459 50008.5 5000850 -32612 32526 4788.66 478866 -125 127 -1.1 -110 -559 100 10549 99460 1.67867 298.67867 150.17867 15017.86786 1.67867 298.67868 150.17868 15017.86802 1.67867 298.67867 150.17867 15017.86700 2020-01-01 2020-01-02 2020-01-01 00:09:19 2020-01-02 03:37:40 2020-01-01 00:09:19.000 2020-01-02 03:37:40.000 559 99460 50009.5 5000950 559 99460 50009.5 5000950 -32611 32527 4789.66 478966 -128 127 -2.66 -266 -56 102 10046 99956 0.16816 300.16816 150.16816 15166.98498 0.16816 300.16818 150.16816 15166.98512 0.16816 300.16816 150.16816 15166.98416 2020-01-01 2020-01-02 2020-01-01 00:00:56 2020-01-02 03:45:56 2020-01-01 00:00:56.000 2020-01-02 03:45:56.000 56 99956 50006 5050606 56 99956 50006 5050606 -32513 32422 4585.009900990099 463086 -127 124 -2.712871287128713 -274 -560 100 10550 99461 1.68168 298.68168 150.18168 15018.16816 1.68168 298.68167 150.18168 15018.16815 1.68168 298.68168 150.18168 15018.16800 2020-01-01 2020-01-02 2020-01-01 00:09:20 2020-01-02 03:37:41 2020-01-01 00:09:20.000 2020-01-02 03:37:41.000 560 99461 50010.5 5001050 560 99461 50010.5 5001050 -32610 32528 4790.66 479066 -128 127 -4.22 -422 -561 100 10551 99462 1.68468 298.68468 150.18468 15018.46846 1.68468 298.6847 150.18468 15018.46826 1.68468 298.68468 150.18468 15018.46800 2020-01-01 2020-01-02 2020-01-01 00:09:21 2020-01-02 03:37:42 2020-01-01 00:09:21.000 2020-01-02 03:37:42.000 561 99462 50011.5 5001150 561 99462 50011.5 5001150 -32609 32529 4791.66 479166 -128 123 -5.78 -578 -562 100 10552 99463 1.68768 298.68768 150.18768 15018.76876 1.68768 298.68768 150.18768 15018.76856 1.68768 298.68768 150.18768 15018.76800 2020-01-01 2020-01-02 2020-01-01 00:09:22 2020-01-02 03:37:43 2020-01-01 00:09:22.000 2020-01-02 03:37:43.000 562 99463 50012.5 5001250 562 99463 50012.5 5001250 -32608 32530 4792.66 479266 -127 124 -4.78 -478 -563 100 10553 99464 1.69069 298.69069 150.19069 15019.06906 1.69069 298.6907 150.19069 15019.06915 1.69069 298.69069 150.19069 15019.06900 2020-01-01 2020-01-02 2020-01-01 00:09:23 2020-01-02 03:37:44 2020-01-01 00:09:23.000 2020-01-02 03:37:44.000 563 99464 50013.5 5001350 563 99464 50013.5 5001350 -32607 32531 4793.66 479366 -126 125 -3.78 -378 -564 100 10554 99465 1.69369 298.69369 150.19369 15019.36936 1.69369 298.6937 150.19369 15019.36948 1.69369 298.69369 150.19369 15019.36900 2020-01-01 2020-01-02 2020-01-01 00:09:24 2020-01-02 03:37:45 2020-01-01 00:09:24.000 2020-01-02 03:37:45.000 564 99465 50014.5 5001450 564 99465 50014.5 5001450 -32606 32532 4794.66 479466 -125 126 -2.78 -278 -565 100 10555 99466 1.69669 298.69669 150.19669 15019.66966 1.69669 298.6967 150.19669 15019.66962 1.69669 298.69669 150.19669 15019.66900 2020-01-01 2020-01-02 2020-01-01 00:09:25 2020-01-02 03:37:46 2020-01-01 00:09:25.000 2020-01-02 03:37:46.000 565 99466 50015.5 5001550 565 99466 50015.5 5001550 -32605 32533 4795.66 479566 -124 127 -1.78 -178 -566 100 10556 99467 1.69969 298.69969 150.19969 15019.96996 1.69969 298.6997 150.1997 15019.97037 1.69969 298.69969 150.19969 15019.96900 2020-01-01 2020-01-02 2020-01-01 00:09:26 2020-01-02 03:37:47 2020-01-01 00:09:26.000 2020-01-02 03:37:47.000 566 99467 50016.5 5001650 566 99467 50016.5 5001650 -32604 32534 4796.66 479666 -128 127 -3.34 -334 -567 100 10557 99468 1.7027 298.7027 150.2027 15020.27027 1.7027 298.7027 150.2027 15020.27003 1.70270 298.70270 150.20270 15020.27000 2020-01-01 2020-01-02 2020-01-01 00:09:27 2020-01-02 03:37:48 2020-01-01 00:09:27.000 2020-01-02 03:37:48.000 567 99468 50017.5 5001750 567 99468 50017.5 5001750 -32603 32535 4797.66 479766 -128 123 -4.9 -490 -568 100 10558 99469 1.7057 298.7057 150.2057 15020.57057 1.7057 298.70572 150.2057 15020.57066 1.70570 298.70570 150.20570 15020.57000 2020-01-01 2020-01-02 2020-01-01 00:09:28 2020-01-02 03:37:49 2020-01-01 00:09:28.000 2020-01-02 03:37:49.000 568 99469 50018.5 5001850 568 99469 50018.5 5001850 -32602 32536 4798.66 479866 -127 124 -3.9 -390 -569 100 10559 99470 1.7087 298.7087 150.2087 15020.87087 1.7087 298.7087 150.2087 15020.87095 1.70870 298.70870 150.20870 15020.87000 2020-01-01 2020-01-02 2020-01-01 00:09:29 2020-01-02 03:37:50 2020-01-01 00:09:29.000 2020-01-02 03:37:50.000 569 99470 50019.5 5001950 569 99470 50019.5 5001950 -32601 32537 4799.66 479966 -126 125 -2.9 -290 -57 102 10047 99957 0.17117 300.17117 150.17117 15167.28828 0.17117 300.17117 150.17117 15167.28841 0.17117 300.17117 150.17117 15167.28817 2020-01-01 2020-01-02 2020-01-01 00:00:57 2020-01-02 03:45:57 2020-01-01 00:00:57.000 2020-01-02 03:45:57.000 57 99957 50007 5050707 57 99957 50007 5050707 -32512 32423 4586.009900990099 463187 -126 125 -1.7128712871287128 -173 -570 100 10560 99471 1.71171 298.71171 150.21171 15021.17117 1.71171 298.7117 150.21171 15021.17109 1.71171 298.71171 150.21171 15021.17100 2020-01-01 2020-01-02 2020-01-01 00:09:30 2020-01-02 03:37:51 2020-01-01 00:09:30.000 2020-01-02 03:37:51.000 570 99471 50020.5 5002050 570 99471 50020.5 5002050 -32600 32538 4800.66 480066 -125 126 -1.9 -190 -571 100 10561 99472 1.71471 298.71471 150.21471 15021.47147 1.71471 298.71472 150.21471 15021.47184 1.71471 298.71471 150.21471 15021.47100 2020-01-01 2020-01-02 2020-01-01 00:09:31 2020-01-02 03:37:52 2020-01-01 00:09:31.000 2020-01-02 03:37:52.000 571 99472 50021.5 5002150 571 99472 50021.5 5002150 -32599 32539 4801.66 480166 -124 127 -0.9 -90 -572 100 10562 99473 1.71771 298.71771 150.21771 15021.77177 1.71771 298.7177 150.21771 15021.7715 1.71771 298.71771 150.21771 15021.77100 2020-01-01 2020-01-02 2020-01-01 00:09:32 2020-01-02 03:37:53 2020-01-01 00:09:32.000 2020-01-02 03:37:53.000 572 99473 50022.5 5002250 572 99473 50022.5 5002250 -32598 32540 4802.66 480266 -128 127 -2.46 -246 -573 100 10563 99474 1.72072 298.72072 150.22072 15022.07207 1.72072 298.72073 150.22072 15022.07212 1.72072 298.72072 150.22072 15022.07200 2020-01-01 2020-01-02 2020-01-01 00:09:33 2020-01-02 03:37:54 2020-01-01 00:09:33.000 2020-01-02 03:37:54.000 573 99474 50023.5 5002350 573 99474 50023.5 5002350 -32597 32541 4803.66 480366 -128 123 -4.02 -402 +548 100 10538 99449 1.64564 298.64564 150.14564 15014.56456 1.64564 298.64566 150.14564 15014.56474 1.64564 298.64564 150.14564000000013 15014.56400 2020-01-01 2020-01-02 2020-01-01 00:09:08 2020-01-02 03:37:29 2020-01-01 00:09:08.000 2020-01-02 03:37:29.000 548 99449 49998.5 4999850 548 99449 49998.5 4999850 -32622 32516 4778.66 477866 -128 123 -3.42 -342 +549 100 10539 99450 1.64864 298.64864 150.14864 15014.86486 1.64864 298.64865 150.14865 15014.86504 1.64864 298.64864 150.14863999999994 15014.86400 2020-01-01 2020-01-02 2020-01-01 00:09:09 2020-01-02 03:37:30 2020-01-01 00:09:09.000 2020-01-02 03:37:30.000 549 99450 49999.5 4999950 549 99450 49999.5 4999950 -32621 32517 4779.66 477966 -127 124 -2.42 -242 +55 102 10045 99955 0.16516 300.16516 150.16516 15166.68168 0.16516 300.16516 150.16516 15166.68151 0.16516 300.16516 150.16515999999996 15166.68116 2020-01-01 2020-01-02 2020-01-01 00:00:55 2020-01-02 03:45:55 2020-01-01 00:00:55.000 2020-01-02 03:45:55.000 55 99955 50005 5050505 55 99955 50005 5050505 -32514 32421 4584.009900990099 462985 -128 123 -3.712871287128713 -375 +550 100 10540 99451 1.65165 298.65165 150.15165 15015.16516 1.65165 298.65164 150.15165 15015.16521 1.65165 298.65165 150.15164999999982 15015.16500 2020-01-01 2020-01-02 2020-01-01 00:09:10 2020-01-02 03:37:31 2020-01-01 00:09:10.000 2020-01-02 03:37:31.000 550 99451 50000.5 5000050 550 99451 50000.5 5000050 -32620 32518 4780.66 478066 -126 125 -1.42 -142 +551 100 10541 99452 1.65465 298.65465 150.15465 15015.46546 1.65465 298.65466 150.15465 15015.46533 1.65465 298.65465 150.1546500000003 15015.46500 2020-01-01 2020-01-02 2020-01-01 00:09:11 2020-01-02 03:37:32 2020-01-01 00:09:11.000 2020-01-02 03:37:32.000 551 99452 50001.5 5000150 551 99452 50001.5 5000150 -32619 32519 4781.66 478166 -125 126 -0.42 -42 +552 100 10542 99453 1.65765 298.65765 150.15765 15015.76576 1.65765 298.65765 150.15765 15015.76562 1.65765 298.65765 150.1576499999998 15015.76500 2020-01-01 2020-01-02 2020-01-01 00:09:12 2020-01-02 03:37:33 2020-01-01 00:09:12.000 2020-01-02 03:37:33.000 552 99453 50002.5 5000250 552 99453 50002.5 5000250 -32618 32520 4782.66 478266 -124 127 0.58 58 +553 100 10543 99454 1.66066 298.66066 150.16066 15016.06606 1.66066 298.66068 150.16066 15016.06621 1.66066 298.66066 150.16065999999967 15016.06600 2020-01-01 2020-01-02 2020-01-01 00:09:13 2020-01-02 03:37:34 2020-01-01 00:09:13.000 2020-01-02 03:37:34.000 553 99454 50003.5 5000350 553 99454 50003.5 5000350 -32617 32521 4783.66 478366 -128 127 -0.98 -98 +554 100 10544 99455 1.66366 298.66366 150.16366 15016.36636 1.66366 298.66367 150.16366 15016.36651 1.66366 298.66366 150.16366000000014 15016.36600 2020-01-01 2020-01-02 2020-01-01 00:09:14 2020-01-02 03:37:35 2020-01-01 00:09:14.000 2020-01-02 03:37:35.000 554 99455 50004.5 5000450 554 99455 50004.5 5000450 -32616 32522 4784.66 478466 -128 127 -2.54 -254 +555 100 10545 99456 1.66666 298.66666 150.16666 15016.66666 1.66666 298.66666 150.16666 15016.66668 1.66666 298.66666 150.16666000000018 15016.66600 2020-01-01 2020-01-02 2020-01-01 00:09:15 2020-01-02 03:37:36 2020-01-01 00:09:15.000 2020-01-02 03:37:36.000 555 99456 50005.5 5000550 555 99456 50005.5 5000550 -32615 32523 4785.66 478566 -128 124 -4.1 -410 +556 100 10546 99457 1.66966 298.66966 150.16966 15016.96696 1.66966 298.66968 150.16966 15016.9668 1.66966 298.66966 150.16965999999968 15016.96600 2020-01-01 2020-01-02 2020-01-01 00:09:16 2020-01-02 03:37:37 2020-01-01 00:09:16.000 2020-01-02 03:37:37.000 556 99457 50006.5 5000650 556 99457 50006.5 5000650 -32614 32524 4786.66 478666 -127 125 -3.1 -310 +557 100 10547 99458 1.67267 298.67267 150.17267 15017.26726 1.67267 298.67267 150.17267 15017.26709 1.67267 298.67267 150.17266999999995 15017.26700 2020-01-01 2020-01-02 2020-01-01 00:09:17 2020-01-02 03:37:38 2020-01-01 00:09:17.000 2020-01-02 03:37:38.000 557 99458 50007.5 5000750 557 99458 50007.5 5000750 -32613 32525 4787.66 478766 -126 126 -2.1 -210 +558 100 10548 99459 1.67567 298.67567 150.17567 15017.56756 1.67567 298.6757 150.17567 15017.56769 1.67567 298.67567 150.17567000000003 15017.56700 2020-01-01 2020-01-02 2020-01-01 00:09:18 2020-01-02 03:37:39 2020-01-01 00:09:18.000 2020-01-02 03:37:39.000 558 99459 50008.5 5000850 558 99459 50008.5 5000850 -32612 32526 4788.66 478866 -125 127 -1.1 -110 +559 100 10549 99460 1.67867 298.67867 150.17867 15017.86786 1.67867 298.67868 150.17868 15017.86802 1.67867 298.67867 150.17866999999987 15017.86700 2020-01-01 2020-01-02 2020-01-01 00:09:19 2020-01-02 03:37:40 2020-01-01 00:09:19.000 2020-01-02 03:37:40.000 559 99460 50009.5 5000950 559 99460 50009.5 5000950 -32611 32527 4789.66 478966 -128 127 -2.66 -266 +56 102 10046 99956 0.16816 300.16816 150.16816 15166.98498 0.16816 300.16818 150.16816 15166.98512 0.16816 300.16816 150.16815999999986 15166.98416 2020-01-01 2020-01-02 2020-01-01 00:00:56 2020-01-02 03:45:56 2020-01-01 00:00:56.000 2020-01-02 03:45:56.000 56 99956 50006 5050606 56 99956 50006 5050606 -32513 32422 4585.009900990099 463086 -127 124 -2.712871287128713 -274 +560 100 10550 99461 1.68168 298.68168 150.18168 15018.16816 1.68168 298.68167 150.18168 15018.16815 1.68168 298.68168 150.1816799999998 15018.16800 2020-01-01 2020-01-02 2020-01-01 00:09:20 2020-01-02 03:37:41 2020-01-01 00:09:20.000 2020-01-02 03:37:41.000 560 99461 50010.5 5001050 560 99461 50010.5 5001050 -32610 32528 4790.66 479066 -128 127 -4.22 -422 +561 100 10551 99462 1.68468 298.68468 150.18468 15018.46846 1.68468 298.6847 150.18468 15018.46826 1.68468 298.68468 150.18468000000018 15018.46800 2020-01-01 2020-01-02 2020-01-01 00:09:21 2020-01-02 03:37:42 2020-01-01 00:09:21.000 2020-01-02 03:37:42.000 561 99462 50011.5 5001150 561 99462 50011.5 5001150 -32609 32529 4791.66 479166 -128 123 -5.78 -578 +562 100 10552 99463 1.68768 298.68768 150.18768 15018.76876 1.68768 298.68768 150.18768 15018.76856 1.68768 298.68768 150.18767999999972 15018.76800 2020-01-01 2020-01-02 2020-01-01 00:09:22 2020-01-02 03:37:43 2020-01-01 00:09:22.000 2020-01-02 03:37:43.000 562 99463 50012.5 5001250 562 99463 50012.5 5001250 -32608 32530 4792.66 479266 -127 124 -4.78 -478 +563 100 10553 99464 1.69069 298.69069 150.19069 15019.06906 1.69069 298.6907 150.19069 15019.06915 1.69069 298.69069 150.19068999999996 15019.06900 2020-01-01 2020-01-02 2020-01-01 00:09:23 2020-01-02 03:37:44 2020-01-01 00:09:23.000 2020-01-02 03:37:44.000 563 99464 50013.5 5001350 563 99464 50013.5 5001350 -32607 32531 4793.66 479366 -126 125 -3.78 -378 +564 100 10554 99465 1.69369 298.69369 150.19369 15019.36936 1.69369 298.6937 150.19369 15019.36948 1.69369 298.69369 150.19369000000003 15019.36900 2020-01-01 2020-01-02 2020-01-01 00:09:24 2020-01-02 03:37:45 2020-01-01 00:09:24.000 2020-01-02 03:37:45.000 564 99465 50014.5 5001450 564 99465 50014.5 5001450 -32606 32532 4794.66 479466 -125 126 -2.78 -278 +565 100 10555 99466 1.69669 298.69669 150.19669 15019.66966 1.69669 298.6967 150.19669 15019.66962 1.69669 298.69669 150.19669000000013 15019.66900 2020-01-01 2020-01-02 2020-01-01 00:09:25 2020-01-02 03:37:46 2020-01-01 00:09:25.000 2020-01-02 03:37:46.000 565 99466 50015.5 5001550 565 99466 50015.5 5001550 -32605 32533 4795.66 479566 -124 127 -1.78 -178 +566 100 10556 99467 1.69969 298.69969 150.19969 15019.96996 1.69969 298.6997 150.1997 15019.97037 1.69969 298.69969 150.19968999999998 15019.96900 2020-01-01 2020-01-02 2020-01-01 00:09:26 2020-01-02 03:37:47 2020-01-01 00:09:26.000 2020-01-02 03:37:47.000 566 99467 50016.5 5001650 566 99467 50016.5 5001650 -32604 32534 4796.66 479666 -128 127 -3.34 -334 +567 100 10557 99468 1.7027 298.7027 150.2027 15020.27027 1.7027 298.7027 150.2027 15020.27003 1.70270 298.70270 150.20269999999985 15020.27000 2020-01-01 2020-01-02 2020-01-01 00:09:27 2020-01-02 03:37:48 2020-01-01 00:09:27.000 2020-01-02 03:37:48.000 567 99468 50017.5 5001750 567 99468 50017.5 5001750 -32603 32535 4797.66 479766 -128 123 -4.9 -490 +568 100 10558 99469 1.7057 298.7057 150.2057 15020.57057 1.7057 298.70572 150.2057 15020.57066 1.70570 298.70570 150.20570000000032 15020.57000 2020-01-01 2020-01-02 2020-01-01 00:09:28 2020-01-02 03:37:49 2020-01-01 00:09:28.000 2020-01-02 03:37:49.000 568 99469 50018.5 5001850 568 99469 50018.5 5001850 -32602 32536 4798.66 479866 -127 124 -3.9 -390 +569 100 10559 99470 1.7087 298.7087 150.2087 15020.87087 1.7087 298.7087 150.2087 15020.87095 1.70870 298.70870 150.20869999999982 15020.87000 2020-01-01 2020-01-02 2020-01-01 00:09:29 2020-01-02 03:37:50 2020-01-01 00:09:29.000 2020-01-02 03:37:50.000 569 99470 50019.5 5001950 569 99470 50019.5 5001950 -32601 32537 4799.66 479966 -126 125 -2.9 -290 +57 102 10047 99957 0.17117 300.17117 150.17117 15167.28828 0.17117 300.17117 150.17117 15167.28841 0.17117 300.17117 150.17116999999973 15167.28817 2020-01-01 2020-01-02 2020-01-01 00:00:57 2020-01-02 03:45:57 2020-01-01 00:00:57.000 2020-01-02 03:45:57.000 57 99957 50007 5050707 57 99957 50007 5050707 -32512 32423 4586.009900990099 463187 -126 125 -1.7128712871287128 -173 +570 100 10560 99471 1.71171 298.71171 150.21171 15021.17117 1.71171 298.7117 150.21171 15021.17109 1.71171 298.71171 150.2117099999997 15021.17100 2020-01-01 2020-01-02 2020-01-01 00:09:30 2020-01-02 03:37:51 2020-01-01 00:09:30.000 2020-01-02 03:37:51.000 570 99471 50020.5 5002050 570 99471 50020.5 5002050 -32600 32538 4800.66 480066 -125 126 -1.9 -190 +571 100 10561 99472 1.71471 298.71471 150.21471 15021.47147 1.71471 298.71472 150.21471 15021.47184 1.71471 298.71471 150.21471000000017 15021.47100 2020-01-01 2020-01-02 2020-01-01 00:09:31 2020-01-02 03:37:52 2020-01-01 00:09:31.000 2020-01-02 03:37:52.000 571 99472 50021.5 5002150 571 99472 50021.5 5002150 -32599 32539 4801.66 480166 -124 127 -0.9 -90 +572 100 10562 99473 1.71771 298.71771 150.21771 15021.77177 1.71771 298.7177 150.21771 15021.7715 1.71771 298.71771 150.2177100000002 15021.77100 2020-01-01 2020-01-02 2020-01-01 00:09:32 2020-01-02 03:37:53 2020-01-01 00:09:32.000 2020-01-02 03:37:53.000 572 99473 50022.5 5002250 572 99473 50022.5 5002250 -32598 32540 4802.66 480266 -128 127 -2.46 -246 +573 100 10563 99474 1.72072 298.72072 150.22072 15022.07207 1.72072 298.72073 150.22072 15022.07212 1.72072 298.72072 150.22071999999986 15022.07200 2020-01-01 2020-01-02 2020-01-01 00:09:33 2020-01-02 03:37:54 2020-01-01 00:09:33.000 2020-01-02 03:37:54.000 573 99474 50023.5 5002350 573 99474 50023.5 5002350 -32597 32541 4803.66 480366 -128 123 -4.02 -402 574 100 10564 99475 1.72372 298.72372 150.22372 15022.37237 1.72372 298.72372 150.22372 15022.37243 1.72372 298.72372 150.22372 15022.37200 2020-01-01 2020-01-02 2020-01-01 00:09:34 2020-01-02 03:37:55 2020-01-01 00:09:34.000 2020-01-02 03:37:55.000 574 99475 50024.5 5002450 574 99475 50024.5 5002450 -32596 32542 4804.66 480466 -127 124 -3.02 -302 -575 100 10565 99476 1.72672 298.72672 150.22672 15022.67267 1.72672 298.7267 150.22672 15022.67272 1.72672 298.72672 150.22672 15022.67200 2020-01-01 2020-01-02 2020-01-01 00:09:35 2020-01-02 03:37:56 2020-01-01 00:09:35.000 2020-01-02 03:37:56.000 575 99476 50025.5 5002550 575 99476 50025.5 5002550 -32595 32543 4805.66 480566 -126 125 -2.02 -202 -576 100 10566 99477 1.72972 298.72972 150.22972 15022.97297 1.72972 298.72974 150.22973 15022.97332 1.72972 298.72972 150.22972 15022.97200 2020-01-01 2020-01-02 2020-01-01 00:09:36 2020-01-02 03:37:57 2020-01-01 00:09:36.000 2020-01-02 03:37:57.000 576 99477 50026.5 5002650 576 99477 50026.5 5002650 -32594 32544 4806.66 480666 -125 126 -1.02 -102 -577 100 10567 99478 1.73273 298.73273 150.23273 15023.27327 1.73273 298.73273 150.23272 15023.27297 1.73273 298.73273 150.23273 15023.27300 2020-01-01 2020-01-02 2020-01-01 00:09:37 2020-01-02 03:37:58 2020-01-01 00:09:37.000 2020-01-02 03:37:58.000 577 99478 50027.5 5002750 577 99478 50027.5 5002750 -32593 32545 4807.66 480766 -124 127 -0.02 -2 -578 100 10568 99479 1.73573 298.73573 150.23573 15023.57357 1.73573 298.73575 150.23573 15023.57359 1.73573 298.73573 150.23573 15023.57300 2020-01-01 2020-01-02 2020-01-01 00:09:38 2020-01-02 03:37:59 2020-01-01 00:09:38.000 2020-01-02 03:37:59.000 578 99479 50028.5 5002850 578 99479 50028.5 5002850 -32592 32546 4808.66 480866 -128 127 -1.58 -158 -579 100 10569 99480 1.73873 298.73873 150.23873 15023.87387 1.73873 298.73874 150.23873 15023.8739 1.73873 298.73873 150.23873 15023.87300 2020-01-01 2020-01-02 2020-01-01 00:09:39 2020-01-02 03:38:00 2020-01-01 00:09:39.000 2020-01-02 03:38:00.000 579 99480 50029.5 5002950 579 99480 50029.5 5002950 -32591 32547 4809.66 480966 -128 123 -3.14 -314 -58 102 10048 99958 0.17417 300.17417 150.17417 15167.59159 0.17417 300.17416 150.17417 15167.59159 0.17417 300.17417 150.17417 15167.59117 2020-01-01 2020-01-02 2020-01-01 00:00:58 2020-01-02 03:45:58 2020-01-01 00:00:58.000 2020-01-02 03:45:58.000 58 99958 50008 5050808 58 99958 50008 5050808 -32511 32424 4587.009900990099 463288 -125 126 -0.7128712871287128 -72 +575 100 10565 99476 1.72672 298.72672 150.22672 15022.67267 1.72672 298.7267 150.22672 15022.67272 1.72672 298.72672 150.22672000000006 15022.67200 2020-01-01 2020-01-02 2020-01-01 00:09:35 2020-01-02 03:37:56 2020-01-01 00:09:35.000 2020-01-02 03:37:56.000 575 99476 50025.5 5002550 575 99476 50025.5 5002550 -32595 32543 4805.66 480566 -126 125 -2.02 -202 +576 100 10566 99477 1.72972 298.72972 150.22972 15022.97297 1.72972 298.72974 150.22973 15022.97332 1.72972 298.72972 150.22971999999987 15022.97200 2020-01-01 2020-01-02 2020-01-01 00:09:36 2020-01-02 03:37:57 2020-01-01 00:09:36.000 2020-01-02 03:37:57.000 576 99477 50026.5 5002650 576 99477 50026.5 5002650 -32594 32544 4806.66 480666 -125 126 -1.02 -102 +577 100 10567 99478 1.73273 298.73273 150.23273 15023.27327 1.73273 298.73273 150.23272 15023.27297 1.73273 298.73273 150.23272999999983 15023.27300 2020-01-01 2020-01-02 2020-01-01 00:09:37 2020-01-02 03:37:58 2020-01-01 00:09:37.000 2020-01-02 03:37:58.000 577 99478 50027.5 5002750 577 99478 50027.5 5002750 -32593 32545 4807.66 480766 -124 127 -0.02 -2 +578 100 10568 99479 1.73573 298.73573 150.23573 15023.57357 1.73573 298.73575 150.23573 15023.57359 1.73573 298.73573 150.23573000000022 15023.57300 2020-01-01 2020-01-02 2020-01-01 00:09:38 2020-01-02 03:37:59 2020-01-01 00:09:38.000 2020-01-02 03:37:59.000 578 99479 50028.5 5002850 578 99479 50028.5 5002850 -32592 32546 4808.66 480866 -128 127 -1.58 -158 +579 100 10569 99480 1.73873 298.73873 150.23873 15023.87387 1.73873 298.73874 150.23873 15023.8739 1.73873 298.73873 150.23872999999972 15023.87300 2020-01-01 2020-01-02 2020-01-01 00:09:39 2020-01-02 03:38:00 2020-01-01 00:09:39.000 2020-01-02 03:38:00.000 579 99480 50029.5 5002950 579 99480 50029.5 5002950 -32591 32547 4809.66 480966 -128 123 -3.14 -314 +58 102 10048 99958 0.17417 300.17417 150.17417 15167.59159 0.17417 300.17416 150.17417 15167.59159 0.17417 300.17417 150.17417000000015 15167.59117 2020-01-01 2020-01-02 2020-01-01 00:00:58 2020-01-02 03:45:58 2020-01-01 00:00:58.000 2020-01-02 03:45:58.000 58 99958 50008 5050808 58 99958 50008 5050808 -32511 32424 4587.009900990099 463288 -125 126 -0.7128712871287128 -72 580 100 10570 99481 1.74174 298.74174 150.24174 15024.17417 1.74174 298.74173 150.24174 15024.17419 1.74174 298.74174 150.24174 15024.17400 2020-01-01 2020-01-02 2020-01-01 00:09:40 2020-01-02 03:38:01 2020-01-01 00:09:40.000 2020-01-02 03:38:01.000 580 99481 50030.5 5003050 580 99481 50030.5 5003050 -32590 32548 4810.66 481066 -127 124 -2.14 -214 -581 100 10571 99482 1.74474 298.74474 150.24474 15024.47447 1.74474 298.74475 150.24474 15024.47478 1.74474 298.74474 150.24474 15024.47400 2020-01-01 2020-01-02 2020-01-01 00:09:41 2020-01-02 03:38:02 2020-01-01 00:09:41.000 2020-01-02 03:38:02.000 581 99482 50031.5 5003150 581 99482 50031.5 5003150 -32589 32549 4811.66 481166 -126 125 -1.14 -114 -582 100 10572 99483 1.74774 298.74774 150.24774 15024.77477 1.74774 298.74774 150.24774 15024.77447 1.74774 298.74774 150.24774 15024.77400 2020-01-01 2020-01-02 2020-01-01 00:09:42 2020-01-02 03:38:03 2020-01-01 00:09:42.000 2020-01-02 03:38:03.000 582 99483 50032.5 5003250 582 99483 50032.5 5003250 -32588 32550 4812.66 481266 -125 126 -0.14 -14 -583 100 10573 99484 1.75075 298.75075 150.25075 15025.07507 1.75075 298.75076 150.25075 15025.07507 1.75075 298.75075 150.25075 15025.07500 2020-01-01 2020-01-02 2020-01-01 00:09:43 2020-01-02 03:38:04 2020-01-01 00:09:43.000 2020-01-02 03:38:04.000 583 99484 50033.5 5003350 583 99484 50033.5 5003350 -32587 32551 4813.66 481366 -124 127 0.86 86 -584 100 10574 99485 1.75375 298.75375 150.25375 15025.37537 1.75375 298.75375 150.25375 15025.37536 1.75375 298.75375 150.25375 15025.37500 2020-01-01 2020-01-02 2020-01-01 00:09:44 2020-01-02 03:38:05 2020-01-01 00:09:44.000 2020-01-02 03:38:05.000 584 99485 50034.5 5003450 584 99485 50034.5 5003450 -32586 32552 4814.66 481466 -128 127 -0.7 -70 -585 100 10575 99486 1.75675 298.75675 150.25675 15025.67567 1.75675 298.75674 150.25675 15025.67566 1.75675 298.75675 150.25675 15025.67500 2020-01-01 2020-01-02 2020-01-01 00:09:45 2020-01-02 03:38:06 2020-01-01 00:09:45.000 2020-01-02 03:38:06.000 585 99486 50035.5 5003550 585 99486 50035.5 5003550 -32585 32553 4815.66 481566 -128 127 -2.26 -226 -586 100 10576 99487 1.75975 298.75975 150.25975 15025.97597 1.75975 298.75977 150.25976 15025.97625 1.75975 298.75975 150.25975 15025.97500 2020-01-01 2020-01-02 2020-01-01 00:09:46 2020-01-02 03:38:07 2020-01-01 00:09:46.000 2020-01-02 03:38:07.000 586 99487 50036.5 5003650 586 99487 50036.5 5003650 -32584 32554 4816.66 481666 -128 123 -3.82 -382 -587 100 10577 99488 1.76276 298.76276 150.26276 15026.27627 1.76276 298.76276 150.26275 15026.27594 1.76276 298.76276 150.26276 15026.27600 2020-01-01 2020-01-02 2020-01-01 00:09:47 2020-01-02 03:38:08 2020-01-01 00:09:47.000 2020-01-02 03:38:08.000 587 99488 50037.5 5003750 587 99488 50037.5 5003750 -32583 32555 4817.66 481766 -127 124 -2.82 -282 -588 100 10578 99489 1.76576 298.76576 150.26576 15026.57657 1.76576 298.76578 150.26576 15026.57654 1.76576 298.76576 150.26576 15026.57600 2020-01-01 2020-01-02 2020-01-01 00:09:48 2020-01-02 03:38:09 2020-01-01 00:09:48.000 2020-01-02 03:38:09.000 588 99489 50038.5 5003850 588 99489 50038.5 5003850 -32582 32556 4818.66 481866 -126 125 -1.82 -182 -589 100 10579 99490 1.76876 298.76876 150.26876 15026.87687 1.76876 298.76877 150.26876 15026.87683 1.76876 298.76876 150.26876 15026.87600 2020-01-01 2020-01-02 2020-01-01 00:09:49 2020-01-02 03:38:10 2020-01-01 00:09:49.000 2020-01-02 03:38:10.000 589 99490 50039.5 5003950 589 99490 50039.5 5003950 -32581 32557 4819.66 481966 -125 126 -0.82 -82 -59 102 10049 99959 0.17717 300.17717 150.17717 15167.89489 0.17717 300.1772 150.17717 15167.8947 0.17717 300.17717 150.17717 15167.89417 2020-01-01 2020-01-02 2020-01-01 00:00:59 2020-01-02 03:45:59 2020-01-01 00:00:59.000 2020-01-02 03:45:59.000 59 99959 50009 5050909 59 99959 50009 5050909 -32510 32425 4588.009900990099 463389 -124 127 0.2871287128712871 29 -590 100 10580 99491 1.77177 298.77177 150.27177 15027.17717 1.77177 298.77176 150.27177 15027.17713 1.77177 298.77177 150.27177 15027.17700 2020-01-01 2020-01-02 2020-01-01 00:09:50 2020-01-02 03:38:11 2020-01-01 00:09:50.000 2020-01-02 03:38:11.000 590 99491 50040.5 5004050 590 99491 50040.5 5004050 -32580 32558 4820.66 482066 -124 127 0.18 18 +581 100 10571 99482 1.74474 298.74474 150.24474 15024.47447 1.74474 298.74475 150.24474 15024.47478 1.74474 298.74474 150.24474000000006 15024.47400 2020-01-01 2020-01-02 2020-01-01 00:09:41 2020-01-02 03:38:02 2020-01-01 00:09:41.000 2020-01-02 03:38:02.000 581 99482 50031.5 5003150 581 99482 50031.5 5003150 -32589 32549 4811.66 481166 -126 125 -1.14 -114 +582 100 10572 99483 1.74774 298.74774 150.24774 15024.77477 1.74774 298.74774 150.24774 15024.77447 1.74774 298.74774 150.2477400000001 15024.77400 2020-01-01 2020-01-02 2020-01-01 00:09:42 2020-01-02 03:38:03 2020-01-01 00:09:42.000 2020-01-02 03:38:03.000 582 99483 50032.5 5003250 582 99483 50032.5 5003250 -32588 32550 4812.66 481266 -125 126 -0.14 -14 +583 100 10573 99484 1.75075 298.75075 150.25075 15025.07507 1.75075 298.75076 150.25075 15025.07507 1.75075 298.75075 150.25074999999984 15025.07500 2020-01-01 2020-01-02 2020-01-01 00:09:43 2020-01-02 03:38:04 2020-01-01 00:09:43.000 2020-01-02 03:38:04.000 583 99484 50033.5 5003350 583 99484 50033.5 5003350 -32587 32551 4813.66 481366 -124 127 0.86 86 +584 100 10574 99485 1.75375 298.75375 150.25375 15025.37537 1.75375 298.75375 150.25375 15025.37536 1.75375 298.75375 150.25374999999988 15025.37500 2020-01-01 2020-01-02 2020-01-01 00:09:44 2020-01-02 03:38:05 2020-01-01 00:09:44.000 2020-01-02 03:38:05.000 584 99485 50034.5 5003450 584 99485 50034.5 5003450 -32586 32552 4814.66 481466 -128 127 -0.7 -70 +585 100 10575 99486 1.75675 298.75675 150.25675 15025.67567 1.75675 298.75674 150.25675 15025.67566 1.75675 298.75675 150.25675000000007 15025.67500 2020-01-01 2020-01-02 2020-01-01 00:09:45 2020-01-02 03:38:06 2020-01-01 00:09:45.000 2020-01-02 03:38:06.000 585 99486 50035.5 5003550 585 99486 50035.5 5003550 -32585 32553 4815.66 481566 -128 127 -2.26 -226 +586 100 10576 99487 1.75975 298.75975 150.25975 15025.97597 1.75975 298.75977 150.25976 15025.97625 1.75975 298.75975 150.25974999999985 15025.97500 2020-01-01 2020-01-02 2020-01-01 00:09:46 2020-01-02 03:38:07 2020-01-01 00:09:46.000 2020-01-02 03:38:07.000 586 99487 50036.5 5003650 586 99487 50036.5 5003650 -32584 32554 4816.66 481666 -128 123 -3.82 -382 +587 100 10577 99488 1.76276 298.76276 150.26276 15026.27627 1.76276 298.76276 150.26275 15026.27594 1.76276 298.76276 150.26275999999973 15026.27600 2020-01-01 2020-01-02 2020-01-01 00:09:47 2020-01-02 03:38:08 2020-01-01 00:09:47.000 2020-01-02 03:38:08.000 587 99488 50037.5 5003750 587 99488 50037.5 5003750 -32583 32555 4817.66 481766 -127 124 -2.82 -282 +588 100 10578 99489 1.76576 298.76576 150.26576 15026.57657 1.76576 298.76578 150.26576 15026.57654 1.76576 298.76576 150.2657600000002 15026.57600 2020-01-01 2020-01-02 2020-01-01 00:09:48 2020-01-02 03:38:09 2020-01-01 00:09:48.000 2020-01-02 03:38:09.000 588 99489 50038.5 5003850 588 99489 50038.5 5003850 -32582 32556 4818.66 481866 -126 125 -1.82 -182 +589 100 10579 99490 1.76876 298.76876 150.26876 15026.87687 1.76876 298.76877 150.26876 15026.87683 1.76876 298.76876 150.26876000000024 15026.87600 2020-01-01 2020-01-02 2020-01-01 00:09:49 2020-01-02 03:38:10 2020-01-01 00:09:49.000 2020-01-02 03:38:10.000 589 99490 50039.5 5003950 589 99490 50039.5 5003950 -32581 32557 4819.66 481966 -125 126 -0.82 -82 +59 102 10049 99959 0.17717 300.17717 150.17717 15167.89489 0.17717 300.1772 150.17717 15167.8947 0.17717 300.17717 150.17717000000025 15167.89417 2020-01-01 2020-01-02 2020-01-01 00:00:59 2020-01-02 03:45:59 2020-01-01 00:00:59.000 2020-01-02 03:45:59.000 59 99959 50009 5050909 59 99959 50009 5050909 -32510 32425 4588.009900990099 463389 -124 127 0.2871287128712871 29 +590 100 10580 99491 1.77177 298.77177 150.27177 15027.17717 1.77177 298.77176 150.27177 15027.17713 1.77177 298.77177 150.2717699999999 15027.17700 2020-01-01 2020-01-02 2020-01-01 00:09:50 2020-01-02 03:38:11 2020-01-01 00:09:50.000 2020-01-02 03:38:11.000 590 99491 50040.5 5004050 590 99491 50040.5 5004050 -32580 32558 4820.66 482066 -124 127 0.18 18 591 100 10581 99492 1.77477 298.77477 150.27477 15027.47747 1.77477 298.77478 150.27477 15027.47775 1.77477 298.77477 150.27477 15027.47700 2020-01-01 2020-01-02 2020-01-01 00:09:51 2020-01-02 03:38:12 2020-01-01 00:09:51.000 2020-01-02 03:38:12.000 591 99492 50041.5 5004150 591 99492 50041.5 5004150 -32579 32559 4821.66 482166 -128 127 -1.38 -138 -592 100 10582 99493 1.77777 298.77777 150.27777 15027.77777 1.77777 298.77777 150.27777 15027.77742 1.77777 298.77777 150.27777 15027.77700 2020-01-01 2020-01-02 2020-01-01 00:09:52 2020-01-02 03:38:13 2020-01-01 00:09:52.000 2020-01-02 03:38:13.000 592 99493 50042.5 5004250 592 99493 50042.5 5004250 -32578 32560 4822.66 482266 -128 123 -2.94 -294 -593 100 10583 99494 1.78078 298.78078 150.28078 15028.07807 1.78078 298.7808 150.28078 15028.078 1.78078 298.78078 150.28078 15028.07800 2020-01-01 2020-01-02 2020-01-01 00:09:53 2020-01-02 03:38:14 2020-01-01 00:09:53.000 2020-01-02 03:38:14.000 593 99494 50043.5 5004350 593 99494 50043.5 5004350 -32577 32561 4823.66 482366 -127 124 -1.94 -194 -594 100 10584 99495 1.78378 298.78378 150.28378 15028.37837 1.78378 298.78378 150.28378 15028.3783 1.78378 298.78378 150.28378 15028.37800 2020-01-01 2020-01-02 2020-01-01 00:09:54 2020-01-02 03:38:15 2020-01-01 00:09:54.000 2020-01-02 03:38:15.000 594 99495 50044.5 5004450 594 99495 50044.5 5004450 -32576 32562 4824.66 482466 -126 125 -0.94 -94 -595 100 10585 99496 1.78678 298.78678 150.28678 15028.67867 1.78678 298.78677 150.28678 15028.6786 1.78678 298.78678 150.28678 15028.67800 2020-01-01 2020-01-02 2020-01-01 00:09:55 2020-01-02 03:38:16 2020-01-01 00:09:55.000 2020-01-02 03:38:16.000 595 99496 50045.5 5004550 595 99496 50045.5 5004550 -32575 32563 4825.66 482566 -125 126 0.06 6 -596 100 10586 99497 1.78978 298.78978 150.28978 15028.97897 1.78978 298.7898 150.28979 15028.97922 1.78978 298.78978 150.28978 15028.97800 2020-01-01 2020-01-02 2020-01-01 00:09:56 2020-01-02 03:38:17 2020-01-01 00:09:56.000 2020-01-02 03:38:17.000 596 99497 50046.5 5004650 596 99497 50046.5 5004650 -32574 32564 4826.66 482666 -124 127 1.06 106 -597 100 10587 99498 1.79279 298.79279 150.29279 15029.27927 1.79279 298.7928 150.29278 15029.27888 1.79279 298.79279 150.29279 15029.27900 2020-01-01 2020-01-02 2020-01-01 00:09:57 2020-01-02 03:38:18 2020-01-01 00:09:57.000 2020-01-02 03:38:18.000 597 99498 50047.5 5004750 597 99498 50047.5 5004750 -32573 32565 4827.66 482766 -128 127 -0.5 -50 -598 100 10588 99499 1.79579 298.79579 150.29579 15029.57957 1.79579 298.7958 150.29579 15029.57964 1.79579 298.79579 150.29579 15029.57900 2020-01-01 2020-01-02 2020-01-01 00:09:58 2020-01-02 03:38:19 2020-01-01 00:09:58.000 2020-01-02 03:38:19.000 598 99499 50048.5 5004850 598 99499 50048.5 5004850 -32572 32566 4828.66 482866 -128 123 -2.06 -206 -599 100 10589 99500 1.79879 298.79879 150.29879 15029.87987 1.79879 298.7988 150.29879 15029.87977 1.79879 298.79879 150.29879 15029.87900 2020-01-01 2020-01-02 2020-01-01 00:09:59 2020-01-02 03:38:20 2020-01-01 00:09:59.000 2020-01-02 03:38:20.000 599 99500 50049.5 5004950 599 99500 50049.5 5004950 -32571 32567 4829.66 482966 -127 124 -1.06 -106 -6 102 1005 9996 0.01801 300.01801 150.01801 15151.81981 0.01801 300.018 150.01801 15151.81978 0.01801 300.01801 150.01801 15151.81901 2020-01-01 2020-01-02 2020-01-01 00:00:06 2020-01-02 03:45:06 2020-01-01 00:00:06.000 2020-01-02 03:45:06.000 6 99906 49956 5045556 6 99906 49956 5045556 -32563 32372 4535.009900990099 458036 -127 124 -2.01980198019802 -204 -60 102 10050 99960 0.18018 300.18018 150.18018 15168.19819 0.18018 300.18018 150.18017 15168.198 0.18018 300.18018 150.18018 15168.19818 2020-01-01 2020-01-02 2020-01-01 00:01:00 2020-01-02 03:46:00 2020-01-01 00:01:00.000 2020-01-02 03:46:00.000 60 99960 50010 5051010 60 99960 50010 5051010 -32509 32426 4589.009900990099 463490 -128 127 -1.2475247524752475 -126 -600 100 10590 99501 1.8018 298.8018 150.3018 15030.18018 1.8018 298.8018 150.3018 15030.1801 1.80180 298.80180 150.30180 15030.18000 2020-01-01 2020-01-02 2020-01-01 00:10:00 2020-01-02 03:38:21 2020-01-01 00:10:00.000 2020-01-02 03:38:21.000 600 99501 50050.5 5005050 600 99501 50050.5 5005050 -32570 32568 4830.66 483066 -126 125 -0.06 -6 -601 100 10591 99502 1.8048 298.8048 150.3048 15030.48048 1.8048 298.8048 150.3048 15030.4807 1.80480 298.80480 150.30480 15030.48000 2020-01-01 2020-01-02 2020-01-01 00:10:01 2020-01-02 03:38:22 2020-01-01 00:10:01.000 2020-01-02 03:38:22.000 601 99502 50051.5 5005150 601 99502 50051.5 5005150 -32569 32569 4831.66 483166 -125 126 0.94 94 -602 100 10592 99503 1.8078 298.8078 150.3078 15030.78078 1.8078 298.8078 150.3078 15030.78035 1.80780 298.80780 150.30780 15030.78000 2020-01-01 2020-01-02 2020-01-01 00:10:02 2020-01-02 03:38:23 2020-01-01 00:10:02.000 2020-01-02 03:38:23.000 602 99503 50052.5 5005250 602 99503 50052.5 5005250 -32568 32570 4832.66 483266 -124 127 1.94 194 -603 100 10593 99504 1.81081 298.81081 150.31081 15031.08108 1.81081 298.81082 150.31081 15031.0811 1.81081 298.81081 150.31081 15031.08100 2020-01-01 2020-01-02 2020-01-01 00:10:03 2020-01-02 03:38:24 2020-01-01 00:10:03.000 2020-01-02 03:38:24.000 603 99504 50053.5 5005350 603 99504 50053.5 5005350 -32567 32571 4833.66 483366 -128 127 0.38 38 -604 100 10594 99505 1.81381 298.81381 150.31381 15031.38138 1.81381 298.8138 150.31381 15031.38124 1.81381 298.81381 150.31381 15031.38100 2020-01-01 2020-01-02 2020-01-01 00:10:04 2020-01-02 03:38:25 2020-01-01 00:10:04.000 2020-01-02 03:38:25.000 604 99505 50054.5 5005450 604 99505 50054.5 5005450 -32566 32572 4834.66 483466 -128 123 -1.18 -118 -605 100 10595 99506 1.81681 298.81681 150.31681 15031.68168 1.81681 298.8168 150.31681 15031.68157 1.81681 298.81681 150.31681 15031.68100 2020-01-01 2020-01-02 2020-01-01 00:10:05 2020-01-02 03:38:26 2020-01-01 00:10:05.000 2020-01-02 03:38:26.000 605 99506 50055.5 5005550 605 99506 50055.5 5005550 -32565 32573 4835.66 483566 -127 124 -0.18 -18 -606 100 10596 99507 1.81981 298.81981 150.31981 15031.98198 1.81981 298.81982 150.31982 15031.98217 1.81981 298.81981 150.31981 15031.98100 2020-01-01 2020-01-02 2020-01-01 00:10:06 2020-01-02 03:38:27 2020-01-01 00:10:06.000 2020-01-02 03:38:27.000 606 99507 50056.5 5005650 606 99507 50056.5 5005650 -32564 32574 4836.66 483666 -126 125 0.82 82 -607 100 10597 99508 1.82282 298.82282 150.32282 15032.28228 1.82282 298.8228 150.32282 15032.28246 1.82282 298.82282 150.32282 15032.28200 2020-01-01 2020-01-02 2020-01-01 00:10:07 2020-01-02 03:38:28 2020-01-01 00:10:07.000 2020-01-02 03:38:28.000 607 99508 50057.5 5005750 607 99508 50057.5 5005750 -32563 32575 4837.66 483766 -125 126 1.82 182 +592 100 10582 99493 1.77777 298.77777 150.27777 15027.77777 1.77777 298.77777 150.27777 15027.77742 1.77777 298.77777 150.2777700000001 15027.77700 2020-01-01 2020-01-02 2020-01-01 00:09:52 2020-01-02 03:38:13 2020-01-01 00:09:52.000 2020-01-02 03:38:13.000 592 99493 50042.5 5004250 592 99493 50042.5 5004250 -32578 32560 4822.66 482266 -128 123 -2.94 -294 +593 100 10583 99494 1.78078 298.78078 150.28078 15028.07807 1.78078 298.7808 150.28078 15028.078 1.78078 298.78078 150.28077999999974 15028.07800 2020-01-01 2020-01-02 2020-01-01 00:09:53 2020-01-02 03:38:14 2020-01-01 00:09:53.000 2020-01-02 03:38:14.000 593 99494 50043.5 5004350 593 99494 50043.5 5004350 -32577 32561 4823.66 482366 -127 124 -1.94 -194 +594 100 10584 99495 1.78378 298.78378 150.28378 15028.37837 1.78378 298.78378 150.28378 15028.3783 1.78378 298.78378 150.28377999999984 15028.37800 2020-01-01 2020-01-02 2020-01-01 00:09:54 2020-01-02 03:38:15 2020-01-01 00:09:54.000 2020-01-02 03:38:15.000 594 99495 50044.5 5004450 594 99495 50044.5 5004450 -32576 32562 4824.66 482466 -126 125 -0.94 -94 +595 100 10585 99496 1.78678 298.78678 150.28678 15028.67867 1.78678 298.78677 150.28678 15028.6786 1.78678 298.78678 150.28678000000025 15028.67800 2020-01-01 2020-01-02 2020-01-01 00:09:55 2020-01-02 03:38:16 2020-01-01 00:09:55.000 2020-01-02 03:38:16.000 595 99496 50045.5 5004550 595 99496 50045.5 5004550 -32575 32563 4825.66 482566 -125 126 0.06 6 +596 100 10586 99497 1.78978 298.78978 150.28978 15028.97897 1.78978 298.7898 150.28979 15028.97922 1.78978 298.78978 150.28977999999975 15028.97800 2020-01-01 2020-01-02 2020-01-01 00:09:56 2020-01-02 03:38:17 2020-01-01 00:09:56.000 2020-01-02 03:38:17.000 596 99497 50046.5 5004650 596 99497 50046.5 5004650 -32574 32564 4826.66 482666 -124 127 1.06 106 +597 100 10587 99498 1.79279 298.79279 150.29279 15029.27927 1.79279 298.7928 150.29278 15029.27888 1.79279 298.79279 150.29279000000002 15029.27900 2020-01-01 2020-01-02 2020-01-01 00:09:57 2020-01-02 03:38:18 2020-01-01 00:09:57.000 2020-01-02 03:38:18.000 597 99498 50047.5 5004750 597 99498 50047.5 5004750 -32573 32565 4827.66 482766 -128 127 -0.5 -50 +598 100 10588 99499 1.79579 298.79579 150.29579 15029.57957 1.79579 298.7958 150.29579 15029.57964 1.79579 298.79579 150.2957900000001 15029.57900 2020-01-01 2020-01-02 2020-01-01 00:09:58 2020-01-02 03:38:19 2020-01-01 00:09:58.000 2020-01-02 03:38:19.000 598 99499 50048.5 5004850 598 99499 50048.5 5004850 -32572 32566 4828.66 482866 -128 123 -2.06 -206 +599 100 10589 99500 1.79879 298.79879 150.29879 15029.87987 1.79879 298.7988 150.29879 15029.87977 1.79879 298.79879 150.29879000000014 15029.87900 2020-01-01 2020-01-02 2020-01-01 00:09:59 2020-01-02 03:38:20 2020-01-01 00:09:59.000 2020-01-02 03:38:20.000 599 99500 50049.5 5004950 599 99500 50049.5 5004950 -32571 32567 4829.66 482966 -127 124 -1.06 -106 +6 102 1005 9996 0.01801 300.01801 150.01801 15151.81981 0.01801 300.018 150.01801 15151.81978 0.01801 300.01801 150.0180099999998 15151.81901 2020-01-01 2020-01-02 2020-01-01 00:00:06 2020-01-02 03:45:06 2020-01-01 00:00:06.000 2020-01-02 03:45:06.000 6 99906 49956 5045556 6 99906 49956 5045556 -32563 32372 4535.009900990099 458036 -127 124 -2.01980198019802 -204 +60 102 10050 99960 0.18018 300.18018 150.18018 15168.19819 0.18018 300.18018 150.18017 15168.198 0.18018 300.18018 150.1801799999999 15168.19818 2020-01-01 2020-01-02 2020-01-01 00:01:00 2020-01-02 03:46:00 2020-01-01 00:01:00.000 2020-01-02 03:46:00.000 60 99960 50010 5051010 60 99960 50010 5051010 -32509 32426 4589.009900990099 463490 -128 127 -1.2475247524752475 -126 +600 100 10590 99501 1.8018 298.8018 150.3018 15030.18018 1.8018 298.8018 150.3018 15030.1801 1.80180 298.80180 150.30179999999987 15030.18000 2020-01-01 2020-01-02 2020-01-01 00:10:00 2020-01-02 03:38:21 2020-01-01 00:10:00.000 2020-01-02 03:38:21.000 600 99501 50050.5 5005050 600 99501 50050.5 5005050 -32570 32568 4830.66 483066 -126 125 -0.06 -6 +601 100 10591 99502 1.8048 298.8048 150.3048 15030.48048 1.8048 298.8048 150.3048 15030.4807 1.80480 298.80480 150.30479999999991 15030.48000 2020-01-01 2020-01-02 2020-01-01 00:10:01 2020-01-02 03:38:22 2020-01-01 00:10:01.000 2020-01-02 03:38:22.000 601 99502 50051.5 5005150 601 99502 50051.5 5005150 -32569 32569 4831.66 483166 -125 126 0.94 94 +602 100 10592 99503 1.8078 298.8078 150.3078 15030.78078 1.8078 298.8078 150.3078 15030.78035 1.80780 298.80780 150.3078 15030.78000 2020-01-01 2020-01-02 2020-01-01 00:10:02 2020-01-02 03:38:23 2020-01-01 00:10:02.000 2020-01-02 03:38:23.000 602 99503 50052.5 5005250 602 99503 50052.5 5005250 -32568 32570 4832.66 483266 -124 127 1.94 194 +603 100 10593 99504 1.81081 298.81081 150.31081 15031.08108 1.81081 298.81082 150.31081 15031.0811 1.81081 298.81081 150.31081000000026 15031.08100 2020-01-01 2020-01-02 2020-01-01 00:10:03 2020-01-02 03:38:24 2020-01-01 00:10:03.000 2020-01-02 03:38:24.000 603 99504 50053.5 5005350 603 99504 50053.5 5005350 -32567 32571 4833.66 483366 -128 127 0.38 38 +604 100 10594 99505 1.81381 298.81381 150.31381 15031.38138 1.81381 298.8138 150.31381 15031.38124 1.81381 298.81381 150.31380999999976 15031.38100 2020-01-01 2020-01-02 2020-01-01 00:10:04 2020-01-02 03:38:25 2020-01-01 00:10:04.000 2020-01-02 03:38:25.000 604 99505 50054.5 5005450 604 99505 50054.5 5005450 -32566 32572 4834.66 483466 -128 123 -1.18 -118 +605 100 10595 99506 1.81681 298.81681 150.31681 15031.68168 1.81681 298.8168 150.31681 15031.68157 1.81681 298.81681 150.31681000000015 15031.68100 2020-01-01 2020-01-02 2020-01-01 00:10:05 2020-01-02 03:38:26 2020-01-01 00:10:05.000 2020-01-02 03:38:26.000 605 99506 50055.5 5005550 605 99506 50055.5 5005550 -32565 32573 4835.66 483566 -127 124 -0.18 -18 +606 100 10596 99507 1.81981 298.81981 150.31981 15031.98198 1.81981 298.81982 150.31982 15031.98217 1.81981 298.81981 150.31981000000027 15031.98100 2020-01-01 2020-01-02 2020-01-01 00:10:06 2020-01-02 03:38:27 2020-01-01 00:10:06.000 2020-01-02 03:38:27.000 606 99507 50056.5 5005650 606 99507 50056.5 5005650 -32564 32574 4836.66 483666 -126 125 0.82 82 +607 100 10597 99508 1.82282 298.82282 150.32282 15032.28228 1.82282 298.8228 150.32282 15032.28246 1.82282 298.82282 150.32281999999992 15032.28200 2020-01-01 2020-01-02 2020-01-01 00:10:07 2020-01-02 03:38:28 2020-01-01 00:10:07.000 2020-01-02 03:38:28.000 607 99508 50057.5 5005750 607 99508 50057.5 5005750 -32563 32575 4837.66 483766 -125 126 1.82 182 608 100 10598 99509 1.82582 298.82582 150.32582 15032.58258 1.82582 298.82584 150.32582 15032.58258 1.82582 298.82582 150.32582 15032.58200 2020-01-01 2020-01-02 2020-01-01 00:10:08 2020-01-02 03:38:29 2020-01-01 00:10:08.000 2020-01-02 03:38:29.000 608 99509 50058.5 5005850 608 99509 50058.5 5005850 -32562 32576 4838.66 483866 -124 127 2.82 282 -609 100 10599 99510 1.82882 298.82882 150.32882 15032.88288 1.82882 298.82883 150.32882 15032.88274 1.82882 298.82882 150.32882 15032.88200 2020-01-01 2020-01-02 2020-01-01 00:10:09 2020-01-02 03:38:30 2020-01-01 00:10:09.000 2020-01-02 03:38:30.000 609 99510 50059.5 5005950 609 99510 50059.5 5005950 -32561 32577 4839.66 483966 -128 127 1.26 126 +609 100 10599 99510 1.82882 298.82882 150.32882 15032.88288 1.82882 298.82883 150.32882 15032.88274 1.82882 298.82882 150.32882000000006 15032.88200 2020-01-01 2020-01-02 2020-01-01 00:10:09 2020-01-02 03:38:30 2020-01-01 00:10:09.000 2020-01-02 03:38:30.000 609 99510 50059.5 5005950 609 99510 50059.5 5005950 -32561 32577 4839.66 483966 -128 127 1.26 126 61 102 10051 99961 0.18318 300.18318 150.18318 15168.5015 0.18318 300.1832 150.18318 15168.5016 0.18318 300.18318 150.18318 15168.50118 2020-01-01 2020-01-02 2020-01-01 00:01:01 2020-01-02 03:46:01 2020-01-01 00:01:01.000 2020-01-02 03:46:01.000 61 99961 50011 5051111 61 99961 50011 5051111 -32508 32427 4590.009900990099 463591 -128 123 -2.782178217821782 -281 -610 100 10600 99511 1.83183 298.83183 150.33183 15033.18318 1.83183 298.83182 150.33183 15033.18304 1.83183 298.83183 150.33183 15033.18300 2020-01-01 2020-01-02 2020-01-01 00:10:10 2020-01-02 03:38:31 2020-01-01 00:10:10.000 2020-01-02 03:38:31.000 610 99511 50060.5 5006050 610 99511 50060.5 5006050 -32560 32578 4840.66 484066 -128 127 -0.3 -30 -611 100 10601 99512 1.83483 298.83483 150.33483 15033.48348 1.83483 298.83484 150.33483 15033.48363 1.83483 298.83483 150.33483 15033.48300 2020-01-01 2020-01-02 2020-01-01 00:10:11 2020-01-02 03:38:32 2020-01-01 00:10:11.000 2020-01-02 03:38:32.000 611 99512 50061.5 5006150 611 99512 50061.5 5006150 -32559 32579 4841.66 484166 -128 123 -1.86 -186 -612 100 10602 99513 1.83783 298.83783 150.33783 15033.78378 1.83783 298.83783 150.33783 15033.78393 1.83783 298.83783 150.33783 15033.78300 2020-01-01 2020-01-02 2020-01-01 00:10:12 2020-01-02 03:38:33 2020-01-01 00:10:12.000 2020-01-02 03:38:33.000 612 99513 50062.5 5006250 612 99513 50062.5 5006250 -32558 32580 4842.66 484266 -127 124 -0.86 -86 -613 100 10603 99514 1.84084 298.84084 150.34084 15034.08408 1.84084 298.84085 150.34084 15034.08405 1.84084 298.84084 150.34084 15034.08400 2020-01-01 2020-01-02 2020-01-01 00:10:13 2020-01-02 03:38:34 2020-01-01 00:10:13.000 2020-01-02 03:38:34.000 613 99514 50063.5 5006350 613 99514 50063.5 5006350 -32557 32581 4843.66 484366 -126 125 0.14 14 -614 100 10604 99515 1.84384 298.84384 150.34384 15034.38438 1.84384 298.84384 150.34384 15034.38421 1.84384 298.84384 150.34384 15034.38400 2020-01-01 2020-01-02 2020-01-01 00:10:14 2020-01-02 03:38:35 2020-01-01 00:10:14.000 2020-01-02 03:38:35.000 614 99515 50064.5 5006450 614 99515 50064.5 5006450 -32556 32582 4844.66 484466 -125 126 1.14 114 -615 100 10605 99516 1.84684 298.84684 150.34684 15034.68468 1.84684 298.84683 150.34684 15034.68452 1.84684 298.84684 150.34684 15034.68400 2020-01-01 2020-01-02 2020-01-01 00:10:15 2020-01-02 03:38:36 2020-01-01 00:10:15.000 2020-01-02 03:38:36.000 615 99516 50065.5 5006550 615 99516 50065.5 5006550 -32555 32583 4845.66 484566 -124 127 2.14 214 -616 100 10606 99517 1.84984 298.84984 150.34984 15034.98498 1.84984 298.84985 150.34985 15034.98527 1.84984 298.84984 150.34984 15034.98400 2020-01-01 2020-01-02 2020-01-01 00:10:16 2020-01-02 03:38:37 2020-01-01 00:10:16.000 2020-01-02 03:38:37.000 616 99517 50066.5 5006650 616 99517 50066.5 5006650 -32554 32584 4846.66 484666 -128 127 0.58 58 +610 100 10600 99511 1.83183 298.83183 150.33183 15033.18318 1.83183 298.83182 150.33183 15033.18304 1.83183 298.83183 150.33182999999994 15033.18300 2020-01-01 2020-01-02 2020-01-01 00:10:10 2020-01-02 03:38:31 2020-01-01 00:10:10.000 2020-01-02 03:38:31.000 610 99511 50060.5 5006050 610 99511 50060.5 5006050 -32560 32578 4840.66 484066 -128 127 -0.3 -30 +611 100 10601 99512 1.83483 298.83483 150.33483 15033.48348 1.83483 298.83484 150.33483 15033.48363 1.83483 298.83483 150.3348299999999 15033.48300 2020-01-01 2020-01-02 2020-01-01 00:10:11 2020-01-02 03:38:32 2020-01-01 00:10:11.000 2020-01-02 03:38:32.000 611 99512 50061.5 5006150 611 99512 50061.5 5006150 -32559 32579 4841.66 484166 -128 123 -1.86 -186 +612 100 10602 99513 1.83783 298.83783 150.33783 15033.78378 1.83783 298.83783 150.33783 15033.78393 1.83783 298.83783 150.3378300000001 15033.78300 2020-01-01 2020-01-02 2020-01-01 00:10:12 2020-01-02 03:38:33 2020-01-01 00:10:12.000 2020-01-02 03:38:33.000 612 99513 50062.5 5006250 612 99513 50062.5 5006250 -32558 32580 4842.66 484266 -127 124 -0.86 -86 +613 100 10603 99514 1.84084 298.84084 150.34084 15034.08408 1.84084 298.84085 150.34084 15034.08405 1.84084 298.84084 150.34084000000004 15034.08400 2020-01-01 2020-01-02 2020-01-01 00:10:13 2020-01-02 03:38:34 2020-01-01 00:10:13.000 2020-01-02 03:38:34.000 613 99514 50063.5 5006350 613 99514 50063.5 5006350 -32557 32581 4843.66 484366 -126 125 0.14 14 +614 100 10604 99515 1.84384 298.84384 150.34384 15034.38438 1.84384 298.84384 150.34384 15034.38421 1.84384 298.84384 150.34383999999983 15034.38400 2020-01-01 2020-01-02 2020-01-01 00:10:14 2020-01-02 03:38:35 2020-01-01 00:10:14.000 2020-01-02 03:38:35.000 614 99515 50064.5 5006450 614 99515 50064.5 5006450 -32556 32582 4844.66 484466 -125 126 1.14 114 +615 100 10605 99516 1.84684 298.84684 150.34684 15034.68468 1.84684 298.84683 150.34684 15034.68452 1.84684 298.84684 150.34684000000007 15034.68400 2020-01-01 2020-01-02 2020-01-01 00:10:15 2020-01-02 03:38:36 2020-01-01 00:10:15.000 2020-01-02 03:38:36.000 615 99516 50065.5 5006550 615 99516 50065.5 5006550 -32555 32583 4845.66 484566 -124 127 2.14 214 +616 100 10606 99517 1.84984 298.84984 150.34984 15034.98498 1.84984 298.84985 150.34985 15034.98527 1.84984 298.84984 150.34984000000003 15034.98400 2020-01-01 2020-01-02 2020-01-01 00:10:16 2020-01-02 03:38:37 2020-01-01 00:10:16.000 2020-01-02 03:38:37.000 616 99517 50066.5 5006650 616 99517 50066.5 5006650 -32554 32584 4846.66 484666 -128 127 0.58 58 617 100 10607 99518 1.85285 298.85285 150.35285 15035.28528 1.85285 298.85284 150.35285 15035.2854 1.85285 298.85285 150.35285 15035.28500 2020-01-01 2020-01-02 2020-01-01 00:10:17 2020-01-02 03:38:38 2020-01-01 00:10:17.000 2020-01-02 03:38:38.000 617 99518 50067.5 5006750 617 99518 50067.5 5006750 -32553 32585 4847.66 484766 -128 123 -0.98 -98 -618 100 10608 99519 1.85585 298.85585 150.35585 15035.58558 1.85585 298.85587 150.35585 15035.58551 1.85585 298.85585 150.35585 15035.58500 2020-01-01 2020-01-02 2020-01-01 00:10:18 2020-01-02 03:38:39 2020-01-01 00:10:18.000 2020-01-02 03:38:39.000 618 99519 50068.5 5006850 618 99519 50068.5 5006850 -32552 32586 4848.66 484866 -127 124 0.02 2 +618 100 10608 99519 1.85585 298.85585 150.35585 15035.58558 1.85585 298.85587 150.35585 15035.58551 1.85585 298.85585 150.35584999999995 15035.58500 2020-01-01 2020-01-02 2020-01-01 00:10:18 2020-01-02 03:38:39 2020-01-01 00:10:18.000 2020-01-02 03:38:39.000 618 99519 50068.5 5006850 618 99519 50068.5 5006850 -32552 32586 4848.66 484866 -127 124 0.02 2 619 100 10609 99520 1.85885 298.85885 150.35885 15035.88588 1.85885 298.85886 150.35885 15035.88568 1.85885 298.85885 150.35885 15035.88500 2020-01-01 2020-01-02 2020-01-01 00:10:19 2020-01-02 03:38:40 2020-01-01 00:10:19.000 2020-01-02 03:38:40.000 619 99520 50069.5 5006950 619 99520 50069.5 5006950 -32551 32587 4849.66 484966 -126 125 1.02 102 -62 102 10052 99962 0.18618 300.18618 150.18618 15168.8048 0.18618 300.1862 150.18618 15168.80494 0.18618 300.18618 150.18618 15168.80418 2020-01-01 2020-01-02 2020-01-01 00:01:02 2020-01-02 03:46:02 2020-01-01 00:01:02.000 2020-01-02 03:46:02.000 62 99962 50012 5051212 62 99962 50012 5051212 -32507 32428 4591.009900990099 463692 -127 124 -1.7821782178217822 -180 -620 100 10610 99521 1.86186 298.86186 150.36186 15036.18618 1.86186 298.86185 150.36185 15036.18598 1.86186 298.86186 150.36186 15036.18600 2020-01-01 2020-01-02 2020-01-01 00:10:20 2020-01-02 03:38:41 2020-01-01 00:10:20.000 2020-01-02 03:38:41.000 620 99521 50070.5 5007050 620 99521 50070.5 5007050 -32550 32588 4850.66 485066 -125 126 2.02 202 -621 100 10611 99522 1.86486 298.86486 150.36486 15036.48648 1.86486 298.86487 150.36486 15036.48673 1.86486 298.86486 150.36486 15036.48600 2020-01-01 2020-01-02 2020-01-01 00:10:21 2020-01-02 03:38:42 2020-01-01 00:10:21.000 2020-01-02 03:38:42.000 621 99522 50071.5 5007150 621 99522 50071.5 5007150 -32549 32589 4851.66 485166 -124 127 3.02 302 -622 100 10612 99523 1.86786 298.86786 150.36786 15036.78678 1.86786 298.86786 150.36786 15036.78687 1.86786 298.86786 150.36786 15036.78600 2020-01-01 2020-01-02 2020-01-01 00:10:22 2020-01-02 03:38:43 2020-01-01 00:10:22.000 2020-01-02 03:38:43.000 622 99523 50072.5 5007250 622 99523 50072.5 5007250 -32548 32590 4852.66 485266 -128 127 1.46 146 -623 100 10613 99524 1.87087 298.87087 150.37087 15037.08708 1.87087 298.87088 150.37087 15037.08702 1.87087 298.87087 150.37087 15037.08700 2020-01-01 2020-01-02 2020-01-01 00:10:23 2020-01-02 03:38:44 2020-01-01 00:10:23.000 2020-01-02 03:38:44.000 623 99524 50073.5 5007350 623 99524 50073.5 5007350 -32547 32591 4853.66 485366 -128 123 -0.1 -10 -624 100 10614 99525 1.87387 298.87387 150.37387 15037.38738 1.87387 298.87387 150.37387 15037.38716 1.87387 298.87387 150.37387 15037.38700 2020-01-01 2020-01-02 2020-01-01 00:10:24 2020-01-02 03:38:45 2020-01-01 00:10:24.000 2020-01-02 03:38:45.000 624 99525 50074.5 5007450 624 99525 50074.5 5007450 -32546 32592 4854.66 485466 -127 124 0.9 90 -625 100 10615 99526 1.87687 298.87687 150.37687 15037.68768 1.87687 298.8769 150.37687 15037.68791 1.87687 298.87687 150.37687 15037.68700 2020-01-01 2020-01-02 2020-01-01 00:10:25 2020-01-02 03:38:46 2020-01-01 00:10:25.000 2020-01-02 03:38:46.000 625 99526 50075.5 5007550 625 99526 50075.5 5007550 -32545 32593 4855.66 485566 -126 125 1.9 190 -626 100 10616 99527 1.87987 298.87987 150.37987 15037.98798 1.87987 298.87988 150.37988 15037.9882 1.87987 298.87987 150.37987 15037.98700 2020-01-01 2020-01-02 2020-01-01 00:10:26 2020-01-02 03:38:47 2020-01-01 00:10:26.000 2020-01-02 03:38:47.000 626 99527 50076.5 5007650 626 99527 50076.5 5007650 -32544 32594 4856.66 485666 -125 126 2.9 290 -627 100 10617 99528 1.88288 298.88288 150.38288 15038.28828 1.88288 298.88287 150.38288 15038.28834 1.88288 298.88288 150.38288 15038.28800 2020-01-01 2020-01-02 2020-01-01 00:10:27 2020-01-02 03:38:48 2020-01-01 00:10:27.000 2020-01-02 03:38:48.000 627 99528 50077.5 5007750 627 99528 50077.5 5007750 -32543 32595 4857.66 485766 -124 127 3.9 390 -628 100 10618 99529 1.88588 298.88588 150.38588 15038.58858 1.88588 298.8859 150.38588 15038.58849 1.88588 298.88588 150.38588 15038.58800 2020-01-01 2020-01-02 2020-01-01 00:10:28 2020-01-02 03:38:49 2020-01-01 00:10:28.000 2020-01-02 03:38:49.000 628 99529 50078.5 5007850 628 99529 50078.5 5007850 -32542 32596 4858.66 485866 -128 127 2.34 234 -629 100 10619 99530 1.88888 298.88888 150.38888 15038.88888 1.88888 298.8889 150.38888 15038.88862 1.88888 298.88888 150.38888 15038.88800 2020-01-01 2020-01-02 2020-01-01 00:10:29 2020-01-02 03:38:50 2020-01-01 00:10:29.000 2020-01-02 03:38:50.000 629 99530 50079.5 5007950 629 99530 50079.5 5007950 -32541 32597 4859.66 485966 -128 123 0.78 78 -63 102 10053 99963 0.18918 300.18918 150.18918 15169.1081 0.18918 300.18918 150.18918 15169.10808 0.18918 300.18918 150.18918 15169.10718 2020-01-01 2020-01-02 2020-01-01 00:01:03 2020-01-02 03:46:03 2020-01-01 00:01:03.000 2020-01-02 03:46:03.000 63 99963 50013 5051313 63 99963 50013 5051313 -32506 32429 4592.009900990099 463793 -126 125 -0.7821782178217822 -79 -630 100 10620 99531 1.89189 298.89189 150.39189 15039.18918 1.89189 298.8919 150.39189 15039.18937 1.89189 298.89189 150.39189 15039.18900 2020-01-01 2020-01-02 2020-01-01 00:10:30 2020-01-02 03:38:51 2020-01-01 00:10:30.000 2020-01-02 03:38:51.000 630 99531 50080.5 5008050 630 99531 50080.5 5008050 -32540 32598 4860.66 486066 -127 124 1.78 178 -631 100 10621 99532 1.89489 298.89489 150.39489 15039.48948 1.89489 298.8949 150.39489 15039.48968 1.89489 298.89489 150.39489 15039.48900 2020-01-01 2020-01-02 2020-01-01 00:10:31 2020-01-02 03:38:52 2020-01-01 00:10:31.000 2020-01-02 03:38:52.000 631 99532 50081.5 5008150 631 99532 50081.5 5008150 -32539 32599 4861.66 486166 -126 125 2.78 278 -632 100 10622 99533 1.89789 298.89789 150.39789 15039.78978 1.89789 298.8979 150.39789 15039.78984 1.89789 298.89789 150.39789 15039.78900 2020-01-01 2020-01-02 2020-01-01 00:10:32 2020-01-02 03:38:53 2020-01-01 00:10:32.000 2020-01-02 03:38:53.000 632 99533 50082.5 5008250 632 99533 50082.5 5008250 -32538 32600 4862.66 486266 -125 126 3.78 378 -633 100 10623 99534 1.9009 298.9009 150.4009 15040.09009 1.9009 298.9009 150.40089 15040.08996 1.90090 298.90090 150.40090 15040.09000 2020-01-01 2020-01-02 2020-01-01 00:10:33 2020-01-02 03:38:54 2020-01-01 00:10:33.000 2020-01-02 03:38:54.000 633 99534 50083.5 5008350 633 99534 50083.5 5008350 -32537 32601 4863.66 486366 -124 127 4.78 478 -634 100 10624 99535 1.9039 298.9039 150.4039 15040.39039 1.9039 298.9039 150.4039 15040.39009 1.90390 298.90390 150.40390 15040.39000 2020-01-01 2020-01-02 2020-01-01 00:10:34 2020-01-02 03:38:55 2020-01-01 00:10:34.000 2020-01-02 03:38:55.000 634 99535 50084.5 5008450 634 99535 50084.5 5008450 -32536 32602 4864.66 486466 -128 127 3.22 322 -635 100 10625 99536 1.9069 298.9069 150.4069 15040.69069 1.9069 298.90692 150.4069 15040.69084 1.90690 298.90690 150.40690 15040.69000 2020-01-01 2020-01-02 2020-01-01 00:10:35 2020-01-02 03:38:56 2020-01-01 00:10:35.000 2020-01-02 03:38:56.000 635 99536 50085.5 5008550 635 99536 50085.5 5008550 -32535 32603 4865.66 486566 -128 127 1.66 166 -636 100 10626 99537 1.9099 298.9099 150.4099 15040.99099 1.90991 298.9099 150.40991 15040.99115 1.90990 298.90990 150.40990 15040.99000 2020-01-01 2020-01-02 2020-01-01 00:10:36 2020-01-02 03:38:57 2020-01-01 00:10:36.000 2020-01-02 03:38:57.000 636 99537 50086.5 5008650 636 99537 50086.5 5008650 -32534 32604 4866.66 486666 -128 124 0.1 10 -637 100 10627 99538 1.91291 298.91291 150.41291 15041.29129 1.91291 298.9129 150.41291 15041.29131 1.91291 298.91291 150.41291 15041.29100 2020-01-01 2020-01-02 2020-01-01 00:10:37 2020-01-02 03:38:58 2020-01-01 00:10:37.000 2020-01-02 03:38:58.000 637 99538 50087.5 5008750 637 99538 50087.5 5008750 -32533 32605 4867.66 486766 -127 125 1.1 110 -638 100 10628 99539 1.91591 298.91591 150.41591 15041.59159 1.91591 298.91592 150.41591 15041.59143 1.91591 298.91591 150.41591 15041.59100 2020-01-01 2020-01-02 2020-01-01 00:10:38 2020-01-02 03:38:59 2020-01-01 00:10:38.000 2020-01-02 03:38:59.000 638 99539 50088.5 5008850 638 99539 50088.5 5008850 -32532 32606 4868.66 486866 -126 126 2.1 210 -639 100 10629 99540 1.91891 298.91891 150.41891 15041.89189 1.91891 298.9189 150.41891 15041.89172 1.91891 298.91891 150.41891 15041.89100 2020-01-01 2020-01-02 2020-01-01 00:10:39 2020-01-02 03:39:00 2020-01-01 00:10:39.000 2020-01-02 03:39:00.000 639 99540 50089.5 5008950 639 99540 50089.5 5008950 -32531 32607 4869.66 486966 -125 127 3.1 310 -64 102 10054 99964 0.19219 300.19219 150.19219 15169.41141 0.19219 300.1922 150.19219 15169.41184 0.19219 300.19219 150.19219 15169.41119 2020-01-01 2020-01-02 2020-01-01 00:01:04 2020-01-02 03:46:04 2020-01-01 00:01:04.000 2020-01-02 03:46:04.000 64 99964 50014 5051414 64 99964 50014 5051414 -32505 32430 4593.009900990099 463894 -125 126 0.21782178217821782 22 +62 102 10052 99962 0.18618 300.18618 150.18618 15168.8048 0.18618 300.1862 150.18618 15168.80494 0.18618 300.18618 150.18618000000004 15168.80418 2020-01-01 2020-01-02 2020-01-01 00:01:02 2020-01-02 03:46:02 2020-01-01 00:01:02.000 2020-01-02 03:46:02.000 62 99962 50012 5051212 62 99962 50012 5051212 -32507 32428 4591.009900990099 463692 -127 124 -1.7821782178217822 -180 +620 100 10610 99521 1.86186 298.86186 150.36186 15036.18618 1.86186 298.86185 150.36185 15036.18598 1.86186 298.86186 150.3618600000001 15036.18600 2020-01-01 2020-01-02 2020-01-01 00:10:20 2020-01-02 03:38:41 2020-01-01 00:10:20.000 2020-01-02 03:38:41.000 620 99521 50070.5 5007050 620 99521 50070.5 5007050 -32550 32588 4850.66 485066 -125 126 2.02 202 +621 100 10611 99522 1.86486 298.86486 150.36486 15036.48648 1.86486 298.86487 150.36486 15036.48673 1.86486 298.86486 150.3648599999999 15036.48600 2020-01-01 2020-01-02 2020-01-01 00:10:21 2020-01-02 03:38:42 2020-01-01 00:10:21.000 2020-01-02 03:38:42.000 621 99522 50071.5 5007150 621 99522 50071.5 5007150 -32549 32589 4851.66 485166 -124 127 3.02 302 +622 100 10612 99523 1.86786 298.86786 150.36786 15036.78678 1.86786 298.86786 150.36786 15036.78687 1.86786 298.86786 150.36786000000006 15036.78600 2020-01-01 2020-01-02 2020-01-01 00:10:22 2020-01-02 03:38:43 2020-01-01 00:10:22.000 2020-01-02 03:38:43.000 622 99523 50072.5 5007250 622 99523 50072.5 5007250 -32548 32590 4852.66 485266 -128 127 1.46 146 +623 100 10613 99524 1.87087 298.87087 150.37087 15037.08708 1.87087 298.87088 150.37087 15037.08702 1.87087 298.87087 150.37087000000008 15037.08700 2020-01-01 2020-01-02 2020-01-01 00:10:23 2020-01-02 03:38:44 2020-01-01 00:10:23.000 2020-01-02 03:38:44.000 623 99524 50073.5 5007350 623 99524 50073.5 5007350 -32547 32591 4853.66 485366 -128 123 -0.1 -10 +624 100 10614 99525 1.87387 298.87387 150.37387 15037.38738 1.87387 298.87387 150.37387 15037.38716 1.87387 298.87387 150.37386999999998 15037.38700 2020-01-01 2020-01-02 2020-01-01 00:10:24 2020-01-02 03:38:45 2020-01-01 00:10:24.000 2020-01-02 03:38:45.000 624 99525 50074.5 5007450 624 99525 50074.5 5007450 -32546 32592 4854.66 485466 -127 124 0.9 90 +625 100 10615 99526 1.87687 298.87687 150.37687 15037.68768 1.87687 298.8769 150.37687 15037.68791 1.87687 298.87687 150.37687000000003 15037.68700 2020-01-01 2020-01-02 2020-01-01 00:10:25 2020-01-02 03:38:46 2020-01-01 00:10:25.000 2020-01-02 03:38:46.000 625 99526 50075.5 5007550 625 99526 50075.5 5007550 -32545 32593 4855.66 485566 -126 125 1.9 190 +626 100 10616 99527 1.87987 298.87987 150.37987 15037.98798 1.87987 298.87988 150.37988 15037.9882 1.87987 298.87987 150.37987000000004 15037.98700 2020-01-01 2020-01-02 2020-01-01 00:10:26 2020-01-02 03:38:47 2020-01-01 00:10:26.000 2020-01-02 03:38:47.000 626 99527 50076.5 5007650 626 99527 50076.5 5007650 -32544 32594 4856.66 485666 -125 126 2.9 290 +627 100 10617 99528 1.88288 298.88288 150.38288 15038.28828 1.88288 298.88287 150.38288 15038.28834 1.88288 298.88288 150.38287999999997 15038.28800 2020-01-01 2020-01-02 2020-01-01 00:10:27 2020-01-02 03:38:48 2020-01-01 00:10:27.000 2020-01-02 03:38:48.000 627 99528 50077.5 5007750 627 99528 50077.5 5007750 -32543 32595 4857.66 485766 -124 127 3.9 390 +628 100 10618 99529 1.88588 298.88588 150.38588 15038.58858 1.88588 298.8859 150.38588 15038.58849 1.88588 298.88588 150.3858799999999 15038.58800 2020-01-01 2020-01-02 2020-01-01 00:10:28 2020-01-02 03:38:49 2020-01-01 00:10:28.000 2020-01-02 03:38:49.000 628 99529 50078.5 5007850 628 99529 50078.5 5007850 -32542 32596 4858.66 485866 -128 127 2.34 234 +629 100 10619 99530 1.88888 298.88888 150.38888 15038.88888 1.88888 298.8889 150.38888 15038.88862 1.88888 298.88888 150.38888000000014 15038.88800 2020-01-01 2020-01-02 2020-01-01 00:10:29 2020-01-02 03:38:50 2020-01-01 00:10:29.000 2020-01-02 03:38:50.000 629 99530 50079.5 5007950 629 99530 50079.5 5007950 -32541 32597 4859.66 485966 -128 123 0.78 78 +63 102 10053 99963 0.18918 300.18918 150.18918 15169.1081 0.18918 300.18918 150.18918 15169.10808 0.18918 300.18918 150.18917999999996 15169.10718 2020-01-01 2020-01-02 2020-01-01 00:01:03 2020-01-02 03:46:03 2020-01-01 00:01:03.000 2020-01-02 03:46:03.000 63 99963 50013 5051313 63 99963 50013 5051313 -32506 32429 4592.009900990099 463793 -126 125 -0.7821782178217822 -79 +630 100 10620 99531 1.89189 298.89189 150.39189 15039.18918 1.89189 298.8919 150.39189 15039.18937 1.89189 298.89189 150.39189000000005 15039.18900 2020-01-01 2020-01-02 2020-01-01 00:10:30 2020-01-02 03:38:51 2020-01-01 00:10:30.000 2020-01-02 03:38:51.000 630 99531 50080.5 5008050 630 99531 50080.5 5008050 -32540 32598 4860.66 486066 -127 124 1.78 178 +631 100 10621 99532 1.89489 298.89489 150.39489 15039.48948 1.89489 298.8949 150.39489 15039.48968 1.89489 298.89489 150.39488999999986 15039.48900 2020-01-01 2020-01-02 2020-01-01 00:10:31 2020-01-02 03:38:52 2020-01-01 00:10:31.000 2020-01-02 03:38:52.000 631 99532 50081.5 5008150 631 99532 50081.5 5008150 -32539 32599 4861.66 486166 -126 125 2.78 278 +632 100 10622 99533 1.89789 298.89789 150.39789 15039.78978 1.89789 298.8979 150.39789 15039.78984 1.89789 298.89789 150.3978900000001 15039.78900 2020-01-01 2020-01-02 2020-01-01 00:10:32 2020-01-02 03:38:53 2020-01-01 00:10:32.000 2020-01-02 03:38:53.000 632 99533 50082.5 5008250 632 99533 50082.5 5008250 -32538 32600 4862.66 486266 -125 126 3.78 378 +633 100 10623 99534 1.9009 298.9009 150.4009 15040.09009 1.9009 298.9009 150.40089 15040.08996 1.90090 298.90090 150.4009 15040.09000 2020-01-01 2020-01-02 2020-01-01 00:10:33 2020-01-02 03:38:54 2020-01-01 00:10:33.000 2020-01-02 03:38:54.000 633 99534 50083.5 5008350 633 99534 50083.5 5008350 -32537 32601 4863.66 486366 -124 127 4.78 478 +634 100 10624 99535 1.9039 298.9039 150.4039 15040.39039 1.9039 298.9039 150.4039 15040.39009 1.90390 298.90390 150.40389999999996 15040.39000 2020-01-01 2020-01-02 2020-01-01 00:10:34 2020-01-02 03:38:55 2020-01-01 00:10:34.000 2020-01-02 03:38:55.000 634 99535 50084.5 5008450 634 99535 50084.5 5008450 -32536 32602 4864.66 486466 -128 127 3.22 322 +635 100 10625 99536 1.9069 298.9069 150.4069 15040.69069 1.9069 298.90692 150.4069 15040.69084 1.90690 298.90690 150.40689999999998 15040.69000 2020-01-01 2020-01-02 2020-01-01 00:10:35 2020-01-02 03:38:56 2020-01-01 00:10:35.000 2020-01-02 03:38:56.000 635 99536 50085.5 5008550 635 99536 50085.5 5008550 -32535 32603 4865.66 486566 -128 127 1.66 166 +636 100 10626 99537 1.9099 298.9099 150.4099 15040.99099 1.90991 298.9099 150.40991 15040.99115 1.90990 298.90990 150.40990000000002 15040.99000 2020-01-01 2020-01-02 2020-01-01 00:10:36 2020-01-02 03:38:57 2020-01-01 00:10:36.000 2020-01-02 03:38:57.000 636 99537 50086.5 5008650 636 99537 50086.5 5008650 -32534 32604 4866.66 486666 -128 124 0.1 10 +637 100 10627 99538 1.91291 298.91291 150.41291 15041.29129 1.91291 298.9129 150.41291 15041.29131 1.91291 298.91291 150.4129099999999 15041.29100 2020-01-01 2020-01-02 2020-01-01 00:10:37 2020-01-02 03:38:58 2020-01-01 00:10:37.000 2020-01-02 03:38:58.000 637 99538 50087.5 5008750 637 99538 50087.5 5008750 -32533 32605 4867.66 486766 -127 125 1.1 110 +638 100 10628 99539 1.91591 298.91591 150.41591 15041.59159 1.91591 298.91592 150.41591 15041.59143 1.91591 298.91591 150.41590999999994 15041.59100 2020-01-01 2020-01-02 2020-01-01 00:10:38 2020-01-02 03:38:59 2020-01-01 00:10:38.000 2020-01-02 03:38:59.000 638 99539 50088.5 5008850 638 99539 50088.5 5008850 -32532 32606 4868.66 486866 -126 126 2.1 210 +639 100 10629 99540 1.91891 298.91891 150.41891 15041.89189 1.91891 298.9189 150.41891 15041.89172 1.91891 298.91891 150.4189100000001 15041.89100 2020-01-01 2020-01-02 2020-01-01 00:10:39 2020-01-02 03:39:00 2020-01-01 00:10:39.000 2020-01-02 03:39:00.000 639 99540 50089.5 5008950 639 99540 50089.5 5008950 -32531 32607 4869.66 486966 -125 127 3.1 310 +64 102 10054 99964 0.19219 300.19219 150.19219 15169.41141 0.19219 300.1922 150.19219 15169.41184 0.19219 300.19219 150.19218999999993 15169.41119 2020-01-01 2020-01-02 2020-01-01 00:01:04 2020-01-02 03:46:04 2020-01-01 00:01:04.000 2020-01-02 03:46:04.000 64 99964 50014 5051414 64 99964 50014 5051414 -32505 32430 4593.009900990099 463894 -125 126 0.21782178217821782 22 640 100 10630 99541 1.92192 298.92192 150.42192 15042.19219 1.92192 298.92194 150.42192 15042.19232 1.92192 298.92192 150.42192 15042.19200 2020-01-01 2020-01-02 2020-01-01 00:10:40 2020-01-02 03:39:01 2020-01-01 00:10:40.000 2020-01-02 03:39:01.000 640 99541 50090.5 5009050 640 99541 50090.5 5009050 -32530 32608 4870.66 487066 -128 127 1.54 154 -641 100 10631 99542 1.92492 298.92492 150.42492 15042.49249 1.92492 298.92493 150.42492 15042.49265 1.92492 298.92492 150.42492 15042.49200 2020-01-01 2020-01-02 2020-01-01 00:10:41 2020-01-02 03:39:02 2020-01-01 00:10:41.000 2020-01-02 03:39:02.000 641 99542 50091.5 5009150 641 99542 50091.5 5009150 -32529 32609 4871.66 487166 -128 127 -0.02 -2 -642 100 10632 99543 1.92792 298.92792 150.42792 15042.79279 1.92792 298.92792 150.42792 15042.79278 1.92792 298.92792 150.42792 15042.79200 2020-01-01 2020-01-02 2020-01-01 00:10:42 2020-01-02 03:39:03 2020-01-01 00:10:42.000 2020-01-02 03:39:03.000 642 99543 50092.5 5009250 642 99543 50092.5 5009250 -32528 32610 4872.66 487266 -128 123 -1.58 -158 -643 100 10633 99544 1.93093 298.93093 150.43093 15043.09309 1.93093 298.93094 150.43092 15043.0929 1.93093 298.93093 150.43093 15043.09300 2020-01-01 2020-01-02 2020-01-01 00:10:43 2020-01-02 03:39:04 2020-01-01 00:10:43.000 2020-01-02 03:39:04.000 643 99544 50093.5 5009350 643 99544 50093.5 5009350 -32527 32611 4873.66 487366 -127 124 -0.58 -58 -644 100 10634 99545 1.93393 298.93393 150.43393 15043.39339 1.93393 298.93393 150.43393 15043.39319 1.93393 298.93393 150.43393 15043.39300 2020-01-01 2020-01-02 2020-01-01 00:10:44 2020-01-02 03:39:05 2020-01-01 00:10:44.000 2020-01-02 03:39:05.000 644 99545 50094.5 5009450 644 99545 50094.5 5009450 -32526 32612 4874.66 487466 -126 125 0.42 42 -645 100 10635 99546 1.93693 298.93693 150.43693 15043.69369 1.93693 298.93695 150.43693 15043.69379 1.93693 298.93693 150.43693 15043.69300 2020-01-01 2020-01-02 2020-01-01 00:10:45 2020-01-02 03:39:06 2020-01-01 00:10:45.000 2020-01-02 03:39:06.000 645 99546 50095.5 5009550 645 99546 50095.5 5009550 -32525 32613 4875.66 487566 -125 126 1.42 142 -646 100 10636 99547 1.93993 298.93993 150.43993 15043.99399 1.93994 298.93994 150.43994 15043.99412 1.93993 298.93993 150.43993 15043.99300 2020-01-01 2020-01-02 2020-01-01 00:10:46 2020-01-02 03:39:07 2020-01-01 00:10:46.000 2020-01-02 03:39:07.000 646 99547 50096.5 5009650 646 99547 50096.5 5009650 -32524 32614 4876.66 487666 -124 127 2.42 242 -647 100 10637 99548 1.94294 298.94294 150.44294 15044.29429 1.94294 298.94293 150.44294 15044.29425 1.94294 298.94294 150.44294 15044.29400 2020-01-01 2020-01-02 2020-01-01 00:10:47 2020-01-02 03:39:08 2020-01-01 00:10:47.000 2020-01-02 03:39:08.000 647 99548 50097.5 5009750 647 99548 50097.5 5009750 -32523 32615 4877.66 487766 -128 127 0.86 86 -648 100 10638 99549 1.94594 298.94594 150.44594 15044.59459 1.94594 298.94595 150.44595 15044.595 1.94594 298.94594 150.44594 15044.59400 2020-01-01 2020-01-02 2020-01-01 00:10:48 2020-01-02 03:39:09 2020-01-01 00:10:48.000 2020-01-02 03:39:09.000 648 99549 50098.5 5009850 648 99549 50098.5 5009850 -32522 32616 4878.66 487866 -128 123 -0.7 -70 -649 100 10639 99550 1.94894 298.94894 150.44894 15044.89489 1.94894 298.94894 150.44894 15044.89467 1.94894 298.94894 150.44894 15044.89400 2020-01-01 2020-01-02 2020-01-01 00:10:49 2020-01-02 03:39:10 2020-01-01 00:10:49.000 2020-01-02 03:39:10.000 649 99550 50099.5 5009950 649 99550 50099.5 5009950 -32521 32617 4879.66 487966 -127 124 0.3 30 -65 102 10055 99965 0.19519 300.19519 150.19519 15169.71471 0.19519 300.1952 150.19519 15169.71448 0.19519 300.19519 150.19519 15169.71419 2020-01-01 2020-01-02 2020-01-01 00:01:05 2020-01-02 03:46:05 2020-01-01 00:01:05.000 2020-01-02 03:46:05.000 65 99965 50015 5051515 65 99965 50015 5051515 -32504 32431 4594.009900990099 463995 -124 127 1.2178217821782178 123 +641 100 10631 99542 1.92492 298.92492 150.42492 15042.49249 1.92492 298.92493 150.42492 15042.49265 1.92492 298.92492 150.42492000000001 15042.49200 2020-01-01 2020-01-02 2020-01-01 00:10:41 2020-01-02 03:39:02 2020-01-01 00:10:41.000 2020-01-02 03:39:02.000 641 99542 50091.5 5009150 641 99542 50091.5 5009150 -32529 32609 4871.66 487166 -128 127 -0.02 -2 +642 100 10632 99543 1.92792 298.92792 150.42792 15042.79279 1.92792 298.92792 150.42792 15042.79278 1.92792 298.92792 150.42792000000006 15042.79200 2020-01-01 2020-01-02 2020-01-01 00:10:42 2020-01-02 03:39:03 2020-01-01 00:10:42.000 2020-01-02 03:39:03.000 642 99543 50092.5 5009250 642 99543 50092.5 5009250 -32528 32610 4872.66 487266 -128 123 -1.58 -158 +643 100 10633 99544 1.93093 298.93093 150.43093 15043.09309 1.93093 298.93094 150.43092 15043.0929 1.93093 298.93093 150.43093000000016 15043.09300 2020-01-01 2020-01-02 2020-01-01 00:10:43 2020-01-02 03:39:04 2020-01-01 00:10:43.000 2020-01-02 03:39:04.000 643 99544 50093.5 5009350 643 99544 50093.5 5009350 -32527 32611 4873.66 487366 -127 124 -0.58 -58 +644 100 10634 99545 1.93393 298.93393 150.43393 15043.39339 1.93393 298.93393 150.43393 15043.39319 1.93393 298.93393 150.43392999999995 15043.39300 2020-01-01 2020-01-02 2020-01-01 00:10:44 2020-01-02 03:39:05 2020-01-01 00:10:44.000 2020-01-02 03:39:05.000 644 99545 50094.5 5009450 644 99545 50094.5 5009450 -32526 32612 4874.66 487466 -126 125 0.42 42 +645 100 10635 99546 1.93693 298.93693 150.43693 15043.69369 1.93693 298.93695 150.43693 15043.69379 1.93693 298.93693 150.43692999999993 15043.69300 2020-01-01 2020-01-02 2020-01-01 00:10:45 2020-01-02 03:39:06 2020-01-01 00:10:45.000 2020-01-02 03:39:06.000 645 99546 50095.5 5009550 645 99546 50095.5 5009550 -32525 32613 4875.66 487566 -125 126 1.42 142 +646 100 10636 99547 1.93993 298.93993 150.43993 15043.99399 1.93994 298.93994 150.43994 15043.99412 1.93993 298.93993 150.43993000000017 15043.99300 2020-01-01 2020-01-02 2020-01-01 00:10:46 2020-01-02 03:39:07 2020-01-01 00:10:46.000 2020-01-02 03:39:07.000 646 99547 50096.5 5009650 646 99547 50096.5 5009650 -32524 32614 4876.66 487666 -124 127 2.42 242 +647 100 10637 99548 1.94294 298.94294 150.44294 15044.29429 1.94294 298.94293 150.44294 15044.29425 1.94294 298.94294 150.44294000000005 15044.29400 2020-01-01 2020-01-02 2020-01-01 00:10:47 2020-01-02 03:39:08 2020-01-01 00:10:47.000 2020-01-02 03:39:08.000 647 99548 50097.5 5009750 647 99548 50097.5 5009750 -32523 32615 4877.66 487766 -128 127 0.86 86 +648 100 10638 99549 1.94594 298.94594 150.44594 15044.59459 1.94594 298.94595 150.44595 15044.595 1.94594 298.94594 150.4459399999999 15044.59400 2020-01-01 2020-01-02 2020-01-01 00:10:48 2020-01-02 03:39:09 2020-01-01 00:10:48.000 2020-01-02 03:39:09.000 648 99549 50098.5 5009850 648 99549 50098.5 5009850 -32522 32616 4878.66 487866 -128 123 -0.7 -70 +649 100 10639 99550 1.94894 298.94894 150.44894 15044.89489 1.94894 298.94894 150.44894 15044.89467 1.94894 298.94894 150.4489400000001 15044.89400 2020-01-01 2020-01-02 2020-01-01 00:10:49 2020-01-02 03:39:10 2020-01-01 00:10:49.000 2020-01-02 03:39:10.000 649 99550 50099.5 5009950 649 99550 50099.5 5009950 -32521 32617 4879.66 487966 -127 124 0.3 30 +65 102 10055 99965 0.19519 300.19519 150.19519 15169.71471 0.19519 300.1952 150.19519 15169.71448 0.19519 300.19519 150.19519000000008 15169.71419 2020-01-01 2020-01-02 2020-01-01 00:01:05 2020-01-02 03:46:05 2020-01-01 00:01:05.000 2020-01-02 03:46:05.000 65 99965 50015 5051515 65 99965 50015 5051515 -32504 32431 4594.009900990099 463995 -124 127 1.2178217821782178 123 650 100 10640 99551 1.95195 298.95195 150.45195 15045.19519 1.95195 298.95197 150.45195 15045.19525 1.95195 298.95195 150.45195 15045.19500 2020-01-01 2020-01-02 2020-01-01 00:10:50 2020-01-02 03:39:11 2020-01-01 00:10:50.000 2020-01-02 03:39:11.000 650 99551 50100.5 5010050 650 99551 50100.5 5010050 -32520 32618 4880.66 488066 -126 125 1.3 130 -651 100 10641 99552 1.95495 298.95495 150.45495 15045.49549 1.95495 298.95496 150.45495 15045.49558 1.95495 298.95495 150.45495 15045.49500 2020-01-01 2020-01-02 2020-01-01 00:10:51 2020-01-02 03:39:12 2020-01-01 00:10:51.000 2020-01-02 03:39:12.000 651 99552 50101.5 5010150 651 99552 50101.5 5010150 -32519 32619 4881.66 488166 -125 126 2.3 230 +651 100 10641 99552 1.95495 298.95495 150.45495 15045.49549 1.95495 298.95496 150.45495 15045.49558 1.95495 298.95495 150.45494999999994 15045.49500 2020-01-01 2020-01-02 2020-01-01 00:10:51 2020-01-02 03:39:12 2020-01-01 00:10:51.000 2020-01-02 03:39:12.000 651 99552 50101.5 5010150 651 99552 50101.5 5010150 -32519 32619 4881.66 488166 -125 126 2.3 230 652 100 10642 99553 1.95795 298.95795 150.45795 15045.79579 1.95795 298.95795 150.45795 15045.79572 1.95795 298.95795 150.45795 15045.79500 2020-01-01 2020-01-02 2020-01-01 00:10:52 2020-01-02 03:39:13 2020-01-01 00:10:52.000 2020-01-02 03:39:13.000 652 99553 50102.5 5010250 652 99553 50102.5 5010250 -32518 32620 4882.66 488266 -124 127 3.3 330 -653 100 10643 99554 1.96096 298.96096 150.46096 15046.09609 1.96096 298.96097 150.46096 15046.09647 1.96096 298.96096 150.46096 15046.09600 2020-01-01 2020-01-02 2020-01-01 00:10:53 2020-01-02 03:39:14 2020-01-01 00:10:53.000 2020-01-02 03:39:14.000 653 99554 50103.5 5010350 653 99554 50103.5 5010350 -32517 32621 4883.66 488366 -128 127 1.74 174 -654 100 10644 99555 1.96396 298.96396 150.46396 15046.39639 1.96396 298.96396 150.46396 15046.39613 1.96396 298.96396 150.46396 15046.39600 2020-01-01 2020-01-02 2020-01-01 00:10:54 2020-01-02 03:39:15 2020-01-01 00:10:54.000 2020-01-02 03:39:15.000 654 99555 50104.5 5010450 654 99555 50104.5 5010450 -32516 32622 4884.66 488466 -128 123 0.18 18 -655 100 10645 99556 1.96696 298.96696 150.46696 15046.69669 1.96696 298.96698 150.46696 15046.69676 1.96696 298.96696 150.46696 15046.69600 2020-01-01 2020-01-02 2020-01-01 00:10:55 2020-01-02 03:39:16 2020-01-01 00:10:55.000 2020-01-02 03:39:16.000 655 99556 50105.5 5010550 655 99556 50105.5 5010550 -32515 32623 4885.66 488566 -127 124 1.18 118 -656 100 10646 99557 1.96996 298.96996 150.46996 15046.99699 1.96997 298.96997 150.46997 15046.99706 1.96996 298.96996 150.46996 15046.99600 2020-01-01 2020-01-02 2020-01-01 00:10:56 2020-01-02 03:39:17 2020-01-01 00:10:56.000 2020-01-02 03:39:17.000 656 99557 50106.5 5010650 656 99557 50106.5 5010650 -32514 32624 4886.66 488666 -126 125 2.18 218 -657 100 10647 99558 1.97297 298.97297 150.47297 15047.29729 1.97297 298.97296 150.47297 15047.29735 1.97297 298.97297 150.47297 15047.29700 2020-01-01 2020-01-02 2020-01-01 00:10:57 2020-01-02 03:39:18 2020-01-01 00:10:57.000 2020-01-02 03:39:18.000 657 99558 50107.5 5010750 657 99558 50107.5 5010750 -32513 32625 4887.66 488766 -125 126 3.18 318 -658 100 10648 99559 1.97597 298.97597 150.47597 15047.59759 1.97597 298.97598 150.47597 15047.59794 1.97597 298.97597 150.47597 15047.59700 2020-01-01 2020-01-02 2020-01-01 00:10:58 2020-01-02 03:39:19 2020-01-01 00:10:58.000 2020-01-02 03:39:19.000 658 99559 50108.5 5010850 658 99559 50108.5 5010850 -32512 32626 4888.66 488866 -124 127 4.18 418 -659 100 10649 99560 1.97897 298.97897 150.47897 15047.89789 1.97897 298.97897 150.47897 15047.8976 1.97897 298.97897 150.47897 15047.89700 2020-01-01 2020-01-02 2020-01-01 00:10:59 2020-01-02 03:39:20 2020-01-01 00:10:59.000 2020-01-02 03:39:20.000 659 99560 50109.5 5010950 659 99560 50109.5 5010950 -32511 32627 4889.66 488966 -128 127 2.62 262 -66 102 10056 99966 0.19819 300.19819 150.19819 15170.01801 0.19819 300.1982 150.19819 15170.01808 0.19819 300.19819 150.19819 15170.01719 2020-01-01 2020-01-02 2020-01-01 00:01:06 2020-01-02 03:46:06 2020-01-01 00:01:06.000 2020-01-02 03:46:06.000 66 99966 50016 5051616 66 99966 50016 5051616 -32503 32432 4595.009900990099 464096 -128 127 -0.31683168316831684 -32 -660 100 10650 99561 1.98198 298.98198 150.48198 15048.19819 1.98198 298.982 150.48198 15048.19822 1.98198 298.98198 150.48198 15048.19800 2020-01-01 2020-01-02 2020-01-01 00:11:00 2020-01-02 03:39:21 2020-01-01 00:11:00.000 2020-01-02 03:39:21.000 660 99561 50110.5 5011050 660 99561 50110.5 5011050 -32510 32628 4890.66 489066 -128 127 1.06 106 -661 100 10651 99562 1.98498 298.98498 150.48498 15048.49849 1.98498 298.985 150.48498 15048.49853 1.98498 298.98498 150.48498 15048.49800 2020-01-01 2020-01-02 2020-01-01 00:11:01 2020-01-02 03:39:22 2020-01-01 00:11:01.000 2020-01-02 03:39:22.000 661 99562 50111.5 5011150 661 99562 50111.5 5011150 -32509 32629 4891.66 489166 -128 124 -0.5 -50 -662 100 10652 99563 1.98798 298.98798 150.48798 15048.79879 1.98798 298.98798 150.48798 15048.79882 1.98798 298.98798 150.48798 15048.79800 2020-01-01 2020-01-02 2020-01-01 00:11:02 2020-01-02 03:39:23 2020-01-01 00:11:02.000 2020-01-02 03:39:23.000 662 99563 50112.5 5011250 662 99563 50112.5 5011250 -32508 32630 4892.66 489266 -127 125 0.5 50 -663 100 10653 99564 1.99099 298.99099 150.49099 15049.09909 1.99099 298.991 150.49099 15049.09942 1.99099 298.99099 150.49099 15049.09900 2020-01-01 2020-01-02 2020-01-01 00:11:03 2020-01-02 03:39:24 2020-01-01 00:11:03.000 2020-01-02 03:39:24.000 663 99564 50113.5 5011350 663 99564 50113.5 5011350 -32507 32631 4893.66 489366 -126 126 1.5 150 -664 100 10654 99565 1.99399 298.99399 150.49399 15049.39939 1.99399 298.994 150.49399 15049.39911 1.99399 298.99399 150.49399 15049.39900 2020-01-01 2020-01-02 2020-01-01 00:11:04 2020-01-02 03:39:25 2020-01-01 00:11:04.000 2020-01-02 03:39:25.000 664 99565 50114.5 5011450 664 99565 50114.5 5011450 -32506 32632 4894.66 489466 -125 127 2.5 250 -665 100 10655 99566 1.99699 298.99699 150.49699 15049.69969 1.99699 298.997 150.49699 15049.6997 1.99699 298.99699 150.49699 15049.69900 2020-01-01 2020-01-02 2020-01-01 00:11:05 2020-01-02 03:39:26 2020-01-01 00:11:05.000 2020-01-02 03:39:26.000 665 99566 50115.5 5011550 665 99566 50115.5 5011550 -32505 32633 4895.66 489566 -128 127 0.94 94 -666 100 10656 99567 2 299 150.5 15050 2 299 150.5 15050 2.00000 299.00000 150.50000 15050.00000 2020-01-01 2020-01-02 2020-01-01 00:11:06 2020-01-02 03:39:27 2020-01-01 00:11:06.000 2020-01-02 03:39:27.000 666 99567 50116.5 5011650 666 99567 50116.5 5011650 -32504 32634 4896.66 489666 -128 127 -0.62 -62 -667 100 10657 99568 2.003 299.003 150.503 15050.3003 2.003 299.003 150.503 15050.30029 2.00300 299.00300 150.50300 15050.30000 2020-01-01 2020-01-02 2020-01-01 00:11:07 2020-01-02 03:39:28 2020-01-01 00:11:07.000 2020-01-02 03:39:28.000 667 99568 50117.5 5011750 667 99568 50117.5 5011750 -32503 32635 4897.66 489766 -128 123 -2.18 -218 -668 100 10658 99569 2.006 299.006 150.506 15050.6006 2.006 299.006 150.506 15050.60089 2.00600 299.00600 150.50600 15050.60000 2020-01-01 2020-01-02 2020-01-01 00:11:08 2020-01-02 03:39:29 2020-01-01 00:11:08.000 2020-01-02 03:39:29.000 668 99569 50118.5 5011850 668 99569 50118.5 5011850 -32502 32636 4898.66 489866 -127 124 -1.18 -118 -669 100 10659 99570 2.009 299.009 150.509 15050.9009 2.009 299.009 150.509 15050.90057 2.00900 299.00900 150.50900 15050.90000 2020-01-01 2020-01-02 2020-01-01 00:11:09 2020-01-02 03:39:30 2020-01-01 00:11:09.000 2020-01-02 03:39:30.000 669 99570 50119.5 5011950 669 99570 50119.5 5011950 -32501 32637 4899.66 489966 -126 125 -0.18 -18 -67 102 10057 99967 0.2012 300.2012 150.2012 15170.32132 0.2012 300.2012 150.2012 15170.32142 0.20120 300.20120 150.20120 15170.32120 2020-01-01 2020-01-02 2020-01-01 00:01:07 2020-01-02 03:46:07 2020-01-01 00:01:07.000 2020-01-02 03:46:07.000 67 99967 50017 5051717 67 99967 50017 5051717 -32502 32433 4596.009900990099 464197 -128 127 -1.8514851485148516 -187 -670 100 10660 99571 2.01201 299.01201 150.51201 15051.2012 2.01201 299.01202 150.51201 15051.20117 2.01201 299.01201 150.51201 15051.20100 2020-01-01 2020-01-02 2020-01-01 00:11:10 2020-01-02 03:39:31 2020-01-01 00:11:10.000 2020-01-02 03:39:31.000 670 99571 50120.5 5012050 670 99571 50120.5 5012050 -32500 32638 4900.66 490066 -125 126 0.82 82 -671 100 10661 99572 2.01501 299.01501 150.51501 15051.5015 2.01501 299.015 150.51501 15051.50146 2.01501 299.01501 150.51501 15051.50100 2020-01-01 2020-01-02 2020-01-01 00:11:11 2020-01-02 03:39:32 2020-01-01 00:11:11.000 2020-01-02 03:39:32.000 671 99572 50121.5 5012150 671 99572 50121.5 5012150 -32499 32639 4901.66 490166 -124 127 1.82 182 -672 100 10662 99573 2.01801 299.01801 150.51801 15051.8018 2.01801 299.018 150.51801 15051.80176 2.01801 299.01801 150.51801 15051.80100 2020-01-01 2020-01-02 2020-01-01 00:11:12 2020-01-02 03:39:33 2020-01-01 00:11:12.000 2020-01-02 03:39:33.000 672 99573 50122.5 5012250 672 99573 50122.5 5012250 -32498 32640 4902.66 490266 -128 127 0.26 26 -673 100 10663 99574 2.02102 299.02102 150.52102 15052.1021 2.02102 299.02103 150.52102 15052.1024 2.02102 299.02102 150.52102 15052.10200 2020-01-01 2020-01-02 2020-01-01 00:11:13 2020-01-02 03:39:34 2020-01-01 00:11:13.000 2020-01-02 03:39:34.000 673 99574 50123.5 5012350 673 99574 50123.5 5012350 -32497 32641 4903.66 490366 -128 123 -1.3 -130 -674 100 10664 99575 2.02402 299.02402 150.52402 15052.4024 2.02402 299.02402 150.52402 15052.40204 2.02402 299.02402 150.52402 15052.40200 2020-01-01 2020-01-02 2020-01-01 00:11:14 2020-01-02 03:39:35 2020-01-01 00:11:14.000 2020-01-02 03:39:35.000 674 99575 50124.5 5012450 674 99575 50124.5 5012450 -32496 32642 4904.66 490466 -127 124 -0.3 -30 -675 100 10665 99576 2.02702 299.02702 150.52702 15052.7027 2.02702 299.02704 150.52702 15052.70264 2.02702 299.02702 150.52702 15052.70200 2020-01-01 2020-01-02 2020-01-01 00:11:15 2020-01-02 03:39:36 2020-01-01 00:11:15.000 2020-01-02 03:39:36.000 675 99576 50125.5 5012550 675 99576 50125.5 5012550 -32495 32643 4905.66 490566 -126 125 0.7 70 -676 100 10666 99577 2.03003 299.03003 150.53003 15053.003 2.03003 299.03003 150.53002 15053.00293 2.03003 299.03003 150.53003 15053.00300 2020-01-01 2020-01-02 2020-01-01 00:11:16 2020-01-02 03:39:37 2020-01-01 00:11:16.000 2020-01-02 03:39:37.000 676 99577 50126.5 5012650 676 99577 50126.5 5012650 -32494 32644 4906.66 490666 -125 126 1.7 170 +653 100 10643 99554 1.96096 298.96096 150.46096 15046.09609 1.96096 298.96097 150.46096 15046.09647 1.96096 298.96096 150.46096000000006 15046.09600 2020-01-01 2020-01-02 2020-01-01 00:10:53 2020-01-02 03:39:14 2020-01-01 00:10:53.000 2020-01-02 03:39:14.000 653 99554 50103.5 5010350 653 99554 50103.5 5010350 -32517 32621 4883.66 488366 -128 127 1.74 174 +654 100 10644 99555 1.96396 298.96396 150.46396 15046.39639 1.96396 298.96396 150.46396 15046.39613 1.96396 298.96396 150.46395999999993 15046.39600 2020-01-01 2020-01-02 2020-01-01 00:10:54 2020-01-02 03:39:15 2020-01-01 00:10:54.000 2020-01-02 03:39:15.000 654 99555 50104.5 5010450 654 99555 50104.5 5010450 -32516 32622 4884.66 488466 -128 123 0.18 18 +655 100 10645 99556 1.96696 298.96696 150.46696 15046.69669 1.96696 298.96698 150.46696 15046.69676 1.96696 298.96696 150.46695999999991 15046.69600 2020-01-01 2020-01-02 2020-01-01 00:10:55 2020-01-02 03:39:16 2020-01-01 00:10:55.000 2020-01-02 03:39:16.000 655 99556 50105.5 5010550 655 99556 50105.5 5010550 -32515 32623 4885.66 488566 -127 124 1.18 118 +656 100 10646 99557 1.96996 298.96996 150.46996 15046.99699 1.96997 298.96997 150.46997 15046.99706 1.96996 298.96996 150.46996000000007 15046.99600 2020-01-01 2020-01-02 2020-01-01 00:10:56 2020-01-02 03:39:17 2020-01-01 00:10:56.000 2020-01-02 03:39:17.000 656 99557 50106.5 5010650 656 99557 50106.5 5010650 -32514 32624 4886.66 488666 -126 125 2.18 218 +657 100 10647 99558 1.97297 298.97297 150.47297 15047.29729 1.97297 298.97296 150.47297 15047.29735 1.97297 298.97297 150.47297000000003 15047.29700 2020-01-01 2020-01-02 2020-01-01 00:10:57 2020-01-02 03:39:18 2020-01-01 00:10:57.000 2020-01-02 03:39:18.000 657 99558 50107.5 5010750 657 99558 50107.5 5010750 -32513 32625 4887.66 488766 -125 126 3.18 318 +658 100 10648 99559 1.97597 298.97597 150.47597 15047.59759 1.97597 298.97598 150.47597 15047.59794 1.97597 298.97597 150.47597000000002 15047.59700 2020-01-01 2020-01-02 2020-01-01 00:10:58 2020-01-02 03:39:19 2020-01-01 00:10:58.000 2020-01-02 03:39:19.000 658 99559 50108.5 5010850 658 99559 50108.5 5010850 -32512 32626 4888.66 488866 -124 127 4.18 418 +659 100 10649 99560 1.97897 298.97897 150.47897 15047.89789 1.97897 298.97897 150.47897 15047.8976 1.97897 298.97897 150.4789700000001 15047.89700 2020-01-01 2020-01-02 2020-01-01 00:10:59 2020-01-02 03:39:20 2020-01-01 00:10:59.000 2020-01-02 03:39:20.000 659 99560 50109.5 5010950 659 99560 50109.5 5010950 -32511 32627 4889.66 488966 -128 127 2.62 262 +66 102 10056 99966 0.19819 300.19819 150.19819 15170.01801 0.19819 300.1982 150.19819 15170.01808 0.19819 300.19819 150.19818999999993 15170.01719 2020-01-01 2020-01-02 2020-01-01 00:01:06 2020-01-02 03:46:06 2020-01-01 00:01:06.000 2020-01-02 03:46:06.000 66 99966 50016 5051616 66 99966 50016 5051616 -32503 32432 4595.009900990099 464096 -128 127 -0.31683168316831684 -32 +660 100 10650 99561 1.98198 298.98198 150.48198 15048.19819 1.98198 298.982 150.48198 15048.19822 1.98198 298.98198 150.48197999999996 15048.19800 2020-01-01 2020-01-02 2020-01-01 00:11:00 2020-01-02 03:39:21 2020-01-01 00:11:00.000 2020-01-02 03:39:21.000 660 99561 50110.5 5011050 660 99561 50110.5 5011050 -32510 32628 4890.66 489066 -128 127 1.06 106 +661 100 10651 99562 1.98498 298.98498 150.48498 15048.49849 1.98498 298.985 150.48498 15048.49853 1.98498 298.98498 150.48497999999998 15048.49800 2020-01-01 2020-01-02 2020-01-01 00:11:01 2020-01-02 03:39:22 2020-01-01 00:11:01.000 2020-01-02 03:39:22.000 661 99562 50111.5 5011150 661 99562 50111.5 5011150 -32509 32629 4891.66 489166 -128 124 -0.5 -50 +662 100 10652 99563 1.98798 298.98798 150.48798 15048.79879 1.98798 298.98798 150.48798 15048.79882 1.98798 298.98798 150.4879799999999 15048.79800 2020-01-01 2020-01-02 2020-01-01 00:11:02 2020-01-02 03:39:23 2020-01-01 00:11:02.000 2020-01-02 03:39:23.000 662 99563 50112.5 5011250 662 99563 50112.5 5011250 -32508 32630 4892.66 489266 -127 125 0.5 50 +663 100 10653 99564 1.99099 298.99099 150.49099 15049.09909 1.99099 298.991 150.49099 15049.09942 1.99099 298.99099 150.4909900000001 15049.09900 2020-01-01 2020-01-02 2020-01-01 00:11:03 2020-01-02 03:39:24 2020-01-01 00:11:03.000 2020-01-02 03:39:24.000 663 99564 50113.5 5011350 663 99564 50113.5 5011350 -32507 32631 4893.66 489366 -126 126 1.5 150 +664 100 10654 99565 1.99399 298.99399 150.49399 15049.39939 1.99399 298.994 150.49399 15049.39911 1.99399 298.99399 150.49399000000005 15049.39900 2020-01-01 2020-01-02 2020-01-01 00:11:04 2020-01-02 03:39:25 2020-01-01 00:11:04.000 2020-01-02 03:39:25.000 664 99565 50114.5 5011450 664 99565 50114.5 5011450 -32506 32632 4894.66 489466 -125 127 2.5 250 +665 100 10655 99566 1.99699 298.99699 150.49699 15049.69969 1.99699 298.997 150.49699 15049.6997 1.99699 298.99699 150.49698999999993 15049.69900 2020-01-01 2020-01-02 2020-01-01 00:11:05 2020-01-02 03:39:26 2020-01-01 00:11:05.000 2020-01-02 03:39:26.000 665 99566 50115.5 5011550 665 99566 50115.5 5011550 -32505 32633 4895.66 489566 -128 127 0.94 94 +666 100 10656 99567 2 299 150.5 15050 2 299 150.5 15050 2.00000 299.00000 150.5 15050.00000 2020-01-01 2020-01-02 2020-01-01 00:11:06 2020-01-02 03:39:27 2020-01-01 00:11:06.000 2020-01-02 03:39:27.000 666 99567 50116.5 5011650 666 99567 50116.5 5011650 -32504 32634 4896.66 489666 -128 127 -0.62 -62 +667 100 10657 99568 2.003 299.003 150.503 15050.3003 2.003 299.003 150.503 15050.30029 2.00300 299.00300 150.50300000000004 15050.30000 2020-01-01 2020-01-02 2020-01-01 00:11:07 2020-01-02 03:39:28 2020-01-01 00:11:07.000 2020-01-02 03:39:28.000 667 99568 50117.5 5011750 667 99568 50117.5 5011750 -32503 32635 4897.66 489766 -128 123 -2.18 -218 +668 100 10658 99569 2.006 299.006 150.506 15050.6006 2.006 299.006 150.506 15050.60089 2.00600 299.00600 150.50599999999997 15050.60000 2020-01-01 2020-01-02 2020-01-01 00:11:08 2020-01-02 03:39:29 2020-01-01 00:11:08.000 2020-01-02 03:39:29.000 668 99569 50118.5 5011850 668 99569 50118.5 5011850 -32502 32636 4898.66 489866 -127 124 -1.18 -118 +669 100 10659 99570 2.009 299.009 150.509 15050.9009 2.009 299.009 150.509 15050.90057 2.00900 299.00900 150.509 15050.90000 2020-01-01 2020-01-02 2020-01-01 00:11:09 2020-01-02 03:39:30 2020-01-01 00:11:09.000 2020-01-02 03:39:30.000 669 99570 50119.5 5011950 669 99570 50119.5 5011950 -32501 32637 4899.66 489966 -126 125 -0.18 -18 +67 102 10057 99967 0.2012 300.2012 150.2012 15170.32132 0.2012 300.2012 150.2012 15170.32142 0.20120 300.20120 150.20120000000003 15170.32120 2020-01-01 2020-01-02 2020-01-01 00:01:07 2020-01-02 03:46:07 2020-01-01 00:01:07.000 2020-01-02 03:46:07.000 67 99967 50017 5051717 67 99967 50017 5051717 -32502 32433 4596.009900990099 464197 -128 127 -1.8514851485148516 -187 +670 100 10660 99571 2.01201 299.01201 150.51201 15051.2012 2.01201 299.01202 150.51201 15051.20117 2.01201 299.01201 150.5120100000001 15051.20100 2020-01-01 2020-01-02 2020-01-01 00:11:10 2020-01-02 03:39:31 2020-01-01 00:11:10.000 2020-01-02 03:39:31.000 670 99571 50120.5 5012050 670 99571 50120.5 5012050 -32500 32638 4900.66 490066 -125 126 0.82 82 +671 100 10661 99572 2.01501 299.01501 150.51501 15051.5015 2.01501 299.015 150.51501 15051.50146 2.01501 299.01501 150.51500999999993 15051.50100 2020-01-01 2020-01-02 2020-01-01 00:11:11 2020-01-02 03:39:32 2020-01-01 00:11:11.000 2020-01-02 03:39:32.000 671 99572 50121.5 5012150 671 99572 50121.5 5012150 -32499 32639 4901.66 490166 -124 127 1.82 182 +672 100 10662 99573 2.01801 299.01801 150.51801 15051.8018 2.01801 299.018 150.51801 15051.80176 2.01801 299.01801 150.5180099999999 15051.80100 2020-01-01 2020-01-02 2020-01-01 00:11:12 2020-01-02 03:39:33 2020-01-01 00:11:12.000 2020-01-02 03:39:33.000 672 99573 50122.5 5012250 672 99573 50122.5 5012250 -32498 32640 4902.66 490266 -128 127 0.26 26 +673 100 10663 99574 2.02102 299.02102 150.52102 15052.1021 2.02102 299.02103 150.52102 15052.1024 2.02102 299.02102 150.52102000000005 15052.10200 2020-01-01 2020-01-02 2020-01-01 00:11:13 2020-01-02 03:39:34 2020-01-01 00:11:13.000 2020-01-02 03:39:34.000 673 99574 50123.5 5012350 673 99574 50123.5 5012350 -32497 32641 4903.66 490366 -128 123 -1.3 -130 +674 100 10664 99575 2.02402 299.02402 150.52402 15052.4024 2.02402 299.02402 150.52402 15052.40204 2.02402 299.02402 150.52402000000004 15052.40200 2020-01-01 2020-01-02 2020-01-01 00:11:14 2020-01-02 03:39:35 2020-01-01 00:11:14.000 2020-01-02 03:39:35.000 674 99575 50124.5 5012450 674 99575 50124.5 5012450 -32496 32642 4904.66 490466 -127 124 -0.3 -30 +675 100 10665 99576 2.02702 299.02702 150.52702 15052.7027 2.02702 299.02704 150.52702 15052.70264 2.02702 299.02702 150.52701999999982 15052.70200 2020-01-01 2020-01-02 2020-01-01 00:11:15 2020-01-02 03:39:36 2020-01-01 00:11:15.000 2020-01-02 03:39:36.000 675 99576 50125.5 5012550 675 99576 50125.5 5012550 -32495 32643 4905.66 490566 -126 125 0.7 70 +676 100 10666 99577 2.03003 299.03003 150.53003 15053.003 2.03003 299.03003 150.53002 15053.00293 2.03003 299.03003 150.53002999999993 15053.00300 2020-01-01 2020-01-02 2020-01-01 00:11:16 2020-01-02 03:39:37 2020-01-01 00:11:16.000 2020-01-02 03:39:37.000 676 99577 50126.5 5012650 676 99577 50126.5 5012650 -32494 32644 4906.66 490666 -125 126 1.7 170 677 100 10667 99578 2.03303 299.03303 150.53303 15053.3033 2.03303 299.03302 150.53303 15053.30323 2.03303 299.03303 150.53303 15053.30300 2020-01-01 2020-01-02 2020-01-01 00:11:17 2020-01-02 03:39:38 2020-01-01 00:11:17.000 2020-01-02 03:39:38.000 677 99578 50127.5 5012750 677 99578 50127.5 5012750 -32493 32645 4907.66 490766 -124 127 2.7 270 -678 100 10668 99579 2.03603 299.03603 150.53603 15053.6036 2.03603 299.03604 150.53603 15053.60387 2.03603 299.03603 150.53603 15053.60300 2020-01-01 2020-01-02 2020-01-01 00:11:18 2020-01-02 03:39:39 2020-01-01 00:11:18.000 2020-01-02 03:39:39.000 678 99579 50128.5 5012850 678 99579 50128.5 5012850 -32492 32646 4908.66 490866 -128 127 1.14 114 -679 100 10669 99580 2.03903 299.03903 150.53903 15053.9039 2.03903 299.03903 150.53903 15053.90351 2.03903 299.03903 150.53903 15053.90300 2020-01-01 2020-01-02 2020-01-01 00:11:19 2020-01-02 03:39:40 2020-01-01 00:11:19.000 2020-01-02 03:39:40.000 679 99580 50129.5 5012950 679 99580 50129.5 5012950 -32491 32647 4909.66 490966 -128 123 -0.42 -42 -68 102 10058 99968 0.2042 300.2042 150.2042 15170.62462 0.2042 300.2042 150.2042 15170.62457 0.20420 300.20420 150.20420 15170.62420 2020-01-01 2020-01-02 2020-01-01 00:01:08 2020-01-02 03:46:08 2020-01-01 00:01:08.000 2020-01-02 03:46:08.000 68 99968 50018 5051818 68 99968 50018 5051818 -32501 32434 4597.009900990099 464298 -128 124 -3.386138613861386 -342 -680 100 10670 99581 2.04204 299.04204 150.54204 15054.2042 2.04204 299.04205 150.54204 15054.20426 2.04204 299.04204 150.54204 15054.20400 2020-01-01 2020-01-02 2020-01-01 00:11:20 2020-01-02 03:39:41 2020-01-01 00:11:20.000 2020-01-02 03:39:41.000 680 99581 50130.5 5013050 680 99581 50130.5 5013050 -32490 32648 4910.66 491066 -127 124 0.58 58 -681 100 10671 99582 2.04504 299.04504 150.54504 15054.5045 2.04504 299.04504 150.54504 15054.5044 2.04504 299.04504 150.54504 15054.50400 2020-01-01 2020-01-02 2020-01-01 00:11:21 2020-01-02 03:39:42 2020-01-01 00:11:21.000 2020-01-02 03:39:42.000 681 99582 50131.5 5013150 681 99582 50131.5 5013150 -32489 32649 4911.66 491166 -126 125 1.58 158 -682 100 10672 99583 2.04804 299.04804 150.54804 15054.8048 2.04804 299.04803 150.54804 15054.80474 2.04804 299.04804 150.54804 15054.80400 2020-01-01 2020-01-02 2020-01-01 00:11:22 2020-01-02 03:39:43 2020-01-01 00:11:22.000 2020-01-02 03:39:43.000 682 99583 50132.5 5013250 682 99583 50132.5 5013250 -32488 32650 4912.66 491266 -125 126 2.58 258 +678 100 10668 99579 2.03603 299.03603 150.53603 15053.6036 2.03603 299.03604 150.53603 15053.60387 2.03603 299.03603 150.53602999999998 15053.60300 2020-01-01 2020-01-02 2020-01-01 00:11:18 2020-01-02 03:39:39 2020-01-01 00:11:18.000 2020-01-02 03:39:39.000 678 99579 50128.5 5012850 678 99579 50128.5 5012850 -32492 32646 4908.66 490866 -128 127 1.14 114 +679 100 10669 99580 2.03903 299.03903 150.53903 15053.9039 2.03903 299.03903 150.53903 15053.90351 2.03903 299.03903 150.53902999999994 15053.90300 2020-01-01 2020-01-02 2020-01-01 00:11:19 2020-01-02 03:39:40 2020-01-01 00:11:19.000 2020-01-02 03:39:40.000 679 99580 50129.5 5012950 679 99580 50129.5 5012950 -32491 32647 4909.66 490966 -128 123 -0.42 -42 +68 102 10058 99968 0.2042 300.2042 150.2042 15170.62462 0.2042 300.2042 150.2042 15170.62457 0.20420 300.20420 150.20420000000004 15170.62420 2020-01-01 2020-01-02 2020-01-01 00:01:08 2020-01-02 03:46:08 2020-01-01 00:01:08.000 2020-01-02 03:46:08.000 68 99968 50018 5051818 68 99968 50018 5051818 -32501 32434 4597.009900990099 464298 -128 124 -3.386138613861386 -342 +680 100 10670 99581 2.04204 299.04204 150.54204 15054.2042 2.04204 299.04205 150.54204 15054.20426 2.04204 299.04204 150.54204000000004 15054.20400 2020-01-01 2020-01-02 2020-01-01 00:11:20 2020-01-02 03:39:41 2020-01-01 00:11:20.000 2020-01-02 03:39:41.000 680 99581 50130.5 5013050 680 99581 50130.5 5013050 -32490 32648 4910.66 491066 -127 124 0.58 58 +681 100 10671 99582 2.04504 299.04504 150.54504 15054.5045 2.04504 299.04504 150.54504 15054.5044 2.04504 299.04504 150.54504000000009 15054.50400 2020-01-01 2020-01-02 2020-01-01 00:11:21 2020-01-02 03:39:42 2020-01-01 00:11:21.000 2020-01-02 03:39:42.000 681 99582 50131.5 5013150 681 99582 50131.5 5013150 -32489 32649 4911.66 491166 -126 125 1.58 158 +682 100 10672 99583 2.04804 299.04804 150.54804 15054.8048 2.04804 299.04803 150.54804 15054.80474 2.04804 299.04804 150.5480399999999 15054.80400 2020-01-01 2020-01-02 2020-01-01 00:11:22 2020-01-02 03:39:43 2020-01-01 00:11:22.000 2020-01-02 03:39:43.000 682 99583 50132.5 5013250 682 99583 50132.5 5013250 -32488 32650 4912.66 491266 -125 126 2.58 258 683 100 10673 99584 2.05105 299.05105 150.55105 15055.1051 2.05105 299.05106 150.55105 15055.10533 2.05105 299.05105 150.55105 15055.10500 2020-01-01 2020-01-02 2020-01-01 00:11:23 2020-01-02 03:39:44 2020-01-01 00:11:23.000 2020-01-02 03:39:44.000 683 99584 50133.5 5013350 683 99584 50133.5 5013350 -32487 32651 4913.66 491366 -124 127 3.58 358 -684 100 10674 99585 2.05405 299.05405 150.55405 15055.4054 2.05405 299.05405 150.55404 15055.40498 2.05405 299.05405 150.55405 15055.40500 2020-01-01 2020-01-02 2020-01-01 00:11:24 2020-01-02 03:39:45 2020-01-01 00:11:24.000 2020-01-02 03:39:45.000 684 99585 50134.5 5013450 684 99585 50134.5 5013450 -32486 32652 4914.66 491466 -128 127 2.02 202 -685 100 10675 99586 2.05705 299.05705 150.55705 15055.7057 2.05705 299.05707 150.55705 15055.70573 2.05705 299.05705 150.55705 15055.70500 2020-01-01 2020-01-02 2020-01-01 00:11:25 2020-01-02 03:39:46 2020-01-01 00:11:25.000 2020-01-02 03:39:46.000 685 99586 50135.5 5013550 685 99586 50135.5 5013550 -32485 32653 4915.66 491566 -128 127 0.46 46 -686 100 10676 99587 2.06006 299.06006 150.56006 15056.006 2.06006 299.06006 150.56005 15056.00587 2.06006 299.06006 150.56006 15056.00600 2020-01-01 2020-01-02 2020-01-01 00:11:26 2020-01-02 03:39:47 2020-01-01 00:11:26.000 2020-01-02 03:39:47.000 686 99587 50136.5 5013650 686 99587 50136.5 5013650 -32484 32654 4916.66 491666 -128 124 -1.1 -110 -687 100 10677 99588 2.06306 299.06306 150.56306 15056.3063 2.06306 299.06305 150.56306 15056.30621 2.06306 299.06306 150.56306 15056.30600 2020-01-01 2020-01-02 2020-01-01 00:11:27 2020-01-02 03:39:48 2020-01-01 00:11:27.000 2020-01-02 03:39:48.000 687 99588 50137.5 5013750 687 99588 50137.5 5013750 -32483 32655 4917.66 491766 -127 125 -0.1 -10 -688 100 10678 99589 2.06606 299.06606 150.56606 15056.6066 2.06606 299.06607 150.56606 15056.60681 2.06606 299.06606 150.56606 15056.60600 2020-01-01 2020-01-02 2020-01-01 00:11:28 2020-01-02 03:39:49 2020-01-01 00:11:28.000 2020-01-02 03:39:49.000 688 99589 50138.5 5013850 688 99589 50138.5 5013850 -32482 32656 4918.66 491866 -126 126 0.9 90 -689 100 10679 99590 2.06906 299.06906 150.56906 15056.9069 2.06906 299.06906 150.56907 15056.9071 2.06906 299.06906 150.56906 15056.90600 2020-01-01 2020-01-02 2020-01-01 00:11:29 2020-01-02 03:39:50 2020-01-01 00:11:29.000 2020-01-02 03:39:50.000 689 99590 50139.5 5013950 689 99590 50139.5 5013950 -32481 32657 4919.66 491966 -125 127 1.9 190 -69 102 10059 99969 0.2072 300.2072 150.2072 15170.92792 0.2072 300.2072 150.20721 15170.92832 0.20720 300.20720 150.20720 15170.92720 2020-01-01 2020-01-02 2020-01-01 00:01:09 2020-01-02 03:46:09 2020-01-01 00:01:09.000 2020-01-02 03:46:09.000 69 99969 50019 5051919 69 99969 50019 5051919 -32500 32435 4598.009900990099 464399 -127 125 -2.386138613861386 -241 -690 100 10680 99591 2.07207 299.07207 150.57207 15057.2072 2.07207 299.07208 150.57207 15057.2072 2.07207 299.07207 150.57207 15057.20700 2020-01-01 2020-01-02 2020-01-01 00:11:30 2020-01-02 03:39:51 2020-01-01 00:11:30.000 2020-01-02 03:39:51.000 690 99591 50140.5 5014050 690 99591 50140.5 5014050 -32480 32658 4920.66 492066 -128 127 0.34 34 -691 100 10681 99592 2.07507 299.07507 150.57507 15057.5075 2.07507 299.07507 150.57507 15057.50734 2.07507 299.07507 150.57507 15057.50700 2020-01-01 2020-01-02 2020-01-01 00:11:31 2020-01-02 03:39:52 2020-01-01 00:11:31.000 2020-01-02 03:39:52.000 691 99592 50141.5 5014150 691 99592 50141.5 5014150 -32479 32659 4921.66 492166 -128 127 -1.22 -122 -692 100 10682 99593 2.07807 299.07807 150.57807 15057.8078 2.07807 299.07806 150.57807 15057.80767 2.07807 299.07807 150.57807 15057.80700 2020-01-01 2020-01-02 2020-01-01 00:11:32 2020-01-02 03:39:53 2020-01-01 00:11:32.000 2020-01-02 03:39:53.000 692 99593 50142.5 5014250 692 99593 50142.5 5014250 -32478 32660 4922.66 492266 -128 123 -2.78 -278 -693 100 10683 99594 2.08108 299.08108 150.58108 15058.1081 2.08108 299.0811 150.58108 15058.10827 2.08108 299.08108 150.58108 15058.10800 2020-01-01 2020-01-02 2020-01-01 00:11:33 2020-01-02 03:39:54 2020-01-01 00:11:33.000 2020-01-02 03:39:54.000 693 99594 50143.5 5014350 693 99594 50143.5 5014350 -32477 32661 4923.66 492366 -127 124 -1.78 -178 +684 100 10674 99585 2.05405 299.05405 150.55405 15055.4054 2.05405 299.05405 150.55404 15055.40498 2.05405 299.05405 150.55405000000007 15055.40500 2020-01-01 2020-01-02 2020-01-01 00:11:24 2020-01-02 03:39:45 2020-01-01 00:11:24.000 2020-01-02 03:39:45.000 684 99585 50134.5 5013450 684 99585 50134.5 5013450 -32486 32652 4914.66 491466 -128 127 2.02 202 +685 100 10675 99586 2.05705 299.05705 150.55705 15055.7057 2.05705 299.05707 150.55705 15055.70573 2.05705 299.05705 150.55704999999998 15055.70500 2020-01-01 2020-01-02 2020-01-01 00:11:25 2020-01-02 03:39:46 2020-01-01 00:11:25.000 2020-01-02 03:39:46.000 685 99586 50135.5 5013550 685 99586 50135.5 5013550 -32485 32653 4915.66 491566 -128 127 0.46 46 +686 100 10676 99587 2.06006 299.06006 150.56006 15056.006 2.06006 299.06006 150.56005 15056.00587 2.06006 299.06006 150.5600599999999 15056.00600 2020-01-01 2020-01-02 2020-01-01 00:11:26 2020-01-02 03:39:47 2020-01-01 00:11:26.000 2020-01-02 03:39:47.000 686 99587 50136.5 5013650 686 99587 50136.5 5013650 -32484 32654 4916.66 491666 -128 124 -1.1 -110 +687 100 10677 99588 2.06306 299.06306 150.56306 15056.3063 2.06306 299.06305 150.56306 15056.30621 2.06306 299.06306 150.56306000000012 15056.30600 2020-01-01 2020-01-02 2020-01-01 00:11:27 2020-01-02 03:39:48 2020-01-01 00:11:27.000 2020-01-02 03:39:48.000 687 99588 50137.5 5013750 687 99588 50137.5 5013750 -32483 32655 4917.66 491766 -127 125 -0.1 -10 +688 100 10678 99589 2.06606 299.06606 150.56606 15056.6066 2.06606 299.06607 150.56606 15056.60681 2.06606 299.06606 150.56605999999996 15056.60600 2020-01-01 2020-01-02 2020-01-01 00:11:28 2020-01-02 03:39:49 2020-01-01 00:11:28.000 2020-01-02 03:39:49.000 688 99589 50138.5 5013850 688 99589 50138.5 5013850 -32482 32656 4918.66 491866 -126 126 0.9 90 +689 100 10679 99590 2.06906 299.06906 150.56906 15056.9069 2.06906 299.06906 150.56907 15056.9071 2.06906 299.06906 150.5690599999999 15056.90600 2020-01-01 2020-01-02 2020-01-01 00:11:29 2020-01-02 03:39:50 2020-01-01 00:11:29.000 2020-01-02 03:39:50.000 689 99590 50139.5 5013950 689 99590 50139.5 5013950 -32481 32657 4919.66 491966 -125 127 1.9 190 +69 102 10059 99969 0.2072 300.2072 150.2072 15170.92792 0.2072 300.2072 150.20721 15170.92832 0.20720 300.20720 150.20720000000003 15170.92720 2020-01-01 2020-01-02 2020-01-01 00:01:09 2020-01-02 03:46:09 2020-01-01 00:01:09.000 2020-01-02 03:46:09.000 69 99969 50019 5051919 69 99969 50019 5051919 -32500 32435 4598.009900990099 464399 -127 125 -2.386138613861386 -241 +690 100 10680 99591 2.07207 299.07207 150.57207 15057.2072 2.07207 299.07208 150.57207 15057.2072 2.07207 299.07207 150.57207000000008 15057.20700 2020-01-01 2020-01-02 2020-01-01 00:11:30 2020-01-02 03:39:51 2020-01-01 00:11:30.000 2020-01-02 03:39:51.000 690 99591 50140.5 5014050 690 99591 50140.5 5014050 -32480 32658 4920.66 492066 -128 127 0.34 34 +691 100 10681 99592 2.07507 299.07507 150.57507 15057.5075 2.07507 299.07507 150.57507 15057.50734 2.07507 299.07507 150.57507000000004 15057.50700 2020-01-01 2020-01-02 2020-01-01 00:11:31 2020-01-02 03:39:52 2020-01-01 00:11:31.000 2020-01-02 03:39:52.000 691 99592 50141.5 5014150 691 99592 50141.5 5014150 -32479 32659 4921.66 492166 -128 127 -1.22 -122 +692 100 10682 99593 2.07807 299.07807 150.57807 15057.8078 2.07807 299.07806 150.57807 15057.80767 2.07807 299.07807 150.57806999999985 15057.80700 2020-01-01 2020-01-02 2020-01-01 00:11:32 2020-01-02 03:39:53 2020-01-01 00:11:32.000 2020-01-02 03:39:53.000 692 99593 50142.5 5014250 692 99593 50142.5 5014250 -32478 32660 4922.66 492266 -128 123 -2.78 -278 +693 100 10683 99594 2.08108 299.08108 150.58108 15058.1081 2.08108 299.0811 150.58108 15058.10827 2.08108 299.08108 150.58107999999996 15058.10800 2020-01-01 2020-01-02 2020-01-01 00:11:33 2020-01-02 03:39:54 2020-01-01 00:11:33.000 2020-01-02 03:39:54.000 693 99594 50143.5 5014350 693 99594 50143.5 5014350 -32477 32661 4923.66 492366 -127 124 -1.78 -178 694 100 10684 99595 2.08408 299.08408 150.58408 15058.4084 2.08408 299.08408 150.58408 15058.40857 2.08408 299.08408 150.58408 15058.40800 2020-01-01 2020-01-02 2020-01-01 00:11:34 2020-01-02 03:39:55 2020-01-01 00:11:34.000 2020-01-02 03:39:55.000 694 99595 50144.5 5014450 694 99595 50144.5 5014450 -32476 32662 4924.66 492466 -126 125 -0.78 -78 -695 100 10685 99596 2.08708 299.08708 150.58708 15058.7087 2.08708 299.0871 150.58708 15058.70867 2.08708 299.08708 150.58708 15058.70800 2020-01-01 2020-01-02 2020-01-01 00:11:35 2020-01-02 03:39:56 2020-01-01 00:11:35.000 2020-01-02 03:39:56.000 695 99596 50145.5 5014550 695 99596 50145.5 5014550 -32475 32663 4925.66 492566 -125 126 0.22 22 -696 100 10686 99597 2.09009 299.09009 150.59009 15059.009 2.09009 299.0901 150.59008 15059.00885 2.09009 299.09009 150.59009 15059.00900 2020-01-01 2020-01-02 2020-01-01 00:11:36 2020-01-02 03:39:57 2020-01-01 00:11:36.000 2020-01-02 03:39:57.000 696 99597 50146.5 5014650 696 99597 50146.5 5014650 -32474 32664 4926.66 492666 -124 127 1.22 122 -697 100 10687 99598 2.09309 299.09309 150.59309 15059.3093 2.09309 299.09308 150.59309 15059.30915 2.09309 299.09309 150.59309 15059.30900 2020-01-01 2020-01-02 2020-01-01 00:11:37 2020-01-02 03:39:58 2020-01-01 00:11:37.000 2020-01-02 03:39:58.000 697 99598 50147.5 5014750 697 99598 50147.5 5014750 -32473 32665 4927.66 492766 -128 127 -0.34 -34 -698 100 10688 99599 2.09609 299.09609 150.59609 15059.6096 2.09609 299.0961 150.59609 15059.6099 2.09609 299.09609 150.59609 15059.60900 2020-01-01 2020-01-02 2020-01-01 00:11:38 2020-01-02 03:39:59 2020-01-01 00:11:38.000 2020-01-02 03:39:59.000 698 99599 50148.5 5014850 698 99599 50148.5 5014850 -32472 32666 4928.66 492866 -128 123 -1.9 -190 -699 100 10689 99600 2.09909 299.09909 150.59909 15059.9099 2.09909 299.0991 150.5991 15059.91003 2.09909 299.09909 150.59909 15059.90900 2020-01-01 2020-01-02 2020-01-01 00:11:39 2020-01-02 03:40:00 2020-01-01 00:11:39.000 2020-01-02 03:40:00.000 699 99600 50149.5 5014950 699 99600 50149.5 5014950 -32471 32667 4929.66 492966 -127 124 -0.9 -90 -7 102 1006 9997 0.02102 300.02102 150.02102 15152.12312 0.02102 300.02103 150.02102 15152.12342 0.02102 300.02102 150.02102 15152.12302 2020-01-01 2020-01-02 2020-01-01 00:00:07 2020-01-02 03:45:07 2020-01-01 00:00:07.000 2020-01-02 03:45:07.000 7 99907 49957 5045657 7 99907 49957 5045657 -32562 32373 4536.009900990099 458137 -126 125 -1.0198019801980198 -103 -70 102 10060 99970 0.21021 300.21021 150.21021 15171.23123 0.21021 300.2102 150.2102 15171.23097 0.21021 300.21021 150.21021 15171.23121 2020-01-01 2020-01-02 2020-01-01 00:01:10 2020-01-02 03:46:10 2020-01-01 00:01:10.000 2020-01-02 03:46:10.000 70 99970 50020 5052020 70 99970 50020 5052020 -32499 32436 4599.009900990099 464500 -126 126 -1.386138613861386 -140 -700 100 10690 99601 2.1021 299.1021 150.6021 15060.21021 2.1021 299.1021 150.6021 15060.21014 2.10210 299.10210 150.60210 15060.21000 2020-01-01 2020-01-02 2020-01-01 00:11:40 2020-01-02 03:40:01 2020-01-01 00:11:40.000 2020-01-02 03:40:01.000 700 99601 50150.5 5015050 700 99601 50150.5 5015050 -32470 32668 4930.66 493066 -126 125 0.1 10 -701 100 10691 99602 2.1051 299.1051 150.6051 15060.51051 2.1051 299.1051 150.6051 15060.51031 2.10510 299.10510 150.60510 15060.51000 2020-01-01 2020-01-02 2020-01-01 00:11:41 2020-01-02 03:40:02 2020-01-01 00:11:41.000 2020-01-02 03:40:02.000 701 99602 50151.5 5015150 701 99602 50151.5 5015150 -32469 32669 4931.66 493166 -125 126 1.1 110 -702 100 10692 99603 2.1081 299.1081 150.6081 15060.81081 2.1081 299.1081 150.6081 15060.81062 2.10810 299.10810 150.60810 15060.81000 2020-01-01 2020-01-02 2020-01-01 00:11:42 2020-01-02 03:40:03 2020-01-01 00:11:42.000 2020-01-02 03:40:03.000 702 99603 50152.5 5015250 702 99603 50152.5 5015250 -32468 32670 4932.66 493266 -124 127 2.1 210 -703 100 10693 99604 2.11111 299.11111 150.61111 15061.11111 2.11111 299.1111 150.61111 15061.11137 2.11111 299.11111 150.61111 15061.11100 2020-01-01 2020-01-02 2020-01-01 00:11:43 2020-01-02 03:40:04 2020-01-01 00:11:43.000 2020-01-02 03:40:04.000 703 99604 50153.5 5015350 703 99604 50153.5 5015350 -32467 32671 4933.66 493366 -128 127 0.54 54 -704 100 10694 99605 2.11411 299.11411 150.61411 15061.41141 2.11411 299.1141 150.61411 15061.41151 2.11411 299.11411 150.61411 15061.41100 2020-01-01 2020-01-02 2020-01-01 00:11:44 2020-01-02 03:40:05 2020-01-01 00:11:44.000 2020-01-02 03:40:05.000 704 99605 50154.5 5015450 704 99605 50154.5 5015450 -32466 32672 4934.66 493466 -128 123 -1.02 -102 -705 100 10695 99606 2.11711 299.11711 150.61711 15061.71171 2.11711 299.11713 150.61711 15061.71165 2.11711 299.11711 150.61711 15061.71100 2020-01-01 2020-01-02 2020-01-01 00:11:45 2020-01-02 03:40:06 2020-01-01 00:11:45.000 2020-01-02 03:40:06.000 705 99606 50155.5 5015550 705 99606 50155.5 5015550 -32465 32673 4935.66 493566 -127 124 -0.02 -2 -706 100 10696 99607 2.12012 299.12012 150.62012 15062.01201 2.12012 299.12012 150.62011 15062.01179 2.12012 299.12012 150.62012 15062.01200 2020-01-01 2020-01-02 2020-01-01 00:11:46 2020-01-02 03:40:07 2020-01-01 00:11:46.000 2020-01-02 03:40:07.000 706 99607 50156.5 5015650 706 99607 50156.5 5015650 -32464 32674 4936.66 493666 -126 125 0.98 98 -707 100 10697 99608 2.12312 299.12312 150.62312 15062.31231 2.12312 299.1231 150.62312 15062.31208 2.12312 299.12312 150.62312 15062.31200 2020-01-01 2020-01-02 2020-01-01 00:11:47 2020-01-02 03:40:08 2020-01-01 00:11:47.000 2020-01-02 03:40:08.000 707 99608 50157.5 5015750 707 99608 50157.5 5015750 -32463 32675 4937.66 493766 -125 126 1.98 198 -708 100 10698 99609 2.12612 299.12612 150.62612 15062.61261 2.12612 299.12613 150.62612 15062.61283 2.12612 299.12612 150.62612 15062.61200 2020-01-01 2020-01-02 2020-01-01 00:11:48 2020-01-02 03:40:09 2020-01-01 00:11:48.000 2020-01-02 03:40:09.000 708 99609 50158.5 5015850 708 99609 50158.5 5015850 -32462 32676 4938.66 493866 -124 127 2.98 298 -709 100 10699 99610 2.12912 299.12912 150.62912 15062.91291 2.12912 299.12912 150.62912 15062.91298 2.12912 299.12912 150.62912 15062.91200 2020-01-01 2020-01-02 2020-01-01 00:11:49 2020-01-02 03:40:10 2020-01-01 00:11:49.000 2020-01-02 03:40:10.000 709 99610 50159.5 5015950 709 99610 50159.5 5015950 -32461 32677 4939.66 493966 -128 127 1.42 142 -71 102 10061 99971 0.21321 300.21321 150.21321 15171.53453 0.21321 300.21323 150.21321 15171.5346 0.21321 300.21321 150.21321 15171.53421 2020-01-01 2020-01-02 2020-01-01 00:01:11 2020-01-02 03:46:11 2020-01-01 00:01:11.000 2020-01-02 03:46:11.000 71 99971 50021 5052121 71 99971 50021 5052121 -32498 32437 4600.009900990099 464601 -125 127 -0.38613861386138615 -39 +695 100 10685 99596 2.08708 299.08708 150.58708 15058.7087 2.08708 299.0871 150.58708 15058.70867 2.08708 299.08708 150.58707999999987 15058.70800 2020-01-01 2020-01-02 2020-01-01 00:11:35 2020-01-02 03:39:56 2020-01-01 00:11:35.000 2020-01-02 03:39:56.000 695 99596 50145.5 5014550 695 99596 50145.5 5014550 -32475 32663 4925.66 492566 -125 126 0.22 22 +696 100 10686 99597 2.09009 299.09009 150.59009 15059.009 2.09009 299.0901 150.59008 15059.00885 2.09009 299.09009 150.5900899999998 15059.00900 2020-01-01 2020-01-02 2020-01-01 00:11:36 2020-01-02 03:39:57 2020-01-01 00:11:36.000 2020-01-02 03:39:57.000 696 99597 50146.5 5014650 696 99597 50146.5 5014650 -32474 32664 4926.66 492666 -124 127 1.22 122 +697 100 10687 99598 2.09309 299.09309 150.59309 15059.3093 2.09309 299.09308 150.59309 15059.30915 2.09309 299.09309 150.5930900000002 15059.30900 2020-01-01 2020-01-02 2020-01-01 00:11:37 2020-01-02 03:39:58 2020-01-01 00:11:37.000 2020-01-02 03:39:58.000 697 99598 50147.5 5014750 697 99598 50147.5 5014750 -32473 32665 4927.66 492766 -128 127 -0.34 -34 +698 100 10688 99599 2.09609 299.09609 150.59609 15059.6096 2.09609 299.0961 150.59609 15059.6099 2.09609 299.09609 150.59608999999972 15059.60900 2020-01-01 2020-01-02 2020-01-01 00:11:38 2020-01-02 03:39:59 2020-01-01 00:11:38.000 2020-01-02 03:39:59.000 698 99599 50148.5 5014850 698 99599 50148.5 5014850 -32472 32666 4928.66 492866 -128 123 -1.9 -190 +699 100 10689 99600 2.09909 299.09909 150.59909 15059.9099 2.09909 299.0991 150.5991 15059.91003 2.09909 299.09909 150.59908999999982 15059.90900 2020-01-01 2020-01-02 2020-01-01 00:11:39 2020-01-02 03:40:00 2020-01-01 00:11:39.000 2020-01-02 03:40:00.000 699 99600 50149.5 5014950 699 99600 50149.5 5014950 -32471 32667 4929.66 492966 -127 124 -0.9 -90 +7 102 1006 9997 0.02102 300.02102 150.02102 15152.12312 0.02102 300.02103 150.02102 15152.12342 0.02102 300.02102 150.0210200000001 15152.12302 2020-01-01 2020-01-02 2020-01-01 00:00:07 2020-01-02 03:45:07 2020-01-01 00:00:07.000 2020-01-02 03:45:07.000 7 99907 49957 5045657 7 99907 49957 5045657 -32562 32373 4536.009900990099 458137 -126 125 -1.0198019801980198 -103 +70 102 10060 99970 0.21021 300.21021 150.21021 15171.23123 0.21021 300.2102 150.2102 15171.23097 0.21021 300.21021 150.21020999999988 15171.23121 2020-01-01 2020-01-02 2020-01-01 00:01:10 2020-01-02 03:46:10 2020-01-01 00:01:10.000 2020-01-02 03:46:10.000 70 99970 50020 5052020 70 99970 50020 5052020 -32499 32436 4599.009900990099 464500 -126 126 -1.386138613861386 -140 +700 100 10690 99601 2.1021 299.1021 150.6021 15060.21021 2.1021 299.1021 150.6021 15060.21014 2.10210 299.10210 150.60210000000004 15060.21000 2020-01-01 2020-01-02 2020-01-01 00:11:40 2020-01-02 03:40:01 2020-01-01 00:11:40.000 2020-01-02 03:40:01.000 700 99601 50150.5 5015050 700 99601 50150.5 5015050 -32470 32668 4930.66 493066 -126 125 0.1 10 +701 100 10691 99602 2.1051 299.1051 150.6051 15060.51051 2.1051 299.1051 150.6051 15060.51031 2.10510 299.10510 150.60510000000014 15060.51000 2020-01-01 2020-01-02 2020-01-01 00:11:41 2020-01-02 03:40:02 2020-01-01 00:11:41.000 2020-01-02 03:40:02.000 701 99602 50151.5 5015150 701 99602 50151.5 5015150 -32469 32669 4931.66 493166 -125 126 1.1 110 +702 100 10692 99603 2.1081 299.1081 150.6081 15060.81081 2.1081 299.1081 150.6081 15060.81062 2.10810 299.10810 150.60809999999998 15060.81000 2020-01-01 2020-01-02 2020-01-01 00:11:42 2020-01-02 03:40:03 2020-01-01 00:11:42.000 2020-01-02 03:40:03.000 702 99603 50152.5 5015250 702 99603 50152.5 5015250 -32468 32670 4932.66 493266 -124 127 2.1 210 +703 100 10693 99604 2.11111 299.11111 150.61111 15061.11111 2.11111 299.1111 150.61111 15061.11137 2.11111 299.11111 150.61110999999985 15061.11100 2020-01-01 2020-01-02 2020-01-01 00:11:43 2020-01-02 03:40:04 2020-01-01 00:11:43.000 2020-01-02 03:40:04.000 703 99604 50153.5 5015350 703 99604 50153.5 5015350 -32467 32671 4933.66 493366 -128 127 0.54 54 +704 100 10694 99605 2.11411 299.11411 150.61411 15061.41141 2.11411 299.1141 150.61411 15061.41151 2.11411 299.11411 150.61411000000032 15061.41100 2020-01-01 2020-01-02 2020-01-01 00:11:44 2020-01-02 03:40:05 2020-01-01 00:11:44.000 2020-01-02 03:40:05.000 704 99605 50154.5 5015450 704 99605 50154.5 5015450 -32466 32672 4934.66 493466 -128 123 -1.02 -102 +705 100 10695 99606 2.11711 299.11711 150.61711 15061.71171 2.11711 299.11713 150.61711 15061.71165 2.11711 299.11711 150.61710999999983 15061.71100 2020-01-01 2020-01-02 2020-01-01 00:11:45 2020-01-02 03:40:06 2020-01-01 00:11:45.000 2020-01-02 03:40:06.000 705 99606 50155.5 5015550 705 99606 50155.5 5015550 -32465 32673 4935.66 493566 -127 124 -0.02 -2 +706 100 10696 99607 2.12012 299.12012 150.62012 15062.01201 2.12012 299.12012 150.62011 15062.01179 2.12012 299.12012 150.6201199999997 15062.01200 2020-01-01 2020-01-02 2020-01-01 00:11:46 2020-01-02 03:40:07 2020-01-01 00:11:46.000 2020-01-02 03:40:07.000 706 99607 50156.5 5015650 706 99607 50156.5 5015650 -32464 32674 4936.66 493666 -126 125 0.98 98 +707 100 10697 99608 2.12312 299.12312 150.62312 15062.31231 2.12312 299.1231 150.62312 15062.31208 2.12312 299.12312 150.62312000000017 15062.31200 2020-01-01 2020-01-02 2020-01-01 00:11:47 2020-01-02 03:40:08 2020-01-01 00:11:47.000 2020-01-02 03:40:08.000 707 99608 50157.5 5015750 707 99608 50157.5 5015750 -32463 32675 4937.66 493766 -125 126 1.98 198 +708 100 10698 99609 2.12612 299.12612 150.62612 15062.61261 2.12612 299.12613 150.62612 15062.61283 2.12612 299.12612 150.6261200000002 15062.61200 2020-01-01 2020-01-02 2020-01-01 00:11:48 2020-01-02 03:40:09 2020-01-01 00:11:48.000 2020-01-02 03:40:09.000 708 99609 50158.5 5015850 708 99609 50158.5 5015850 -32462 32676 4938.66 493866 -124 127 2.98 298 +709 100 10699 99610 2.12912 299.12912 150.62912 15062.91291 2.12912 299.12912 150.62912 15062.91298 2.12912 299.12912 150.62911999999972 15062.91200 2020-01-01 2020-01-02 2020-01-01 00:11:49 2020-01-02 03:40:10 2020-01-01 00:11:49.000 2020-01-02 03:40:10.000 709 99610 50159.5 5015950 709 99610 50159.5 5015950 -32461 32677 4939.66 493966 -128 127 1.42 142 +71 102 10061 99971 0.21321 300.21321 150.21321 15171.53453 0.21321 300.21323 150.21321 15171.5346 0.21321 300.21321 150.21320999999992 15171.53421 2020-01-01 2020-01-02 2020-01-01 00:01:11 2020-01-02 03:46:11 2020-01-01 00:01:11.000 2020-01-02 03:46:11.000 71 99971 50021 5052121 71 99971 50021 5052121 -32498 32437 4600.009900990099 464601 -125 127 -0.38613861386138615 -39 710 100 10700 99611 2.13213 299.13213 150.63213 15063.21321 2.13213 299.13214 150.63213 15063.21311 2.13213 299.13213 150.63213 15063.21300 2020-01-01 2020-01-02 2020-01-01 00:11:50 2020-01-02 03:40:11 2020-01-01 00:11:50.000 2020-01-02 03:40:11.000 710 99611 50160.5 5016050 710 99611 50160.5 5016050 -32460 32678 4940.66 494066 -128 127 -0.14 -14 -711 100 10701 99612 2.13513 299.13513 150.63513 15063.51351 2.13513 299.13513 150.63513 15063.51325 2.13513 299.13513 150.63513 15063.51300 2020-01-01 2020-01-02 2020-01-01 00:11:51 2020-01-02 03:40:12 2020-01-01 00:11:51.000 2020-01-02 03:40:12.000 711 99612 50161.5 5016150 711 99612 50161.5 5016150 -32459 32679 4941.66 494166 -128 124 -1.7 -170 -712 100 10702 99613 2.13813 299.13813 150.63813 15063.81381 2.13813 299.13815 150.63814 15063.81401 2.13813 299.13813 150.63813 15063.81300 2020-01-01 2020-01-02 2020-01-01 00:11:52 2020-01-02 03:40:13 2020-01-01 00:11:52.000 2020-01-02 03:40:13.000 712 99613 50162.5 5016250 712 99613 50162.5 5016250 -32458 32680 4942.66 494266 -127 125 -0.7 -70 -713 100 10703 99614 2.14114 299.14114 150.64114 15064.11411 2.14114 299.14114 150.64114 15064.11431 2.14114 299.14114 150.64114 15064.11400 2020-01-01 2020-01-02 2020-01-01 00:11:53 2020-01-02 03:40:14 2020-01-01 00:11:53.000 2020-01-02 03:40:14.000 713 99614 50163.5 5016350 713 99614 50163.5 5016350 -32457 32681 4943.66 494366 -126 126 0.3 30 -714 100 10704 99615 2.14414 299.14414 150.64414 15064.41441 2.14414 299.14413 150.64414 15064.41448 2.14414 299.14414 150.64414 15064.41400 2020-01-01 2020-01-02 2020-01-01 00:11:54 2020-01-02 03:40:15 2020-01-01 00:11:54.000 2020-01-02 03:40:15.000 714 99615 50164.5 5016450 714 99615 50164.5 5016450 -32456 32682 4944.66 494466 -125 127 1.3 130 -715 100 10705 99616 2.14714 299.14714 150.64714 15064.71471 2.14714 299.14716 150.64714 15064.71458 2.14714 299.14714 150.64714 15064.71400 2020-01-01 2020-01-02 2020-01-01 00:11:55 2020-01-02 03:40:16 2020-01-01 00:11:55.000 2020-01-02 03:40:16.000 715 99616 50165.5 5016550 715 99616 50165.5 5016550 -32455 32683 4945.66 494566 -128 127 -0.26 -26 +711 100 10701 99612 2.13513 299.13513 150.63513 15063.51351 2.13513 299.13513 150.63513 15063.51325 2.13513 299.13513 150.63513000000006 15063.51300 2020-01-01 2020-01-02 2020-01-01 00:11:51 2020-01-02 03:40:12 2020-01-01 00:11:51.000 2020-01-02 03:40:12.000 711 99612 50161.5 5016150 711 99612 50161.5 5016150 -32459 32679 4941.66 494166 -128 124 -1.7 -170 +712 100 10702 99613 2.13813 299.13813 150.63813 15063.81381 2.13813 299.13815 150.63814 15063.81401 2.13813 299.13813 150.63812999999988 15063.81300 2020-01-01 2020-01-02 2020-01-01 00:11:52 2020-01-02 03:40:13 2020-01-01 00:11:52.000 2020-01-02 03:40:13.000 712 99613 50162.5 5016250 712 99613 50162.5 5016250 -32458 32680 4942.66 494266 -127 125 -0.7 -70 +713 100 10703 99614 2.14114 299.14114 150.64114 15064.11411 2.14114 299.14114 150.64114 15064.11431 2.14114 299.14114 150.64113999999984 15064.11400 2020-01-01 2020-01-02 2020-01-01 00:11:53 2020-01-02 03:40:14 2020-01-01 00:11:53.000 2020-01-02 03:40:14.000 713 99614 50163.5 5016350 713 99614 50163.5 5016350 -32457 32681 4943.66 494366 -126 126 0.3 30 +714 100 10704 99615 2.14414 299.14414 150.64414 15064.41441 2.14414 299.14413 150.64414 15064.41448 2.14414 299.14414 150.64414000000022 15064.41400 2020-01-01 2020-01-02 2020-01-01 00:11:54 2020-01-02 03:40:15 2020-01-01 00:11:54.000 2020-01-02 03:40:15.000 714 99615 50164.5 5016450 714 99615 50164.5 5016450 -32456 32682 4944.66 494466 -125 127 1.3 130 +715 100 10705 99616 2.14714 299.14714 150.64714 15064.71471 2.14714 299.14716 150.64714 15064.71458 2.14714 299.14714 150.64713999999972 15064.71400 2020-01-01 2020-01-02 2020-01-01 00:11:55 2020-01-02 03:40:16 2020-01-01 00:11:55.000 2020-01-02 03:40:16.000 715 99616 50165.5 5016550 715 99616 50165.5 5016550 -32455 32683 4945.66 494566 -128 127 -0.26 -26 716 100 10706 99617 2.15015 299.15015 150.65015 15065.01501 2.15015 299.15015 150.65014 15065.01472 2.15015 299.15015 150.65015 15065.01500 2020-01-01 2020-01-02 2020-01-01 00:11:56 2020-01-02 03:40:17 2020-01-01 00:11:56.000 2020-01-02 03:40:17.000 716 99617 50166.5 5016650 716 99617 50166.5 5016650 -32454 32684 4946.66 494666 -128 127 -1.82 -182 -717 100 10707 99618 2.15315 299.15315 150.65315 15065.31531 2.15315 299.15317 150.65315 15065.31547 2.15315 299.15315 150.65315 15065.31500 2020-01-01 2020-01-02 2020-01-01 00:11:57 2020-01-02 03:40:18 2020-01-01 00:11:57.000 2020-01-02 03:40:18.000 717 99618 50167.5 5016750 717 99618 50167.5 5016750 -32453 32685 4947.66 494766 -128 123 -3.38 -338 -718 100 10708 99619 2.15615 299.15615 150.65615 15065.61561 2.15615 299.15616 150.65615 15065.61578 2.15615 299.15615 150.65615 15065.61500 2020-01-01 2020-01-02 2020-01-01 00:11:58 2020-01-02 03:40:19 2020-01-01 00:11:58.000 2020-01-02 03:40:19.000 718 99619 50168.5 5016850 718 99619 50168.5 5016850 -32452 32686 4948.66 494866 -127 124 -2.38 -238 +717 100 10707 99618 2.15315 299.15315 150.65315 15065.31531 2.15315 299.15317 150.65315 15065.31547 2.15315 299.15315 150.65315000000007 15065.31500 2020-01-01 2020-01-02 2020-01-01 00:11:57 2020-01-02 03:40:18 2020-01-01 00:11:57.000 2020-01-02 03:40:18.000 717 99618 50167.5 5016750 717 99618 50167.5 5016750 -32453 32685 4947.66 494766 -128 123 -3.38 -338 +718 100 10708 99619 2.15615 299.15615 150.65615 15065.61561 2.15615 299.15616 150.65615 15065.61578 2.15615 299.15615 150.6561500000001 15065.61500 2020-01-01 2020-01-02 2020-01-01 00:11:58 2020-01-02 03:40:19 2020-01-01 00:11:58.000 2020-01-02 03:40:19.000 718 99619 50168.5 5016850 718 99619 50168.5 5016850 -32452 32686 4948.66 494866 -127 124 -2.38 -238 719 100 10709 99620 2.15915 299.15915 150.65915 15065.91591 2.15915 299.15915 150.65915 15065.91595 2.15915 299.15915 150.65915 15065.91500 2020-01-01 2020-01-02 2020-01-01 00:11:59 2020-01-02 03:40:20 2020-01-01 00:11:59.000 2020-01-02 03:40:20.000 719 99620 50169.5 5016950 719 99620 50169.5 5016950 -32451 32687 4949.66 494966 -126 125 -1.38 -138 72 102 10062 99972 0.21621 300.21621 150.21621 15171.83783 0.21621 300.21622 150.21621 15171.83791 0.21621 300.21621 150.21621 15171.83721 2020-01-01 2020-01-02 2020-01-01 00:01:12 2020-01-02 03:46:12 2020-01-01 00:01:12.000 2020-01-02 03:46:12.000 72 99972 50022 5052222 72 99972 50022 5052222 -32497 32438 4601.009900990099 464702 -128 127 -1.9207920792079207 -194 -720 100 10710 99621 2.16216 299.16216 150.66216 15066.21621 2.16216 299.16217 150.66216 15066.21606 2.16216 299.16216 150.66216 15066.21600 2020-01-01 2020-01-02 2020-01-01 00:12:00 2020-01-02 03:40:21 2020-01-01 00:12:00.000 2020-01-02 03:40:21.000 720 99621 50170.5 5017050 720 99621 50170.5 5017050 -32450 32688 4950.66 495066 -125 126 -0.38 -38 -721 100 10711 99622 2.16516 299.16516 150.66516 15066.51651 2.16516 299.16516 150.66516 15066.51635 2.16516 299.16516 150.66516 15066.51600 2020-01-01 2020-01-02 2020-01-01 00:12:01 2020-01-02 03:40:22 2020-01-01 00:12:01.000 2020-01-02 03:40:22.000 721 99622 50171.5 5017150 721 99622 50171.5 5017150 -32449 32689 4951.66 495166 -124 127 0.62 62 -722 100 10712 99623 2.16816 299.16816 150.66816 15066.81681 2.16816 299.16818 150.66816 15066.81695 2.16816 299.16816 150.66816 15066.81600 2020-01-01 2020-01-02 2020-01-01 00:12:02 2020-01-02 03:40:23 2020-01-01 00:12:02.000 2020-01-02 03:40:23.000 722 99623 50172.5 5017250 722 99623 50172.5 5017250 -32448 32690 4952.66 495266 -128 127 -0.94 -94 -723 100 10713 99624 2.17117 299.17117 150.67117 15067.11711 2.17117 299.17117 150.67117 15067.11724 2.17117 299.17117 150.67117 15067.11700 2020-01-01 2020-01-02 2020-01-01 00:12:03 2020-01-02 03:40:24 2020-01-01 00:12:03.000 2020-01-02 03:40:24.000 723 99624 50173.5 5017350 723 99624 50173.5 5017350 -32447 32691 4953.66 495366 -128 123 -2.5 -250 -724 100 10714 99625 2.17417 299.17417 150.67417 15067.41741 2.17417 299.17416 150.67417 15067.41742 2.17417 299.17417 150.67417 15067.41700 2020-01-01 2020-01-02 2020-01-01 00:12:04 2020-01-02 03:40:25 2020-01-01 00:12:04.000 2020-01-02 03:40:25.000 724 99625 50174.5 5017450 724 99625 50174.5 5017450 -32446 32692 4954.66 495466 -127 124 -1.5 -150 -725 100 10715 99626 2.17717 299.17717 150.67717 15067.71771 2.17717 299.1772 150.67717 15067.71752 2.17717 299.17717 150.67717 15067.71700 2020-01-01 2020-01-02 2020-01-01 00:12:05 2020-01-02 03:40:26 2020-01-01 00:12:05.000 2020-01-02 03:40:26.000 725 99626 50175.5 5017550 725 99626 50175.5 5017550 -32445 32693 4955.66 495566 -126 125 -0.5 -50 -726 100 10716 99627 2.18018 299.18018 150.68018 15068.01801 2.18018 299.18018 150.68017 15068.01782 2.18018 299.18018 150.68018 15068.01800 2020-01-01 2020-01-02 2020-01-01 00:12:06 2020-01-02 03:40:27 2020-01-01 00:12:06.000 2020-01-02 03:40:27.000 726 99627 50176.5 5017650 726 99627 50176.5 5017650 -32444 32694 4956.66 495666 -125 126 0.5 50 +720 100 10710 99621 2.16216 299.16216 150.66216 15066.21621 2.16216 299.16217 150.66216 15066.21606 2.16216 299.16216 150.6621599999999 15066.21600 2020-01-01 2020-01-02 2020-01-01 00:12:00 2020-01-02 03:40:21 2020-01-01 00:12:00.000 2020-01-02 03:40:21.000 720 99621 50170.5 5017050 720 99621 50170.5 5017050 -32450 32688 4950.66 495066 -125 126 -0.38 -38 +721 100 10711 99622 2.16516 299.16516 150.66516 15066.51651 2.16516 299.16516 150.66516 15066.51635 2.16516 299.16516 150.66515999999996 15066.51600 2020-01-01 2020-01-02 2020-01-01 00:12:01 2020-01-02 03:40:22 2020-01-01 00:12:01.000 2020-01-02 03:40:22.000 721 99622 50171.5 5017150 721 99622 50171.5 5017150 -32449 32689 4951.66 495166 -124 127 0.62 62 +722 100 10712 99623 2.16816 299.16816 150.66816 15066.81681 2.16816 299.16818 150.66816 15066.81695 2.16816 299.16816 150.66815999999986 15066.81600 2020-01-01 2020-01-02 2020-01-01 00:12:02 2020-01-02 03:40:23 2020-01-01 00:12:02.000 2020-01-02 03:40:23.000 722 99623 50172.5 5017250 722 99623 50172.5 5017250 -32448 32690 4952.66 495266 -128 127 -0.94 -94 +723 100 10713 99624 2.17117 299.17117 150.67117 15067.11711 2.17117 299.17117 150.67117 15067.11724 2.17117 299.17117 150.67116999999973 15067.11700 2020-01-01 2020-01-02 2020-01-01 00:12:03 2020-01-02 03:40:24 2020-01-01 00:12:03.000 2020-01-02 03:40:24.000 723 99624 50173.5 5017350 723 99624 50173.5 5017350 -32447 32691 4953.66 495366 -128 123 -2.5 -250 +724 100 10714 99625 2.17417 299.17417 150.67417 15067.41741 2.17417 299.17416 150.67417 15067.41742 2.17417 299.17417 150.67417000000015 15067.41700 2020-01-01 2020-01-02 2020-01-01 00:12:04 2020-01-02 03:40:25 2020-01-01 00:12:04.000 2020-01-02 03:40:25.000 724 99625 50174.5 5017450 724 99625 50174.5 5017450 -32446 32692 4954.66 495466 -127 124 -1.5 -150 +725 100 10715 99626 2.17717 299.17717 150.67717 15067.71771 2.17717 299.1772 150.67717 15067.71752 2.17717 299.17717 150.67717000000025 15067.71700 2020-01-01 2020-01-02 2020-01-01 00:12:05 2020-01-02 03:40:26 2020-01-01 00:12:05.000 2020-01-02 03:40:26.000 725 99626 50175.5 5017550 725 99626 50175.5 5017550 -32445 32693 4955.66 495566 -126 125 -0.5 -50 +726 100 10716 99627 2.18018 299.18018 150.68018 15068.01801 2.18018 299.18018 150.68017 15068.01782 2.18018 299.18018 150.6801799999999 15068.01800 2020-01-01 2020-01-02 2020-01-01 00:12:06 2020-01-02 03:40:27 2020-01-01 00:12:06.000 2020-01-02 03:40:27.000 726 99627 50176.5 5017650 726 99627 50176.5 5017650 -32444 32694 4956.66 495666 -125 126 0.5 50 727 100 10717 99628 2.18318 299.18318 150.68318 15068.31831 2.18318 299.1832 150.68318 15068.31842 2.18318 299.18318 150.68318 15068.31800 2020-01-01 2020-01-02 2020-01-01 00:12:07 2020-01-02 03:40:28 2020-01-01 00:12:07.000 2020-01-02 03:40:28.000 727 99628 50177.5 5017750 727 99628 50177.5 5017750 -32443 32695 4957.66 495766 -124 127 1.5 150 -728 100 10718 99629 2.18618 299.18618 150.68618 15068.61861 2.18618 299.1862 150.68618 15068.61875 2.18618 299.18618 150.68618 15068.61800 2020-01-01 2020-01-02 2020-01-01 00:12:08 2020-01-02 03:40:29 2020-01-01 00:12:08.000 2020-01-02 03:40:29.000 728 99629 50178.5 5017850 728 99629 50178.5 5017850 -32442 32696 4958.66 495866 -128 127 -0.06 -6 -729 100 10719 99630 2.18918 299.18918 150.68918 15068.91891 2.18918 299.18918 150.68918 15068.91889 2.18918 299.18918 150.68918 15068.91800 2020-01-01 2020-01-02 2020-01-01 00:12:09 2020-01-02 03:40:30 2020-01-01 00:12:09.000 2020-01-02 03:40:30.000 729 99630 50179.5 5017950 729 99630 50179.5 5017950 -32441 32697 4959.66 495966 -128 123 -1.62 -162 -73 102 10063 99973 0.21921 300.21921 150.21921 15172.14114 0.21921 300.2192 150.21921 15172.14121 0.21921 300.21921 150.21921 15172.14021 2020-01-01 2020-01-02 2020-01-01 00:01:13 2020-01-02 03:46:13 2020-01-01 00:01:13.000 2020-01-02 03:46:13.000 73 99973 50023 5052323 73 99973 50023 5052323 -32496 32439 4602.009900990099 464803 -128 127 -3.4554455445544554 -349 -730 100 10720 99631 2.19219 299.19219 150.69219 15069.21921 2.19219 299.1922 150.69219 15069.21965 2.19219 299.19219 150.69219 15069.21900 2020-01-01 2020-01-02 2020-01-01 00:12:10 2020-01-02 03:40:31 2020-01-01 00:12:10.000 2020-01-02 03:40:31.000 730 99631 50180.5 5018050 730 99631 50180.5 5018050 -32440 32698 4960.66 496066 -127 124 -0.62 -62 -731 100 10721 99632 2.19519 299.19519 150.69519 15069.51951 2.19519 299.1952 150.69519 15069.51928 2.19519 299.19519 150.69519 15069.51900 2020-01-01 2020-01-02 2020-01-01 00:12:11 2020-01-02 03:40:32 2020-01-01 00:12:11.000 2020-01-02 03:40:32.000 731 99632 50181.5 5018150 731 99632 50181.5 5018150 -32439 32699 4961.66 496166 -126 125 0.38 38 -732 100 10722 99633 2.19819 299.19819 150.69819 15069.81981 2.19819 299.1982 150.69819 15069.81988 2.19819 299.19819 150.69819 15069.81900 2020-01-01 2020-01-02 2020-01-01 00:12:12 2020-01-02 03:40:33 2020-01-01 00:12:12.000 2020-01-02 03:40:33.000 732 99633 50182.5 5018250 732 99633 50182.5 5018250 -32438 32700 4962.66 496266 -125 126 1.38 138 -733 100 10723 99634 2.2012 299.2012 150.7012 15070.12012 2.2012 299.2012 150.7012 15070.12022 2.20120 299.20120 150.70120 15070.12000 2020-01-01 2020-01-02 2020-01-01 00:12:13 2020-01-02 03:40:34 2020-01-01 00:12:13.000 2020-01-02 03:40:34.000 733 99634 50183.5 5018350 733 99634 50183.5 5018350 -32437 32701 4963.66 496366 -124 127 2.38 238 -734 100 10724 99635 2.2042 299.2042 150.7042 15070.42042 2.2042 299.2042 150.7042 15070.42036 2.20420 299.20420 150.70420 15070.42000 2020-01-01 2020-01-02 2020-01-01 00:12:14 2020-01-02 03:40:35 2020-01-01 00:12:14.000 2020-01-02 03:40:35.000 734 99635 50184.5 5018450 734 99635 50184.5 5018450 -32436 32702 4964.66 496466 -128 127 0.82 82 -735 100 10725 99636 2.2072 299.2072 150.7072 15070.72072 2.2072 299.2072 150.70721 15070.72111 2.20720 299.20720 150.70720 15070.72000 2020-01-01 2020-01-02 2020-01-01 00:12:15 2020-01-02 03:40:36 2020-01-01 00:12:15.000 2020-01-02 03:40:36.000 735 99636 50185.5 5018550 735 99636 50185.5 5018550 -32435 32703 4965.66 496566 -128 127 -0.74 -74 -736 100 10726 99637 2.21021 299.21021 150.71021 15071.02102 2.21021 299.2102 150.7102 15071.02076 2.21021 299.21021 150.71021 15071.02100 2020-01-01 2020-01-02 2020-01-01 00:12:16 2020-01-02 03:40:37 2020-01-01 00:12:16.000 2020-01-02 03:40:37.000 736 99637 50186.5 5018650 736 99637 50186.5 5018650 -32434 32704 4966.66 496666 -128 124 -2.3 -230 -737 100 10727 99638 2.21321 299.21321 150.71321 15071.32132 2.21321 299.21323 150.71321 15071.32139 2.21321 299.21321 150.71321 15071.32100 2020-01-01 2020-01-02 2020-01-01 00:12:17 2020-01-02 03:40:38 2020-01-01 00:12:17.000 2020-01-02 03:40:38.000 737 99638 50187.5 5018750 737 99638 50187.5 5018750 -32433 32705 4967.66 496766 -127 125 -1.3 -130 +728 100 10718 99629 2.18618 299.18618 150.68618 15068.61861 2.18618 299.1862 150.68618 15068.61875 2.18618 299.18618 150.6861800000001 15068.61800 2020-01-01 2020-01-02 2020-01-01 00:12:08 2020-01-02 03:40:29 2020-01-01 00:12:08.000 2020-01-02 03:40:29.000 728 99629 50178.5 5017850 728 99629 50178.5 5017850 -32442 32696 4958.66 495866 -128 127 -0.06 -6 +729 100 10719 99630 2.18918 299.18918 150.68918 15068.91891 2.18918 299.18918 150.68918 15068.91889 2.18918 299.18918 150.6891799999999 15068.91800 2020-01-01 2020-01-02 2020-01-01 00:12:09 2020-01-02 03:40:30 2020-01-01 00:12:09.000 2020-01-02 03:40:30.000 729 99630 50179.5 5017950 729 99630 50179.5 5017950 -32441 32697 4959.66 495966 -128 123 -1.62 -162 +73 102 10063 99973 0.21921 300.21921 150.21921 15172.14114 0.21921 300.2192 150.21921 15172.14121 0.21921 300.21921 150.2192099999999 15172.14021 2020-01-01 2020-01-02 2020-01-01 00:01:13 2020-01-02 03:46:13 2020-01-01 00:01:13.000 2020-01-02 03:46:13.000 73 99973 50023 5052323 73 99973 50023 5052323 -32496 32439 4602.009900990099 464803 -128 127 -3.4554455445544554 -349 +730 100 10720 99631 2.19219 299.19219 150.69219 15069.21921 2.19219 299.1922 150.69219 15069.21965 2.19219 299.19219 150.69218999999984 15069.21900 2020-01-01 2020-01-02 2020-01-01 00:12:10 2020-01-02 03:40:31 2020-01-01 00:12:10.000 2020-01-02 03:40:31.000 730 99631 50180.5 5018050 730 99631 50180.5 5018050 -32440 32698 4960.66 496066 -127 124 -0.62 -62 +731 100 10721 99632 2.19519 299.19519 150.69519 15069.51951 2.19519 299.1952 150.69519 15069.51928 2.19519 299.19519 150.69519000000025 15069.51900 2020-01-01 2020-01-02 2020-01-01 00:12:11 2020-01-02 03:40:32 2020-01-01 00:12:11.000 2020-01-02 03:40:32.000 731 99632 50181.5 5018150 731 99632 50181.5 5018150 -32439 32699 4961.66 496166 -126 125 0.38 38 +732 100 10722 99633 2.19819 299.19819 150.69819 15069.81981 2.19819 299.1982 150.69819 15069.81988 2.19819 299.19819 150.69818999999976 15069.81900 2020-01-01 2020-01-02 2020-01-01 00:12:12 2020-01-02 03:40:33 2020-01-01 00:12:12.000 2020-01-02 03:40:33.000 732 99633 50182.5 5018250 732 99633 50182.5 5018250 -32438 32700 4962.66 496266 -125 126 1.38 138 +733 100 10723 99634 2.2012 299.2012 150.7012 15070.12012 2.2012 299.2012 150.7012 15070.12022 2.20120 299.20120 150.70120000000003 15070.12000 2020-01-01 2020-01-02 2020-01-01 00:12:13 2020-01-02 03:40:34 2020-01-01 00:12:13.000 2020-01-02 03:40:34.000 733 99634 50183.5 5018350 733 99634 50183.5 5018350 -32437 32701 4963.66 496366 -124 127 2.38 238 +734 100 10724 99635 2.2042 299.2042 150.7042 15070.42042 2.2042 299.2042 150.7042 15070.42036 2.20420 299.20420 150.7042000000001 15070.42000 2020-01-01 2020-01-02 2020-01-01 00:12:14 2020-01-02 03:40:35 2020-01-01 00:12:14.000 2020-01-02 03:40:35.000 734 99635 50184.5 5018450 734 99635 50184.5 5018450 -32436 32702 4964.66 496466 -128 127 0.82 82 +735 100 10725 99636 2.2072 299.2072 150.7072 15070.72072 2.2072 299.2072 150.70721 15070.72111 2.20720 299.20720 150.70720000000014 15070.72000 2020-01-01 2020-01-02 2020-01-01 00:12:15 2020-01-02 03:40:36 2020-01-01 00:12:15.000 2020-01-02 03:40:36.000 735 99636 50185.5 5018550 735 99636 50185.5 5018550 -32435 32703 4965.66 496566 -128 127 -0.74 -74 +736 100 10726 99637 2.21021 299.21021 150.71021 15071.02102 2.21021 299.2102 150.7102 15071.02076 2.21021 299.21021 150.71020999999988 15071.02100 2020-01-01 2020-01-02 2020-01-01 00:12:16 2020-01-02 03:40:37 2020-01-01 00:12:16.000 2020-01-02 03:40:37.000 736 99637 50186.5 5018650 736 99637 50186.5 5018650 -32434 32704 4966.66 496666 -128 124 -2.3 -230 +737 100 10727 99638 2.21321 299.21321 150.71321 15071.32132 2.21321 299.21323 150.71321 15071.32139 2.21321 299.21321 150.71320999999992 15071.32100 2020-01-01 2020-01-02 2020-01-01 00:12:17 2020-01-02 03:40:38 2020-01-01 00:12:17.000 2020-01-02 03:40:38.000 737 99638 50187.5 5018750 737 99638 50187.5 5018750 -32433 32705 4967.66 496766 -127 125 -1.3 -130 738 100 10728 99639 2.21621 299.21621 150.71621 15071.62162 2.21621 299.21622 150.71621 15071.62169 2.21621 299.21621 150.71621 15071.62100 2020-01-01 2020-01-02 2020-01-01 00:12:18 2020-01-02 03:40:39 2020-01-01 00:12:18.000 2020-01-02 03:40:39.000 738 99639 50188.5 5018850 738 99639 50188.5 5018850 -32432 32706 4968.66 496866 -126 126 -0.3 -30 -739 100 10729 99640 2.21921 299.21921 150.71921 15071.92192 2.21921 299.2192 150.71921 15071.92199 2.21921 299.21921 150.71921 15071.92100 2020-01-01 2020-01-02 2020-01-01 00:12:19 2020-01-02 03:40:40 2020-01-01 00:12:19.000 2020-01-02 03:40:40.000 739 99640 50189.5 5018950 739 99640 50189.5 5018950 -32431 32707 4969.66 496966 -125 127 0.7 70 -74 102 10064 99974 0.22222 300.22222 150.22222 15172.44444 0.22222 300.22223 150.22222 15172.4448 0.22222 300.22222 150.22222 15172.44422 2020-01-01 2020-01-02 2020-01-01 00:01:14 2020-01-02 03:46:14 2020-01-01 00:01:14.000 2020-01-02 03:46:14.000 74 99974 50024 5052424 74 99974 50024 5052424 -32495 32440 4603.009900990099 464904 -128 123 -4.99009900990099 -504 -740 100 10730 99641 2.22222 299.22222 150.72222 15072.22222 2.22222 299.22223 150.72222 15072.22258 2.22222 299.22222 150.72222 15072.22200 2020-01-01 2020-01-02 2020-01-01 00:12:20 2020-01-02 03:40:41 2020-01-01 00:12:20.000 2020-01-02 03:40:41.000 740 99641 50190.5 5019050 740 99641 50190.5 5019050 -32430 32708 4970.66 497066 -128 127 -0.86 -86 -741 100 10731 99642 2.22522 299.22522 150.72522 15072.52252 2.22522 299.22522 150.72522 15072.52223 2.22522 299.22522 150.72522 15072.52200 2020-01-01 2020-01-02 2020-01-01 00:12:21 2020-01-02 03:40:42 2020-01-01 00:12:21.000 2020-01-02 03:40:42.000 741 99642 50191.5 5019150 741 99642 50191.5 5019150 -32429 32709 4971.66 497166 -128 127 -2.42 -242 -742 100 10732 99643 2.22822 299.22822 150.72822 15072.82282 2.22822 299.22824 150.72822 15072.82286 2.22822 299.22822 150.72822 15072.82200 2020-01-01 2020-01-02 2020-01-01 00:12:22 2020-01-02 03:40:43 2020-01-01 00:12:22.000 2020-01-02 03:40:43.000 742 99643 50192.5 5019250 742 99643 50192.5 5019250 -32428 32710 4972.66 497266 -128 123 -3.98 -398 -743 100 10733 99644 2.23123 299.23123 150.73123 15073.12312 2.23123 299.23123 150.73123 15073.12316 2.23123 299.23123 150.73123 15073.12300 2020-01-01 2020-01-02 2020-01-01 00:12:23 2020-01-02 03:40:44 2020-01-01 00:12:23.000 2020-01-02 03:40:44.000 743 99644 50193.5 5019350 743 99644 50193.5 5019350 -32427 32711 4973.66 497366 -127 124 -2.98 -298 -744 100 10734 99645 2.23423 299.23423 150.73423 15073.42342 2.23423 299.23422 150.73423 15073.42345 2.23423 299.23423 150.73423 15073.42300 2020-01-01 2020-01-02 2020-01-01 00:12:24 2020-01-02 03:40:45 2020-01-01 00:12:24.000 2020-01-02 03:40:45.000 744 99645 50194.5 5019450 744 99645 50194.5 5019450 -32426 32712 4974.66 497466 -126 125 -1.98 -198 -745 100 10735 99646 2.23723 299.23723 150.73723 15073.72372 2.23723 299.23724 150.73724 15073.72405 2.23723 299.23723 150.73723 15073.72300 2020-01-01 2020-01-02 2020-01-01 00:12:25 2020-01-02 03:40:46 2020-01-01 00:12:25.000 2020-01-02 03:40:46.000 745 99646 50195.5 5019550 745 99646 50195.5 5019550 -32425 32713 4975.66 497566 -125 126 -0.98 -98 -746 100 10736 99647 2.24024 299.24024 150.74024 15074.02402 2.24024 299.24023 150.74023 15074.02373 2.24024 299.24024 150.74024 15074.02400 2020-01-01 2020-01-02 2020-01-01 00:12:26 2020-01-02 03:40:47 2020-01-01 00:12:26.000 2020-01-02 03:40:47.000 746 99647 50196.5 5019650 746 99647 50196.5 5019650 -32424 32714 4976.66 497666 -124 127 0.02 2 -747 100 10737 99648 2.24324 299.24324 150.74324 15074.32432 2.24324 299.24326 150.74324 15074.32433 2.24324 299.24324 150.74324 15074.32400 2020-01-01 2020-01-02 2020-01-01 00:12:27 2020-01-02 03:40:48 2020-01-01 00:12:27.000 2020-01-02 03:40:48.000 747 99648 50197.5 5019750 747 99648 50197.5 5019750 -32423 32715 4977.66 497766 -128 127 -1.54 -154 -748 100 10738 99649 2.24624 299.24624 150.74624 15074.62462 2.24624 299.24625 150.74624 15074.62463 2.24624 299.24624 150.74624 15074.62400 2020-01-01 2020-01-02 2020-01-01 00:12:28 2020-01-02 03:40:49 2020-01-01 00:12:28.000 2020-01-02 03:40:49.000 748 99649 50198.5 5019850 748 99649 50198.5 5019850 -32422 32716 4978.66 497866 -128 123 -3.1 -310 -749 100 10739 99650 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924 15074.92400 2020-01-01 2020-01-02 2020-01-01 00:12:29 2020-01-02 03:40:50 2020-01-01 00:12:29.000 2020-01-02 03:40:50.000 749 99650 50199.5 5019950 749 99650 50199.5 5019950 -32421 32717 4979.66 497966 -127 124 -2.1 -210 -75 102 10065 99975 0.22522 300.22522 150.22522 15172.74774 0.22522 300.22522 150.22522 15172.74745 0.22522 300.22522 150.22522 15172.74722 2020-01-01 2020-01-02 2020-01-01 00:01:15 2020-01-02 03:46:15 2020-01-01 00:01:15.000 2020-01-02 03:46:15.000 75 99975 50025 5052525 75 99975 50025 5052525 -32494 32441 4604.009900990099 465005 -127 124 -3.99009900990099 -403 -750 100 10740 99651 2.25225 299.25225 150.75225 15075.22522 2.25225 299.25226 150.75225 15075.22552 2.25225 299.25225 150.75225 15075.22500 2020-01-01 2020-01-02 2020-01-01 00:12:30 2020-01-02 03:40:51 2020-01-01 00:12:30.000 2020-01-02 03:40:51.000 750 99651 50200.5 5020050 750 99651 50200.5 5020050 -32420 32718 4980.66 498066 -126 125 -1.1 -110 -751 100 10741 99652 2.25525 299.25525 150.75525 15075.52552 2.25525 299.25525 150.75525 15075.5252 2.25525 299.25525 150.75525 15075.52500 2020-01-01 2020-01-02 2020-01-01 00:12:31 2020-01-02 03:40:52 2020-01-01 00:12:31.000 2020-01-02 03:40:52.000 751 99652 50201.5 5020150 751 99652 50201.5 5020150 -32419 32719 4981.66 498166 -125 126 -0.1 -10 -752 100 10742 99653 2.25825 299.25825 150.75825 15075.82582 2.25825 299.25827 150.75825 15075.8258 2.25825 299.25825 150.75825 15075.82500 2020-01-01 2020-01-02 2020-01-01 00:12:32 2020-01-02 03:40:53 2020-01-01 00:12:32.000 2020-01-02 03:40:53.000 752 99653 50202.5 5020250 752 99653 50202.5 5020250 -32418 32720 4982.66 498266 -124 127 0.9 90 -753 100 10743 99654 2.26126 299.26126 150.76126 15076.12612 2.26126 299.26126 150.76126 15076.12609 2.26126 299.26126 150.76126 15076.12600 2020-01-01 2020-01-02 2020-01-01 00:12:33 2020-01-02 03:40:54 2020-01-01 00:12:33.000 2020-01-02 03:40:54.000 753 99654 50203.5 5020350 753 99654 50203.5 5020350 -32417 32721 4983.66 498366 -128 127 -0.66 -66 -754 100 10744 99655 2.26426 299.26426 150.76426 15076.42642 2.26426 299.26425 150.76426 15076.4264 2.26426 299.26426 150.76426 15076.42600 2020-01-01 2020-01-02 2020-01-01 00:12:34 2020-01-02 03:40:55 2020-01-01 00:12:34.000 2020-01-02 03:40:55.000 754 99655 50204.5 5020450 754 99655 50204.5 5020450 -32416 32722 4984.66 498466 -128 123 -2.22 -222 -755 100 10745 99656 2.26726 299.26726 150.76726 15076.72672 2.26726 299.26727 150.76727 15076.72703 2.26726 299.26726 150.76726 15076.72600 2020-01-01 2020-01-02 2020-01-01 00:12:35 2020-01-02 03:40:56 2020-01-01 00:12:35.000 2020-01-02 03:40:56.000 755 99656 50205.5 5020550 755 99656 50205.5 5020550 -32415 32723 4985.66 498566 -127 124 -1.22 -122 -756 100 10746 99657 2.27027 299.27027 150.77027 15077.02702 2.27027 299.27026 150.77026 15077.02667 2.27027 299.27027 150.77027 15077.02700 2020-01-01 2020-01-02 2020-01-01 00:12:36 2020-01-02 03:40:57 2020-01-01 00:12:36.000 2020-01-02 03:40:57.000 756 99657 50206.5 5020650 756 99657 50206.5 5020650 -32414 32724 4986.66 498666 -126 125 -0.22 -22 -757 100 10747 99658 2.27327 299.27327 150.77327 15077.32732 2.27327 299.2733 150.77327 15077.32727 2.27327 299.27327 150.77327 15077.32700 2020-01-01 2020-01-02 2020-01-01 00:12:37 2020-01-02 03:40:58 2020-01-01 00:12:37.000 2020-01-02 03:40:58.000 757 99658 50207.5 5020750 757 99658 50207.5 5020750 -32413 32725 4987.66 498766 -125 126 0.78 78 -758 100 10748 99659 2.27627 299.27627 150.77627 15077.62762 2.27627 299.27628 150.77627 15077.62756 2.27627 299.27627 150.77627 15077.62700 2020-01-01 2020-01-02 2020-01-01 00:12:38 2020-01-02 03:40:59 2020-01-01 00:12:38.000 2020-01-02 03:40:59.000 758 99659 50208.5 5020850 758 99659 50208.5 5020850 -32412 32726 4988.66 498866 -124 127 1.78 178 -759 100 10749 99660 2.27927 299.27927 150.77927 15077.92792 2.27927 299.27927 150.77927 15077.92787 2.27927 299.27927 150.77927 15077.92700 2020-01-01 2020-01-02 2020-01-01 00:12:39 2020-01-02 03:41:00 2020-01-01 00:12:39.000 2020-01-02 03:41:00.000 759 99660 50209.5 5020950 759 99660 50209.5 5020950 -32411 32727 4989.66 498966 -128 127 0.22 22 -76 102 10066 99976 0.22822 300.22822 150.22822 15173.05105 0.22822 300.22824 150.22822 15173.05109 0.22822 300.22822 150.22822 15173.05022 2020-01-01 2020-01-02 2020-01-01 00:01:16 2020-01-02 03:46:16 2020-01-01 00:01:16.000 2020-01-02 03:46:16.000 76 99976 50026 5052626 76 99976 50026 5052626 -32493 32442 4605.009900990099 465106 -126 125 -2.99009900990099 -302 -760 100 10750 99661 2.28228 299.28228 150.78228 15078.22822 2.28228 299.2823 150.78228 15078.2285 2.28228 299.28228 150.78228 15078.22800 2020-01-01 2020-01-02 2020-01-01 00:12:40 2020-01-02 03:41:01 2020-01-01 00:12:40.000 2020-01-02 03:41:01.000 760 99661 50210.5 5021050 760 99661 50210.5 5021050 -32410 32728 4990.66 499066 -128 127 -1.34 -134 -761 100 10751 99662 2.28528 299.28528 150.78528 15078.52852 2.28528 299.28528 150.78528 15078.52814 2.28528 299.28528 150.78528 15078.52800 2020-01-01 2020-01-02 2020-01-01 00:12:41 2020-01-02 03:41:02 2020-01-01 00:12:41.000 2020-01-02 03:41:02.000 761 99662 50211.5 5021150 761 99662 50211.5 5021150 -32409 32729 4991.66 499166 -128 124 -2.9 -290 -762 100 10752 99663 2.28828 299.28828 150.78828 15078.82882 2.28828 299.2883 150.78828 15078.82889 2.28828 299.28828 150.78828 15078.82800 2020-01-01 2020-01-02 2020-01-01 00:12:42 2020-01-02 03:41:03 2020-01-01 00:12:42.000 2020-01-02 03:41:03.000 762 99663 50212.5 5021250 762 99663 50212.5 5021250 -32408 32730 4992.66 499266 -127 125 -1.9 -190 -763 100 10753 99664 2.29129 299.29129 150.79129 15079.12912 2.29129 299.2913 150.79129 15079.12904 2.29129 299.29129 150.79129 15079.12900 2020-01-01 2020-01-02 2020-01-01 00:12:43 2020-01-02 03:41:04 2020-01-01 00:12:43.000 2020-01-02 03:41:04.000 763 99664 50213.5 5021350 763 99664 50213.5 5021350 -32407 32731 4993.66 499366 -126 126 -0.9 -90 -764 100 10754 99665 2.29429 299.29429 150.79429 15079.42942 2.29429 299.29428 150.79429 15079.42933 2.29429 299.29429 150.79429 15079.42900 2020-01-01 2020-01-02 2020-01-01 00:12:44 2020-01-02 03:41:05 2020-01-01 00:12:44.000 2020-01-02 03:41:05.000 764 99665 50214.5 5021450 764 99665 50214.5 5021450 -32406 32732 4994.66 499466 -125 127 0.1 10 -765 100 10755 99666 2.29729 299.29729 150.79729 15079.72972 2.29729 299.2973 150.79729 15079.72996 2.29729 299.29729 150.79729 15079.72900 2020-01-01 2020-01-02 2020-01-01 00:12:45 2020-01-02 03:41:06 2020-01-01 00:12:45.000 2020-01-02 03:41:06.000 765 99666 50215.5 5021550 765 99666 50215.5 5021550 -32405 32733 4995.66 499566 -128 127 -1.46 -146 -766 100 10756 99667 2.3003 299.3003 150.8003 15080.03003 2.3003 299.3003 150.80029 15080.02961 2.30030 299.30030 150.80030 15080.03000 2020-01-01 2020-01-02 2020-01-01 00:12:46 2020-01-02 03:41:07 2020-01-01 00:12:46.000 2020-01-02 03:41:07.000 766 99667 50216.5 5021650 766 99667 50216.5 5021650 -32404 32734 4996.66 499666 -128 127 -3.02 -302 -767 100 10757 99668 2.3033 299.3033 150.8033 15080.33033 2.3033 299.3033 150.8033 15080.33036 2.30330 299.30330 150.80330 15080.33000 2020-01-01 2020-01-02 2020-01-01 00:12:47 2020-01-02 03:41:08 2020-01-01 00:12:47.000 2020-01-02 03:41:08.000 767 99668 50217.5 5021750 767 99668 50217.5 5021750 -32403 32735 4997.66 499766 -128 123 -4.58 -458 -768 100 10758 99669 2.3063 299.3063 150.8063 15080.63063 2.3063 299.3063 150.8063 15080.6305 2.30630 299.30630 150.80630 15080.63000 2020-01-01 2020-01-02 2020-01-01 00:12:48 2020-01-02 03:41:09 2020-01-01 00:12:48.000 2020-01-02 03:41:09.000 768 99669 50218.5 5021850 768 99669 50218.5 5021850 -32402 32736 4998.66 499866 -127 124 -3.58 -358 -769 100 10759 99670 2.3093 299.3093 150.8093 15080.93093 2.3093 299.3093 150.8093 15080.93084 2.30930 299.30930 150.80930 15080.93000 2020-01-01 2020-01-02 2020-01-01 00:12:49 2020-01-02 03:41:10 2020-01-01 00:12:49.000 2020-01-02 03:41:10.000 769 99670 50219.5 5021950 769 99670 50219.5 5021950 -32401 32737 4999.66 499966 -126 125 -2.58 -258 -77 102 10067 99977 0.23123 300.23123 150.23123 15173.35435 0.23123 300.23123 150.23123 15173.35439 0.23123 300.23123 150.23123 15173.35423 2020-01-01 2020-01-02 2020-01-01 00:01:17 2020-01-02 03:46:17 2020-01-01 00:01:17.000 2020-01-02 03:46:17.000 77 99977 50027 5052727 77 99977 50027 5052727 -32492 32443 4606.009900990099 465207 -125 126 -1.99009900990099 -201 -770 100 10760 99671 2.31231 299.31231 150.81231 15081.23123 2.31231 299.31232 150.81231 15081.23144 2.31231 299.31231 150.81231 15081.23100 2020-01-01 2020-01-02 2020-01-01 00:12:50 2020-01-02 03:41:11 2020-01-01 00:12:50.000 2020-01-02 03:41:11.000 770 99671 50220.5 5022050 770 99671 50220.5 5022050 -32400 32738 5000.66 500066 -125 126 -1.58 -158 -771 100 10761 99672 2.31531 299.31531 150.81531 15081.53153 2.31531 299.3153 150.81531 15081.53173 2.31531 299.31531 150.81531 15081.53100 2020-01-01 2020-01-02 2020-01-01 00:12:51 2020-01-02 03:41:12 2020-01-01 00:12:51.000 2020-01-02 03:41:12.000 771 99672 50221.5 5022150 771 99672 50221.5 5022150 -32399 32739 5001.66 500166 -124 127 -0.58 -58 -772 100 10762 99673 2.31831 299.31831 150.81831 15081.83183 2.31831 299.31833 150.81831 15081.83183 2.31831 299.31831 150.81831 15081.83100 2020-01-01 2020-01-02 2020-01-01 00:12:52 2020-01-02 03:41:13 2020-01-01 00:12:52.000 2020-01-02 03:41:13.000 772 99673 50222.5 5022250 772 99673 50222.5 5022250 -32398 32740 5002.66 500266 -128 127 -2.14 -214 -773 100 10763 99674 2.32132 299.32132 150.82132 15082.13213 2.32132 299.32132 150.82131 15082.13197 2.32132 299.32132 150.82132 15082.13200 2020-01-01 2020-01-02 2020-01-01 00:12:53 2020-01-02 03:41:14 2020-01-01 00:12:53.000 2020-01-02 03:41:14.000 773 99674 50223.5 5022350 773 99674 50223.5 5022350 -32397 32741 5003.66 500366 -128 123 -3.7 -370 -774 100 10764 99675 2.32432 299.32432 150.82432 15082.43243 2.32432 299.3243 150.82432 15082.43231 2.32432 299.32432 150.82432 15082.43200 2020-01-01 2020-01-02 2020-01-01 00:12:54 2020-01-02 03:41:15 2020-01-01 00:12:54.000 2020-01-02 03:41:15.000 774 99675 50224.5 5022450 774 99675 50224.5 5022450 -32396 32742 5004.66 500466 -127 124 -2.7 -270 -775 100 10765 99676 2.32732 299.32732 150.82732 15082.73273 2.32732 299.32733 150.82732 15082.73291 2.32732 299.32732 150.82732 15082.73200 2020-01-01 2020-01-02 2020-01-01 00:12:55 2020-01-02 03:41:16 2020-01-01 00:12:55.000 2020-01-02 03:41:16.000 775 99676 50225.5 5022550 775 99676 50225.5 5022550 -32395 32743 5005.66 500566 -126 125 -1.7 -170 -776 100 10766 99677 2.33033 299.33033 150.83033 15083.03303 2.33033 299.33032 150.83033 15083.0332 2.33033 299.33033 150.83033 15083.03300 2020-01-01 2020-01-02 2020-01-01 00:12:56 2020-01-02 03:41:17 2020-01-01 00:12:56.000 2020-01-02 03:41:17.000 776 99677 50226.5 5022650 776 99677 50226.5 5022650 -32394 32744 5006.66 500666 -125 126 -0.7 -70 +739 100 10729 99640 2.21921 299.21921 150.71921 15071.92192 2.21921 299.2192 150.71921 15071.92199 2.21921 299.21921 150.7192099999999 15071.92100 2020-01-01 2020-01-02 2020-01-01 00:12:19 2020-01-02 03:40:40 2020-01-01 00:12:19.000 2020-01-02 03:40:40.000 739 99640 50189.5 5018950 739 99640 50189.5 5018950 -32431 32707 4969.66 496966 -125 127 0.7 70 +74 102 10064 99974 0.22222 300.22222 150.22222 15172.44444 0.22222 300.22223 150.22222 15172.4448 0.22222 300.22222 150.22221999999977 15172.44422 2020-01-01 2020-01-02 2020-01-01 00:01:14 2020-01-02 03:46:14 2020-01-01 00:01:14.000 2020-01-02 03:46:14.000 74 99974 50024 5052424 74 99974 50024 5052424 -32495 32440 4603.009900990099 464904 -128 123 -4.99009900990099 -504 +740 100 10730 99641 2.22222 299.22222 150.72222 15072.22222 2.22222 299.22223 150.72222 15072.22258 2.22222 299.22222 150.72221999999977 15072.22200 2020-01-01 2020-01-02 2020-01-01 00:12:20 2020-01-02 03:40:41 2020-01-01 00:12:20.000 2020-01-02 03:40:41.000 740 99641 50190.5 5019050 740 99641 50190.5 5019050 -32430 32708 4970.66 497066 -128 127 -0.86 -86 +741 100 10731 99642 2.22522 299.22522 150.72522 15072.52252 2.22522 299.22522 150.72522 15072.52223 2.22522 299.22522 150.72522000000015 15072.52200 2020-01-01 2020-01-02 2020-01-01 00:12:21 2020-01-02 03:40:42 2020-01-01 00:12:21.000 2020-01-02 03:40:42.000 741 99642 50191.5 5019150 741 99642 50191.5 5019150 -32429 32709 4971.66 497166 -128 127 -2.42 -242 +742 100 10732 99643 2.22822 299.22822 150.72822 15072.82282 2.22822 299.22824 150.72822 15072.82286 2.22822 299.22822 150.72822000000028 15072.82200 2020-01-01 2020-01-02 2020-01-01 00:12:22 2020-01-02 03:40:43 2020-01-01 00:12:22.000 2020-01-02 03:40:43.000 742 99643 50192.5 5019250 742 99643 50192.5 5019250 -32428 32710 4972.66 497266 -128 123 -3.98 -398 +743 100 10733 99644 2.23123 299.23123 150.73123 15073.12312 2.23123 299.23123 150.73123 15073.12316 2.23123 299.23123 150.73122999999993 15073.12300 2020-01-01 2020-01-02 2020-01-01 00:12:23 2020-01-02 03:40:44 2020-01-01 00:12:23.000 2020-01-02 03:40:44.000 743 99644 50193.5 5019350 743 99644 50193.5 5019350 -32427 32711 4973.66 497366 -127 124 -2.98 -298 +744 100 10734 99645 2.23423 299.23423 150.73423 15073.42342 2.23423 299.23422 150.73423 15073.42345 2.23423 299.23423 150.73423000000003 15073.42300 2020-01-01 2020-01-02 2020-01-01 00:12:24 2020-01-02 03:40:45 2020-01-01 00:12:24.000 2020-01-02 03:40:45.000 744 99645 50194.5 5019450 744 99645 50194.5 5019450 -32426 32712 4974.66 497466 -126 125 -1.98 -198 +745 100 10735 99646 2.23723 299.23723 150.73723 15073.72372 2.23723 299.23724 150.73724 15073.72405 2.23723 299.23723 150.73723000000012 15073.72300 2020-01-01 2020-01-02 2020-01-01 00:12:25 2020-01-02 03:40:46 2020-01-01 00:12:25.000 2020-01-02 03:40:46.000 745 99646 50195.5 5019550 745 99646 50195.5 5019550 -32425 32713 4975.66 497566 -125 126 -0.98 -98 +746 100 10736 99647 2.24024 299.24024 150.74024 15074.02402 2.24024 299.24023 150.74023 15074.02373 2.24024 299.24024 150.74023999999977 15074.02400 2020-01-01 2020-01-02 2020-01-01 00:12:26 2020-01-02 03:40:47 2020-01-01 00:12:26.000 2020-01-02 03:40:47.000 746 99647 50196.5 5019650 746 99647 50196.5 5019650 -32424 32714 4976.66 497666 -124 127 0.02 2 +747 100 10737 99648 2.24324 299.24324 150.74324 15074.32432 2.24324 299.24326 150.74324 15074.32433 2.24324 299.24324 150.74323999999982 15074.32400 2020-01-01 2020-01-02 2020-01-01 00:12:27 2020-01-02 03:40:48 2020-01-01 00:12:27.000 2020-01-02 03:40:48.000 747 99648 50197.5 5019750 747 99648 50197.5 5019750 -32423 32715 4977.66 497766 -128 127 -1.54 -154 +748 100 10738 99649 2.24624 299.24624 150.74624 15074.62462 2.24624 299.24625 150.74624 15074.62463 2.24624 299.24624 150.74624000000028 15074.62400 2020-01-01 2020-01-02 2020-01-01 00:12:28 2020-01-02 03:40:49 2020-01-01 00:12:28.000 2020-01-02 03:40:49.000 748 99649 50198.5 5019850 748 99649 50198.5 5019850 -32422 32716 4978.66 497866 -128 123 -3.1 -310 +749 100 10739 99650 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.7492399999998 15074.92400 2020-01-01 2020-01-02 2020-01-01 00:12:29 2020-01-02 03:40:50 2020-01-01 00:12:29.000 2020-01-02 03:40:50.000 749 99650 50199.5 5019950 749 99650 50199.5 5019950 -32421 32717 4979.66 497966 -127 124 -2.1 -210 +75 102 10065 99975 0.22522 300.22522 150.22522 15172.74774 0.22522 300.22522 150.22522 15172.74745 0.22522 300.22522 150.22522000000015 15172.74722 2020-01-01 2020-01-02 2020-01-01 00:01:15 2020-01-02 03:46:15 2020-01-01 00:01:15.000 2020-01-02 03:46:15.000 75 99975 50025 5052525 75 99975 50025 5052525 -32494 32441 4604.009900990099 465005 -127 124 -3.99009900990099 -403 +750 100 10740 99651 2.25225 299.25225 150.75225 15075.22522 2.25225 299.25226 150.75225 15075.22552 2.25225 299.25225 150.75224999999966 15075.22500 2020-01-01 2020-01-02 2020-01-01 00:12:30 2020-01-02 03:40:51 2020-01-01 00:12:30.000 2020-01-02 03:40:51.000 750 99651 50200.5 5020050 750 99651 50200.5 5020050 -32420 32718 4980.66 498066 -126 125 -1.1 -110 +751 100 10741 99652 2.25525 299.25525 150.75525 15075.52552 2.25525 299.25525 150.75525 15075.5252 2.25525 299.25525 150.75525000000013 15075.52500 2020-01-01 2020-01-02 2020-01-01 00:12:31 2020-01-02 03:40:52 2020-01-01 00:12:31.000 2020-01-02 03:40:52.000 751 99652 50201.5 5020150 751 99652 50201.5 5020150 -32419 32719 4981.66 498166 -125 126 -0.1 -10 +752 100 10742 99653 2.25825 299.25825 150.75825 15075.82582 2.25825 299.25827 150.75825 15075.8258 2.25825 299.25825 150.75825000000017 15075.82500 2020-01-01 2020-01-02 2020-01-01 00:12:32 2020-01-02 03:40:53 2020-01-01 00:12:32.000 2020-01-02 03:40:53.000 752 99653 50202.5 5020250 752 99653 50202.5 5020250 -32418 32720 4982.66 498266 -124 127 0.9 90 +753 100 10743 99654 2.26126 299.26126 150.76126 15076.12612 2.26126 299.26126 150.76126 15076.12609 2.26126 299.26126 150.7612599999999 15076.12600 2020-01-01 2020-01-02 2020-01-01 00:12:33 2020-01-02 03:40:54 2020-01-01 00:12:33.000 2020-01-02 03:40:54.000 753 99654 50203.5 5020350 753 99654 50203.5 5020350 -32417 32721 4983.66 498366 -128 127 -0.66 -66 +754 100 10744 99655 2.26426 299.26426 150.76426 15076.42642 2.26426 299.26425 150.76426 15076.4264 2.26426 299.26426 150.76425999999995 15076.42600 2020-01-01 2020-01-02 2020-01-01 00:12:34 2020-01-02 03:40:55 2020-01-01 00:12:34.000 2020-01-02 03:40:55.000 754 99655 50204.5 5020450 754 99655 50204.5 5020450 -32416 32722 4984.66 498466 -128 123 -2.22 -222 +755 100 10745 99656 2.26726 299.26726 150.76726 15076.72672 2.26726 299.26727 150.76727 15076.72703 2.26726 299.26726 150.76726000000002 15076.72600 2020-01-01 2020-01-02 2020-01-01 00:12:35 2020-01-02 03:40:56 2020-01-01 00:12:35.000 2020-01-02 03:40:56.000 755 99656 50205.5 5020550 755 99656 50205.5 5020550 -32415 32723 4985.66 498566 -127 124 -1.22 -122 +756 100 10746 99657 2.27027 299.27027 150.77027 15077.02702 2.27027 299.27026 150.77026 15077.02667 2.27027 299.27027 150.77027000000027 15077.02700 2020-01-01 2020-01-02 2020-01-01 00:12:36 2020-01-02 03:40:57 2020-01-01 00:12:36.000 2020-01-02 03:40:57.000 756 99657 50206.5 5020650 756 99657 50206.5 5020650 -32414 32724 4986.66 498666 -126 125 -0.22 -22 +757 100 10747 99658 2.27327 299.27327 150.77327 15077.32732 2.27327 299.2733 150.77327 15077.32727 2.27327 299.27327 150.7732699999998 15077.32700 2020-01-01 2020-01-02 2020-01-01 00:12:37 2020-01-02 03:40:58 2020-01-01 00:12:37.000 2020-01-02 03:40:58.000 757 99658 50207.5 5020750 757 99658 50207.5 5020750 -32413 32725 4987.66 498766 -125 126 0.78 78 +758 100 10748 99659 2.27627 299.27627 150.77627 15077.62762 2.27627 299.27628 150.77627 15077.62756 2.27627 299.27627 150.77627000000018 15077.62700 2020-01-01 2020-01-02 2020-01-01 00:12:38 2020-01-02 03:40:59 2020-01-01 00:12:38.000 2020-01-02 03:40:59.000 758 99659 50208.5 5020850 758 99659 50208.5 5020850 -32412 32726 4988.66 498866 -124 127 1.78 178 +759 100 10749 99660 2.27927 299.27927 150.77927 15077.92792 2.27927 299.27927 150.77927 15077.92787 2.27927 299.27927 150.7792699999997 15077.92700 2020-01-01 2020-01-02 2020-01-01 00:12:39 2020-01-02 03:41:00 2020-01-01 00:12:39.000 2020-01-02 03:41:00.000 759 99660 50209.5 5020950 759 99660 50209.5 5020950 -32411 32727 4989.66 498966 -128 127 0.22 22 +76 102 10066 99976 0.22822 300.22822 150.22822 15173.05105 0.22822 300.22824 150.22822 15173.05109 0.22822 300.22822 150.22822000000028 15173.05022 2020-01-01 2020-01-02 2020-01-01 00:01:16 2020-01-02 03:46:16 2020-01-01 00:01:16.000 2020-01-02 03:46:16.000 76 99976 50026 5052626 76 99976 50026 5052626 -32493 32442 4605.009900990099 465106 -126 125 -2.99009900990099 -302 +760 100 10750 99661 2.28228 299.28228 150.78228 15078.22822 2.28228 299.2823 150.78228 15078.2285 2.28228 299.28228 150.78227999999996 15078.22800 2020-01-01 2020-01-02 2020-01-01 00:12:40 2020-01-02 03:41:01 2020-01-01 00:12:40.000 2020-01-02 03:41:01.000 760 99661 50210.5 5021050 760 99661 50210.5 5021050 -32410 32728 4990.66 499066 -128 127 -1.34 -134 +761 100 10751 99662 2.28528 299.28528 150.78528 15078.52852 2.28528 299.28528 150.78528 15078.52814 2.28528 299.28528 150.78528000000003 15078.52800 2020-01-01 2020-01-02 2020-01-01 00:12:41 2020-01-02 03:41:02 2020-01-01 00:12:41.000 2020-01-02 03:41:02.000 761 99662 50211.5 5021150 761 99662 50211.5 5021150 -32409 32729 4991.66 499166 -128 124 -2.9 -290 +762 100 10752 99663 2.28828 299.28828 150.78828 15078.82882 2.28828 299.2883 150.78828 15078.82889 2.28828 299.28828 150.78828000000013 15078.82800 2020-01-01 2020-01-02 2020-01-01 00:12:42 2020-01-02 03:41:03 2020-01-01 00:12:42.000 2020-01-02 03:41:03.000 762 99663 50212.5 5021250 762 99663 50212.5 5021250 -32408 32730 4992.66 499266 -127 125 -1.9 -190 +763 100 10753 99664 2.29129 299.29129 150.79129 15079.12912 2.29129 299.2913 150.79129 15079.12904 2.29129 299.29129 150.7912899999998 15079.12900 2020-01-01 2020-01-02 2020-01-01 00:12:43 2020-01-02 03:41:04 2020-01-01 00:12:43.000 2020-01-02 03:41:04.000 763 99664 50213.5 5021350 763 99664 50213.5 5021350 -32407 32731 4993.66 499366 -126 126 -0.9 -90 +764 100 10754 99665 2.29429 299.29429 150.79429 15079.42942 2.29429 299.29428 150.79429 15079.42933 2.29429 299.29429 150.79428999999985 15079.42900 2020-01-01 2020-01-02 2020-01-01 00:12:44 2020-01-02 03:41:05 2020-01-01 00:12:44.000 2020-01-02 03:41:05.000 764 99665 50214.5 5021450 764 99665 50214.5 5021450 -32406 32732 4994.66 499466 -125 127 0.1 10 +765 100 10755 99666 2.29729 299.29729 150.79729 15079.72972 2.29729 299.2973 150.79729 15079.72996 2.29729 299.29729 150.79729000000032 15079.72900 2020-01-01 2020-01-02 2020-01-01 00:12:45 2020-01-02 03:41:06 2020-01-01 00:12:45.000 2020-01-02 03:41:06.000 765 99666 50215.5 5021550 765 99666 50215.5 5021550 -32405 32733 4995.66 499566 -128 127 -1.46 -146 +766 100 10756 99667 2.3003 299.3003 150.8003 15080.03003 2.3003 299.3003 150.80029 15080.02961 2.30030 299.30030 150.8003000000002 15080.03000 2020-01-01 2020-01-02 2020-01-01 00:12:46 2020-01-02 03:41:07 2020-01-01 00:12:46.000 2020-01-02 03:41:07.000 766 99667 50216.5 5021650 766 99667 50216.5 5021650 -32404 32734 4996.66 499666 -128 127 -3.02 -302 +767 100 10757 99668 2.3033 299.3033 150.8033 15080.33033 2.3033 299.3033 150.8033 15080.33036 2.30330 299.30330 150.8032999999997 15080.33000 2020-01-01 2020-01-02 2020-01-01 00:12:47 2020-01-02 03:41:08 2020-01-01 00:12:47.000 2020-01-02 03:41:08.000 767 99668 50217.5 5021750 767 99668 50217.5 5021750 -32403 32735 4997.66 499766 -128 123 -4.58 -458 +768 100 10758 99669 2.3063 299.3063 150.8063 15080.63063 2.3063 299.3063 150.8063 15080.6305 2.30630 299.30630 150.80630000000016 15080.63000 2020-01-01 2020-01-02 2020-01-01 00:12:48 2020-01-02 03:41:09 2020-01-01 00:12:48.000 2020-01-02 03:41:09.000 768 99669 50218.5 5021850 768 99669 50218.5 5021850 -32402 32736 4998.66 499866 -127 124 -3.58 -358 +769 100 10759 99670 2.3093 299.3093 150.8093 15080.93093 2.3093 299.3093 150.8093 15080.93084 2.30930 299.30930 150.8093000000002 15080.93000 2020-01-01 2020-01-02 2020-01-01 00:12:49 2020-01-02 03:41:10 2020-01-01 00:12:49.000 2020-01-02 03:41:10.000 769 99670 50219.5 5021950 769 99670 50219.5 5021950 -32401 32737 4999.66 499966 -126 125 -2.58 -258 +77 102 10067 99977 0.23123 300.23123 150.23123 15173.35435 0.23123 300.23123 150.23123 15173.35439 0.23123 300.23123 150.23122999999993 15173.35423 2020-01-01 2020-01-02 2020-01-01 00:01:17 2020-01-02 03:46:17 2020-01-01 00:01:17.000 2020-01-02 03:46:17.000 77 99977 50027 5052727 77 99977 50027 5052727 -32492 32443 4606.009900990099 465207 -125 126 -1.99009900990099 -201 +770 100 10760 99671 2.31231 299.31231 150.81231 15081.23123 2.31231 299.31232 150.81231 15081.23144 2.31231 299.31231 150.81230999999988 15081.23100 2020-01-01 2020-01-02 2020-01-01 00:12:50 2020-01-02 03:41:11 2020-01-01 00:12:50.000 2020-01-02 03:41:11.000 770 99671 50220.5 5022050 770 99671 50220.5 5022050 -32400 32738 5000.66 500066 -125 126 -1.58 -158 +771 100 10761 99672 2.31531 299.31531 150.81531 15081.53153 2.31531 299.3153 150.81531 15081.53173 2.31531 299.31531 150.81530999999998 15081.53100 2020-01-01 2020-01-02 2020-01-01 00:12:51 2020-01-02 03:41:12 2020-01-01 00:12:51.000 2020-01-02 03:41:12.000 771 99672 50221.5 5022150 771 99672 50221.5 5022150 -32399 32739 5001.66 500166 -124 127 -0.58 -58 +772 100 10762 99673 2.31831 299.31831 150.81831 15081.83183 2.31831 299.31833 150.81831 15081.83183 2.31831 299.31831 150.81831000000005 15081.83100 2020-01-01 2020-01-02 2020-01-01 00:12:52 2020-01-02 03:41:13 2020-01-01 00:12:52.000 2020-01-02 03:41:13.000 772 99673 50222.5 5022250 772 99673 50222.5 5022250 -32398 32740 5002.66 500266 -128 127 -2.14 -214 +773 100 10763 99674 2.32132 299.32132 150.82132 15082.13213 2.32132 299.32132 150.82131 15082.13197 2.32132 299.32132 150.82131999999973 15082.13200 2020-01-01 2020-01-02 2020-01-01 00:12:53 2020-01-02 03:41:14 2020-01-01 00:12:53.000 2020-01-02 03:41:14.000 773 99674 50223.5 5022350 773 99674 50223.5 5022350 -32397 32741 5003.66 500366 -128 123 -3.7 -370 +774 100 10764 99675 2.32432 299.32432 150.82432 15082.43243 2.32432 299.3243 150.82432 15082.43231 2.32432 299.32432 150.82431999999983 15082.43200 2020-01-01 2020-01-02 2020-01-01 00:12:54 2020-01-02 03:41:15 2020-01-01 00:12:54.000 2020-01-02 03:41:15.000 774 99675 50224.5 5022450 774 99675 50224.5 5022450 -32396 32742 5004.66 500466 -127 124 -2.7 -270 +775 100 10765 99676 2.32732 299.32732 150.82732 15082.73273 2.32732 299.32733 150.82732 15082.73291 2.32732 299.32732 150.8273200000002 15082.73200 2020-01-01 2020-01-02 2020-01-01 00:12:55 2020-01-02 03:41:16 2020-01-01 00:12:55.000 2020-01-02 03:41:16.000 775 99676 50225.5 5022550 775 99676 50225.5 5022550 -32395 32743 5005.66 500566 -126 125 -1.7 -170 +776 100 10766 99677 2.33033 299.33033 150.83033 15083.03303 2.33033 299.33032 150.83033 15083.0332 2.33033 299.33033 150.8303300000001 15083.03300 2020-01-01 2020-01-02 2020-01-01 00:12:56 2020-01-02 03:41:17 2020-01-01 00:12:56.000 2020-01-02 03:41:17.000 776 99677 50226.5 5022650 776 99677 50226.5 5022650 -32394 32744 5006.66 500666 -125 126 -0.7 -70 777 100 10767 99678 2.33333 299.33333 150.83333 15083.33333 2.33333 299.33334 150.83333 15083.3333 2.33333 299.33333 150.83333 15083.33300 2020-01-01 2020-01-02 2020-01-01 00:12:57 2020-01-02 03:41:18 2020-01-01 00:12:57.000 2020-01-02 03:41:18.000 777 99678 50227.5 5022750 777 99678 50227.5 5022750 -32393 32745 5007.66 500766 -124 127 0.3 30 -778 100 10768 99679 2.33633 299.33633 150.83633 15083.63363 2.33633 299.33633 150.83633 15083.63348 2.33633 299.33633 150.83633 15083.63300 2020-01-01 2020-01-02 2020-01-01 00:12:58 2020-01-02 03:41:19 2020-01-01 00:12:58.000 2020-01-02 03:41:19.000 778 99679 50228.5 5022850 778 99679 50228.5 5022850 -32392 32746 5008.66 500866 -128 127 -1.26 -126 -779 100 10769 99680 2.33933 299.33933 150.83933 15083.93393 2.33933 299.33932 150.83933 15083.93378 2.33933 299.33933 150.83933 15083.93300 2020-01-01 2020-01-02 2020-01-01 00:12:59 2020-01-02 03:41:20 2020-01-01 00:12:59.000 2020-01-02 03:41:20.000 779 99680 50229.5 5022950 779 99680 50229.5 5022950 -32391 32747 5009.66 500966 -128 123 -2.82 -282 +778 100 10768 99679 2.33633 299.33633 150.83633 15083.63363 2.33633 299.33633 150.83633 15083.63348 2.33633 299.33633 150.83633000000006 15083.63300 2020-01-01 2020-01-02 2020-01-01 00:12:58 2020-01-02 03:41:19 2020-01-01 00:12:58.000 2020-01-02 03:41:19.000 778 99679 50228.5 5022850 778 99679 50228.5 5022850 -32392 32746 5008.66 500866 -128 127 -1.26 -126 +779 100 10769 99680 2.33933 299.33933 150.83933 15083.93393 2.33933 299.33932 150.83933 15083.93378 2.33933 299.33933 150.8393300000001 15083.93300 2020-01-01 2020-01-02 2020-01-01 00:12:59 2020-01-02 03:41:20 2020-01-01 00:12:59.000 2020-01-02 03:41:20.000 779 99680 50229.5 5022950 779 99680 50229.5 5022950 -32391 32747 5009.66 500966 -128 123 -2.82 -282 78 102 10068 99978 0.23423 300.23423 150.23423 15173.65765 0.23423 300.23422 150.23423 15173.65769 0.23423 300.23423 150.23423 15173.65723 2020-01-01 2020-01-02 2020-01-01 00:01:18 2020-01-02 03:46:18 2020-01-01 00:01:18.000 2020-01-02 03:46:18.000 78 99978 50028 5052828 78 99978 50028 5052828 -32491 32444 4607.009900990099 465308 -124 127 -0.9900990099009901 -100 -780 100 10770 99681 2.34234 299.34234 150.84234 15084.23423 2.34234 299.34235 150.84234 15084.23437 2.34234 299.34234 150.84234 15084.23400 2020-01-01 2020-01-02 2020-01-01 00:13:00 2020-01-02 03:41:21 2020-01-01 00:13:00.000 2020-01-02 03:41:21.000 780 99681 50230.5 5023050 780 99681 50230.5 5023050 -32390 32748 5010.66 501066 -127 124 -1.82 -182 -781 100 10771 99682 2.34534 299.34534 150.84534 15084.53453 2.34534 299.34534 150.84534 15084.53467 2.34534 299.34534 150.84534 15084.53400 2020-01-01 2020-01-02 2020-01-01 00:13:01 2020-01-02 03:41:22 2020-01-01 00:13:01.000 2020-01-02 03:41:22.000 781 99682 50231.5 5023150 781 99682 50231.5 5023150 -32389 32749 5011.66 501166 -126 125 -0.82 -82 -782 100 10772 99683 2.34834 299.34834 150.84834 15084.83483 2.34834 299.34836 150.84834 15084.83477 2.34834 299.34834 150.84834 15084.83400 2020-01-01 2020-01-02 2020-01-01 00:13:02 2020-01-02 03:41:23 2020-01-01 00:13:02.000 2020-01-02 03:41:23.000 782 99683 50232.5 5023250 782 99683 50232.5 5023250 -32388 32750 5012.66 501266 -125 126 0.18 18 -783 100 10773 99684 2.35135 299.35135 150.85135 15085.13513 2.35135 299.35135 150.85134 15085.13495 2.35135 299.35135 150.85135 15085.13500 2020-01-01 2020-01-02 2020-01-01 00:13:03 2020-01-02 03:41:24 2020-01-01 00:13:03.000 2020-01-02 03:41:24.000 783 99684 50233.5 5023350 783 99684 50233.5 5023350 -32387 32751 5013.66 501366 -124 127 1.18 118 -784 100 10774 99685 2.35435 299.35435 150.85435 15085.43543 2.35435 299.35434 150.85435 15085.43525 2.35435 299.35435 150.85435 15085.43500 2020-01-01 2020-01-02 2020-01-01 00:13:04 2020-01-02 03:41:25 2020-01-01 00:13:04.000 2020-01-02 03:41:25.000 784 99685 50234.5 5023450 784 99685 50234.5 5023450 -32386 32752 5014.66 501466 -128 127 -0.38 -38 -785 100 10775 99686 2.35735 299.35735 150.85735 15085.73573 2.35735 299.35736 150.85736 15085.736 2.35735 299.35735 150.85735 15085.73500 2020-01-01 2020-01-02 2020-01-01 00:13:05 2020-01-02 03:41:26 2020-01-01 00:13:05.000 2020-01-02 03:41:26.000 785 99686 50235.5 5023550 785 99686 50235.5 5023550 -32385 32753 5015.66 501566 -128 127 -1.94 -194 -786 100 10776 99687 2.36036 299.36036 150.86036 15086.03603 2.36036 299.36035 150.86036 15086.03614 2.36036 299.36036 150.86036 15086.03600 2020-01-01 2020-01-02 2020-01-01 00:13:06 2020-01-02 03:41:27 2020-01-01 00:13:06.000 2020-01-02 03:41:27.000 786 99687 50236.5 5023650 786 99687 50236.5 5023650 -32384 32754 5016.66 501666 -128 124 -3.5 -350 -787 100 10777 99688 2.36336 299.36336 150.86336 15086.33633 2.36336 299.36337 150.86336 15086.33628 2.36336 299.36336 150.86336 15086.33600 2020-01-01 2020-01-02 2020-01-01 00:13:07 2020-01-02 03:41:28 2020-01-01 00:13:07.000 2020-01-02 03:41:28.000 787 99688 50237.5 5023750 787 99688 50237.5 5023750 -32383 32755 5017.66 501766 -127 125 -2.5 -250 +780 100 10770 99681 2.34234 299.34234 150.84234 15084.23423 2.34234 299.34235 150.84234 15084.23437 2.34234 299.34234 150.84233999999995 15084.23400 2020-01-01 2020-01-02 2020-01-01 00:13:00 2020-01-02 03:41:21 2020-01-01 00:13:00.000 2020-01-02 03:41:21.000 780 99681 50230.5 5023050 780 99681 50230.5 5023050 -32390 32748 5010.66 501066 -127 124 -1.82 -182 +781 100 10771 99682 2.34534 299.34534 150.84534 15084.53453 2.34534 299.34534 150.84534 15084.53467 2.34534 299.34534 150.84533999999994 15084.53400 2020-01-01 2020-01-02 2020-01-01 00:13:01 2020-01-02 03:41:22 2020-01-01 00:13:01.000 2020-01-02 03:41:22.000 781 99682 50231.5 5023150 781 99682 50231.5 5023150 -32389 32749 5011.66 501166 -126 125 -0.82 -82 +782 100 10772 99683 2.34834 299.34834 150.84834 15084.83483 2.34834 299.34836 150.84834 15084.83477 2.34834 299.34834 150.84834000000018 15084.83400 2020-01-01 2020-01-02 2020-01-01 00:13:02 2020-01-02 03:41:23 2020-01-01 00:13:02.000 2020-01-02 03:41:23.000 782 99683 50232.5 5023250 782 99683 50232.5 5023250 -32388 32750 5012.66 501266 -125 126 0.18 18 +783 100 10773 99684 2.35135 299.35135 150.85135 15085.13513 2.35135 299.35135 150.85134 15085.13495 2.35135 299.35135 150.85135000000005 15085.13500 2020-01-01 2020-01-02 2020-01-01 00:13:03 2020-01-02 03:41:24 2020-01-01 00:13:03.000 2020-01-02 03:41:24.000 783 99684 50233.5 5023350 783 99684 50233.5 5023350 -32387 32751 5013.66 501366 -124 127 1.18 118 +784 100 10774 99685 2.35435 299.35435 150.85435 15085.43543 2.35435 299.35434 150.85435 15085.43525 2.35435 299.35435 150.8543499999999 15085.43500 2020-01-01 2020-01-02 2020-01-01 00:13:04 2020-01-02 03:41:25 2020-01-01 00:13:04.000 2020-01-02 03:41:25.000 784 99685 50234.5 5023450 784 99685 50234.5 5023450 -32386 32752 5014.66 501466 -128 127 -0.38 -38 +785 100 10775 99686 2.35735 299.35735 150.85735 15085.73573 2.35735 299.35736 150.85736 15085.736 2.35735 299.35735 150.8573500000001 15085.73500 2020-01-01 2020-01-02 2020-01-01 00:13:05 2020-01-02 03:41:26 2020-01-01 00:13:05.000 2020-01-02 03:41:26.000 785 99686 50235.5 5023550 785 99686 50235.5 5023550 -32385 32753 5015.66 501566 -128 127 -1.94 -194 +786 100 10776 99687 2.36036 299.36036 150.86036 15086.03603 2.36036 299.36035 150.86036 15086.03614 2.36036 299.36036 150.86036000000001 15086.03600 2020-01-01 2020-01-02 2020-01-01 00:13:06 2020-01-02 03:41:27 2020-01-01 00:13:06.000 2020-01-02 03:41:27.000 786 99687 50236.5 5023650 786 99687 50236.5 5023650 -32384 32754 5016.66 501666 -128 124 -3.5 -350 +787 100 10777 99688 2.36336 299.36336 150.86336 15086.33633 2.36336 299.36337 150.86336 15086.33628 2.36336 299.36336 150.86335999999994 15086.33600 2020-01-01 2020-01-02 2020-01-01 00:13:07 2020-01-02 03:41:28 2020-01-01 00:13:07.000 2020-01-02 03:41:28.000 787 99688 50237.5 5023750 787 99688 50237.5 5023750 -32383 32755 5017.66 501766 -127 125 -2.5 -250 788 100 10778 99689 2.36636 299.36636 150.86636 15086.63663 2.36636 299.36636 150.86636 15086.63641 2.36636 299.36636 150.86636 15086.63600 2020-01-01 2020-01-02 2020-01-01 00:13:08 2020-01-02 03:41:29 2020-01-01 00:13:08.000 2020-01-02 03:41:29.000 788 99689 50238.5 5023850 788 99689 50238.5 5023850 -32382 32756 5018.66 501866 -126 126 -1.5 -150 -789 100 10779 99690 2.36936 299.36936 150.86936 15086.93693 2.36936 299.36935 150.86936 15086.93672 2.36936 299.36936 150.86936 15086.93600 2020-01-01 2020-01-02 2020-01-01 00:13:09 2020-01-02 03:41:30 2020-01-01 00:13:09.000 2020-01-02 03:41:30.000 789 99690 50239.5 5023950 789 99690 50239.5 5023950 -32381 32757 5019.66 501966 -125 127 -0.5 -50 -79 102 10069 99979 0.23723 300.23723 150.23723 15173.96096 0.23723 300.23724 150.23724 15173.96129 0.23723 300.23723 150.23723 15173.96023 2020-01-01 2020-01-02 2020-01-01 00:01:19 2020-01-02 03:46:19 2020-01-01 00:01:19.000 2020-01-02 03:46:19.000 79 99979 50029 5052929 79 99979 50029 5052929 -32490 32445 4608.009900990099 465409 -128 127 -2.5247524752475248 -255 -790 100 10780 99691 2.37237 299.37237 150.87237 15087.23723 2.37237 299.37238 150.87237 15087.23747 2.37237 299.37237 150.87237 15087.23700 2020-01-01 2020-01-02 2020-01-01 00:13:10 2020-01-02 03:41:31 2020-01-01 00:13:10.000 2020-01-02 03:41:31.000 790 99691 50240.5 5024050 790 99691 50240.5 5024050 -32380 32758 5020.66 502066 -128 127 -2.06 -206 -791 100 10781 99692 2.37537 299.37537 150.87537 15087.53753 2.37537 299.37537 150.87537 15087.53761 2.37537 299.37537 150.87537 15087.53700 2020-01-01 2020-01-02 2020-01-01 00:13:11 2020-01-02 03:41:32 2020-01-01 00:13:11.000 2020-01-02 03:41:32.000 791 99692 50241.5 5024150 791 99692 50241.5 5024150 -32379 32759 5021.66 502166 -128 127 -3.62 -362 -792 100 10782 99693 2.37837 299.37837 150.87837 15087.83783 2.37837 299.3784 150.87837 15087.83775 2.37837 299.37837 150.87837 15087.83700 2020-01-01 2020-01-02 2020-01-01 00:13:12 2020-01-02 03:41:33 2020-01-01 00:13:12.000 2020-01-02 03:41:33.000 792 99693 50242.5 5024250 792 99693 50242.5 5024250 -32378 32760 5022.66 502266 -128 123 -5.18 -518 +789 100 10779 99690 2.36936 299.36936 150.86936 15086.93693 2.36936 299.36935 150.86936 15086.93672 2.36936 299.36936 150.86936000000003 15086.93600 2020-01-01 2020-01-02 2020-01-01 00:13:09 2020-01-02 03:41:30 2020-01-01 00:13:09.000 2020-01-02 03:41:30.000 789 99690 50239.5 5023950 789 99690 50239.5 5023950 -32381 32757 5019.66 501966 -125 127 -0.5 -50 +79 102 10069 99979 0.23723 300.23723 150.23723 15173.96096 0.23723 300.23724 150.23724 15173.96129 0.23723 300.23723 150.23723000000007 15173.96023 2020-01-01 2020-01-02 2020-01-01 00:01:19 2020-01-02 03:46:19 2020-01-01 00:01:19.000 2020-01-02 03:46:19.000 79 99979 50029 5052929 79 99979 50029 5052929 -32490 32445 4608.009900990099 465409 -128 127 -2.5247524752475248 -255 +790 100 10780 99691 2.37237 299.37237 150.87237 15087.23723 2.37237 299.37238 150.87237 15087.23747 2.37237 299.37237 150.87236999999993 15087.23700 2020-01-01 2020-01-02 2020-01-01 00:13:10 2020-01-02 03:41:31 2020-01-01 00:13:10.000 2020-01-02 03:41:31.000 790 99691 50240.5 5024050 790 99691 50240.5 5024050 -32380 32758 5020.66 502066 -128 127 -2.06 -206 +791 100 10781 99692 2.37537 299.37537 150.87537 15087.53753 2.37537 299.37537 150.87537 15087.53761 2.37537 299.37537 150.87536999999992 15087.53700 2020-01-01 2020-01-02 2020-01-01 00:13:11 2020-01-02 03:41:32 2020-01-01 00:13:11.000 2020-01-02 03:41:32.000 791 99692 50241.5 5024150 791 99692 50241.5 5024150 -32379 32759 5021.66 502166 -128 127 -3.62 -362 +792 100 10782 99693 2.37837 299.37837 150.87837 15087.83783 2.37837 299.3784 150.87837 15087.83775 2.37837 299.37837 150.87837000000007 15087.83700 2020-01-01 2020-01-02 2020-01-01 00:13:12 2020-01-02 03:41:33 2020-01-01 00:13:12.000 2020-01-02 03:41:33.000 792 99693 50242.5 5024250 792 99693 50242.5 5024250 -32378 32760 5022.66 502266 -128 123 -5.18 -518 793 100 10783 99694 2.38138 299.38138 150.88138 15088.13813 2.38138 299.38138 150.88137 15088.13789 2.38138 299.38138 150.88138 15088.13800 2020-01-01 2020-01-02 2020-01-01 00:13:13 2020-01-02 03:41:34 2020-01-01 00:13:13.000 2020-01-02 03:41:34.000 793 99694 50243.5 5024350 793 99694 50243.5 5024350 -32377 32761 5023.66 502366 -127 124 -4.18 -418 -794 100 10784 99695 2.38438 299.38438 150.88438 15088.43843 2.38438 299.3844 150.88438 15088.43864 2.38438 299.38438 150.88438 15088.43800 2020-01-01 2020-01-02 2020-01-01 00:13:14 2020-01-02 03:41:35 2020-01-01 00:13:14.000 2020-01-02 03:41:35.000 794 99695 50244.5 5024450 794 99695 50244.5 5024450 -32376 32762 5024.66 502466 -126 125 -3.18 -318 -795 100 10785 99696 2.38738 299.38738 150.88738 15088.73873 2.38738 299.3874 150.88738 15088.73894 2.38738 299.38738 150.88738 15088.73800 2020-01-01 2020-01-02 2020-01-01 00:13:15 2020-01-02 03:41:36 2020-01-01 00:13:15.000 2020-01-02 03:41:36.000 795 99696 50245.5 5024550 795 99696 50245.5 5024550 -32375 32763 5025.66 502566 -125 126 -2.18 -218 -796 100 10786 99697 2.39039 299.39039 150.89039 15089.03903 2.39039 299.39038 150.89039 15089.03908 2.39039 299.39039 150.89039 15089.03900 2020-01-01 2020-01-02 2020-01-01 00:13:16 2020-01-02 03:41:37 2020-01-01 00:13:16.000 2020-01-02 03:41:37.000 796 99697 50246.5 5024650 796 99697 50246.5 5024650 -32374 32764 5026.66 502666 -124 127 -1.18 -118 -797 100 10787 99698 2.39339 299.39339 150.89339 15089.33933 2.39339 299.3934 150.89339 15089.33921 2.39339 299.39339 150.89339 15089.33900 2020-01-01 2020-01-02 2020-01-01 00:13:17 2020-01-02 03:41:38 2020-01-01 00:13:17.000 2020-01-02 03:41:38.000 797 99698 50247.5 5024750 797 99698 50247.5 5024750 -32373 32765 5027.66 502766 -128 127 -2.74 -274 -798 100 10788 99699 2.39639 299.39639 150.89639 15089.63963 2.39639 299.3964 150.89639 15089.63936 2.39639 299.39639 150.89639 15089.63900 2020-01-01 2020-01-02 2020-01-01 00:13:18 2020-01-02 03:41:39 2020-01-01 00:13:18.000 2020-01-02 03:41:39.000 798 99699 50248.5 5024850 798 99699 50248.5 5024850 -32372 32766 5028.66 502866 -128 123 -4.3 -430 -799 100 10789 99700 2.39939 299.39939 150.89939 15089.93993 2.39939 299.3994 150.8994 15089.94011 2.39939 299.39939 150.89939 15089.93900 2020-01-01 2020-01-02 2020-01-01 00:13:19 2020-01-02 03:41:40 2020-01-01 00:13:19.000 2020-01-02 03:41:40.000 799 99700 50249.5 5024950 799 99700 50249.5 5024950 -32371 32767 5029.66 502966 -127 124 -3.3 -330 -8 102 1007 9998 0.02402 300.02402 150.02402 15152.42642 0.02402 300.02402 150.02402 15152.42607 0.02402 300.02402 150.02402 15152.42602 2020-01-01 2020-01-02 2020-01-01 00:00:08 2020-01-02 03:45:08 2020-01-01 00:00:08.000 2020-01-02 03:45:08.000 8 99908 49958 5045758 8 99908 49958 5045758 -32561 32374 4537.009900990099 458238 -125 126 -0.019801980198019802 -2 -80 102 10070 99980 0.24024 300.24024 150.24024 15174.26426 0.24024 300.24023 150.24023 15174.26397 0.24024 300.24024 150.24024 15174.26424 2020-01-01 2020-01-02 2020-01-01 00:01:20 2020-01-02 03:46:20 2020-01-01 00:01:20.000 2020-01-02 03:46:20.000 80 99980 50030 5053030 80 99980 50030 5053030 -32489 32446 4609.009900990099 465510 -128 123 -4.0594059405940595 -410 -800 100 10790 99701 2.4024 299.4024 150.9024 15090.24024 2.4024 299.4024 150.9024 15090.24041 2.40240 299.40240 150.90240 15090.24000 2020-01-01 2020-01-02 2020-01-01 00:13:20 2020-01-02 03:41:41 2020-01-01 00:13:20.000 2020-01-02 03:41:41.000 800 99701 50250.5 5025050 800 99701 50250.5 5025050 -32768 32167 4375.3 437530 -126 125 -2.3 -230 -801 100 10791 99702 2.4054 299.4054 150.9054 15090.54054 2.4054 299.4054 150.9054 15090.54058 2.40540 299.40540 150.90540 15090.54000 2020-01-01 2020-01-02 2020-01-01 00:13:21 2020-01-02 03:41:42 2020-01-01 00:13:21.000 2020-01-02 03:41:42.000 801 99702 50251.5 5025150 801 99702 50251.5 5025150 -32767 32168 4376.3 437630 -125 126 -1.3 -130 -802 100 10792 99703 2.4084 299.4084 150.9084 15090.84084 2.4084 299.40842 150.9084 15090.84069 2.40840 299.40840 150.90840 15090.84000 2020-01-01 2020-01-02 2020-01-01 00:13:22 2020-01-02 03:41:43 2020-01-01 00:13:22.000 2020-01-02 03:41:43.000 802 99703 50252.5 5025250 802 99703 50252.5 5025250 -32766 32169 4377.3 437730 -124 127 -0.3 -30 -803 100 10793 99704 2.41141 299.41141 150.91141 15091.14114 2.41141 299.4114 150.9114 15091.14098 2.41141 299.41141 150.91141 15091.14100 2020-01-01 2020-01-02 2020-01-01 00:13:23 2020-01-02 03:41:44 2020-01-01 00:13:23.000 2020-01-02 03:41:44.000 803 99704 50253.5 5025350 803 99704 50253.5 5025350 -32765 32170 4378.3 437830 -128 127 -1.86 -186 -804 100 10794 99705 2.41441 299.41441 150.91441 15091.44144 2.41441 299.41443 150.91441 15091.44158 2.41441 299.41441 150.91441 15091.44100 2020-01-01 2020-01-02 2020-01-01 00:13:24 2020-01-02 03:41:45 2020-01-01 00:13:24.000 2020-01-02 03:41:45.000 804 99705 50254.5 5025450 804 99705 50254.5 5025450 -32764 32171 4379.3 437930 -128 123 -3.42 -342 +794 100 10784 99695 2.38438 299.38438 150.88438 15088.43843 2.38438 299.3844 150.88438 15088.43864 2.38438 299.38438 150.88438000000002 15088.43800 2020-01-01 2020-01-02 2020-01-01 00:13:14 2020-01-02 03:41:35 2020-01-01 00:13:14.000 2020-01-02 03:41:35.000 794 99695 50244.5 5024450 794 99695 50244.5 5024450 -32376 32762 5024.66 502466 -126 125 -3.18 -318 +795 100 10785 99696 2.38738 299.38738 150.88738 15088.73873 2.38738 299.3874 150.88738 15088.73894 2.38738 299.38738 150.88738000000006 15088.73800 2020-01-01 2020-01-02 2020-01-01 00:13:15 2020-01-02 03:41:36 2020-01-01 00:13:15.000 2020-01-02 03:41:36.000 795 99696 50245.5 5024550 795 99696 50245.5 5024550 -32375 32763 5025.66 502566 -125 126 -2.18 -218 +796 100 10786 99697 2.39039 299.39039 150.89039 15089.03903 2.39039 299.39038 150.89039 15089.03908 2.39039 299.39039 150.89038999999997 15089.03900 2020-01-01 2020-01-02 2020-01-01 00:13:16 2020-01-02 03:41:37 2020-01-01 00:13:16.000 2020-01-02 03:41:37.000 796 99697 50246.5 5024650 796 99697 50246.5 5024650 -32374 32764 5026.66 502666 -124 127 -1.18 -118 +797 100 10787 99698 2.39339 299.39339 150.89339 15089.33933 2.39339 299.3934 150.89339 15089.33921 2.39339 299.39339 150.89338999999998 15089.33900 2020-01-01 2020-01-02 2020-01-01 00:13:17 2020-01-02 03:41:38 2020-01-01 00:13:17.000 2020-01-02 03:41:38.000 797 99698 50247.5 5024750 797 99698 50247.5 5024750 -32373 32765 5027.66 502766 -128 127 -2.74 -274 +798 100 10788 99699 2.39639 299.39639 150.89639 15089.63963 2.39639 299.3964 150.89639 15089.63936 2.39639 299.39639 150.8963899999999 15089.63900 2020-01-01 2020-01-02 2020-01-01 00:13:18 2020-01-02 03:41:39 2020-01-01 00:13:18.000 2020-01-02 03:41:39.000 798 99699 50248.5 5024850 798 99699 50248.5 5024850 -32372 32766 5028.66 502866 -128 123 -4.3 -430 +799 100 10789 99700 2.39939 299.39939 150.89939 15089.93993 2.39939 299.3994 150.8994 15089.94011 2.39939 299.39939 150.89938999999998 15089.93900 2020-01-01 2020-01-02 2020-01-01 00:13:19 2020-01-02 03:41:40 2020-01-01 00:13:19.000 2020-01-02 03:41:40.000 799 99700 50249.5 5024950 799 99700 50249.5 5024950 -32371 32767 5029.66 502966 -127 124 -3.3 -330 +8 102 1007 9998 0.02402 300.02402 150.02402 15152.42642 0.02402 300.02402 150.02402 15152.42607 0.02402 300.02402 150.02402000000004 15152.42602 2020-01-01 2020-01-02 2020-01-01 00:00:08 2020-01-02 03:45:08 2020-01-01 00:00:08.000 2020-01-02 03:45:08.000 8 99908 49958 5045758 8 99908 49958 5045758 -32561 32374 4537.009900990099 458238 -125 126 -0.019801980198019802 -2 +80 102 10070 99980 0.24024 300.24024 150.24024 15174.26426 0.24024 300.24023 150.24023 15174.26397 0.24024 300.24024 150.24023999999994 15174.26424 2020-01-01 2020-01-02 2020-01-01 00:01:20 2020-01-02 03:46:20 2020-01-01 00:01:20.000 2020-01-02 03:46:20.000 80 99980 50030 5053030 80 99980 50030 5053030 -32489 32446 4609.009900990099 465510 -128 123 -4.0594059405940595 -410 +800 100 10790 99701 2.4024 299.4024 150.9024 15090.24024 2.4024 299.4024 150.9024 15090.24041 2.40240 299.40240 150.90240000000006 15090.24000 2020-01-01 2020-01-02 2020-01-01 00:13:20 2020-01-02 03:41:41 2020-01-01 00:13:20.000 2020-01-02 03:41:41.000 800 99701 50250.5 5025050 800 99701 50250.5 5025050 -32768 32167 4375.3 437530 -126 125 -2.3 -230 +801 100 10791 99702 2.4054 299.4054 150.9054 15090.54054 2.4054 299.4054 150.9054 15090.54058 2.40540 299.40540 150.90539999999993 15090.54000 2020-01-01 2020-01-02 2020-01-01 00:13:21 2020-01-02 03:41:42 2020-01-01 00:13:21.000 2020-01-02 03:41:42.000 801 99702 50251.5 5025150 801 99702 50251.5 5025150 -32767 32168 4376.3 437630 -125 126 -1.3 -130 +802 100 10792 99703 2.4084 299.4084 150.9084 15090.84084 2.4084 299.40842 150.9084 15090.84069 2.40840 299.40840 150.90840000000003 15090.84000 2020-01-01 2020-01-02 2020-01-01 00:13:22 2020-01-02 03:41:43 2020-01-01 00:13:22.000 2020-01-02 03:41:43.000 802 99703 50252.5 5025250 802 99703 50252.5 5025250 -32766 32169 4377.3 437730 -124 127 -0.3 -30 +803 100 10793 99704 2.41141 299.41141 150.91141 15091.14114 2.41141 299.4114 150.9114 15091.14098 2.41141 299.41141 150.91141000000005 15091.14100 2020-01-01 2020-01-02 2020-01-01 00:13:23 2020-01-02 03:41:44 2020-01-01 00:13:23.000 2020-01-02 03:41:44.000 803 99704 50253.5 5025350 803 99704 50253.5 5025350 -32765 32170 4378.3 437830 -128 127 -1.86 -186 +804 100 10794 99705 2.41441 299.41441 150.91441 15091.44144 2.41441 299.41443 150.91441 15091.44158 2.41441 299.41441 150.91440999999998 15091.44100 2020-01-01 2020-01-02 2020-01-01 00:13:24 2020-01-02 03:41:45 2020-01-01 00:13:24.000 2020-01-02 03:41:45.000 804 99705 50254.5 5025450 804 99705 50254.5 5025450 -32764 32171 4379.3 437930 -128 123 -3.42 -342 805 100 10795 99706 2.41741 299.41741 150.91741 15091.74174 2.41741 299.41742 150.91741 15091.74188 2.41741 299.41741 150.91741 15091.74100 2020-01-01 2020-01-02 2020-01-01 00:13:25 2020-01-02 03:41:46 2020-01-01 00:13:25.000 2020-01-02 03:41:46.000 805 99706 50255.5 5025550 805 99706 50255.5 5025550 -32763 32172 4380.3 438030 -127 124 -2.42 -242 -806 100 10796 99707 2.42042 299.42042 150.92042 15092.04204 2.42042 299.4204 150.92042 15092.04205 2.42042 299.42042 150.92042 15092.04200 2020-01-01 2020-01-02 2020-01-01 00:13:26 2020-01-02 03:41:47 2020-01-01 00:13:26.000 2020-01-02 03:41:47.000 806 99707 50256.5 5025650 806 99707 50256.5 5025650 -32762 32173 4381.3 438130 -126 125 -1.42 -142 -807 100 10797 99708 2.42342 299.42342 150.92342 15092.34234 2.42342 299.42343 150.92342 15092.34216 2.42342 299.42342 150.92342 15092.34200 2020-01-01 2020-01-02 2020-01-01 00:13:27 2020-01-02 03:41:48 2020-01-01 00:13:27.000 2020-01-02 03:41:48.000 807 99708 50257.5 5025750 807 99708 50257.5 5025750 -32761 32174 4382.3 438230 -125 126 -0.42 -42 -808 100 10798 99709 2.42642 299.42642 150.92642 15092.64264 2.42642 299.42642 150.92642 15092.64245 2.42642 299.42642 150.92642 15092.64200 2020-01-01 2020-01-02 2020-01-01 00:13:28 2020-01-02 03:41:49 2020-01-01 00:13:28.000 2020-01-02 03:41:49.000 808 99709 50258.5 5025850 808 99709 50258.5 5025850 -32760 32175 4383.3 438330 -124 127 0.58 58 -809 100 10799 99710 2.42942 299.42942 150.92942 15092.94294 2.42942 299.42944 150.92943 15092.94305 2.42942 299.42942 150.92942 15092.94200 2020-01-01 2020-01-02 2020-01-01 00:13:29 2020-01-02 03:41:50 2020-01-01 00:13:29.000 2020-01-02 03:41:50.000 809 99710 50259.5 5025950 809 99710 50259.5 5025950 -32759 32176 4384.3 438430 -128 127 -0.98 -98 -81 102 10071 99981 0.24324 300.24324 150.24324 15174.56756 0.24324 300.24326 150.24324 15174.56758 0.24324 300.24324 150.24324 15174.56724 2020-01-01 2020-01-02 2020-01-01 00:01:21 2020-01-02 03:46:21 2020-01-01 00:01:21.000 2020-01-02 03:46:21.000 81 99981 50031 5053131 81 99981 50031 5053131 -32488 32447 4610.009900990099 465611 -127 124 -3.0594059405940595 -309 -810 100 10800 99711 2.43243 299.43243 150.93243 15093.24324 2.43243 299.43243 150.93243 15093.24338 2.43243 299.43243 150.93243 15093.24300 2020-01-01 2020-01-02 2020-01-01 00:13:30 2020-01-02 03:41:51 2020-01-01 00:13:30.000 2020-01-02 03:41:51.000 810 99711 50260.5 5026050 810 99711 50260.5 5026050 -32758 32177 4385.3 438530 -128 127 -2.54 -254 -811 100 10801 99712 2.43543 299.43543 150.93543 15093.54354 2.43543 299.43542 150.93543 15093.54353 2.43543 299.43543 150.93543 15093.54300 2020-01-01 2020-01-02 2020-01-01 00:13:31 2020-01-02 03:41:52 2020-01-01 00:13:31.000 2020-01-02 03:41:52.000 811 99712 50261.5 5026150 811 99712 50261.5 5026150 -32757 32178 4386.3 438630 -128 124 -4.1 -410 -812 100 10802 99713 2.43843 299.43843 150.93843 15093.84384 2.43843 299.43845 150.93844 15093.84428 2.43843 299.43843 150.93843 15093.84300 2020-01-01 2020-01-02 2020-01-01 00:13:32 2020-01-02 03:41:53 2020-01-01 00:13:32.000 2020-01-02 03:41:53.000 812 99713 50262.5 5026250 812 99713 50262.5 5026250 -32756 32179 4387.3 438730 -127 125 -3.1 -310 -813 100 10803 99714 2.44144 299.44144 150.94144 15094.14414 2.44144 299.44144 150.94143 15094.14392 2.44144 299.44144 150.94144 15094.14400 2020-01-01 2020-01-02 2020-01-01 00:13:33 2020-01-02 03:41:54 2020-01-01 00:13:33.000 2020-01-02 03:41:54.000 813 99714 50263.5 5026350 813 99714 50263.5 5026350 -32755 32180 4388.3 438830 -126 126 -2.1 -210 +806 100 10796 99707 2.42042 299.42042 150.92042 15092.04204 2.42042 299.4204 150.92042 15092.04205 2.42042 299.42042 150.9204200000001 15092.04200 2020-01-01 2020-01-02 2020-01-01 00:13:26 2020-01-02 03:41:47 2020-01-01 00:13:26.000 2020-01-02 03:41:47.000 806 99707 50256.5 5025650 806 99707 50256.5 5025650 -32762 32173 4381.3 438130 -126 125 -1.42 -142 +807 100 10797 99708 2.42342 299.42342 150.92342 15092.34234 2.42342 299.42343 150.92342 15092.34216 2.42342 299.42342 150.92341999999994 15092.34200 2020-01-01 2020-01-02 2020-01-01 00:13:27 2020-01-02 03:41:48 2020-01-01 00:13:27.000 2020-01-02 03:41:48.000 807 99708 50257.5 5025750 807 99708 50257.5 5025750 -32761 32174 4382.3 438230 -125 126 -0.42 -42 +808 100 10798 99709 2.42642 299.42642 150.92642 15092.64264 2.42642 299.42642 150.92642 15092.64245 2.42642 299.42642 150.9264199999999 15092.64200 2020-01-01 2020-01-02 2020-01-01 00:13:28 2020-01-02 03:41:49 2020-01-01 00:13:28.000 2020-01-02 03:41:49.000 808 99709 50258.5 5025850 808 99709 50258.5 5025850 -32760 32175 4383.3 438330 -124 127 0.58 58 +809 100 10799 99710 2.42942 299.42942 150.92942 15092.94294 2.42942 299.42944 150.92943 15092.94305 2.42942 299.42942 150.9294200000001 15092.94200 2020-01-01 2020-01-02 2020-01-01 00:13:29 2020-01-02 03:41:50 2020-01-01 00:13:29.000 2020-01-02 03:41:50.000 809 99710 50259.5 5025950 809 99710 50259.5 5025950 -32759 32176 4384.3 438430 -128 127 -0.98 -98 +81 102 10071 99981 0.24324 300.24324 150.24324 15174.56756 0.24324 300.24326 150.24324 15174.56758 0.24324 300.24324 150.2432399999999 15174.56724 2020-01-01 2020-01-02 2020-01-01 00:01:21 2020-01-02 03:46:21 2020-01-01 00:01:21.000 2020-01-02 03:46:21.000 81 99981 50031 5053131 81 99981 50031 5053131 -32488 32447 4610.009900990099 465611 -127 124 -3.0594059405940595 -309 +810 100 10800 99711 2.43243 299.43243 150.93243 15093.24324 2.43243 299.43243 150.93243 15093.24338 2.43243 299.43243 150.93243000000004 15093.24300 2020-01-01 2020-01-02 2020-01-01 00:13:30 2020-01-02 03:41:51 2020-01-01 00:13:30.000 2020-01-02 03:41:51.000 810 99711 50260.5 5026050 810 99711 50260.5 5026050 -32758 32177 4385.3 438530 -128 127 -2.54 -254 +811 100 10801 99712 2.43543 299.43543 150.93543 15093.54354 2.43543 299.43542 150.93543 15093.54353 2.43543 299.43543 150.93542999999983 15093.54300 2020-01-01 2020-01-02 2020-01-01 00:13:31 2020-01-02 03:41:52 2020-01-01 00:13:31.000 2020-01-02 03:41:52.000 811 99712 50261.5 5026150 811 99712 50261.5 5026150 -32757 32178 4386.3 438630 -128 124 -4.1 -410 +812 100 10802 99713 2.43843 299.43843 150.93843 15093.84384 2.43843 299.43845 150.93844 15093.84428 2.43843 299.43843 150.93843000000007 15093.84300 2020-01-01 2020-01-02 2020-01-01 00:13:32 2020-01-02 03:41:53 2020-01-01 00:13:32.000 2020-01-02 03:41:53.000 812 99713 50262.5 5026250 812 99713 50262.5 5026250 -32756 32179 4387.3 438730 -127 125 -3.1 -310 +813 100 10803 99714 2.44144 299.44144 150.94144 15094.14414 2.44144 299.44144 150.94143 15094.14392 2.44144 299.44144 150.94143999999997 15094.14400 2020-01-01 2020-01-02 2020-01-01 00:13:33 2020-01-02 03:41:54 2020-01-01 00:13:33.000 2020-01-02 03:41:54.000 813 99714 50263.5 5026350 813 99714 50263.5 5026350 -32755 32180 4388.3 438830 -126 126 -2.1 -210 814 100 10804 99715 2.44444 299.44444 150.94444 15094.44444 2.44444 299.44446 150.94444 15094.44452 2.44444 299.44444 150.94444 15094.44400 2020-01-01 2020-01-02 2020-01-01 00:13:34 2020-01-02 03:41:55 2020-01-01 00:13:34.000 2020-01-02 03:41:55.000 814 99715 50264.5 5026450 814 99715 50264.5 5026450 -32754 32181 4389.3 438930 -125 127 -1.1 -110 -815 100 10805 99716 2.44744 299.44744 150.94744 15094.74474 2.44744 299.44745 150.94744 15094.74485 2.44744 299.44744 150.94744 15094.74400 2020-01-01 2020-01-02 2020-01-01 00:13:35 2020-01-02 03:41:56 2020-01-01 00:13:35.000 2020-01-02 03:41:56.000 815 99716 50265.5 5026550 815 99716 50265.5 5026550 -32753 32182 4390.3 439030 -128 127 -2.66 -266 -816 100 10806 99717 2.45045 299.45045 150.95045 15095.04504 2.45045 299.45044 150.95045 15095.045 2.45045 299.45045 150.95045 15095.04500 2020-01-01 2020-01-02 2020-01-01 00:13:36 2020-01-02 03:41:57 2020-01-01 00:13:36.000 2020-01-02 03:41:57.000 816 99717 50266.5 5026650 816 99717 50266.5 5026650 -32752 32183 4391.3 439130 -128 127 -4.22 -422 -817 100 10807 99718 2.45345 299.45345 150.95345 15095.34534 2.45345 299.45346 150.95345 15095.34574 2.45345 299.45345 150.95345 15095.34500 2020-01-01 2020-01-02 2020-01-01 00:13:37 2020-01-02 03:41:58 2020-01-01 00:13:37.000 2020-01-02 03:41:58.000 817 99718 50267.5 5026750 817 99718 50267.5 5026750 -32751 32184 4392.3 439230 -128 123 -5.78 -578 -818 100 10808 99719 2.45645 299.45645 150.95645 15095.64564 2.45645 299.45645 150.95645 15095.64539 2.45645 299.45645 150.95645 15095.64500 2020-01-01 2020-01-02 2020-01-01 00:13:38 2020-01-02 03:41:59 2020-01-01 00:13:38.000 2020-01-02 03:41:59.000 818 99719 50268.5 5026850 818 99719 50268.5 5026850 -32750 32185 4393.3 439330 -127 124 -4.78 -478 -819 100 10809 99720 2.45945 299.45945 150.95945 15095.94594 2.45945 299.45947 150.95946 15095.94602 2.45945 299.45945 150.95945 15095.94500 2020-01-01 2020-01-02 2020-01-01 00:13:39 2020-01-02 03:42:00 2020-01-01 00:13:39.000 2020-01-02 03:42:00.000 819 99720 50269.5 5026950 819 99720 50269.5 5026950 -32749 32186 4394.3 439430 -126 125 -3.78 -378 -82 102 10072 99982 0.24624 300.24624 150.24624 15174.87087 0.24624 300.24625 150.24624 15174.87088 0.24624 300.24624 150.24624 15174.87024 2020-01-01 2020-01-02 2020-01-01 00:01:22 2020-01-02 03:46:22 2020-01-01 00:01:22.000 2020-01-02 03:46:22.000 82 99982 50032 5053232 82 99982 50032 5053232 -32487 32448 4611.009900990099 465712 -126 125 -2.0594059405940595 -208 -820 100 10810 99721 2.46246 299.46246 150.96246 15096.24624 2.46246 299.46246 150.96246 15096.24633 2.46246 299.46246 150.96246 15096.24600 2020-01-01 2020-01-02 2020-01-01 00:13:40 2020-01-02 03:42:01 2020-01-01 00:13:40.000 2020-01-02 03:42:01.000 820 99721 50270.5 5027050 820 99721 50270.5 5027050 -32748 32187 4395.3 439530 -125 126 -2.78 -278 -821 100 10811 99722 2.46546 299.46546 150.96546 15096.54654 2.46546 299.46545 150.96546 15096.54646 2.46546 299.46546 150.96546 15096.54600 2020-01-01 2020-01-02 2020-01-01 00:13:41 2020-01-02 03:42:02 2020-01-01 00:13:41.000 2020-01-02 03:42:02.000 821 99722 50271.5 5027150 821 99722 50271.5 5027150 -32747 32188 4396.3 439630 -124 127 -1.78 -178 -822 100 10812 99723 2.46846 299.46846 150.96846 15096.84684 2.46846 299.46848 150.96847 15096.84721 2.46846 299.46846 150.96846 15096.84600 2020-01-01 2020-01-02 2020-01-01 00:13:42 2020-01-02 03:42:03 2020-01-01 00:13:42.000 2020-01-02 03:42:03.000 822 99723 50272.5 5027250 822 99723 50272.5 5027250 -32746 32189 4397.3 439730 -128 127 -3.34 -334 -823 100 10813 99724 2.47147 299.47147 150.97147 15097.14714 2.47147 299.47147 150.97146 15097.14686 2.47147 299.47147 150.97147 15097.14700 2020-01-01 2020-01-02 2020-01-01 00:13:43 2020-01-02 03:42:04 2020-01-01 00:13:43.000 2020-01-02 03:42:04.000 823 99724 50273.5 5027350 823 99724 50273.5 5027350 -32745 32190 4398.3 439830 -128 123 -4.9 -490 -824 100 10814 99725 2.47447 299.47447 150.97447 15097.44744 2.47447 299.4745 150.97447 15097.44749 2.47447 299.47447 150.97447 15097.44700 2020-01-01 2020-01-02 2020-01-01 00:13:44 2020-01-02 03:42:05 2020-01-01 00:13:44.000 2020-01-02 03:42:05.000 824 99725 50274.5 5027450 824 99725 50274.5 5027450 -32744 32191 4399.3 439930 -127 124 -3.9 -390 -825 100 10815 99726 2.47747 299.47747 150.97747 15097.74774 2.47747 299.47748 150.97747 15097.74779 2.47747 299.47747 150.97747 15097.74700 2020-01-01 2020-01-02 2020-01-01 00:13:45 2020-01-02 03:42:06 2020-01-01 00:13:45.000 2020-01-02 03:42:06.000 825 99726 50275.5 5027550 825 99726 50275.5 5027550 -32743 32192 4400.3 440030 -126 125 -2.9 -290 -826 100 10816 99727 2.48048 299.48048 150.98048 15098.04804 2.48048 299.48047 150.98048 15098.04809 2.48048 299.48048 150.98048 15098.04800 2020-01-01 2020-01-02 2020-01-01 00:13:46 2020-01-02 03:42:07 2020-01-01 00:13:46.000 2020-01-02 03:42:07.000 826 99727 50276.5 5027650 826 99727 50276.5 5027650 -32742 32193 4401.3 440130 -125 126 -1.9 -190 -827 100 10817 99728 2.48348 299.48348 150.98348 15098.34834 2.48348 299.4835 150.98348 15098.34869 2.48348 299.48348 150.98348 15098.34800 2020-01-01 2020-01-02 2020-01-01 00:13:47 2020-01-02 03:42:08 2020-01-01 00:13:47.000 2020-01-02 03:42:08.000 827 99728 50277.5 5027750 827 99728 50277.5 5027750 -32741 32194 4402.3 440230 -124 127 -0.9 -90 -828 100 10818 99729 2.48648 299.48648 150.98648 15098.64864 2.48648 299.48648 150.98648 15098.64837 2.48648 299.48648 150.98648 15098.64800 2020-01-01 2020-01-02 2020-01-01 00:13:48 2020-01-02 03:42:09 2020-01-01 00:13:48.000 2020-01-02 03:42:09.000 828 99729 50278.5 5027850 828 99729 50278.5 5027850 -32740 32195 4403.3 440330 -128 127 -2.46 -246 -829 100 10819 99730 2.48948 299.48948 150.98948 15098.94894 2.48948 299.4895 150.98948 15098.94896 2.48948 299.48948 150.98948 15098.94800 2020-01-01 2020-01-02 2020-01-01 00:13:49 2020-01-02 03:42:10 2020-01-01 00:13:49.000 2020-01-02 03:42:10.000 829 99730 50279.5 5027950 829 99730 50279.5 5027950 -32739 32196 4404.3 440430 -128 123 -4.02 -402 -83 102 10073 99983 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924 15175.17324 2020-01-01 2020-01-02 2020-01-01 00:01:23 2020-01-02 03:46:23 2020-01-01 00:01:23.000 2020-01-02 03:46:23.000 83 99983 50033 5053333 83 99983 50033 5053333 -32486 32449 4612.009900990099 465813 -125 126 -1.0594059405940595 -107 +815 100 10805 99716 2.44744 299.44744 150.94744 15094.74474 2.44744 299.44745 150.94744 15094.74485 2.44744 299.44744 150.94743999999994 15094.74400 2020-01-01 2020-01-02 2020-01-01 00:13:35 2020-01-02 03:41:56 2020-01-01 00:13:35.000 2020-01-02 03:41:56.000 815 99716 50265.5 5026550 815 99716 50265.5 5026550 -32753 32182 4390.3 439030 -128 127 -2.66 -266 +816 100 10806 99717 2.45045 299.45045 150.95045 15095.04504 2.45045 299.45044 150.95045 15095.045 2.45045 299.45045 150.95045000000005 15095.04500 2020-01-01 2020-01-02 2020-01-01 00:13:36 2020-01-02 03:41:57 2020-01-01 00:13:36.000 2020-01-02 03:41:57.000 816 99717 50266.5 5026650 816 99717 50266.5 5026650 -32752 32183 4391.3 439130 -128 127 -4.22 -422 +817 100 10807 99718 2.45345 299.45345 150.95345 15095.34534 2.45345 299.45346 150.95345 15095.34574 2.45345 299.45345 150.9534500000001 15095.34500 2020-01-01 2020-01-02 2020-01-01 00:13:37 2020-01-02 03:41:58 2020-01-01 00:13:37.000 2020-01-02 03:41:58.000 817 99718 50267.5 5026750 817 99718 50267.5 5026750 -32751 32184 4392.3 439230 -128 123 -5.78 -578 +818 100 10808 99719 2.45645 299.45645 150.95645 15095.64564 2.45645 299.45645 150.95645 15095.64539 2.45645 299.45645 150.9564499999999 15095.64500 2020-01-01 2020-01-02 2020-01-01 00:13:38 2020-01-02 03:41:59 2020-01-01 00:13:38.000 2020-01-02 03:41:59.000 818 99719 50268.5 5026850 818 99719 50268.5 5026850 -32750 32185 4393.3 439330 -127 124 -4.78 -478 +819 100 10809 99720 2.45945 299.45945 150.95945 15095.94594 2.45945 299.45947 150.95946 15095.94602 2.45945 299.45945 150.95945000000006 15095.94500 2020-01-01 2020-01-02 2020-01-01 00:13:39 2020-01-02 03:42:00 2020-01-01 00:13:39.000 2020-01-02 03:42:00.000 819 99720 50269.5 5026950 819 99720 50269.5 5026950 -32749 32186 4394.3 439430 -126 125 -3.78 -378 +82 102 10072 99982 0.24624 300.24624 150.24624 15174.87087 0.24624 300.24625 150.24624 15174.87088 0.24624 300.24624 150.2462400000001 15174.87024 2020-01-01 2020-01-02 2020-01-01 00:01:22 2020-01-02 03:46:22 2020-01-01 00:01:22.000 2020-01-02 03:46:22.000 82 99982 50032 5053232 82 99982 50032 5053232 -32487 32448 4611.009900990099 465712 -126 125 -2.0594059405940595 -208 +820 100 10810 99721 2.46246 299.46246 150.96246 15096.24624 2.46246 299.46246 150.96246 15096.24633 2.46246 299.46246 150.96246000000008 15096.24600 2020-01-01 2020-01-02 2020-01-01 00:13:40 2020-01-02 03:42:01 2020-01-01 00:13:40.000 2020-01-02 03:42:01.000 820 99721 50270.5 5027050 820 99721 50270.5 5027050 -32748 32187 4395.3 439530 -125 126 -2.78 -278 +821 100 10811 99722 2.46546 299.46546 150.96546 15096.54654 2.46546 299.46545 150.96546 15096.54646 2.46546 299.46546 150.96545999999998 15096.54600 2020-01-01 2020-01-02 2020-01-01 00:13:41 2020-01-02 03:42:02 2020-01-01 00:13:41.000 2020-01-02 03:42:02.000 821 99722 50271.5 5027150 821 99722 50271.5 5027150 -32747 32188 4396.3 439630 -124 127 -1.78 -178 +822 100 10812 99723 2.46846 299.46846 150.96846 15096.84684 2.46846 299.46848 150.96847 15096.84721 2.46846 299.46846 150.96846000000002 15096.84600 2020-01-01 2020-01-02 2020-01-01 00:13:42 2020-01-02 03:42:03 2020-01-01 00:13:42.000 2020-01-02 03:42:03.000 822 99723 50272.5 5027250 822 99723 50272.5 5027250 -32746 32189 4397.3 439730 -128 127 -3.34 -334 +823 100 10813 99724 2.47147 299.47147 150.97147 15097.14714 2.47147 299.47147 150.97146 15097.14686 2.47147 299.47147 150.97147000000012 15097.14700 2020-01-01 2020-01-02 2020-01-01 00:13:43 2020-01-02 03:42:04 2020-01-01 00:13:43.000 2020-01-02 03:42:04.000 823 99724 50273.5 5027350 823 99724 50273.5 5027350 -32745 32190 4398.3 439830 -128 123 -4.9 -490 +824 100 10814 99725 2.47447 299.47447 150.97447 15097.44744 2.47447 299.4745 150.97447 15097.44749 2.47447 299.47447 150.97446999999997 15097.44700 2020-01-01 2020-01-02 2020-01-01 00:13:44 2020-01-02 03:42:05 2020-01-01 00:13:44.000 2020-01-02 03:42:05.000 824 99725 50274.5 5027450 824 99725 50274.5 5027450 -32744 32191 4399.3 439930 -127 124 -3.9 -390 +825 100 10815 99726 2.47747 299.47747 150.97747 15097.74774 2.47747 299.47748 150.97747 15097.74779 2.47747 299.47747 150.9774699999999 15097.74700 2020-01-01 2020-01-02 2020-01-01 00:13:45 2020-01-02 03:42:06 2020-01-01 00:13:45.000 2020-01-02 03:42:06.000 825 99726 50275.5 5027550 825 99726 50275.5 5027550 -32743 32192 4400.3 440030 -126 125 -2.9 -290 +826 100 10816 99727 2.48048 299.48048 150.98048 15098.04804 2.48048 299.48047 150.98048 15098.04809 2.48048 299.48048 150.98048000000009 15098.04800 2020-01-01 2020-01-02 2020-01-01 00:13:46 2020-01-02 03:42:07 2020-01-01 00:13:46.000 2020-01-02 03:42:07.000 826 99727 50276.5 5027650 826 99727 50276.5 5027650 -32742 32193 4401.3 440130 -125 126 -1.9 -190 +827 100 10817 99728 2.48348 299.48348 150.98348 15098.34834 2.48348 299.4835 150.98348 15098.34869 2.48348 299.48348 150.98348000000004 15098.34800 2020-01-01 2020-01-02 2020-01-01 00:13:47 2020-01-02 03:42:08 2020-01-01 00:13:47.000 2020-01-02 03:42:08.000 827 99728 50277.5 5027750 827 99728 50277.5 5027750 -32741 32194 4402.3 440230 -124 127 -0.9 -90 +828 100 10818 99729 2.48648 299.48648 150.98648 15098.64864 2.48648 299.48648 150.98648 15098.64837 2.48648 299.48648 150.98647999999986 15098.64800 2020-01-01 2020-01-02 2020-01-01 00:13:48 2020-01-02 03:42:09 2020-01-01 00:13:48.000 2020-01-02 03:42:09.000 828 99729 50278.5 5027850 828 99729 50278.5 5027850 -32740 32195 4403.3 440330 -128 127 -2.46 -246 +829 100 10819 99730 2.48948 299.48948 150.98948 15098.94894 2.48948 299.4895 150.98948 15098.94896 2.48948 299.48948 150.9894800000001 15098.94800 2020-01-01 2020-01-02 2020-01-01 00:13:49 2020-01-02 03:42:10 2020-01-01 00:13:49.000 2020-01-02 03:42:10.000 829 99730 50279.5 5027950 829 99730 50279.5 5027950 -32739 32196 4404.3 440430 -128 123 -4.02 -402 +83 102 10073 99983 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24923999999996 15175.17324 2020-01-01 2020-01-02 2020-01-01 00:01:23 2020-01-02 03:46:23 2020-01-01 00:01:23.000 2020-01-02 03:46:23.000 83 99983 50033 5053333 83 99983 50033 5053333 -32486 32449 4612.009900990099 465813 -125 126 -1.0594059405940595 -107 830 100 10820 99731 2.49249 299.49249 150.99249 15099.24924 2.49249 299.4925 150.99249 15099.24926 2.49249 299.49249 150.99249 15099.24900 2020-01-01 2020-01-02 2020-01-01 00:13:50 2020-01-02 03:42:11 2020-01-01 00:13:50.000 2020-01-02 03:42:11.000 830 99731 50280.5 5028050 830 99731 50280.5 5028050 -32738 32197 4405.3 440530 -127 124 -3.02 -302 -831 100 10821 99732 2.49549 299.49549 150.99549 15099.54954 2.49549 299.49548 150.99549 15099.54956 2.49549 299.49549 150.99549 15099.54900 2020-01-01 2020-01-02 2020-01-01 00:13:51 2020-01-02 03:42:12 2020-01-01 00:13:51.000 2020-01-02 03:42:12.000 831 99732 50281.5 5028150 831 99732 50281.5 5028150 -32737 32198 4406.3 440630 -126 125 -2.02 -202 -832 100 10822 99733 2.49849 299.49849 150.99849 15099.84984 2.49849 299.4985 150.9985 15099.85016 2.49849 299.49849 150.99849 15099.84900 2020-01-01 2020-01-02 2020-01-01 00:13:52 2020-01-02 03:42:13 2020-01-01 00:13:52.000 2020-01-02 03:42:13.000 832 99733 50282.5 5028250 832 99733 50282.5 5028250 -32736 32199 4407.3 440730 -125 126 -1.02 -102 -833 100 10823 99734 2.5015 299.5015 151.0015 15100.15015 2.5015 299.5015 151.00149 15100.14983 2.50150 299.50150 151.00150 15100.15000 2020-01-01 2020-01-02 2020-01-01 00:13:53 2020-01-02 03:42:14 2020-01-01 00:13:53.000 2020-01-02 03:42:14.000 833 99734 50283.5 5028350 833 99734 50283.5 5028350 -32735 32200 4408.3 440830 -124 127 -0.02 -2 -834 100 10824 99735 2.5045 299.5045 151.0045 15100.45045 2.5045 299.50452 151.0045 15100.45043 2.50450 299.50450 151.00450 15100.45000 2020-01-01 2020-01-02 2020-01-01 00:13:54 2020-01-02 03:42:15 2020-01-01 00:13:54.000 2020-01-02 03:42:15.000 834 99735 50284.5 5028450 834 99735 50284.5 5028450 -32734 32201 4409.3 440930 -128 127 -1.58 -158 -835 100 10825 99736 2.5075 299.5075 151.0075 15100.75075 2.5075 299.5075 151.0075 15100.75073 2.50750 299.50750 151.00750 15100.75000 2020-01-01 2020-01-02 2020-01-01 00:13:55 2020-01-02 03:42:16 2020-01-01 00:13:55.000 2020-01-02 03:42:16.000 835 99736 50285.5 5028550 835 99736 50285.5 5028550 -32733 32202 4410.3 441030 -128 123 -3.14 -314 -836 100 10826 99737 2.51051 299.51051 151.01051 15101.05105 2.51051 299.5105 151.01051 15101.05103 2.51051 299.51051 151.01051 15101.05100 2020-01-01 2020-01-02 2020-01-01 00:13:56 2020-01-02 03:42:17 2020-01-01 00:13:56.000 2020-01-02 03:42:17.000 836 99737 50286.5 5028650 836 99737 50286.5 5028650 -32732 32203 4411.3 441130 -127 124 -2.14 -214 +831 100 10821 99732 2.49549 299.49549 150.99549 15099.54954 2.49549 299.49548 150.99549 15099.54956 2.49549 299.49549 150.99548999999996 15099.54900 2020-01-01 2020-01-02 2020-01-01 00:13:51 2020-01-02 03:42:12 2020-01-01 00:13:51.000 2020-01-02 03:42:12.000 831 99732 50281.5 5028150 831 99732 50281.5 5028150 -32737 32198 4406.3 440630 -126 125 -2.02 -202 +832 100 10822 99733 2.49849 299.49849 150.99849 15099.84984 2.49849 299.4985 150.9985 15099.85016 2.49849 299.49849 150.99848999999998 15099.84900 2020-01-01 2020-01-02 2020-01-01 00:13:52 2020-01-02 03:42:13 2020-01-01 00:13:52.000 2020-01-02 03:42:13.000 832 99733 50282.5 5028250 832 99733 50282.5 5028250 -32736 32199 4407.3 440730 -125 126 -1.02 -102 +833 100 10823 99734 2.5015 299.5015 151.0015 15100.15015 2.5015 299.5015 151.00149 15100.14983 2.50150 299.50150 151.00150000000008 15100.15000 2020-01-01 2020-01-02 2020-01-01 00:13:53 2020-01-02 03:42:14 2020-01-01 00:13:53.000 2020-01-02 03:42:14.000 833 99734 50283.5 5028350 833 99734 50283.5 5028350 -32735 32200 4408.3 440830 -124 127 -0.02 -2 +834 100 10824 99735 2.5045 299.5045 151.0045 15100.45045 2.5045 299.50452 151.0045 15100.45043 2.50450 299.50450 151.00449999999992 15100.45000 2020-01-01 2020-01-02 2020-01-01 00:13:54 2020-01-02 03:42:15 2020-01-01 00:13:54.000 2020-01-02 03:42:15.000 834 99735 50284.5 5028450 834 99735 50284.5 5028450 -32734 32201 4409.3 440930 -128 127 -1.58 -158 +835 100 10825 99736 2.5075 299.5075 151.0075 15100.75075 2.5075 299.5075 151.0075 15100.75073 2.50750 299.50750 151.00749999999994 15100.75000 2020-01-01 2020-01-02 2020-01-01 00:13:55 2020-01-02 03:42:16 2020-01-01 00:13:55.000 2020-01-02 03:42:16.000 835 99736 50285.5 5028550 835 99736 50285.5 5028550 -32733 32202 4410.3 441030 -128 123 -3.14 -314 +836 100 10826 99737 2.51051 299.51051 151.01051 15101.05105 2.51051 299.5105 151.01051 15101.05103 2.51051 299.51051 151.01051000000004 15101.05100 2020-01-01 2020-01-02 2020-01-01 00:13:56 2020-01-02 03:42:17 2020-01-01 00:13:56.000 2020-01-02 03:42:17.000 836 99737 50286.5 5028650 836 99737 50286.5 5028650 -32732 32203 4411.3 441130 -127 124 -2.14 -214 837 100 10827 99738 2.51351 299.51351 151.01351 15101.35135 2.51351 299.51352 151.01351 15101.35162 2.51351 299.51351 151.01351 15101.35100 2020-01-01 2020-01-02 2020-01-01 00:13:57 2020-01-02 03:42:18 2020-01-01 00:13:57.000 2020-01-02 03:42:18.000 837 99738 50287.5 5028750 837 99738 50287.5 5028750 -32731 32204 4412.3 441230 -126 125 -1.14 -114 838 100 10828 99739 2.51651 299.51651 151.01651 15101.65165 2.51651 299.5165 151.01651 15101.6513 2.51651 299.51651 151.01651 15101.65100 2020-01-01 2020-01-02 2020-01-01 00:13:58 2020-01-02 03:42:19 2020-01-01 00:13:58.000 2020-01-02 03:42:19.000 838 99739 50288.5 5028850 838 99739 50288.5 5028850 -32730 32205 4413.3 441330 -125 126 -0.14 -14 -839 100 10829 99740 2.51951 299.51951 151.01951 15101.95195 2.51951 299.51953 151.01951 15101.9519 2.51951 299.51951 151.01951 15101.95100 2020-01-01 2020-01-02 2020-01-01 00:13:59 2020-01-02 03:42:20 2020-01-01 00:13:59.000 2020-01-02 03:42:20.000 839 99740 50289.5 5028950 839 99740 50289.5 5028950 -32729 32206 4414.3 441430 -124 127 0.86 86 -84 102 10074 99984 0.25225 300.25225 150.25225 15175.47747 0.25225 300.25226 150.25225 15175.47778 0.25225 300.25225 150.25225 15175.47725 2020-01-01 2020-01-02 2020-01-01 00:01:24 2020-01-02 03:46:24 2020-01-01 00:01:24.000 2020-01-02 03:46:24.000 84 99984 50034 5053434 84 99984 50034 5053434 -32485 32450 4613.009900990099 465914 -124 127 -0.0594059405940594 -6 -840 100 10830 99741 2.52252 299.52252 151.02252 15102.25225 2.52252 299.52252 151.02252 15102.2522 2.52252 299.52252 151.02252 15102.25200 2020-01-01 2020-01-02 2020-01-01 00:14:00 2020-01-02 03:42:21 2020-01-01 00:14:00.000 2020-01-02 03:42:21.000 840 99741 50290.5 5029050 840 99741 50290.5 5029050 -32728 32207 4415.3 441530 -128 127 -0.7 -70 -841 100 10831 99742 2.52552 299.52552 151.02552 15102.55255 2.52552 299.5255 151.02552 15102.5525 2.52552 299.52552 151.02552 15102.55200 2020-01-01 2020-01-02 2020-01-01 00:14:01 2020-01-02 03:42:22 2020-01-01 00:14:01.000 2020-01-02 03:42:22.000 841 99742 50291.5 5029150 841 99742 50291.5 5029150 -32727 32208 4416.3 441630 -128 127 -2.26 -226 -842 100 10832 99743 2.52852 299.52852 151.02852 15102.85285 2.52852 299.52853 151.02853 15102.85313 2.52852 299.52852 151.02852 15102.85200 2020-01-01 2020-01-02 2020-01-01 00:14:02 2020-01-02 03:42:23 2020-01-01 00:14:02.000 2020-01-02 03:42:23.000 842 99743 50292.5 5029250 842 99743 50292.5 5029250 -32726 32209 4417.3 441730 -128 123 -3.82 -382 -843 100 10833 99744 2.53153 299.53153 151.03153 15103.15315 2.53153 299.53152 151.03152 15103.15278 2.53153 299.53153 151.03153 15103.15300 2020-01-01 2020-01-02 2020-01-01 00:14:03 2020-01-02 03:42:24 2020-01-01 00:14:03.000 2020-01-02 03:42:24.000 843 99744 50293.5 5029350 843 99744 50293.5 5029350 -32725 32210 4418.3 441830 -127 124 -2.82 -282 -844 100 10834 99745 2.53453 299.53453 151.03453 15103.45345 2.53453 299.53455 151.03453 15103.45353 2.53453 299.53453 151.03453 15103.45300 2020-01-01 2020-01-02 2020-01-01 00:14:04 2020-01-02 03:42:25 2020-01-01 00:14:04.000 2020-01-02 03:42:25.000 844 99745 50294.5 5029450 844 99745 50294.5 5029450 -32724 32211 4419.3 441930 -126 125 -1.82 -182 -845 100 10835 99746 2.53753 299.53753 151.03753 15103.75375 2.53753 299.53754 151.03753 15103.75366 2.53753 299.53753 151.03753 15103.75300 2020-01-01 2020-01-02 2020-01-01 00:14:05 2020-01-02 03:42:26 2020-01-01 00:14:05.000 2020-01-02 03:42:26.000 845 99746 50295.5 5029550 845 99746 50295.5 5029550 -32723 32212 4420.3 442030 -125 126 -0.82 -82 +839 100 10829 99740 2.51951 299.51951 151.01951 15101.95195 2.51951 299.51953 151.01951 15101.9519 2.51951 299.51951 151.01951000000005 15101.95100 2020-01-01 2020-01-02 2020-01-01 00:13:59 2020-01-02 03:42:20 2020-01-01 00:13:59.000 2020-01-02 03:42:20.000 839 99740 50289.5 5028950 839 99740 50289.5 5028950 -32729 32206 4414.3 441430 -124 127 0.86 86 +84 102 10074 99984 0.25225 300.25225 150.25225 15175.47747 0.25225 300.25226 150.25225 15175.47778 0.25225 300.25225 150.25224999999986 15175.47725 2020-01-01 2020-01-02 2020-01-01 00:01:24 2020-01-02 03:46:24 2020-01-01 00:01:24.000 2020-01-02 03:46:24.000 84 99984 50034 5053434 84 99984 50034 5053434 -32485 32450 4613.009900990099 465914 -124 127 -0.0594059405940594 -6 +840 100 10830 99741 2.52252 299.52252 151.02252 15102.25225 2.52252 299.52252 151.02252 15102.2522 2.52252 299.52252 151.02252000000016 15102.25200 2020-01-01 2020-01-02 2020-01-01 00:14:00 2020-01-02 03:42:21 2020-01-01 00:14:00.000 2020-01-02 03:42:21.000 840 99741 50290.5 5029050 840 99741 50290.5 5029050 -32728 32207 4415.3 441530 -128 127 -0.7 -70 +841 100 10831 99742 2.52552 299.52552 151.02552 15102.55255 2.52552 299.5255 151.02552 15102.5525 2.52552 299.52552 151.02551999999994 15102.55200 2020-01-01 2020-01-02 2020-01-01 00:14:01 2020-01-02 03:42:22 2020-01-01 00:14:01.000 2020-01-02 03:42:22.000 841 99742 50291.5 5029150 841 99742 50291.5 5029150 -32727 32208 4416.3 441630 -128 127 -2.26 -226 +842 100 10832 99743 2.52852 299.52852 151.02852 15102.85285 2.52852 299.52853 151.02853 15102.85313 2.52852 299.52852 151.02851999999993 15102.85200 2020-01-01 2020-01-02 2020-01-01 00:14:02 2020-01-02 03:42:23 2020-01-01 00:14:02.000 2020-01-02 03:42:23.000 842 99743 50292.5 5029250 842 99743 50292.5 5029250 -32726 32209 4417.3 441730 -128 123 -3.82 -382 +843 100 10833 99744 2.53153 299.53153 151.03153 15103.15315 2.53153 299.53152 151.03152 15103.15278 2.53153 299.53153 151.0315300000001 15103.15300 2020-01-01 2020-01-02 2020-01-01 00:14:03 2020-01-02 03:42:24 2020-01-01 00:14:03.000 2020-01-02 03:42:24.000 843 99744 50293.5 5029350 843 99744 50293.5 5029350 -32725 32210 4418.3 441830 -127 124 -2.82 -282 +844 100 10834 99745 2.53453 299.53453 151.03453 15103.45345 2.53453 299.53455 151.03453 15103.45353 2.53453 299.53453 151.03453000000005 15103.45300 2020-01-01 2020-01-02 2020-01-01 00:14:04 2020-01-02 03:42:25 2020-01-01 00:14:04.000 2020-01-02 03:42:25.000 844 99745 50294.5 5029450 844 99745 50294.5 5029450 -32724 32211 4419.3 441930 -126 125 -1.82 -182 +845 100 10835 99746 2.53753 299.53753 151.03753 15103.75375 2.53753 299.53754 151.03753 15103.75366 2.53753 299.53753 151.0375299999999 15103.75300 2020-01-01 2020-01-02 2020-01-01 00:14:05 2020-01-02 03:42:26 2020-01-01 00:14:05.000 2020-01-02 03:42:26.000 845 99746 50295.5 5029550 845 99746 50295.5 5029550 -32723 32212 4420.3 442030 -125 126 -0.82 -82 846 100 10836 99747 2.54054 299.54054 151.04054 15104.05405 2.54054 299.54053 151.04053 15104.05397 2.54054 299.54054 151.04054 15104.05400 2020-01-01 2020-01-02 2020-01-01 00:14:06 2020-01-02 03:42:27 2020-01-01 00:14:06.000 2020-01-02 03:42:27.000 846 99747 50296.5 5029650 846 99747 50296.5 5029650 -32722 32213 4421.3 442130 -124 127 0.18 18 847 100 10837 99748 2.54354 299.54354 151.04354 15104.35435 2.54354 299.54355 151.04354 15104.3546 2.54354 299.54354 151.04354 15104.35400 2020-01-01 2020-01-02 2020-01-01 00:14:07 2020-01-02 03:42:28 2020-01-01 00:14:07.000 2020-01-02 03:42:28.000 847 99748 50297.5 5029750 847 99748 50297.5 5029750 -32721 32214 4422.3 442230 -128 127 -1.38 -138 -848 100 10838 99749 2.54654 299.54654 151.04654 15104.65465 2.54654 299.54654 151.04654 15104.65425 2.54654 299.54654 151.04654 15104.65400 2020-01-01 2020-01-02 2020-01-01 00:14:08 2020-01-02 03:42:29 2020-01-01 00:14:08.000 2020-01-02 03:42:29.000 848 99749 50298.5 5029850 848 99749 50298.5 5029850 -32720 32215 4423.3 442330 -128 123 -2.94 -294 +848 100 10838 99749 2.54654 299.54654 151.04654 15104.65465 2.54654 299.54654 151.04654 15104.65425 2.54654 299.54654 151.04653999999994 15104.65400 2020-01-01 2020-01-02 2020-01-01 00:14:08 2020-01-02 03:42:29 2020-01-01 00:14:08.000 2020-01-02 03:42:29.000 848 99749 50298.5 5029850 848 99749 50298.5 5029850 -32720 32215 4423.3 442330 -128 123 -2.94 -294 849 100 10839 99750 2.54954 299.54954 151.04954 15104.95495 2.54954 299.54956 151.04954 15104.95499 2.54954 299.54954 151.04954 15104.95400 2020-01-01 2020-01-02 2020-01-01 00:14:09 2020-01-02 03:42:30 2020-01-01 00:14:09.000 2020-01-02 03:42:30.000 849 99750 50299.5 5029950 849 99750 50299.5 5029950 -32719 32216 4424.3 442430 -127 124 -1.94 -194 -85 102 10075 99985 0.25525 300.25525 150.25525 15175.78078 0.25525 300.25525 150.25525 15175.78046 0.25525 300.25525 150.25525 15175.78025 2020-01-01 2020-01-02 2020-01-01 00:01:25 2020-01-02 03:46:25 2020-01-01 00:01:25.000 2020-01-02 03:46:25.000 85 99985 50035 5053535 85 99985 50035 5053535 -32484 32451 4614.009900990099 466015 -128 127 -1.5940594059405941 -161 -850 100 10840 99751 2.55255 299.55255 151.05255 15105.25525 2.55255 299.55255 151.05255 15105.25514 2.55255 299.55255 151.05255 15105.25500 2020-01-01 2020-01-02 2020-01-01 00:14:10 2020-01-02 03:42:31 2020-01-01 00:14:10.000 2020-01-02 03:42:31.000 850 99751 50300.5 5030050 850 99751 50300.5 5030050 -32718 32217 4425.3 442530 -126 125 -0.94 -94 -851 100 10841 99752 2.55555 299.55555 151.05555 15105.55555 2.55555 299.55554 151.05555 15105.55547 2.55555 299.55555 151.05555 15105.55500 2020-01-01 2020-01-02 2020-01-01 00:14:11 2020-01-02 03:42:32 2020-01-01 00:14:11.000 2020-01-02 03:42:32.000 851 99752 50301.5 5030150 851 99752 50301.5 5030150 -32717 32218 4426.3 442630 -125 126 0.06 6 -852 100 10842 99753 2.55855 299.55855 151.05855 15105.85585 2.55855 299.55856 151.05856 15105.85607 2.55855 299.55855 151.05855 15105.85500 2020-01-01 2020-01-02 2020-01-01 00:14:12 2020-01-02 03:42:33 2020-01-01 00:14:12.000 2020-01-02 03:42:33.000 852 99753 50302.5 5030250 852 99753 50302.5 5030250 -32716 32219 4427.3 442730 -124 127 1.06 106 -853 100 10843 99754 2.56156 299.56156 151.06156 15106.15615 2.56156 299.56155 151.06155 15106.15571 2.56156 299.56156 151.06156 15106.15600 2020-01-01 2020-01-02 2020-01-01 00:14:13 2020-01-02 03:42:34 2020-01-01 00:14:13.000 2020-01-02 03:42:34.000 853 99754 50303.5 5030350 853 99754 50303.5 5030350 -32715 32220 4428.3 442830 -128 127 -0.5 -50 -854 100 10844 99755 2.56456 299.56456 151.06456 15106.45645 2.56456 299.56458 151.06456 15106.45646 2.56456 299.56456 151.06456 15106.45600 2020-01-01 2020-01-02 2020-01-01 00:14:14 2020-01-02 03:42:35 2020-01-01 00:14:14.000 2020-01-02 03:42:35.000 854 99755 50304.5 5030450 854 99755 50304.5 5030450 -32714 32221 4429.3 442930 -128 123 -2.06 -206 -855 100 10845 99756 2.56756 299.56756 151.06756 15106.75675 2.56756 299.56757 151.06756 15106.75661 2.56756 299.56756 151.06756 15106.75600 2020-01-01 2020-01-02 2020-01-01 00:14:15 2020-01-02 03:42:36 2020-01-01 00:14:15.000 2020-01-02 03:42:36.000 855 99756 50305.5 5030550 855 99756 50305.5 5030550 -32713 32222 4430.3 443030 -127 124 -1.06 -106 -856 100 10846 99757 2.57057 299.57057 151.07057 15107.05705 2.57057 299.57056 151.07056 15107.05694 2.57057 299.57057 151.07057 15107.05700 2020-01-01 2020-01-02 2020-01-01 00:14:16 2020-01-02 03:42:37 2020-01-01 00:14:16.000 2020-01-02 03:42:37.000 856 99757 50306.5 5030650 856 99757 50306.5 5030650 -32712 32223 4431.3 443130 -126 125 -0.06 -6 -857 100 10847 99758 2.57357 299.57357 151.07357 15107.35735 2.57357 299.57358 151.07357 15107.35754 2.57357 299.57357 151.07357 15107.35700 2020-01-01 2020-01-02 2020-01-01 00:14:17 2020-01-02 03:42:38 2020-01-01 00:14:17.000 2020-01-02 03:42:38.000 857 99758 50307.5 5030750 857 99758 50307.5 5030750 -32711 32224 4432.3 443230 -125 126 0.94 94 -858 100 10848 99759 2.57657 299.57657 151.07657 15107.65765 2.57657 299.57657 151.07657 15107.65783 2.57657 299.57657 151.07657 15107.65700 2020-01-01 2020-01-02 2020-01-01 00:14:18 2020-01-02 03:42:39 2020-01-01 00:14:18.000 2020-01-02 03:42:39.000 858 99759 50308.5 5030850 858 99759 50308.5 5030850 -32710 32225 4433.3 443330 -124 127 1.94 194 -859 100 10849 99760 2.57957 299.57957 151.07957 15107.95795 2.57957 299.5796 151.07957 15107.95794 2.57957 299.57957 151.07957 15107.95700 2020-01-01 2020-01-02 2020-01-01 00:14:19 2020-01-02 03:42:40 2020-01-01 00:14:19.000 2020-01-02 03:42:40.000 859 99760 50309.5 5030950 859 99760 50309.5 5030950 -32709 32226 4434.3 443430 -128 127 0.38 38 -86 102 10076 99986 0.25825 300.25825 150.25825 15176.08408 0.25825 300.25827 150.25825 15176.08406 0.25825 300.25825 150.25825 15176.08325 2020-01-01 2020-01-02 2020-01-01 00:01:26 2020-01-02 03:46:26 2020-01-01 00:01:26.000 2020-01-02 03:46:26.000 86 99986 50036 5053636 86 99986 50036 5053636 -32483 32452 4615.009900990099 466116 -128 123 -3.128712871287129 -316 -860 100 10850 99761 2.58258 299.58258 151.08258 15108.25825 2.58258 299.58258 151.08258 15108.25811 2.58258 299.58258 151.08258 15108.25800 2020-01-01 2020-01-02 2020-01-01 00:14:20 2020-01-02 03:42:41 2020-01-01 00:14:20.000 2020-01-02 03:42:41.000 860 99761 50310.5 5031050 860 99761 50310.5 5031050 -32708 32227 4435.3 443530 -128 123 -1.18 -118 -861 100 10851 99762 2.58558 299.58558 151.08558 15108.55855 2.58558 299.58557 151.08558 15108.55841 2.58558 299.58558 151.08558 15108.55800 2020-01-01 2020-01-02 2020-01-01 00:14:21 2020-01-02 03:42:42 2020-01-01 00:14:21.000 2020-01-02 03:42:42.000 861 99762 50311.5 5031150 861 99762 50311.5 5031150 -32707 32228 4436.3 443630 -127 124 -0.18 -18 -862 100 10852 99763 2.58858 299.58858 151.08858 15108.85885 2.58858 299.5886 151.08859 15108.85901 2.58858 299.58858 151.08858 15108.85800 2020-01-01 2020-01-02 2020-01-01 00:14:22 2020-01-02 03:42:43 2020-01-01 00:14:22.000 2020-01-02 03:42:43.000 862 99763 50312.5 5031250 862 99763 50312.5 5031250 -32706 32229 4437.3 443730 -126 125 0.82 82 +85 102 10075 99985 0.25525 300.25525 150.25525 15175.78078 0.25525 300.25525 150.25525 15175.78046 0.25525 300.25525 150.25525000000007 15175.78025 2020-01-01 2020-01-02 2020-01-01 00:01:25 2020-01-02 03:46:25 2020-01-01 00:01:25.000 2020-01-02 03:46:25.000 85 99985 50035 5053535 85 99985 50035 5053535 -32484 32451 4614.009900990099 466015 -128 127 -1.5940594059405941 -161 +850 100 10840 99751 2.55255 299.55255 151.05255 15105.25525 2.55255 299.55255 151.05255 15105.25514 2.55255 299.55255 151.05255000000008 15105.25500 2020-01-01 2020-01-02 2020-01-01 00:14:10 2020-01-02 03:42:31 2020-01-01 00:14:10.000 2020-01-02 03:42:31.000 850 99751 50300.5 5030050 850 99751 50300.5 5030050 -32718 32217 4425.3 442530 -126 125 -0.94 -94 +851 100 10841 99752 2.55555 299.55555 151.05555 15105.55555 2.55555 299.55554 151.05555 15105.55547 2.55555 299.55555 151.05554999999993 15105.55500 2020-01-01 2020-01-02 2020-01-01 00:14:11 2020-01-02 03:42:32 2020-01-01 00:14:11.000 2020-01-02 03:42:32.000 851 99752 50301.5 5030150 851 99752 50301.5 5030150 -32717 32218 4426.3 442630 -125 126 0.06 6 +852 100 10842 99753 2.55855 299.55855 151.05855 15105.85585 2.55855 299.55856 151.05856 15105.85607 2.55855 299.55855 151.05854999999997 15105.85500 2020-01-01 2020-01-02 2020-01-01 00:14:12 2020-01-02 03:42:33 2020-01-01 00:14:12.000 2020-01-02 03:42:33.000 852 99753 50302.5 5030250 852 99753 50302.5 5030250 -32716 32219 4427.3 442730 -124 127 1.06 106 +853 100 10843 99754 2.56156 299.56156 151.06156 15106.15615 2.56156 299.56155 151.06155 15106.15571 2.56156 299.56156 151.06156000000007 15106.15600 2020-01-01 2020-01-02 2020-01-01 00:14:13 2020-01-02 03:42:34 2020-01-01 00:14:13.000 2020-01-02 03:42:34.000 853 99754 50303.5 5030350 853 99754 50303.5 5030350 -32715 32220 4428.3 442830 -128 127 -0.5 -50 +854 100 10844 99755 2.56456 299.56456 151.06456 15106.45645 2.56456 299.56458 151.06456 15106.45646 2.56456 299.56456 151.06456000000003 15106.45600 2020-01-01 2020-01-02 2020-01-01 00:14:14 2020-01-02 03:42:35 2020-01-01 00:14:14.000 2020-01-02 03:42:35.000 854 99755 50304.5 5030450 854 99755 50304.5 5030450 -32714 32221 4429.3 442930 -128 123 -2.06 -206 +855 100 10845 99756 2.56756 299.56756 151.06756 15106.75675 2.56756 299.56757 151.06756 15106.75661 2.56756 299.56756 151.06756000000001 15106.75600 2020-01-01 2020-01-02 2020-01-01 00:14:15 2020-01-02 03:42:36 2020-01-01 00:14:15.000 2020-01-02 03:42:36.000 855 99756 50305.5 5030550 855 99756 50305.5 5030550 -32713 32222 4430.3 443030 -127 124 -1.06 -106 +856 100 10846 99757 2.57057 299.57057 151.07057 15107.05705 2.57057 299.57056 151.07056 15107.05694 2.57057 299.57057 151.07056999999995 15107.05700 2020-01-01 2020-01-02 2020-01-01 00:14:16 2020-01-02 03:42:37 2020-01-01 00:14:16.000 2020-01-02 03:42:37.000 856 99757 50306.5 5030650 856 99757 50306.5 5030650 -32712 32223 4431.3 443130 -126 125 -0.06 -6 +857 100 10847 99758 2.57357 299.57357 151.07357 15107.35735 2.57357 299.57358 151.07357 15107.35754 2.57357 299.57357 151.07356999999996 15107.35700 2020-01-01 2020-01-02 2020-01-01 00:14:17 2020-01-02 03:42:38 2020-01-01 00:14:17.000 2020-01-02 03:42:38.000 857 99758 50307.5 5030750 857 99758 50307.5 5030750 -32711 32224 4432.3 443230 -125 126 0.94 94 +858 100 10848 99759 2.57657 299.57657 151.07657 15107.65765 2.57657 299.57657 151.07657 15107.65783 2.57657 299.57657 151.07656999999998 15107.65700 2020-01-01 2020-01-02 2020-01-01 00:14:18 2020-01-02 03:42:39 2020-01-01 00:14:18.000 2020-01-02 03:42:39.000 858 99759 50308.5 5030850 858 99759 50308.5 5030850 -32710 32225 4433.3 443330 -124 127 1.94 194 +859 100 10849 99760 2.57957 299.57957 151.07957 15107.95795 2.57957 299.5796 151.07957 15107.95794 2.57957 299.57957 151.07956999999993 15107.95700 2020-01-01 2020-01-02 2020-01-01 00:14:19 2020-01-02 03:42:40 2020-01-01 00:14:19.000 2020-01-02 03:42:40.000 859 99760 50309.5 5030950 859 99760 50309.5 5030950 -32709 32226 4434.3 443430 -128 127 0.38 38 +86 102 10076 99986 0.25825 300.25825 150.25825 15176.08408 0.25825 300.25827 150.25825 15176.08406 0.25825 300.25825 150.25825000000003 15176.08325 2020-01-01 2020-01-02 2020-01-01 00:01:26 2020-01-02 03:46:26 2020-01-01 00:01:26.000 2020-01-02 03:46:26.000 86 99986 50036 5053636 86 99986 50036 5053636 -32483 32452 4615.009900990099 466116 -128 123 -3.128712871287129 -316 +860 100 10850 99761 2.58258 299.58258 151.08258 15108.25825 2.58258 299.58258 151.08258 15108.25811 2.58258 299.58258 151.08258000000012 15108.25800 2020-01-01 2020-01-02 2020-01-01 00:14:20 2020-01-02 03:42:41 2020-01-01 00:14:20.000 2020-01-02 03:42:41.000 860 99761 50310.5 5031050 860 99761 50310.5 5031050 -32708 32227 4435.3 443530 -128 123 -1.18 -118 +861 100 10851 99762 2.58558 299.58558 151.08558 15108.55855 2.58558 299.58557 151.08558 15108.55841 2.58558 299.58558 151.08558000000005 15108.55800 2020-01-01 2020-01-02 2020-01-01 00:14:21 2020-01-02 03:42:42 2020-01-01 00:14:21.000 2020-01-02 03:42:42.000 861 99762 50311.5 5031150 861 99762 50311.5 5031150 -32707 32228 4436.3 443630 -127 124 -0.18 -18 +862 100 10852 99763 2.58858 299.58858 151.08858 15108.85885 2.58858 299.5886 151.08859 15108.85901 2.58858 299.58858 151.08857999999992 15108.85800 2020-01-01 2020-01-02 2020-01-01 00:14:22 2020-01-02 03:42:43 2020-01-01 00:14:22.000 2020-01-02 03:42:43.000 862 99763 50312.5 5031250 862 99763 50312.5 5031250 -32706 32229 4437.3 443730 -126 125 0.82 82 863 100 10853 99764 2.59159 299.59159 151.09159 15109.15915 2.59159 299.59158 151.09159 15109.1593 2.59159 299.59159 151.09159 15109.15900 2020-01-01 2020-01-02 2020-01-01 00:14:23 2020-01-02 03:42:44 2020-01-01 00:14:23.000 2020-01-02 03:42:44.000 863 99764 50313.5 5031350 863 99764 50313.5 5031350 -32705 32230 4438.3 443830 -125 126 1.82 182 -864 100 10854 99765 2.59459 299.59459 151.09459 15109.45945 2.59459 299.5946 151.09459 15109.45941 2.59459 299.59459 151.09459 15109.45900 2020-01-01 2020-01-02 2020-01-01 00:14:24 2020-01-02 03:42:45 2020-01-01 00:14:24.000 2020-01-02 03:42:45.000 864 99765 50314.5 5031450 864 99765 50314.5 5031450 -32704 32231 4439.3 443930 -124 127 2.82 282 -865 100 10855 99766 2.59759 299.59759 151.09759 15109.75975 2.59759 299.5976 151.09759 15109.75958 2.59759 299.59759 151.09759 15109.75900 2020-01-01 2020-01-02 2020-01-01 00:14:25 2020-01-02 03:42:46 2020-01-01 00:14:25.000 2020-01-02 03:42:46.000 865 99766 50315.5 5031550 865 99766 50315.5 5031550 -32703 32232 4440.3 444030 -128 127 1.26 126 -866 100 10856 99767 2.6006 299.6006 151.1006 15110.06006 2.6006 299.6006 151.10059 15110.05988 2.60060 299.60060 151.10060 15110.06000 2020-01-01 2020-01-02 2020-01-01 00:14:26 2020-01-02 03:42:47 2020-01-01 00:14:26.000 2020-01-02 03:42:47.000 866 99767 50316.5 5031650 866 99767 50316.5 5031650 -32702 32233 4441.3 444130 -128 127 -0.3 -30 -867 100 10857 99768 2.6036 299.6036 151.1036 15110.36036 2.6036 299.6036 151.1036 15110.36063 2.60360 299.60360 151.10360 15110.36000 2020-01-01 2020-01-02 2020-01-01 00:14:27 2020-01-02 03:42:48 2020-01-01 00:14:27.000 2020-01-02 03:42:48.000 867 99768 50317.5 5031750 867 99768 50317.5 5031750 -32701 32234 4442.3 444230 -128 123 -1.86 -186 -868 100 10858 99769 2.6066 299.6066 151.1066 15110.66066 2.6066 299.6066 151.1066 15110.66078 2.60660 299.60660 151.10660 15110.66000 2020-01-01 2020-01-02 2020-01-01 00:14:28 2020-01-02 03:42:49 2020-01-01 00:14:28.000 2020-01-02 03:42:49.000 868 99769 50318.5 5031850 868 99769 50318.5 5031850 -32700 32235 4443.3 444330 -127 124 -0.86 -86 -869 100 10859 99770 2.6096 299.6096 151.1096 15110.96096 2.6096 299.60962 151.1096 15110.96091 2.60960 299.60960 151.10960 15110.96000 2020-01-01 2020-01-02 2020-01-01 00:14:29 2020-01-02 03:42:50 2020-01-01 00:14:29.000 2020-01-02 03:42:50.000 869 99770 50319.5 5031950 869 99770 50319.5 5031950 -32699 32236 4444.3 444430 -126 125 0.14 14 -87 102 10077 99987 0.26126 300.26126 150.26126 15176.38738 0.26126 300.26126 150.26126 15176.38736 0.26126 300.26126 150.26126 15176.38726 2020-01-01 2020-01-02 2020-01-01 00:01:27 2020-01-02 03:46:27 2020-01-01 00:01:27.000 2020-01-02 03:46:27.000 87 99987 50037 5053737 87 99987 50037 5053737 -32482 32453 4616.009900990099 466217 -127 124 -2.128712871287129 -215 -870 100 10860 99771 2.61261 299.61261 151.11261 15111.26126 2.61261 299.6126 151.11261 15111.26105 2.61261 299.61261 151.11261 15111.26100 2020-01-01 2020-01-02 2020-01-01 00:14:30 2020-01-02 03:42:51 2020-01-01 00:14:30.000 2020-01-02 03:42:51.000 870 99771 50320.5 5032050 870 99771 50320.5 5032050 -32698 32237 4445.3 444530 -125 126 1.14 114 -871 100 10861 99772 2.61561 299.61561 151.11561 15111.56156 2.61561 299.6156 151.11561 15111.56135 2.61561 299.61561 151.11561 15111.56100 2020-01-01 2020-01-02 2020-01-01 00:14:31 2020-01-02 03:42:52 2020-01-01 00:14:31.000 2020-01-02 03:42:52.000 871 99772 50321.5 5032150 871 99772 50321.5 5032150 -32697 32238 4446.3 444630 -124 127 2.14 214 -872 100 10862 99773 2.61861 299.61861 151.11861 15111.86186 2.61861 299.61862 151.11862 15111.8621 2.61861 299.61861 151.11861 15111.86100 2020-01-01 2020-01-02 2020-01-01 00:14:32 2020-01-02 03:42:53 2020-01-01 00:14:32.000 2020-01-02 03:42:53.000 872 99773 50322.5 5032250 872 99773 50322.5 5032250 -32696 32239 4447.3 444730 -128 127 0.58 58 -873 100 10863 99774 2.62162 299.62162 151.12162 15112.16216 2.62162 299.6216 151.12162 15112.16224 2.62162 299.62162 151.12162 15112.16200 2020-01-01 2020-01-02 2020-01-01 00:14:33 2020-01-02 03:42:54 2020-01-01 00:14:33.000 2020-01-02 03:42:54.000 873 99774 50323.5 5032350 873 99774 50323.5 5032350 -32695 32240 4448.3 444830 -128 123 -0.98 -98 -874 100 10864 99775 2.62462 299.62462 151.12462 15112.46246 2.62462 299.62463 151.12462 15112.46238 2.62462 299.62462 151.12462 15112.46200 2020-01-01 2020-01-02 2020-01-01 00:14:34 2020-01-02 03:42:55 2020-01-01 00:14:34.000 2020-01-02 03:42:55.000 874 99775 50324.5 5032450 874 99775 50324.5 5032450 -32694 32241 4449.3 444930 -127 124 0.02 2 -875 100 10865 99776 2.62762 299.62762 151.12762 15112.76276 2.62762 299.62762 151.12762 15112.76252 2.62762 299.62762 151.12762 15112.76200 2020-01-01 2020-01-02 2020-01-01 00:14:35 2020-01-02 03:42:56 2020-01-01 00:14:35.000 2020-01-02 03:42:56.000 875 99776 50325.5 5032550 875 99776 50325.5 5032550 -32693 32242 4450.3 445030 -126 125 1.02 102 -876 100 10866 99777 2.63063 299.63063 151.13063 15113.06306 2.63063 299.63065 151.13063 15113.06327 2.63063 299.63063 151.13063 15113.06300 2020-01-01 2020-01-02 2020-01-01 00:14:36 2020-01-02 03:42:57 2020-01-01 00:14:36.000 2020-01-02 03:42:57.000 876 99777 50326.5 5032650 876 99777 50326.5 5032650 -32692 32243 4451.3 445130 -125 126 2.02 202 -877 100 10867 99778 2.63363 299.63363 151.13363 15113.36336 2.63363 299.63364 151.13363 15113.36358 2.63363 299.63363 151.13363 15113.36300 2020-01-01 2020-01-02 2020-01-01 00:14:37 2020-01-02 03:42:58 2020-01-01 00:14:37.000 2020-01-02 03:42:58.000 877 99778 50327.5 5032750 877 99778 50327.5 5032750 -32691 32244 4452.3 445230 -124 127 3.02 302 -878 100 10868 99779 2.63663 299.63663 151.13663 15113.66366 2.63663 299.63663 151.13663 15113.66371 2.63663 299.63663 151.13663 15113.66300 2020-01-01 2020-01-02 2020-01-01 00:14:38 2020-01-02 03:42:59 2020-01-01 00:14:38.000 2020-01-02 03:42:59.000 878 99779 50328.5 5032850 878 99779 50328.5 5032850 -32690 32245 4453.3 445330 -128 127 1.46 146 -879 100 10869 99780 2.63963 299.63963 151.13963 15113.96396 2.63963 299.63965 151.13963 15113.96385 2.63963 299.63963 151.13963 15113.96300 2020-01-01 2020-01-02 2020-01-01 00:14:39 2020-01-02 03:43:00 2020-01-01 00:14:39.000 2020-01-02 03:43:00.000 879 99780 50329.5 5032950 879 99780 50329.5 5032950 -32689 32246 4454.3 445430 -128 123 -0.1 -10 -88 102 10078 99988 0.26426 300.26426 150.26426 15176.69069 0.26426 300.26425 150.26426 15176.69066 0.26426 300.26426 150.26426 15176.69026 2020-01-01 2020-01-02 2020-01-01 00:01:28 2020-01-02 03:46:28 2020-01-01 00:01:28.000 2020-01-02 03:46:28.000 88 99988 50038 5053838 88 99988 50038 5053838 -32481 32454 4617.009900990099 466318 -126 125 -1.1287128712871286 -114 +864 100 10854 99765 2.59459 299.59459 151.09459 15109.45945 2.59459 299.5946 151.09459 15109.45941 2.59459 299.59459 151.09459000000004 15109.45900 2020-01-01 2020-01-02 2020-01-01 00:14:24 2020-01-02 03:42:45 2020-01-01 00:14:24.000 2020-01-02 03:42:45.000 864 99765 50314.5 5031450 864 99765 50314.5 5031450 -32704 32231 4439.3 443930 -124 127 2.82 282 +865 100 10855 99766 2.59759 299.59759 151.09759 15109.75975 2.59759 299.5976 151.09759 15109.75958 2.59759 299.59759 151.09759000000008 15109.75900 2020-01-01 2020-01-02 2020-01-01 00:14:25 2020-01-02 03:42:46 2020-01-01 00:14:25.000 2020-01-02 03:42:46.000 865 99766 50315.5 5031550 865 99766 50315.5 5031550 -32703 32232 4440.3 444030 -128 127 1.26 126 +866 100 10856 99767 2.6006 299.6006 151.1006 15110.06006 2.6006 299.6006 151.10059 15110.05988 2.60060 299.60060 151.10059999999984 15110.06000 2020-01-01 2020-01-02 2020-01-01 00:14:26 2020-01-02 03:42:47 2020-01-01 00:14:26.000 2020-01-02 03:42:47.000 866 99767 50316.5 5031650 866 99767 50316.5 5031650 -32702 32233 4441.3 444130 -128 127 -0.3 -30 +867 100 10857 99768 2.6036 299.6036 151.1036 15110.36036 2.6036 299.6036 151.1036 15110.36063 2.60360 299.60360 151.10360000000026 15110.36000 2020-01-01 2020-01-02 2020-01-01 00:14:27 2020-01-02 03:42:48 2020-01-01 00:14:27.000 2020-01-02 03:42:48.000 867 99768 50317.5 5031750 867 99768 50317.5 5031750 -32701 32234 4442.3 444230 -128 123 -1.86 -186 +868 100 10858 99769 2.6066 299.6066 151.1066 15110.66066 2.6066 299.6066 151.1066 15110.66078 2.60660 299.60660 151.10659999999976 15110.66000 2020-01-01 2020-01-02 2020-01-01 00:14:28 2020-01-02 03:42:49 2020-01-01 00:14:28.000 2020-01-02 03:42:49.000 868 99769 50318.5 5031850 868 99769 50318.5 5031850 -32700 32235 4443.3 444330 -127 124 -0.86 -86 +869 100 10859 99770 2.6096 299.6096 151.1096 15110.96096 2.6096 299.60962 151.1096 15110.96091 2.60960 299.60960 151.1095999999998 15110.96000 2020-01-01 2020-01-02 2020-01-01 00:14:29 2020-01-02 03:42:50 2020-01-01 00:14:29.000 2020-01-02 03:42:50.000 869 99770 50319.5 5031950 869 99770 50319.5 5031950 -32699 32236 4444.3 444430 -126 125 0.14 14 +87 102 10077 99987 0.26126 300.26126 150.26126 15176.38738 0.26126 300.26126 150.26126 15176.38736 0.26126 300.26126 150.2612599999999 15176.38726 2020-01-01 2020-01-02 2020-01-01 00:01:27 2020-01-02 03:46:27 2020-01-01 00:01:27.000 2020-01-02 03:46:27.000 87 99987 50037 5053737 87 99987 50037 5053737 -32482 32453 4616.009900990099 466217 -127 124 -2.128712871287129 -215 +870 100 10860 99771 2.61261 299.61261 151.11261 15111.26126 2.61261 299.6126 151.11261 15111.26105 2.61261 299.61261 151.1126100000001 15111.26100 2020-01-01 2020-01-02 2020-01-01 00:14:30 2020-01-02 03:42:51 2020-01-01 00:14:30.000 2020-01-02 03:42:51.000 870 99771 50320.5 5032050 870 99771 50320.5 5032050 -32698 32237 4445.3 444530 -125 126 1.14 114 +871 100 10861 99772 2.61561 299.61561 151.11561 15111.56156 2.61561 299.6156 151.11561 15111.56135 2.61561 299.61561 151.11561000000017 15111.56100 2020-01-01 2020-01-02 2020-01-01 00:14:31 2020-01-02 03:42:52 2020-01-01 00:14:31.000 2020-01-02 03:42:52.000 871 99772 50321.5 5032150 871 99772 50321.5 5032150 -32697 32238 4446.3 444630 -124 127 2.14 214 +872 100 10862 99773 2.61861 299.61861 151.11861 15111.86186 2.61861 299.61862 151.11862 15111.8621 2.61861 299.61861 151.11860999999993 15111.86100 2020-01-01 2020-01-02 2020-01-01 00:14:32 2020-01-02 03:42:53 2020-01-01 00:14:32.000 2020-01-02 03:42:53.000 872 99773 50322.5 5032250 872 99773 50322.5 5032250 -32696 32239 4447.3 444730 -128 127 0.58 58 +873 100 10863 99774 2.62162 299.62162 151.12162 15112.16216 2.62162 299.6216 151.12162 15112.16224 2.62162 299.62162 151.12161999999992 15112.16200 2020-01-01 2020-01-02 2020-01-01 00:14:33 2020-01-02 03:42:54 2020-01-01 00:14:33.000 2020-01-02 03:42:54.000 873 99774 50323.5 5032350 873 99774 50323.5 5032350 -32695 32240 4448.3 444830 -128 123 -0.98 -98 +874 100 10864 99775 2.62462 299.62462 151.12462 15112.46246 2.62462 299.62463 151.12462 15112.46238 2.62462 299.62462 151.12462000000002 15112.46200 2020-01-01 2020-01-02 2020-01-01 00:14:34 2020-01-02 03:42:55 2020-01-01 00:14:34.000 2020-01-02 03:42:55.000 874 99775 50324.5 5032450 874 99775 50324.5 5032450 -32694 32241 4449.3 444930 -127 124 0.02 2 +875 100 10865 99776 2.62762 299.62762 151.12762 15112.76276 2.62762 299.62762 151.12762 15112.76252 2.62762 299.62762 151.12761999999987 15112.76200 2020-01-01 2020-01-02 2020-01-01 00:14:35 2020-01-02 03:42:56 2020-01-01 00:14:35.000 2020-01-02 03:42:56.000 875 99776 50325.5 5032550 875 99776 50325.5 5032550 -32693 32242 4450.3 445030 -126 125 1.02 102 +876 100 10866 99777 2.63063 299.63063 151.13063 15113.06306 2.63063 299.63065 151.13063 15113.06327 2.63063 299.63063 151.13062999999977 15113.06300 2020-01-01 2020-01-02 2020-01-01 00:14:36 2020-01-02 03:42:57 2020-01-01 00:14:36.000 2020-01-02 03:42:57.000 876 99777 50326.5 5032650 876 99777 50326.5 5032650 -32692 32243 4451.3 445130 -125 126 2.02 202 +877 100 10867 99778 2.63363 299.63363 151.13363 15113.36336 2.63363 299.63364 151.13363 15113.36358 2.63363 299.63363 151.13363000000015 15113.36300 2020-01-01 2020-01-02 2020-01-01 00:14:37 2020-01-02 03:42:58 2020-01-01 00:14:37.000 2020-01-02 03:42:58.000 877 99778 50327.5 5032750 877 99778 50327.5 5032750 -32691 32244 4452.3 445230 -124 127 3.02 302 +878 100 10868 99779 2.63663 299.63663 151.13663 15113.66366 2.63663 299.63663 151.13663 15113.66371 2.63663 299.63663 151.13663000000028 15113.66300 2020-01-01 2020-01-02 2020-01-01 00:14:38 2020-01-02 03:42:59 2020-01-01 00:14:38.000 2020-01-02 03:42:59.000 878 99779 50328.5 5032850 878 99779 50328.5 5032850 -32690 32245 4453.3 445330 -128 127 1.46 146 +879 100 10869 99780 2.63963 299.63963 151.13963 15113.96396 2.63963 299.63965 151.13963 15113.96385 2.63963 299.63963 151.13962999999978 15113.96300 2020-01-01 2020-01-02 2020-01-01 00:14:39 2020-01-02 03:43:00 2020-01-01 00:14:39.000 2020-01-02 03:43:00.000 879 99780 50329.5 5032950 879 99780 50329.5 5032950 -32689 32246 4454.3 445430 -128 123 -0.1 -10 +88 102 10078 99988 0.26426 300.26426 150.26426 15176.69069 0.26426 300.26425 150.26426 15176.69066 0.26426 300.26426 150.26425999999995 15176.69026 2020-01-01 2020-01-02 2020-01-01 00:01:28 2020-01-02 03:46:28 2020-01-01 00:01:28.000 2020-01-02 03:46:28.000 88 99988 50038 5053838 88 99988 50038 5053838 -32481 32454 4617.009900990099 466318 -126 125 -1.1287128712871286 -114 880 100 10870 99781 2.64264 299.64264 151.14264 15114.26426 2.64264 299.64264 151.14263 15114.26399 2.64264 299.64264 151.14264 15114.26400 2020-01-01 2020-01-02 2020-01-01 00:14:40 2020-01-02 03:43:01 2020-01-01 00:14:40.000 2020-01-02 03:43:01.000 880 99781 50330.5 5033050 880 99781 50330.5 5033050 -32688 32247 4455.3 445530 -127 124 0.9 90 -881 100 10871 99782 2.64564 299.64564 151.14564 15114.56456 2.64564 299.64566 151.14564 15114.56474 2.64564 299.64564 151.14564 15114.56400 2020-01-01 2020-01-02 2020-01-01 00:14:41 2020-01-02 03:43:02 2020-01-01 00:14:41.000 2020-01-02 03:43:02.000 881 99782 50331.5 5033150 881 99782 50331.5 5033150 -32687 32248 4456.3 445630 -126 125 1.9 190 -882 100 10872 99783 2.64864 299.64864 151.14864 15114.86486 2.64864 299.64865 151.14865 15114.86504 2.64864 299.64864 151.14864 15114.86400 2020-01-01 2020-01-02 2020-01-01 00:14:42 2020-01-02 03:43:03 2020-01-01 00:14:42.000 2020-01-02 03:43:03.000 882 99783 50332.5 5033250 882 99783 50332.5 5033250 -32686 32249 4457.3 445730 -125 126 2.9 290 -883 100 10873 99784 2.65165 299.65165 151.15165 15115.16516 2.65165 299.65164 151.15165 15115.16522 2.65165 299.65165 151.15165 15115.16500 2020-01-01 2020-01-02 2020-01-01 00:14:43 2020-01-02 03:43:04 2020-01-01 00:14:43.000 2020-01-02 03:43:04.000 883 99784 50333.5 5033350 883 99784 50333.5 5033350 -32685 32250 4458.3 445830 -124 127 3.9 390 -884 100 10874 99785 2.65465 299.65465 151.15465 15115.46546 2.65465 299.65466 151.15465 15115.46532 2.65465 299.65465 151.15465 15115.46500 2020-01-01 2020-01-02 2020-01-01 00:14:44 2020-01-02 03:43:05 2020-01-01 00:14:44.000 2020-01-02 03:43:05.000 884 99785 50334.5 5033450 884 99785 50334.5 5033450 -32684 32251 4459.3 445930 -128 127 2.34 234 -885 100 10875 99786 2.65765 299.65765 151.15765 15115.76576 2.65765 299.65765 151.15765 15115.76562 2.65765 299.65765 151.15765 15115.76500 2020-01-01 2020-01-02 2020-01-01 00:14:45 2020-01-02 03:43:06 2020-01-01 00:14:45.000 2020-01-02 03:43:06.000 885 99786 50335.5 5033550 885 99786 50335.5 5033550 -32683 32252 4460.3 446030 -128 123 0.78 78 -886 100 10876 99787 2.66066 299.66066 151.16066 15116.06606 2.66066 299.66068 151.16066 15116.06621 2.66066 299.66066 151.16066 15116.06600 2020-01-01 2020-01-02 2020-01-01 00:14:46 2020-01-02 03:43:07 2020-01-01 00:14:46.000 2020-01-02 03:43:07.000 886 99787 50336.5 5033650 886 99787 50336.5 5033650 -32682 32253 4461.3 446130 -127 124 1.78 178 -887 100 10877 99788 2.66366 299.66366 151.16366 15116.36636 2.66366 299.66367 151.16366 15116.36651 2.66366 299.66366 151.16366 15116.36600 2020-01-01 2020-01-02 2020-01-01 00:14:47 2020-01-02 03:43:08 2020-01-01 00:14:47.000 2020-01-02 03:43:08.000 887 99788 50337.5 5033750 887 99788 50337.5 5033750 -32681 32254 4462.3 446230 -126 125 2.78 278 -888 100 10878 99789 2.66666 299.66666 151.16666 15116.66666 2.66666 299.66666 151.16666 15116.66669 2.66666 299.66666 151.16666 15116.66600 2020-01-01 2020-01-02 2020-01-01 00:14:48 2020-01-02 03:43:09 2020-01-01 00:14:48.000 2020-01-02 03:43:09.000 888 99789 50338.5 5033850 888 99789 50338.5 5033850 -32680 32255 4463.3 446330 -125 126 3.78 378 -889 100 10879 99790 2.66966 299.66966 151.16966 15116.96696 2.66966 299.66968 151.16966 15116.96679 2.66966 299.66966 151.16966 15116.96600 2020-01-01 2020-01-02 2020-01-01 00:14:49 2020-01-02 03:43:10 2020-01-01 00:14:49.000 2020-01-02 03:43:10.000 889 99790 50339.5 5033950 889 99790 50339.5 5033950 -32679 32256 4464.3 446430 -124 127 4.78 478 -89 102 10079 99989 0.26726 300.26726 150.26726 15176.99399 0.26726 300.26727 150.26727 15176.9943 0.26726 300.26726 150.26726 15176.99326 2020-01-01 2020-01-02 2020-01-01 00:01:29 2020-01-02 03:46:29 2020-01-01 00:01:29.000 2020-01-02 03:46:29.000 89 99989 50039 5053939 89 99989 50039 5053939 -32480 32455 4618.009900990099 466419 -125 126 -0.12871287128712872 -13 -890 100 10880 99791 2.67267 299.67267 151.17267 15117.26726 2.67267 299.67267 151.17267 15117.26708 2.67267 299.67267 151.17267 15117.26700 2020-01-01 2020-01-02 2020-01-01 00:14:50 2020-01-02 03:43:11 2020-01-01 00:14:50.000 2020-01-02 03:43:11.000 890 99791 50340.5 5034050 890 99791 50340.5 5034050 -32678 32257 4465.3 446530 -128 127 3.22 322 -891 100 10881 99792 2.67567 299.67567 151.17567 15117.56756 2.67567 299.6757 151.17567 15117.56768 2.67567 299.67567 151.17567 15117.56700 2020-01-01 2020-01-02 2020-01-01 00:14:51 2020-01-02 03:43:12 2020-01-01 00:14:51.000 2020-01-02 03:43:12.000 891 99792 50341.5 5034150 891 99792 50341.5 5034150 -32677 32258 4466.3 446630 -128 127 1.66 166 -892 100 10882 99793 2.67867 299.67867 151.17867 15117.86786 2.67867 299.67868 151.17868 15117.86802 2.67867 299.67867 151.17867 15117.86700 2020-01-01 2020-01-02 2020-01-01 00:14:52 2020-01-02 03:43:13 2020-01-01 00:14:52.000 2020-01-02 03:43:13.000 892 99793 50342.5 5034250 892 99793 50342.5 5034250 -32676 32259 4467.3 446730 -128 124 0.1 10 -893 100 10883 99794 2.68168 299.68168 151.18168 15118.16816 2.68168 299.68167 151.18168 15118.16816 2.68168 299.68168 151.18168 15118.16800 2020-01-01 2020-01-02 2020-01-01 00:14:53 2020-01-02 03:43:14 2020-01-01 00:14:53.000 2020-01-02 03:43:14.000 893 99794 50343.5 5034350 893 99794 50343.5 5034350 -32675 32260 4468.3 446830 -127 125 1.1 110 -894 100 10884 99795 2.68468 299.68468 151.18468 15118.46846 2.68468 299.6847 151.18468 15118.46826 2.68468 299.68468 151.18468 15118.46800 2020-01-01 2020-01-02 2020-01-01 00:14:54 2020-01-02 03:43:15 2020-01-01 00:14:54.000 2020-01-02 03:43:15.000 894 99795 50344.5 5034450 894 99795 50344.5 5034450 -32674 32261 4469.3 446930 -126 126 2.1 210 -895 100 10885 99796 2.68768 299.68768 151.18768 15118.76876 2.68768 299.68768 151.18768 15118.76855 2.68768 299.68768 151.18768 15118.76800 2020-01-01 2020-01-02 2020-01-01 00:14:55 2020-01-02 03:43:16 2020-01-01 00:14:55.000 2020-01-02 03:43:16.000 895 99796 50345.5 5034550 895 99796 50345.5 5034550 -32673 32262 4470.3 447030 -125 127 3.1 310 -896 100 10886 99797 2.69069 299.69069 151.19069 15119.06906 2.69069 299.6907 151.19069 15119.06915 2.69069 299.69069 151.19069 15119.06900 2020-01-01 2020-01-02 2020-01-01 00:14:56 2020-01-02 03:43:17 2020-01-01 00:14:56.000 2020-01-02 03:43:17.000 896 99797 50346.5 5034650 896 99797 50346.5 5034650 -32672 32263 4471.3 447130 -128 127 1.54 154 -897 100 10887 99798 2.69369 299.69369 151.19369 15119.36936 2.69369 299.6937 151.19369 15119.36949 2.69369 299.69369 151.19369 15119.36900 2020-01-01 2020-01-02 2020-01-01 00:14:57 2020-01-02 03:43:18 2020-01-01 00:14:57.000 2020-01-02 03:43:18.000 897 99798 50347.5 5034750 897 99798 50347.5 5034750 -32671 32264 4472.3 447230 -128 127 -0.02 -2 -898 100 10888 99799 2.69669 299.69669 151.19669 15119.66966 2.69669 299.6967 151.19669 15119.66963 2.69669 299.69669 151.19669 15119.66900 2020-01-01 2020-01-02 2020-01-01 00:14:58 2020-01-02 03:43:19 2020-01-01 00:14:58.000 2020-01-02 03:43:19.000 898 99799 50348.5 5034850 898 99799 50348.5 5034850 -32670 32265 4473.3 447330 -128 123 -1.58 -158 -899 100 10889 99800 2.69969 299.69969 151.19969 15119.96996 2.69969 299.6997 151.1997 15119.97038 2.69969 299.69969 151.19969 15119.96900 2020-01-01 2020-01-02 2020-01-01 00:14:59 2020-01-02 03:43:20 2020-01-01 00:14:59.000 2020-01-02 03:43:20.000 899 99800 50349.5 5034950 899 99800 50349.5 5034950 -32669 32266 4474.3 447430 -127 124 -0.58 -58 -9 102 1008 9999 0.02702 300.02702 150.02702 15152.72972 0.02702 300.02704 150.02702 15152.72966 0.02702 300.02702 150.02702 15152.72902 2020-01-01 2020-01-02 2020-01-01 00:00:09 2020-01-02 03:45:09 2020-01-01 00:00:09.000 2020-01-02 03:45:09.000 9 99909 49959 5045859 9 99909 49959 5045859 -32560 32375 4538.009900990099 458339 -124 127 0.9801980198019802 99 -90 102 10080 99990 0.27027 300.27027 150.27027 15177.29729 0.27027 300.27026 150.27026 15177.29694 0.27027 300.27027 150.27027 15177.29727 2020-01-01 2020-01-02 2020-01-01 00:01:30 2020-01-02 03:46:30 2020-01-01 00:01:30.000 2020-01-02 03:46:30.000 90 99990 50040 5054040 90 99990 50040 5054040 -32479 32456 4619.009900990099 466520 -124 127 0.8712871287128713 88 -900 100 10890 99801 2.7027 299.7027 151.2027 15120.27027 2.7027 299.7027 151.2027 15120.27003 2.70270 299.70270 151.20270 15120.27000 2020-01-01 2020-01-02 2020-01-01 00:15:00 2020-01-02 03:43:21 2020-01-01 00:15:00.000 2020-01-02 03:43:21.000 900 99801 50350.5 5035050 900 99801 50350.5 5035050 -32668 32267 4475.3 447530 -126 125 0.42 42 -901 100 10891 99802 2.7057 299.7057 151.2057 15120.57057 2.7057 299.70572 151.2057 15120.57066 2.70570 299.70570 151.20570 15120.57000 2020-01-01 2020-01-02 2020-01-01 00:15:01 2020-01-02 03:43:22 2020-01-01 00:15:01.000 2020-01-02 03:43:22.000 901 99802 50351.5 5035150 901 99802 50351.5 5035150 -32667 32268 4476.3 447630 -125 126 1.42 142 -902 100 10892 99803 2.7087 299.7087 151.2087 15120.87087 2.7087 299.7087 151.2087 15120.87095 2.70870 299.70870 151.20870 15120.87000 2020-01-01 2020-01-02 2020-01-01 00:15:02 2020-01-02 03:43:23 2020-01-01 00:15:02.000 2020-01-02 03:43:23.000 902 99803 50352.5 5035250 902 99803 50352.5 5035250 -32666 32269 4477.3 447730 -124 127 2.42 242 -903 100 10893 99804 2.71171 299.71171 151.21171 15121.17117 2.71171 299.7117 151.21171 15121.1711 2.71171 299.71171 151.21171 15121.17100 2020-01-01 2020-01-02 2020-01-01 00:15:03 2020-01-02 03:43:24 2020-01-01 00:15:03.000 2020-01-02 03:43:24.000 903 99804 50353.5 5035350 903 99804 50353.5 5035350 -32665 32270 4478.3 447830 -128 127 0.86 86 -904 100 10894 99805 2.71471 299.71471 151.21471 15121.47147 2.71471 299.71472 151.21471 15121.47185 2.71471 299.71471 151.21471 15121.47100 2020-01-01 2020-01-02 2020-01-01 00:15:04 2020-01-02 03:43:25 2020-01-01 00:15:04.000 2020-01-02 03:43:25.000 904 99805 50354.5 5035450 904 99805 50354.5 5035450 -32664 32271 4479.3 447930 -128 123 -0.7 -70 -905 100 10895 99806 2.71771 299.71771 151.21771 15121.77177 2.71771 299.7177 151.21771 15121.77149 2.71771 299.71771 151.21771 15121.77100 2020-01-01 2020-01-02 2020-01-01 00:15:05 2020-01-02 03:43:26 2020-01-01 00:15:05.000 2020-01-02 03:43:26.000 905 99806 50355.5 5035550 905 99806 50355.5 5035550 -32663 32272 4480.3 448030 -127 124 0.3 30 -906 100 10896 99807 2.72072 299.72072 151.22072 15122.07207 2.72072 299.72073 151.22072 15122.07212 2.72072 299.72072 151.22072 15122.07200 2020-01-01 2020-01-02 2020-01-01 00:15:06 2020-01-02 03:43:27 2020-01-01 00:15:06.000 2020-01-02 03:43:27.000 906 99807 50356.5 5035650 906 99807 50356.5 5035650 -32662 32273 4481.3 448130 -126 125 1.3 130 +881 100 10871 99782 2.64564 299.64564 151.14564 15114.56456 2.64564 299.64566 151.14564 15114.56474 2.64564 299.64564 151.14564000000013 15114.56400 2020-01-01 2020-01-02 2020-01-01 00:14:41 2020-01-02 03:43:02 2020-01-01 00:14:41.000 2020-01-02 03:43:02.000 881 99782 50331.5 5033150 881 99782 50331.5 5033150 -32687 32248 4456.3 445630 -126 125 1.9 190 +882 100 10872 99783 2.64864 299.64864 151.14864 15114.86486 2.64864 299.64865 151.14865 15114.86504 2.64864 299.64864 151.14863999999992 15114.86400 2020-01-01 2020-01-02 2020-01-01 00:14:42 2020-01-02 03:43:03 2020-01-01 00:14:42.000 2020-01-02 03:43:03.000 882 99783 50332.5 5033250 882 99783 50332.5 5033250 -32686 32249 4457.3 445730 -125 126 2.9 290 +883 100 10873 99784 2.65165 299.65165 151.15165 15115.16516 2.65165 299.65164 151.15165 15115.16522 2.65165 299.65165 151.15164999999982 15115.16500 2020-01-01 2020-01-02 2020-01-01 00:14:43 2020-01-02 03:43:04 2020-01-01 00:14:43.000 2020-01-02 03:43:04.000 883 99784 50333.5 5033350 883 99784 50333.5 5033350 -32685 32250 4458.3 445830 -124 127 3.9 390 +884 100 10874 99785 2.65465 299.65465 151.15465 15115.46546 2.65465 299.65466 151.15465 15115.46532 2.65465 299.65465 151.1546500000003 15115.46500 2020-01-01 2020-01-02 2020-01-01 00:14:44 2020-01-02 03:43:05 2020-01-01 00:14:44.000 2020-01-02 03:43:05.000 884 99785 50334.5 5033450 884 99785 50334.5 5033450 -32684 32251 4459.3 445930 -128 127 2.34 234 +885 100 10875 99786 2.65765 299.65765 151.15765 15115.76576 2.65765 299.65765 151.15765 15115.76562 2.65765 299.65765 151.1576499999998 15115.76500 2020-01-01 2020-01-02 2020-01-01 00:14:45 2020-01-02 03:43:06 2020-01-01 00:14:45.000 2020-01-02 03:43:06.000 885 99786 50335.5 5033550 885 99786 50335.5 5033550 -32683 32252 4460.3 446030 -128 123 0.78 78 +886 100 10876 99787 2.66066 299.66066 151.16066 15116.06606 2.66066 299.66068 151.16066 15116.06621 2.66066 299.66066 151.16065999999967 15116.06600 2020-01-01 2020-01-02 2020-01-01 00:14:46 2020-01-02 03:43:07 2020-01-01 00:14:46.000 2020-01-02 03:43:07.000 886 99787 50336.5 5033650 886 99787 50336.5 5033650 -32682 32253 4461.3 446130 -127 124 1.78 178 +887 100 10877 99788 2.66366 299.66366 151.16366 15116.36636 2.66366 299.66367 151.16366 15116.36651 2.66366 299.66366 151.16366000000014 15116.36600 2020-01-01 2020-01-02 2020-01-01 00:14:47 2020-01-02 03:43:08 2020-01-01 00:14:47.000 2020-01-02 03:43:08.000 887 99788 50337.5 5033750 887 99788 50337.5 5033750 -32681 32254 4462.3 446230 -126 125 2.78 278 +888 100 10878 99789 2.66666 299.66666 151.16666 15116.66666 2.66666 299.66666 151.16666 15116.66669 2.66666 299.66666 151.16666000000018 15116.66600 2020-01-01 2020-01-02 2020-01-01 00:14:48 2020-01-02 03:43:09 2020-01-01 00:14:48.000 2020-01-02 03:43:09.000 888 99789 50338.5 5033850 888 99789 50338.5 5033850 -32680 32255 4463.3 446330 -125 126 3.78 378 +889 100 10879 99790 2.66966 299.66966 151.16966 15116.96696 2.66966 299.66968 151.16966 15116.96679 2.66966 299.66966 151.16965999999968 15116.96600 2020-01-01 2020-01-02 2020-01-01 00:14:49 2020-01-02 03:43:10 2020-01-01 00:14:49.000 2020-01-02 03:43:10.000 889 99790 50339.5 5033950 889 99790 50339.5 5033950 -32679 32256 4464.3 446430 -124 127 4.78 478 +89 102 10079 99989 0.26726 300.26726 150.26726 15176.99399 0.26726 300.26727 150.26727 15176.9943 0.26726 300.26726 150.26726000000002 15176.99326 2020-01-01 2020-01-02 2020-01-01 00:01:29 2020-01-02 03:46:29 2020-01-01 00:01:29.000 2020-01-02 03:46:29.000 89 99989 50039 5053939 89 99989 50039 5053939 -32480 32455 4618.009900990099 466419 -125 126 -0.12871287128712872 -13 +890 100 10880 99791 2.67267 299.67267 151.17267 15117.26726 2.67267 299.67267 151.17267 15117.26708 2.67267 299.67267 151.17266999999995 15117.26700 2020-01-01 2020-01-02 2020-01-01 00:14:50 2020-01-02 03:43:11 2020-01-01 00:14:50.000 2020-01-02 03:43:11.000 890 99791 50340.5 5034050 890 99791 50340.5 5034050 -32678 32257 4465.3 446530 -128 127 3.22 322 +891 100 10881 99792 2.67567 299.67567 151.17567 15117.56756 2.67567 299.6757 151.17567 15117.56768 2.67567 299.67567 151.17567000000005 15117.56700 2020-01-01 2020-01-02 2020-01-01 00:14:51 2020-01-02 03:43:12 2020-01-01 00:14:51.000 2020-01-02 03:43:12.000 891 99792 50341.5 5034150 891 99792 50341.5 5034150 -32677 32258 4466.3 446630 -128 127 1.66 166 +892 100 10882 99793 2.67867 299.67867 151.17867 15117.86786 2.67867 299.67868 151.17868 15117.86802 2.67867 299.67867 151.17866999999987 15117.86700 2020-01-01 2020-01-02 2020-01-01 00:14:52 2020-01-02 03:43:13 2020-01-01 00:14:52.000 2020-01-02 03:43:13.000 892 99793 50342.5 5034250 892 99793 50342.5 5034250 -32676 32259 4467.3 446730 -128 124 0.1 10 +893 100 10883 99794 2.68168 299.68168 151.18168 15118.16816 2.68168 299.68167 151.18168 15118.16816 2.68168 299.68168 151.1816799999998 15118.16800 2020-01-01 2020-01-02 2020-01-01 00:14:53 2020-01-02 03:43:14 2020-01-01 00:14:53.000 2020-01-02 03:43:14.000 893 99794 50343.5 5034350 893 99794 50343.5 5034350 -32675 32260 4468.3 446830 -127 125 1.1 110 +894 100 10884 99795 2.68468 299.68468 151.18468 15118.46846 2.68468 299.6847 151.18468 15118.46826 2.68468 299.68468 151.18468000000018 15118.46800 2020-01-01 2020-01-02 2020-01-01 00:14:54 2020-01-02 03:43:15 2020-01-01 00:14:54.000 2020-01-02 03:43:15.000 894 99795 50344.5 5034450 894 99795 50344.5 5034450 -32674 32261 4469.3 446930 -126 126 2.1 210 +895 100 10885 99796 2.68768 299.68768 151.18768 15118.76876 2.68768 299.68768 151.18768 15118.76855 2.68768 299.68768 151.18767999999972 15118.76800 2020-01-01 2020-01-02 2020-01-01 00:14:55 2020-01-02 03:43:16 2020-01-01 00:14:55.000 2020-01-02 03:43:16.000 895 99796 50345.5 5034550 895 99796 50345.5 5034550 -32673 32262 4470.3 447030 -125 127 3.1 310 +896 100 10886 99797 2.69069 299.69069 151.19069 15119.06906 2.69069 299.6907 151.19069 15119.06915 2.69069 299.69069 151.19068999999993 15119.06900 2020-01-01 2020-01-02 2020-01-01 00:14:56 2020-01-02 03:43:17 2020-01-01 00:14:56.000 2020-01-02 03:43:17.000 896 99797 50346.5 5034650 896 99797 50346.5 5034650 -32672 32263 4471.3 447130 -128 127 1.54 154 +897 100 10887 99798 2.69369 299.69369 151.19369 15119.36936 2.69369 299.6937 151.19369 15119.36949 2.69369 299.69369 151.19369000000003 15119.36900 2020-01-01 2020-01-02 2020-01-01 00:14:57 2020-01-02 03:43:18 2020-01-01 00:14:57.000 2020-01-02 03:43:18.000 897 99798 50347.5 5034750 897 99798 50347.5 5034750 -32671 32264 4472.3 447230 -128 127 -0.02 -2 +898 100 10888 99799 2.69669 299.69669 151.19669 15119.66966 2.69669 299.6967 151.19669 15119.66963 2.69669 299.69669 151.19669000000013 15119.66900 2020-01-01 2020-01-02 2020-01-01 00:14:58 2020-01-02 03:43:19 2020-01-01 00:14:58.000 2020-01-02 03:43:19.000 898 99799 50348.5 5034850 898 99799 50348.5 5034850 -32670 32265 4473.3 447330 -128 123 -1.58 -158 +899 100 10889 99800 2.69969 299.69969 151.19969 15119.96996 2.69969 299.6997 151.1997 15119.97038 2.69969 299.69969 151.19968999999995 15119.96900 2020-01-01 2020-01-02 2020-01-01 00:14:59 2020-01-02 03:43:20 2020-01-01 00:14:59.000 2020-01-02 03:43:20.000 899 99800 50349.5 5034950 899 99800 50349.5 5034950 -32669 32266 4474.3 447430 -127 124 -0.58 -58 +9 102 1008 9999 0.02702 300.02702 150.02702 15152.72972 0.02702 300.02704 150.02702 15152.72966 0.02702 300.02702 150.02701999999977 15152.72902 2020-01-01 2020-01-02 2020-01-01 00:00:09 2020-01-02 03:45:09 2020-01-01 00:00:09.000 2020-01-02 03:45:09.000 9 99909 49959 5045859 9 99909 49959 5045859 -32560 32375 4538.009900990099 458339 -124 127 0.9801980198019802 99 +90 102 10080 99990 0.27027 300.27027 150.27027 15177.29729 0.27027 300.27026 150.27026 15177.29694 0.27027 300.27027 150.27027000000027 15177.29727 2020-01-01 2020-01-02 2020-01-01 00:01:30 2020-01-02 03:46:30 2020-01-01 00:01:30.000 2020-01-02 03:46:30.000 90 99990 50040 5054040 90 99990 50040 5054040 -32479 32456 4619.009900990099 466520 -124 127 0.8712871287128713 88 +900 100 10890 99801 2.7027 299.7027 151.2027 15120.27027 2.7027 299.7027 151.2027 15120.27003 2.70270 299.70270 151.20269999999985 15120.27000 2020-01-01 2020-01-02 2020-01-01 00:15:00 2020-01-02 03:43:21 2020-01-01 00:15:00.000 2020-01-02 03:43:21.000 900 99801 50350.5 5035050 900 99801 50350.5 5035050 -32668 32267 4475.3 447530 -126 125 0.42 42 +901 100 10891 99802 2.7057 299.7057 151.2057 15120.57057 2.7057 299.70572 151.2057 15120.57066 2.70570 299.70570 151.20570000000032 15120.57000 2020-01-01 2020-01-02 2020-01-01 00:15:01 2020-01-02 03:43:22 2020-01-01 00:15:01.000 2020-01-02 03:43:22.000 901 99802 50351.5 5035150 901 99802 50351.5 5035150 -32667 32268 4476.3 447630 -125 126 1.42 142 +902 100 10892 99803 2.7087 299.7087 151.2087 15120.87087 2.7087 299.7087 151.2087 15120.87095 2.70870 299.70870 151.20869999999982 15120.87000 2020-01-01 2020-01-02 2020-01-01 00:15:02 2020-01-02 03:43:23 2020-01-01 00:15:02.000 2020-01-02 03:43:23.000 902 99803 50352.5 5035250 902 99803 50352.5 5035250 -32666 32269 4477.3 447730 -124 127 2.42 242 +903 100 10893 99804 2.71171 299.71171 151.21171 15121.17117 2.71171 299.7117 151.21171 15121.1711 2.71171 299.71171 151.2117099999997 15121.17100 2020-01-01 2020-01-02 2020-01-01 00:15:03 2020-01-02 03:43:24 2020-01-01 00:15:03.000 2020-01-02 03:43:24.000 903 99804 50353.5 5035350 903 99804 50353.5 5035350 -32665 32270 4478.3 447830 -128 127 0.86 86 +904 100 10894 99805 2.71471 299.71471 151.21471 15121.47147 2.71471 299.71472 151.21471 15121.47185 2.71471 299.71471 151.21471000000017 15121.47100 2020-01-01 2020-01-02 2020-01-01 00:15:04 2020-01-02 03:43:25 2020-01-01 00:15:04.000 2020-01-02 03:43:25.000 904 99805 50354.5 5035450 904 99805 50354.5 5035450 -32664 32271 4479.3 447930 -128 123 -0.7 -70 +905 100 10895 99806 2.71771 299.71771 151.21771 15121.77177 2.71771 299.7177 151.21771 15121.77149 2.71771 299.71771 151.2177100000002 15121.77100 2020-01-01 2020-01-02 2020-01-01 00:15:05 2020-01-02 03:43:26 2020-01-01 00:15:05.000 2020-01-02 03:43:26.000 905 99806 50355.5 5035550 905 99806 50355.5 5035550 -32663 32272 4480.3 448030 -127 124 0.3 30 +906 100 10896 99807 2.72072 299.72072 151.22072 15122.07207 2.72072 299.72073 151.22072 15122.07212 2.72072 299.72072 151.22071999999986 15122.07200 2020-01-01 2020-01-02 2020-01-01 00:15:06 2020-01-02 03:43:27 2020-01-01 00:15:06.000 2020-01-02 03:43:27.000 906 99807 50356.5 5035650 906 99807 50356.5 5035650 -32662 32273 4481.3 448130 -126 125 1.3 130 907 100 10897 99808 2.72372 299.72372 151.22372 15122.37237 2.72372 299.72372 151.22372 15122.37243 2.72372 299.72372 151.22372 15122.37200 2020-01-01 2020-01-02 2020-01-01 00:15:07 2020-01-02 03:43:28 2020-01-01 00:15:07.000 2020-01-02 03:43:28.000 907 99808 50357.5 5035750 907 99808 50357.5 5035750 -32661 32274 4482.3 448230 -125 126 2.3 230 -908 100 10898 99809 2.72672 299.72672 151.22672 15122.67267 2.72672 299.7267 151.22672 15122.67272 2.72672 299.72672 151.22672 15122.67200 2020-01-01 2020-01-02 2020-01-01 00:15:08 2020-01-02 03:43:29 2020-01-01 00:15:08.000 2020-01-02 03:43:29.000 908 99809 50358.5 5035850 908 99809 50358.5 5035850 -32660 32275 4483.3 448330 -124 127 3.3 330 -909 100 10899 99810 2.72972 299.72972 151.22972 15122.97297 2.72972 299.72974 151.22973 15122.97332 2.72972 299.72972 151.22972 15122.97200 2020-01-01 2020-01-02 2020-01-01 00:15:09 2020-01-02 03:43:30 2020-01-01 00:15:09.000 2020-01-02 03:43:30.000 909 99810 50359.5 5035950 909 99810 50359.5 5035950 -32659 32276 4484.3 448430 -128 127 1.74 174 -91 102 10081 99991 0.27327 300.27327 150.27327 15177.6006 0.27327 300.2733 150.27327 15177.60054 0.27327 300.27327 150.27327 15177.60027 2020-01-01 2020-01-02 2020-01-01 00:01:31 2020-01-02 03:46:31 2020-01-01 00:01:31.000 2020-01-02 03:46:31.000 91 99991 50041 5054141 91 99991 50041 5054141 -32478 32457 4620.009900990099 466621 -128 127 -0.6633663366336634 -67 -910 100 10900 99811 2.73273 299.73273 151.23273 15123.27327 2.73273 299.73273 151.23272 15123.27296 2.73273 299.73273 151.23273 15123.27300 2020-01-01 2020-01-02 2020-01-01 00:15:10 2020-01-02 03:43:31 2020-01-01 00:15:10.000 2020-01-02 03:43:31.000 910 99811 50360.5 5036050 910 99811 50360.5 5036050 -32658 32277 4485.3 448530 -128 123 0.18 18 -911 100 10901 99812 2.73573 299.73573 151.23573 15123.57357 2.73573 299.73575 151.23573 15123.57359 2.73573 299.73573 151.23573 15123.57300 2020-01-01 2020-01-02 2020-01-01 00:15:11 2020-01-02 03:43:32 2020-01-01 00:15:11.000 2020-01-02 03:43:32.000 911 99812 50361.5 5036150 911 99812 50361.5 5036150 -32657 32278 4486.3 448630 -127 124 1.18 118 -912 100 10902 99813 2.73873 299.73873 151.23873 15123.87387 2.73873 299.73874 151.23873 15123.8739 2.73873 299.73873 151.23873 15123.87300 2020-01-01 2020-01-02 2020-01-01 00:15:12 2020-01-02 03:43:33 2020-01-01 00:15:12.000 2020-01-02 03:43:33.000 912 99813 50362.5 5036250 912 99813 50362.5 5036250 -32656 32279 4487.3 448730 -126 125 2.18 218 -913 100 10903 99814 2.74174 299.74174 151.24174 15124.17417 2.74174 299.74173 151.24174 15124.17419 2.74174 299.74174 151.24174 15124.17400 2020-01-01 2020-01-02 2020-01-01 00:15:13 2020-01-02 03:43:34 2020-01-01 00:15:13.000 2020-01-02 03:43:34.000 913 99814 50363.5 5036350 913 99814 50363.5 5036350 -32655 32280 4488.3 448830 -125 126 3.18 318 -914 100 10904 99815 2.74474 299.74474 151.24474 15124.47447 2.74474 299.74475 151.24474 15124.47479 2.74474 299.74474 151.24474 15124.47400 2020-01-01 2020-01-02 2020-01-01 00:15:14 2020-01-02 03:43:35 2020-01-01 00:15:14.000 2020-01-02 03:43:35.000 914 99815 50364.5 5036450 914 99815 50364.5 5036450 -32654 32281 4489.3 448930 -124 127 4.18 418 -915 100 10905 99816 2.74774 299.74774 151.24774 15124.77477 2.74774 299.74774 151.24774 15124.77447 2.74774 299.74774 151.24774 15124.77400 2020-01-01 2020-01-02 2020-01-01 00:15:15 2020-01-02 03:43:36 2020-01-01 00:15:15.000 2020-01-02 03:43:36.000 915 99816 50365.5 5036550 915 99816 50365.5 5036550 -32653 32282 4490.3 449030 -128 127 2.62 262 -916 100 10906 99817 2.75075 299.75075 151.25075 15125.07507 2.75075 299.75076 151.25075 15125.07507 2.75075 299.75075 151.25075 15125.07500 2020-01-01 2020-01-02 2020-01-01 00:15:16 2020-01-02 03:43:37 2020-01-01 00:15:16.000 2020-01-02 03:43:37.000 916 99817 50366.5 5036650 916 99817 50366.5 5036650 -32652 32283 4491.3 449130 -128 127 1.06 106 -917 100 10907 99818 2.75375 299.75375 151.25375 15125.37537 2.75375 299.75375 151.25375 15125.37536 2.75375 299.75375 151.25375 15125.37500 2020-01-01 2020-01-02 2020-01-01 00:15:17 2020-01-02 03:43:38 2020-01-01 00:15:17.000 2020-01-02 03:43:38.000 917 99818 50367.5 5036750 917 99818 50367.5 5036750 -32651 32284 4492.3 449230 -128 124 -0.5 -50 -918 100 10908 99819 2.75675 299.75675 151.25675 15125.67567 2.75675 299.75674 151.25675 15125.67566 2.75675 299.75675 151.25675 15125.67500 2020-01-01 2020-01-02 2020-01-01 00:15:18 2020-01-02 03:43:39 2020-01-01 00:15:18.000 2020-01-02 03:43:39.000 918 99819 50368.5 5036850 918 99819 50368.5 5036850 -32650 32285 4493.3 449330 -127 125 0.5 50 -919 100 10909 99820 2.75975 299.75975 151.25975 15125.97597 2.75975 299.75977 151.25976 15125.97626 2.75975 299.75975 151.25975 15125.97500 2020-01-01 2020-01-02 2020-01-01 00:15:19 2020-01-02 03:43:40 2020-01-01 00:15:19.000 2020-01-02 03:43:40.000 919 99820 50369.5 5036950 919 99820 50369.5 5036950 -32649 32286 4494.3 449430 -126 126 1.5 150 -92 102 10082 99992 0.27627 300.27627 150.27627 15177.9039 0.27627 300.27628 150.27627 15177.90384 0.27627 300.27627 150.27627 15177.90327 2020-01-01 2020-01-02 2020-01-01 00:01:32 2020-01-02 03:46:32 2020-01-01 00:01:32.000 2020-01-02 03:46:32.000 92 99992 50042 5054242 92 99992 50042 5054242 -32477 32458 4621.009900990099 466722 -128 123 -2.198019801980198 -222 -920 100 10910 99821 2.76276 299.76276 151.26276 15126.27627 2.76276 299.76276 151.26275 15126.27594 2.76276 299.76276 151.26276 15126.27600 2020-01-01 2020-01-02 2020-01-01 00:15:20 2020-01-02 03:43:41 2020-01-01 00:15:20.000 2020-01-02 03:43:41.000 920 99821 50370.5 5037050 920 99821 50370.5 5037050 -32648 32287 4495.3 449530 -125 127 2.5 250 -921 100 10911 99822 2.76576 299.76576 151.26576 15126.57657 2.76576 299.76578 151.26576 15126.57654 2.76576 299.76576 151.26576 15126.57600 2020-01-01 2020-01-02 2020-01-01 00:15:21 2020-01-02 03:43:42 2020-01-01 00:15:21.000 2020-01-02 03:43:42.000 921 99822 50371.5 5037150 921 99822 50371.5 5037150 -32647 32288 4496.3 449630 -128 127 0.94 94 -922 100 10912 99823 2.76876 299.76876 151.26876 15126.87687 2.76876 299.76877 151.26876 15126.87683 2.76876 299.76876 151.26876 15126.87600 2020-01-01 2020-01-02 2020-01-01 00:15:22 2020-01-02 03:43:43 2020-01-01 00:15:22.000 2020-01-02 03:43:43.000 922 99823 50372.5 5037250 922 99823 50372.5 5037250 -32646 32289 4497.3 449730 -128 127 -0.62 -62 -923 100 10913 99824 2.77177 299.77177 151.27177 15127.17717 2.77177 299.77176 151.27177 15127.17713 2.77177 299.77177 151.27177 15127.17700 2020-01-01 2020-01-02 2020-01-01 00:15:23 2020-01-02 03:43:44 2020-01-01 00:15:23.000 2020-01-02 03:43:44.000 923 99824 50373.5 5037350 923 99824 50373.5 5037350 -32645 32290 4498.3 449830 -128 123 -2.18 -218 +908 100 10898 99809 2.72672 299.72672 151.22672 15122.67267 2.72672 299.7267 151.22672 15122.67272 2.72672 299.72672 151.22672000000009 15122.67200 2020-01-01 2020-01-02 2020-01-01 00:15:08 2020-01-02 03:43:29 2020-01-01 00:15:08.000 2020-01-02 03:43:29.000 908 99809 50358.5 5035850 908 99809 50358.5 5035850 -32660 32275 4483.3 448330 -124 127 3.3 330 +909 100 10899 99810 2.72972 299.72972 151.22972 15122.97297 2.72972 299.72974 151.22973 15122.97332 2.72972 299.72972 151.22971999999987 15122.97200 2020-01-01 2020-01-02 2020-01-01 00:15:09 2020-01-02 03:43:30 2020-01-01 00:15:09.000 2020-01-02 03:43:30.000 909 99810 50359.5 5035950 909 99810 50359.5 5035950 -32659 32276 4484.3 448430 -128 127 1.74 174 +91 102 10081 99991 0.27327 300.27327 150.27327 15177.6006 0.27327 300.2733 150.27327 15177.60054 0.27327 300.27327 150.2732699999998 15177.60027 2020-01-01 2020-01-02 2020-01-01 00:01:31 2020-01-02 03:46:31 2020-01-01 00:01:31.000 2020-01-02 03:46:31.000 91 99991 50041 5054141 91 99991 50041 5054141 -32478 32457 4620.009900990099 466621 -128 127 -0.6633663366336634 -67 +910 100 10900 99811 2.73273 299.73273 151.23273 15123.27327 2.73273 299.73273 151.23272 15123.27296 2.73273 299.73273 151.23272999999983 15123.27300 2020-01-01 2020-01-02 2020-01-01 00:15:10 2020-01-02 03:43:31 2020-01-01 00:15:10.000 2020-01-02 03:43:31.000 910 99811 50360.5 5036050 910 99811 50360.5 5036050 -32658 32277 4485.3 448530 -128 123 0.18 18 +911 100 10901 99812 2.73573 299.73573 151.23573 15123.57357 2.73573 299.73575 151.23573 15123.57359 2.73573 299.73573 151.23573000000022 15123.57300 2020-01-01 2020-01-02 2020-01-01 00:15:11 2020-01-02 03:43:32 2020-01-01 00:15:11.000 2020-01-02 03:43:32.000 911 99812 50361.5 5036150 911 99812 50361.5 5036150 -32657 32278 4486.3 448630 -127 124 1.18 118 +912 100 10902 99813 2.73873 299.73873 151.23873 15123.87387 2.73873 299.73874 151.23873 15123.8739 2.73873 299.73873 151.23872999999972 15123.87300 2020-01-01 2020-01-02 2020-01-01 00:15:12 2020-01-02 03:43:33 2020-01-01 00:15:12.000 2020-01-02 03:43:33.000 912 99813 50362.5 5036250 912 99813 50362.5 5036250 -32656 32279 4487.3 448730 -126 125 2.18 218 +913 100 10903 99814 2.74174 299.74174 151.24174 15124.17417 2.74174 299.74173 151.24174 15124.17419 2.74174 299.74174 151.24173999999996 15124.17400 2020-01-01 2020-01-02 2020-01-01 00:15:13 2020-01-02 03:43:34 2020-01-01 00:15:13.000 2020-01-02 03:43:34.000 913 99814 50363.5 5036350 913 99814 50363.5 5036350 -32655 32280 4488.3 448830 -125 126 3.18 318 +914 100 10904 99815 2.74474 299.74474 151.24474 15124.47447 2.74474 299.74475 151.24474 15124.47479 2.74474 299.74474 151.24474000000006 15124.47400 2020-01-01 2020-01-02 2020-01-01 00:15:14 2020-01-02 03:43:35 2020-01-01 00:15:14.000 2020-01-02 03:43:35.000 914 99815 50364.5 5036450 914 99815 50364.5 5036450 -32654 32281 4489.3 448930 -124 127 4.18 418 +915 100 10905 99816 2.74774 299.74774 151.24774 15124.77477 2.74774 299.74774 151.24774 15124.77447 2.74774 299.74774 151.24774000000014 15124.77400 2020-01-01 2020-01-02 2020-01-01 00:15:15 2020-01-02 03:43:36 2020-01-01 00:15:15.000 2020-01-02 03:43:36.000 915 99816 50365.5 5036550 915 99816 50365.5 5036550 -32653 32282 4490.3 449030 -128 127 2.62 262 +916 100 10906 99817 2.75075 299.75075 151.25075 15125.07507 2.75075 299.75076 151.25075 15125.07507 2.75075 299.75075 151.2507499999998 15125.07500 2020-01-01 2020-01-02 2020-01-01 00:15:16 2020-01-02 03:43:37 2020-01-01 00:15:16.000 2020-01-02 03:43:37.000 916 99817 50366.5 5036650 916 99817 50366.5 5036650 -32652 32283 4491.3 449130 -128 127 1.06 106 +917 100 10907 99818 2.75375 299.75375 151.25375 15125.37537 2.75375 299.75375 151.25375 15125.37536 2.75375 299.75375 151.25374999999988 15125.37500 2020-01-01 2020-01-02 2020-01-01 00:15:17 2020-01-02 03:43:38 2020-01-01 00:15:17.000 2020-01-02 03:43:38.000 917 99818 50367.5 5036750 917 99818 50367.5 5036750 -32651 32284 4492.3 449230 -128 124 -0.5 -50 +918 100 10908 99819 2.75675 299.75675 151.25675 15125.67567 2.75675 299.75674 151.25675 15125.67566 2.75675 299.75675 151.25675000000007 15125.67500 2020-01-01 2020-01-02 2020-01-01 00:15:18 2020-01-02 03:43:39 2020-01-01 00:15:18.000 2020-01-02 03:43:39.000 918 99819 50368.5 5036850 918 99819 50368.5 5036850 -32650 32285 4493.3 449330 -127 125 0.5 50 +919 100 10909 99820 2.75975 299.75975 151.25975 15125.97597 2.75975 299.75977 151.25976 15125.97626 2.75975 299.75975 151.25974999999983 15125.97500 2020-01-01 2020-01-02 2020-01-01 00:15:19 2020-01-02 03:43:40 2020-01-01 00:15:19.000 2020-01-02 03:43:40.000 919 99820 50369.5 5036950 919 99820 50369.5 5036950 -32649 32286 4494.3 449430 -126 126 1.5 150 +92 102 10082 99992 0.27627 300.27627 150.27627 15177.9039 0.27627 300.27628 150.27627 15177.90384 0.27627 300.27627 150.27627000000018 15177.90327 2020-01-01 2020-01-02 2020-01-01 00:01:32 2020-01-02 03:46:32 2020-01-01 00:01:32.000 2020-01-02 03:46:32.000 92 99992 50042 5054242 92 99992 50042 5054242 -32477 32458 4621.009900990099 466722 -128 123 -2.198019801980198 -222 +920 100 10910 99821 2.76276 299.76276 151.26276 15126.27627 2.76276 299.76276 151.26275 15126.27594 2.76276 299.76276 151.26275999999973 15126.27600 2020-01-01 2020-01-02 2020-01-01 00:15:20 2020-01-02 03:43:41 2020-01-01 00:15:20.000 2020-01-02 03:43:41.000 920 99821 50370.5 5037050 920 99821 50370.5 5037050 -32648 32287 4495.3 449530 -125 127 2.5 250 +921 100 10911 99822 2.76576 299.76576 151.26576 15126.57657 2.76576 299.76578 151.26576 15126.57654 2.76576 299.76576 151.2657600000002 15126.57600 2020-01-01 2020-01-02 2020-01-01 00:15:21 2020-01-02 03:43:42 2020-01-01 00:15:21.000 2020-01-02 03:43:42.000 921 99822 50371.5 5037150 921 99822 50371.5 5037150 -32647 32288 4496.3 449630 -128 127 0.94 94 +922 100 10912 99823 2.76876 299.76876 151.26876 15126.87687 2.76876 299.76877 151.26876 15126.87683 2.76876 299.76876 151.26876000000024 15126.87600 2020-01-01 2020-01-02 2020-01-01 00:15:22 2020-01-02 03:43:43 2020-01-01 00:15:22.000 2020-01-02 03:43:43.000 922 99823 50372.5 5037250 922 99823 50372.5 5037250 -32646 32289 4497.3 449730 -128 127 -0.62 -62 +923 100 10913 99824 2.77177 299.77177 151.27177 15127.17717 2.77177 299.77176 151.27177 15127.17713 2.77177 299.77177 151.2717699999999 15127.17700 2020-01-01 2020-01-02 2020-01-01 00:15:23 2020-01-02 03:43:44 2020-01-01 00:15:23.000 2020-01-02 03:43:44.000 923 99824 50373.5 5037350 923 99824 50373.5 5037350 -32645 32290 4498.3 449830 -128 123 -2.18 -218 924 100 10914 99825 2.77477 299.77477 151.27477 15127.47747 2.77477 299.77478 151.27477 15127.47776 2.77477 299.77477 151.27477 15127.47700 2020-01-01 2020-01-02 2020-01-01 00:15:24 2020-01-02 03:43:45 2020-01-01 00:15:24.000 2020-01-02 03:43:45.000 924 99825 50374.5 5037450 924 99825 50374.5 5037450 -32644 32291 4499.3 449930 -127 124 -1.18 -118 -925 100 10915 99826 2.77777 299.77777 151.27777 15127.77777 2.77777 299.77777 151.27777 15127.77741 2.77777 299.77777 151.27777 15127.77700 2020-01-01 2020-01-02 2020-01-01 00:15:25 2020-01-02 03:43:46 2020-01-01 00:15:25.000 2020-01-02 03:43:46.000 925 99826 50375.5 5037550 925 99826 50375.5 5037550 -32643 32292 4500.3 450030 -126 125 -0.18 -18 -926 100 10916 99827 2.78078 299.78078 151.28078 15128.07807 2.78078 299.7808 151.28078 15128.078 2.78078 299.78078 151.28078 15128.07800 2020-01-01 2020-01-02 2020-01-01 00:15:26 2020-01-02 03:43:47 2020-01-01 00:15:26.000 2020-01-02 03:43:47.000 926 99827 50376.5 5037650 926 99827 50376.5 5037650 -32642 32293 4501.3 450130 -125 126 0.82 82 -927 100 10917 99828 2.78378 299.78378 151.28378 15128.37837 2.78378 299.78378 151.28378 15128.3783 2.78378 299.78378 151.28378 15128.37800 2020-01-01 2020-01-02 2020-01-01 00:15:27 2020-01-02 03:43:48 2020-01-01 00:15:27.000 2020-01-02 03:43:48.000 927 99828 50377.5 5037750 927 99828 50377.5 5037750 -32641 32294 4502.3 450230 -124 127 1.82 182 -928 100 10918 99829 2.78678 299.78678 151.28678 15128.67867 2.78678 299.78677 151.28678 15128.6786 2.78678 299.78678 151.28678 15128.67800 2020-01-01 2020-01-02 2020-01-01 00:15:28 2020-01-02 03:43:49 2020-01-01 00:15:28.000 2020-01-02 03:43:49.000 928 99829 50378.5 5037850 928 99829 50378.5 5037850 -32640 32295 4503.3 450330 -128 127 0.26 26 -929 100 10919 99830 2.78978 299.78978 151.28978 15128.97897 2.78978 299.7898 151.28979 15128.97923 2.78978 299.78978 151.28978 15128.97800 2020-01-01 2020-01-02 2020-01-01 00:15:29 2020-01-02 03:43:50 2020-01-01 00:15:29.000 2020-01-02 03:43:50.000 929 99830 50379.5 5037950 929 99830 50379.5 5037950 -32639 32296 4504.3 450430 -128 123 -1.3 -130 -93 102 10083 99993 0.27927 300.27927 150.27927 15178.2072 0.27927 300.27927 150.27927 15178.20715 0.27927 300.27927 150.27927 15178.20627 2020-01-01 2020-01-02 2020-01-01 00:01:33 2020-01-02 03:46:33 2020-01-01 00:01:33.000 2020-01-02 03:46:33.000 93 99993 50043 5054343 93 99993 50043 5054343 -32476 32459 4622.009900990099 466823 -127 124 -1.198019801980198 -121 +925 100 10915 99826 2.77777 299.77777 151.27777 15127.77777 2.77777 299.77777 151.27777 15127.77741 2.77777 299.77777 151.2777700000001 15127.77700 2020-01-01 2020-01-02 2020-01-01 00:15:25 2020-01-02 03:43:46 2020-01-01 00:15:25.000 2020-01-02 03:43:46.000 925 99826 50375.5 5037550 925 99826 50375.5 5037550 -32643 32292 4500.3 450030 -126 125 -0.18 -18 +926 100 10916 99827 2.78078 299.78078 151.28078 15128.07807 2.78078 299.7808 151.28078 15128.078 2.78078 299.78078 151.28077999999974 15128.07800 2020-01-01 2020-01-02 2020-01-01 00:15:26 2020-01-02 03:43:47 2020-01-01 00:15:26.000 2020-01-02 03:43:47.000 926 99827 50376.5 5037650 926 99827 50376.5 5037650 -32642 32293 4501.3 450130 -125 126 0.82 82 +927 100 10917 99828 2.78378 299.78378 151.28378 15128.37837 2.78378 299.78378 151.28378 15128.3783 2.78378 299.78378 151.28377999999984 15128.37800 2020-01-01 2020-01-02 2020-01-01 00:15:27 2020-01-02 03:43:48 2020-01-01 00:15:27.000 2020-01-02 03:43:48.000 927 99828 50377.5 5037750 927 99828 50377.5 5037750 -32641 32294 4502.3 450230 -124 127 1.82 182 +928 100 10918 99829 2.78678 299.78678 151.28678 15128.67867 2.78678 299.78677 151.28678 15128.6786 2.78678 299.78678 151.28678000000025 15128.67800 2020-01-01 2020-01-02 2020-01-01 00:15:28 2020-01-02 03:43:49 2020-01-01 00:15:28.000 2020-01-02 03:43:49.000 928 99829 50378.5 5037850 928 99829 50378.5 5037850 -32640 32295 4503.3 450330 -128 127 0.26 26 +929 100 10919 99830 2.78978 299.78978 151.28978 15128.97897 2.78978 299.7898 151.28979 15128.97923 2.78978 299.78978 151.28977999999975 15128.97800 2020-01-01 2020-01-02 2020-01-01 00:15:29 2020-01-02 03:43:50 2020-01-01 00:15:29.000 2020-01-02 03:43:50.000 929 99830 50379.5 5037950 929 99830 50379.5 5037950 -32639 32296 4504.3 450430 -128 123 -1.3 -130 +93 102 10083 99993 0.27927 300.27927 150.27927 15178.2072 0.27927 300.27927 150.27927 15178.20715 0.27927 300.27927 150.2792699999997 15178.20627 2020-01-01 2020-01-02 2020-01-01 00:01:33 2020-01-02 03:46:33 2020-01-01 00:01:33.000 2020-01-02 03:46:33.000 93 99993 50043 5054343 93 99993 50043 5054343 -32476 32459 4622.009900990099 466823 -127 124 -1.198019801980198 -121 930 100 10920 99831 2.79279 299.79279 151.29279 15129.27927 2.79279 299.7928 151.29278 15129.27888 2.79279 299.79279 151.29279 15129.27900 2020-01-01 2020-01-02 2020-01-01 00:15:30 2020-01-02 03:43:51 2020-01-01 00:15:30.000 2020-01-02 03:43:51.000 930 99831 50380.5 5038050 930 99831 50380.5 5038050 -32638 32297 4505.3 450530 -127 124 -0.3 -30 -931 100 10921 99832 2.79579 299.79579 151.29579 15129.57957 2.79579 299.7958 151.29579 15129.57963 2.79579 299.79579 151.29579 15129.57900 2020-01-01 2020-01-02 2020-01-01 00:15:31 2020-01-02 03:43:52 2020-01-01 00:15:31.000 2020-01-02 03:43:52.000 931 99832 50381.5 5038150 931 99832 50381.5 5038150 -32637 32298 4506.3 450630 -126 125 0.7 70 -932 100 10922 99833 2.79879 299.79879 151.29879 15129.87987 2.79879 299.7988 151.29879 15129.87977 2.79879 299.79879 151.29879 15129.87900 2020-01-01 2020-01-02 2020-01-01 00:15:32 2020-01-02 03:43:53 2020-01-01 00:15:32.000 2020-01-02 03:43:53.000 932 99833 50382.5 5038250 932 99833 50382.5 5038250 -32636 32299 4507.3 450730 -125 126 1.7 170 -933 100 10923 99834 2.8018 299.8018 151.3018 15130.18018 2.8018 299.8018 151.3018 15130.18011 2.80180 299.80180 151.30180 15130.18000 2020-01-01 2020-01-02 2020-01-01 00:15:33 2020-01-02 03:43:54 2020-01-01 00:15:33.000 2020-01-02 03:43:54.000 933 99834 50383.5 5038350 933 99834 50383.5 5038350 -32635 32300 4508.3 450830 -124 127 2.7 270 -934 100 10924 99835 2.8048 299.8048 151.3048 15130.48048 2.8048 299.8048 151.3048 15130.48071 2.80480 299.80480 151.30480 15130.48000 2020-01-01 2020-01-02 2020-01-01 00:15:34 2020-01-02 03:43:55 2020-01-01 00:15:34.000 2020-01-02 03:43:55.000 934 99835 50384.5 5038450 934 99835 50384.5 5038450 -32634 32301 4509.3 450930 -128 127 1.14 114 -935 100 10925 99836 2.8078 299.8078 151.3078 15130.78078 2.8078 299.8078 151.3078 15130.78034 2.80780 299.80780 151.30780 15130.78000 2020-01-01 2020-01-02 2020-01-01 00:15:35 2020-01-02 03:43:56 2020-01-01 00:15:35.000 2020-01-02 03:43:56.000 935 99836 50385.5 5038550 935 99836 50385.5 5038550 -32633 32302 4510.3 451030 -128 123 -0.42 -42 -936 100 10926 99837 2.81081 299.81081 151.31081 15131.08108 2.81081 299.81082 151.31081 15131.0811 2.81081 299.81081 151.31081 15131.08100 2020-01-01 2020-01-02 2020-01-01 00:15:36 2020-01-02 03:43:57 2020-01-01 00:15:36.000 2020-01-02 03:43:57.000 936 99837 50386.5 5038650 936 99837 50386.5 5038650 -32632 32303 4511.3 451130 -127 124 0.58 58 -937 100 10927 99838 2.81381 299.81381 151.31381 15131.38138 2.81381 299.8138 151.31381 15131.38124 2.81381 299.81381 151.31381 15131.38100 2020-01-01 2020-01-02 2020-01-01 00:15:37 2020-01-02 03:43:58 2020-01-01 00:15:37.000 2020-01-02 03:43:58.000 937 99838 50387.5 5038750 937 99838 50387.5 5038750 -32631 32304 4512.3 451230 -126 125 1.58 158 -938 100 10928 99839 2.81681 299.81681 151.31681 15131.68168 2.81681 299.8168 151.31681 15131.68157 2.81681 299.81681 151.31681 15131.68100 2020-01-01 2020-01-02 2020-01-01 00:15:38 2020-01-02 03:43:59 2020-01-01 00:15:38.000 2020-01-02 03:43:59.000 938 99839 50388.5 5038850 938 99839 50388.5 5038850 -32630 32305 4513.3 451330 -125 126 2.58 258 -939 100 10929 99840 2.81981 299.81981 151.31981 15131.98198 2.81982 299.81982 151.31982 15131.98217 2.81981 299.81981 151.31981 15131.98100 2020-01-01 2020-01-02 2020-01-01 00:15:39 2020-01-02 03:44:00 2020-01-01 00:15:39.000 2020-01-02 03:44:00.000 939 99840 50389.5 5038950 939 99840 50389.5 5038950 -32629 32306 4514.3 451430 -124 127 3.58 358 -94 102 10084 99994 0.28228 300.28228 150.28228 15178.51051 0.28228 300.2823 150.28228 15178.51078 0.28228 300.28228 150.28228 15178.51028 2020-01-01 2020-01-02 2020-01-01 00:01:34 2020-01-02 03:46:34 2020-01-01 00:01:34.000 2020-01-02 03:46:34.000 94 99994 50044 5054444 94 99994 50044 5054444 -32475 32460 4623.009900990099 466924 -126 125 -0.19801980198019803 -20 -940 100 10930 99841 2.82282 299.82282 151.32282 15132.28228 2.82282 299.8228 151.32282 15132.28247 2.82282 299.82282 151.32282 15132.28200 2020-01-01 2020-01-02 2020-01-01 00:15:40 2020-01-02 03:44:01 2020-01-01 00:15:40.000 2020-01-02 03:44:01.000 940 99841 50390.5 5039050 940 99841 50390.5 5039050 -32628 32307 4515.3 451530 -128 127 2.02 202 -941 100 10931 99842 2.82582 299.82582 151.32582 15132.58258 2.82582 299.82584 151.32582 15132.58257 2.82582 299.82582 151.32582 15132.58200 2020-01-01 2020-01-02 2020-01-01 00:15:41 2020-01-02 03:44:02 2020-01-01 00:15:41.000 2020-01-02 03:44:02.000 941 99842 50391.5 5039150 941 99842 50391.5 5039150 -32627 32308 4516.3 451630 -128 127 0.46 46 -942 100 10932 99843 2.82882 299.82882 151.32882 15132.88288 2.82882 299.82883 151.32882 15132.88275 2.82882 299.82882 151.32882 15132.88200 2020-01-01 2020-01-02 2020-01-01 00:15:42 2020-01-02 03:44:03 2020-01-01 00:15:42.000 2020-01-02 03:44:03.000 942 99843 50392.5 5039250 942 99843 50392.5 5039250 -32626 32309 4517.3 451730 -128 124 -1.1 -110 -943 100 10933 99844 2.83183 299.83183 151.33183 15133.18318 2.83183 299.83182 151.33183 15133.18304 2.83183 299.83183 151.33183 15133.18300 2020-01-01 2020-01-02 2020-01-01 00:15:43 2020-01-02 03:44:04 2020-01-01 00:15:43.000 2020-01-02 03:44:04.000 943 99844 50393.5 5039350 943 99844 50393.5 5039350 -32625 32310 4518.3 451830 -127 125 -0.1 -10 -944 100 10934 99845 2.83483 299.83483 151.33483 15133.48348 2.83483 299.83484 151.33483 15133.48364 2.83483 299.83483 151.33483 15133.48300 2020-01-01 2020-01-02 2020-01-01 00:15:44 2020-01-02 03:44:05 2020-01-01 00:15:44.000 2020-01-02 03:44:05.000 944 99845 50394.5 5039450 944 99845 50394.5 5039450 -32624 32311 4519.3 451930 -126 126 0.9 90 -945 100 10935 99846 2.83783 299.83783 151.33783 15133.78378 2.83783 299.83783 151.33783 15133.78393 2.83783 299.83783 151.33783 15133.78300 2020-01-01 2020-01-02 2020-01-01 00:15:45 2020-01-02 03:44:06 2020-01-01 00:15:45.000 2020-01-02 03:44:06.000 945 99846 50395.5 5039550 945 99846 50395.5 5039550 -32623 32312 4520.3 452030 -125 127 1.9 190 -946 100 10936 99847 2.84084 299.84084 151.34084 15134.08408 2.84084 299.84085 151.34084 15134.08404 2.84084 299.84084 151.34084 15134.08400 2020-01-01 2020-01-02 2020-01-01 00:15:46 2020-01-02 03:44:07 2020-01-01 00:15:46.000 2020-01-02 03:44:07.000 946 99847 50396.5 5039650 946 99847 50396.5 5039650 -32622 32313 4521.3 452130 -128 127 0.34 34 -947 100 10937 99848 2.84384 299.84384 151.34384 15134.38438 2.84384 299.84384 151.34384 15134.38421 2.84384 299.84384 151.34384 15134.38400 2020-01-01 2020-01-02 2020-01-01 00:15:47 2020-01-02 03:44:08 2020-01-01 00:15:47.000 2020-01-02 03:44:08.000 947 99848 50397.5 5039750 947 99848 50397.5 5039750 -32621 32314 4522.3 452230 -128 127 -1.22 -122 -948 100 10938 99849 2.84684 299.84684 151.34684 15134.68468 2.84684 299.84683 151.34684 15134.68452 2.84684 299.84684 151.34684 15134.68400 2020-01-01 2020-01-02 2020-01-01 00:15:48 2020-01-02 03:44:09 2020-01-01 00:15:48.000 2020-01-02 03:44:09.000 948 99849 50398.5 5039850 948 99849 50398.5 5039850 -32620 32315 4523.3 452330 -128 123 -2.78 -278 -949 100 10939 99850 2.84984 299.84984 151.34984 15134.98498 2.84985 299.84985 151.34985 15134.98527 2.84984 299.84984 151.34984 15134.98400 2020-01-01 2020-01-02 2020-01-01 00:15:49 2020-01-02 03:44:10 2020-01-01 00:15:49.000 2020-01-02 03:44:10.000 949 99850 50399.5 5039950 949 99850 50399.5 5039950 -32619 32316 4524.3 452430 -127 124 -1.78 -178 -95 102 10085 99995 0.28528 300.28528 150.28528 15178.81381 0.28528 300.28528 150.28528 15178.81343 0.28528 300.28528 150.28528 15178.81328 2020-01-01 2020-01-02 2020-01-01 00:01:35 2020-01-02 03:46:35 2020-01-01 00:01:35.000 2020-01-02 03:46:35.000 95 99995 50045 5054545 95 99995 50045 5054545 -32474 32461 4624.009900990099 467025 -125 126 0.801980198019802 81 -950 100 10940 99851 2.85285 299.85285 151.35285 15135.28528 2.85285 299.85284 151.35285 15135.28541 2.85285 299.85285 151.35285 15135.28500 2020-01-01 2020-01-02 2020-01-01 00:15:50 2020-01-02 03:44:11 2020-01-01 00:15:50.000 2020-01-02 03:44:11.000 950 99851 50400.5 5040050 950 99851 50400.5 5040050 -32618 32317 4525.3 452530 -126 125 -0.78 -78 -951 100 10941 99852 2.85585 299.85585 151.35585 15135.58558 2.85585 299.85587 151.35585 15135.58551 2.85585 299.85585 151.35585 15135.58500 2020-01-01 2020-01-02 2020-01-01 00:15:51 2020-01-02 03:44:12 2020-01-01 00:15:51.000 2020-01-02 03:44:12.000 951 99852 50401.5 5040150 951 99852 50401.5 5040150 -32617 32318 4526.3 452630 -125 126 0.22 22 +931 100 10921 99832 2.79579 299.79579 151.29579 15129.57957 2.79579 299.7958 151.29579 15129.57963 2.79579 299.79579 151.2957900000001 15129.57900 2020-01-01 2020-01-02 2020-01-01 00:15:31 2020-01-02 03:43:52 2020-01-01 00:15:31.000 2020-01-02 03:43:52.000 931 99832 50381.5 5038150 931 99832 50381.5 5038150 -32637 32298 4506.3 450630 -126 125 0.7 70 +932 100 10922 99833 2.79879 299.79879 151.29879 15129.87987 2.79879 299.7988 151.29879 15129.87977 2.79879 299.79879 151.29879000000017 15129.87900 2020-01-01 2020-01-02 2020-01-01 00:15:32 2020-01-02 03:43:53 2020-01-01 00:15:32.000 2020-01-02 03:43:53.000 932 99833 50382.5 5038250 932 99833 50382.5 5038250 -32636 32299 4507.3 450730 -125 126 1.7 170 +933 100 10923 99834 2.8018 299.8018 151.3018 15130.18018 2.8018 299.8018 151.3018 15130.18011 2.80180 299.80180 151.30179999999984 15130.18000 2020-01-01 2020-01-02 2020-01-01 00:15:33 2020-01-02 03:43:54 2020-01-01 00:15:33.000 2020-01-02 03:43:54.000 933 99834 50383.5 5038350 933 99834 50383.5 5038350 -32635 32300 4508.3 450830 -124 127 2.7 270 +934 100 10924 99835 2.8048 299.8048 151.3048 15130.48048 2.8048 299.8048 151.3048 15130.48071 2.80480 299.80480 151.30479999999991 15130.48000 2020-01-01 2020-01-02 2020-01-01 00:15:34 2020-01-02 03:43:55 2020-01-01 00:15:34.000 2020-01-02 03:43:55.000 934 99835 50384.5 5038450 934 99835 50384.5 5038450 -32634 32301 4509.3 450930 -128 127 1.14 114 +935 100 10925 99836 2.8078 299.8078 151.3078 15130.78078 2.8078 299.8078 151.3078 15130.78034 2.80780 299.80780 151.30780000000001 15130.78000 2020-01-01 2020-01-02 2020-01-01 00:15:35 2020-01-02 03:43:56 2020-01-01 00:15:35.000 2020-01-02 03:43:56.000 935 99836 50385.5 5038550 935 99836 50385.5 5038550 -32633 32302 4510.3 451030 -128 123 -0.42 -42 +936 100 10926 99837 2.81081 299.81081 151.31081 15131.08108 2.81081 299.81082 151.31081 15131.0811 2.81081 299.81081 151.31081000000026 15131.08100 2020-01-01 2020-01-02 2020-01-01 00:15:36 2020-01-02 03:43:57 2020-01-01 00:15:36.000 2020-01-02 03:43:57.000 936 99837 50386.5 5038650 936 99837 50386.5 5038650 -32632 32303 4511.3 451130 -127 124 0.58 58 +937 100 10927 99838 2.81381 299.81381 151.31381 15131.38138 2.81381 299.8138 151.31381 15131.38124 2.81381 299.81381 151.31380999999976 15131.38100 2020-01-01 2020-01-02 2020-01-01 00:15:37 2020-01-02 03:43:58 2020-01-01 00:15:37.000 2020-01-02 03:43:58.000 937 99838 50387.5 5038750 937 99838 50387.5 5038750 -32631 32304 4512.3 451230 -126 125 1.58 158 +938 100 10928 99839 2.81681 299.81681 151.31681 15131.68168 2.81681 299.8168 151.31681 15131.68157 2.81681 299.81681 151.31681000000015 15131.68100 2020-01-01 2020-01-02 2020-01-01 00:15:38 2020-01-02 03:43:59 2020-01-01 00:15:38.000 2020-01-02 03:43:59.000 938 99839 50388.5 5038850 938 99839 50388.5 5038850 -32630 32305 4513.3 451330 -125 126 2.58 258 +939 100 10929 99840 2.81981 299.81981 151.31981 15131.98198 2.81982 299.81982 151.31982 15131.98217 2.81981 299.81981 151.31981000000027 15131.98100 2020-01-01 2020-01-02 2020-01-01 00:15:39 2020-01-02 03:44:00 2020-01-01 00:15:39.000 2020-01-02 03:44:00.000 939 99840 50389.5 5038950 939 99840 50389.5 5038950 -32629 32306 4514.3 451430 -124 127 3.58 358 +94 102 10084 99994 0.28228 300.28228 150.28228 15178.51051 0.28228 300.2823 150.28228 15178.51078 0.28228 300.28228 150.28227999999996 15178.51028 2020-01-01 2020-01-02 2020-01-01 00:01:34 2020-01-02 03:46:34 2020-01-01 00:01:34.000 2020-01-02 03:46:34.000 94 99994 50044 5054444 94 99994 50044 5054444 -32475 32460 4623.009900990099 466924 -126 125 -0.19801980198019803 -20 +940 100 10930 99841 2.82282 299.82282 151.32282 15132.28228 2.82282 299.8228 151.32282 15132.28247 2.82282 299.82282 151.3228199999999 15132.28200 2020-01-01 2020-01-02 2020-01-01 00:15:40 2020-01-02 03:44:01 2020-01-01 00:15:40.000 2020-01-02 03:44:01.000 940 99841 50390.5 5039050 940 99841 50390.5 5039050 -32628 32307 4515.3 451530 -128 127 2.02 202 +941 100 10931 99842 2.82582 299.82582 151.32582 15132.58258 2.82582 299.82584 151.32582 15132.58257 2.82582 299.82582 151.32582000000002 15132.58200 2020-01-01 2020-01-02 2020-01-01 00:15:41 2020-01-02 03:44:02 2020-01-01 00:15:41.000 2020-01-02 03:44:02.000 941 99842 50391.5 5039150 941 99842 50391.5 5039150 -32627 32308 4516.3 451630 -128 127 0.46 46 +942 100 10932 99843 2.82882 299.82882 151.32882 15132.88288 2.82882 299.82883 151.32882 15132.88275 2.82882 299.82882 151.32882000000012 15132.88200 2020-01-01 2020-01-02 2020-01-01 00:15:42 2020-01-02 03:44:03 2020-01-01 00:15:42.000 2020-01-02 03:44:03.000 942 99843 50392.5 5039250 942 99843 50392.5 5039250 -32626 32309 4517.3 451730 -128 124 -1.1 -110 +943 100 10933 99844 2.83183 299.83183 151.33183 15133.18318 2.83183 299.83182 151.33183 15133.18304 2.83183 299.83183 151.33182999999977 15133.18300 2020-01-01 2020-01-02 2020-01-01 00:15:43 2020-01-02 03:44:04 2020-01-01 00:15:43.000 2020-01-02 03:44:04.000 943 99844 50393.5 5039350 943 99844 50393.5 5039350 -32625 32310 4518.3 451830 -127 125 -0.1 -10 +944 100 10934 99845 2.83483 299.83483 151.33483 15133.48348 2.83483 299.83484 151.33483 15133.48364 2.83483 299.83483 151.3348299999998 15133.48300 2020-01-01 2020-01-02 2020-01-01 00:15:44 2020-01-02 03:44:05 2020-01-01 00:15:44.000 2020-01-02 03:44:05.000 944 99845 50394.5 5039450 944 99845 50394.5 5039450 -32624 32311 4519.3 451930 -126 126 0.9 90 +945 100 10935 99846 2.83783 299.83783 151.33783 15133.78378 2.83783 299.83783 151.33783 15133.78393 2.83783 299.83783 151.33783000000028 15133.78300 2020-01-01 2020-01-02 2020-01-01 00:15:45 2020-01-02 03:44:06 2020-01-01 00:15:45.000 2020-01-02 03:44:06.000 945 99846 50395.5 5039550 945 99846 50395.5 5039550 -32623 32312 4520.3 452030 -125 127 1.9 190 +946 100 10936 99847 2.84084 299.84084 151.34084 15134.08408 2.84084 299.84085 151.34084 15134.08404 2.84084 299.84084 151.34084000000018 15134.08400 2020-01-01 2020-01-02 2020-01-01 00:15:46 2020-01-02 03:44:07 2020-01-01 00:15:46.000 2020-01-02 03:44:07.000 946 99847 50396.5 5039650 946 99847 50396.5 5039650 -32622 32313 4521.3 452130 -128 127 0.34 34 +947 100 10937 99848 2.84384 299.84384 151.34384 15134.38438 2.84384 299.84384 151.34384 15134.38421 2.84384 299.84384 151.34383999999966 15134.38400 2020-01-01 2020-01-02 2020-01-01 00:15:47 2020-01-02 03:44:08 2020-01-01 00:15:47.000 2020-01-02 03:44:08.000 947 99848 50397.5 5039750 947 99848 50397.5 5039750 -32621 32314 4522.3 452230 -128 127 -1.22 -122 +948 100 10938 99849 2.84684 299.84684 151.34684 15134.68468 2.84684 299.84683 151.34684 15134.68452 2.84684 299.84684 151.34684000000013 15134.68400 2020-01-01 2020-01-02 2020-01-01 00:15:48 2020-01-02 03:44:09 2020-01-01 00:15:48.000 2020-01-02 03:44:09.000 948 99849 50398.5 5039850 948 99849 50398.5 5039850 -32620 32315 4523.3 452330 -128 123 -2.78 -278 +949 100 10939 99850 2.84984 299.84984 151.34984 15134.98498 2.84985 299.84985 151.34985 15134.98527 2.84984 299.84984 151.34984000000017 15134.98400 2020-01-01 2020-01-02 2020-01-01 00:15:49 2020-01-02 03:44:10 2020-01-01 00:15:49.000 2020-01-02 03:44:10.000 949 99850 50399.5 5039950 949 99850 50399.5 5039950 -32619 32316 4524.3 452430 -127 124 -1.78 -178 +95 102 10085 99995 0.28528 300.28528 150.28528 15178.81381 0.28528 300.28528 150.28528 15178.81343 0.28528 300.28528 150.28528000000003 15178.81328 2020-01-01 2020-01-02 2020-01-01 00:01:35 2020-01-02 03:46:35 2020-01-01 00:01:35.000 2020-01-02 03:46:35.000 95 99995 50045 5054545 95 99995 50045 5054545 -32474 32461 4624.009900990099 467025 -125 126 0.801980198019802 81 +950 100 10940 99851 2.85285 299.85285 151.35285 15135.28528 2.85285 299.85284 151.35285 15135.28541 2.85285 299.85285 151.35284999999993 15135.28500 2020-01-01 2020-01-02 2020-01-01 00:15:50 2020-01-02 03:44:11 2020-01-01 00:15:50.000 2020-01-02 03:44:11.000 950 99851 50400.5 5040050 950 99851 50400.5 5040050 -32618 32317 4525.3 452530 -126 125 -0.78 -78 +951 100 10941 99852 2.85585 299.85585 151.35585 15135.58558 2.85585 299.85587 151.35585 15135.58551 2.85585 299.85585 151.35584999999995 15135.58500 2020-01-01 2020-01-02 2020-01-01 00:15:51 2020-01-02 03:44:12 2020-01-01 00:15:51.000 2020-01-02 03:44:12.000 951 99852 50401.5 5040150 951 99852 50401.5 5040150 -32617 32318 4526.3 452630 -125 126 0.22 22 952 100 10942 99853 2.85885 299.85885 151.35885 15135.88588 2.85885 299.85886 151.35885 15135.88568 2.85885 299.85885 151.35885 15135.88500 2020-01-01 2020-01-02 2020-01-01 00:15:52 2020-01-02 03:44:13 2020-01-01 00:15:52.000 2020-01-02 03:44:13.000 952 99853 50402.5 5040250 952 99853 50402.5 5040250 -32616 32319 4527.3 452730 -124 127 1.22 122 -953 100 10943 99854 2.86186 299.86186 151.36186 15136.18618 2.86186 299.86185 151.36185 15136.18598 2.86186 299.86186 151.36186 15136.18600 2020-01-01 2020-01-02 2020-01-01 00:15:53 2020-01-02 03:44:14 2020-01-01 00:15:53.000 2020-01-02 03:44:14.000 953 99854 50403.5 5040350 953 99854 50403.5 5040350 -32615 32320 4528.3 452830 -128 127 -0.34 -34 -954 100 10944 99855 2.86486 299.86486 151.36486 15136.48648 2.86486 299.86487 151.36486 15136.48674 2.86486 299.86486 151.36486 15136.48600 2020-01-01 2020-01-02 2020-01-01 00:15:54 2020-01-02 03:44:15 2020-01-01 00:15:54.000 2020-01-02 03:44:15.000 954 99855 50404.5 5040450 954 99855 50404.5 5040450 -32614 32321 4529.3 452930 -128 123 -1.9 -190 -955 100 10945 99856 2.86786 299.86786 151.36786 15136.78678 2.86786 299.86786 151.36786 15136.78688 2.86786 299.86786 151.36786 15136.78600 2020-01-01 2020-01-02 2020-01-01 00:15:55 2020-01-02 03:44:16 2020-01-01 00:15:55.000 2020-01-02 03:44:16.000 955 99856 50405.5 5040550 955 99856 50405.5 5040550 -32613 32322 4530.3 453030 -127 124 -0.9 -90 -956 100 10946 99857 2.87087 299.87087 151.37087 15137.08708 2.87087 299.87088 151.37087 15137.08701 2.87087 299.87087 151.37087 15137.08700 2020-01-01 2020-01-02 2020-01-01 00:15:56 2020-01-02 03:44:17 2020-01-01 00:15:56.000 2020-01-02 03:44:17.000 956 99857 50406.5 5040650 956 99857 50406.5 5040650 -32612 32323 4531.3 453130 -126 125 0.1 10 -957 100 10947 99858 2.87387 299.87387 151.37387 15137.38738 2.87387 299.87387 151.37387 15137.38716 2.87387 299.87387 151.37387 15137.38700 2020-01-01 2020-01-02 2020-01-01 00:15:57 2020-01-02 03:44:18 2020-01-01 00:15:57.000 2020-01-02 03:44:18.000 957 99858 50407.5 5040750 957 99858 50407.5 5040750 -32611 32324 4532.3 453230 -125 126 1.1 110 -958 100 10948 99859 2.87687 299.87687 151.37687 15137.68768 2.87687 299.8769 151.37687 15137.68791 2.87687 299.87687 151.37687 15137.68700 2020-01-01 2020-01-02 2020-01-01 00:15:58 2020-01-02 03:44:19 2020-01-01 00:15:58.000 2020-01-02 03:44:19.000 958 99859 50408.5 5040850 958 99859 50408.5 5040850 -32610 32325 4533.3 453330 -124 127 2.1 210 -959 100 10949 99860 2.87987 299.87987 151.37987 15137.98798 2.87988 299.87988 151.37988 15137.9882 2.87987 299.87987 151.37987 15137.98700 2020-01-01 2020-01-02 2020-01-01 00:15:59 2020-01-02 03:44:20 2020-01-01 00:15:59.000 2020-01-02 03:44:20.000 959 99860 50409.5 5040950 959 99860 50409.5 5040950 -32609 32326 4534.3 453430 -128 127 0.54 54 -96 102 10086 99996 0.28828 300.28828 150.28828 15179.11711 0.28828 300.2883 150.28828 15179.11718 0.28828 300.28828 150.28828 15179.11628 2020-01-01 2020-01-02 2020-01-01 00:01:36 2020-01-02 03:46:36 2020-01-01 00:01:36.000 2020-01-02 03:46:36.000 96 99996 50046 5054646 96 99996 50046 5054646 -32473 32462 4625.009900990099 467126 -124 127 1.801980198019802 182 -960 100 10950 99861 2.88288 299.88288 151.38288 15138.28828 2.88288 299.88287 151.38288 15138.28834 2.88288 299.88288 151.38288 15138.28800 2020-01-01 2020-01-02 2020-01-01 00:16:00 2020-01-02 03:44:21 2020-01-01 00:16:00.000 2020-01-02 03:44:21.000 960 99861 50410.5 5041050 960 99861 50410.5 5041050 -32608 32327 4535.3 453530 -128 123 -1.02 -102 -961 100 10951 99862 2.88588 299.88588 151.38588 15138.58858 2.88588 299.8859 151.38588 15138.58848 2.88588 299.88588 151.38588 15138.58800 2020-01-01 2020-01-02 2020-01-01 00:16:01 2020-01-02 03:44:22 2020-01-01 00:16:01.000 2020-01-02 03:44:22.000 961 99862 50411.5 5041150 961 99862 50411.5 5041150 -32607 32328 4536.3 453630 -127 124 -0.02 -2 -962 100 10952 99863 2.88888 299.88888 151.38888 15138.88888 2.88888 299.8889 151.38888 15138.88862 2.88888 299.88888 151.38888 15138.88800 2020-01-01 2020-01-02 2020-01-01 00:16:02 2020-01-02 03:44:23 2020-01-01 00:16:02.000 2020-01-02 03:44:23.000 962 99863 50412.5 5041250 962 99863 50412.5 5041250 -32606 32329 4537.3 453730 -126 125 0.98 98 -963 100 10953 99864 2.89189 299.89189 151.39189 15139.18918 2.89189 299.8919 151.39189 15139.18937 2.89189 299.89189 151.39189 15139.18900 2020-01-01 2020-01-02 2020-01-01 00:16:03 2020-01-02 03:44:24 2020-01-01 00:16:03.000 2020-01-02 03:44:24.000 963 99864 50413.5 5041350 963 99864 50413.5 5041350 -32605 32330 4538.3 453830 -125 126 1.98 198 -964 100 10954 99865 2.89489 299.89489 151.39489 15139.48948 2.89489 299.8949 151.39489 15139.48968 2.89489 299.89489 151.39489 15139.48900 2020-01-01 2020-01-02 2020-01-01 00:16:04 2020-01-02 03:44:25 2020-01-01 00:16:04.000 2020-01-02 03:44:25.000 964 99865 50414.5 5041450 964 99865 50414.5 5041450 -32604 32331 4539.3 453930 -124 127 2.98 298 -965 100 10955 99866 2.89789 299.89789 151.39789 15139.78978 2.89789 299.8979 151.39789 15139.78985 2.89789 299.89789 151.39789 15139.78900 2020-01-01 2020-01-02 2020-01-01 00:16:05 2020-01-02 03:44:26 2020-01-01 00:16:05.000 2020-01-02 03:44:26.000 965 99866 50415.5 5041550 965 99866 50415.5 5041550 -32603 32332 4540.3 454030 -128 127 1.42 142 -966 100 10956 99867 2.9009 299.9009 151.4009 15140.09009 2.9009 299.9009 151.40089 15140.08996 2.90090 299.90090 151.40090 15140.09000 2020-01-01 2020-01-02 2020-01-01 00:16:06 2020-01-02 03:44:27 2020-01-01 00:16:06.000 2020-01-02 03:44:27.000 966 99867 50416.5 5041650 966 99867 50416.5 5041650 -32602 32333 4541.3 454130 -128 127 -0.14 -14 -967 100 10957 99868 2.9039 299.9039 151.4039 15140.39039 2.9039 299.9039 151.4039 15140.39009 2.90390 299.90390 151.40390 15140.39000 2020-01-01 2020-01-02 2020-01-01 00:16:07 2020-01-02 03:44:28 2020-01-01 00:16:07.000 2020-01-02 03:44:28.000 967 99868 50417.5 5041750 967 99868 50417.5 5041750 -32601 32334 4542.3 454230 -128 124 -1.7 -170 -968 100 10958 99869 2.9069 299.9069 151.4069 15140.69069 2.9069 299.90692 151.4069 15140.69084 2.90690 299.90690 151.40690 15140.69000 2020-01-01 2020-01-02 2020-01-01 00:16:08 2020-01-02 03:44:29 2020-01-01 00:16:08.000 2020-01-02 03:44:29.000 968 99869 50418.5 5041850 968 99869 50418.5 5041850 -32600 32335 4543.3 454330 -127 125 -0.7 -70 -969 100 10959 99870 2.9099 299.9099 151.4099 15140.99099 2.90991 299.9099 151.40991 15140.99114 2.90990 299.90990 151.40990 15140.99000 2020-01-01 2020-01-02 2020-01-01 00:16:09 2020-01-02 03:44:30 2020-01-01 00:16:09.000 2020-01-02 03:44:30.000 969 99870 50419.5 5041950 969 99870 50419.5 5041950 -32599 32336 4544.3 454430 -126 126 0.3 30 -97 102 10087 99997 0.29129 300.29129 150.29129 15179.42042 0.29129 300.2913 150.29129 15179.42033 0.29129 300.29129 150.29129 15179.42029 2020-01-01 2020-01-02 2020-01-01 00:01:37 2020-01-02 03:46:37 2020-01-01 00:01:37.000 2020-01-02 03:46:37.000 97 99997 50047 5054747 97 99997 50047 5054747 -32472 32463 4626.009900990099 467227 -128 127 0.26732673267326734 27 -970 100 10960 99871 2.91291 299.91291 151.41291 15141.29129 2.91291 299.9129 151.41291 15141.29132 2.91291 299.91291 151.41291 15141.29100 2020-01-01 2020-01-02 2020-01-01 00:16:10 2020-01-02 03:44:31 2020-01-01 00:16:10.000 2020-01-02 03:44:31.000 970 99871 50420.5 5042050 970 99871 50420.5 5042050 -32598 32337 4545.3 454530 -125 127 1.3 130 -971 100 10961 99872 2.91591 299.91591 151.41591 15141.59159 2.91591 299.91592 151.41591 15141.59142 2.91591 299.91591 151.41591 15141.59100 2020-01-01 2020-01-02 2020-01-01 00:16:11 2020-01-02 03:44:32 2020-01-01 00:16:11.000 2020-01-02 03:44:32.000 971 99872 50421.5 5042150 971 99872 50421.5 5042150 -32597 32338 4546.3 454630 -128 127 -0.26 -26 -972 100 10962 99873 2.91891 299.91891 151.41891 15141.89189 2.91891 299.9189 151.41891 15141.89172 2.91891 299.91891 151.41891 15141.89100 2020-01-01 2020-01-02 2020-01-01 00:16:12 2020-01-02 03:44:33 2020-01-01 00:16:12.000 2020-01-02 03:44:33.000 972 99873 50422.5 5042250 972 99873 50422.5 5042250 -32596 32339 4547.3 454730 -128 127 -1.82 -182 +953 100 10943 99854 2.86186 299.86186 151.36186 15136.18618 2.86186 299.86185 151.36185 15136.18598 2.86186 299.86186 151.3618600000001 15136.18600 2020-01-01 2020-01-02 2020-01-01 00:15:53 2020-01-02 03:44:14 2020-01-01 00:15:53.000 2020-01-02 03:44:14.000 953 99854 50403.5 5040350 953 99854 50403.5 5040350 -32615 32320 4528.3 452830 -128 127 -0.34 -34 +954 100 10944 99855 2.86486 299.86486 151.36486 15136.48648 2.86486 299.86487 151.36486 15136.48674 2.86486 299.86486 151.3648599999999 15136.48600 2020-01-01 2020-01-02 2020-01-01 00:15:54 2020-01-02 03:44:15 2020-01-01 00:15:54.000 2020-01-02 03:44:15.000 954 99855 50404.5 5040450 954 99855 50404.5 5040450 -32614 32321 4529.3 452930 -128 123 -1.9 -190 +955 100 10945 99856 2.86786 299.86786 151.36786 15136.78678 2.86786 299.86786 151.36786 15136.78688 2.86786 299.86786 151.36786000000006 15136.78600 2020-01-01 2020-01-02 2020-01-01 00:15:55 2020-01-02 03:44:16 2020-01-01 00:15:55.000 2020-01-02 03:44:16.000 955 99856 50405.5 5040550 955 99856 50405.5 5040550 -32613 32322 4530.3 453030 -127 124 -0.9 -90 +956 100 10946 99857 2.87087 299.87087 151.37087 15137.08708 2.87087 299.87088 151.37087 15137.08701 2.87087 299.87087 151.37087000000008 15137.08700 2020-01-01 2020-01-02 2020-01-01 00:15:56 2020-01-02 03:44:17 2020-01-01 00:15:56.000 2020-01-02 03:44:17.000 956 99857 50406.5 5040650 956 99857 50406.5 5040650 -32612 32323 4531.3 453130 -126 125 0.1 10 +957 100 10947 99858 2.87387 299.87387 151.37387 15137.38738 2.87387 299.87387 151.37387 15137.38716 2.87387 299.87387 151.37386999999998 15137.38700 2020-01-01 2020-01-02 2020-01-01 00:15:57 2020-01-02 03:44:18 2020-01-01 00:15:57.000 2020-01-02 03:44:18.000 957 99858 50407.5 5040750 957 99858 50407.5 5040750 -32611 32324 4532.3 453230 -125 126 1.1 110 +958 100 10948 99859 2.87687 299.87687 151.37687 15137.68768 2.87687 299.8769 151.37687 15137.68791 2.87687 299.87687 151.37687000000003 15137.68700 2020-01-01 2020-01-02 2020-01-01 00:15:58 2020-01-02 03:44:19 2020-01-01 00:15:58.000 2020-01-02 03:44:19.000 958 99859 50408.5 5040850 958 99859 50408.5 5040850 -32610 32325 4533.3 453330 -124 127 2.1 210 +959 100 10949 99860 2.87987 299.87987 151.37987 15137.98798 2.87988 299.87988 151.37988 15137.9882 2.87987 299.87987 151.37987000000004 15137.98700 2020-01-01 2020-01-02 2020-01-01 00:15:59 2020-01-02 03:44:20 2020-01-01 00:15:59.000 2020-01-02 03:44:20.000 959 99860 50409.5 5040950 959 99860 50409.5 5040950 -32609 32326 4534.3 453430 -128 127 0.54 54 +96 102 10086 99996 0.28828 300.28828 150.28828 15179.11711 0.28828 300.2883 150.28828 15179.11718 0.28828 300.28828 150.28828000000001 15179.11628 2020-01-01 2020-01-02 2020-01-01 00:01:36 2020-01-02 03:46:36 2020-01-01 00:01:36.000 2020-01-02 03:46:36.000 96 99996 50046 5054646 96 99996 50046 5054646 -32473 32462 4625.009900990099 467126 -124 127 1.801980198019802 182 +960 100 10950 99861 2.88288 299.88288 151.38288 15138.28828 2.88288 299.88287 151.38288 15138.28834 2.88288 299.88288 151.38287999999997 15138.28800 2020-01-01 2020-01-02 2020-01-01 00:16:00 2020-01-02 03:44:21 2020-01-01 00:16:00.000 2020-01-02 03:44:21.000 960 99861 50410.5 5041050 960 99861 50410.5 5041050 -32608 32327 4535.3 453530 -128 123 -1.02 -102 +961 100 10951 99862 2.88588 299.88588 151.38588 15138.58858 2.88588 299.8859 151.38588 15138.58848 2.88588 299.88588 151.3858799999999 15138.58800 2020-01-01 2020-01-02 2020-01-01 00:16:01 2020-01-02 03:44:22 2020-01-01 00:16:01.000 2020-01-02 03:44:22.000 961 99862 50411.5 5041150 961 99862 50411.5 5041150 -32607 32328 4536.3 453630 -127 124 -0.02 -2 +962 100 10952 99863 2.88888 299.88888 151.38888 15138.88888 2.88888 299.8889 151.38888 15138.88862 2.88888 299.88888 151.38888000000014 15138.88800 2020-01-01 2020-01-02 2020-01-01 00:16:02 2020-01-02 03:44:23 2020-01-01 00:16:02.000 2020-01-02 03:44:23.000 962 99863 50412.5 5041250 962 99863 50412.5 5041250 -32606 32329 4537.3 453730 -126 125 0.98 98 +963 100 10953 99864 2.89189 299.89189 151.39189 15139.18918 2.89189 299.8919 151.39189 15139.18937 2.89189 299.89189 151.39189000000005 15139.18900 2020-01-01 2020-01-02 2020-01-01 00:16:03 2020-01-02 03:44:24 2020-01-01 00:16:03.000 2020-01-02 03:44:24.000 963 99864 50413.5 5041350 963 99864 50413.5 5041350 -32605 32330 4538.3 453830 -125 126 1.98 198 +964 100 10954 99865 2.89489 299.89489 151.39489 15139.48948 2.89489 299.8949 151.39489 15139.48968 2.89489 299.89489 151.39488999999986 15139.48900 2020-01-01 2020-01-02 2020-01-01 00:16:04 2020-01-02 03:44:25 2020-01-01 00:16:04.000 2020-01-02 03:44:25.000 964 99865 50414.5 5041450 964 99865 50414.5 5041450 -32604 32331 4539.3 453930 -124 127 2.98 298 +965 100 10955 99866 2.89789 299.89789 151.39789 15139.78978 2.89789 299.8979 151.39789 15139.78985 2.89789 299.89789 151.3978900000001 15139.78900 2020-01-01 2020-01-02 2020-01-01 00:16:05 2020-01-02 03:44:26 2020-01-01 00:16:05.000 2020-01-02 03:44:26.000 965 99866 50415.5 5041550 965 99866 50415.5 5041550 -32603 32332 4540.3 454030 -128 127 1.42 142 +966 100 10956 99867 2.9009 299.9009 151.4009 15140.09009 2.9009 299.9009 151.40089 15140.08996 2.90090 299.90090 151.4009 15140.09000 2020-01-01 2020-01-02 2020-01-01 00:16:06 2020-01-02 03:44:27 2020-01-01 00:16:06.000 2020-01-02 03:44:27.000 966 99867 50416.5 5041650 966 99867 50416.5 5041650 -32602 32333 4541.3 454130 -128 127 -0.14 -14 +967 100 10957 99868 2.9039 299.9039 151.4039 15140.39039 2.9039 299.9039 151.4039 15140.39009 2.90390 299.90390 151.40389999999996 15140.39000 2020-01-01 2020-01-02 2020-01-01 00:16:07 2020-01-02 03:44:28 2020-01-01 00:16:07.000 2020-01-02 03:44:28.000 967 99868 50417.5 5041750 967 99868 50417.5 5041750 -32601 32334 4542.3 454230 -128 124 -1.7 -170 +968 100 10958 99869 2.9069 299.9069 151.4069 15140.69069 2.9069 299.90692 151.4069 15140.69084 2.90690 299.90690 151.40689999999998 15140.69000 2020-01-01 2020-01-02 2020-01-01 00:16:08 2020-01-02 03:44:29 2020-01-01 00:16:08.000 2020-01-02 03:44:29.000 968 99869 50418.5 5041850 968 99869 50418.5 5041850 -32600 32335 4543.3 454330 -127 125 -0.7 -70 +969 100 10959 99870 2.9099 299.9099 151.4099 15140.99099 2.90991 299.9099 151.40991 15140.99114 2.90990 299.90990 151.40990000000002 15140.99000 2020-01-01 2020-01-02 2020-01-01 00:16:09 2020-01-02 03:44:30 2020-01-01 00:16:09.000 2020-01-02 03:44:30.000 969 99870 50419.5 5041950 969 99870 50419.5 5041950 -32599 32336 4544.3 454430 -126 126 0.3 30 +97 102 10087 99997 0.29129 300.29129 150.29129 15179.42042 0.29129 300.2913 150.29129 15179.42033 0.29129 300.29129 150.29128999999998 15179.42029 2020-01-01 2020-01-02 2020-01-01 00:01:37 2020-01-02 03:46:37 2020-01-01 00:01:37.000 2020-01-02 03:46:37.000 97 99997 50047 5054747 97 99997 50047 5054747 -32472 32463 4626.009900990099 467227 -128 127 0.26732673267326734 27 +970 100 10960 99871 2.91291 299.91291 151.41291 15141.29129 2.91291 299.9129 151.41291 15141.29132 2.91291 299.91291 151.41290999999993 15141.29100 2020-01-01 2020-01-02 2020-01-01 00:16:10 2020-01-02 03:44:31 2020-01-01 00:16:10.000 2020-01-02 03:44:31.000 970 99871 50420.5 5042050 970 99871 50420.5 5042050 -32598 32337 4545.3 454530 -125 127 1.3 130 +971 100 10961 99872 2.91591 299.91591 151.41591 15141.59159 2.91591 299.91592 151.41591 15141.59142 2.91591 299.91591 151.41590999999994 15141.59100 2020-01-01 2020-01-02 2020-01-01 00:16:11 2020-01-02 03:44:32 2020-01-01 00:16:11.000 2020-01-02 03:44:32.000 971 99872 50421.5 5042150 971 99872 50421.5 5042150 -32597 32338 4546.3 454630 -128 127 -0.26 -26 +972 100 10962 99873 2.91891 299.91891 151.41891 15141.89189 2.91891 299.9189 151.41891 15141.89172 2.91891 299.91891 151.4189100000001 15141.89100 2020-01-01 2020-01-02 2020-01-01 00:16:12 2020-01-02 03:44:33 2020-01-01 00:16:12.000 2020-01-02 03:44:33.000 972 99873 50422.5 5042250 972 99873 50422.5 5042250 -32596 32339 4547.3 454730 -128 127 -1.82 -182 973 100 10963 99874 2.92192 299.92192 151.42192 15142.19219 2.92192 299.92194 151.42192 15142.19232 2.92192 299.92192 151.42192 15142.19200 2020-01-01 2020-01-02 2020-01-01 00:16:13 2020-01-02 03:44:34 2020-01-01 00:16:13.000 2020-01-02 03:44:34.000 973 99874 50423.5 5042350 973 99874 50423.5 5042350 -32595 32340 4548.3 454830 -128 123 -3.38 -338 -974 100 10964 99875 2.92492 299.92492 151.42492 15142.49249 2.92492 299.92493 151.42492 15142.49265 2.92492 299.92492 151.42492 15142.49200 2020-01-01 2020-01-02 2020-01-01 00:16:14 2020-01-02 03:44:35 2020-01-01 00:16:14.000 2020-01-02 03:44:35.000 974 99875 50424.5 5042450 974 99875 50424.5 5042450 -32594 32341 4549.3 454930 -127 124 -2.38 -238 -975 100 10965 99876 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792 15142.79200 2020-01-01 2020-01-02 2020-01-01 00:16:15 2020-01-02 03:44:36 2020-01-01 00:16:15.000 2020-01-02 03:44:36.000 975 99876 50425.5 5042550 975 99876 50425.5 5042550 -32593 32342 4550.3 455030 -126 125 -1.38 -138 -976 100 10966 99877 2.93093 299.93093 151.43093 15143.09309 2.93093 299.93094 151.43092 15143.09289 2.93093 299.93093 151.43093 15143.09300 2020-01-01 2020-01-02 2020-01-01 00:16:16 2020-01-02 03:44:37 2020-01-01 00:16:16.000 2020-01-02 03:44:37.000 976 99877 50426.5 5042650 976 99877 50426.5 5042650 -32592 32343 4551.3 455130 -125 126 -0.38 -38 -977 100 10967 99878 2.93393 299.93393 151.43393 15143.39339 2.93393 299.93393 151.43393 15143.39318 2.93393 299.93393 151.43393 15143.39300 2020-01-01 2020-01-02 2020-01-01 00:16:17 2020-01-02 03:44:38 2020-01-01 00:16:17.000 2020-01-02 03:44:38.000 977 99878 50427.5 5042750 977 99878 50427.5 5042750 -32591 32344 4552.3 455230 -124 127 0.62 62 -978 100 10968 99879 2.93693 299.93693 151.43693 15143.69369 2.93693 299.93695 151.43693 15143.69378 2.93693 299.93693 151.43693 15143.69300 2020-01-01 2020-01-02 2020-01-01 00:16:18 2020-01-02 03:44:39 2020-01-01 00:16:18.000 2020-01-02 03:44:39.000 978 99879 50428.5 5042850 978 99879 50428.5 5042850 -32590 32345 4553.3 455330 -128 127 -0.94 -94 -979 100 10969 99880 2.93993 299.93993 151.43993 15143.99399 2.93994 299.93994 151.43994 15143.99412 2.93993 299.93993 151.43993 15143.99300 2020-01-01 2020-01-02 2020-01-01 00:16:19 2020-01-02 03:44:40 2020-01-01 00:16:19.000 2020-01-02 03:44:40.000 979 99880 50429.5 5042950 979 99880 50429.5 5042950 -32589 32346 4554.3 455430 -128 123 -2.5 -250 -98 102 10088 99998 0.29429 300.29429 150.29429 15179.72372 0.29429 300.29428 150.29429 15179.72363 0.29429 300.29429 150.29429 15179.72329 2020-01-01 2020-01-02 2020-01-01 00:01:38 2020-01-02 03:46:38 2020-01-01 00:01:38.000 2020-01-02 03:46:38.000 98 99998 50048 5054848 98 99998 50048 5054848 -32471 32464 4627.009900990099 467328 -128 127 -1.2673267326732673 -128 -980 100 10970 99881 2.94294 299.94294 151.44294 15144.29429 2.94294 299.94293 151.44294 15144.29426 2.94294 299.94294 151.44294 15144.29400 2020-01-01 2020-01-02 2020-01-01 00:16:20 2020-01-02 03:44:41 2020-01-01 00:16:20.000 2020-01-02 03:44:41.000 980 99881 50430.5 5043050 980 99881 50430.5 5043050 -32588 32347 4555.3 455530 -127 124 -1.5 -150 -981 100 10971 99882 2.94594 299.94594 151.44594 15144.59459 2.94594 299.94595 151.44595 15144.59501 2.94594 299.94594 151.44594 15144.59400 2020-01-01 2020-01-02 2020-01-01 00:16:21 2020-01-02 03:44:42 2020-01-01 00:16:21.000 2020-01-02 03:44:42.000 981 99882 50431.5 5043150 981 99882 50431.5 5043150 -32587 32348 4556.3 455630 -126 125 -0.5 -50 -982 100 10972 99883 2.94894 299.94894 151.44894 15144.89489 2.94894 299.94894 151.44894 15144.89466 2.94894 299.94894 151.44894 15144.89400 2020-01-01 2020-01-02 2020-01-01 00:16:22 2020-01-02 03:44:43 2020-01-01 00:16:22.000 2020-01-02 03:44:43.000 982 99883 50432.5 5043250 982 99883 50432.5 5043250 -32586 32349 4557.3 455730 -125 126 0.5 50 +974 100 10964 99875 2.92492 299.92492 151.42492 15142.49249 2.92492 299.92493 151.42492 15142.49265 2.92492 299.92492 151.42492000000001 15142.49200 2020-01-01 2020-01-02 2020-01-01 00:16:14 2020-01-02 03:44:35 2020-01-01 00:16:14.000 2020-01-02 03:44:35.000 974 99875 50424.5 5042450 974 99875 50424.5 5042450 -32594 32341 4549.3 454930 -127 124 -2.38 -238 +975 100 10965 99876 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792000000006 15142.79200 2020-01-01 2020-01-02 2020-01-01 00:16:15 2020-01-02 03:44:36 2020-01-01 00:16:15.000 2020-01-02 03:44:36.000 975 99876 50425.5 5042550 975 99876 50425.5 5042550 -32593 32342 4550.3 455030 -126 125 -1.38 -138 +976 100 10966 99877 2.93093 299.93093 151.43093 15143.09309 2.93093 299.93094 151.43092 15143.09289 2.93093 299.93093 151.43093000000016 15143.09300 2020-01-01 2020-01-02 2020-01-01 00:16:16 2020-01-02 03:44:37 2020-01-01 00:16:16.000 2020-01-02 03:44:37.000 976 99877 50426.5 5042650 976 99877 50426.5 5042650 -32592 32343 4551.3 455130 -125 126 -0.38 -38 +977 100 10967 99878 2.93393 299.93393 151.43393 15143.39339 2.93393 299.93393 151.43393 15143.39318 2.93393 299.93393 151.43392999999995 15143.39300 2020-01-01 2020-01-02 2020-01-01 00:16:17 2020-01-02 03:44:38 2020-01-01 00:16:17.000 2020-01-02 03:44:38.000 977 99878 50427.5 5042750 977 99878 50427.5 5042750 -32591 32344 4552.3 455230 -124 127 0.62 62 +978 100 10968 99879 2.93693 299.93693 151.43693 15143.69369 2.93693 299.93695 151.43693 15143.69378 2.93693 299.93693 151.43692999999993 15143.69300 2020-01-01 2020-01-02 2020-01-01 00:16:18 2020-01-02 03:44:39 2020-01-01 00:16:18.000 2020-01-02 03:44:39.000 978 99879 50428.5 5042850 978 99879 50428.5 5042850 -32590 32345 4553.3 455330 -128 127 -0.94 -94 +979 100 10969 99880 2.93993 299.93993 151.43993 15143.99399 2.93994 299.93994 151.43994 15143.99412 2.93993 299.93993 151.43993000000017 15143.99300 2020-01-01 2020-01-02 2020-01-01 00:16:19 2020-01-02 03:44:40 2020-01-01 00:16:19.000 2020-01-02 03:44:40.000 979 99880 50429.5 5042950 979 99880 50429.5 5042950 -32589 32346 4554.3 455430 -128 123 -2.5 -250 +98 102 10088 99998 0.29429 300.29429 150.29429 15179.72372 0.29429 300.29428 150.29429 15179.72363 0.29429 300.29429 150.29428999999993 15179.72329 2020-01-01 2020-01-02 2020-01-01 00:01:38 2020-01-02 03:46:38 2020-01-01 00:01:38.000 2020-01-02 03:46:38.000 98 99998 50048 5054848 98 99998 50048 5054848 -32471 32464 4627.009900990099 467328 -128 127 -1.2673267326732673 -128 +980 100 10970 99881 2.94294 299.94294 151.44294 15144.29429 2.94294 299.94293 151.44294 15144.29426 2.94294 299.94294 151.44294000000005 15144.29400 2020-01-01 2020-01-02 2020-01-01 00:16:20 2020-01-02 03:44:41 2020-01-01 00:16:20.000 2020-01-02 03:44:41.000 980 99881 50430.5 5043050 980 99881 50430.5 5043050 -32588 32347 4555.3 455530 -127 124 -1.5 -150 +981 100 10971 99882 2.94594 299.94594 151.44594 15144.59459 2.94594 299.94595 151.44595 15144.59501 2.94594 299.94594 151.4459399999999 15144.59400 2020-01-01 2020-01-02 2020-01-01 00:16:21 2020-01-02 03:44:42 2020-01-01 00:16:21.000 2020-01-02 03:44:42.000 981 99882 50431.5 5043150 981 99882 50431.5 5043150 -32587 32348 4556.3 455630 -126 125 -0.5 -50 +982 100 10972 99883 2.94894 299.94894 151.44894 15144.89489 2.94894 299.94894 151.44894 15144.89466 2.94894 299.94894 151.4489400000001 15144.89400 2020-01-01 2020-01-02 2020-01-01 00:16:22 2020-01-02 03:44:43 2020-01-01 00:16:22.000 2020-01-02 03:44:43.000 982 99883 50432.5 5043250 982 99883 50432.5 5043250 -32586 32349 4557.3 455730 -125 126 0.5 50 983 100 10973 99884 2.95195 299.95195 151.45195 15145.19519 2.95195 299.95197 151.45195 15145.19525 2.95195 299.95195 151.45195 15145.19500 2020-01-01 2020-01-02 2020-01-01 00:16:23 2020-01-02 03:44:44 2020-01-01 00:16:23.000 2020-01-02 03:44:44.000 983 99884 50433.5 5043350 983 99884 50433.5 5043350 -32585 32350 4558.3 455830 -124 127 1.5 150 -984 100 10974 99885 2.95495 299.95495 151.45495 15145.49549 2.95495 299.95496 151.45495 15145.49559 2.95495 299.95495 151.45495 15145.49500 2020-01-01 2020-01-02 2020-01-01 00:16:24 2020-01-02 03:44:45 2020-01-01 00:16:24.000 2020-01-02 03:44:45.000 984 99885 50434.5 5043450 984 99885 50434.5 5043450 -32584 32351 4559.3 455930 -128 127 -0.06 -6 +984 100 10974 99885 2.95495 299.95495 151.45495 15145.49549 2.95495 299.95496 151.45495 15145.49559 2.95495 299.95495 151.45494999999994 15145.49500 2020-01-01 2020-01-02 2020-01-01 00:16:24 2020-01-02 03:44:45 2020-01-01 00:16:24.000 2020-01-02 03:44:45.000 984 99885 50434.5 5043450 984 99885 50434.5 5043450 -32584 32351 4559.3 455930 -128 127 -0.06 -6 985 100 10975 99886 2.95795 299.95795 151.45795 15145.79579 2.95795 299.95795 151.45795 15145.79573 2.95795 299.95795 151.45795 15145.79500 2020-01-01 2020-01-02 2020-01-01 00:16:25 2020-01-02 03:44:46 2020-01-01 00:16:25.000 2020-01-02 03:44:46.000 985 99886 50435.5 5043550 985 99886 50435.5 5043550 -32583 32352 4560.3 456030 -128 123 -1.62 -162 -986 100 10976 99887 2.96096 299.96096 151.46096 15146.09609 2.96096 299.96097 151.46096 15146.09648 2.96096 299.96096 151.46096 15146.09600 2020-01-01 2020-01-02 2020-01-01 00:16:26 2020-01-02 03:44:47 2020-01-01 00:16:26.000 2020-01-02 03:44:47.000 986 99887 50436.5 5043650 986 99887 50436.5 5043650 -32582 32353 4561.3 456130 -127 124 -0.62 -62 -987 100 10977 99888 2.96396 299.96396 151.46396 15146.39639 2.96396 299.96396 151.46396 15146.39612 2.96396 299.96396 151.46396 15146.39600 2020-01-01 2020-01-02 2020-01-01 00:16:27 2020-01-02 03:44:48 2020-01-01 00:16:27.000 2020-01-02 03:44:48.000 987 99888 50437.5 5043750 987 99888 50437.5 5043750 -32581 32354 4562.3 456230 -126 125 0.38 38 -988 100 10978 99889 2.96696 299.96696 151.46696 15146.69669 2.96696 299.96698 151.46696 15146.69676 2.96696 299.96696 151.46696 15146.69600 2020-01-01 2020-01-02 2020-01-01 00:16:28 2020-01-02 03:44:49 2020-01-01 00:16:28.000 2020-01-02 03:44:49.000 988 99889 50438.5 5043850 988 99889 50438.5 5043850 -32580 32355 4563.3 456330 -125 126 1.38 138 -989 100 10979 99890 2.96996 299.96996 151.46996 15146.99699 2.96997 299.96997 151.46997 15146.99706 2.96996 299.96996 151.46996 15146.99600 2020-01-01 2020-01-02 2020-01-01 00:16:29 2020-01-02 03:44:50 2020-01-01 00:16:29.000 2020-01-02 03:44:50.000 989 99890 50439.5 5043950 989 99890 50439.5 5043950 -32579 32356 4564.3 456430 -124 127 2.38 238 -99 102 10089 99999 0.29729 300.29729 150.29729 15180.02702 0.29729 300.2973 150.29729 15180.02726 0.29729 300.29729 150.29729 15180.02629 2020-01-01 2020-01-02 2020-01-01 00:01:39 2020-01-02 03:46:39 2020-01-01 00:01:39.000 2020-01-02 03:46:39.000 99 99999 50049 5054949 99 99999 50049 5054949 -32470 32465 4628.009900990099 467429 -128 123 -2.801980198019802 -283 -990 100 10980 99891 2.97297 299.97297 151.47297 15147.29729 2.97297 299.97296 151.47297 15147.29735 2.97297 299.97297 151.47297 15147.29700 2020-01-01 2020-01-02 2020-01-01 00:16:30 2020-01-02 03:44:51 2020-01-01 00:16:30.000 2020-01-02 03:44:51.000 990 99891 50440.5 5044050 990 99891 50440.5 5044050 -32578 32357 4565.3 456530 -128 127 0.82 82 -991 100 10981 99892 2.97597 299.97597 151.47597 15147.59759 2.97597 299.97598 151.47597 15147.59795 2.97597 299.97597 151.47597 15147.59700 2020-01-01 2020-01-02 2020-01-01 00:16:31 2020-01-02 03:44:52 2020-01-01 00:16:31.000 2020-01-02 03:44:52.000 991 99892 50441.5 5044150 991 99892 50441.5 5044150 -32577 32358 4566.3 456630 -128 127 -0.74 -74 -992 100 10982 99893 2.97897 299.97897 151.47897 15147.89789 2.97897 299.97897 151.47897 15147.89759 2.97897 299.97897 151.47897 15147.89700 2020-01-01 2020-01-02 2020-01-01 00:16:32 2020-01-02 03:44:53 2020-01-01 00:16:32.000 2020-01-02 03:44:53.000 992 99893 50442.5 5044250 992 99893 50442.5 5044250 -32576 32359 4567.3 456730 -128 124 -2.3 -230 -993 100 10983 99894 2.98198 299.98198 151.48198 15148.19819 2.98198 299.982 151.48198 15148.19823 2.98198 299.98198 151.48198 15148.19800 2020-01-01 2020-01-02 2020-01-01 00:16:33 2020-01-02 03:44:54 2020-01-01 00:16:33.000 2020-01-02 03:44:54.000 993 99894 50443.5 5044350 993 99894 50443.5 5044350 -32575 32360 4568.3 456830 -127 125 -1.3 -130 -994 100 10984 99895 2.98498 299.98498 151.48498 15148.49849 2.98498 299.985 151.48498 15148.49853 2.98498 299.98498 151.48498 15148.49800 2020-01-01 2020-01-02 2020-01-01 00:16:34 2020-01-02 03:44:55 2020-01-01 00:16:34.000 2020-01-02 03:44:55.000 994 99895 50444.5 5044450 994 99895 50444.5 5044450 -32574 32361 4569.3 456930 -126 126 -0.3 -30 -995 100 10985 99896 2.98798 299.98798 151.48798 15148.79879 2.98798 299.98798 151.48798 15148.79882 2.98798 299.98798 151.48798 15148.79800 2020-01-01 2020-01-02 2020-01-01 00:16:35 2020-01-02 03:44:56 2020-01-01 00:16:35.000 2020-01-02 03:44:56.000 995 99896 50445.5 5044550 995 99896 50445.5 5044550 -32573 32362 4570.3 457030 -125 127 0.7 70 -996 100 10986 99897 2.99099 299.99099 151.49099 15149.09909 2.99099 299.991 151.49099 15149.09942 2.99099 299.99099 151.49099 15149.09900 2020-01-01 2020-01-02 2020-01-01 00:16:36 2020-01-02 03:44:57 2020-01-01 00:16:36.000 2020-01-02 03:44:57.000 996 99897 50446.5 5044650 996 99897 50446.5 5044650 -32572 32363 4571.3 457130 -128 127 -0.86 -86 -997 100 10987 99898 2.99399 299.99399 151.49399 15149.39939 2.99399 299.994 151.49399 15149.3991 2.99399 299.99399 151.49399 15149.39900 2020-01-01 2020-01-02 2020-01-01 00:16:37 2020-01-02 03:44:58 2020-01-01 00:16:37.000 2020-01-02 03:44:58.000 997 99898 50447.5 5044750 997 99898 50447.5 5044750 -32571 32364 4572.3 457230 -128 127 -2.42 -242 -998 100 10988 99899 2.99699 299.99699 151.49699 15149.69969 2.99699 299.997 151.49699 15149.6997 2.99699 299.99699 151.49699 15149.69900 2020-01-01 2020-01-02 2020-01-01 00:16:38 2020-01-02 03:44:59 2020-01-01 00:16:38.000 2020-01-02 03:44:59.000 998 99899 50448.5 5044850 998 99899 50448.5 5044850 -32570 32365 4573.3 457330 -128 123 -3.98 -398 +986 100 10976 99887 2.96096 299.96096 151.46096 15146.09609 2.96096 299.96097 151.46096 15146.09648 2.96096 299.96096 151.46096000000009 15146.09600 2020-01-01 2020-01-02 2020-01-01 00:16:26 2020-01-02 03:44:47 2020-01-01 00:16:26.000 2020-01-02 03:44:47.000 986 99887 50436.5 5043650 986 99887 50436.5 5043650 -32582 32353 4561.3 456130 -127 124 -0.62 -62 +987 100 10977 99888 2.96396 299.96396 151.46396 15146.39639 2.96396 299.96396 151.46396 15146.39612 2.96396 299.96396 151.46395999999993 15146.39600 2020-01-01 2020-01-02 2020-01-01 00:16:27 2020-01-02 03:44:48 2020-01-01 00:16:27.000 2020-01-02 03:44:48.000 987 99888 50437.5 5043750 987 99888 50437.5 5043750 -32581 32354 4562.3 456230 -126 125 0.38 38 +988 100 10978 99889 2.96696 299.96696 151.46696 15146.69669 2.96696 299.96698 151.46696 15146.69676 2.96696 299.96696 151.46695999999991 15146.69600 2020-01-01 2020-01-02 2020-01-01 00:16:28 2020-01-02 03:44:49 2020-01-01 00:16:28.000 2020-01-02 03:44:49.000 988 99889 50438.5 5043850 988 99889 50438.5 5043850 -32580 32355 4563.3 456330 -125 126 1.38 138 +989 100 10979 99890 2.96996 299.96996 151.46996 15146.99699 2.96997 299.96997 151.46997 15146.99706 2.96996 299.96996 151.46996000000007 15146.99600 2020-01-01 2020-01-02 2020-01-01 00:16:29 2020-01-02 03:44:50 2020-01-01 00:16:29.000 2020-01-02 03:44:50.000 989 99890 50439.5 5043950 989 99890 50439.5 5043950 -32579 32356 4564.3 456430 -124 127 2.38 238 +99 102 10089 99999 0.29729 300.29729 150.29729 15180.02702 0.29729 300.2973 150.29729 15180.02726 0.29729 300.29729 150.29729000000012 15180.02629 2020-01-01 2020-01-02 2020-01-01 00:01:39 2020-01-02 03:46:39 2020-01-01 00:01:39.000 2020-01-02 03:46:39.000 99 99999 50049 5054949 99 99999 50049 5054949 -32470 32465 4628.009900990099 467429 -128 123 -2.801980198019802 -283 +990 100 10980 99891 2.97297 299.97297 151.47297 15147.29729 2.97297 299.97296 151.47297 15147.29735 2.97297 299.97297 151.47297000000003 15147.29700 2020-01-01 2020-01-02 2020-01-01 00:16:30 2020-01-02 03:44:51 2020-01-01 00:16:30.000 2020-01-02 03:44:51.000 990 99891 50440.5 5044050 990 99891 50440.5 5044050 -32578 32357 4565.3 456530 -128 127 0.82 82 +991 100 10981 99892 2.97597 299.97597 151.47597 15147.59759 2.97597 299.97598 151.47597 15147.59795 2.97597 299.97597 151.47597000000002 15147.59700 2020-01-01 2020-01-02 2020-01-01 00:16:31 2020-01-02 03:44:52 2020-01-01 00:16:31.000 2020-01-02 03:44:52.000 991 99892 50441.5 5044150 991 99892 50441.5 5044150 -32577 32358 4566.3 456630 -128 127 -0.74 -74 +992 100 10982 99893 2.97897 299.97897 151.47897 15147.89789 2.97897 299.97897 151.47897 15147.89759 2.97897 299.97897 151.4789700000001 15147.89700 2020-01-01 2020-01-02 2020-01-01 00:16:32 2020-01-02 03:44:53 2020-01-01 00:16:32.000 2020-01-02 03:44:53.000 992 99893 50442.5 5044250 992 99893 50442.5 5044250 -32576 32359 4567.3 456730 -128 124 -2.3 -230 +993 100 10983 99894 2.98198 299.98198 151.48198 15148.19819 2.98198 299.982 151.48198 15148.19823 2.98198 299.98198 151.48197999999996 15148.19800 2020-01-01 2020-01-02 2020-01-01 00:16:33 2020-01-02 03:44:54 2020-01-01 00:16:33.000 2020-01-02 03:44:54.000 993 99894 50443.5 5044350 993 99894 50443.5 5044350 -32575 32360 4568.3 456830 -127 125 -1.3 -130 +994 100 10984 99895 2.98498 299.98498 151.48498 15148.49849 2.98498 299.985 151.48498 15148.49853 2.98498 299.98498 151.48497999999998 15148.49800 2020-01-01 2020-01-02 2020-01-01 00:16:34 2020-01-02 03:44:55 2020-01-01 00:16:34.000 2020-01-02 03:44:55.000 994 99895 50444.5 5044450 994 99895 50444.5 5044450 -32574 32361 4569.3 456930 -126 126 -0.3 -30 +995 100 10985 99896 2.98798 299.98798 151.48798 15148.79879 2.98798 299.98798 151.48798 15148.79882 2.98798 299.98798 151.4879799999999 15148.79800 2020-01-01 2020-01-02 2020-01-01 00:16:35 2020-01-02 03:44:56 2020-01-01 00:16:35.000 2020-01-02 03:44:56.000 995 99896 50445.5 5044550 995 99896 50445.5 5044550 -32573 32362 4570.3 457030 -125 127 0.7 70 +996 100 10986 99897 2.99099 299.99099 151.49099 15149.09909 2.99099 299.991 151.49099 15149.09942 2.99099 299.99099 151.49099000000012 15149.09900 2020-01-01 2020-01-02 2020-01-01 00:16:36 2020-01-02 03:44:57 2020-01-01 00:16:36.000 2020-01-02 03:44:57.000 996 99897 50446.5 5044650 996 99897 50446.5 5044650 -32572 32363 4571.3 457130 -128 127 -0.86 -86 +997 100 10987 99898 2.99399 299.99399 151.49399 15149.39939 2.99399 299.994 151.49399 15149.3991 2.99399 299.99399 151.49399000000005 15149.39900 2020-01-01 2020-01-02 2020-01-01 00:16:37 2020-01-02 03:44:58 2020-01-01 00:16:37.000 2020-01-02 03:44:58.000 997 99898 50447.5 5044750 997 99898 50447.5 5044750 -32571 32364 4572.3 457230 -128 127 -2.42 -242 +998 100 10988 99899 2.99699 299.99699 151.49699 15149.69969 2.99699 299.997 151.49699 15149.6997 2.99699 299.99699 151.49698999999993 15149.69900 2020-01-01 2020-01-02 2020-01-01 00:16:38 2020-01-02 03:44:59 2020-01-01 00:16:38.000 2020-01-02 03:44:59.000 998 99899 50448.5 5044850 998 99899 50448.5 5044850 -32570 32365 4573.3 457330 -128 123 -3.98 -398 ---- select with states ---- -1 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N -2 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N -3 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N -4 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N -5 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N -0 2 0 99900 0 300 150 15150 0 300 150 15150 0.00000 300.00000 150.00000 15150.00000 2020-01-01 2020-01-02 2020-01-01 00:00:00 2020-01-02 03:45:00 2020-01-01 00:00:00.000 2020-01-02 03:45:00.000 0 99900 49950 5044950 0 99900 49950 5044950 -32569 32366 4529.009900990099 457430 -127 124 -2.9504950495049505 -298 -1 2 1 9991 0.003 300.003 150.003 15150.3033 0.003 300.003 150.003 15150.30329 0.00300 300.00300 150.00300 15150.30300 2020-01-01 2020-01-02 2020-01-01 00:00:01 2020-01-02 03:45:01 2020-01-01 00:00:01.000 2020-01-02 03:45:01.000 1 99901 49951 5045051 1 99901 49951 5045051 -32568 32367 4530.009900990099 457531 -126 125 -1.9504950495049505 -197 -10 2 10 99910 0.03003 300.03003 150.03003 15153.03303 0.03003 300.03003 150.03002 15153.03296 0.03003 300.03003 150.03003 15153.03303 2020-01-01 2020-01-02 2020-01-01 00:00:10 2020-01-02 03:45:10 2020-01-01 00:00:10.000 2020-01-02 03:45:10.000 10 99910 49960 5045960 10 99910 49960 5045960 -32559 32376 4539.009900990099 458440 -128 127 -0.5544554455445545 -56 -100 2 100 99001 0.3003 297.3003 148.8003 14880.03003 0.3003 297.3003 148.80029 14880.02962 0.30030 297.30030 148.80030 14880.03000 2020-01-01 2020-01-02 2020-01-01 00:01:40 2020-01-02 03:30:01 2020-01-01 00:01:40.000 2020-01-02 03:30:01.000 100 99001 49550.5 4955050 100 99001 49550.5 4955050 -32469 32466 4986.02 498602 -127 124 -0.86 -86 -101 2 10091 99002 0.3033 297.3033 148.8033 14880.33033 0.3033 297.3033 148.8033 14880.33035 0.30330 297.30330 148.80330 14880.33000 2020-01-01 2020-01-02 2020-01-01 00:01:41 2020-01-02 03:30:02 2020-01-01 00:01:41.000 2020-01-02 03:30:02.000 101 99002 49551.5 4955150 101 99002 49551.5 4955150 -32468 32467 4987.02 498702 -126 125 0.14 14 -102 2 10092 99003 0.3063 297.3063 148.8063 14880.63063 0.3063 297.3063 148.8063 14880.6305 0.30630 297.30630 148.80630 14880.63000 2020-01-01 2020-01-02 2020-01-01 00:01:42 2020-01-02 03:30:03 2020-01-01 00:01:42.000 2020-01-02 03:30:03.000 102 99003 49552.5 4955250 102 99003 49552.5 4955250 -32467 32468 4988.02 498802 -125 126 1.14 114 -103 2 10093 99004 0.3093 297.3093 148.8093 14880.93093 0.3093 297.3093 148.8093 14880.93085 0.30930 297.30930 148.80930 14880.93000 2020-01-01 2020-01-02 2020-01-01 00:01:43 2020-01-02 03:30:04 2020-01-01 00:01:43.000 2020-01-02 03:30:04.000 103 99004 49553.5 4955350 103 99004 49553.5 4955350 -32466 32469 4989.02 498902 -124 127 2.14 214 -104 2 10094 99005 0.31231 297.31231 148.81231 14881.23123 0.31231 297.31232 148.81231 14881.23144 0.31231 297.31231 148.81231 14881.23100 2020-01-01 2020-01-02 2020-01-01 00:01:44 2020-01-02 03:30:05 2020-01-01 00:01:44.000 2020-01-02 03:30:05.000 104 99005 49554.5 4955450 104 99005 49554.5 4955450 -32465 32470 4990.02 499002 -128 127 0.58 58 -105 2 10095 99006 0.31531 297.31531 148.81531 14881.53153 0.31531 297.3153 148.81531 14881.53174 0.31531 297.31531 148.81531 14881.53100 2020-01-01 2020-01-02 2020-01-01 00:01:45 2020-01-02 03:30:06 2020-01-01 00:01:45.000 2020-01-02 03:30:06.000 105 99006 49555.5 4955550 105 99006 49555.5 4955550 -32464 32471 4991.02 499102 -128 123 -0.98 -98 -106 2 10096 99007 0.31831 297.31831 148.81831 14881.83183 0.31831 297.31833 148.81831 14881.83182 0.31831 297.31831 148.81831 14881.83100 2020-01-01 2020-01-02 2020-01-01 00:01:46 2020-01-02 03:30:07 2020-01-01 00:01:46.000 2020-01-02 03:30:07.000 106 99007 49556.5 4955650 106 99007 49556.5 4955650 -32463 32472 4992.02 499202 -127 124 0.02 2 -107 2 10097 99008 0.32132 297.32132 148.82132 14882.13213 0.32132 297.32132 148.82131 14882.13197 0.32132 297.32132 148.82132 14882.13200 2020-01-01 2020-01-02 2020-01-01 00:01:47 2020-01-02 03:30:08 2020-01-01 00:01:47.000 2020-01-02 03:30:08.000 107 99008 49557.5 4955750 107 99008 49557.5 4955750 -32462 32473 4993.02 499302 -126 125 1.02 102 -108 2 10098 99009 0.32432 297.32432 148.82432 14882.43243 0.32432 297.3243 148.82432 14882.43232 0.32432 297.32432 148.82432 14882.43200 2020-01-01 2020-01-02 2020-01-01 00:01:48 2020-01-02 03:30:09 2020-01-01 00:01:48.000 2020-01-02 03:30:09.000 108 99009 49558.5 4955850 108 99009 49558.5 4955850 -32461 32474 4994.02 499402 -125 126 2.02 202 -109 2 10099 99010 0.32732 297.32732 148.82732 14882.73273 0.32732 297.32733 148.82732 14882.7329 0.32732 297.32732 148.82732 14882.73200 2020-01-01 2020-01-02 2020-01-01 00:01:49 2020-01-02 03:30:10 2020-01-01 00:01:49.000 2020-01-02 03:30:10.000 109 99010 49559.5 4955950 109 99010 49559.5 4955950 -32460 32475 4995.02 499502 -124 127 3.02 302 +0 2 0 99900 0 300 150 15150 0 300 150 15150 0.00000 300.00000 150 15150.00000 2020-01-01 2020-01-02 2020-01-01 00:00:00 2020-01-02 03:45:00 2020-01-01 00:00:00.000 2020-01-02 03:45:00.000 0 99900 49950 5044950 0 99900 49950 5044950 -32569 32366 4529.009900990099 457430 -127 124 -2.9504950495049505 -298 +1 2 1 9991 0.003 300.003 150.003 15150.3033 0.003 300.003 150.003 15150.30329 0.00300 300.00300 150.00300000000007 15150.30300 2020-01-01 2020-01-02 2020-01-01 00:00:01 2020-01-02 03:45:01 2020-01-01 00:00:01.000 2020-01-02 03:45:01.000 1 99901 49951 5045051 1 99901 49951 5045051 -32568 32367 4530.009900990099 457531 -126 125 -1.9504950495049505 -197 +10 2 10 99910 0.03003 300.03003 150.03003 15153.03303 0.03003 300.03003 150.03002 15153.03296 0.03003 300.03003 150.03002999999993 15153.03303 2020-01-01 2020-01-02 2020-01-01 00:00:10 2020-01-02 03:45:10 2020-01-01 00:00:10.000 2020-01-02 03:45:10.000 10 99910 49960 5045960 10 99910 49960 5045960 -32559 32376 4539.009900990099 458440 -128 127 -0.5544554455445545 -56 +100 2 100 99001 0.3003 297.3003 148.8003 14880.03003 0.3003 297.3003 148.80029 14880.02962 0.30030 297.30030 148.8003000000001 14880.03000 2020-01-01 2020-01-02 2020-01-01 00:01:40 2020-01-02 03:30:01 2020-01-01 00:01:40.000 2020-01-02 03:30:01.000 100 99001 49550.5 4955050 100 99001 49550.5 4955050 -32469 32466 4986.02 498602 -127 124 -0.86 -86 +101 2 10091 99002 0.3033 297.3033 148.8033 14880.33033 0.3033 297.3033 148.8033 14880.33035 0.30330 297.30330 148.80329999999978 14880.33000 2020-01-01 2020-01-02 2020-01-01 00:01:41 2020-01-02 03:30:02 2020-01-01 00:01:41.000 2020-01-02 03:30:02.000 101 99002 49551.5 4955150 101 99002 49551.5 4955150 -32468 32467 4987.02 498702 -126 125 0.14 14 +102 2 10092 99003 0.3063 297.3063 148.8063 14880.63063 0.3063 297.3063 148.8063 14880.6305 0.30630 297.30630 148.80630000000014 14880.63000 2020-01-01 2020-01-02 2020-01-01 00:01:42 2020-01-02 03:30:03 2020-01-01 00:01:42.000 2020-01-02 03:30:03.000 102 99003 49552.5 4955250 102 99003 49552.5 4955250 -32467 32468 4988.02 498802 -125 126 1.14 114 +103 2 10093 99004 0.3093 297.3093 148.8093 14880.93093 0.3093 297.3093 148.8093 14880.93085 0.30930 297.30930 148.80930000000012 14880.93000 2020-01-01 2020-01-02 2020-01-01 00:01:43 2020-01-02 03:30:04 2020-01-01 00:01:43.000 2020-01-02 03:30:04.000 103 99004 49553.5 4955350 103 99004 49553.5 4955350 -32466 32469 4989.02 498902 -124 127 2.14 214 +104 2 10094 99005 0.31231 297.31231 148.81231 14881.23123 0.31231 297.31232 148.81231 14881.23144 0.31231 297.31231 148.81230999999988 14881.23100 2020-01-01 2020-01-02 2020-01-01 00:01:44 2020-01-02 03:30:05 2020-01-01 00:01:44.000 2020-01-02 03:30:05.000 104 99005 49554.5 4955450 104 99005 49554.5 4955450 -32465 32470 4990.02 499002 -128 127 0.58 58 +105 2 10095 99006 0.31531 297.31531 148.81531 14881.53153 0.31531 297.3153 148.81531 14881.53174 0.31531 297.31531 148.81530999999998 14881.53100 2020-01-01 2020-01-02 2020-01-01 00:01:45 2020-01-02 03:30:06 2020-01-01 00:01:45.000 2020-01-02 03:30:06.000 105 99006 49555.5 4955550 105 99006 49555.5 4955550 -32464 32471 4991.02 499102 -128 123 -0.98 -98 +106 2 10096 99007 0.31831 297.31831 148.81831 14881.83183 0.31831 297.31833 148.81831 14881.83182 0.31831 297.31831 148.81831000000003 14881.83100 2020-01-01 2020-01-02 2020-01-01 00:01:46 2020-01-02 03:30:07 2020-01-01 00:01:46.000 2020-01-02 03:30:07.000 106 99007 49556.5 4955650 106 99007 49556.5 4955650 -32463 32472 4992.02 499202 -127 124 0.02 2 +107 2 10097 99008 0.32132 297.32132 148.82132 14882.13213 0.32132 297.32132 148.82131 14882.13197 0.32132 297.32132 148.8213199999998 14882.13200 2020-01-01 2020-01-02 2020-01-01 00:01:47 2020-01-02 03:30:08 2020-01-01 00:01:47.000 2020-01-02 03:30:08.000 107 99008 49557.5 4955750 107 99008 49557.5 4955750 -32462 32473 4993.02 499302 -126 125 1.02 102 +108 2 10098 99009 0.32432 297.32432 148.82432 14882.43243 0.32432 297.3243 148.82432 14882.43232 0.32432 297.32432 148.82431999999986 14882.43200 2020-01-01 2020-01-02 2020-01-01 00:01:48 2020-01-02 03:30:09 2020-01-01 00:01:48.000 2020-01-02 03:30:09.000 108 99009 49558.5 4955850 108 99009 49558.5 4955850 -32461 32474 4994.02 499402 -125 126 2.02 202 +109 2 10099 99010 0.32732 297.32732 148.82732 14882.73273 0.32732 297.32733 148.82732 14882.7329 0.32732 297.32732 148.82732000000016 14882.73200 2020-01-01 2020-01-02 2020-01-01 00:01:49 2020-01-02 03:30:10 2020-01-01 00:01:49.000 2020-01-02 03:30:10.000 109 99010 49559.5 4955950 109 99010 49559.5 4955950 -32460 32475 4995.02 499502 -124 127 3.02 302 11 2 10001 99911 0.03303 300.03303 150.03303 15153.33633 0.03303 300.03302 150.03303 15153.33627 0.03303 300.03303 150.03303 15153.33603 2020-01-01 2020-01-02 2020-01-01 00:00:11 2020-01-02 03:45:11 2020-01-01 00:00:11.000 2020-01-02 03:45:11.000 11 99911 49961 5046061 11 99911 49961 5046061 -32558 32377 4540.009900990099 458541 -128 123 -2.089108910891089 -211 -110 2 10100 99011 0.33033 297.33033 148.83033 14883.03303 0.33033 297.33032 148.83033 14883.03321 0.33033 297.33033 148.83033 14883.03300 2020-01-01 2020-01-02 2020-01-01 00:01:50 2020-01-02 03:30:11 2020-01-01 00:01:50.000 2020-01-02 03:30:11.000 110 99011 49560.5 4956050 110 99011 49560.5 4956050 -32459 32476 4996.02 499602 -128 127 1.46 146 -111 2 10101 99012 0.33333 297.33333 148.83333 14883.33333 0.33333 297.33334 148.83333 14883.33329 0.33333 297.33333 148.83333 14883.33300 2020-01-01 2020-01-02 2020-01-01 00:01:51 2020-01-02 03:30:12 2020-01-01 00:01:51.000 2020-01-02 03:30:12.000 111 99012 49561.5 4956150 111 99012 49561.5 4956150 -32458 32477 4997.02 499702 -128 123 -0.1 -10 -112 2 10102 99013 0.33633 297.33633 148.83633 14883.63363 0.33633 297.33633 148.83633 14883.63348 0.33633 297.33633 148.83633 14883.63300 2020-01-01 2020-01-02 2020-01-01 00:01:52 2020-01-02 03:30:13 2020-01-01 00:01:52.000 2020-01-02 03:30:13.000 112 99013 49562.5 4956250 112 99013 49562.5 4956250 -32457 32478 4998.02 499802 -127 124 0.9 90 -113 2 10103 99014 0.33933 297.33933 148.83933 14883.93393 0.33933 297.33932 148.83933 14883.9338 0.33933 297.33933 148.83933 14883.93300 2020-01-01 2020-01-02 2020-01-01 00:01:53 2020-01-02 03:30:14 2020-01-01 00:01:53.000 2020-01-02 03:30:14.000 113 99014 49563.5 4956350 113 99014 49563.5 4956350 -32456 32479 4999.02 499902 -126 125 1.9 190 -114 2 10104 99015 0.34234 297.34234 148.84234 14884.23423 0.34234 297.34235 148.84234 14884.23437 0.34234 297.34234 148.84234 14884.23400 2020-01-01 2020-01-02 2020-01-01 00:01:54 2020-01-02 03:30:15 2020-01-01 00:01:54.000 2020-01-02 03:30:15.000 114 99015 49564.5 4956450 114 99015 49564.5 4956450 -32455 32480 5000.02 500002 -125 126 2.9 290 -115 2 10105 99016 0.34534 297.34534 148.84534 14884.53453 0.34534 297.34534 148.84534 14884.53468 0.34534 297.34534 148.84534 14884.53400 2020-01-01 2020-01-02 2020-01-01 00:01:55 2020-01-02 03:30:16 2020-01-01 00:01:55.000 2020-01-02 03:30:16.000 115 99016 49565.5 4956550 115 99016 49565.5 4956550 -32454 32481 5001.02 500102 -124 127 3.9 390 -116 2 10106 99017 0.34834 297.34834 148.84834 14884.83483 0.34834 297.34836 148.84834 14884.83476 0.34834 297.34834 148.84834 14884.83400 2020-01-01 2020-01-02 2020-01-01 00:01:56 2020-01-02 03:30:17 2020-01-01 00:01:56.000 2020-01-02 03:30:17.000 116 99017 49566.5 4956650 116 99017 49566.5 4956650 -32453 32482 5002.02 500202 -128 127 2.34 234 -117 2 10107 99018 0.35135 297.35135 148.85135 14885.13513 0.35135 297.35135 148.85134 14885.13495 0.35135 297.35135 148.85135 14885.13500 2020-01-01 2020-01-02 2020-01-01 00:01:57 2020-01-02 03:30:18 2020-01-01 00:01:57.000 2020-01-02 03:30:18.000 117 99018 49567.5 4956750 117 99018 49567.5 4956750 -32452 32483 5003.02 500302 -128 123 0.78 78 -118 2 10108 99019 0.35435 297.35435 148.85435 14885.43543 0.35435 297.35434 148.85435 14885.43526 0.35435 297.35435 148.85435 14885.43500 2020-01-01 2020-01-02 2020-01-01 00:01:58 2020-01-02 03:30:19 2020-01-01 00:01:58.000 2020-01-02 03:30:19.000 118 99019 49568.5 4956850 118 99019 49568.5 4956850 -32451 32484 5004.02 500402 -127 124 1.78 178 -119 2 10109 99020 0.35735 297.35735 148.85735 14885.73573 0.35735 297.35736 148.85736 14885.736 0.35735 297.35735 148.85735 14885.73500 2020-01-01 2020-01-02 2020-01-01 00:01:59 2020-01-02 03:30:20 2020-01-01 00:01:59.000 2020-01-02 03:30:20.000 119 99020 49569.5 4956950 119 99020 49569.5 4956950 -32450 32485 5005.02 500502 -126 125 2.78 278 -12 2 10002 99912 0.03603 300.03603 150.03603 15153.63963 0.03603 300.03604 150.03603 15153.6399 0.03603 300.03603 150.03603 15153.63903 2020-01-01 2020-01-02 2020-01-01 00:00:12 2020-01-02 03:45:12 2020-01-01 00:00:12.000 2020-01-02 03:45:12.000 12 99912 49962 5046162 12 99912 49962 5046162 -32557 32378 4541.009900990099 458642 -127 124 -1.0891089108910892 -110 -120 2 10110 99021 0.36036 297.36036 148.86036 14886.03603 0.36036 297.36035 148.86036 14886.03615 0.36036 297.36036 148.86036 14886.03600 2020-01-01 2020-01-02 2020-01-01 00:02:00 2020-01-02 03:30:21 2020-01-01 00:02:00.000 2020-01-02 03:30:21.000 120 99021 49570.5 4957050 120 99021 49570.5 4957050 -32449 32486 5006.02 500602 -125 126 3.78 378 -121 2 10111 99022 0.36336 297.36336 148.86336 14886.33633 0.36336 297.36337 148.86336 14886.33627 0.36336 297.36336 148.86336 14886.33600 2020-01-01 2020-01-02 2020-01-01 00:02:01 2020-01-02 03:30:22 2020-01-01 00:02:01.000 2020-01-02 03:30:22.000 121 99022 49571.5 4957150 121 99022 49571.5 4957150 -32448 32487 5007.02 500702 -124 127 4.78 478 +110 2 10100 99011 0.33033 297.33033 148.83033 14883.03303 0.33033 297.33032 148.83033 14883.03321 0.33033 297.33033 148.83033000000006 14883.03300 2020-01-01 2020-01-02 2020-01-01 00:01:50 2020-01-02 03:30:11 2020-01-01 00:01:50.000 2020-01-02 03:30:11.000 110 99011 49560.5 4956050 110 99011 49560.5 4956050 -32459 32476 4996.02 499602 -128 127 1.46 146 +111 2 10101 99012 0.33333 297.33333 148.83333 14883.33333 0.33333 297.33334 148.83333 14883.33329 0.33333 297.33333 148.83332999999996 14883.33300 2020-01-01 2020-01-02 2020-01-01 00:01:51 2020-01-02 03:30:12 2020-01-01 00:01:51.000 2020-01-02 03:30:12.000 111 99012 49561.5 4956150 111 99012 49561.5 4956150 -32458 32477 4997.02 499702 -128 123 -0.1 -10 +112 2 10102 99013 0.33633 297.33633 148.83633 14883.63363 0.33633 297.33633 148.83633 14883.63348 0.33633 297.33633 148.83633000000006 14883.63300 2020-01-01 2020-01-02 2020-01-01 00:01:52 2020-01-02 03:30:13 2020-01-01 00:01:52.000 2020-01-02 03:30:13.000 112 99013 49562.5 4956250 112 99013 49562.5 4956250 -32457 32478 4998.02 499802 -127 124 0.9 90 +113 2 10103 99014 0.33933 297.33933 148.83933 14883.93393 0.33933 297.33932 148.83933 14883.9338 0.33933 297.33933 148.83933000000007 14883.93300 2020-01-01 2020-01-02 2020-01-01 00:01:53 2020-01-02 03:30:14 2020-01-01 00:01:53.000 2020-01-02 03:30:14.000 113 99014 49563.5 4956350 113 99014 49563.5 4956350 -32456 32479 4999.02 499902 -126 125 1.9 190 +114 2 10104 99015 0.34234 297.34234 148.84234 14884.23423 0.34234 297.34235 148.84234 14884.23437 0.34234 297.34234 148.84233999999992 14884.23400 2020-01-01 2020-01-02 2020-01-01 00:01:54 2020-01-02 03:30:15 2020-01-01 00:01:54.000 2020-01-02 03:30:15.000 114 99015 49564.5 4956450 114 99015 49564.5 4956450 -32455 32480 5000.02 500002 -125 126 2.9 290 +115 2 10105 99016 0.34534 297.34534 148.84534 14884.53453 0.34534 297.34534 148.84534 14884.53468 0.34534 297.34534 148.8453399999999 14884.53400 2020-01-01 2020-01-02 2020-01-01 00:01:55 2020-01-02 03:30:16 2020-01-01 00:01:55.000 2020-01-02 03:30:16.000 115 99016 49565.5 4956550 115 99016 49565.5 4956550 -32454 32481 5001.02 500102 -124 127 3.9 390 +116 2 10106 99017 0.34834 297.34834 148.84834 14884.83483 0.34834 297.34836 148.84834 14884.83476 0.34834 297.34834 148.84834000000004 14884.83400 2020-01-01 2020-01-02 2020-01-01 00:01:56 2020-01-02 03:30:17 2020-01-01 00:01:56.000 2020-01-02 03:30:17.000 116 99017 49566.5 4956650 116 99017 49566.5 4956650 -32453 32482 5002.02 500202 -128 127 2.34 234 +117 2 10107 99018 0.35135 297.35135 148.85135 14885.13513 0.35135 297.35135 148.85134 14885.13495 0.35135 297.35135 148.85135000000014 14885.13500 2020-01-01 2020-01-02 2020-01-01 00:01:57 2020-01-02 03:30:18 2020-01-01 00:01:57.000 2020-01-02 03:30:18.000 117 99018 49567.5 4956750 117 99018 49567.5 4956750 -32452 32483 5003.02 500302 -128 123 0.78 78 +118 2 10108 99019 0.35435 297.35435 148.85435 14885.43543 0.35435 297.35434 148.85435 14885.43526 0.35435 297.35435 148.8543499999998 14885.43500 2020-01-01 2020-01-02 2020-01-01 00:01:58 2020-01-02 03:30:19 2020-01-01 00:01:58.000 2020-01-02 03:30:19.000 118 99019 49568.5 4956850 118 99019 49568.5 4956850 -32451 32484 5004.02 500402 -127 124 1.78 178 +119 2 10109 99020 0.35735 297.35735 148.85735 14885.73573 0.35735 297.35736 148.85736 14885.736 0.35735 297.35735 148.85735000000017 14885.73500 2020-01-01 2020-01-02 2020-01-01 00:01:59 2020-01-02 03:30:20 2020-01-01 00:01:59.000 2020-01-02 03:30:20.000 119 99020 49569.5 4956950 119 99020 49569.5 4956950 -32450 32485 5005.02 500502 -126 125 2.78 278 +12 2 10002 99912 0.03603 300.03603 150.03603 15153.63963 0.03603 300.03604 150.03603 15153.6399 0.03603 300.03603 150.03602999999995 15153.63903 2020-01-01 2020-01-02 2020-01-01 00:00:12 2020-01-02 03:45:12 2020-01-01 00:00:12.000 2020-01-02 03:45:12.000 12 99912 49962 5046162 12 99912 49962 5046162 -32557 32378 4541.009900990099 458642 -127 124 -1.0891089108910892 -110 +120 2 10110 99021 0.36036 297.36036 148.86036 14886.03603 0.36036 297.36035 148.86036 14886.03615 0.36036 297.36036 148.86036000000007 14886.03600 2020-01-01 2020-01-02 2020-01-01 00:02:00 2020-01-02 03:30:21 2020-01-01 00:02:00.000 2020-01-02 03:30:21.000 120 99021 49570.5 4957050 120 99021 49570.5 4957050 -32449 32486 5006.02 500602 -125 126 3.78 378 +121 2 10111 99022 0.36336 297.36336 148.86336 14886.33633 0.36336 297.36337 148.86336 14886.33627 0.36336 297.36336 148.86335999999991 14886.33600 2020-01-01 2020-01-02 2020-01-01 00:02:01 2020-01-02 03:30:22 2020-01-01 00:02:01.000 2020-01-02 03:30:22.000 121 99022 49571.5 4957150 121 99022 49571.5 4957150 -32448 32487 5007.02 500702 -124 127 4.78 478 122 2 10112 99023 0.36636 297.36636 148.86636 14886.63663 0.36636 297.36636 148.86636 14886.63642 0.36636 297.36636 148.86636 14886.63600 2020-01-01 2020-01-02 2020-01-01 00:02:02 2020-01-02 03:30:23 2020-01-01 00:02:02.000 2020-01-02 03:30:23.000 122 99023 49572.5 4957250 122 99023 49572.5 4957250 -32447 32488 5008.02 500802 -128 127 3.22 322 -123 2 10113 99024 0.36936 297.36936 148.86936 14886.93693 0.36936 297.36935 148.86936 14886.93673 0.36936 297.36936 148.86936 14886.93600 2020-01-01 2020-01-02 2020-01-01 00:02:03 2020-01-02 03:30:24 2020-01-01 00:02:03.000 2020-01-02 03:30:24.000 123 99024 49573.5 4957350 123 99024 49573.5 4957350 -32446 32489 5009.02 500902 -128 127 1.66 166 -124 2 10114 99025 0.37237 297.37237 148.87237 14887.23723 0.37237 297.37238 148.87237 14887.23746 0.37237 297.37237 148.87237 14887.23700 2020-01-01 2020-01-02 2020-01-01 00:02:04 2020-01-02 03:30:25 2020-01-01 00:02:04.000 2020-01-02 03:30:25.000 124 99025 49574.5 4957450 124 99025 49574.5 4957450 -32445 32490 5010.02 501002 -128 124 0.1 10 -125 2 10115 99026 0.37537 297.37537 148.87537 14887.53753 0.37537 297.37537 148.87537 14887.53762 0.37537 297.37537 148.87537 14887.53700 2020-01-01 2020-01-02 2020-01-01 00:02:05 2020-01-02 03:30:26 2020-01-01 00:02:05.000 2020-01-02 03:30:26.000 125 99026 49575.5 4957550 125 99026 49575.5 4957550 -32444 32491 5011.02 501102 -127 125 1.1 110 -126 2 10116 99027 0.37837 297.37837 148.87837 14887.83783 0.37837 297.3784 148.87837 14887.83774 0.37837 297.37837 148.87837 14887.83700 2020-01-01 2020-01-02 2020-01-01 00:02:06 2020-01-02 03:30:27 2020-01-01 00:02:06.000 2020-01-02 03:30:27.000 126 99027 49576.5 4957650 126 99027 49576.5 4957650 -32443 32492 5012.02 501202 -126 126 2.1 210 -127 2 10117 99028 0.38138 297.38138 148.88138 14888.13813 0.38138 297.38138 148.88137 14888.13789 0.38138 297.38138 148.88138 14888.13800 2020-01-01 2020-01-02 2020-01-01 00:02:07 2020-01-02 03:30:28 2020-01-01 00:02:07.000 2020-01-02 03:30:28.000 127 99028 49577.5 4957750 127 99028 49577.5 4957750 -32442 32493 5013.02 501302 -125 127 3.1 310 +123 2 10113 99024 0.36936 297.36936 148.86936 14886.93693 0.36936 297.36935 148.86936 14886.93673 0.36936 297.36936 148.86936000000003 14886.93600 2020-01-01 2020-01-02 2020-01-01 00:02:03 2020-01-02 03:30:24 2020-01-01 00:02:03.000 2020-01-02 03:30:24.000 123 99024 49573.5 4957350 123 99024 49573.5 4957350 -32446 32489 5009.02 500902 -128 127 1.66 166 +124 2 10114 99025 0.37237 297.37237 148.87237 14887.23723 0.37237 297.37238 148.87237 14887.23746 0.37237 297.37237 148.8723699999998 14887.23700 2020-01-01 2020-01-02 2020-01-01 00:02:04 2020-01-02 03:30:25 2020-01-01 00:02:04.000 2020-01-02 03:30:25.000 124 99025 49574.5 4957450 124 99025 49574.5 4957450 -32445 32490 5010.02 501002 -128 124 0.1 10 +125 2 10115 99026 0.37537 297.37537 148.87537 14887.53753 0.37537 297.37537 148.87537 14887.53762 0.37537 297.37537 148.87536999999986 14887.53700 2020-01-01 2020-01-02 2020-01-01 00:02:05 2020-01-02 03:30:26 2020-01-01 00:02:05.000 2020-01-02 03:30:26.000 125 99026 49575.5 4957550 125 99026 49575.5 4957550 -32444 32491 5011.02 501102 -127 125 1.1 110 +126 2 10116 99027 0.37837 297.37837 148.87837 14887.83783 0.37837 297.3784 148.87837 14887.83774 0.37837 297.37837 148.87837000000016 14887.83700 2020-01-01 2020-01-02 2020-01-01 00:02:06 2020-01-02 03:30:27 2020-01-01 00:02:06.000 2020-01-02 03:30:27.000 126 99027 49576.5 4957650 126 99027 49576.5 4957650 -32443 32492 5012.02 501202 -126 126 2.1 210 +127 2 10117 99028 0.38138 297.38138 148.88138 14888.13813 0.38138 297.38138 148.88137 14888.13789 0.38138 297.38138 148.8813800000001 14888.13800 2020-01-01 2020-01-02 2020-01-01 00:02:07 2020-01-02 03:30:28 2020-01-01 00:02:07.000 2020-01-02 03:30:28.000 127 99028 49577.5 4957750 127 99028 49577.5 4957750 -32442 32493 5013.02 501302 -125 127 3.1 310 128 2 10118 99029 0.38438 297.38438 148.88438 14888.43843 0.38438 297.3844 148.88438 14888.43862 0.38438 297.38438 148.88438 14888.43800 2020-01-01 2020-01-02 2020-01-01 00:02:08 2020-01-02 03:30:29 2020-01-01 00:02:08.000 2020-01-02 03:30:29.000 128 99029 49578.5 4957850 128 99029 49578.5 4957850 -32441 32494 5014.02 501402 -128 127 1.54 154 -129 2 10119 99030 0.38738 297.38738 148.88738 14888.73873 0.38738 297.3874 148.88738 14888.73894 0.38738 297.38738 148.88738 14888.73800 2020-01-01 2020-01-02 2020-01-01 00:02:09 2020-01-02 03:30:30 2020-01-01 00:02:09.000 2020-01-02 03:30:30.000 129 99030 49579.5 4957950 129 99030 49579.5 4957950 -32440 32495 5015.02 501502 -128 127 -0.02 -2 -13 2 10003 99913 0.03903 300.03903 150.03903 15153.94294 0.03903 300.03903 150.03903 15153.94255 0.03903 300.03903 150.03903 15153.94203 2020-01-01 2020-01-02 2020-01-01 00:00:13 2020-01-02 03:45:13 2020-01-01 00:00:13.000 2020-01-02 03:45:13.000 13 99913 49963 5046263 13 99913 49963 5046263 -32556 32379 4542.009900990099 458743 -126 125 -0.0891089108910891 -9 -130 2 10120 99031 0.39039 297.39039 148.89039 14889.03903 0.39039 297.39038 148.89039 14889.03909 0.39039 297.39039 148.89039 14889.03900 2020-01-01 2020-01-02 2020-01-01 00:02:10 2020-01-02 03:30:31 2020-01-01 00:02:10.000 2020-01-02 03:30:31.000 130 99031 49580.5 4958050 130 99031 49580.5 4958050 -32439 32496 5016.02 501602 -128 123 -1.58 -158 -131 2 10121 99032 0.39339 297.39339 148.89339 14889.33933 0.39339 297.3934 148.89339 14889.33921 0.39339 297.39339 148.89339 14889.33900 2020-01-01 2020-01-02 2020-01-01 00:02:11 2020-01-02 03:30:32 2020-01-01 00:02:11.000 2020-01-02 03:30:32.000 131 99032 49581.5 4958150 131 99032 49581.5 4958150 -32438 32497 5017.02 501702 -127 124 -0.58 -58 -132 2 10122 99033 0.39639 297.39639 148.89639 14889.63963 0.39639 297.3964 148.89639 14889.63936 0.39639 297.39639 148.89639 14889.63900 2020-01-01 2020-01-02 2020-01-01 00:02:12 2020-01-02 03:30:33 2020-01-01 00:02:12.000 2020-01-02 03:30:33.000 132 99033 49582.5 4958250 132 99033 49582.5 4958250 -32437 32498 5018.02 501802 -126 125 0.42 42 -133 2 10123 99034 0.39939 297.39939 148.89939 14889.93993 0.39939 297.3994 148.8994 14889.94009 0.39939 297.39939 148.89939 14889.93900 2020-01-01 2020-01-02 2020-01-01 00:02:13 2020-01-02 03:30:34 2020-01-01 00:02:13.000 2020-01-02 03:30:34.000 133 99034 49583.5 4958350 133 99034 49583.5 4958350 -32436 32499 5019.02 501902 -125 126 1.42 142 -134 2 10124 99035 0.4024 297.4024 148.9024 14890.24024 0.4024 297.4024 148.9024 14890.24041 0.40240 297.40240 148.90240 14890.24000 2020-01-01 2020-01-02 2020-01-01 00:02:14 2020-01-02 03:30:35 2020-01-01 00:02:14.000 2020-01-02 03:30:35.000 134 99035 49584.5 4958450 134 99035 49584.5 4958450 -32435 32500 5020.02 502002 -124 127 2.42 242 -135 2 10125 99036 0.4054 297.4054 148.9054 14890.54054 0.4054 297.4054 148.9054 14890.54059 0.40540 297.40540 148.90540 14890.54000 2020-01-01 2020-01-02 2020-01-01 00:02:15 2020-01-02 03:30:36 2020-01-01 00:02:15.000 2020-01-02 03:30:36.000 135 99036 49585.5 4958550 135 99036 49585.5 4958550 -32434 32501 5021.02 502102 -128 127 0.86 86 -136 2 10126 99037 0.4084 297.4084 148.9084 14890.84084 0.4084 297.40842 148.9084 14890.84068 0.40840 297.40840 148.90840 14890.84000 2020-01-01 2020-01-02 2020-01-01 00:02:16 2020-01-02 03:30:37 2020-01-01 00:02:16.000 2020-01-02 03:30:37.000 136 99037 49586.5 4958650 136 99037 49586.5 4958650 -32433 32502 5022.02 502202 -128 123 -0.7 -70 -137 2 10127 99038 0.41141 297.41141 148.91141 14891.14114 0.41141 297.4114 148.9114 14891.14099 0.41141 297.41141 148.91141 14891.14100 2020-01-01 2020-01-02 2020-01-01 00:02:17 2020-01-02 03:30:38 2020-01-01 00:02:17.000 2020-01-02 03:30:38.000 137 99038 49587.5 4958750 137 99038 49587.5 4958750 -32432 32503 5023.02 502302 -127 124 0.3 30 -138 2 10128 99039 0.41441 297.41441 148.91441 14891.44144 0.41441 297.41443 148.91441 14891.44157 0.41441 297.41441 148.91441 14891.44100 2020-01-01 2020-01-02 2020-01-01 00:02:18 2020-01-02 03:30:39 2020-01-01 00:02:18.000 2020-01-02 03:30:39.000 138 99039 49588.5 4958850 138 99039 49588.5 4958850 -32431 32504 5024.02 502402 -126 125 1.3 130 +129 2 10119 99030 0.38738 297.38738 148.88738 14888.73873 0.38738 297.3874 148.88738 14888.73894 0.38738 297.38738 148.8873800000001 14888.73800 2020-01-01 2020-01-02 2020-01-01 00:02:09 2020-01-02 03:30:30 2020-01-01 00:02:09.000 2020-01-02 03:30:30.000 129 99030 49579.5 4957950 129 99030 49579.5 4957950 -32440 32495 5015.02 501502 -128 127 -0.02 -2 +13 2 10003 99913 0.03903 300.03903 150.03903 15153.94294 0.03903 300.03903 150.03903 15153.94255 0.03903 300.03903 150.03902999999994 15153.94203 2020-01-01 2020-01-02 2020-01-01 00:00:13 2020-01-02 03:45:13 2020-01-01 00:00:13.000 2020-01-02 03:45:13.000 13 99913 49963 5046263 13 99913 49963 5046263 -32556 32379 4542.009900990099 458743 -126 125 -0.0891089108910891 -9 +130 2 10120 99031 0.39039 297.39039 148.89039 14889.03903 0.39039 297.39038 148.89039 14889.03909 0.39039 297.39039 148.89038999999994 14889.03900 2020-01-01 2020-01-02 2020-01-01 00:02:10 2020-01-02 03:30:31 2020-01-01 00:02:10.000 2020-01-02 03:30:31.000 130 99031 49580.5 4958050 130 99031 49580.5 4958050 -32439 32496 5016.02 501602 -128 123 -1.58 -158 +131 2 10121 99032 0.39339 297.39339 148.89339 14889.33933 0.39339 297.3934 148.89339 14889.33921 0.39339 297.39339 148.89338999999995 14889.33900 2020-01-01 2020-01-02 2020-01-01 00:02:11 2020-01-02 03:30:32 2020-01-01 00:02:11.000 2020-01-02 03:30:32.000 131 99032 49581.5 4958150 131 99032 49581.5 4958150 -32438 32497 5017.02 501702 -127 124 -0.58 -58 +132 2 10122 99033 0.39639 297.39639 148.89639 14889.63963 0.39639 297.3964 148.89639 14889.63936 0.39639 297.39639 148.8963899999999 14889.63900 2020-01-01 2020-01-02 2020-01-01 00:02:12 2020-01-02 03:30:33 2020-01-01 00:02:12.000 2020-01-02 03:30:33.000 132 99033 49582.5 4958250 132 99033 49582.5 4958250 -32437 32498 5018.02 501802 -126 125 0.42 42 +133 2 10123 99034 0.39939 297.39939 148.89939 14889.93993 0.39939 297.3994 148.8994 14889.94009 0.39939 297.39939 148.89938999999998 14889.93900 2020-01-01 2020-01-02 2020-01-01 00:02:13 2020-01-02 03:30:34 2020-01-01 00:02:13.000 2020-01-02 03:30:34.000 133 99034 49583.5 4958350 133 99034 49583.5 4958350 -32436 32499 5019.02 501902 -125 126 1.42 142 +134 2 10124 99035 0.4024 297.4024 148.9024 14890.24024 0.4024 297.4024 148.9024 14890.24041 0.40240 297.40240 148.90240000000017 14890.24000 2020-01-01 2020-01-02 2020-01-01 00:02:14 2020-01-02 03:30:35 2020-01-01 00:02:14.000 2020-01-02 03:30:35.000 134 99035 49584.5 4958450 134 99035 49584.5 4958450 -32435 32500 5020.02 502002 -124 127 2.42 242 +135 2 10125 99036 0.4054 297.4054 148.9054 14890.54054 0.4054 297.4054 148.9054 14890.54059 0.40540 297.40540 148.90539999999984 14890.54000 2020-01-01 2020-01-02 2020-01-01 00:02:15 2020-01-02 03:30:36 2020-01-01 00:02:15.000 2020-01-02 03:30:36.000 135 99036 49585.5 4958550 135 99036 49585.5 4958550 -32434 32501 5021.02 502102 -128 127 0.86 86 +136 2 10126 99037 0.4084 297.4084 148.9084 14890.84084 0.4084 297.40842 148.9084 14890.84068 0.40840 297.40840 148.9084000000001 14890.84000 2020-01-01 2020-01-02 2020-01-01 00:02:16 2020-01-02 03:30:37 2020-01-01 00:02:16.000 2020-01-02 03:30:37.000 136 99037 49586.5 4958650 136 99037 49586.5 4958650 -32433 32502 5022.02 502202 -128 123 -0.7 -70 +137 2 10127 99038 0.41141 297.41141 148.91141 14891.14114 0.41141 297.4114 148.9114 14891.14099 0.41141 297.41141 148.91141000000007 14891.14100 2020-01-01 2020-01-02 2020-01-01 00:02:17 2020-01-02 03:30:38 2020-01-01 00:02:17.000 2020-01-02 03:30:38.000 137 99038 49587.5 4958750 137 99038 49587.5 4958750 -32432 32503 5023.02 502302 -127 124 0.3 30 +138 2 10128 99039 0.41441 297.41441 148.91441 14891.44144 0.41441 297.41443 148.91441 14891.44157 0.41441 297.41441 148.91440999999995 14891.44100 2020-01-01 2020-01-02 2020-01-01 00:02:18 2020-01-02 03:30:39 2020-01-01 00:02:18.000 2020-01-02 03:30:39.000 138 99039 49588.5 4958850 138 99039 49588.5 4958850 -32431 32504 5024.02 502402 -126 125 1.3 130 139 2 10129 99040 0.41741 297.41741 148.91741 14891.74174 0.41741 297.41742 148.91741 14891.74188 0.41741 297.41741 148.91741 14891.74100 2020-01-01 2020-01-02 2020-01-01 00:02:19 2020-01-02 03:30:40 2020-01-01 00:02:19.000 2020-01-02 03:30:40.000 139 99040 49589.5 4958950 139 99040 49589.5 4958950 -32430 32505 5025.02 502502 -125 126 2.3 230 -14 2 10004 99914 0.04204 300.04204 150.04204 15154.24624 0.04204 300.04205 150.04204 15154.2463 0.04204 300.04204 150.04204 15154.24604 2020-01-01 2020-01-02 2020-01-01 00:00:14 2020-01-02 03:45:14 2020-01-01 00:00:14.000 2020-01-02 03:45:14.000 14 99914 49964 5046364 14 99914 49964 5046364 -32555 32380 4543.009900990099 458844 -125 126 0.9108910891089109 92 -140 2 10130 99041 0.42042 297.42042 148.92042 14892.04204 0.42042 297.4204 148.92042 14892.04206 0.42042 297.42042 148.92042 14892.04200 2020-01-01 2020-01-02 2020-01-01 00:02:20 2020-01-02 03:30:41 2020-01-01 00:02:20.000 2020-01-02 03:30:41.000 140 99041 49590.5 4959050 140 99041 49590.5 4959050 -32429 32506 5026.02 502602 -124 127 3.3 330 -141 2 10131 99042 0.42342 297.42342 148.92342 14892.34234 0.42342 297.42343 148.92342 14892.34215 0.42342 297.42342 148.92342 14892.34200 2020-01-01 2020-01-02 2020-01-01 00:02:21 2020-01-02 03:30:42 2020-01-01 00:02:21.000 2020-01-02 03:30:42.000 141 99042 49591.5 4959150 141 99042 49591.5 4959150 -32428 32507 5027.02 502702 -128 127 1.74 174 -142 2 10132 99043 0.42642 297.42642 148.92642 14892.64264 0.42642 297.42642 148.92642 14892.64246 0.42642 297.42642 148.92642 14892.64200 2020-01-01 2020-01-02 2020-01-01 00:02:22 2020-01-02 03:30:43 2020-01-01 00:02:22.000 2020-01-02 03:30:43.000 142 99043 49592.5 4959250 142 99043 49592.5 4959250 -32427 32508 5028.02 502802 -128 123 0.18 18 -143 2 10133 99044 0.42942 297.42942 148.92942 14892.94294 0.42942 297.42944 148.92943 14892.94304 0.42942 297.42942 148.92942 14892.94200 2020-01-01 2020-01-02 2020-01-01 00:02:23 2020-01-02 03:30:44 2020-01-01 00:02:23.000 2020-01-02 03:30:44.000 143 99044 49593.5 4959350 143 99044 49593.5 4959350 -32426 32509 5029.02 502902 -127 124 1.18 118 -144 2 10134 99045 0.43243 297.43243 148.93243 14893.24324 0.43243 297.43243 148.93243 14893.24338 0.43243 297.43243 148.93243 14893.24300 2020-01-01 2020-01-02 2020-01-01 00:02:24 2020-01-02 03:30:45 2020-01-01 00:02:24.000 2020-01-02 03:30:45.000 144 99045 49594.5 4959450 144 99045 49594.5 4959450 -32425 32510 5030.02 503002 -126 125 2.18 218 -145 2 10135 99046 0.43543 297.43543 148.93543 14893.54354 0.43543 297.43542 148.93543 14893.54354 0.43543 297.43543 148.93543 14893.54300 2020-01-01 2020-01-02 2020-01-01 00:02:25 2020-01-02 03:30:46 2020-01-01 00:02:25.000 2020-01-02 03:30:46.000 145 99046 49595.5 4959550 145 99046 49595.5 4959550 -32424 32511 5031.02 503102 -125 126 3.18 318 -146 2 10136 99047 0.43843 297.43843 148.93843 14893.84384 0.43843 297.43845 148.93844 14893.84427 0.43843 297.43843 148.93843 14893.84300 2020-01-01 2020-01-02 2020-01-01 00:02:26 2020-01-02 03:30:47 2020-01-01 00:02:26.000 2020-01-02 03:30:47.000 146 99047 49596.5 4959650 146 99047 49596.5 4959650 -32423 32512 5032.02 503202 -124 127 4.18 418 +14 2 10004 99914 0.04204 300.04204 150.04204 15154.24624 0.04204 300.04205 150.04204 15154.2463 0.04204 300.04204 150.0420400000001 15154.24604 2020-01-01 2020-01-02 2020-01-01 00:00:14 2020-01-02 03:45:14 2020-01-01 00:00:14.000 2020-01-02 03:45:14.000 14 99914 49964 5046364 14 99914 49964 5046364 -32555 32380 4543.009900990099 458844 -125 126 0.9108910891089109 92 +140 2 10130 99041 0.42042 297.42042 148.92042 14892.04204 0.42042 297.4204 148.92042 14892.04206 0.42042 297.42042 148.9204200000002 14892.04200 2020-01-01 2020-01-02 2020-01-01 00:02:20 2020-01-02 03:30:41 2020-01-01 00:02:20.000 2020-01-02 03:30:41.000 140 99041 49590.5 4959050 140 99041 49590.5 4959050 -32429 32506 5026.02 502602 -124 127 3.3 330 +141 2 10131 99042 0.42342 297.42342 148.92342 14892.34234 0.42342 297.42343 148.92342 14892.34215 0.42342 297.42342 148.92341999999988 14892.34200 2020-01-01 2020-01-02 2020-01-01 00:02:21 2020-01-02 03:30:42 2020-01-01 00:02:21.000 2020-01-02 03:30:42.000 141 99042 49591.5 4959150 141 99042 49591.5 4959150 -32428 32507 5027.02 502702 -128 127 1.74 174 +142 2 10132 99043 0.42642 297.42642 148.92642 14892.64264 0.42642 297.42642 148.92642 14892.64246 0.42642 297.42642 148.92641999999984 14892.64200 2020-01-01 2020-01-02 2020-01-01 00:02:22 2020-01-02 03:30:43 2020-01-01 00:02:22.000 2020-01-02 03:30:43.000 142 99043 49592.5 4959250 142 99043 49592.5 4959250 -32427 32508 5028.02 502802 -128 123 0.18 18 +143 2 10133 99044 0.42942 297.42942 148.92942 14892.94294 0.42942 297.42944 148.92943 14892.94304 0.42942 297.42942 148.92942000000016 14892.94200 2020-01-01 2020-01-02 2020-01-01 00:02:23 2020-01-02 03:30:44 2020-01-01 00:02:23.000 2020-01-02 03:30:44.000 143 99044 49593.5 4959350 143 99044 49593.5 4959350 -32426 32509 5029.02 502902 -127 124 1.18 118 +144 2 10134 99045 0.43243 297.43243 148.93243 14893.24324 0.43243 297.43243 148.93243 14893.24338 0.43243 297.43243 148.93243000000012 14893.24300 2020-01-01 2020-01-02 2020-01-01 00:02:24 2020-01-02 03:30:45 2020-01-01 00:02:24.000 2020-01-02 03:30:45.000 144 99045 49594.5 4959450 144 99045 49594.5 4959450 -32425 32510 5030.02 503002 -126 125 2.18 218 +145 2 10135 99046 0.43543 297.43543 148.93543 14893.54354 0.43543 297.43542 148.93543 14893.54354 0.43543 297.43543 148.93542999999983 14893.54300 2020-01-01 2020-01-02 2020-01-01 00:02:25 2020-01-02 03:30:46 2020-01-01 00:02:25.000 2020-01-02 03:30:46.000 145 99046 49595.5 4959550 145 99046 49595.5 4959550 -32424 32511 5031.02 503102 -125 126 3.18 318 +146 2 10136 99047 0.43843 297.43843 148.93843 14893.84384 0.43843 297.43845 148.93844 14893.84427 0.43843 297.43843 148.9384300000001 14893.84300 2020-01-01 2020-01-02 2020-01-01 00:02:26 2020-01-02 03:30:47 2020-01-01 00:02:26.000 2020-01-02 03:30:47.000 146 99047 49596.5 4959650 146 99047 49596.5 4959650 -32423 32512 5032.02 503202 -124 127 4.18 418 147 2 10137 99048 0.44144 297.44144 148.94144 14894.14414 0.44144 297.44144 148.94143 14894.14392 0.44144 297.44144 148.94144 14894.14400 2020-01-01 2020-01-02 2020-01-01 00:02:27 2020-01-02 03:30:48 2020-01-01 00:02:27.000 2020-01-02 03:30:48.000 147 99048 49597.5 4959750 147 99048 49597.5 4959750 -32422 32513 5033.02 503302 -128 127 2.62 262 -148 2 10138 99049 0.44444 297.44444 148.94444 14894.44444 0.44444 297.44446 148.94444 14894.4445 0.44444 297.44444 148.94444 14894.44400 2020-01-01 2020-01-02 2020-01-01 00:02:28 2020-01-02 03:30:49 2020-01-01 00:02:28.000 2020-01-02 03:30:49.000 148 99049 49598.5 4959850 148 99049 49598.5 4959850 -32421 32514 5034.02 503402 -128 127 1.06 106 -149 2 10139 99050 0.44744 297.44744 148.94744 14894.74474 0.44744 297.44745 148.94744 14894.74485 0.44744 297.44744 148.94744 14894.74400 2020-01-01 2020-01-02 2020-01-01 00:02:29 2020-01-02 03:30:50 2020-01-01 00:02:29.000 2020-01-02 03:30:50.000 149 99050 49599.5 4959950 149 99050 49599.5 4959950 -32420 32515 5035.02 503502 -128 124 -0.5 -50 -15 2 10005 99915 0.04504 300.04504 150.04504 15154.54954 0.04504 300.04504 150.04504 15154.54945 0.04504 300.04504 150.04504 15154.54904 2020-01-01 2020-01-02 2020-01-01 00:00:15 2020-01-02 03:45:15 2020-01-01 00:00:15.000 2020-01-02 03:45:15.000 15 99915 49965 5046465 15 99915 49965 5046465 -32554 32381 4544.009900990099 458945 -124 127 1.9108910891089108 193 -150 2 10140 99051 0.45045 297.45045 148.95045 14895.04504 0.45045 297.45044 148.95045 14895.04501 0.45045 297.45045 148.95045 14895.04500 2020-01-01 2020-01-02 2020-01-01 00:02:30 2020-01-02 03:30:51 2020-01-01 00:02:30.000 2020-01-02 03:30:51.000 150 99051 49600.5 4960050 150 99051 49600.5 4960050 -32419 32516 5036.02 503602 -127 125 0.5 50 -151 2 10141 99052 0.45345 297.45345 148.95345 14895.34534 0.45345 297.45346 148.95345 14895.34574 0.45345 297.45345 148.95345 14895.34500 2020-01-01 2020-01-02 2020-01-01 00:02:31 2020-01-02 03:30:52 2020-01-01 00:02:31.000 2020-01-02 03:30:52.000 151 99052 49601.5 4960150 151 99052 49601.5 4960150 -32418 32517 5037.02 503702 -126 126 1.5 150 -152 2 10142 99053 0.45645 297.45645 148.95645 14895.64564 0.45645 297.45645 148.95645 14895.6454 0.45645 297.45645 148.95645 14895.64500 2020-01-01 2020-01-02 2020-01-01 00:02:32 2020-01-02 03:30:53 2020-01-01 00:02:32.000 2020-01-02 03:30:53.000 152 99053 49602.5 4960250 152 99053 49602.5 4960250 -32417 32518 5038.02 503802 -125 127 2.5 250 -153 2 10143 99054 0.45945 297.45945 148.95945 14895.94594 0.45945 297.45947 148.95946 14895.94601 0.45945 297.45945 148.95945 14895.94500 2020-01-01 2020-01-02 2020-01-01 00:02:33 2020-01-02 03:30:54 2020-01-01 00:02:33.000 2020-01-02 03:30:54.000 153 99054 49603.5 4960350 153 99054 49603.5 4960350 -32416 32519 5039.02 503902 -128 127 0.94 94 -154 2 10144 99055 0.46246 297.46246 148.96246 14896.24624 0.46246 297.46246 148.96246 14896.24633 0.46246 297.46246 148.96246 14896.24600 2020-01-01 2020-01-02 2020-01-01 00:02:34 2020-01-02 03:30:55 2020-01-01 00:02:34.000 2020-01-02 03:30:55.000 154 99055 49604.5 4960450 154 99055 49604.5 4960450 -32415 32520 5040.02 504002 -128 127 -0.62 -62 -155 2 10145 99056 0.46546 297.46546 148.96546 14896.54654 0.46546 297.46545 148.96546 14896.54647 0.46546 297.46546 148.96546 14896.54600 2020-01-01 2020-01-02 2020-01-01 00:02:35 2020-01-02 03:30:56 2020-01-01 00:02:35.000 2020-01-02 03:30:56.000 155 99056 49605.5 4960550 155 99056 49605.5 4960550 -32414 32521 5041.02 504102 -128 123 -2.18 -218 -156 2 10146 99057 0.46846 297.46846 148.96846 14896.84684 0.46846 297.46848 148.96847 14896.84721 0.46846 297.46846 148.96846 14896.84600 2020-01-01 2020-01-02 2020-01-01 00:02:36 2020-01-02 03:30:57 2020-01-01 00:02:36.000 2020-01-02 03:30:57.000 156 99057 49606.5 4960650 156 99057 49606.5 4960650 -32413 32522 5042.02 504202 -127 124 -1.18 -118 -157 2 10147 99058 0.47147 297.47147 148.97147 14897.14714 0.47147 297.47147 148.97146 14897.14687 0.47147 297.47147 148.97147 14897.14700 2020-01-01 2020-01-02 2020-01-01 00:02:37 2020-01-02 03:30:58 2020-01-01 00:02:37.000 2020-01-02 03:30:58.000 157 99058 49607.5 4960750 157 99058 49607.5 4960750 -32412 32523 5043.02 504302 -126 125 -0.18 -18 -158 2 10148 99059 0.47447 297.47447 148.97447 14897.44744 0.47447 297.4745 148.97447 14897.44748 0.47447 297.47447 148.97447 14897.44700 2020-01-01 2020-01-02 2020-01-01 00:02:38 2020-01-02 03:30:59 2020-01-01 00:02:38.000 2020-01-02 03:30:59.000 158 99059 49608.5 4960850 158 99059 49608.5 4960850 -32411 32524 5044.02 504402 -125 126 0.82 82 -159 2 10149 99060 0.47747 297.47747 148.97747 14897.74774 0.47747 297.47748 148.97747 14897.74779 0.47747 297.47747 148.97747 14897.74700 2020-01-01 2020-01-02 2020-01-01 00:02:39 2020-01-02 03:31:00 2020-01-01 00:02:39.000 2020-01-02 03:31:00.000 159 99060 49609.5 4960950 159 99060 49609.5 4960950 -32410 32525 5045.02 504502 -124 127 1.82 182 -16 2 10006 99916 0.04804 300.04804 150.04804 15154.85285 0.04804 300.04803 150.04804 15154.85279 0.04804 300.04804 150.04804 15154.85204 2020-01-01 2020-01-02 2020-01-01 00:00:16 2020-01-02 03:45:16 2020-01-01 00:00:16.000 2020-01-02 03:45:16.000 16 99916 49966 5046566 16 99916 49966 5046566 -32553 32382 4545.009900990099 459046 -128 127 0.37623762376237624 38 -160 2 10150 99061 0.48048 297.48048 148.98048 14898.04804 0.48048 297.48047 148.98048 14898.0481 0.48048 297.48048 148.98048 14898.04800 2020-01-01 2020-01-02 2020-01-01 00:02:40 2020-01-02 03:31:01 2020-01-01 00:02:40.000 2020-01-02 03:31:01.000 160 99061 49610.5 4961050 160 99061 49610.5 4961050 -32409 32526 5046.02 504602 -128 127 0.26 26 -161 2 10151 99062 0.48348 297.48348 148.98348 14898.34834 0.48348 297.4835 148.98348 14898.34868 0.48348 297.48348 148.98348 14898.34800 2020-01-01 2020-01-02 2020-01-01 00:02:41 2020-01-02 03:31:02 2020-01-01 00:02:41.000 2020-01-02 03:31:02.000 161 99062 49611.5 4961150 161 99062 49611.5 4961150 -32408 32527 5047.02 504702 -128 123 -1.3 -130 -162 2 10152 99063 0.48648 297.48648 148.98648 14898.64864 0.48648 297.48648 148.98648 14898.64837 0.48648 297.48648 148.98648 14898.64800 2020-01-01 2020-01-02 2020-01-01 00:02:42 2020-01-02 03:31:03 2020-01-01 00:02:42.000 2020-01-02 03:31:03.000 162 99063 49612.5 4961250 162 99063 49612.5 4961250 -32407 32528 5048.02 504802 -127 124 -0.3 -30 -163 2 10153 99064 0.48948 297.48948 148.98948 14898.94894 0.48948 297.4895 148.98948 14898.94895 0.48948 297.48948 148.98948 14898.94800 2020-01-01 2020-01-02 2020-01-01 00:02:43 2020-01-02 03:31:04 2020-01-01 00:02:43.000 2020-01-02 03:31:04.000 163 99064 49613.5 4961350 163 99064 49613.5 4961350 -32406 32529 5049.02 504902 -126 125 0.7 70 -164 2 10154 99065 0.49249 297.49249 148.99249 14899.24924 0.49249 297.4925 148.99249 14899.24926 0.49249 297.49249 148.99249 14899.24900 2020-01-01 2020-01-02 2020-01-01 00:02:44 2020-01-02 03:31:05 2020-01-01 00:02:44.000 2020-01-02 03:31:05.000 164 99065 49614.5 4961450 164 99065 49614.5 4961450 -32405 32530 5050.02 505002 -125 126 1.7 170 -165 2 10155 99066 0.49549 297.49549 148.99549 14899.54954 0.49549 297.49548 148.99549 14899.54957 0.49549 297.49549 148.99549 14899.54900 2020-01-01 2020-01-02 2020-01-01 00:02:45 2020-01-02 03:31:06 2020-01-01 00:02:45.000 2020-01-02 03:31:06.000 165 99066 49615.5 4961550 165 99066 49615.5 4961550 -32404 32531 5051.02 505102 -124 127 2.7 270 -166 2 10156 99067 0.49849 297.49849 148.99849 14899.84984 0.49849 297.4985 148.9985 14899.85015 0.49849 297.49849 148.99849 14899.84900 2020-01-01 2020-01-02 2020-01-01 00:02:46 2020-01-02 03:31:07 2020-01-01 00:02:46.000 2020-01-02 03:31:07.000 166 99067 49616.5 4961650 166 99067 49616.5 4961650 -32403 32532 5052.02 505202 -128 127 1.14 114 -167 2 10157 99068 0.5015 297.5015 149.0015 14900.15015 0.5015 297.5015 149.00149 14900.14984 0.50150 297.50150 149.00150 14900.15000 2020-01-01 2020-01-02 2020-01-01 00:02:47 2020-01-02 03:31:08 2020-01-01 00:02:47.000 2020-01-02 03:31:08.000 167 99068 49617.5 4961750 167 99068 49617.5 4961750 -32402 32533 5053.02 505302 -128 123 -0.42 -42 -168 2 10158 99069 0.5045 297.5045 149.0045 14900.45045 0.5045 297.50452 149.0045 14900.45042 0.50450 297.50450 149.00450 14900.45000 2020-01-01 2020-01-02 2020-01-01 00:02:48 2020-01-02 03:31:09 2020-01-01 00:02:48.000 2020-01-02 03:31:09.000 168 99069 49618.5 4961850 168 99069 49618.5 4961850 -32401 32534 5054.02 505402 -127 124 0.58 58 -169 2 10159 99070 0.5075 297.5075 149.0075 14900.75075 0.5075 297.5075 149.0075 14900.75073 0.50750 297.50750 149.00750 14900.75000 2020-01-01 2020-01-02 2020-01-01 00:02:49 2020-01-02 03:31:10 2020-01-01 00:02:49.000 2020-01-02 03:31:10.000 169 99070 49619.5 4961950 169 99070 49619.5 4961950 -32400 32535 5055.02 505502 -126 125 1.58 158 +148 2 10138 99049 0.44444 297.44444 148.94444 14894.44444 0.44444 297.44446 148.94444 14894.4445 0.44444 297.44444 148.94443999999996 14894.44400 2020-01-01 2020-01-02 2020-01-01 00:02:28 2020-01-02 03:30:49 2020-01-01 00:02:28.000 2020-01-02 03:30:49.000 148 99049 49598.5 4959850 148 99049 49598.5 4959850 -32421 32514 5034.02 503402 -128 127 1.06 106 +149 2 10139 99050 0.44744 297.44744 148.94744 14894.74474 0.44744 297.44745 148.94744 14894.74485 0.44744 297.44744 148.94743999999994 14894.74400 2020-01-01 2020-01-02 2020-01-01 00:02:29 2020-01-02 03:30:50 2020-01-01 00:02:29.000 2020-01-02 03:30:50.000 149 99050 49599.5 4959950 149 99050 49599.5 4959950 -32420 32515 5035.02 503502 -128 124 -0.5 -50 +15 2 10005 99915 0.04504 300.04504 150.04504 15154.54954 0.04504 300.04504 150.04504 15154.54945 0.04504 300.04504 150.04504000000017 15154.54904 2020-01-01 2020-01-02 2020-01-01 00:00:15 2020-01-02 03:45:15 2020-01-01 00:00:15.000 2020-01-02 03:45:15.000 15 99915 49965 5046465 15 99915 49965 5046465 -32554 32381 4544.009900990099 458945 -124 127 1.9108910891089108 193 +150 2 10140 99051 0.45045 297.45045 148.95045 14895.04504 0.45045 297.45044 148.95045 14895.04501 0.45045 297.45045 148.95045000000013 14895.04500 2020-01-01 2020-01-02 2020-01-01 00:02:30 2020-01-02 03:30:51 2020-01-01 00:02:30.000 2020-01-02 03:30:51.000 150 99051 49600.5 4960050 150 99051 49600.5 4960050 -32419 32516 5036.02 503602 -127 125 0.5 50 +151 2 10141 99052 0.45345 297.45345 148.95345 14895.34534 0.45345 297.45346 148.95345 14895.34574 0.45345 297.45345 148.9534500000002 14895.34500 2020-01-01 2020-01-02 2020-01-01 00:02:31 2020-01-02 03:30:52 2020-01-01 00:02:31.000 2020-01-02 03:30:52.000 151 99052 49601.5 4960150 151 99052 49601.5 4960150 -32418 32517 5037.02 503702 -126 126 1.5 150 +152 2 10142 99053 0.45645 297.45645 148.95645 14895.64564 0.45645 297.45645 148.95645 14895.6454 0.45645 297.45645 148.95644999999985 14895.64500 2020-01-01 2020-01-02 2020-01-01 00:02:32 2020-01-02 03:30:53 2020-01-01 00:02:32.000 2020-01-02 03:30:53.000 152 99053 49602.5 4960250 152 99053 49602.5 4960250 -32417 32518 5038.02 503802 -125 127 2.5 250 +153 2 10143 99054 0.45945 297.45945 148.95945 14895.94594 0.45945 297.45947 148.95946 14895.94601 0.45945 297.45945 148.95945000000015 14895.94500 2020-01-01 2020-01-02 2020-01-01 00:02:33 2020-01-02 03:30:54 2020-01-01 00:02:33.000 2020-01-02 03:30:54.000 153 99054 49603.5 4960350 153 99054 49603.5 4960350 -32416 32519 5039.02 503902 -128 127 0.94 94 +154 2 10144 99055 0.46246 297.46246 148.96246 14896.24624 0.46246 297.46246 148.96246 14896.24633 0.46246 297.46246 148.9624600000001 14896.24600 2020-01-01 2020-01-02 2020-01-01 00:02:34 2020-01-02 03:30:55 2020-01-01 00:02:34.000 2020-01-02 03:30:55.000 154 99055 49604.5 4960450 154 99055 49604.5 4960450 -32415 32520 5040.02 504002 -128 127 -0.62 -62 +155 2 10145 99056 0.46546 297.46546 148.96546 14896.54654 0.46546 297.46545 148.96546 14896.54647 0.46546 297.46546 148.96545999999995 14896.54600 2020-01-01 2020-01-02 2020-01-01 00:02:35 2020-01-02 03:30:56 2020-01-01 00:02:35.000 2020-01-02 03:30:56.000 155 99056 49605.5 4960550 155 99056 49605.5 4960550 -32414 32521 5041.02 504102 -128 123 -2.18 -218 +156 2 10146 99057 0.46846 297.46846 148.96846 14896.84684 0.46846 297.46848 148.96847 14896.84721 0.46846 297.46846 148.96846000000002 14896.84600 2020-01-01 2020-01-02 2020-01-01 00:02:36 2020-01-02 03:30:57 2020-01-01 00:02:36.000 2020-01-02 03:30:57.000 156 99057 49606.5 4960650 156 99057 49606.5 4960650 -32413 32522 5042.02 504202 -127 124 -1.18 -118 +157 2 10147 99058 0.47147 297.47147 148.97147 14897.14714 0.47147 297.47147 148.97146 14897.14687 0.47147 297.47147 148.97147000000024 14897.14700 2020-01-01 2020-01-02 2020-01-01 00:02:37 2020-01-02 03:30:58 2020-01-01 00:02:37.000 2020-01-02 03:30:58.000 157 99058 49607.5 4960750 157 99058 49607.5 4960750 -32412 32523 5043.02 504302 -126 125 -0.18 -18 +158 2 10148 99059 0.47447 297.47447 148.97447 14897.44744 0.47447 297.4745 148.97447 14897.44748 0.47447 297.47447 148.97446999999985 14897.44700 2020-01-01 2020-01-02 2020-01-01 00:02:38 2020-01-02 03:30:59 2020-01-01 00:02:38.000 2020-01-02 03:30:59.000 158 99059 49608.5 4960850 158 99059 49608.5 4960850 -32411 32524 5044.02 504402 -125 126 0.82 82 +159 2 10149 99060 0.47747 297.47747 148.97747 14897.74774 0.47747 297.47748 148.97747 14897.74779 0.47747 297.47747 148.97746999999987 14897.74700 2020-01-01 2020-01-02 2020-01-01 00:02:39 2020-01-02 03:31:00 2020-01-01 00:02:39.000 2020-01-02 03:31:00.000 159 99060 49609.5 4960950 159 99060 49609.5 4960950 -32410 32525 5045.02 504502 -124 127 1.82 182 +16 2 10006 99916 0.04804 300.04804 150.04804 15154.85285 0.04804 300.04803 150.04804 15154.85279 0.04804 300.04804 150.04803999999982 15154.85204 2020-01-01 2020-01-02 2020-01-01 00:00:16 2020-01-02 03:45:16 2020-01-01 00:00:16.000 2020-01-02 03:45:16.000 16 99916 49966 5046566 16 99916 49966 5046566 -32553 32382 4545.009900990099 459046 -128 127 0.37623762376237624 38 +160 2 10150 99061 0.48048 297.48048 148.98048 14898.04804 0.48048 297.48047 148.98048 14898.0481 0.48048 297.48048 148.9804800000001 14898.04800 2020-01-01 2020-01-02 2020-01-01 00:02:40 2020-01-02 03:31:01 2020-01-01 00:02:40.000 2020-01-02 03:31:01.000 160 99061 49610.5 4961050 160 99061 49610.5 4961050 -32409 32526 5046.02 504602 -128 127 0.26 26 +161 2 10151 99062 0.48348 297.48348 148.98348 14898.34834 0.48348 297.4835 148.98348 14898.34868 0.48348 297.48348 148.98348000000016 14898.34800 2020-01-01 2020-01-02 2020-01-01 00:02:41 2020-01-02 03:31:02 2020-01-01 00:02:41.000 2020-01-02 03:31:02.000 161 99062 49611.5 4961150 161 99062 49611.5 4961150 -32408 32527 5047.02 504702 -128 123 -1.3 -130 +162 2 10152 99063 0.48648 297.48648 148.98648 14898.64864 0.48648 297.48648 148.98648 14898.64837 0.48648 297.48648 148.98647999999974 14898.64800 2020-01-01 2020-01-02 2020-01-01 00:02:42 2020-01-02 03:31:03 2020-01-01 00:02:42.000 2020-01-02 03:31:03.000 162 99063 49612.5 4961250 162 99063 49612.5 4961250 -32407 32528 5048.02 504802 -127 124 -0.3 -30 +163 2 10153 99064 0.48948 297.48948 148.98948 14898.94894 0.48948 297.4895 148.98948 14898.94895 0.48948 297.48948 148.98948000000013 14898.94800 2020-01-01 2020-01-02 2020-01-01 00:02:43 2020-01-02 03:31:04 2020-01-01 00:02:43.000 2020-01-02 03:31:04.000 163 99064 49613.5 4961350 163 99064 49613.5 4961350 -32406 32529 5049.02 504902 -126 125 0.7 70 +164 2 10154 99065 0.49249 297.49249 148.99249 14899.24924 0.49249 297.4925 148.99249 14899.24926 0.49249 297.49249 148.99249000000006 14899.24900 2020-01-01 2020-01-02 2020-01-01 00:02:44 2020-01-02 03:31:05 2020-01-01 00:02:44.000 2020-01-02 03:31:05.000 164 99065 49614.5 4961450 164 99065 49614.5 4961450 -32405 32530 5050.02 505002 -125 126 1.7 170 +165 2 10155 99066 0.49549 297.49549 148.99549 14899.54954 0.49549 297.49548 148.99549 14899.54957 0.49549 297.49549 148.99548999999988 14899.54900 2020-01-01 2020-01-02 2020-01-01 00:02:45 2020-01-02 03:31:06 2020-01-01 00:02:45.000 2020-01-02 03:31:06.000 165 99066 49615.5 4961550 165 99066 49615.5 4961550 -32404 32531 5051.02 505102 -124 127 2.7 270 +166 2 10156 99067 0.49849 297.49849 148.99849 14899.84984 0.49849 297.4985 148.9985 14899.85015 0.49849 297.49849 148.99848999999998 14899.84900 2020-01-01 2020-01-02 2020-01-01 00:02:46 2020-01-02 03:31:07 2020-01-01 00:02:46.000 2020-01-02 03:31:07.000 166 99067 49616.5 4961650 166 99067 49616.5 4961650 -32403 32532 5052.02 505202 -128 127 1.14 114 +167 2 10157 99068 0.5015 297.5015 149.0015 14900.15015 0.5015 297.5015 149.00149 14900.14984 0.50150 297.50150 149.00150000000016 14900.15000 2020-01-01 2020-01-02 2020-01-01 00:02:47 2020-01-02 03:31:08 2020-01-01 00:02:47.000 2020-01-02 03:31:08.000 167 99068 49617.5 4961750 167 99068 49617.5 4961750 -32402 32533 5053.02 505302 -128 123 -0.42 -42 +168 2 10158 99069 0.5045 297.5045 149.0045 14900.45045 0.5045 297.50452 149.0045 14900.45042 0.50450 297.50450 149.00449999999978 14900.45000 2020-01-01 2020-01-02 2020-01-01 00:02:48 2020-01-02 03:31:09 2020-01-01 00:02:48.000 2020-01-02 03:31:09.000 168 99069 49618.5 4961850 168 99069 49618.5 4961850 -32401 32534 5054.02 505402 -127 124 0.58 58 +169 2 10159 99070 0.5075 297.5075 149.0075 14900.75075 0.5075 297.5075 149.0075 14900.75073 0.50750 297.50750 149.00749999999985 14900.75000 2020-01-01 2020-01-02 2020-01-01 00:02:49 2020-01-02 03:31:10 2020-01-01 00:02:49.000 2020-01-02 03:31:10.000 169 99070 49619.5 4961950 169 99070 49619.5 4961950 -32400 32535 5055.02 505502 -126 125 1.58 158 17 2 10007 99917 0.05105 300.05105 150.05105 15155.15615 0.05105 300.05106 150.05105 15155.15638 0.05105 300.05105 150.05105 15155.15605 2020-01-01 2020-01-02 2020-01-01 00:00:17 2020-01-02 03:45:17 2020-01-01 00:00:17.000 2020-01-02 03:45:17.000 17 99917 49967 5046667 17 99917 49967 5046667 -32552 32383 4546.009900990099 459147 -128 127 -1.1584158415841583 -117 -170 2 10160 99071 0.51051 297.51051 149.01051 14901.05105 0.51051 297.5105 149.01051 14901.05104 0.51051 297.51051 149.01051 14901.05100 2020-01-01 2020-01-02 2020-01-01 00:02:50 2020-01-02 03:31:11 2020-01-01 00:02:50.000 2020-01-02 03:31:11.000 170 99071 49620.5 4962050 170 99071 49620.5 4962050 -32399 32536 5056.02 505602 -125 126 2.58 258 -171 2 10161 99072 0.51351 297.51351 149.01351 14901.35135 0.51351 297.51352 149.01351 14901.35162 0.51351 297.51351 149.01351 14901.35100 2020-01-01 2020-01-02 2020-01-01 00:02:51 2020-01-02 03:31:12 2020-01-01 00:02:51.000 2020-01-02 03:31:12.000 171 99072 49621.5 4962150 171 99072 49621.5 4962150 -32398 32537 5057.02 505702 -124 127 3.58 358 -172 2 10162 99073 0.51651 297.51651 149.01651 14901.65165 0.51651 297.5165 149.01651 14901.65131 0.51651 297.51651 149.01651 14901.65100 2020-01-01 2020-01-02 2020-01-01 00:02:52 2020-01-02 03:31:13 2020-01-01 00:02:52.000 2020-01-02 03:31:13.000 172 99073 49622.5 4962250 172 99073 49622.5 4962250 -32397 32538 5058.02 505802 -128 127 2.02 202 -173 2 10163 99074 0.51951 297.51951 149.01951 14901.95195 0.51951 297.51953 149.01951 14901.95189 0.51951 297.51951 149.01951 14901.95100 2020-01-01 2020-01-02 2020-01-01 00:02:53 2020-01-02 03:31:14 2020-01-01 00:02:53.000 2020-01-02 03:31:14.000 173 99074 49623.5 4962350 173 99074 49623.5 4962350 -32396 32539 5059.02 505902 -128 127 0.46 46 -174 2 10164 99075 0.52252 297.52252 149.02252 14902.25225 0.52252 297.52252 149.02252 14902.2522 0.52252 297.52252 149.02252 14902.25200 2020-01-01 2020-01-02 2020-01-01 00:02:54 2020-01-02 03:31:15 2020-01-01 00:02:54.000 2020-01-02 03:31:15.000 174 99075 49624.5 4962450 174 99075 49624.5 4962450 -32395 32540 5060.02 506002 -128 124 -1.1 -110 -175 2 10165 99076 0.52552 297.52552 149.02552 14902.55255 0.52552 297.5255 149.02552 14902.55251 0.52552 297.52552 149.02552 14902.55200 2020-01-01 2020-01-02 2020-01-01 00:02:55 2020-01-02 03:31:16 2020-01-01 00:02:55.000 2020-01-02 03:31:16.000 175 99076 49625.5 4962550 175 99076 49625.5 4962550 -32394 32541 5061.02 506102 -127 125 -0.1 -10 -176 2 10166 99077 0.52852 297.52852 149.02852 14902.85285 0.52852 297.52853 149.02853 14902.85312 0.52852 297.52852 149.02852 14902.85200 2020-01-01 2020-01-02 2020-01-01 00:02:56 2020-01-02 03:31:17 2020-01-01 00:02:56.000 2020-01-02 03:31:17.000 176 99077 49626.5 4962650 176 99077 49626.5 4962650 -32393 32542 5062.02 506202 -126 126 0.9 90 -177 2 10167 99078 0.53153 297.53153 149.03153 14903.15315 0.53153 297.53152 149.03152 14903.15278 0.53153 297.53153 149.03153 14903.15300 2020-01-01 2020-01-02 2020-01-01 00:02:57 2020-01-02 03:31:18 2020-01-01 00:02:57.000 2020-01-02 03:31:18.000 177 99078 49627.5 4962750 177 99078 49627.5 4962750 -32392 32543 5063.02 506302 -125 127 1.9 190 -178 2 10168 99079 0.53453 297.53453 149.03453 14903.45345 0.53453 297.53455 149.03453 14903.45352 0.53453 297.53453 149.03453 14903.45300 2020-01-01 2020-01-02 2020-01-01 00:02:58 2020-01-02 03:31:19 2020-01-01 00:02:58.000 2020-01-02 03:31:19.000 178 99079 49628.5 4962850 178 99079 49628.5 4962850 -32391 32544 5064.02 506402 -128 127 0.34 34 -179 2 10169 99080 0.53753 297.53753 149.03753 14903.75375 0.53753 297.53754 149.03753 14903.75366 0.53753 297.53753 149.03753 14903.75300 2020-01-01 2020-01-02 2020-01-01 00:02:59 2020-01-02 03:31:20 2020-01-01 00:02:59.000 2020-01-02 03:31:20.000 179 99080 49629.5 4962950 179 99080 49629.5 4962950 -32390 32545 5065.02 506502 -128 127 -1.22 -122 -18 2 10008 99918 0.05405 300.05405 150.05405 15155.45945 0.05405 300.05405 150.05404 15155.45903 0.05405 300.05405 150.05405 15155.45905 2020-01-01 2020-01-02 2020-01-01 00:00:18 2020-01-02 03:45:18 2020-01-01 00:00:18.000 2020-01-02 03:45:18.000 18 99918 49968 5046768 18 99918 49968 5046768 -32551 32384 4547.009900990099 459248 -128 124 -2.6930693069306932 -272 +170 2 10160 99071 0.51051 297.51051 149.01051 14901.05105 0.51051 297.5105 149.01051 14901.05104 0.51051 297.51051 149.01051000000004 14901.05100 2020-01-01 2020-01-02 2020-01-01 00:02:50 2020-01-02 03:31:11 2020-01-01 00:02:50.000 2020-01-02 03:31:11.000 170 99071 49620.5 4962050 170 99071 49620.5 4962050 -32399 32536 5056.02 505602 -125 126 2.58 258 +171 2 10161 99072 0.51351 297.51351 149.01351 14901.35135 0.51351 297.51352 149.01351 14901.35162 0.51351 297.51351 149.01351000000005 14901.35100 2020-01-01 2020-01-02 2020-01-01 00:02:51 2020-01-02 03:31:12 2020-01-01 00:02:51.000 2020-01-02 03:31:12.000 171 99072 49621.5 4962150 171 99072 49621.5 4962150 -32398 32537 5057.02 505702 -124 127 3.58 358 +172 2 10162 99073 0.51651 297.51651 149.01651 14901.65165 0.51651 297.5165 149.01651 14901.65131 0.51651 297.51651 149.01650999999998 14901.65100 2020-01-01 2020-01-02 2020-01-01 00:02:52 2020-01-02 03:31:13 2020-01-01 00:02:52.000 2020-01-02 03:31:13.000 172 99073 49622.5 4962250 172 99073 49622.5 4962250 -32397 32538 5058.02 505802 -128 127 2.02 202 +173 2 10163 99074 0.51951 297.51951 149.01951 14901.95195 0.51951 297.51953 149.01951 14901.95189 0.51951 297.51951 149.01951000000005 14901.95100 2020-01-01 2020-01-02 2020-01-01 00:02:53 2020-01-02 03:31:14 2020-01-01 00:02:53.000 2020-01-02 03:31:14.000 173 99074 49623.5 4962350 173 99074 49623.5 4962350 -32396 32539 5059.02 505902 -128 127 0.46 46 +174 2 10164 99075 0.52252 297.52252 149.02252 14902.25225 0.52252 297.52252 149.02252 14902.2522 0.52252 297.52252 149.02252000000024 14902.25200 2020-01-01 2020-01-02 2020-01-01 00:02:54 2020-01-02 03:31:15 2020-01-01 00:02:54.000 2020-01-02 03:31:15.000 174 99075 49624.5 4962450 174 99075 49624.5 4962450 -32395 32540 5060.02 506002 -128 124 -1.1 -110 +175 2 10165 99076 0.52552 297.52552 149.02552 14902.55255 0.52552 297.5255 149.02552 14902.55251 0.52552 297.52552 149.02551999999986 14902.55200 2020-01-01 2020-01-02 2020-01-01 00:02:55 2020-01-02 03:31:16 2020-01-01 00:02:55.000 2020-01-02 03:31:16.000 175 99076 49625.5 4962550 175 99076 49625.5 4962550 -32394 32541 5061.02 506102 -127 125 -0.1 -10 +176 2 10166 99077 0.52852 297.52852 149.02852 14902.85285 0.52852 297.52853 149.02853 14902.85312 0.52852 297.52852 149.0285199999999 14902.85200 2020-01-01 2020-01-02 2020-01-01 00:02:56 2020-01-02 03:31:17 2020-01-01 00:02:56.000 2020-01-02 03:31:17.000 176 99077 49626.5 4962650 176 99077 49626.5 4962650 -32393 32542 5062.02 506202 -126 126 0.9 90 +177 2 10167 99078 0.53153 297.53153 149.03153 14903.15315 0.53153 297.53152 149.03152 14903.15278 0.53153 297.53153 149.03153000000015 14903.15300 2020-01-01 2020-01-02 2020-01-01 00:02:57 2020-01-02 03:31:18 2020-01-01 00:02:57.000 2020-01-02 03:31:18.000 177 99078 49627.5 4962750 177 99078 49627.5 4962750 -32392 32543 5063.02 506302 -125 127 1.9 190 +178 2 10168 99079 0.53453 297.53453 149.03453 14903.45345 0.53453 297.53455 149.03453 14903.45352 0.53453 297.53453 149.03453000000016 14903.45300 2020-01-01 2020-01-02 2020-01-01 00:02:58 2020-01-02 03:31:19 2020-01-01 00:02:58.000 2020-01-02 03:31:19.000 178 99079 49628.5 4962850 178 99079 49628.5 4962850 -32391 32544 5064.02 506402 -128 127 0.34 34 +179 2 10169 99080 0.53753 297.53753 149.03753 14903.75375 0.53753 297.53754 149.03753 14903.75366 0.53753 297.53753 149.0375299999998 14903.75300 2020-01-01 2020-01-02 2020-01-01 00:02:59 2020-01-02 03:31:20 2020-01-01 00:02:59.000 2020-01-02 03:31:20.000 179 99080 49629.5 4962950 179 99080 49629.5 4962950 -32390 32545 5065.02 506502 -128 127 -1.22 -122 +18 2 10008 99918 0.05405 300.05405 150.05405 15155.45945 0.05405 300.05405 150.05404 15155.45903 0.05405 300.05405 150.05405000000007 15155.45905 2020-01-01 2020-01-02 2020-01-01 00:00:18 2020-01-02 03:45:18 2020-01-01 00:00:18.000 2020-01-02 03:45:18.000 18 99918 49968 5046768 18 99918 49968 5046768 -32551 32384 4547.009900990099 459248 -128 124 -2.6930693069306932 -272 180 2 10170 99081 0.54054 297.54054 149.04054 14904.05405 0.54054 297.54053 149.04053 14904.05398 0.54054 297.54054 149.04054 14904.05400 2020-01-01 2020-01-02 2020-01-01 00:03:00 2020-01-02 03:31:21 2020-01-01 00:03:00.000 2020-01-02 03:31:21.000 180 99081 49630.5 4963050 180 99081 49630.5 4963050 -32389 32546 5066.02 506602 -128 123 -2.78 -278 -181 2 10171 99082 0.54354 297.54354 149.04354 14904.35435 0.54354 297.54355 149.04354 14904.35459 0.54354 297.54354 149.04354 14904.35400 2020-01-01 2020-01-02 2020-01-01 00:03:01 2020-01-02 03:31:22 2020-01-01 00:03:01.000 2020-01-02 03:31:22.000 181 99082 49631.5 4963150 181 99082 49631.5 4963150 -32388 32547 5067.02 506702 -127 124 -1.78 -178 -182 2 10172 99083 0.54654 297.54654 149.04654 14904.65465 0.54654 297.54654 149.04654 14904.65425 0.54654 297.54654 149.04654 14904.65400 2020-01-01 2020-01-02 2020-01-01 00:03:02 2020-01-02 03:31:23 2020-01-01 00:03:02.000 2020-01-02 03:31:23.000 182 99083 49632.5 4963250 182 99083 49632.5 4963250 -32387 32548 5068.02 506802 -126 125 -0.78 -78 -183 2 10173 99084 0.54954 297.54954 149.04954 14904.95495 0.54954 297.54956 149.04954 14904.95498 0.54954 297.54954 149.04954 14904.95400 2020-01-01 2020-01-02 2020-01-01 00:03:03 2020-01-02 03:31:24 2020-01-01 00:03:03.000 2020-01-02 03:31:24.000 183 99084 49633.5 4963350 183 99084 49633.5 4963350 -32386 32549 5069.02 506902 -125 126 0.22 22 -184 2 10174 99085 0.55255 297.55255 149.05255 14905.25525 0.55255 297.55255 149.05255 14905.25514 0.55255 297.55255 149.05255 14905.25500 2020-01-01 2020-01-02 2020-01-01 00:03:04 2020-01-02 03:31:25 2020-01-01 00:03:04.000 2020-01-02 03:31:25.000 184 99085 49634.5 4963450 184 99085 49634.5 4963450 -32385 32550 5070.02 507002 -124 127 1.22 122 -185 2 10175 99086 0.55555 297.55555 149.05555 14905.55555 0.55555 297.55554 149.05555 14905.55549 0.55555 297.55555 149.05555 14905.55500 2020-01-01 2020-01-02 2020-01-01 00:03:05 2020-01-02 03:31:26 2020-01-01 00:03:05.000 2020-01-02 03:31:26.000 185 99086 49635.5 4963550 185 99086 49635.5 4963550 -32384 32551 5071.02 507102 -128 127 -0.34 -34 -186 2 10176 99087 0.55855 297.55855 149.05855 14905.85585 0.55855 297.55856 149.05856 14905.85607 0.55855 297.55855 149.05855 14905.85500 2020-01-01 2020-01-02 2020-01-01 00:03:06 2020-01-02 03:31:27 2020-01-01 00:03:06.000 2020-01-02 03:31:27.000 186 99087 49636.5 4963650 186 99087 49636.5 4963650 -32383 32552 5072.02 507202 -128 123 -1.9 -190 -187 2 10177 99088 0.56156 297.56156 149.06156 14906.15615 0.56156 297.56155 149.06155 14906.15572 0.56156 297.56156 149.06156 14906.15600 2020-01-01 2020-01-02 2020-01-01 00:03:07 2020-01-02 03:31:28 2020-01-01 00:03:07.000 2020-01-02 03:31:28.000 187 99088 49637.5 4963750 187 99088 49637.5 4963750 -32382 32553 5073.02 507302 -127 124 -0.9 -90 -188 2 10178 99089 0.56456 297.56456 149.06456 14906.45645 0.56456 297.56458 149.06456 14906.45645 0.56456 297.56456 149.06456 14906.45600 2020-01-01 2020-01-02 2020-01-01 00:03:08 2020-01-02 03:31:29 2020-01-01 00:03:08.000 2020-01-02 03:31:29.000 188 99089 49638.5 4963850 188 99089 49638.5 4963850 -32381 32554 5074.02 507402 -126 125 0.1 10 +181 2 10171 99082 0.54354 297.54354 149.04354 14904.35435 0.54354 297.54355 149.04354 14904.35459 0.54354 297.54354 149.04354000000006 14904.35400 2020-01-01 2020-01-02 2020-01-01 00:03:01 2020-01-02 03:31:22 2020-01-01 00:03:01.000 2020-01-02 03:31:22.000 181 99082 49631.5 4963150 181 99082 49631.5 4963150 -32388 32547 5067.02 506702 -127 124 -1.78 -178 +182 2 10172 99083 0.54654 297.54654 149.04654 14904.65465 0.54654 297.54654 149.04654 14904.65425 0.54654 297.54654 149.0465399999999 14904.65400 2020-01-01 2020-01-02 2020-01-01 00:03:02 2020-01-02 03:31:23 2020-01-01 00:03:02.000 2020-01-02 03:31:23.000 182 99083 49632.5 4963250 182 99083 49632.5 4963250 -32387 32548 5068.02 506802 -126 125 -0.78 -78 +183 2 10173 99084 0.54954 297.54954 149.04954 14904.95495 0.54954 297.54956 149.04954 14904.95498 0.54954 297.54954 149.04953999999998 14904.95400 2020-01-01 2020-01-02 2020-01-01 00:03:03 2020-01-02 03:31:24 2020-01-01 00:03:03.000 2020-01-02 03:31:24.000 183 99084 49633.5 4963350 183 99084 49633.5 4963350 -32386 32549 5069.02 506902 -125 126 0.22 22 +184 2 10174 99085 0.55255 297.55255 149.05255 14905.25525 0.55255 297.55255 149.05255 14905.25514 0.55255 297.55255 149.05255000000017 14905.25500 2020-01-01 2020-01-02 2020-01-01 00:03:04 2020-01-02 03:31:25 2020-01-01 00:03:04.000 2020-01-02 03:31:25.000 184 99085 49634.5 4963450 184 99085 49634.5 4963450 -32385 32550 5070.02 507002 -124 127 1.22 122 +185 2 10175 99086 0.55555 297.55555 149.05555 14905.55555 0.55555 297.55554 149.05555 14905.55549 0.55555 297.55555 149.05554999999978 14905.55500 2020-01-01 2020-01-02 2020-01-01 00:03:05 2020-01-02 03:31:26 2020-01-01 00:03:05.000 2020-01-02 03:31:26.000 185 99086 49635.5 4963550 185 99086 49635.5 4963550 -32384 32551 5071.02 507102 -128 127 -0.34 -34 +186 2 10176 99087 0.55855 297.55855 149.05855 14905.85585 0.55855 297.55856 149.05856 14905.85607 0.55855 297.55855 149.05854999999988 14905.85500 2020-01-01 2020-01-02 2020-01-01 00:03:06 2020-01-02 03:31:27 2020-01-01 00:03:06.000 2020-01-02 03:31:27.000 186 99087 49636.5 4963650 186 99087 49636.5 4963650 -32383 32552 5072.02 507202 -128 123 -1.9 -190 +187 2 10177 99088 0.56156 297.56156 149.06156 14906.15615 0.56156 297.56155 149.06155 14906.15572 0.56156 297.56156 149.06156000000007 14906.15600 2020-01-01 2020-01-02 2020-01-01 00:03:07 2020-01-02 03:31:28 2020-01-01 00:03:07.000 2020-01-02 03:31:28.000 187 99088 49637.5 4963750 187 99088 49637.5 4963750 -32382 32553 5073.02 507302 -127 124 -0.9 -90 +188 2 10178 99089 0.56456 297.56456 149.06456 14906.45645 0.56456 297.56458 149.06456 14906.45645 0.56456 297.56456 149.06456000000009 14906.45600 2020-01-01 2020-01-02 2020-01-01 00:03:08 2020-01-02 03:31:29 2020-01-01 00:03:08.000 2020-01-02 03:31:29.000 188 99089 49638.5 4963850 188 99089 49638.5 4963850 -32381 32554 5074.02 507402 -126 125 0.1 10 189 2 10179 99090 0.56756 297.56756 149.06756 14906.75675 0.56756 297.56757 149.06756 14906.75661 0.56756 297.56756 149.06756 14906.75600 2020-01-01 2020-01-02 2020-01-01 00:03:09 2020-01-02 03:31:30 2020-01-01 00:03:09.000 2020-01-02 03:31:30.000 189 99090 49639.5 4963950 189 99090 49639.5 4963950 -32380 32555 5075.02 507502 -125 126 1.1 110 19 2 10009 99919 0.05705 300.05705 150.05705 15155.76276 0.05705 300.05707 150.05705 15155.76279 0.05705 300.05705 150.05705 15155.76205 2020-01-01 2020-01-02 2020-01-01 00:00:19 2020-01-02 03:45:19 2020-01-01 00:00:19.000 2020-01-02 03:45:19.000 19 99919 49969 5046869 19 99919 49969 5046869 -32550 32385 4548.009900990099 459349 -127 125 -1.693069306930693 -171 -190 2 10180 99091 0.57057 297.57057 149.07057 14907.05705 0.57057 297.57056 149.07056 14907.05695 0.57057 297.57057 149.07057 14907.05700 2020-01-01 2020-01-02 2020-01-01 00:03:10 2020-01-02 03:31:31 2020-01-01 00:03:10.000 2020-01-02 03:31:31.000 190 99091 49640.5 4964050 190 99091 49640.5 4964050 -32379 32556 5076.02 507602 -124 127 2.1 210 +190 2 10180 99091 0.57057 297.57057 149.07057 14907.05705 0.57057 297.57056 149.07056 14907.05695 0.57057 297.57057 149.0705699999999 14907.05700 2020-01-01 2020-01-02 2020-01-01 00:03:10 2020-01-02 03:31:31 2020-01-01 00:03:10.000 2020-01-02 03:31:31.000 190 99091 49640.5 4964050 190 99091 49640.5 4964050 -32379 32556 5076.02 507602 -124 127 2.1 210 191 2 10181 99092 0.57357 297.57357 149.07357 14907.35735 0.57357 297.57358 149.07357 14907.35753 0.57357 297.57357 149.07357 14907.35700 2020-01-01 2020-01-02 2020-01-01 00:03:11 2020-01-02 03:31:32 2020-01-01 00:03:11.000 2020-01-02 03:31:32.000 191 99092 49641.5 4964150 191 99092 49641.5 4964150 -32378 32557 5077.02 507702 -128 127 0.54 54 -192 2 10182 99093 0.57657 297.57657 149.07657 14907.65765 0.57657 297.57657 149.07657 14907.65784 0.57657 297.57657 149.07657 14907.65700 2020-01-01 2020-01-02 2020-01-01 00:03:12 2020-01-02 03:31:33 2020-01-01 00:03:12.000 2020-01-02 03:31:33.000 192 99093 49642.5 4964250 192 99093 49642.5 4964250 -32377 32558 5078.02 507802 -128 123 -1.02 -102 -193 2 10183 99094 0.57957 297.57957 149.07957 14907.95795 0.57957 297.5796 149.07957 14907.95793 0.57957 297.57957 149.07957 14907.95700 2020-01-01 2020-01-02 2020-01-01 00:03:13 2020-01-02 03:31:34 2020-01-01 00:03:13.000 2020-01-02 03:31:34.000 193 99094 49643.5 4964350 193 99094 49643.5 4964350 -32376 32559 5079.02 507902 -127 124 -0.02 -2 -194 2 10184 99095 0.58258 297.58258 149.08258 14908.25825 0.58258 297.58258 149.08258 14908.25811 0.58258 297.58258 149.08258 14908.25800 2020-01-01 2020-01-02 2020-01-01 00:03:14 2020-01-02 03:31:35 2020-01-01 00:03:14.000 2020-01-02 03:31:35.000 194 99095 49644.5 4964450 194 99095 49644.5 4964450 -32375 32560 5080.02 508002 -126 125 0.98 98 -195 2 10185 99096 0.58558 297.58558 149.08558 14908.55855 0.58558 297.58557 149.08558 14908.55842 0.58558 297.58558 149.08558 14908.55800 2020-01-01 2020-01-02 2020-01-01 00:03:15 2020-01-02 03:31:36 2020-01-01 00:03:15.000 2020-01-02 03:31:36.000 195 99096 49645.5 4964550 195 99096 49645.5 4964550 -32374 32561 5081.02 508102 -125 126 1.98 198 -196 2 10186 99097 0.58858 297.58858 149.08858 14908.85885 0.58858 297.5886 149.08859 14908.859 0.58858 297.58858 149.08858 14908.85800 2020-01-01 2020-01-02 2020-01-01 00:03:16 2020-01-02 03:31:37 2020-01-01 00:03:16.000 2020-01-02 03:31:37.000 196 99097 49646.5 4964650 196 99097 49646.5 4964650 -32373 32562 5082.02 508202 -124 127 2.98 298 +192 2 10182 99093 0.57657 297.57657 149.07657 14907.65765 0.57657 297.57657 149.07657 14907.65784 0.57657 297.57657 149.07656999999992 14907.65700 2020-01-01 2020-01-02 2020-01-01 00:03:12 2020-01-02 03:31:33 2020-01-01 00:03:12.000 2020-01-02 03:31:33.000 192 99093 49642.5 4964250 192 99093 49642.5 4964250 -32377 32558 5078.02 507802 -128 123 -1.02 -102 +193 2 10183 99094 0.57957 297.57957 149.07957 14907.95795 0.57957 297.5796 149.07957 14907.95793 0.57957 297.57957 149.0795699999999 14907.95700 2020-01-01 2020-01-02 2020-01-01 00:03:13 2020-01-02 03:31:34 2020-01-01 00:03:13.000 2020-01-02 03:31:34.000 193 99094 49643.5 4964350 193 99094 49643.5 4964350 -32376 32559 5079.02 507902 -127 124 -0.02 -2 +194 2 10184 99095 0.58258 297.58258 149.08258 14908.25825 0.58258 297.58258 149.08258 14908.25811 0.58258 297.58258 149.08258000000015 14908.25800 2020-01-01 2020-01-02 2020-01-01 00:03:14 2020-01-02 03:31:35 2020-01-01 00:03:14.000 2020-01-02 03:31:35.000 194 99095 49644.5 4964450 194 99095 49644.5 4964450 -32375 32560 5080.02 508002 -126 125 0.98 98 +195 2 10185 99096 0.58558 297.58558 149.08558 14908.55855 0.58558 297.58557 149.08558 14908.55842 0.58558 297.58558 149.0855800000002 14908.55800 2020-01-01 2020-01-02 2020-01-01 00:03:15 2020-01-02 03:31:36 2020-01-01 00:03:15.000 2020-01-02 03:31:36.000 195 99096 49645.5 4964550 195 99096 49645.5 4964550 -32374 32561 5081.02 508102 -125 126 1.98 198 +196 2 10186 99097 0.58858 297.58858 149.08858 14908.85885 0.58858 297.5886 149.08859 14908.859 0.58858 297.58858 149.08857999999984 14908.85800 2020-01-01 2020-01-02 2020-01-01 00:03:16 2020-01-02 03:31:37 2020-01-01 00:03:16.000 2020-01-02 03:31:37.000 196 99097 49646.5 4964650 196 99097 49646.5 4964650 -32373 32562 5082.02 508202 -124 127 2.98 298 197 2 10187 99098 0.59159 297.59159 149.09159 14909.15915 0.59159 297.59158 149.09159 14909.15931 0.59159 297.59159 149.09159 14909.15900 2020-01-01 2020-01-02 2020-01-01 00:03:17 2020-01-02 03:31:38 2020-01-01 00:03:17.000 2020-01-02 03:31:38.000 197 99098 49647.5 4964750 197 99098 49647.5 4964750 -32372 32563 5083.02 508302 -128 127 1.42 142 -198 2 10188 99099 0.59459 297.59459 149.09459 14909.45945 0.59459 297.5946 149.09459 14909.4594 0.59459 297.59459 149.09459 14909.45900 2020-01-01 2020-01-02 2020-01-01 00:03:18 2020-01-02 03:31:39 2020-01-01 00:03:18.000 2020-01-02 03:31:39.000 198 99099 49648.5 4964850 198 99099 49648.5 4964850 -32371 32564 5084.02 508402 -128 127 -0.14 -14 -199 2 10189 99100 0.59759 297.59759 149.09759 14909.75975 0.59759 297.5976 149.09759 14909.75958 0.59759 297.59759 149.09759 14909.75900 2020-01-01 2020-01-02 2020-01-01 00:03:19 2020-01-02 03:31:40 2020-01-01 00:03:19.000 2020-01-02 03:31:40.000 199 99100 49649.5 4964950 199 99100 49649.5 4964950 -32370 32565 5085.02 508502 -128 124 -1.7 -170 -2 2 1001 9992 0.006 300.006 150.006 15150.6066 0.006 300.006 150.006 15150.6069 0.00600 300.00600 150.00600 15150.60600 2020-01-01 2020-01-02 2020-01-01 00:00:02 2020-01-02 03:45:02 2020-01-01 00:00:02.000 2020-01-02 03:45:02.000 2 99902 49952 5045152 2 99902 49952 5045152 -32567 32368 4531.009900990099 457632 -125 126 -0.9504950495049505 -96 -20 2 10010 99920 0.06006 300.06006 150.06006 15156.06606 0.06006 300.06006 150.06005 15156.06593 0.06006 300.06006 150.06006 15156.06606 2020-01-01 2020-01-02 2020-01-01 00:00:20 2020-01-02 03:45:20 2020-01-01 00:00:20.000 2020-01-02 03:45:20.000 20 99920 49970 5046970 20 99920 49970 5046970 -32549 32386 4549.009900990099 459450 -126 126 -0.693069306930693 -70 -200 2 10190 99101 0.6006 297.6006 149.1006 14910.06006 0.6006 297.6006 149.10059 14910.0599 0.60060 297.60060 149.10060 14910.06000 2020-01-01 2020-01-02 2020-01-01 00:03:20 2020-01-02 03:31:41 2020-01-01 00:03:20.000 2020-01-02 03:31:41.000 200 99101 49650.5 4965050 200 99101 49650.5 4965050 -32369 32566 5086.02 508602 -127 125 -0.7 -70 -201 2 10191 99102 0.6036 297.6036 149.1036 14910.36036 0.6036 297.6036 149.1036 14910.36063 0.60360 297.60360 149.10360 14910.36000 2020-01-01 2020-01-02 2020-01-01 00:03:21 2020-01-02 03:31:42 2020-01-01 00:03:21.000 2020-01-02 03:31:42.000 201 99102 49651.5 4965150 201 99102 49651.5 4965150 -32368 32567 5087.02 508702 -126 126 0.3 30 -202 2 10192 99103 0.6066 297.6066 149.1066 14910.66066 0.6066 297.6066 149.1066 14910.66078 0.60660 297.60660 149.10660 14910.66000 2020-01-01 2020-01-02 2020-01-01 00:03:22 2020-01-02 03:31:43 2020-01-01 00:03:22.000 2020-01-02 03:31:43.000 202 99103 49652.5 4965250 202 99103 49652.5 4965250 -32367 32568 5088.02 508802 -125 127 1.3 130 -203 2 10193 99104 0.6096 297.6096 149.1096 14910.96096 0.6096 297.60962 149.1096 14910.9609 0.60960 297.60960 149.10960 14910.96000 2020-01-01 2020-01-02 2020-01-01 00:03:23 2020-01-02 03:31:44 2020-01-01 00:03:23.000 2020-01-02 03:31:44.000 203 99104 49653.5 4965350 203 99104 49653.5 4965350 -32366 32569 5089.02 508902 -128 127 -0.26 -26 -204 2 10194 99105 0.61261 297.61261 149.11261 14911.26126 0.61261 297.6126 149.11261 14911.26105 0.61261 297.61261 149.11261 14911.26100 2020-01-01 2020-01-02 2020-01-01 00:03:24 2020-01-02 03:31:45 2020-01-01 00:03:24.000 2020-01-02 03:31:45.000 204 99105 49654.5 4965450 204 99105 49654.5 4965450 -32365 32570 5090.02 509002 -128 127 -1.82 -182 -205 2 10195 99106 0.61561 297.61561 149.11561 14911.56156 0.61561 297.6156 149.11561 14911.56137 0.61561 297.61561 149.11561 14911.56100 2020-01-01 2020-01-02 2020-01-01 00:03:25 2020-01-02 03:31:46 2020-01-01 00:03:25.000 2020-01-02 03:31:46.000 205 99106 49655.5 4965550 205 99106 49655.5 4965550 -32364 32571 5091.02 509102 -128 123 -3.38 -338 -206 2 10196 99107 0.61861 297.61861 149.11861 14911.86186 0.61861 297.61862 149.11862 14911.8621 0.61861 297.61861 149.11861 14911.86100 2020-01-01 2020-01-02 2020-01-01 00:03:26 2020-01-02 03:31:47 2020-01-01 00:03:26.000 2020-01-02 03:31:47.000 206 99107 49656.5 4965650 206 99107 49656.5 4965650 -32363 32572 5092.02 509202 -127 124 -2.38 -238 -207 2 10197 99108 0.62162 297.62162 149.12162 14912.16216 0.62162 297.6216 149.12162 14912.16225 0.62162 297.62162 149.12162 14912.16200 2020-01-01 2020-01-02 2020-01-01 00:03:27 2020-01-02 03:31:48 2020-01-01 00:03:27.000 2020-01-02 03:31:48.000 207 99108 49657.5 4965750 207 99108 49657.5 4965750 -32362 32573 5093.02 509302 -126 125 -1.38 -138 +198 2 10188 99099 0.59459 297.59459 149.09459 14909.45945 0.59459 297.5946 149.09459 14909.4594 0.59459 297.59459 149.09459000000007 14909.45900 2020-01-01 2020-01-02 2020-01-01 00:03:18 2020-01-02 03:31:39 2020-01-01 00:03:18.000 2020-01-02 03:31:39.000 198 99099 49648.5 4964850 198 99099 49648.5 4964850 -32371 32564 5084.02 508402 -128 127 -0.14 -14 +199 2 10189 99100 0.59759 297.59759 149.09759 14909.75975 0.59759 297.5976 149.09759 14909.75958 0.59759 297.59759 149.09758999999994 14909.75900 2020-01-01 2020-01-02 2020-01-01 00:03:19 2020-01-02 03:31:40 2020-01-01 00:03:19.000 2020-01-02 03:31:40.000 199 99100 49649.5 4964950 199 99100 49649.5 4964950 -32370 32565 5085.02 508502 -128 124 -1.7 -170 +2 2 1001 9992 0.006 300.006 150.006 15150.6066 0.006 300.006 150.006 15150.6069 0.00600 300.00600 150.00599999999991 15150.60600 2020-01-01 2020-01-02 2020-01-01 00:00:02 2020-01-02 03:45:02 2020-01-01 00:00:02.000 2020-01-02 03:45:02.000 2 99902 49952 5045152 2 99902 49952 5045152 -32567 32368 4531.009900990099 457632 -125 126 -0.9504950495049505 -96 +20 2 10010 99920 0.06006 300.06006 150.06006 15156.06606 0.06006 300.06006 150.06005 15156.06593 0.06006 300.06006 150.06005999999988 15156.06606 2020-01-01 2020-01-02 2020-01-01 00:00:20 2020-01-02 03:45:20 2020-01-01 00:00:20.000 2020-01-02 03:45:20.000 20 99920 49970 5046970 20 99920 49970 5046970 -32549 32386 4549.009900990099 459450 -126 126 -0.693069306930693 -70 +200 2 10190 99101 0.6006 297.6006 149.1006 14910.06006 0.6006 297.6006 149.10059 14910.0599 0.60060 297.60060 149.10059999999987 14910.06000 2020-01-01 2020-01-02 2020-01-01 00:03:20 2020-01-02 03:31:41 2020-01-01 00:03:20.000 2020-01-02 03:31:41.000 200 99101 49650.5 4965050 200 99101 49650.5 4965050 -32369 32566 5086.02 508602 -127 125 -0.7 -70 +201 2 10191 99102 0.6036 297.6036 149.1036 14910.36036 0.6036 297.6036 149.1036 14910.36063 0.60360 297.60360 149.10360000000017 14910.36000 2020-01-01 2020-01-02 2020-01-01 00:03:21 2020-01-02 03:31:42 2020-01-01 00:03:21.000 2020-01-02 03:31:42.000 201 99102 49651.5 4965150 201 99102 49651.5 4965150 -32368 32567 5087.02 508702 -126 126 0.3 30 +202 2 10192 99103 0.6066 297.6066 149.1066 14910.66066 0.6066 297.6066 149.1066 14910.66078 0.60660 297.60660 149.10659999999982 14910.66000 2020-01-01 2020-01-02 2020-01-01 00:03:22 2020-01-02 03:31:43 2020-01-01 00:03:22.000 2020-01-02 03:31:43.000 202 99103 49652.5 4965250 202 99103 49652.5 4965250 -32367 32568 5088.02 508802 -125 127 1.3 130 +203 2 10193 99104 0.6096 297.6096 149.1096 14910.96096 0.6096 297.60962 149.1096 14910.9609 0.60960 297.60960 149.10959999999983 14910.96000 2020-01-01 2020-01-02 2020-01-01 00:03:23 2020-01-02 03:31:44 2020-01-01 00:03:23.000 2020-01-02 03:31:44.000 203 99104 49653.5 4965350 203 99104 49653.5 4965350 -32366 32569 5089.02 508902 -128 127 -0.26 -26 +204 2 10194 99105 0.61261 297.61261 149.11261 14911.26126 0.61261 297.6126 149.11261 14911.26105 0.61261 297.61261 149.1126100000001 14911.26100 2020-01-01 2020-01-02 2020-01-01 00:03:24 2020-01-02 03:31:45 2020-01-01 00:03:24.000 2020-01-02 03:31:45.000 204 99105 49654.5 4965450 204 99105 49654.5 4965450 -32365 32570 5090.02 509002 -128 127 -1.82 -182 +205 2 10195 99106 0.61561 297.61561 149.11561 14911.56156 0.61561 297.6156 149.11561 14911.56137 0.61561 297.61561 149.11561000000012 14911.56100 2020-01-01 2020-01-02 2020-01-01 00:03:25 2020-01-02 03:31:46 2020-01-01 00:03:25.000 2020-01-02 03:31:46.000 205 99106 49655.5 4965550 205 99106 49655.5 4965550 -32364 32571 5091.02 509102 -128 123 -3.38 -338 +206 2 10196 99107 0.61861 297.61861 149.11861 14911.86186 0.61861 297.61862 149.11862 14911.8621 0.61861 297.61861 149.1186099999999 14911.86100 2020-01-01 2020-01-02 2020-01-01 00:03:26 2020-01-02 03:31:47 2020-01-01 00:03:26.000 2020-01-02 03:31:47.000 206 99107 49656.5 4965650 206 99107 49656.5 4965650 -32363 32572 5092.02 509202 -127 124 -2.38 -238 +207 2 10197 99108 0.62162 297.62162 149.12162 14912.16216 0.62162 297.6216 149.12162 14912.16225 0.62162 297.62162 149.12161999999992 14912.16200 2020-01-01 2020-01-02 2020-01-01 00:03:27 2020-01-02 03:31:48 2020-01-01 00:03:27.000 2020-01-02 03:31:48.000 207 99108 49657.5 4965750 207 99108 49657.5 4965750 -32362 32573 5093.02 509302 -126 125 -1.38 -138 208 2 10198 99109 0.62462 297.62462 149.12462 14912.46246 0.62462 297.62463 149.12462 14912.46237 0.62462 297.62462 149.12462 14912.46200 2020-01-01 2020-01-02 2020-01-01 00:03:28 2020-01-02 03:31:49 2020-01-01 00:03:28.000 2020-01-02 03:31:49.000 208 99109 49658.5 4965850 208 99109 49658.5 4965850 -32361 32574 5094.02 509402 -125 126 -0.38 -38 -209 2 10199 99110 0.62762 297.62762 149.12762 14912.76276 0.62762 297.62762 149.12762 14912.76253 0.62762 297.62762 149.12762 14912.76200 2020-01-01 2020-01-02 2020-01-01 00:03:29 2020-01-02 03:31:50 2020-01-01 00:03:29.000 2020-01-02 03:31:50.000 209 99110 49659.5 4965950 209 99110 49659.5 4965950 -32360 32575 5095.02 509502 -124 127 0.62 62 -21 2 10011 99921 0.06306 300.06306 150.06306 15156.36936 0.06306 300.06305 150.06306 15156.36927 0.06306 300.06306 150.06306 15156.36906 2020-01-01 2020-01-02 2020-01-01 00:00:21 2020-01-02 03:45:21 2020-01-01 00:00:21.000 2020-01-02 03:45:21.000 21 99921 49971 5047071 21 99921 49971 5047071 -32548 32387 4550.009900990099 459551 -125 127 0.3069306930693069 31 -210 2 10200 99111 0.63063 297.63063 149.13063 14913.06306 0.63063 297.63065 149.13063 14913.06326 0.63063 297.63063 149.13063 14913.06300 2020-01-01 2020-01-02 2020-01-01 00:03:30 2020-01-02 03:31:51 2020-01-01 00:03:30.000 2020-01-02 03:31:51.000 210 99111 49660.5 4966050 210 99111 49660.5 4966050 -32359 32576 5096.02 509602 -128 127 -0.94 -94 -211 2 10201 99112 0.63363 297.63363 149.13363 14913.36336 0.63363 297.63364 149.13363 14913.36357 0.63363 297.63363 149.13363 14913.36300 2020-01-01 2020-01-02 2020-01-01 00:03:31 2020-01-02 03:31:52 2020-01-01 00:03:31.000 2020-01-02 03:31:52.000 211 99112 49661.5 4966150 211 99112 49661.5 4966150 -32358 32577 5097.02 509702 -128 123 -2.5 -250 -212 2 10202 99113 0.63663 297.63663 149.13663 14913.66366 0.63663 297.63663 149.13663 14913.66372 0.63663 297.63663 149.13663 14913.66300 2020-01-01 2020-01-02 2020-01-01 00:03:32 2020-01-02 03:31:53 2020-01-01 00:03:32.000 2020-01-02 03:31:53.000 212 99113 49662.5 4966250 212 99113 49662.5 4966250 -32357 32578 5098.02 509802 -127 124 -1.5 -150 -213 2 10203 99114 0.63963 297.63963 149.13963 14913.96396 0.63963 297.63965 149.13963 14913.96384 0.63963 297.63963 149.13963 14913.96300 2020-01-01 2020-01-02 2020-01-01 00:03:33 2020-01-02 03:31:54 2020-01-01 00:03:33.000 2020-01-02 03:31:54.000 213 99114 49663.5 4966350 213 99114 49663.5 4966350 -32356 32579 5099.02 509902 -126 125 -0.5 -50 +209 2 10199 99110 0.62762 297.62762 149.12762 14912.76276 0.62762 297.62762 149.12762 14912.76253 0.62762 297.62762 149.12761999999992 14912.76200 2020-01-01 2020-01-02 2020-01-01 00:03:29 2020-01-02 03:31:50 2020-01-01 00:03:29.000 2020-01-02 03:31:50.000 209 99110 49659.5 4965950 209 99110 49659.5 4965950 -32360 32575 5095.02 509502 -124 127 0.62 62 +21 2 10011 99921 0.06306 300.06306 150.06306 15156.36936 0.06306 300.06305 150.06306 15156.36927 0.06306 300.06306 150.0630600000002 15156.36906 2020-01-01 2020-01-02 2020-01-01 00:00:21 2020-01-02 03:45:21 2020-01-01 00:00:21.000 2020-01-02 03:45:21.000 21 99921 49971 5047071 21 99921 49971 5047071 -32548 32387 4550.009900990099 459551 -125 127 0.3069306930693069 31 +210 2 10200 99111 0.63063 297.63063 149.13063 14913.06306 0.63063 297.63065 149.13063 14913.06326 0.63063 297.63063 149.13062999999983 14913.06300 2020-01-01 2020-01-02 2020-01-01 00:03:30 2020-01-02 03:31:51 2020-01-01 00:03:30.000 2020-01-02 03:31:51.000 210 99111 49660.5 4966050 210 99111 49660.5 4966050 -32359 32576 5096.02 509602 -128 127 -0.94 -94 +211 2 10201 99112 0.63363 297.63363 149.13363 14913.36336 0.63363 297.63364 149.13363 14913.36357 0.63363 297.63363 149.13363000000012 14913.36300 2020-01-01 2020-01-02 2020-01-01 00:03:31 2020-01-02 03:31:52 2020-01-01 00:03:31.000 2020-01-02 03:31:52.000 211 99112 49661.5 4966150 211 99112 49661.5 4966150 -32358 32577 5097.02 509702 -128 123 -2.5 -250 +212 2 10202 99113 0.63663 297.63663 149.13663 14913.66366 0.63663 297.63663 149.13663 14913.66372 0.63663 297.63663 149.1366300000002 14913.66300 2020-01-01 2020-01-02 2020-01-01 00:03:32 2020-01-02 03:31:53 2020-01-01 00:03:32.000 2020-01-02 03:31:53.000 212 99113 49662.5 4966250 212 99113 49662.5 4966250 -32357 32578 5098.02 509802 -127 124 -1.5 -150 +213 2 10203 99114 0.63963 297.63963 149.13963 14913.96396 0.63963 297.63965 149.13963 14913.96384 0.63963 297.63963 149.13962999999984 14913.96300 2020-01-01 2020-01-02 2020-01-01 00:03:33 2020-01-02 03:31:54 2020-01-01 00:03:33.000 2020-01-02 03:31:54.000 213 99114 49663.5 4966350 213 99114 49663.5 4966350 -32356 32579 5099.02 509902 -126 125 -0.5 -50 214 2 10204 99115 0.64264 297.64264 149.14264 14914.26426 0.64264 297.64264 149.14263 14914.26399 0.64264 297.64264 149.14264 14914.26400 2020-01-01 2020-01-02 2020-01-01 00:03:34 2020-01-02 03:31:55 2020-01-01 00:03:34.000 2020-01-02 03:31:55.000 214 99115 49664.5 4966450 214 99115 49664.5 4966450 -32355 32580 5100.02 510002 -125 126 0.5 50 -215 2 10205 99116 0.64564 297.64564 149.14564 14914.56456 0.64564 297.64566 149.14564 14914.56473 0.64564 297.64564 149.14564 14914.56400 2020-01-01 2020-01-02 2020-01-01 00:03:35 2020-01-02 03:31:56 2020-01-01 00:03:35.000 2020-01-02 03:31:56.000 215 99116 49665.5 4966550 215 99116 49665.5 4966550 -32354 32581 5101.02 510102 -124 127 1.5 150 -216 2 10206 99117 0.64864 297.64864 149.14864 14914.86486 0.64864 297.64865 149.14865 14914.86504 0.64864 297.64864 149.14864 14914.86400 2020-01-01 2020-01-02 2020-01-01 00:03:36 2020-01-02 03:31:57 2020-01-01 00:03:36.000 2020-01-02 03:31:57.000 216 99117 49666.5 4966650 216 99117 49666.5 4966650 -32353 32582 5102.02 510202 -128 127 -0.06 -6 -217 2 10207 99118 0.65165 297.65165 149.15165 14915.16516 0.65165 297.65164 149.15165 14915.16523 0.65165 297.65165 149.15165 14915.16500 2020-01-01 2020-01-02 2020-01-01 00:03:37 2020-01-02 03:31:58 2020-01-01 00:03:37.000 2020-01-02 03:31:58.000 217 99118 49667.5 4966750 217 99118 49667.5 4966750 -32352 32583 5103.02 510302 -128 123 -1.62 -162 -218 2 10208 99119 0.65465 297.65465 149.15465 14915.46546 0.65465 297.65466 149.15465 14915.46531 0.65465 297.65465 149.15465 14915.46500 2020-01-01 2020-01-02 2020-01-01 00:03:38 2020-01-02 03:31:59 2020-01-01 00:03:38.000 2020-01-02 03:31:59.000 218 99119 49668.5 4966850 218 99119 49668.5 4966850 -32351 32584 5104.02 510402 -127 124 -0.62 -62 -219 2 10209 99120 0.65765 297.65765 149.15765 14915.76576 0.65765 297.65765 149.15765 14915.76562 0.65765 297.65765 149.15765 14915.76500 2020-01-01 2020-01-02 2020-01-01 00:03:39 2020-01-02 03:32:00 2020-01-01 00:03:39.000 2020-01-02 03:32:00.000 219 99120 49669.5 4966950 219 99120 49669.5 4966950 -32350 32585 5105.02 510502 -126 125 0.38 38 -22 2 10012 99922 0.06606 300.06606 150.06606 15156.67267 0.06606 300.06607 150.06606 15156.67287 0.06606 300.06606 150.06606 15156.67206 2020-01-01 2020-01-02 2020-01-01 00:00:22 2020-01-02 03:45:22 2020-01-01 00:00:22.000 2020-01-02 03:45:22.000 22 99922 49972 5047172 22 99922 49972 5047172 -32547 32388 4551.009900990099 459652 -128 127 -1.2277227722772277 -124 -220 2 10210 99121 0.66066 297.66066 149.16066 14916.06606 0.66066 297.66068 149.16066 14916.06619 0.66066 297.66066 149.16066 14916.06600 2020-01-01 2020-01-02 2020-01-01 00:03:40 2020-01-02 03:32:01 2020-01-01 00:03:40.000 2020-01-02 03:32:01.000 220 99121 49670.5 4967050 220 99121 49670.5 4967050 -32349 32586 5106.02 510602 -125 126 1.38 138 -221 2 10211 99122 0.66366 297.66366 149.16366 14916.36636 0.66366 297.66367 149.16366 14916.36651 0.66366 297.66366 149.16366 14916.36600 2020-01-01 2020-01-02 2020-01-01 00:03:41 2020-01-02 03:32:02 2020-01-01 00:03:41.000 2020-01-02 03:32:02.000 221 99122 49671.5 4967150 221 99122 49671.5 4967150 -32348 32587 5107.02 510702 -124 127 2.38 238 -222 2 10212 99123 0.66666 297.66666 149.16666 14916.66666 0.66666 297.66666 149.16666 14916.6667 0.66666 297.66666 149.16666 14916.66600 2020-01-01 2020-01-02 2020-01-01 00:03:42 2020-01-02 03:32:03 2020-01-01 00:03:42.000 2020-01-02 03:32:03.000 222 99123 49672.5 4967250 222 99123 49672.5 4967250 -32347 32588 5108.02 510802 -128 127 0.82 82 -223 2 10213 99124 0.66966 297.66966 149.16966 14916.96696 0.66966 297.66968 149.16966 14916.96678 0.66966 297.66966 149.16966 14916.96600 2020-01-01 2020-01-02 2020-01-01 00:03:43 2020-01-02 03:32:04 2020-01-01 00:03:43.000 2020-01-02 03:32:04.000 223 99124 49673.5 4967350 223 99124 49673.5 4967350 -32346 32589 5109.02 510902 -128 127 -0.74 -74 -224 2 10214 99125 0.67267 297.67267 149.17267 14917.26726 0.67267 297.67267 149.17267 14917.26709 0.67267 297.67267 149.17267 14917.26700 2020-01-01 2020-01-02 2020-01-01 00:03:44 2020-01-02 03:32:05 2020-01-01 00:03:44.000 2020-01-02 03:32:05.000 224 99125 49674.5 4967450 224 99125 49674.5 4967450 -32345 32590 5110.02 511002 -128 124 -2.3 -230 -225 2 10215 99126 0.67567 297.67567 149.17567 14917.56756 0.67567 297.6757 149.17567 14917.56767 0.67567 297.67567 149.17567 14917.56700 2020-01-01 2020-01-02 2020-01-01 00:03:45 2020-01-02 03:32:06 2020-01-01 00:03:45.000 2020-01-02 03:32:06.000 225 99126 49675.5 4967550 225 99126 49675.5 4967550 -32344 32591 5111.02 511102 -127 125 -1.3 -130 -226 2 10216 99127 0.67867 297.67867 149.17867 14917.86786 0.67867 297.67868 149.17868 14917.86802 0.67867 297.67867 149.17867 14917.86700 2020-01-01 2020-01-02 2020-01-01 00:03:46 2020-01-02 03:32:07 2020-01-01 00:03:46.000 2020-01-02 03:32:07.000 226 99127 49676.5 4967650 226 99127 49676.5 4967650 -32343 32592 5112.02 511202 -126 126 -0.3 -30 -227 2 10217 99128 0.68168 297.68168 149.18168 14918.16816 0.68168 297.68167 149.18168 14918.16817 0.68168 297.68168 149.18168 14918.16800 2020-01-01 2020-01-02 2020-01-01 00:03:47 2020-01-02 03:32:08 2020-01-01 00:03:47.000 2020-01-02 03:32:08.000 227 99128 49677.5 4967750 227 99128 49677.5 4967750 -32342 32593 5113.02 511302 -125 127 0.7 70 -228 2 10218 99129 0.68468 297.68468 149.18468 14918.46846 0.68468 297.6847 149.18468 14918.46825 0.68468 297.68468 149.18468 14918.46800 2020-01-01 2020-01-02 2020-01-01 00:03:48 2020-01-02 03:32:09 2020-01-01 00:03:48.000 2020-01-02 03:32:09.000 228 99129 49678.5 4967850 228 99129 49678.5 4967850 -32341 32594 5114.02 511402 -128 127 -0.86 -86 -229 2 10219 99130 0.68768 297.68768 149.18768 14918.76876 0.68768 297.68768 149.18768 14918.76855 0.68768 297.68768 149.18768 14918.76800 2020-01-01 2020-01-02 2020-01-01 00:03:49 2020-01-02 03:32:10 2020-01-01 00:03:49.000 2020-01-02 03:32:10.000 229 99130 49679.5 4967950 229 99130 49679.5 4967950 -32340 32595 5115.02 511502 -128 127 -2.42 -242 -23 2 10013 99923 0.06906 300.06906 150.06906 15156.97597 0.06906 300.06906 150.06907 15156.97617 0.06906 300.06906 150.06906 15156.97506 2020-01-01 2020-01-02 2020-01-01 00:00:23 2020-01-02 03:45:23 2020-01-01 00:00:23.000 2020-01-02 03:45:23.000 23 99923 49973 5047273 23 99923 49973 5047273 -32546 32389 4552.009900990099 459753 -128 127 -2.762376237623762 -279 -230 2 10220 99131 0.69069 297.69069 149.19069 14919.06906 0.69069 297.6907 149.19069 14919.06914 0.69069 297.69069 149.19069 14919.06900 2020-01-01 2020-01-02 2020-01-01 00:03:50 2020-01-02 03:32:11 2020-01-01 00:03:50.000 2020-01-02 03:32:11.000 230 99131 49680.5 4968050 230 99131 49680.5 4968050 -32339 32596 5116.02 511602 -128 123 -3.98 -398 -231 2 10221 99132 0.69369 297.69369 149.19369 14919.36936 0.69369 297.6937 149.19369 14919.36949 0.69369 297.69369 149.19369 14919.36900 2020-01-01 2020-01-02 2020-01-01 00:03:51 2020-01-02 03:32:12 2020-01-01 00:03:51.000 2020-01-02 03:32:12.000 231 99132 49681.5 4968150 231 99132 49681.5 4968150 -32338 32597 5117.02 511702 -127 124 -2.98 -298 -232 2 10222 99133 0.69669 297.69669 149.19669 14919.66966 0.69669 297.6967 149.19669 14919.66964 0.69669 297.69669 149.19669 14919.66900 2020-01-01 2020-01-02 2020-01-01 00:03:52 2020-01-02 03:32:13 2020-01-01 00:03:52.000 2020-01-02 03:32:13.000 232 99133 49682.5 4968250 232 99133 49682.5 4968250 -32337 32598 5118.02 511802 -126 125 -1.98 -198 -233 2 10223 99134 0.69969 297.69969 149.19969 14919.96996 0.69969 297.6997 149.1997 14919.97037 0.69969 297.69969 149.19969 14919.96900 2020-01-01 2020-01-02 2020-01-01 00:03:53 2020-01-02 03:32:14 2020-01-01 00:03:53.000 2020-01-02 03:32:14.000 233 99134 49683.5 4968350 233 99134 49683.5 4968350 -32336 32599 5119.02 511902 -125 126 -0.98 -98 -234 2 10224 99135 0.7027 297.7027 149.2027 14920.27027 0.7027 297.7027 149.2027 14920.27003 0.70270 297.70270 149.20270 14920.27000 2020-01-01 2020-01-02 2020-01-01 00:03:54 2020-01-02 03:32:15 2020-01-01 00:03:54.000 2020-01-02 03:32:15.000 234 99135 49684.5 4968450 234 99135 49684.5 4968450 -32335 32600 5120.02 512002 -124 127 0.02 2 -235 2 10225 99136 0.7057 297.7057 149.2057 14920.57057 0.7057 297.70572 149.2057 14920.57065 0.70570 297.70570 149.20570 14920.57000 2020-01-01 2020-01-02 2020-01-01 00:03:55 2020-01-02 03:32:16 2020-01-01 00:03:55.000 2020-01-02 03:32:16.000 235 99136 49685.5 4968550 235 99136 49685.5 4968550 -32334 32601 5121.02 512102 -128 127 -1.54 -154 -236 2 10226 99137 0.7087 297.7087 149.2087 14920.87087 0.7087 297.7087 149.2087 14920.87095 0.70870 297.70870 149.20870 14920.87000 2020-01-01 2020-01-02 2020-01-01 00:03:56 2020-01-02 03:32:17 2020-01-01 00:03:56.000 2020-01-02 03:32:17.000 236 99137 49686.5 4968650 236 99137 49686.5 4968650 -32333 32602 5122.02 512202 -128 123 -3.1 -310 -237 2 10227 99138 0.71171 297.71171 149.21171 14921.17117 0.71171 297.7117 149.21171 14921.17111 0.71171 297.71171 149.21171 14921.17100 2020-01-01 2020-01-02 2020-01-01 00:03:57 2020-01-02 03:32:18 2020-01-01 00:03:57.000 2020-01-02 03:32:18.000 237 99138 49687.5 4968750 237 99138 49687.5 4968750 -32332 32603 5123.02 512302 -127 124 -2.1 -210 -238 2 10228 99139 0.71471 297.71471 149.21471 14921.47147 0.71471 297.71472 149.21471 14921.47184 0.71471 297.71471 149.21471 14921.47100 2020-01-01 2020-01-02 2020-01-01 00:03:58 2020-01-02 03:32:19 2020-01-01 00:03:58.000 2020-01-02 03:32:19.000 238 99139 49688.5 4968850 238 99139 49688.5 4968850 -32331 32604 5124.02 512402 -126 125 -1.1 -110 -239 2 10229 99140 0.71771 297.71771 149.21771 14921.77177 0.71771 297.7177 149.21771 14921.7715 0.71771 297.71771 149.21771 14921.77100 2020-01-01 2020-01-02 2020-01-01 00:03:59 2020-01-02 03:32:20 2020-01-01 00:03:59.000 2020-01-02 03:32:20.000 239 99140 49689.5 4968950 239 99140 49689.5 4968950 -32330 32605 5125.02 512502 -125 126 -0.1 -10 -24 2 10014 99924 0.07207 300.07207 150.07207 15157.27927 0.07207 300.07208 150.07207 15157.27928 0.07207 300.07207 150.07207 15157.27907 2020-01-01 2020-01-02 2020-01-01 00:00:24 2020-01-02 03:45:24 2020-01-01 00:00:24.000 2020-01-02 03:45:24.000 24 99924 49974 5047374 24 99924 49974 5047374 -32545 32390 4553.009900990099 459854 -128 123 -4.297029702970297 -434 -240 2 10230 99141 0.72072 297.72072 149.22072 14922.07207 0.72072 297.72073 149.22072 14922.07211 0.72072 297.72072 149.22072 14922.07200 2020-01-01 2020-01-02 2020-01-01 00:04:00 2020-01-02 03:32:21 2020-01-01 00:04:00.000 2020-01-02 03:32:21.000 240 99141 49690.5 4969050 240 99141 49690.5 4969050 -32329 32606 5126.02 512602 -124 127 0.9 90 +215 2 10205 99116 0.64564 297.64564 149.14564 14914.56456 0.64564 297.64566 149.14564 14914.56473 0.64564 297.64564 149.1456400000001 14914.56400 2020-01-01 2020-01-02 2020-01-01 00:03:35 2020-01-02 03:31:56 2020-01-01 00:03:35.000 2020-01-02 03:31:56.000 215 99116 49665.5 4966550 215 99116 49665.5 4966550 -32354 32581 5101.02 510102 -124 127 1.5 150 +216 2 10206 99117 0.64864 297.64864 149.14864 14914.86486 0.64864 297.64865 149.14865 14914.86504 0.64864 297.64864 149.14863999999994 14914.86400 2020-01-01 2020-01-02 2020-01-01 00:03:36 2020-01-02 03:31:57 2020-01-01 00:03:36.000 2020-01-02 03:31:57.000 216 99117 49666.5 4966650 216 99117 49666.5 4966650 -32353 32582 5102.02 510202 -128 127 -0.06 -6 +217 2 10207 99118 0.65165 297.65165 149.15165 14915.16516 0.65165 297.65164 149.15165 14915.16523 0.65165 297.65165 149.15164999999988 14915.16500 2020-01-01 2020-01-02 2020-01-01 00:03:37 2020-01-02 03:31:58 2020-01-01 00:03:37.000 2020-01-02 03:31:58.000 217 99118 49667.5 4966750 217 99118 49667.5 4966750 -32352 32583 5103.02 510302 -128 123 -1.62 -162 +218 2 10208 99119 0.65465 297.65465 149.15465 14915.46546 0.65465 297.65466 149.15465 14915.46531 0.65465 297.65465 149.1546500000002 14915.46500 2020-01-01 2020-01-02 2020-01-01 00:03:38 2020-01-02 03:31:59 2020-01-01 00:03:38.000 2020-01-02 03:31:59.000 218 99119 49668.5 4966850 218 99119 49668.5 4966850 -32351 32584 5104.02 510402 -127 124 -0.62 -62 +219 2 10209 99120 0.65765 297.65765 149.15765 14915.76576 0.65765 297.65765 149.15765 14915.76562 0.65765 297.65765 149.15764999999985 14915.76500 2020-01-01 2020-01-02 2020-01-01 00:03:39 2020-01-02 03:32:00 2020-01-01 00:03:39.000 2020-01-02 03:32:00.000 219 99120 49669.5 4966950 219 99120 49669.5 4966950 -32350 32585 5105.02 510502 -126 125 0.38 38 +22 2 10012 99922 0.06606 300.06606 150.06606 15156.67267 0.06606 300.06607 150.06606 15156.67287 0.06606 300.06606 150.0660599999999 15156.67206 2020-01-01 2020-01-02 2020-01-01 00:00:22 2020-01-02 03:45:22 2020-01-01 00:00:22.000 2020-01-02 03:45:22.000 22 99922 49972 5047172 22 99922 49972 5047172 -32547 32388 4551.009900990099 459652 -128 127 -1.2277227722772277 -124 +220 2 10210 99121 0.66066 297.66066 149.16066 14916.06606 0.66066 297.66068 149.16066 14916.06619 0.66066 297.66066 149.16065999999975 14916.06600 2020-01-01 2020-01-02 2020-01-01 00:03:40 2020-01-02 03:32:01 2020-01-01 00:03:40.000 2020-01-02 03:32:01.000 220 99121 49670.5 4967050 220 99121 49670.5 4967050 -32349 32586 5106.02 510602 -125 126 1.38 138 +221 2 10211 99122 0.66366 297.66366 149.16366 14916.36636 0.66366 297.66367 149.16366 14916.36651 0.66366 297.66366 149.1636600000001 14916.36600 2020-01-01 2020-01-02 2020-01-01 00:03:41 2020-01-02 03:32:02 2020-01-01 00:03:41.000 2020-01-02 03:32:02.000 221 99122 49671.5 4967150 221 99122 49671.5 4967150 -32348 32587 5107.02 510702 -124 127 2.38 238 +222 2 10212 99123 0.66666 297.66666 149.16666 14916.66666 0.66666 297.66666 149.16666 14916.6667 0.66666 297.66666 149.1666600000001 14916.66600 2020-01-01 2020-01-02 2020-01-01 00:03:42 2020-01-02 03:32:03 2020-01-01 00:03:42.000 2020-01-02 03:32:03.000 222 99123 49672.5 4967250 222 99123 49672.5 4967250 -32347 32588 5108.02 510802 -128 127 0.82 82 +223 2 10213 99124 0.66966 297.66966 149.16966 14916.96696 0.66966 297.66968 149.16966 14916.96678 0.66966 297.66966 149.16965999999974 14916.96600 2020-01-01 2020-01-02 2020-01-01 00:03:43 2020-01-02 03:32:04 2020-01-01 00:03:43.000 2020-01-02 03:32:04.000 223 99124 49673.5 4967350 223 99124 49673.5 4967350 -32346 32589 5109.02 510902 -128 127 -0.74 -74 +224 2 10214 99125 0.67267 297.67267 149.17267 14917.26726 0.67267 297.67267 149.17267 14917.26709 0.67267 297.67267 149.17266999999995 14917.26700 2020-01-01 2020-01-02 2020-01-01 00:03:44 2020-01-02 03:32:05 2020-01-01 00:03:44.000 2020-01-02 03:32:05.000 224 99125 49674.5 4967450 224 99125 49674.5 4967450 -32345 32590 5110.02 511002 -128 124 -2.3 -230 +225 2 10215 99126 0.67567 297.67567 149.17567 14917.56756 0.67567 297.6757 149.17567 14917.56767 0.67567 297.67567 149.17567000000003 14917.56700 2020-01-01 2020-01-02 2020-01-01 00:03:45 2020-01-02 03:32:06 2020-01-01 00:03:45.000 2020-01-02 03:32:06.000 225 99126 49675.5 4967550 225 99126 49675.5 4967550 -32344 32591 5111.02 511102 -127 125 -1.3 -130 +226 2 10216 99127 0.67867 297.67867 149.17867 14917.86786 0.67867 297.67868 149.17868 14917.86802 0.67867 297.67867 149.17866999999995 14917.86700 2020-01-01 2020-01-02 2020-01-01 00:03:46 2020-01-02 03:32:07 2020-01-01 00:03:46.000 2020-01-02 03:32:07.000 226 99127 49676.5 4967650 226 99127 49676.5 4967650 -32343 32592 5112.02 511202 -126 126 -0.3 -30 +227 2 10217 99128 0.68168 297.68168 149.18168 14918.16816 0.68168 297.68167 149.18168 14918.16817 0.68168 297.68168 149.18167999999986 14918.16800 2020-01-01 2020-01-02 2020-01-01 00:03:47 2020-01-02 03:32:08 2020-01-01 00:03:47.000 2020-01-02 03:32:08.000 227 99128 49677.5 4967750 227 99128 49677.5 4967750 -32342 32593 5113.02 511302 -125 127 0.7 70 +228 2 10218 99129 0.68468 297.68468 149.18468 14918.46846 0.68468 297.6847 149.18468 14918.46825 0.68468 297.68468 149.18468000000013 14918.46800 2020-01-01 2020-01-02 2020-01-01 00:03:48 2020-01-02 03:32:09 2020-01-01 00:03:48.000 2020-01-02 03:32:09.000 228 99129 49678.5 4967850 228 99129 49678.5 4967850 -32341 32594 5114.02 511402 -128 127 -0.86 -86 +229 2 10219 99130 0.68768 297.68768 149.18768 14918.76876 0.68768 297.68768 149.18768 14918.76855 0.68768 297.68768 149.18767999999983 14918.76800 2020-01-01 2020-01-02 2020-01-01 00:03:49 2020-01-02 03:32:10 2020-01-01 00:03:49.000 2020-01-02 03:32:10.000 229 99130 49679.5 4967950 229 99130 49679.5 4967950 -32340 32595 5115.02 511502 -128 127 -2.42 -242 +23 2 10013 99923 0.06906 300.06906 150.06906 15156.97597 0.06906 300.06906 150.06907 15156.97617 0.06906 300.06906 150.06905999999987 15156.97506 2020-01-01 2020-01-02 2020-01-01 00:00:23 2020-01-02 03:45:23 2020-01-01 00:00:23.000 2020-01-02 03:45:23.000 23 99923 49973 5047273 23 99923 49973 5047273 -32546 32389 4552.009900990099 459753 -128 127 -2.762376237623762 -279 +230 2 10220 99131 0.69069 297.69069 149.19069 14919.06906 0.69069 297.6907 149.19069 14919.06914 0.69069 297.69069 149.19068999999996 14919.06900 2020-01-01 2020-01-02 2020-01-01 00:03:50 2020-01-02 03:32:11 2020-01-01 00:03:50.000 2020-01-02 03:32:11.000 230 99131 49680.5 4968050 230 99131 49680.5 4968050 -32339 32596 5116.02 511602 -128 123 -3.98 -398 +231 2 10221 99132 0.69369 297.69369 149.19369 14919.36936 0.69369 297.6937 149.19369 14919.36949 0.69369 297.69369 149.19369000000003 14919.36900 2020-01-01 2020-01-02 2020-01-01 00:03:51 2020-01-02 03:32:12 2020-01-01 00:03:51.000 2020-01-02 03:32:12.000 231 99132 49681.5 4968150 231 99132 49681.5 4968150 -32338 32597 5117.02 511702 -127 124 -2.98 -298 +232 2 10222 99133 0.69669 297.69669 149.19669 14919.66966 0.69669 297.6967 149.19669 14919.66964 0.69669 297.69669 149.1966900000001 14919.66900 2020-01-01 2020-01-02 2020-01-01 00:03:52 2020-01-02 03:32:13 2020-01-01 00:03:52.000 2020-01-02 03:32:13.000 232 99133 49682.5 4968250 232 99133 49682.5 4968250 -32337 32598 5118.02 511802 -126 125 -1.98 -198 +233 2 10223 99134 0.69969 297.69969 149.19969 14919.96996 0.69969 297.6997 149.1997 14919.97037 0.69969 297.69969 149.19968999999998 14919.96900 2020-01-01 2020-01-02 2020-01-01 00:03:53 2020-01-02 03:32:14 2020-01-01 00:03:53.000 2020-01-02 03:32:14.000 233 99134 49683.5 4968350 233 99134 49683.5 4968350 -32336 32599 5119.02 511902 -125 126 -0.98 -98 +234 2 10224 99135 0.7027 297.7027 149.2027 14920.27027 0.7027 297.7027 149.2027 14920.27003 0.70270 297.70270 149.20269999999988 14920.27000 2020-01-01 2020-01-02 2020-01-01 00:03:54 2020-01-02 03:32:15 2020-01-01 00:03:54.000 2020-01-02 03:32:15.000 234 99135 49684.5 4968450 234 99135 49684.5 4968450 -32335 32600 5120.02 512002 -124 127 0.02 2 +235 2 10225 99136 0.7057 297.7057 149.2057 14920.57057 0.7057 297.70572 149.2057 14920.57065 0.70570 297.70570 149.20570000000023 14920.57000 2020-01-01 2020-01-02 2020-01-01 00:03:55 2020-01-02 03:32:16 2020-01-01 00:03:55.000 2020-01-02 03:32:16.000 235 99136 49685.5 4968550 235 99136 49685.5 4968550 -32334 32601 5121.02 512102 -128 127 -1.54 -154 +236 2 10226 99137 0.7087 297.7087 149.2087 14920.87087 0.7087 297.7087 149.2087 14920.87095 0.70870 297.70870 149.2086999999999 14920.87000 2020-01-01 2020-01-02 2020-01-01 00:03:56 2020-01-02 03:32:17 2020-01-01 00:03:56.000 2020-01-02 03:32:17.000 236 99137 49686.5 4968650 236 99137 49686.5 4968650 -32333 32602 5122.02 512202 -128 123 -3.1 -310 +237 2 10227 99138 0.71171 297.71171 149.21171 14921.17117 0.71171 297.7117 149.21171 14921.17111 0.71171 297.71171 149.21170999999978 14921.17100 2020-01-01 2020-01-02 2020-01-01 00:03:57 2020-01-02 03:32:18 2020-01-01 00:03:57.000 2020-01-02 03:32:18.000 237 99138 49687.5 4968750 237 99138 49687.5 4968750 -32332 32603 5123.02 512302 -127 124 -2.1 -210 +238 2 10228 99139 0.71471 297.71471 149.21471 14921.47147 0.71471 297.71472 149.21471 14921.47184 0.71471 297.71471 149.21471000000014 14921.47100 2020-01-01 2020-01-02 2020-01-01 00:03:58 2020-01-02 03:32:19 2020-01-01 00:03:58.000 2020-01-02 03:32:19.000 238 99139 49688.5 4968850 238 99139 49688.5 4968850 -32331 32604 5124.02 512402 -126 125 -1.1 -110 +239 2 10229 99140 0.71771 297.71771 149.21771 14921.77177 0.71771 297.7177 149.21771 14921.7715 0.71771 297.71771 149.2177100000001 14921.77100 2020-01-01 2020-01-02 2020-01-01 00:03:59 2020-01-02 03:32:20 2020-01-01 00:03:59.000 2020-01-02 03:32:20.000 239 99140 49689.5 4968950 239 99140 49689.5 4968950 -32330 32605 5125.02 512502 -125 126 -0.1 -10 +24 2 10014 99924 0.07207 300.07207 150.07207 15157.27927 0.07207 300.07208 150.07207 15157.27928 0.07207 300.07207 150.0720700000001 15157.27907 2020-01-01 2020-01-02 2020-01-01 00:00:24 2020-01-02 03:45:24 2020-01-01 00:00:24.000 2020-01-02 03:45:24.000 24 99924 49974 5047374 24 99924 49974 5047374 -32545 32390 4553.009900990099 459854 -128 123 -4.297029702970297 -434 +240 2 10230 99141 0.72072 297.72072 149.22072 14922.07207 0.72072 297.72073 149.22072 14922.07211 0.72072 297.72072 149.22071999999991 14922.07200 2020-01-01 2020-01-02 2020-01-01 00:04:00 2020-01-02 03:32:21 2020-01-01 00:04:00.000 2020-01-02 03:32:21.000 240 99141 49690.5 4969050 240 99141 49690.5 4969050 -32329 32606 5126.02 512602 -124 127 0.9 90 241 2 10231 99142 0.72372 297.72372 149.22372 14922.37237 0.72372 297.72372 149.22372 14922.37243 0.72372 297.72372 149.22372 14922.37200 2020-01-01 2020-01-02 2020-01-01 00:04:01 2020-01-02 03:32:22 2020-01-01 00:04:01.000 2020-01-02 03:32:22.000 241 99142 49691.5 4969150 241 99142 49691.5 4969150 -32328 32607 5127.02 512702 -128 127 -0.66 -66 -242 2 10232 99143 0.72672 297.72672 149.22672 14922.67267 0.72672 297.7267 149.22672 14922.67273 0.72672 297.72672 149.22672 14922.67200 2020-01-01 2020-01-02 2020-01-01 00:04:02 2020-01-02 03:32:23 2020-01-01 00:04:02.000 2020-01-02 03:32:23.000 242 99143 49692.5 4969250 242 99143 49692.5 4969250 -32327 32608 5128.02 512802 -128 123 -2.22 -222 -243 2 10233 99144 0.72972 297.72972 149.22972 14922.97297 0.72972 297.72974 149.22973 14922.97332 0.72972 297.72972 149.22972 14922.97200 2020-01-01 2020-01-02 2020-01-01 00:04:03 2020-01-02 03:32:24 2020-01-01 00:04:03.000 2020-01-02 03:32:24.000 243 99144 49693.5 4969350 243 99144 49693.5 4969350 -32326 32609 5129.02 512902 -127 124 -1.22 -122 -244 2 10234 99145 0.73273 297.73273 149.23273 14923.27327 0.73273 297.73273 149.23272 14923.27297 0.73273 297.73273 149.23273 14923.27300 2020-01-01 2020-01-02 2020-01-01 00:04:04 2020-01-02 03:32:25 2020-01-01 00:04:04.000 2020-01-02 03:32:25.000 244 99145 49694.5 4969450 244 99145 49694.5 4969450 -32325 32610 5130.02 513002 -126 125 -0.22 -22 -245 2 10235 99146 0.73573 297.73573 149.23573 14923.57357 0.73573 297.73575 149.23573 14923.57358 0.73573 297.73573 149.23573 14923.57300 2020-01-01 2020-01-02 2020-01-01 00:04:05 2020-01-02 03:32:26 2020-01-01 00:04:05.000 2020-01-02 03:32:26.000 245 99146 49695.5 4969550 245 99146 49695.5 4969550 -32324 32611 5131.02 513102 -125 126 0.78 78 -246 2 10236 99147 0.73873 297.73873 149.23873 14923.87387 0.73873 297.73874 149.23873 14923.8739 0.73873 297.73873 149.23873 14923.87300 2020-01-01 2020-01-02 2020-01-01 00:04:06 2020-01-02 03:32:27 2020-01-01 00:04:06.000 2020-01-02 03:32:27.000 246 99147 49696.5 4969650 246 99147 49696.5 4969650 -32323 32612 5132.02 513202 -124 127 1.78 178 +242 2 10232 99143 0.72672 297.72672 149.22672 14922.67267 0.72672 297.7267 149.22672 14922.67273 0.72672 297.72672 149.22672000000006 14922.67200 2020-01-01 2020-01-02 2020-01-01 00:04:02 2020-01-02 03:32:23 2020-01-01 00:04:02.000 2020-01-02 03:32:23.000 242 99143 49692.5 4969250 242 99143 49692.5 4969250 -32327 32608 5128.02 512802 -128 123 -2.22 -222 +243 2 10233 99144 0.72972 297.72972 149.22972 14922.97297 0.72972 297.72974 149.22973 14922.97332 0.72972 297.72972 149.2297199999999 14922.97200 2020-01-01 2020-01-02 2020-01-01 00:04:03 2020-01-02 03:32:24 2020-01-01 00:04:03.000 2020-01-02 03:32:24.000 243 99144 49693.5 4969350 243 99144 49693.5 4969350 -32326 32609 5129.02 512902 -127 124 -1.22 -122 +244 2 10234 99145 0.73273 297.73273 149.23273 14923.27327 0.73273 297.73273 149.23272 14923.27297 0.73273 297.73273 149.23272999999986 14923.27300 2020-01-01 2020-01-02 2020-01-01 00:04:04 2020-01-02 03:32:25 2020-01-01 00:04:04.000 2020-01-02 03:32:25.000 244 99145 49694.5 4969450 244 99145 49694.5 4969450 -32325 32610 5130.02 513002 -126 125 -0.22 -22 +245 2 10235 99146 0.73573 297.73573 149.23573 14923.57357 0.73573 297.73575 149.23573 14923.57358 0.73573 297.73573 149.23573000000016 14923.57300 2020-01-01 2020-01-02 2020-01-01 00:04:05 2020-01-02 03:32:26 2020-01-01 00:04:05.000 2020-01-02 03:32:26.000 245 99146 49695.5 4969550 245 99146 49695.5 4969550 -32324 32611 5131.02 513102 -125 126 0.78 78 +246 2 10236 99147 0.73873 297.73873 149.23873 14923.87387 0.73873 297.73874 149.23873 14923.8739 0.73873 297.73873 149.2387299999998 14923.87300 2020-01-01 2020-01-02 2020-01-01 00:04:06 2020-01-02 03:32:27 2020-01-01 00:04:06.000 2020-01-02 03:32:27.000 246 99147 49696.5 4969650 246 99147 49696.5 4969650 -32323 32612 5132.02 513202 -124 127 1.78 178 247 2 10237 99148 0.74174 297.74174 149.24174 14924.17417 0.74174 297.74173 149.24174 14924.1742 0.74174 297.74174 149.24174 14924.17400 2020-01-01 2020-01-02 2020-01-01 00:04:07 2020-01-02 03:32:28 2020-01-01 00:04:07.000 2020-01-02 03:32:28.000 247 99148 49697.5 4969750 247 99148 49697.5 4969750 -32322 32613 5133.02 513302 -128 127 0.22 22 -248 2 10238 99149 0.74474 297.74474 149.24474 14924.47447 0.74474 297.74475 149.24474 14924.47478 0.74474 297.74474 149.24474 14924.47400 2020-01-01 2020-01-02 2020-01-01 00:04:08 2020-01-02 03:32:29 2020-01-01 00:04:08.000 2020-01-02 03:32:29.000 248 99149 49698.5 4969850 248 99149 49698.5 4969850 -32321 32614 5134.02 513402 -128 127 -1.34 -134 -249 2 10239 99150 0.74774 297.74774 149.24774 14924.77477 0.74774 297.74774 149.24774 14924.77447 0.74774 297.74774 149.24774 14924.77400 2020-01-01 2020-01-02 2020-01-01 00:04:09 2020-01-02 03:32:30 2020-01-01 00:04:09.000 2020-01-02 03:32:30.000 249 99150 49699.5 4969950 249 99150 49699.5 4969950 -32320 32615 5135.02 513502 -128 124 -2.9 -290 -25 2 10015 99925 0.07507 300.07507 150.07507 15157.58258 0.07507 300.07507 150.07507 15157.58241 0.07507 300.07507 150.07507 15157.58207 2020-01-01 2020-01-02 2020-01-01 00:00:25 2020-01-02 03:45:25 2020-01-01 00:00:25.000 2020-01-02 03:45:25.000 25 99925 49975 5047475 25 99925 49975 5047475 -32544 32391 4554.009900990099 459955 -127 124 -3.297029702970297 -333 -250 2 10240 99151 0.75075 297.75075 149.25075 14925.07507 0.75075 297.75076 149.25075 14925.07506 0.75075 297.75075 149.25075 14925.07500 2020-01-01 2020-01-02 2020-01-01 00:04:10 2020-01-02 03:32:31 2020-01-01 00:04:10.000 2020-01-02 03:32:31.000 250 99151 49700.5 4970050 250 99151 49700.5 4970050 -32319 32616 5136.02 513602 -127 125 -1.9 -190 -251 2 10241 99152 0.75375 297.75375 149.25375 14925.37537 0.75375 297.75375 149.25375 14925.37536 0.75375 297.75375 149.25375 14925.37500 2020-01-01 2020-01-02 2020-01-01 00:04:11 2020-01-02 03:32:32 2020-01-01 00:04:11.000 2020-01-02 03:32:32.000 251 99152 49701.5 4970150 251 99152 49701.5 4970150 -32318 32617 5137.02 513702 -126 126 -0.9 -90 -252 2 10242 99153 0.75675 297.75675 149.25675 14925.67567 0.75675 297.75674 149.25675 14925.67567 0.75675 297.75675 149.25675 14925.67500 2020-01-01 2020-01-02 2020-01-01 00:04:12 2020-01-02 03:32:33 2020-01-01 00:04:12.000 2020-01-02 03:32:33.000 252 99153 49702.5 4970250 252 99153 49702.5 4970250 -32317 32618 5138.02 513802 -125 127 0.1 10 -253 2 10243 99154 0.75975 297.75975 149.25975 14925.97597 0.75975 297.75977 149.25976 14925.97625 0.75975 297.75975 149.25975 14925.97500 2020-01-01 2020-01-02 2020-01-01 00:04:13 2020-01-02 03:32:34 2020-01-01 00:04:13.000 2020-01-02 03:32:34.000 253 99154 49703.5 4970350 253 99154 49703.5 4970350 -32316 32619 5139.02 513902 -128 127 -1.46 -146 -254 2 10244 99155 0.76276 297.76276 149.26276 14926.27627 0.76276 297.76276 149.26275 14926.27594 0.76276 297.76276 149.26276 14926.27600 2020-01-01 2020-01-02 2020-01-01 00:04:14 2020-01-02 03:32:35 2020-01-01 00:04:14.000 2020-01-02 03:32:35.000 254 99155 49704.5 4970450 254 99155 49704.5 4970450 -32315 32620 5140.02 514002 -128 127 -3.02 -302 -255 2 10245 99156 0.76576 297.76576 149.26576 14926.57657 0.76576 297.76578 149.26576 14926.57652 0.76576 297.76576 149.26576 14926.57600 2020-01-01 2020-01-02 2020-01-01 00:04:15 2020-01-02 03:32:36 2020-01-01 00:04:15.000 2020-01-02 03:32:36.000 255 99156 49705.5 4970550 255 99156 49705.5 4970550 -32314 32621 5141.02 514102 -128 123 -4.58 -458 -256 2 10246 99157 0.76876 297.76876 149.26876 14926.87687 0.76876 297.76877 149.26876 14926.87683 0.76876 297.76876 149.26876 14926.87600 2020-01-01 2020-01-02 2020-01-01 00:04:16 2020-01-02 03:32:37 2020-01-01 00:04:16.000 2020-01-02 03:32:37.000 256 99157 49706.5 4970650 256 99157 49706.5 4970650 -32313 32622 5142.02 514202 -127 124 -3.58 -358 -257 2 10247 99158 0.77177 297.77177 149.27177 14927.17717 0.77177 297.77176 149.27177 14927.17714 0.77177 297.77177 149.27177 14927.17700 2020-01-01 2020-01-02 2020-01-01 00:04:17 2020-01-02 03:32:38 2020-01-01 00:04:17.000 2020-01-02 03:32:38.000 257 99158 49707.5 4970750 257 99158 49707.5 4970750 -32312 32623 5143.02 514302 -126 125 -2.58 -258 +248 2 10238 99149 0.74474 297.74474 149.24474 14924.47447 0.74474 297.74475 149.24474 14924.47478 0.74474 297.74474 149.24474000000006 14924.47400 2020-01-01 2020-01-02 2020-01-01 00:04:08 2020-01-02 03:32:29 2020-01-01 00:04:08.000 2020-01-02 03:32:29.000 248 99149 49698.5 4969850 248 99149 49698.5 4969850 -32321 32614 5134.02 513402 -128 127 -1.34 -134 +249 2 10239 99150 0.74774 297.74774 149.24774 14924.77477 0.74774 297.74774 149.24774 14924.77447 0.74774 297.74774 149.24774000000005 14924.77400 2020-01-01 2020-01-02 2020-01-01 00:04:09 2020-01-02 03:32:30 2020-01-01 00:04:09.000 2020-01-02 03:32:30.000 249 99150 49699.5 4969950 249 99150 49699.5 4969950 -32320 32615 5135.02 513502 -128 124 -2.9 -290 +25 2 10015 99925 0.07507 300.07507 150.07507 15157.58258 0.07507 300.07507 150.07507 15157.58241 0.07507 300.07507 150.07507000000012 15157.58207 2020-01-01 2020-01-02 2020-01-01 00:00:25 2020-01-02 03:45:25 2020-01-01 00:00:25.000 2020-01-02 03:45:25.000 25 99925 49975 5047475 25 99925 49975 5047475 -32544 32391 4554.009900990099 459955 -127 124 -3.297029702970297 -333 +250 2 10240 99151 0.75075 297.75075 149.25075 14925.07507 0.75075 297.75076 149.25075 14925.07506 0.75075 297.75075 149.25074999999987 14925.07500 2020-01-01 2020-01-02 2020-01-01 00:04:10 2020-01-02 03:32:31 2020-01-01 00:04:10.000 2020-01-02 03:32:31.000 250 99151 49700.5 4970050 250 99151 49700.5 4970050 -32319 32616 5136.02 513602 -127 125 -1.9 -190 +251 2 10241 99152 0.75375 297.75375 149.25375 14925.37537 0.75375 297.75375 149.25375 14925.37536 0.75375 297.75375 149.2537499999999 14925.37500 2020-01-01 2020-01-02 2020-01-01 00:04:11 2020-01-02 03:32:32 2020-01-01 00:04:11.000 2020-01-02 03:32:32.000 251 99152 49701.5 4970150 251 99152 49701.5 4970150 -32318 32617 5137.02 513702 -126 126 -0.9 -90 +252 2 10242 99153 0.75675 297.75675 149.25675 14925.67567 0.75675 297.75674 149.25675 14925.67567 0.75675 297.75675 149.2567500000001 14925.67500 2020-01-01 2020-01-02 2020-01-01 00:04:12 2020-01-02 03:32:33 2020-01-01 00:04:12.000 2020-01-02 03:32:33.000 252 99153 49702.5 4970250 252 99153 49702.5 4970250 -32317 32618 5138.02 513802 -125 127 0.1 10 +253 2 10243 99154 0.75975 297.75975 149.25975 14925.97597 0.75975 297.75977 149.25976 14925.97625 0.75975 297.75975 149.25974999999988 14925.97500 2020-01-01 2020-01-02 2020-01-01 00:04:13 2020-01-02 03:32:34 2020-01-01 00:04:13.000 2020-01-02 03:32:34.000 253 99154 49703.5 4970350 253 99154 49703.5 4970350 -32316 32619 5139.02 513902 -128 127 -1.46 -146 +254 2 10244 99155 0.76276 297.76276 149.26276 14926.27627 0.76276 297.76276 149.26275 14926.27594 0.76276 297.76276 149.2627599999998 14926.27600 2020-01-01 2020-01-02 2020-01-01 00:04:14 2020-01-02 03:32:35 2020-01-01 00:04:14.000 2020-01-02 03:32:35.000 254 99155 49704.5 4970450 254 99155 49704.5 4970450 -32315 32620 5140.02 514002 -128 127 -3.02 -302 +255 2 10245 99156 0.76576 297.76576 149.26576 14926.57657 0.76576 297.76578 149.26576 14926.57652 0.76576 297.76576 149.26576000000017 14926.57600 2020-01-01 2020-01-02 2020-01-01 00:04:15 2020-01-02 03:32:36 2020-01-01 00:04:15.000 2020-01-02 03:32:36.000 255 99156 49705.5 4970550 255 99156 49705.5 4970550 -32314 32621 5141.02 514102 -128 123 -4.58 -458 +256 2 10246 99157 0.76876 297.76876 149.26876 14926.87687 0.76876 297.76877 149.26876 14926.87683 0.76876 297.76876 149.26876000000016 14926.87600 2020-01-01 2020-01-02 2020-01-01 00:04:16 2020-01-02 03:32:37 2020-01-01 00:04:16.000 2020-01-02 03:32:37.000 256 99157 49706.5 4970650 256 99157 49706.5 4970650 -32313 32622 5142.02 514202 -127 124 -3.58 -358 +257 2 10247 99158 0.77177 297.77177 149.27177 14927.17717 0.77177 297.77176 149.27177 14927.17714 0.77177 297.77177 149.27176999999992 14927.17700 2020-01-01 2020-01-02 2020-01-01 00:04:17 2020-01-02 03:32:38 2020-01-01 00:04:17.000 2020-01-02 03:32:38.000 257 99158 49707.5 4970750 257 99158 49707.5 4970750 -32312 32623 5143.02 514302 -126 125 -2.58 -258 258 2 10248 99159 0.77477 297.77477 149.27477 14927.47747 0.77477 297.77478 149.27477 14927.47776 0.77477 297.77477 149.27477 14927.47700 2020-01-01 2020-01-02 2020-01-01 00:04:18 2020-01-02 03:32:39 2020-01-01 00:04:18.000 2020-01-02 03:32:39.000 258 99159 49708.5 4970850 258 99159 49708.5 4970850 -32311 32624 5144.02 514402 -125 126 -1.58 -158 -259 2 10249 99160 0.77777 297.77777 149.27777 14927.77777 0.77777 297.77777 149.27777 14927.77742 0.77777 297.77777 149.27777 14927.77700 2020-01-01 2020-01-02 2020-01-01 00:04:19 2020-01-02 03:32:40 2020-01-01 00:04:19.000 2020-01-02 03:32:40.000 259 99160 49709.5 4970950 259 99160 49709.5 4970950 -32310 32625 5145.02 514502 -124 127 -0.58 -58 -26 2 10016 99926 0.07807 300.07807 150.07807 15157.88588 0.07807 300.07806 150.07807 15157.88575 0.07807 300.07807 150.07807 15157.88507 2020-01-01 2020-01-02 2020-01-01 00:00:26 2020-01-02 03:45:26 2020-01-01 00:00:26.000 2020-01-02 03:45:26.000 26 99926 49976 5047576 26 99926 49976 5047576 -32543 32392 4555.009900990099 460056 -126 125 -2.297029702970297 -232 -260 2 10250 99161 0.78078 297.78078 149.28078 14928.07807 0.78078 297.7808 149.28077 14928.07799 0.78078 297.78078 149.28078 14928.07800 2020-01-01 2020-01-02 2020-01-01 00:04:20 2020-01-02 03:32:41 2020-01-01 00:04:20.000 2020-01-02 03:32:41.000 260 99161 49710.5 4971050 260 99161 49710.5 4971050 -32309 32626 5146.02 514602 -128 127 -2.14 -214 -261 2 10251 99162 0.78378 297.78378 149.28378 14928.37837 0.78378 297.78378 149.28378 14928.3783 0.78378 297.78378 149.28378 14928.37800 2020-01-01 2020-01-02 2020-01-01 00:04:21 2020-01-02 03:32:42 2020-01-01 00:04:21.000 2020-01-02 03:32:42.000 261 99162 49711.5 4971150 261 99162 49711.5 4971150 -32308 32627 5147.02 514702 -128 123 -3.7 -370 -262 2 10252 99163 0.78678 297.78678 149.28678 14928.67867 0.78678 297.78677 149.28678 14928.67861 0.78678 297.78678 149.28678 14928.67800 2020-01-01 2020-01-02 2020-01-01 00:04:22 2020-01-02 03:32:43 2020-01-01 00:04:22.000 2020-01-02 03:32:43.000 262 99163 49712.5 4971250 262 99163 49712.5 4971250 -32307 32628 5148.02 514802 -127 124 -2.7 -270 -263 2 10253 99164 0.78978 297.78978 149.28978 14928.97897 0.78978 297.7898 149.28979 14928.97923 0.78978 297.78978 149.28978 14928.97800 2020-01-01 2020-01-02 2020-01-01 00:04:23 2020-01-02 03:32:44 2020-01-01 00:04:23.000 2020-01-02 03:32:44.000 263 99164 49713.5 4971350 263 99164 49713.5 4971350 -32306 32629 5149.02 514902 -126 125 -1.7 -170 +259 2 10249 99160 0.77777 297.77777 149.27777 14927.77777 0.77777 297.77777 149.27777 14927.77742 0.77777 297.77777 149.27777000000006 14927.77700 2020-01-01 2020-01-02 2020-01-01 00:04:19 2020-01-02 03:32:40 2020-01-01 00:04:19.000 2020-01-02 03:32:40.000 259 99160 49709.5 4970950 259 99160 49709.5 4970950 -32310 32625 5145.02 514502 -124 127 -0.58 -58 +26 2 10016 99926 0.07807 300.07807 150.07807 15157.88588 0.07807 300.07806 150.07807 15157.88575 0.07807 300.07807 150.0780699999998 15157.88507 2020-01-01 2020-01-02 2020-01-01 00:00:26 2020-01-02 03:45:26 2020-01-01 00:00:26.000 2020-01-02 03:45:26.000 26 99926 49976 5047576 26 99926 49976 5047576 -32543 32392 4555.009900990099 460056 -126 125 -2.297029702970297 -232 +260 2 10250 99161 0.78078 297.78078 149.28078 14928.07807 0.78078 297.7808 149.28077 14928.07799 0.78078 297.78078 149.28077999999982 14928.07800 2020-01-01 2020-01-02 2020-01-01 00:04:20 2020-01-02 03:32:41 2020-01-01 00:04:20.000 2020-01-02 03:32:41.000 260 99161 49710.5 4971050 260 99161 49710.5 4971050 -32309 32626 5146.02 514602 -128 127 -2.14 -214 +261 2 10251 99162 0.78378 297.78378 149.28378 14928.37837 0.78378 297.78378 149.28378 14928.3783 0.78378 297.78378 149.28377999999987 14928.37800 2020-01-01 2020-01-02 2020-01-01 00:04:21 2020-01-02 03:32:42 2020-01-01 00:04:21.000 2020-01-02 03:32:42.000 261 99162 49711.5 4971150 261 99162 49711.5 4971150 -32308 32627 5147.02 514702 -128 123 -3.7 -370 +262 2 10252 99163 0.78678 297.78678 149.28678 14928.67867 0.78678 297.78677 149.28678 14928.67861 0.78678 297.78678 149.2867800000002 14928.67800 2020-01-01 2020-01-02 2020-01-01 00:04:22 2020-01-02 03:32:43 2020-01-01 00:04:22.000 2020-01-02 03:32:43.000 262 99163 49712.5 4971250 262 99163 49712.5 4971250 -32307 32628 5148.02 514802 -127 124 -2.7 -270 +263 2 10253 99164 0.78978 297.78978 149.28978 14928.97897 0.78978 297.7898 149.28979 14928.97923 0.78978 297.78978 149.2897799999999 14928.97800 2020-01-01 2020-01-02 2020-01-01 00:04:23 2020-01-02 03:32:44 2020-01-01 00:04:23.000 2020-01-02 03:32:44.000 263 99164 49713.5 4971350 263 99164 49713.5 4971350 -32306 32629 5149.02 514902 -126 125 -1.7 -170 264 2 10254 99165 0.79279 297.79279 149.29279 14929.27927 0.79279 297.7928 149.29278 14929.27888 0.79279 297.79279 149.29279 14929.27900 2020-01-01 2020-01-02 2020-01-01 00:04:24 2020-01-02 03:32:45 2020-01-01 00:04:24.000 2020-01-02 03:32:45.000 264 99165 49714.5 4971450 264 99165 49714.5 4971450 -32305 32630 5150.02 515002 -125 126 -0.7 -70 -265 2 10255 99166 0.79579 297.79579 149.29579 14929.57957 0.79579 297.7958 149.29579 14929.57962 0.79579 297.79579 149.29579 14929.57900 2020-01-01 2020-01-02 2020-01-01 00:04:25 2020-01-02 03:32:46 2020-01-01 00:04:25.000 2020-01-02 03:32:46.000 265 99166 49715.5 4971550 265 99166 49715.5 4971550 -32304 32631 5151.02 515102 -124 127 0.3 30 -266 2 10256 99167 0.79879 297.79879 149.29879 14929.87987 0.79879 297.7988 149.29879 14929.87977 0.79879 297.79879 149.29879 14929.87900 2020-01-01 2020-01-02 2020-01-01 00:04:26 2020-01-02 03:32:47 2020-01-01 00:04:26.000 2020-01-02 03:32:47.000 266 99167 49716.5 4971650 266 99167 49716.5 4971650 -32303 32632 5152.02 515202 -128 127 -1.26 -126 -267 2 10257 99168 0.8018 297.8018 149.3018 14930.18018 0.8018 297.8018 149.3018 14930.18012 0.80180 297.80180 149.30180 14930.18000 2020-01-01 2020-01-02 2020-01-01 00:04:27 2020-01-02 03:32:48 2020-01-01 00:04:27.000 2020-01-02 03:32:48.000 267 99168 49717.5 4971750 267 99168 49717.5 4971750 -32302 32633 5153.02 515302 -128 123 -2.82 -282 -268 2 10258 99169 0.8048 297.8048 149.3048 14930.48048 0.8048 297.8048 149.3048 14930.4807 0.80480 297.80480 149.30480 14930.48000 2020-01-01 2020-01-02 2020-01-01 00:04:28 2020-01-02 03:32:49 2020-01-01 00:04:28.000 2020-01-02 03:32:49.000 268 99169 49718.5 4971850 268 99169 49718.5 4971850 -32301 32634 5154.02 515402 -127 124 -1.82 -182 -269 2 10259 99170 0.8078 297.8078 149.3078 14930.78078 0.8078 297.8078 149.3078 14930.78035 0.80780 297.80780 149.30780 14930.78000 2020-01-01 2020-01-02 2020-01-01 00:04:29 2020-01-02 03:32:50 2020-01-01 00:04:29.000 2020-01-02 03:32:50.000 269 99170 49719.5 4971950 269 99170 49719.5 4971950 -32300 32635 5155.02 515502 -126 125 -0.82 -82 -27 2 10017 99927 0.08108 300.08108 150.08108 15158.18918 0.08108 300.0811 150.08108 15158.18936 0.08108 300.08108 150.08108 15158.18908 2020-01-01 2020-01-02 2020-01-01 00:00:27 2020-01-02 03:45:27 2020-01-01 00:00:27.000 2020-01-02 03:45:27.000 27 99927 49977 5047677 27 99927 49977 5047677 -32542 32393 4556.009900990099 460157 -125 126 -1.297029702970297 -131 -270 2 10260 99171 0.81081 297.81081 149.31081 14931.08108 0.81081 297.81082 149.31081 14931.08109 0.81081 297.81081 149.31081 14931.08100 2020-01-01 2020-01-02 2020-01-01 00:04:30 2020-01-02 03:32:51 2020-01-01 00:04:30.000 2020-01-02 03:32:51.000 270 99171 49720.5 4972050 270 99171 49720.5 4972050 -32299 32636 5156.02 515602 -125 126 0.18 18 -271 2 10261 99172 0.81381 297.81381 149.31381 14931.38138 0.81381 297.8138 149.31381 14931.38124 0.81381 297.81381 149.31381 14931.38100 2020-01-01 2020-01-02 2020-01-01 00:04:31 2020-01-02 03:32:52 2020-01-01 00:04:31.000 2020-01-02 03:32:52.000 271 99172 49721.5 4972150 271 99172 49721.5 4972150 -32298 32637 5157.02 515702 -124 127 1.18 118 -272 2 10262 99173 0.81681 297.81681 149.31681 14931.68168 0.81681 297.8168 149.31681 14931.68159 0.81681 297.81681 149.31681 14931.68100 2020-01-01 2020-01-02 2020-01-01 00:04:32 2020-01-02 03:32:53 2020-01-01 00:04:32.000 2020-01-02 03:32:53.000 272 99173 49722.5 4972250 272 99173 49722.5 4972250 -32297 32638 5158.02 515802 -128 127 -0.38 -38 -273 2 10263 99174 0.81981 297.81981 149.31981 14931.98198 0.81981 297.81982 149.31982 14931.98217 0.81981 297.81981 149.31981 14931.98100 2020-01-01 2020-01-02 2020-01-01 00:04:33 2020-01-02 03:32:54 2020-01-01 00:04:33.000 2020-01-02 03:32:54.000 273 99174 49723.5 4972350 273 99174 49723.5 4972350 -32296 32639 5159.02 515902 -128 127 -1.94 -194 -274 2 10264 99175 0.82282 297.82282 149.32282 14932.28228 0.82282 297.8228 149.32282 14932.28247 0.82282 297.82282 149.32282 14932.28200 2020-01-01 2020-01-02 2020-01-01 00:04:34 2020-01-02 03:32:55 2020-01-01 00:04:34.000 2020-01-02 03:32:55.000 274 99175 49724.5 4972450 274 99175 49724.5 4972450 -32295 32640 5160.02 516002 -128 124 -3.5 -350 +265 2 10255 99166 0.79579 297.79579 149.29579 14929.57957 0.79579 297.7958 149.29579 14929.57962 0.79579 297.79579 149.2957900000001 14929.57900 2020-01-01 2020-01-02 2020-01-01 00:04:25 2020-01-02 03:32:46 2020-01-01 00:04:25.000 2020-01-02 03:32:46.000 265 99166 49715.5 4971550 265 99166 49715.5 4971550 -32304 32631 5151.02 515102 -124 127 0.3 30 +266 2 10256 99167 0.79879 297.79879 149.29879 14929.87987 0.79879 297.7988 149.29879 14929.87977 0.79879 297.79879 149.2987900000001 14929.87900 2020-01-01 2020-01-02 2020-01-01 00:04:26 2020-01-02 03:32:47 2020-01-01 00:04:26.000 2020-01-02 03:32:47.000 266 99167 49716.5 4971650 266 99167 49716.5 4971650 -32303 32632 5152.02 515202 -128 127 -1.26 -126 +267 2 10257 99168 0.8018 297.8018 149.3018 14930.18018 0.8018 297.8018 149.3018 14930.18012 0.80180 297.80180 149.3017999999999 14930.18000 2020-01-01 2020-01-02 2020-01-01 00:04:27 2020-01-02 03:32:48 2020-01-01 00:04:27.000 2020-01-02 03:32:48.000 267 99168 49717.5 4971750 267 99168 49717.5 4971750 -32302 32633 5153.02 515302 -128 123 -2.82 -282 +268 2 10258 99169 0.8048 297.8048 149.3048 14930.48048 0.8048 297.8048 149.3048 14930.4807 0.80480 297.80480 149.30479999999991 14930.48000 2020-01-01 2020-01-02 2020-01-01 00:04:28 2020-01-02 03:32:49 2020-01-01 00:04:28.000 2020-01-02 03:32:49.000 268 99169 49718.5 4971850 268 99169 49718.5 4971850 -32301 32634 5154.02 515402 -127 124 -1.82 -182 +269 2 10259 99170 0.8078 297.8078 149.3078 14930.78078 0.8078 297.8078 149.3078 14930.78035 0.80780 297.80780 149.3078 14930.78000 2020-01-01 2020-01-02 2020-01-01 00:04:29 2020-01-02 03:32:50 2020-01-01 00:04:29.000 2020-01-02 03:32:50.000 269 99170 49719.5 4971950 269 99170 49719.5 4971950 -32300 32635 5155.02 515502 -126 125 -0.82 -82 +27 2 10017 99927 0.08108 300.08108 150.08108 15158.18918 0.08108 300.0811 150.08108 15158.18936 0.08108 300.08108 150.08107999999996 15158.18908 2020-01-01 2020-01-02 2020-01-01 00:00:27 2020-01-02 03:45:27 2020-01-01 00:00:27.000 2020-01-02 03:45:27.000 27 99927 49977 5047677 27 99927 49977 5047677 -32542 32393 4556.009900990099 460157 -125 126 -1.297029702970297 -131 +270 2 10260 99171 0.81081 297.81081 149.31081 14931.08108 0.81081 297.81082 149.31081 14931.08109 0.81081 297.81081 149.31081000000017 14931.08100 2020-01-01 2020-01-02 2020-01-01 00:04:30 2020-01-02 03:32:51 2020-01-01 00:04:30.000 2020-01-02 03:32:51.000 270 99171 49720.5 4972050 270 99171 49720.5 4972050 -32299 32636 5156.02 515602 -125 126 0.18 18 +271 2 10261 99172 0.81381 297.81381 149.31381 14931.38138 0.81381 297.8138 149.31381 14931.38124 0.81381 297.81381 149.31380999999985 14931.38100 2020-01-01 2020-01-02 2020-01-01 00:04:31 2020-01-02 03:32:52 2020-01-01 00:04:31.000 2020-01-02 03:32:52.000 271 99172 49721.5 4972150 271 99172 49721.5 4972150 -32298 32637 5157.02 515702 -124 127 1.18 118 +272 2 10262 99173 0.81681 297.81681 149.31681 14931.68168 0.81681 297.8168 149.31681 14931.68159 0.81681 297.81681 149.31681000000012 14931.68100 2020-01-01 2020-01-02 2020-01-01 00:04:32 2020-01-02 03:32:53 2020-01-01 00:04:32.000 2020-01-02 03:32:53.000 272 99173 49722.5 4972250 272 99173 49722.5 4972250 -32297 32638 5158.02 515802 -128 127 -0.38 -38 +273 2 10263 99174 0.81981 297.81981 149.31981 14931.98198 0.81981 297.81982 149.31982 14931.98217 0.81981 297.81981 149.3198100000002 14931.98100 2020-01-01 2020-01-02 2020-01-01 00:04:33 2020-01-02 03:32:54 2020-01-01 00:04:33.000 2020-01-02 03:32:54.000 273 99174 49723.5 4972350 273 99174 49723.5 4972350 -32296 32639 5159.02 515902 -128 127 -1.94 -194 +274 2 10264 99175 0.82282 297.82282 149.32282 14932.28228 0.82282 297.8228 149.32282 14932.28247 0.82282 297.82282 149.3228199999999 14932.28200 2020-01-01 2020-01-02 2020-01-01 00:04:34 2020-01-02 03:32:55 2020-01-01 00:04:34.000 2020-01-02 03:32:55.000 274 99175 49724.5 4972450 274 99175 49724.5 4972450 -32295 32640 5160.02 516002 -128 124 -3.5 -350 275 2 10265 99176 0.82582 297.82582 149.32582 14932.58258 0.82582 297.82584 149.32582 14932.58256 0.82582 297.82582 149.32582 14932.58200 2020-01-01 2020-01-02 2020-01-01 00:04:35 2020-01-02 03:32:56 2020-01-01 00:04:35.000 2020-01-02 03:32:56.000 275 99176 49725.5 4972550 275 99176 49725.5 4972550 -32294 32641 5161.02 516102 -127 125 -2.5 -250 -276 2 10266 99177 0.82882 297.82882 149.32882 14932.88288 0.82882 297.82883 149.32882 14932.88275 0.82882 297.82882 149.32882 14932.88200 2020-01-01 2020-01-02 2020-01-01 00:04:36 2020-01-02 03:32:57 2020-01-01 00:04:36.000 2020-01-02 03:32:57.000 276 99177 49726.5 4972650 276 99177 49726.5 4972650 -32293 32642 5162.02 516202 -126 126 -1.5 -150 -277 2 10267 99178 0.83183 297.83183 149.33183 14933.18318 0.83183 297.83182 149.33183 14933.18305 0.83183 297.83183 149.33183 14933.18300 2020-01-01 2020-01-02 2020-01-01 00:04:37 2020-01-02 03:32:58 2020-01-01 00:04:37.000 2020-01-02 03:32:58.000 277 99178 49727.5 4972750 277 99178 49727.5 4972750 -32292 32643 5163.02 516302 -125 127 -0.5 -50 -278 2 10268 99179 0.83483 297.83483 149.33483 14933.48348 0.83483 297.83484 149.33483 14933.48364 0.83483 297.83483 149.33483 14933.48300 2020-01-01 2020-01-02 2020-01-01 00:04:38 2020-01-02 03:32:59 2020-01-01 00:04:38.000 2020-01-02 03:32:59.000 278 99179 49728.5 4972850 278 99179 49728.5 4972850 -32291 32644 5164.02 516402 -128 127 -2.06 -206 -279 2 10269 99180 0.83783 297.83783 149.33783 14933.78378 0.83783 297.83783 149.33783 14933.78394 0.83783 297.83783 149.33783 14933.78300 2020-01-01 2020-01-02 2020-01-01 00:04:39 2020-01-02 03:33:00 2020-01-01 00:04:39.000 2020-01-02 03:33:00.000 279 99180 49729.5 4972950 279 99180 49729.5 4972950 -32290 32645 5165.02 516502 -128 127 -3.62 -362 -28 2 10018 99928 0.08408 300.08408 150.08408 15158.49249 0.08408 300.08408 150.08408 15158.49265 0.08408 300.08408 150.08408 15158.49208 2020-01-01 2020-01-02 2020-01-01 00:00:28 2020-01-02 03:45:28 2020-01-01 00:00:28.000 2020-01-02 03:45:28.000 28 99928 49978 5047778 28 99928 49978 5047778 -32541 32394 4557.009900990099 460258 -124 127 -0.297029702970297 -30 -280 2 10270 99181 0.84084 297.84084 149.34084 14934.08408 0.84084 297.84085 149.34084 14934.08403 0.84084 297.84084 149.34084 14934.08400 2020-01-01 2020-01-02 2020-01-01 00:04:40 2020-01-02 03:33:01 2020-01-01 00:04:40.000 2020-01-02 03:33:01.000 280 99181 49730.5 4973050 280 99181 49730.5 4973050 -32289 32646 5166.02 516602 -128 123 -5.18 -518 -281 2 10271 99182 0.84384 297.84384 149.34384 14934.38438 0.84384 297.84384 149.34384 14934.38421 0.84384 297.84384 149.34384 14934.38400 2020-01-01 2020-01-02 2020-01-01 00:04:41 2020-01-02 03:33:02 2020-01-01 00:04:41.000 2020-01-02 03:33:02.000 281 99182 49731.5 4973150 281 99182 49731.5 4973150 -32288 32647 5167.02 516702 -127 124 -4.18 -418 -282 2 10272 99183 0.84684 297.84684 149.34684 14934.68468 0.84684 297.84683 149.34684 14934.68453 0.84684 297.84684 149.34684 14934.68400 2020-01-01 2020-01-02 2020-01-01 00:04:42 2020-01-02 03:33:03 2020-01-01 00:04:42.000 2020-01-02 03:33:03.000 282 99183 49732.5 4973250 282 99183 49732.5 4973250 -32287 32648 5168.02 516802 -126 125 -3.18 -318 -283 2 10273 99184 0.84984 297.84984 149.34984 14934.98498 0.84984 297.84985 149.34985 14934.98526 0.84984 297.84984 149.34984 14934.98400 2020-01-01 2020-01-02 2020-01-01 00:04:43 2020-01-02 03:33:04 2020-01-01 00:04:43.000 2020-01-02 03:33:04.000 283 99184 49733.5 4973350 283 99184 49733.5 4973350 -32286 32649 5169.02 516902 -125 126 -2.18 -218 -284 2 10274 99185 0.85285 297.85285 149.35285 14935.28528 0.85285 297.85284 149.35285 14935.28542 0.85285 297.85285 149.35285 14935.28500 2020-01-01 2020-01-02 2020-01-01 00:04:44 2020-01-02 03:33:05 2020-01-01 00:04:44.000 2020-01-02 03:33:05.000 284 99185 49734.5 4973450 284 99185 49734.5 4973450 -32285 32650 5170.02 517002 -124 127 -1.18 -118 -285 2 10275 99186 0.85585 297.85585 149.35585 14935.58558 0.85585 297.85587 149.35585 14935.5855 0.85585 297.85585 149.35585 14935.58500 2020-01-01 2020-01-02 2020-01-01 00:04:45 2020-01-02 03:33:06 2020-01-01 00:04:45.000 2020-01-02 03:33:06.000 285 99186 49735.5 4973550 285 99186 49735.5 4973550 -32284 32651 5171.02 517102 -128 127 -2.74 -274 -286 2 10276 99187 0.85885 297.85885 149.35885 14935.88588 0.85885 297.85886 149.35885 14935.88568 0.85885 297.85885 149.35885 14935.88500 2020-01-01 2020-01-02 2020-01-01 00:04:46 2020-01-02 03:33:07 2020-01-01 00:04:46.000 2020-01-02 03:33:07.000 286 99187 49736.5 4973650 286 99187 49736.5 4973650 -32283 32652 5172.02 517202 -128 123 -4.3 -430 -287 2 10277 99188 0.86186 297.86186 149.36186 14936.18618 0.86186 297.86185 149.36186 14936.186 0.86186 297.86186 149.36186 14936.18600 2020-01-01 2020-01-02 2020-01-01 00:04:47 2020-01-02 03:33:08 2020-01-01 00:04:47.000 2020-01-02 03:33:08.000 287 99188 49737.5 4973750 287 99188 49737.5 4973750 -32282 32653 5173.02 517302 -127 124 -3.3 -330 -288 2 10278 99189 0.86486 297.86486 149.36486 14936.48648 0.86486 297.86487 149.36486 14936.48673 0.86486 297.86486 149.36486 14936.48600 2020-01-01 2020-01-02 2020-01-01 00:04:48 2020-01-02 03:33:09 2020-01-01 00:04:48.000 2020-01-02 03:33:09.000 288 99189 49738.5 4973850 288 99189 49738.5 4973850 -32281 32654 5174.02 517402 -126 125 -2.3 -230 -289 2 10279 99190 0.86786 297.86786 149.36786 14936.78678 0.86786 297.86786 149.36786 14936.78688 0.86786 297.86786 149.36786 14936.78600 2020-01-01 2020-01-02 2020-01-01 00:04:49 2020-01-02 03:33:10 2020-01-01 00:04:49.000 2020-01-02 03:33:10.000 289 99190 49739.5 4973950 289 99190 49739.5 4973950 -32280 32655 5175.02 517502 -125 126 -1.3 -130 -29 2 10019 99929 0.08708 300.08708 150.08708 15158.79579 0.08708 300.0871 150.08708 15158.79576 0.08708 300.08708 150.08708 15158.79508 2020-01-01 2020-01-02 2020-01-01 00:00:29 2020-01-02 03:45:29 2020-01-01 00:00:29.000 2020-01-02 03:45:29.000 29 99929 49979 5047879 29 99929 49979 5047879 -32540 32395 4558.009900990099 460359 -128 127 -1.8316831683168318 -185 -290 2 10280 99191 0.87087 297.87087 149.37087 14937.08708 0.87087 297.87088 149.37087 14937.087 0.87087 297.87087 149.37087 14937.08700 2020-01-01 2020-01-02 2020-01-01 00:04:50 2020-01-02 03:33:11 2020-01-01 00:04:50.000 2020-01-02 03:33:11.000 290 99191 49740.5 4974050 290 99191 49740.5 4974050 -32279 32656 5176.02 517602 -124 127 -0.3 -30 -291 2 10281 99192 0.87387 297.87387 149.37387 14937.38738 0.87387 297.87387 149.37387 14937.38716 0.87387 297.87387 149.37387 14937.38700 2020-01-01 2020-01-02 2020-01-01 00:04:51 2020-01-02 03:33:12 2020-01-01 00:04:51.000 2020-01-02 03:33:12.000 291 99192 49741.5 4974150 291 99192 49741.5 4974150 -32278 32657 5177.02 517702 -128 127 -1.86 -186 -292 2 10282 99193 0.87687 297.87687 149.37687 14937.68768 0.87687 297.8769 149.37687 14937.68789 0.87687 297.87687 149.37687 14937.68700 2020-01-01 2020-01-02 2020-01-01 00:04:52 2020-01-02 03:33:13 2020-01-01 00:04:52.000 2020-01-02 03:33:13.000 292 99193 49742.5 4974250 292 99193 49742.5 4974250 -32277 32658 5178.02 517802 -128 123 -3.42 -342 -293 2 10283 99194 0.87987 297.87987 149.37987 14937.98798 0.87987 297.87988 149.37988 14937.9882 0.87987 297.87987 149.37987 14937.98700 2020-01-01 2020-01-02 2020-01-01 00:04:53 2020-01-02 03:33:14 2020-01-01 00:04:53.000 2020-01-02 03:33:14.000 293 99194 49743.5 4974350 293 99194 49743.5 4974350 -32276 32659 5179.02 517902 -127 124 -2.42 -242 -294 2 10284 99195 0.88288 297.88288 149.38288 14938.28828 0.88288 297.88287 149.38288 14938.28835 0.88288 297.88288 149.38288 14938.28800 2020-01-01 2020-01-02 2020-01-01 00:04:54 2020-01-02 03:33:15 2020-01-01 00:04:54.000 2020-01-02 03:33:15.000 294 99195 49744.5 4974450 294 99195 49744.5 4974450 -32275 32660 5180.02 518002 -126 125 -1.42 -142 -295 2 10285 99196 0.88588 297.88588 149.38588 14938.58858 0.88588 297.8859 149.38588 14938.58847 0.88588 297.88588 149.38588 14938.58800 2020-01-01 2020-01-02 2020-01-01 00:04:55 2020-01-02 03:33:16 2020-01-01 00:04:55.000 2020-01-02 03:33:16.000 295 99196 49745.5 4974550 295 99196 49745.5 4974550 -32274 32661 5181.02 518102 -125 126 -0.42 -42 -296 2 10286 99197 0.88888 297.88888 149.38888 14938.88888 0.88888 297.8889 149.38888 14938.88863 0.88888 297.88888 149.38888 14938.88800 2020-01-01 2020-01-02 2020-01-01 00:04:56 2020-01-02 03:33:17 2020-01-01 00:04:56.000 2020-01-02 03:33:17.000 296 99197 49746.5 4974650 296 99197 49746.5 4974650 -32273 32662 5182.02 518202 -124 127 0.58 58 -297 2 10287 99198 0.89189 297.89189 149.39189 14939.18918 0.89189 297.8919 149.39189 14939.18936 0.89189 297.89189 149.39189 14939.18900 2020-01-01 2020-01-02 2020-01-01 00:04:57 2020-01-02 03:33:18 2020-01-01 00:04:57.000 2020-01-02 03:33:18.000 297 99198 49747.5 4974750 297 99198 49747.5 4974750 -32272 32663 5183.02 518302 -128 127 -0.98 -98 -298 2 10288 99199 0.89489 297.89489 149.39489 14939.48948 0.89489 297.8949 149.39489 14939.48967 0.89489 297.89489 149.39489 14939.48900 2020-01-01 2020-01-02 2020-01-01 00:04:58 2020-01-02 03:33:19 2020-01-01 00:04:58.000 2020-01-02 03:33:19.000 298 99199 49748.5 4974850 298 99199 49748.5 4974850 -32271 32664 5184.02 518402 -128 127 -2.54 -254 -299 2 10289 99200 0.89789 297.89789 149.39789 14939.78978 0.89789 297.8979 149.39789 14939.78986 0.89789 297.89789 149.39789 14939.78900 2020-01-01 2020-01-02 2020-01-01 00:04:59 2020-01-02 03:33:20 2020-01-01 00:04:59.000 2020-01-02 03:33:20.000 299 99200 49749.5 4974950 299 99200 49749.5 4974950 -32270 32665 5185.02 518502 -128 124 -4.1 -410 -3 2 1002 9993 0.009 300.009 150.009 15150.9099 0.009 300.009 150.009 15150.90958 0.00900 300.00900 150.00900 15150.90900 2020-01-01 2020-01-02 2020-01-01 00:00:03 2020-01-02 03:45:03 2020-01-01 00:00:03.000 2020-01-02 03:45:03.000 3 99903 49953 5045253 3 99903 49953 5045253 -32566 32369 4532.009900990099 457733 -124 127 0.04950495049504951 5 -30 2 10020 99930 0.09009 300.09009 150.09009 15159.09909 0.09009 300.0901 150.09008 15159.09894 0.09009 300.09009 150.09009 15159.09909 2020-01-01 2020-01-02 2020-01-01 00:00:30 2020-01-02 03:45:30 2020-01-01 00:00:30.000 2020-01-02 03:45:30.000 30 99930 49980 5047980 30 99930 49980 5047980 -32539 32396 4559.009900990099 460460 -128 123 -3.366336633663366 -340 -300 2 10290 99201 0.9009 297.9009 149.4009 14940.09009 0.9009 297.9009 149.40089 14940.08995 0.90090 297.90090 149.40090 14940.09000 2020-01-01 2020-01-02 2020-01-01 00:05:00 2020-01-02 03:33:21 2020-01-01 00:05:00.000 2020-01-02 03:33:21.000 300 99201 49750.5 4975050 300 99201 49750.5 4975050 -32269 32666 5186.02 518602 -127 125 -3.1 -310 -301 2 10291 99202 0.9039 297.9039 149.4039 14940.39039 0.9039 297.9039 149.4039 14940.39009 0.90390 297.90390 149.40390 14940.39000 2020-01-01 2020-01-02 2020-01-01 00:05:01 2020-01-02 03:33:22 2020-01-01 00:05:01.000 2020-01-02 03:33:22.000 301 99202 49751.5 4975150 301 99202 49751.5 4975150 -32268 32667 5187.02 518702 -126 126 -2.1 -210 -302 2 10292 99203 0.9069 297.9069 149.4069 14940.69069 0.9069 297.90692 149.4069 14940.69083 0.90690 297.90690 149.40690 14940.69000 2020-01-01 2020-01-02 2020-01-01 00:05:02 2020-01-02 03:33:23 2020-01-01 00:05:02.000 2020-01-02 03:33:23.000 302 99203 49752.5 4975250 302 99203 49752.5 4975250 -32267 32668 5188.02 518802 -125 127 -1.1 -110 -303 2 10293 99204 0.9099 297.9099 149.4099 14940.99099 0.9099 297.9099 149.40991 14940.99114 0.90990 297.90990 149.40990 14940.99000 2020-01-01 2020-01-02 2020-01-01 00:05:03 2020-01-02 03:33:24 2020-01-01 00:05:03.000 2020-01-02 03:33:24.000 303 99204 49753.5 4975350 303 99204 49753.5 4975350 -32266 32669 5189.02 518902 -128 127 -2.66 -266 -304 2 10294 99205 0.91291 297.91291 149.41291 14941.29129 0.91291 297.9129 149.41291 14941.29133 0.91291 297.91291 149.41291 14941.29100 2020-01-01 2020-01-02 2020-01-01 00:05:04 2020-01-02 03:33:25 2020-01-01 00:05:04.000 2020-01-02 03:33:25.000 304 99205 49754.5 4975450 304 99205 49754.5 4975450 -32265 32670 5190.02 519002 -128 127 -4.22 -422 -305 2 10295 99206 0.91591 297.91591 149.41591 14941.59159 0.91591 297.91592 149.41591 14941.59141 0.91591 297.91591 149.41591 14941.59100 2020-01-01 2020-01-02 2020-01-01 00:05:05 2020-01-02 03:33:26 2020-01-01 00:05:05.000 2020-01-02 03:33:26.000 305 99206 49755.5 4975550 305 99206 49755.5 4975550 -32264 32671 5191.02 519102 -128 123 -5.78 -578 -306 2 10296 99207 0.91891 297.91891 149.41891 14941.89189 0.91891 297.9189 149.41891 14941.89172 0.91891 297.91891 149.41891 14941.89100 2020-01-01 2020-01-02 2020-01-01 00:05:06 2020-01-02 03:33:27 2020-01-01 00:05:06.000 2020-01-02 03:33:27.000 306 99207 49756.5 4975650 306 99207 49756.5 4975650 -32263 32672 5192.02 519202 -127 124 -4.78 -478 -307 2 10297 99208 0.92192 297.92192 149.42192 14942.19219 0.92192 297.92194 149.42192 14942.1923 0.92192 297.92192 149.42192 14942.19200 2020-01-01 2020-01-02 2020-01-01 00:05:07 2020-01-02 03:33:28 2020-01-01 00:05:07.000 2020-01-02 03:33:28.000 307 99208 49757.5 4975750 307 99208 49757.5 4975750 -32262 32673 5193.02 519302 -126 125 -3.78 -378 +276 2 10266 99177 0.82882 297.82882 149.32882 14932.88288 0.82882 297.82883 149.32882 14932.88275 0.82882 297.82882 149.3288200000001 14932.88200 2020-01-01 2020-01-02 2020-01-01 00:04:36 2020-01-02 03:32:57 2020-01-01 00:04:36.000 2020-01-02 03:32:57.000 276 99177 49726.5 4972650 276 99177 49726.5 4972650 -32293 32642 5162.02 516202 -126 126 -1.5 -150 +277 2 10267 99178 0.83183 297.83183 149.33183 14933.18318 0.83183 297.83182 149.33183 14933.18305 0.83183 297.83183 149.33182999999985 14933.18300 2020-01-01 2020-01-02 2020-01-01 00:04:37 2020-01-02 03:32:58 2020-01-01 00:04:37.000 2020-01-02 03:32:58.000 277 99178 49727.5 4972750 277 99178 49727.5 4972750 -32292 32643 5163.02 516302 -125 127 -0.5 -50 +278 2 10268 99179 0.83483 297.83483 149.33483 14933.48348 0.83483 297.83484 149.33483 14933.48364 0.83483 297.83483 149.33482999999984 14933.48300 2020-01-01 2020-01-02 2020-01-01 00:04:38 2020-01-02 03:32:59 2020-01-01 00:04:38.000 2020-01-02 03:32:59.000 278 99179 49728.5 4972850 278 99179 49728.5 4972850 -32291 32644 5164.02 516402 -128 127 -2.06 -206 +279 2 10269 99180 0.83783 297.83783 149.33783 14933.78378 0.83783 297.83783 149.33783 14933.78394 0.83783 297.83783 149.33783000000022 14933.78300 2020-01-01 2020-01-02 2020-01-01 00:04:39 2020-01-02 03:33:00 2020-01-01 00:04:39.000 2020-01-02 03:33:00.000 279 99180 49729.5 4972950 279 99180 49729.5 4972950 -32290 32645 5165.02 516502 -128 127 -3.62 -362 +28 2 10018 99928 0.08408 300.08408 150.08408 15158.49249 0.08408 300.08408 150.08408 15158.49265 0.08408 300.08408 150.08408000000006 15158.49208 2020-01-01 2020-01-02 2020-01-01 00:00:28 2020-01-02 03:45:28 2020-01-01 00:00:28.000 2020-01-02 03:45:28.000 28 99928 49978 5047778 28 99928 49978 5047778 -32541 32394 4557.009900990099 460258 -124 127 -0.297029702970297 -30 +280 2 10270 99181 0.84084 297.84084 149.34084 14934.08408 0.84084 297.84085 149.34084 14934.08403 0.84084 297.84084 149.34084000000007 14934.08400 2020-01-01 2020-01-02 2020-01-01 00:04:40 2020-01-02 03:33:01 2020-01-01 00:04:40.000 2020-01-02 03:33:01.000 280 99181 49730.5 4973050 280 99181 49730.5 4973050 -32289 32646 5166.02 516602 -128 123 -5.18 -518 +281 2 10271 99182 0.84384 297.84384 149.34384 14934.38438 0.84384 297.84384 149.34384 14934.38421 0.84384 297.84384 149.34383999999974 14934.38400 2020-01-01 2020-01-02 2020-01-01 00:04:41 2020-01-02 03:33:02 2020-01-01 00:04:41.000 2020-01-02 03:33:02.000 281 99182 49731.5 4973150 281 99182 49731.5 4973150 -32288 32647 5167.02 516702 -127 124 -4.18 -418 +282 2 10272 99183 0.84684 297.84684 149.34684 14934.68468 0.84684 297.84683 149.34684 14934.68453 0.84684 297.84684 149.3468400000001 14934.68400 2020-01-01 2020-01-02 2020-01-01 00:04:42 2020-01-02 03:33:03 2020-01-01 00:04:42.000 2020-01-02 03:33:03.000 282 99183 49732.5 4973250 282 99183 49732.5 4973250 -32287 32648 5168.02 516802 -126 125 -3.18 -318 +283 2 10273 99184 0.84984 297.84984 149.34984 14934.98498 0.84984 297.84985 149.34985 14934.98526 0.84984 297.84984 149.34984000000014 14934.98400 2020-01-01 2020-01-02 2020-01-01 00:04:43 2020-01-02 03:33:04 2020-01-01 00:04:43.000 2020-01-02 03:33:04.000 283 99184 49733.5 4973350 283 99184 49733.5 4973350 -32286 32649 5169.02 516902 -125 126 -2.18 -218 +284 2 10274 99185 0.85285 297.85285 149.35285 14935.28528 0.85285 297.85284 149.35285 14935.28542 0.85285 297.85285 149.35284999999993 14935.28500 2020-01-01 2020-01-02 2020-01-01 00:04:44 2020-01-02 03:33:05 2020-01-01 00:04:44.000 2020-01-02 03:33:05.000 284 99185 49734.5 4973450 284 99185 49734.5 4973450 -32285 32650 5170.02 517002 -124 127 -1.18 -118 +285 2 10275 99186 0.85585 297.85585 149.35585 14935.58558 0.85585 297.85587 149.35585 14935.5855 0.85585 297.85585 149.35584999999995 14935.58500 2020-01-01 2020-01-02 2020-01-01 00:04:45 2020-01-02 03:33:06 2020-01-01 00:04:45.000 2020-01-02 03:33:06.000 285 99186 49735.5 4973550 285 99186 49735.5 4973550 -32284 32651 5171.02 517102 -128 127 -2.74 -274 +286 2 10276 99187 0.85885 297.85885 149.35885 14935.88588 0.85885 297.85886 149.35885 14935.88568 0.85885 297.85885 149.35885000000005 14935.88500 2020-01-01 2020-01-02 2020-01-01 00:04:46 2020-01-02 03:33:07 2020-01-01 00:04:46.000 2020-01-02 03:33:07.000 286 99187 49736.5 4973650 286 99187 49736.5 4973650 -32283 32652 5172.02 517202 -128 123 -4.3 -430 +287 2 10277 99188 0.86186 297.86186 149.36186 14936.18618 0.86186 297.86185 149.36186 14936.186 0.86186 297.86186 149.3618600000002 14936.18600 2020-01-01 2020-01-02 2020-01-01 00:04:47 2020-01-02 03:33:08 2020-01-01 00:04:47.000 2020-01-02 03:33:08.000 287 99188 49737.5 4973750 287 99188 49737.5 4973750 -32282 32653 5173.02 517302 -127 124 -3.3 -330 +288 2 10278 99189 0.86486 297.86486 149.36486 14936.48648 0.86486 297.86487 149.36486 14936.48673 0.86486 297.86486 149.36485999999985 14936.48600 2020-01-01 2020-01-02 2020-01-01 00:04:48 2020-01-02 03:33:09 2020-01-01 00:04:48.000 2020-01-02 03:33:09.000 288 99189 49738.5 4973850 288 99189 49738.5 4973850 -32281 32654 5174.02 517402 -126 125 -2.3 -230 +289 2 10279 99190 0.86786 297.86786 149.36786 14936.78678 0.86786 297.86786 149.36786 14936.78688 0.86786 297.86786 149.36786000000015 14936.78600 2020-01-01 2020-01-02 2020-01-01 00:04:49 2020-01-02 03:33:10 2020-01-01 00:04:49.000 2020-01-02 03:33:10.000 289 99190 49739.5 4973950 289 99190 49739.5 4973950 -32280 32655 5175.02 517502 -125 126 -1.3 -130 +29 2 10019 99929 0.08708 300.08708 150.08708 15158.79579 0.08708 300.0871 150.08708 15158.79576 0.08708 300.08708 150.0870799999999 15158.79508 2020-01-01 2020-01-02 2020-01-01 00:00:29 2020-01-02 03:45:29 2020-01-01 00:00:29.000 2020-01-02 03:45:29.000 29 99929 49979 5047879 29 99929 49979 5047879 -32540 32395 4558.009900990099 460359 -128 127 -1.8316831683168318 -185 +290 2 10280 99191 0.87087 297.87087 149.37087 14937.08708 0.87087 297.87088 149.37087 14937.087 0.87087 297.87087 149.37087000000008 14937.08700 2020-01-01 2020-01-02 2020-01-01 00:04:50 2020-01-02 03:33:11 2020-01-01 00:04:50.000 2020-01-02 03:33:11.000 290 99191 49740.5 4974050 290 99191 49740.5 4974050 -32279 32656 5176.02 517602 -124 127 -0.3 -30 +291 2 10281 99192 0.87387 297.87387 149.37387 14937.38738 0.87387 297.87387 149.37387 14937.38716 0.87387 297.87387 149.37386999999993 14937.38700 2020-01-01 2020-01-02 2020-01-01 00:04:51 2020-01-02 03:33:12 2020-01-01 00:04:51.000 2020-01-02 03:33:12.000 291 99192 49741.5 4974150 291 99192 49741.5 4974150 -32278 32657 5177.02 517702 -128 127 -1.86 -186 +292 2 10282 99193 0.87687 297.87687 149.37687 14937.68768 0.87687 297.8769 149.37687 14937.68789 0.87687 297.87687 149.37687000000003 14937.68700 2020-01-01 2020-01-02 2020-01-01 00:04:52 2020-01-02 03:33:13 2020-01-01 00:04:52.000 2020-01-02 03:33:13.000 292 99193 49742.5 4974250 292 99193 49742.5 4974250 -32277 32658 5178.02 517802 -128 123 -3.42 -342 +293 2 10283 99194 0.87987 297.87987 149.37987 14937.98798 0.87987 297.87988 149.37988 14937.9882 0.87987 297.87987 149.3798700000001 14937.98700 2020-01-01 2020-01-02 2020-01-01 00:04:53 2020-01-02 03:33:14 2020-01-01 00:04:53.000 2020-01-02 03:33:14.000 293 99194 49743.5 4974350 293 99194 49743.5 4974350 -32276 32659 5179.02 517902 -127 124 -2.42 -242 +294 2 10284 99195 0.88288 297.88288 149.38288 14938.28828 0.88288 297.88287 149.38288 14938.28835 0.88288 297.88288 149.38287999999986 14938.28800 2020-01-01 2020-01-02 2020-01-01 00:04:54 2020-01-02 03:33:15 2020-01-01 00:04:54.000 2020-01-02 03:33:15.000 294 99195 49744.5 4974450 294 99195 49744.5 4974450 -32275 32660 5180.02 518002 -126 125 -1.42 -142 +295 2 10285 99196 0.88588 297.88588 149.38588 14938.58858 0.88588 297.8859 149.38588 14938.58847 0.88588 297.88588 149.38587999999987 14938.58800 2020-01-01 2020-01-02 2020-01-01 00:04:55 2020-01-02 03:33:16 2020-01-01 00:04:55.000 2020-01-02 03:33:16.000 295 99196 49745.5 4974550 295 99196 49745.5 4974550 -32274 32661 5181.02 518102 -125 126 -0.42 -42 +296 2 10286 99197 0.88888 297.88888 149.38888 14938.88888 0.88888 297.8889 149.38888 14938.88863 0.88888 297.88888 149.38888000000023 14938.88800 2020-01-01 2020-01-02 2020-01-01 00:04:56 2020-01-02 03:33:17 2020-01-01 00:04:56.000 2020-01-02 03:33:17.000 296 99197 49746.5 4974650 296 99197 49746.5 4974650 -32273 32662 5182.02 518202 -124 127 0.58 58 +297 2 10287 99198 0.89189 297.89189 149.39189 14939.18918 0.89189 297.8919 149.39189 14939.18936 0.89189 297.89189 149.3918900000001 14939.18900 2020-01-01 2020-01-02 2020-01-01 00:04:57 2020-01-02 03:33:18 2020-01-01 00:04:57.000 2020-01-02 03:33:18.000 297 99198 49747.5 4974750 297 99198 49747.5 4974750 -32272 32663 5183.02 518302 -128 127 -0.98 -98 +298 2 10288 99199 0.89489 297.89489 149.39489 14939.48948 0.89489 297.8949 149.39489 14939.48967 0.89489 297.89489 149.39488999999975 14939.48900 2020-01-01 2020-01-02 2020-01-01 00:04:58 2020-01-02 03:33:19 2020-01-01 00:04:58.000 2020-01-02 03:33:19.000 298 99199 49748.5 4974850 298 99199 49748.5 4974850 -32271 32664 5184.02 518402 -128 127 -2.54 -254 +299 2 10289 99200 0.89789 297.89789 149.39789 14939.78978 0.89789 297.8979 149.39789 14939.78986 0.89789 297.89789 149.39789000000013 14939.78900 2020-01-01 2020-01-02 2020-01-01 00:04:59 2020-01-02 03:33:20 2020-01-01 00:04:59.000 2020-01-02 03:33:20.000 299 99200 49749.5 4974950 299 99200 49749.5 4974950 -32270 32665 5185.02 518502 -128 124 -4.1 -410 +3 2 1002 9993 0.009 300.009 150.009 15150.9099 0.009 300.009 150.009 15150.90958 0.00900 300.00900 150.009 15150.90900 2020-01-01 2020-01-02 2020-01-01 00:00:03 2020-01-02 03:45:03 2020-01-01 00:00:03.000 2020-01-02 03:45:03.000 3 99903 49953 5045253 3 99903 49953 5045253 -32566 32369 4532.009900990099 457733 -124 127 0.04950495049504951 5 +30 2 10020 99930 0.09009 300.09009 150.09009 15159.09909 0.09009 300.0901 150.09008 15159.09894 0.09009 300.09009 150.09008999999986 15159.09909 2020-01-01 2020-01-02 2020-01-01 00:00:30 2020-01-02 03:45:30 2020-01-01 00:00:30.000 2020-01-02 03:45:30.000 30 99930 49980 5047980 30 99930 49980 5047980 -32539 32396 4559.009900990099 460460 -128 123 -3.366336633663366 -340 +300 2 10290 99201 0.9009 297.9009 149.4009 14940.09009 0.9009 297.9009 149.40089 14940.08995 0.90090 297.90090 149.40090000000004 14940.09000 2020-01-01 2020-01-02 2020-01-01 00:05:00 2020-01-02 03:33:21 2020-01-01 00:05:00.000 2020-01-02 03:33:21.000 300 99201 49750.5 4975050 300 99201 49750.5 4975050 -32269 32666 5186.02 518602 -127 125 -3.1 -310 +301 2 10291 99202 0.9039 297.9039 149.4039 14940.39039 0.9039 297.9039 149.4039 14940.39009 0.90390 297.90390 149.4038999999999 14940.39000 2020-01-01 2020-01-02 2020-01-01 00:05:01 2020-01-02 03:33:22 2020-01-01 00:05:01.000 2020-01-02 03:33:22.000 301 99202 49751.5 4975150 301 99202 49751.5 4975150 -32268 32667 5187.02 518702 -126 126 -2.1 -210 +302 2 10292 99203 0.9069 297.9069 149.4069 14940.69069 0.9069 297.90692 149.4069 14940.69083 0.90690 297.90690 149.40689999999998 14940.69000 2020-01-01 2020-01-02 2020-01-01 00:05:02 2020-01-02 03:33:23 2020-01-01 00:05:02.000 2020-01-02 03:33:23.000 302 99203 49752.5 4975250 302 99203 49752.5 4975250 -32267 32668 5188.02 518802 -125 127 -1.1 -110 +303 2 10293 99204 0.9099 297.9099 149.4099 14940.99099 0.9099 297.9099 149.40991 14940.99114 0.90990 297.90990 149.40990000000005 14940.99000 2020-01-01 2020-01-02 2020-01-01 00:05:03 2020-01-02 03:33:24 2020-01-01 00:05:03.000 2020-01-02 03:33:24.000 303 99204 49753.5 4975350 303 99204 49753.5 4975350 -32266 32669 5189.02 518902 -128 127 -2.66 -266 +304 2 10294 99205 0.91291 297.91291 149.41291 14941.29129 0.91291 297.9129 149.41291 14941.29133 0.91291 297.91291 149.41290999999978 14941.29100 2020-01-01 2020-01-02 2020-01-01 00:05:04 2020-01-02 03:33:25 2020-01-01 00:05:04.000 2020-01-02 03:33:25.000 304 99205 49754.5 4975450 304 99205 49754.5 4975450 -32265 32670 5190.02 519002 -128 127 -4.22 -422 +305 2 10295 99206 0.91591 297.91591 149.41591 14941.59159 0.91591 297.91592 149.41591 14941.59141 0.91591 297.91591 149.41590999999985 14941.59100 2020-01-01 2020-01-02 2020-01-01 00:05:05 2020-01-02 03:33:26 2020-01-01 00:05:05.000 2020-01-02 03:33:26.000 305 99206 49755.5 4975550 305 99206 49755.5 4975550 -32264 32671 5191.02 519102 -128 123 -5.78 -578 +306 2 10296 99207 0.91891 297.91891 149.41891 14941.89189 0.91891 297.9189 149.41891 14941.89172 0.91891 297.91891 149.41891000000015 14941.89100 2020-01-01 2020-01-02 2020-01-01 00:05:06 2020-01-02 03:33:27 2020-01-01 00:05:06.000 2020-01-02 03:33:27.000 306 99207 49756.5 4975650 306 99207 49756.5 4975650 -32263 32672 5192.02 519202 -127 124 -4.78 -478 +307 2 10297 99208 0.92192 297.92192 149.42192 14942.19219 0.92192 297.92194 149.42192 14942.1923 0.92192 297.92192 149.42192000000003 14942.19200 2020-01-01 2020-01-02 2020-01-01 00:05:07 2020-01-02 03:33:28 2020-01-01 00:05:07.000 2020-01-02 03:33:28.000 307 99208 49757.5 4975750 307 99208 49757.5 4975750 -32262 32673 5193.02 519302 -126 125 -3.78 -378 308 2 10298 99209 0.92492 297.92492 149.42492 14942.49249 0.92492 297.92493 149.42492 14942.49265 0.92492 297.92492 149.42492 14942.49200 2020-01-01 2020-01-02 2020-01-01 00:05:08 2020-01-02 03:33:29 2020-01-01 00:05:08.000 2020-01-02 03:33:29.000 308 99209 49758.5 4975850 308 99209 49758.5 4975850 -32261 32674 5194.02 519402 -125 126 -2.78 -278 -309 2 10299 99210 0.92792 297.92792 149.42792 14942.79279 0.92792 297.92792 149.42792 14942.7928 0.92792 297.92792 149.42792 14942.79200 2020-01-01 2020-01-02 2020-01-01 00:05:09 2020-01-02 03:33:30 2020-01-01 00:05:09.000 2020-01-02 03:33:30.000 309 99210 49759.5 4975950 309 99210 49759.5 4975950 -32260 32675 5195.02 519502 -124 127 -1.78 -178 -31 2 10021 99931 0.09309 300.09309 150.09309 15159.4024 0.09309 300.09308 150.09309 15159.40224 0.09309 300.09309 150.09309 15159.40209 2020-01-01 2020-01-02 2020-01-01 00:00:31 2020-01-02 03:45:31 2020-01-01 00:00:31.000 2020-01-02 03:45:31.000 31 99931 49981 5048081 31 99931 49981 5048081 -32538 32397 4560.009900990099 460561 -127 124 -2.366336633663366 -239 -310 2 10300 99211 0.93093 297.93093 149.43093 14943.09309 0.93093 297.93094 149.43092 14943.09288 0.93093 297.93093 149.43093 14943.09300 2020-01-01 2020-01-02 2020-01-01 00:05:10 2020-01-02 03:33:31 2020-01-01 00:05:10.000 2020-01-02 03:33:31.000 310 99211 49760.5 4976050 310 99211 49760.5 4976050 -32259 32676 5196.02 519602 -128 127 -3.34 -334 -311 2 10301 99212 0.93393 297.93393 149.43393 14943.39339 0.93393 297.93393 149.43393 14943.39319 0.93393 297.93393 149.43393 14943.39300 2020-01-01 2020-01-02 2020-01-01 00:05:11 2020-01-02 03:33:32 2020-01-01 00:05:11.000 2020-01-02 03:33:32.000 311 99212 49761.5 4976150 311 99212 49761.5 4976150 -32258 32677 5197.02 519702 -128 123 -4.9 -490 -312 2 10302 99213 0.93693 297.93693 149.43693 14943.69369 0.93693 297.93695 149.43693 14943.69377 0.93693 297.93693 149.43693 14943.69300 2020-01-01 2020-01-02 2020-01-01 00:05:12 2020-01-02 03:33:33 2020-01-01 00:05:12.000 2020-01-02 03:33:33.000 312 99213 49762.5 4976250 312 99213 49762.5 4976250 -32257 32678 5198.02 519802 -127 124 -3.9 -390 -313 2 10303 99214 0.93993 297.93993 149.43993 14943.99399 0.93993 297.93994 149.43994 14943.99412 0.93993 297.93993 149.43993 14943.99300 2020-01-01 2020-01-02 2020-01-01 00:05:13 2020-01-02 03:33:34 2020-01-01 00:05:13.000 2020-01-02 03:33:34.000 313 99214 49763.5 4976350 313 99214 49763.5 4976350 -32256 32679 5199.02 519902 -126 125 -2.9 -290 -314 2 10304 99215 0.94294 297.94294 149.44294 14944.29429 0.94294 297.94293 149.44294 14944.29427 0.94294 297.94294 149.44294 14944.29400 2020-01-01 2020-01-02 2020-01-01 00:05:14 2020-01-02 03:33:35 2020-01-01 00:05:14.000 2020-01-02 03:33:35.000 314 99215 49764.5 4976450 314 99215 49764.5 4976450 -32255 32680 5200.02 520002 -125 126 -1.9 -190 -315 2 10305 99216 0.94594 297.94594 149.44594 14944.59459 0.94594 297.94595 149.44595 14944.595 0.94594 297.94594 149.44594 14944.59400 2020-01-01 2020-01-02 2020-01-01 00:05:15 2020-01-02 03:33:36 2020-01-01 00:05:15.000 2020-01-02 03:33:36.000 315 99216 49765.5 4976550 315 99216 49765.5 4976550 -32254 32681 5201.02 520102 -124 127 -0.9 -90 -316 2 10306 99217 0.94894 297.94894 149.44894 14944.89489 0.94894 297.94894 149.44894 14944.89466 0.94894 297.94894 149.44894 14944.89400 2020-01-01 2020-01-02 2020-01-01 00:05:16 2020-01-02 03:33:37 2020-01-01 00:05:16.000 2020-01-02 03:33:37.000 316 99217 49766.5 4976650 316 99217 49766.5 4976650 -32253 32682 5202.02 520202 -128 127 -2.46 -246 -317 2 10307 99218 0.95195 297.95195 149.45195 14945.19519 0.95195 297.95197 149.45195 14945.19524 0.95195 297.95195 149.45195 14945.19500 2020-01-01 2020-01-02 2020-01-01 00:05:17 2020-01-02 03:33:38 2020-01-01 00:05:17.000 2020-01-02 03:33:38.000 317 99218 49767.5 4976750 317 99218 49767.5 4976750 -32252 32683 5203.02 520302 -128 123 -4.02 -402 -318 2 10308 99219 0.95495 297.95495 149.45495 14945.49549 0.95495 297.95496 149.45495 14945.49558 0.95495 297.95495 149.45495 14945.49500 2020-01-01 2020-01-02 2020-01-01 00:05:18 2020-01-02 03:33:39 2020-01-01 00:05:18.000 2020-01-02 03:33:39.000 318 99219 49768.5 4976850 318 99219 49768.5 4976850 -32251 32684 5204.02 520402 -127 124 -3.02 -302 +309 2 10299 99210 0.92792 297.92792 149.42792 14942.79279 0.92792 297.92792 149.42792 14942.7928 0.92792 297.92792 149.42792000000006 14942.79200 2020-01-01 2020-01-02 2020-01-01 00:05:09 2020-01-02 03:33:30 2020-01-01 00:05:09.000 2020-01-02 03:33:30.000 309 99210 49759.5 4975950 309 99210 49759.5 4975950 -32260 32675 5195.02 519502 -124 127 -1.78 -178 +31 2 10021 99931 0.09309 300.09309 150.09309 15159.4024 0.09309 300.09308 150.09309 15159.40224 0.09309 300.09309 150.09309000000013 15159.40209 2020-01-01 2020-01-02 2020-01-01 00:00:31 2020-01-02 03:45:31 2020-01-01 00:00:31.000 2020-01-02 03:45:31.000 31 99931 49981 5048081 31 99931 49981 5048081 -32538 32397 4560.009900990099 460561 -127 124 -2.366336633663366 -239 +310 2 10300 99211 0.93093 297.93093 149.43093 14943.09309 0.93093 297.93094 149.43092 14943.09288 0.93093 297.93093 149.43093000000025 14943.09300 2020-01-01 2020-01-02 2020-01-01 00:05:10 2020-01-02 03:33:31 2020-01-01 00:05:10.000 2020-01-02 03:33:31.000 310 99211 49760.5 4976050 310 99211 49760.5 4976050 -32259 32676 5196.02 519602 -128 127 -3.34 -334 +311 2 10301 99212 0.93393 297.93393 149.43393 14943.39339 0.93393 297.93393 149.43393 14943.39319 0.93393 297.93393 149.43392999999992 14943.39300 2020-01-01 2020-01-02 2020-01-01 00:05:11 2020-01-02 03:33:32 2020-01-01 00:05:11.000 2020-01-02 03:33:32.000 311 99212 49761.5 4976150 311 99212 49761.5 4976150 -32258 32677 5197.02 519702 -128 123 -4.9 -490 +312 2 10302 99213 0.93693 297.93693 149.43693 14943.69369 0.93693 297.93695 149.43693 14943.69377 0.93693 297.93693 149.4369299999999 14943.69300 2020-01-01 2020-01-02 2020-01-01 00:05:12 2020-01-02 03:33:33 2020-01-01 00:05:12.000 2020-01-02 03:33:33.000 312 99213 49762.5 4976250 312 99213 49762.5 4976250 -32257 32678 5198.02 519802 -127 124 -3.9 -390 +313 2 10303 99214 0.93993 297.93993 149.43993 14943.99399 0.93993 297.93994 149.43994 14943.99412 0.93993 297.93993 149.43993000000012 14943.99300 2020-01-01 2020-01-02 2020-01-01 00:05:13 2020-01-02 03:33:34 2020-01-01 00:05:13.000 2020-01-02 03:33:34.000 313 99214 49763.5 4976350 313 99214 49763.5 4976350 -32256 32679 5199.02 519902 -126 125 -2.9 -290 +314 2 10304 99215 0.94294 297.94294 149.44294 14944.29429 0.94294 297.94293 149.44294 14944.29427 0.94294 297.94294 149.44294000000014 14944.29400 2020-01-01 2020-01-02 2020-01-01 00:05:14 2020-01-02 03:33:35 2020-01-01 00:05:14.000 2020-01-02 03:33:35.000 314 99215 49764.5 4976450 314 99215 49764.5 4976450 -32255 32680 5200.02 520002 -125 126 -1.9 -190 +315 2 10305 99216 0.94594 297.94594 149.44594 14944.59459 0.94594 297.94595 149.44595 14944.595 0.94594 297.94594 149.4459399999998 14944.59400 2020-01-01 2020-01-02 2020-01-01 00:05:15 2020-01-02 03:33:36 2020-01-01 00:05:15.000 2020-01-02 03:33:36.000 315 99216 49765.5 4976550 315 99216 49765.5 4976550 -32254 32681 5201.02 520102 -124 127 -0.9 -90 +316 2 10306 99217 0.94894 297.94894 149.44894 14944.89489 0.94894 297.94894 149.44894 14944.89466 0.94894 297.94894 149.44894000000016 14944.89400 2020-01-01 2020-01-02 2020-01-01 00:05:16 2020-01-02 03:33:37 2020-01-01 00:05:16.000 2020-01-02 03:33:37.000 316 99217 49766.5 4976650 316 99217 49766.5 4976650 -32253 32682 5202.02 520202 -128 127 -2.46 -246 +317 2 10307 99218 0.95195 297.95195 149.45195 14945.19519 0.95195 297.95197 149.45195 14945.19524 0.95195 297.95195 149.45195000000004 14945.19500 2020-01-01 2020-01-02 2020-01-01 00:05:17 2020-01-02 03:33:38 2020-01-01 00:05:17.000 2020-01-02 03:33:38.000 317 99218 49767.5 4976750 317 99218 49767.5 4976750 -32252 32683 5203.02 520302 -128 123 -4.02 -402 +318 2 10308 99219 0.95495 297.95495 149.45495 14945.49549 0.95495 297.95496 149.45495 14945.49558 0.95495 297.95495 149.4549499999999 14945.49500 2020-01-01 2020-01-02 2020-01-01 00:05:18 2020-01-02 03:33:39 2020-01-01 00:05:18.000 2020-01-02 03:33:39.000 318 99219 49768.5 4976850 318 99219 49768.5 4976850 -32251 32684 5204.02 520402 -127 124 -3.02 -302 319 2 10309 99220 0.95795 297.95795 149.45795 14945.79579 0.95795 297.95795 149.45795 14945.79574 0.95795 297.95795 149.45795 14945.79500 2020-01-01 2020-01-02 2020-01-01 00:05:19 2020-01-02 03:33:40 2020-01-01 00:05:19.000 2020-01-02 03:33:40.000 319 99220 49769.5 4976950 319 99220 49769.5 4976950 -32250 32685 5205.02 520502 -126 125 -2.02 -202 -32 2 10022 99932 0.09609 300.09609 150.09609 15159.7057 0.09609 300.0961 150.09609 15159.706 0.09609 300.09609 150.09609 15159.70509 2020-01-01 2020-01-02 2020-01-01 00:00:32 2020-01-02 03:45:32 2020-01-01 00:00:32.000 2020-01-02 03:45:32.000 32 99932 49982 5048182 32 99932 49982 5048182 -32537 32398 4561.009900990099 460662 -126 125 -1.3663366336633664 -138 -320 2 10310 99221 0.96096 297.96096 149.46096 14946.09609 0.96096 297.96097 149.46096 14946.09647 0.96096 297.96096 149.46096 14946.09600 2020-01-01 2020-01-02 2020-01-01 00:05:20 2020-01-02 03:33:41 2020-01-01 00:05:20.000 2020-01-02 03:33:41.000 320 99221 49770.5 4977050 320 99221 49770.5 4977050 -32249 32686 5206.02 520602 -125 126 -1.02 -102 -321 2 10311 99222 0.96396 297.96396 149.46396 14946.39639 0.96396 297.96396 149.46396 14946.39613 0.96396 297.96396 149.46396 14946.39600 2020-01-01 2020-01-02 2020-01-01 00:05:21 2020-01-02 03:33:42 2020-01-01 00:05:21.000 2020-01-02 03:33:42.000 321 99222 49771.5 4977150 321 99222 49771.5 4977150 -32248 32687 5207.02 520702 -124 127 -0.02 -2 -322 2 10312 99223 0.96696 297.96696 149.46696 14946.69669 0.96696 297.96698 149.46696 14946.69674 0.96696 297.96696 149.46696 14946.69600 2020-01-01 2020-01-02 2020-01-01 00:05:22 2020-01-02 03:33:43 2020-01-01 00:05:22.000 2020-01-02 03:33:43.000 322 99223 49772.5 4977250 322 99223 49772.5 4977250 -32247 32688 5208.02 520802 -128 127 -1.58 -158 -323 2 10313 99224 0.96996 297.96996 149.46996 14946.99699 0.96997 297.96997 149.46997 14946.99706 0.96996 297.96996 149.46996 14946.99600 2020-01-01 2020-01-02 2020-01-01 00:05:23 2020-01-02 03:33:44 2020-01-01 00:05:23.000 2020-01-02 03:33:44.000 323 99224 49773.5 4977350 323 99224 49773.5 4977350 -32246 32689 5209.02 520902 -128 123 -3.14 -314 -324 2 10314 99225 0.97297 297.97297 149.47297 14947.29729 0.97297 297.97296 149.47297 14947.29737 0.97297 297.97297 149.47297 14947.29700 2020-01-01 2020-01-02 2020-01-01 00:05:24 2020-01-02 03:33:45 2020-01-01 00:05:24.000 2020-01-02 03:33:45.000 324 99225 49774.5 4977450 324 99225 49774.5 4977450 -32245 32690 5210.02 521002 -127 124 -2.14 -214 +32 2 10022 99932 0.09609 300.09609 150.09609 15159.7057 0.09609 300.0961 150.09609 15159.706 0.09609 300.09609 150.0960899999998 15159.70509 2020-01-01 2020-01-02 2020-01-01 00:00:32 2020-01-02 03:45:32 2020-01-01 00:00:32.000 2020-01-02 03:45:32.000 32 99932 49982 5048182 32 99932 49982 5048182 -32537 32398 4561.009900990099 460662 -126 125 -1.3663366336633664 -138 +320 2 10310 99221 0.96096 297.96096 149.46096 14946.09609 0.96096 297.96097 149.46096 14946.09647 0.96096 297.96096 149.46096000000014 14946.09600 2020-01-01 2020-01-02 2020-01-01 00:05:20 2020-01-02 03:33:41 2020-01-01 00:05:20.000 2020-01-02 03:33:41.000 320 99221 49770.5 4977050 320 99221 49770.5 4977050 -32249 32686 5206.02 520602 -125 126 -1.02 -102 +321 2 10311 99222 0.96396 297.96396 149.46396 14946.39639 0.96396 297.96396 149.46396 14946.39613 0.96396 297.96396 149.4639599999998 14946.39600 2020-01-01 2020-01-02 2020-01-01 00:05:21 2020-01-02 03:33:42 2020-01-01 00:05:21.000 2020-01-02 03:33:42.000 321 99222 49771.5 4977150 321 99222 49771.5 4977150 -32248 32687 5207.02 520702 -124 127 -0.02 -2 +322 2 10312 99223 0.96696 297.96696 149.46696 14946.69669 0.96696 297.96698 149.46696 14946.69674 0.96696 297.96696 149.4669599999999 14946.69600 2020-01-01 2020-01-02 2020-01-01 00:05:22 2020-01-02 03:33:43 2020-01-01 00:05:22.000 2020-01-02 03:33:43.000 322 99223 49772.5 4977250 322 99223 49772.5 4977250 -32247 32688 5208.02 520802 -128 127 -1.58 -158 +323 2 10313 99224 0.96996 297.96996 149.46996 14946.99699 0.96997 297.96997 149.46997 14946.99706 0.96996 297.96996 149.46996000000013 14946.99600 2020-01-01 2020-01-02 2020-01-01 00:05:23 2020-01-02 03:33:44 2020-01-01 00:05:23.000 2020-01-02 03:33:44.000 323 99224 49773.5 4977350 323 99224 49773.5 4977350 -32246 32689 5209.02 520902 -128 123 -3.14 -314 +324 2 10314 99225 0.97297 297.97297 149.47297 14947.29729 0.97297 297.97296 149.47297 14947.29737 0.97297 297.97297 149.4729700000001 14947.29700 2020-01-01 2020-01-02 2020-01-01 00:05:24 2020-01-02 03:33:45 2020-01-01 00:05:24.000 2020-01-02 03:33:45.000 324 99225 49774.5 4977450 324 99225 49774.5 4977450 -32245 32690 5210.02 521002 -127 124 -2.14 -214 325 2 10315 99226 0.97597 297.97597 149.47597 14947.59759 0.97597 297.97598 149.47597 14947.59794 0.97597 297.97597 149.47597 14947.59700 2020-01-01 2020-01-02 2020-01-01 00:05:25 2020-01-02 03:33:46 2020-01-01 00:05:25.000 2020-01-02 03:33:46.000 325 99226 49775.5 4977550 325 99226 49775.5 4977550 -32244 32691 5211.02 521102 -126 125 -1.14 -114 -326 2 10316 99227 0.97897 297.97897 149.47897 14947.89789 0.97897 297.97897 149.47897 14947.8976 0.97897 297.97897 149.47897 14947.89700 2020-01-01 2020-01-02 2020-01-01 00:05:26 2020-01-02 03:33:47 2020-01-01 00:05:26.000 2020-01-02 03:33:47.000 326 99227 49776.5 4977650 326 99227 49776.5 4977650 -32243 32692 5212.02 521202 -125 126 -0.14 -14 +326 2 10316 99227 0.97897 297.97897 149.47897 14947.89789 0.97897 297.97897 149.47897 14947.8976 0.97897 297.97897 149.4789700000001 14947.89700 2020-01-01 2020-01-02 2020-01-01 00:05:26 2020-01-02 03:33:47 2020-01-01 00:05:26.000 2020-01-02 03:33:47.000 326 99227 49776.5 4977650 326 99227 49776.5 4977650 -32243 32692 5212.02 521202 -125 126 -0.14 -14 327 2 10317 99228 0.98198 297.98198 149.48198 14948.19819 0.98198 297.982 149.48198 14948.19821 0.98198 297.98198 149.48198 14948.19800 2020-01-01 2020-01-02 2020-01-01 00:05:27 2020-01-02 03:33:48 2020-01-01 00:05:27.000 2020-01-02 03:33:48.000 327 99228 49777.5 4977750 327 99228 49777.5 4977750 -32242 32693 5213.02 521302 -124 127 0.86 86 -328 2 10318 99229 0.98498 297.98498 149.48498 14948.49849 0.98498 297.985 149.48498 14948.49853 0.98498 297.98498 149.48498 14948.49800 2020-01-01 2020-01-02 2020-01-01 00:05:28 2020-01-02 03:33:49 2020-01-01 00:05:28.000 2020-01-02 03:33:49.000 328 99229 49778.5 4977850 328 99229 49778.5 4977850 -32241 32694 5214.02 521402 -128 127 -0.7 -70 -329 2 10319 99230 0.98798 297.98798 149.48798 14948.79879 0.98798 297.98798 149.48798 14948.79883 0.98798 297.98798 149.48798 14948.79800 2020-01-01 2020-01-02 2020-01-01 00:05:29 2020-01-02 03:33:50 2020-01-01 00:05:29.000 2020-01-02 03:33:50.000 329 99230 49779.5 4977950 329 99230 49779.5 4977950 -32240 32695 5215.02 521502 -128 127 -2.26 -226 -33 2 10023 99933 0.09909 300.09909 150.09909 15160.009 0.09909 300.0991 150.0991 15160.00913 0.09909 300.09909 150.09909 15160.00809 2020-01-01 2020-01-02 2020-01-01 00:00:33 2020-01-02 03:45:33 2020-01-01 00:00:33.000 2020-01-02 03:45:33.000 33 99933 49983 5048283 33 99933 49983 5048283 -32536 32399 4562.009900990099 460763 -125 126 -0.36633663366336633 -37 -330 2 10320 99231 0.99099 297.99099 149.49099 14949.09909 0.99099 297.991 149.49099 14949.09941 0.99099 297.99099 149.49099 14949.09900 2020-01-01 2020-01-02 2020-01-01 00:05:30 2020-01-02 03:33:51 2020-01-01 00:05:30.000 2020-01-02 03:33:51.000 330 99231 49780.5 4978050 330 99231 49780.5 4978050 -32239 32696 5216.02 521602 -128 123 -3.82 -382 -331 2 10321 99232 0.99399 297.99399 149.49399 14949.39939 0.99399 297.994 149.49399 14949.39911 0.99399 297.99399 149.49399 14949.39900 2020-01-01 2020-01-02 2020-01-01 00:05:31 2020-01-02 03:33:52 2020-01-01 00:05:31.000 2020-01-02 03:33:52.000 331 99232 49781.5 4978150 331 99232 49781.5 4978150 -32238 32697 5217.02 521702 -127 124 -2.82 -282 -332 2 10322 99233 0.99699 297.99699 149.49699 14949.69969 0.99699 297.997 149.49699 14949.69969 0.99699 297.99699 149.49699 14949.69900 2020-01-01 2020-01-02 2020-01-01 00:05:32 2020-01-02 03:33:53 2020-01-01 00:05:32.000 2020-01-02 03:33:53.000 332 99233 49782.5 4978250 332 99233 49782.5 4978250 -32237 32698 5218.02 521802 -126 125 -1.82 -182 -333 2 10323 99234 1 298 149.5 14950 1 298 149.5 14950 1.00000 298.00000 149.50000 14950.00000 2020-01-01 2020-01-02 2020-01-01 00:05:33 2020-01-02 03:33:54 2020-01-01 00:05:33.000 2020-01-02 03:33:54.000 333 99234 49783.5 4978350 333 99234 49783.5 4978350 -32236 32699 5219.02 521902 -125 126 -0.82 -82 -334 2 10324 99235 1.003 298.003 149.503 14950.3003 1.003 298.003 149.503 14950.30029 1.00300 298.00300 149.50300 14950.30000 2020-01-01 2020-01-02 2020-01-01 00:05:34 2020-01-02 03:33:55 2020-01-01 00:05:34.000 2020-01-02 03:33:55.000 334 99235 49784.5 4978450 334 99235 49784.5 4978450 -32235 32700 5220.02 522002 -124 127 0.18 18 -335 2 10325 99236 1.006 298.006 149.506 14950.6006 1.006 298.006 149.506 14950.60088 1.00600 298.00600 149.50600 14950.60000 2020-01-01 2020-01-02 2020-01-01 00:05:35 2020-01-02 03:33:56 2020-01-01 00:05:35.000 2020-01-02 03:33:56.000 335 99236 49785.5 4978550 335 99236 49785.5 4978550 -32234 32701 5221.02 522102 -128 127 -1.38 -138 -336 2 10326 99237 1.009 298.009 149.509 14950.9009 1.009 298.009 149.509 14950.90057 1.00900 298.00900 149.50900 14950.90000 2020-01-01 2020-01-02 2020-01-01 00:05:36 2020-01-02 03:33:57 2020-01-01 00:05:36.000 2020-01-02 03:33:57.000 336 99237 49786.5 4978650 336 99237 49786.5 4978650 -32233 32702 5222.02 522202 -128 123 -2.94 -294 -337 2 10327 99238 1.01201 298.01201 149.51201 14951.2012 1.01201 298.01202 149.51201 14951.20117 1.01201 298.01201 149.51201 14951.20100 2020-01-01 2020-01-02 2020-01-01 00:05:37 2020-01-02 03:33:58 2020-01-01 00:05:37.000 2020-01-02 03:33:58.000 337 99238 49787.5 4978750 337 99238 49787.5 4978750 -32232 32703 5223.02 522302 -127 124 -1.94 -194 -338 2 10328 99239 1.01501 298.01501 149.51501 14951.5015 1.01501 298.015 149.51501 14951.50146 1.01501 298.01501 149.51501 14951.50100 2020-01-01 2020-01-02 2020-01-01 00:05:38 2020-01-02 03:33:59 2020-01-01 00:05:38.000 2020-01-02 03:33:59.000 338 99239 49788.5 4978850 338 99239 49788.5 4978850 -32231 32704 5224.02 522402 -126 125 -0.94 -94 -339 2 10329 99240 1.01801 298.01801 149.51801 14951.8018 1.01801 298.018 149.51801 14951.80177 1.01801 298.01801 149.51801 14951.80100 2020-01-01 2020-01-02 2020-01-01 00:05:39 2020-01-02 03:34:00 2020-01-01 00:05:39.000 2020-01-02 03:34:00.000 339 99240 49789.5 4978950 339 99240 49789.5 4978950 -32230 32705 5225.02 522502 -125 126 0.06 6 -34 2 10024 99934 0.1021 300.1021 150.1021 15160.31231 0.1021 300.1021 150.1021 15160.31224 0.10210 300.10210 150.10210 15160.31210 2020-01-01 2020-01-02 2020-01-01 00:00:34 2020-01-02 03:45:34 2020-01-01 00:00:34.000 2020-01-02 03:45:34.000 34 99934 49984 5048384 34 99934 49984 5048384 -32535 32400 4563.009900990099 460864 -124 127 0.6336633663366337 64 -340 2 10330 99241 1.02102 298.02102 149.52102 14952.1021 1.02102 298.02103 149.52102 14952.10239 1.02102 298.02102 149.52102 14952.10200 2020-01-01 2020-01-02 2020-01-01 00:05:40 2020-01-02 03:34:01 2020-01-01 00:05:40.000 2020-01-02 03:34:01.000 340 99241 49790.5 4979050 340 99241 49790.5 4979050 -32229 32706 5226.02 522602 -124 127 1.06 106 -341 2 10331 99242 1.02402 298.02402 149.52402 14952.4024 1.02402 298.02402 149.52402 14952.40205 1.02402 298.02402 149.52402 14952.40200 2020-01-01 2020-01-02 2020-01-01 00:05:41 2020-01-02 03:34:02 2020-01-01 00:05:41.000 2020-01-02 03:34:02.000 341 99242 49791.5 4979150 341 99242 49791.5 4979150 -32228 32707 5227.02 522702 -128 127 -0.5 -50 -342 2 10332 99243 1.02702 298.02702 149.52702 14952.7027 1.02702 298.02704 149.52702 14952.70264 1.02702 298.02702 149.52702 14952.70200 2020-01-01 2020-01-02 2020-01-01 00:05:42 2020-01-02 03:34:03 2020-01-01 00:05:42.000 2020-01-02 03:34:03.000 342 99243 49792.5 4979250 342 99243 49792.5 4979250 -32227 32708 5228.02 522802 -128 123 -2.06 -206 -343 2 10333 99244 1.03003 298.03003 149.53003 14953.003 1.03003 298.03003 149.53002 14953.00293 1.03003 298.03003 149.53003 14953.00300 2020-01-01 2020-01-02 2020-01-01 00:05:43 2020-01-02 03:34:04 2020-01-01 00:05:43.000 2020-01-02 03:34:04.000 343 99244 49793.5 4979350 343 99244 49793.5 4979350 -32226 32709 5229.02 522902 -127 124 -1.06 -106 -344 2 10334 99245 1.03303 298.03303 149.53303 14953.3033 1.03303 298.03302 149.53303 14953.30323 1.03303 298.03303 149.53303 14953.30300 2020-01-01 2020-01-02 2020-01-01 00:05:44 2020-01-02 03:34:05 2020-01-01 00:05:44.000 2020-01-02 03:34:05.000 344 99245 49794.5 4979450 344 99245 49794.5 4979450 -32225 32710 5230.02 523002 -126 125 -0.06 -6 -345 2 10335 99246 1.03603 298.03603 149.53603 14953.6036 1.03603 298.03604 149.53603 14953.60386 1.03603 298.03603 149.53603 14953.60300 2020-01-01 2020-01-02 2020-01-01 00:05:45 2020-01-02 03:34:06 2020-01-01 00:05:45.000 2020-01-02 03:34:06.000 345 99246 49795.5 4979550 345 99246 49795.5 4979550 -32224 32711 5231.02 523102 -125 126 0.94 94 -346 2 10336 99247 1.03903 298.03903 149.53903 14953.9039 1.03903 298.03903 149.53903 14953.90352 1.03903 298.03903 149.53903 14953.90300 2020-01-01 2020-01-02 2020-01-01 00:05:46 2020-01-02 03:34:07 2020-01-01 00:05:46.000 2020-01-02 03:34:07.000 346 99247 49796.5 4979650 346 99247 49796.5 4979650 -32223 32712 5232.02 523202 -124 127 1.94 194 -347 2 10337 99248 1.04204 298.04204 149.54204 14954.2042 1.04204 298.04205 149.54204 14954.20427 1.04204 298.04204 149.54204 14954.20400 2020-01-01 2020-01-02 2020-01-01 00:05:47 2020-01-02 03:34:08 2020-01-01 00:05:47.000 2020-01-02 03:34:08.000 347 99248 49797.5 4979750 347 99248 49797.5 4979750 -32222 32713 5233.02 523302 -128 127 0.38 38 -348 2 10338 99249 1.04504 298.04504 149.54504 14954.5045 1.04504 298.04504 149.54504 14954.50441 1.04504 298.04504 149.54504 14954.50400 2020-01-01 2020-01-02 2020-01-01 00:05:48 2020-01-02 03:34:09 2020-01-01 00:05:48.000 2020-01-02 03:34:09.000 348 99249 49798.5 4979850 348 99249 49798.5 4979850 -32221 32714 5234.02 523402 -128 123 -1.18 -118 -349 2 10339 99250 1.04804 298.04804 149.54804 14954.8048 1.04804 298.04803 149.54804 14954.80474 1.04804 298.04804 149.54804 14954.80400 2020-01-01 2020-01-02 2020-01-01 00:05:49 2020-01-02 03:34:10 2020-01-01 00:05:49.000 2020-01-02 03:34:10.000 349 99250 49799.5 4979950 349 99250 49799.5 4979950 -32220 32715 5235.02 523502 -127 124 -0.18 -18 -35 2 10025 99935 0.1051 300.1051 150.1051 15160.61561 0.1051 300.1051 150.1051 15160.61542 0.10510 300.10510 150.10510 15160.61510 2020-01-01 2020-01-02 2020-01-01 00:00:35 2020-01-02 03:45:35 2020-01-01 00:00:35.000 2020-01-02 03:45:35.000 35 99935 49985 5048485 35 99935 49985 5048485 -32534 32401 4564.009900990099 460965 -128 127 -0.900990099009901 -91 +328 2 10318 99229 0.98498 297.98498 149.48498 14948.49849 0.98498 297.985 149.48498 14948.49853 0.98498 297.98498 149.4849799999999 14948.49800 2020-01-01 2020-01-02 2020-01-01 00:05:28 2020-01-02 03:33:49 2020-01-01 00:05:28.000 2020-01-02 03:33:49.000 328 99229 49778.5 4977850 328 99229 49778.5 4977850 -32241 32694 5214.02 521402 -128 127 -0.7 -70 +329 2 10319 99230 0.98798 297.98798 149.48798 14948.79879 0.98798 297.98798 149.48798 14948.79883 0.98798 297.98798 149.4879799999999 14948.79800 2020-01-01 2020-01-02 2020-01-01 00:05:29 2020-01-02 03:33:50 2020-01-01 00:05:29.000 2020-01-02 03:33:50.000 329 99230 49779.5 4977950 329 99230 49779.5 4977950 -32240 32695 5215.02 521502 -128 127 -2.26 -226 +33 2 10023 99933 0.09909 300.09909 150.09909 15160.009 0.09909 300.0991 150.0991 15160.00913 0.09909 300.09909 150.09908999999988 15160.00809 2020-01-01 2020-01-02 2020-01-01 00:00:33 2020-01-02 03:45:33 2020-01-01 00:00:33.000 2020-01-02 03:45:33.000 33 99933 49983 5048283 33 99933 49983 5048283 -32536 32399 4562.009900990099 460763 -125 126 -0.36633663366336633 -37 +330 2 10320 99231 0.99099 297.99099 149.49099 14949.09909 0.99099 297.991 149.49099 14949.09941 0.99099 297.99099 149.49099000000012 14949.09900 2020-01-01 2020-01-02 2020-01-01 00:05:30 2020-01-02 03:33:51 2020-01-01 00:05:30.000 2020-01-02 03:33:51.000 330 99231 49780.5 4978050 330 99231 49780.5 4978050 -32239 32696 5216.02 521602 -128 123 -3.82 -382 +331 2 10321 99232 0.99399 297.99399 149.49399 14949.39939 0.99399 297.994 149.49399 14949.39911 0.99399 297.99399 149.49399000000017 14949.39900 2020-01-01 2020-01-02 2020-01-01 00:05:31 2020-01-02 03:33:52 2020-01-01 00:05:31.000 2020-01-02 03:33:52.000 331 99232 49781.5 4978150 331 99232 49781.5 4978150 -32238 32697 5217.02 521702 -127 124 -2.82 -282 +332 2 10322 99233 0.99699 297.99699 149.49699 14949.69969 0.99699 297.997 149.49699 14949.69969 0.99699 297.99699 149.4969899999998 14949.69900 2020-01-01 2020-01-02 2020-01-01 00:05:32 2020-01-02 03:33:53 2020-01-01 00:05:32.000 2020-01-02 03:33:53.000 332 99233 49782.5 4978250 332 99233 49782.5 4978250 -32237 32698 5218.02 521802 -126 125 -1.82 -182 +333 2 10323 99234 1 298 149.5 14950 1 298 149.5 14950 1.00000 298.00000 149.5 14950.00000 2020-01-01 2020-01-02 2020-01-01 00:05:33 2020-01-02 03:33:54 2020-01-01 00:05:33.000 2020-01-02 03:33:54.000 333 99234 49783.5 4978350 333 99234 49783.5 4978350 -32236 32699 5219.02 521902 -125 126 -0.82 -82 +334 2 10324 99235 1.003 298.003 149.503 14950.3003 1.003 298.003 149.503 14950.30029 1.00300 298.00300 149.50300000000007 14950.30000 2020-01-01 2020-01-02 2020-01-01 00:05:34 2020-01-02 03:33:55 2020-01-01 00:05:34.000 2020-01-02 03:33:55.000 334 99235 49784.5 4978450 334 99235 49784.5 4978450 -32235 32700 5220.02 522002 -124 127 0.18 18 +335 2 10325 99236 1.006 298.006 149.506 14950.6006 1.006 298.006 149.506 14950.60088 1.00600 298.00600 149.50599999999991 14950.60000 2020-01-01 2020-01-02 2020-01-01 00:05:35 2020-01-02 03:33:56 2020-01-01 00:05:35.000 2020-01-02 03:33:56.000 335 99236 49785.5 4978550 335 99236 49785.5 4978550 -32234 32701 5221.02 522102 -128 127 -1.38 -138 +336 2 10326 99237 1.009 298.009 149.509 14950.9009 1.009 298.009 149.509 14950.90057 1.00900 298.00900 149.50900000000001 14950.90000 2020-01-01 2020-01-02 2020-01-01 00:05:36 2020-01-02 03:33:57 2020-01-01 00:05:36.000 2020-01-02 03:33:57.000 336 99237 49786.5 4978650 336 99237 49786.5 4978650 -32233 32702 5222.02 522202 -128 123 -2.94 -294 +337 2 10327 99238 1.01201 298.01201 149.51201 14951.2012 1.01201 298.01202 149.51201 14951.20117 1.01201 298.01201 149.51201000000017 14951.20100 2020-01-01 2020-01-02 2020-01-01 00:05:37 2020-01-02 03:33:58 2020-01-01 00:05:37.000 2020-01-02 03:33:58.000 337 99238 49787.5 4978750 337 99238 49787.5 4978750 -32232 32703 5223.02 522302 -127 124 -1.94 -194 +338 2 10328 99239 1.01501 298.01501 149.51501 14951.5015 1.01501 298.015 149.51501 14951.50146 1.01501 298.01501 149.51500999999982 14951.50100 2020-01-01 2020-01-02 2020-01-01 00:05:38 2020-01-02 03:33:59 2020-01-01 00:05:38.000 2020-01-02 03:33:59.000 338 99239 49788.5 4978850 338 99239 49788.5 4978850 -32231 32704 5224.02 522402 -126 125 -0.94 -94 +339 2 10329 99240 1.01801 298.01801 149.51801 14951.8018 1.01801 298.018 149.51801 14951.80177 1.01801 298.01801 149.51800999999983 14951.80100 2020-01-01 2020-01-02 2020-01-01 00:05:39 2020-01-02 03:34:00 2020-01-01 00:05:39.000 2020-01-02 03:34:00.000 339 99240 49789.5 4978950 339 99240 49789.5 4978950 -32230 32705 5225.02 522502 -125 126 0.06 6 +34 2 10024 99934 0.1021 300.1021 150.1021 15160.31231 0.1021 300.1021 150.1021 15160.31224 0.10210 300.10210 150.10210000000004 15160.31210 2020-01-01 2020-01-02 2020-01-01 00:00:34 2020-01-02 03:45:34 2020-01-01 00:00:34.000 2020-01-02 03:45:34.000 34 99934 49984 5048384 34 99934 49984 5048384 -32535 32400 4563.009900990099 460864 -124 127 0.6336633663366337 64 +340 2 10330 99241 1.02102 298.02102 149.52102 14952.1021 1.02102 298.02103 149.52102 14952.10239 1.02102 298.02102 149.5210200000001 14952.10200 2020-01-01 2020-01-02 2020-01-01 00:05:40 2020-01-02 03:34:01 2020-01-01 00:05:40.000 2020-01-02 03:34:01.000 340 99241 49790.5 4979050 340 99241 49790.5 4979050 -32229 32706 5226.02 522602 -124 127 1.06 106 +341 2 10331 99242 1.02402 298.02402 149.52402 14952.4024 1.02402 298.02402 149.52402 14952.40205 1.02402 298.02402 149.52402000000006 14952.40200 2020-01-01 2020-01-02 2020-01-01 00:05:41 2020-01-02 03:34:02 2020-01-01 00:05:41.000 2020-01-02 03:34:02.000 341 99242 49791.5 4979150 341 99242 49791.5 4979150 -32228 32707 5227.02 522702 -128 127 -0.5 -50 +342 2 10332 99243 1.02702 298.02702 149.52702 14952.7027 1.02702 298.02704 149.52702 14952.70264 1.02702 298.02702 149.52701999999982 14952.70200 2020-01-01 2020-01-02 2020-01-01 00:05:42 2020-01-02 03:34:03 2020-01-01 00:05:42.000 2020-01-02 03:34:03.000 342 99243 49792.5 4979250 342 99243 49792.5 4979250 -32227 32708 5228.02 522802 -128 123 -2.06 -206 +343 2 10333 99244 1.03003 298.03003 149.53003 14953.003 1.03003 298.03003 149.53002 14953.00293 1.03003 298.03003 149.53002999999993 14953.00300 2020-01-01 2020-01-02 2020-01-01 00:05:43 2020-01-02 03:34:04 2020-01-01 00:05:43.000 2020-01-02 03:34:04.000 343 99244 49793.5 4979350 343 99244 49793.5 4979350 -32226 32709 5229.02 522902 -127 124 -1.06 -106 +344 2 10334 99245 1.03303 298.03303 149.53303 14953.3033 1.03303 298.03302 149.53303 14953.30323 1.03303 298.03303 149.53303000000002 14953.30300 2020-01-01 2020-01-02 2020-01-01 00:05:44 2020-01-02 03:34:05 2020-01-01 00:05:44.000 2020-01-02 03:34:05.000 344 99245 49794.5 4979450 344 99245 49794.5 4979450 -32225 32710 5230.02 523002 -126 125 -0.06 -6 +345 2 10335 99246 1.03603 298.03603 149.53603 14953.6036 1.03603 298.03604 149.53603 14953.60386 1.03603 298.03603 149.53602999999993 14953.60300 2020-01-01 2020-01-02 2020-01-01 00:05:45 2020-01-02 03:34:06 2020-01-01 00:05:45.000 2020-01-02 03:34:06.000 345 99246 49795.5 4979550 345 99246 49795.5 4979550 -32224 32711 5231.02 523102 -125 126 0.94 94 +346 2 10336 99247 1.03903 298.03903 149.53903 14953.9039 1.03903 298.03903 149.53903 14953.90352 1.03903 298.03903 149.53902999999994 14953.90300 2020-01-01 2020-01-02 2020-01-01 00:05:46 2020-01-02 03:34:07 2020-01-01 00:05:46.000 2020-01-02 03:34:07.000 346 99247 49796.5 4979650 346 99247 49796.5 4979650 -32223 32712 5232.02 523202 -124 127 1.94 194 +347 2 10337 99248 1.04204 298.04204 149.54204 14954.2042 1.04204 298.04205 149.54204 14954.20427 1.04204 298.04204 149.5420400000001 14954.20400 2020-01-01 2020-01-02 2020-01-01 00:05:47 2020-01-02 03:34:08 2020-01-01 00:05:47.000 2020-01-02 03:34:08.000 347 99248 49797.5 4979750 347 99248 49797.5 4979750 -32222 32713 5233.02 523302 -128 127 0.38 38 +348 2 10338 99249 1.04504 298.04504 149.54504 14954.5045 1.04504 298.04504 149.54504 14954.50441 1.04504 298.04504 149.5450400000002 14954.50400 2020-01-01 2020-01-02 2020-01-01 00:05:48 2020-01-02 03:34:09 2020-01-01 00:05:48.000 2020-01-02 03:34:09.000 348 99249 49798.5 4979850 348 99249 49798.5 4979850 -32221 32714 5234.02 523402 -128 123 -1.18 -118 +349 2 10339 99250 1.04804 298.04804 149.54804 14954.8048 1.04804 298.04803 149.54804 14954.80474 1.04804 298.04804 149.54803999999987 14954.80400 2020-01-01 2020-01-02 2020-01-01 00:05:49 2020-01-02 03:34:10 2020-01-01 00:05:49.000 2020-01-02 03:34:10.000 349 99250 49799.5 4979950 349 99250 49799.5 4979950 -32220 32715 5235.02 523502 -127 124 -0.18 -18 +35 2 10025 99935 0.1051 300.1051 150.1051 15160.61561 0.1051 300.1051 150.1051 15160.61542 0.10510 300.10510 150.10510000000008 15160.61510 2020-01-01 2020-01-02 2020-01-01 00:00:35 2020-01-02 03:45:35 2020-01-01 00:00:35.000 2020-01-02 03:45:35.000 35 99935 49985 5048485 35 99935 49985 5048485 -32534 32401 4564.009900990099 460965 -128 127 -0.900990099009901 -91 350 2 10340 99251 1.05105 298.05105 149.55105 14955.1051 1.05105 298.05106 149.55105 14955.10532 1.05105 298.05105 149.55105 14955.10500 2020-01-01 2020-01-02 2020-01-01 00:05:50 2020-01-02 03:34:11 2020-01-01 00:05:50.000 2020-01-02 03:34:11.000 350 99251 49800.5 4980050 350 99251 49800.5 4980050 -32219 32716 5236.02 523602 -126 125 0.82 82 -351 2 10341 99252 1.05405 298.05405 149.55405 14955.4054 1.05405 298.05405 149.55404 14955.40499 1.05405 298.05405 149.55405 14955.40500 2020-01-01 2020-01-02 2020-01-01 00:05:51 2020-01-02 03:34:12 2020-01-01 00:05:51.000 2020-01-02 03:34:12.000 351 99252 49801.5 4980150 351 99252 49801.5 4980150 -32218 32717 5237.02 523702 -125 126 1.82 182 -352 2 10342 99253 1.05705 298.05705 149.55705 14955.7057 1.05705 298.05707 149.55705 14955.70574 1.05705 298.05705 149.55705 14955.70500 2020-01-01 2020-01-02 2020-01-01 00:05:52 2020-01-02 03:34:13 2020-01-01 00:05:52.000 2020-01-02 03:34:13.000 352 99253 49802.5 4980250 352 99253 49802.5 4980250 -32217 32718 5238.02 523802 -124 127 2.82 282 -353 2 10343 99254 1.06006 298.06006 149.56006 14956.006 1.06006 298.06006 149.56005 14956.00587 1.06006 298.06006 149.56006 14956.00600 2020-01-01 2020-01-02 2020-01-01 00:05:53 2020-01-02 03:34:14 2020-01-01 00:05:53.000 2020-01-02 03:34:14.000 353 99254 49803.5 4980350 353 99254 49803.5 4980350 -32216 32719 5239.02 523902 -128 127 1.26 126 -354 2 10344 99255 1.06306 298.06306 149.56306 14956.3063 1.06306 298.06305 149.56306 14956.3062 1.06306 298.06306 149.56306 14956.30600 2020-01-01 2020-01-02 2020-01-01 00:05:54 2020-01-02 03:34:15 2020-01-01 00:05:54.000 2020-01-02 03:34:15.000 354 99255 49804.5 4980450 354 99255 49804.5 4980450 -32215 32720 5240.02 524002 -128 127 -0.3 -30 -355 2 10345 99256 1.06606 298.06606 149.56606 14956.6066 1.06606 298.06607 149.56606 14956.6068 1.06606 298.06606 149.56606 14956.60600 2020-01-01 2020-01-02 2020-01-01 00:05:55 2020-01-02 03:34:16 2020-01-01 00:05:55.000 2020-01-02 03:34:16.000 355 99256 49805.5 4980550 355 99256 49805.5 4980550 -32214 32721 5241.02 524102 -128 123 -1.86 -186 -356 2 10346 99257 1.06906 298.06906 149.56906 14956.9069 1.06906 298.06906 149.56907 14956.90709 1.06906 298.06906 149.56906 14956.90600 2020-01-01 2020-01-02 2020-01-01 00:05:56 2020-01-02 03:34:17 2020-01-01 00:05:56.000 2020-01-02 03:34:17.000 356 99257 49806.5 4980650 356 99257 49806.5 4980650 -32213 32722 5242.02 524202 -127 124 -0.86 -86 -357 2 10347 99258 1.07207 298.07207 149.57207 14957.2072 1.07207 298.07208 149.57207 14957.20721 1.07207 298.07207 149.57207 14957.20700 2020-01-01 2020-01-02 2020-01-01 00:05:57 2020-01-02 03:34:18 2020-01-01 00:05:57.000 2020-01-02 03:34:18.000 357 99258 49807.5 4980750 357 99258 49807.5 4980750 -32212 32723 5243.02 524302 -126 125 0.14 14 -358 2 10348 99259 1.07507 298.07507 149.57507 14957.5075 1.07507 298.07507 149.57507 14957.50734 1.07507 298.07507 149.57507 14957.50700 2020-01-01 2020-01-02 2020-01-01 00:05:58 2020-01-02 03:34:19 2020-01-01 00:05:58.000 2020-01-02 03:34:19.000 358 99259 49808.5 4980850 358 99259 49808.5 4980850 -32211 32724 5244.02 524402 -125 126 1.14 114 -359 2 10349 99260 1.07807 298.07807 149.57807 14957.8078 1.07807 298.07806 149.57807 14957.80767 1.07807 298.07807 149.57807 14957.80700 2020-01-01 2020-01-02 2020-01-01 00:05:59 2020-01-02 03:34:20 2020-01-01 00:05:59.000 2020-01-02 03:34:20.000 359 99260 49809.5 4980950 359 99260 49809.5 4980950 -32210 32725 5245.02 524502 -124 127 2.14 214 -36 2 10026 99936 0.1081 300.1081 150.1081 15160.91891 0.1081 300.1081 150.1081 15160.91873 0.10810 300.10810 150.10810 15160.91810 2020-01-01 2020-01-02 2020-01-01 00:00:36 2020-01-02 03:45:36 2020-01-01 00:00:36.000 2020-01-02 03:45:36.000 36 99936 49986 5048586 36 99936 49986 5048586 -32533 32402 4565.009900990099 461066 -128 123 -2.4356435643564356 -246 -360 2 10350 99261 1.08108 298.08108 149.58108 14958.1081 1.08108 298.0811 149.58108 14958.10827 1.08108 298.08108 149.58108 14958.10800 2020-01-01 2020-01-02 2020-01-01 00:06:00 2020-01-02 03:34:21 2020-01-01 00:06:00.000 2020-01-02 03:34:21.000 360 99261 49810.5 4981050 360 99261 49810.5 4981050 -32209 32726 5246.02 524602 -128 127 0.58 58 -361 2 10351 99262 1.08408 298.08408 149.58408 14958.4084 1.08408 298.08408 149.58408 14958.40856 1.08408 298.08408 149.58408 14958.40800 2020-01-01 2020-01-02 2020-01-01 00:06:01 2020-01-02 03:34:22 2020-01-01 00:06:01.000 2020-01-02 03:34:22.000 361 99262 49811.5 4981150 361 99262 49811.5 4981150 -32208 32727 5247.02 524702 -128 123 -0.98 -98 -362 2 10352 99263 1.08708 298.08708 149.58708 14958.7087 1.08708 298.0871 149.58708 14958.70868 1.08708 298.08708 149.58708 14958.70800 2020-01-01 2020-01-02 2020-01-01 00:06:02 2020-01-02 03:34:23 2020-01-01 00:06:02.000 2020-01-02 03:34:23.000 362 99263 49812.5 4981250 362 99263 49812.5 4981250 -32207 32728 5248.02 524802 -127 124 0.02 2 -363 2 10353 99264 1.09009 298.09009 149.59009 14959.009 1.09009 298.0901 149.59008 14959.00884 1.09009 298.09009 149.59009 14959.00900 2020-01-01 2020-01-02 2020-01-01 00:06:03 2020-01-02 03:34:24 2020-01-01 00:06:03.000 2020-01-02 03:34:24.000 363 99264 49813.5 4981350 363 99264 49813.5 4981350 -32206 32729 5249.02 524902 -126 125 1.02 102 -364 2 10354 99265 1.09309 298.09309 149.59309 14959.3093 1.09309 298.09308 149.59309 14959.30915 1.09309 298.09309 149.59309 14959.30900 2020-01-01 2020-01-02 2020-01-01 00:06:04 2020-01-02 03:34:25 2020-01-01 00:06:04.000 2020-01-02 03:34:25.000 364 99265 49814.5 4981450 364 99265 49814.5 4981450 -32205 32730 5250.02 525002 -125 126 2.02 202 -365 2 10355 99266 1.09609 298.09609 149.59609 14959.6096 1.09609 298.0961 149.59609 14959.6099 1.09609 298.09609 149.59609 14959.60900 2020-01-01 2020-01-02 2020-01-01 00:06:05 2020-01-02 03:34:26 2020-01-01 00:06:05.000 2020-01-02 03:34:26.000 365 99266 49815.5 4981550 365 99266 49815.5 4981550 -32204 32731 5251.02 525102 -124 127 3.02 302 -366 2 10356 99267 1.09909 298.09909 149.59909 14959.9099 1.09909 298.0991 149.5991 14959.91003 1.09909 298.09909 149.59909 14959.90900 2020-01-01 2020-01-02 2020-01-01 00:06:06 2020-01-02 03:34:27 2020-01-01 00:06:06.000 2020-01-02 03:34:27.000 366 99267 49816.5 4981650 366 99267 49816.5 4981650 -32203 32732 5252.02 525202 -128 127 1.46 146 -367 2 10357 99268 1.1021 298.1021 149.6021 14960.21021 1.1021 298.1021 149.6021 14960.21015 1.10210 298.10210 149.60210 14960.21000 2020-01-01 2020-01-02 2020-01-01 00:06:07 2020-01-02 03:34:28 2020-01-01 00:06:07.000 2020-01-02 03:34:28.000 367 99268 49817.5 4981750 367 99268 49817.5 4981750 -32202 32733 5253.02 525302 -128 123 -0.1 -10 -368 2 10358 99269 1.1051 298.1051 149.6051 14960.51051 1.1051 298.1051 149.6051 14960.51031 1.10510 298.10510 149.60510 14960.51000 2020-01-01 2020-01-02 2020-01-01 00:06:08 2020-01-02 03:34:29 2020-01-01 00:06:08.000 2020-01-02 03:34:29.000 368 99269 49818.5 4981850 368 99269 49818.5 4981850 -32201 32734 5254.02 525402 -127 124 0.9 90 -369 2 10359 99270 1.1081 298.1081 149.6081 14960.81081 1.1081 298.1081 149.6081 14960.81062 1.10810 298.10810 149.60810 14960.81000 2020-01-01 2020-01-02 2020-01-01 00:06:09 2020-01-02 03:34:30 2020-01-01 00:06:09.000 2020-01-02 03:34:30.000 369 99270 49819.5 4981950 369 99270 49819.5 4981950 -32200 32735 5255.02 525502 -126 125 1.9 190 -37 2 10027 99937 0.11111 300.11111 150.11111 15161.22222 0.11111 300.1111 150.11111 15161.22248 0.11111 300.11111 150.11111 15161.22211 2020-01-01 2020-01-02 2020-01-01 00:00:37 2020-01-02 03:45:37 2020-01-01 00:00:37.000 2020-01-02 03:45:37.000 37 99937 49987 5048687 37 99937 49987 5048687 -32532 32403 4566.009900990099 461167 -127 124 -1.4356435643564356 -145 -370 2 10360 99271 1.11111 298.11111 149.61111 14961.11111 1.11111 298.1111 149.61111 14961.11137 1.11111 298.11111 149.61111 14961.11100 2020-01-01 2020-01-02 2020-01-01 00:06:10 2020-01-02 03:34:31 2020-01-01 00:06:10.000 2020-01-02 03:34:31.000 370 99271 49820.5 4982050 370 99271 49820.5 4982050 -32199 32736 5256.02 525602 -125 126 2.9 290 -371 2 10361 99272 1.11411 298.11411 149.61411 14961.41141 1.11411 298.1141 149.61411 14961.4115 1.11411 298.11411 149.61411 14961.41100 2020-01-01 2020-01-02 2020-01-01 00:06:11 2020-01-02 03:34:32 2020-01-01 00:06:11.000 2020-01-02 03:34:32.000 371 99272 49821.5 4982150 371 99272 49821.5 4982150 -32198 32737 5257.02 525702 -124 127 3.9 390 -372 2 10362 99273 1.11711 298.11711 149.61711 14961.71171 1.11711 298.11713 149.61711 14961.71165 1.11711 298.11711 149.61711 14961.71100 2020-01-01 2020-01-02 2020-01-01 00:06:12 2020-01-02 03:34:33 2020-01-01 00:06:12.000 2020-01-02 03:34:33.000 372 99273 49822.5 4982250 372 99273 49822.5 4982250 -32197 32738 5258.02 525802 -128 127 2.34 234 -373 2 10363 99274 1.12012 298.12012 149.62012 14962.01201 1.12012 298.12012 149.62011 14962.01179 1.12012 298.12012 149.62012 14962.01200 2020-01-01 2020-01-02 2020-01-01 00:06:13 2020-01-02 03:34:34 2020-01-01 00:06:13.000 2020-01-02 03:34:34.000 373 99274 49823.5 4982350 373 99274 49823.5 4982350 -32196 32739 5259.02 525902 -128 123 0.78 78 -374 2 10364 99275 1.12312 298.12312 149.62312 14962.31231 1.12312 298.1231 149.62312 14962.31208 1.12312 298.12312 149.62312 14962.31200 2020-01-01 2020-01-02 2020-01-01 00:06:14 2020-01-02 03:34:35 2020-01-01 00:06:14.000 2020-01-02 03:34:35.000 374 99275 49824.5 4982450 374 99275 49824.5 4982450 -32195 32740 5260.02 526002 -127 124 1.78 178 -375 2 10365 99276 1.12612 298.12612 149.62612 14962.61261 1.12612 298.12613 149.62612 14962.61283 1.12612 298.12612 149.62612 14962.61200 2020-01-01 2020-01-02 2020-01-01 00:06:15 2020-01-02 03:34:36 2020-01-01 00:06:15.000 2020-01-02 03:34:36.000 375 99276 49825.5 4982550 375 99276 49825.5 4982550 -32194 32741 5261.02 526102 -126 125 2.78 278 -376 2 10366 99277 1.12912 298.12912 149.62912 14962.91291 1.12912 298.12912 149.62912 14962.91297 1.12912 298.12912 149.62912 14962.91200 2020-01-01 2020-01-02 2020-01-01 00:06:16 2020-01-02 03:34:37 2020-01-01 00:06:16.000 2020-01-02 03:34:37.000 376 99277 49826.5 4982650 376 99277 49826.5 4982650 -32193 32742 5262.02 526202 -125 126 3.78 378 +351 2 10341 99252 1.05405 298.05405 149.55405 14955.4054 1.05405 298.05405 149.55404 14955.40499 1.05405 298.05405 149.55405000000007 14955.40500 2020-01-01 2020-01-02 2020-01-01 00:05:51 2020-01-02 03:34:12 2020-01-01 00:05:51.000 2020-01-02 03:34:12.000 351 99252 49801.5 4980150 351 99252 49801.5 4980150 -32218 32717 5237.02 523702 -125 126 1.82 182 +352 2 10342 99253 1.05705 298.05705 149.55705 14955.7057 1.05705 298.05707 149.55705 14955.70574 1.05705 298.05705 149.55704999999998 14955.70500 2020-01-01 2020-01-02 2020-01-01 00:05:52 2020-01-02 03:34:13 2020-01-01 00:05:52.000 2020-01-02 03:34:13.000 352 99253 49802.5 4980250 352 99253 49802.5 4980250 -32217 32718 5238.02 523802 -124 127 2.82 282 +353 2 10343 99254 1.06006 298.06006 149.56006 14956.006 1.06006 298.06006 149.56005 14956.00587 1.06006 298.06006 149.56005999999985 14956.00600 2020-01-01 2020-01-02 2020-01-01 00:05:53 2020-01-02 03:34:14 2020-01-01 00:05:53.000 2020-01-02 03:34:14.000 353 99254 49803.5 4980350 353 99254 49803.5 4980350 -32216 32719 5239.02 523902 -128 127 1.26 126 +354 2 10344 99255 1.06306 298.06306 149.56306 14956.3063 1.06306 298.06305 149.56306 14956.3062 1.06306 298.06306 149.5630600000002 14956.30600 2020-01-01 2020-01-02 2020-01-01 00:05:54 2020-01-02 03:34:15 2020-01-01 00:05:54.000 2020-01-02 03:34:15.000 354 99255 49804.5 4980450 354 99255 49804.5 4980450 -32215 32720 5240.02 524002 -128 127 -0.3 -30 +355 2 10345 99256 1.06606 298.06606 149.56606 14956.6066 1.06606 298.06607 149.56606 14956.6068 1.06606 298.06606 149.56605999999988 14956.60600 2020-01-01 2020-01-02 2020-01-01 00:05:55 2020-01-02 03:34:16 2020-01-01 00:05:55.000 2020-01-02 03:34:16.000 355 99256 49805.5 4980550 355 99256 49805.5 4980550 -32214 32721 5241.02 524102 -128 123 -1.86 -186 +356 2 10346 99257 1.06906 298.06906 149.56906 14956.9069 1.06906 298.06906 149.56907 14956.90709 1.06906 298.06906 149.56905999999987 14956.90600 2020-01-01 2020-01-02 2020-01-01 00:05:56 2020-01-02 03:34:17 2020-01-01 00:05:56.000 2020-01-02 03:34:17.000 356 99257 49806.5 4980650 356 99257 49806.5 4980650 -32213 32722 5242.02 524202 -127 124 -0.86 -86 +357 2 10347 99258 1.07207 298.07207 149.57207 14957.2072 1.07207 298.07208 149.57207 14957.20721 1.07207 298.07207 149.5720700000001 14957.20700 2020-01-01 2020-01-02 2020-01-01 00:05:57 2020-01-02 03:34:18 2020-01-01 00:05:57.000 2020-01-02 03:34:18.000 357 99258 49807.5 4980750 357 99258 49807.5 4980750 -32212 32723 5243.02 524302 -126 125 0.14 14 +358 2 10348 99259 1.07507 298.07507 149.57507 14957.5075 1.07507 298.07507 149.57507 14957.50734 1.07507 298.07507 149.57507000000012 14957.50700 2020-01-01 2020-01-02 2020-01-01 00:05:58 2020-01-02 03:34:19 2020-01-01 00:05:58.000 2020-01-02 03:34:19.000 358 99259 49808.5 4980850 358 99259 49808.5 4980850 -32211 32724 5244.02 524402 -125 126 1.14 114 +359 2 10349 99260 1.07807 298.07807 149.57807 14957.8078 1.07807 298.07806 149.57807 14957.80767 1.07807 298.07807 149.57806999999977 14957.80700 2020-01-01 2020-01-02 2020-01-01 00:05:59 2020-01-02 03:34:20 2020-01-01 00:05:59.000 2020-01-02 03:34:20.000 359 99260 49809.5 4980950 359 99260 49809.5 4980950 -32210 32725 5245.02 524502 -124 127 2.14 214 +36 2 10026 99936 0.1081 300.1081 150.1081 15160.91891 0.1081 300.1081 150.1081 15160.91873 0.10810 300.10810 150.10809999999995 15160.91810 2020-01-01 2020-01-02 2020-01-01 00:00:36 2020-01-02 03:45:36 2020-01-01 00:00:36.000 2020-01-02 03:45:36.000 36 99936 49986 5048586 36 99936 49986 5048586 -32533 32402 4565.009900990099 461066 -128 123 -2.4356435643564356 -246 +360 2 10350 99261 1.08108 298.08108 149.58108 14958.1081 1.08108 298.0811 149.58108 14958.10827 1.08108 298.08108 149.58107999999996 14958.10800 2020-01-01 2020-01-02 2020-01-01 00:06:00 2020-01-02 03:34:21 2020-01-01 00:06:00.000 2020-01-02 03:34:21.000 360 99261 49810.5 4981050 360 99261 49810.5 4981050 -32209 32726 5246.02 524602 -128 127 0.58 58 +361 2 10351 99262 1.08408 298.08408 149.58408 14958.4084 1.08408 298.08408 149.58408 14958.40856 1.08408 298.08408 149.58408000000006 14958.40800 2020-01-01 2020-01-02 2020-01-01 00:06:01 2020-01-02 03:34:22 2020-01-01 00:06:01.000 2020-01-02 03:34:22.000 361 99262 49811.5 4981150 361 99262 49811.5 4981150 -32208 32727 5247.02 524702 -128 123 -0.98 -98 +362 2 10352 99263 1.08708 298.08708 149.58708 14958.7087 1.08708 298.0871 149.58708 14958.70868 1.08708 298.08708 149.5870799999999 14958.70800 2020-01-01 2020-01-02 2020-01-01 00:06:02 2020-01-02 03:34:23 2020-01-01 00:06:02.000 2020-01-02 03:34:23.000 362 99263 49812.5 4981250 362 99263 49812.5 4981250 -32207 32728 5248.02 524802 -127 124 0.02 2 +363 2 10353 99264 1.09009 298.09009 149.59009 14959.009 1.09009 298.0901 149.59008 14959.00884 1.09009 298.09009 149.59008999999983 14959.00900 2020-01-01 2020-01-02 2020-01-01 00:06:03 2020-01-02 03:34:24 2020-01-01 00:06:03.000 2020-01-02 03:34:24.000 363 99264 49813.5 4981350 363 99264 49813.5 4981350 -32206 32729 5249.02 524902 -126 125 1.02 102 +364 2 10354 99265 1.09309 298.09309 149.59309 14959.3093 1.09309 298.09308 149.59309 14959.30915 1.09309 298.09309 149.59309000000013 14959.30900 2020-01-01 2020-01-02 2020-01-01 00:06:04 2020-01-02 03:34:25 2020-01-01 00:06:04.000 2020-01-02 03:34:25.000 364 99265 49814.5 4981450 364 99265 49814.5 4981450 -32205 32730 5250.02 525002 -125 126 2.02 202 +365 2 10355 99266 1.09609 298.09609 149.59609 14959.6096 1.09609 298.0961 149.59609 14959.6099 1.09609 298.09609 149.59608999999978 14959.60900 2020-01-01 2020-01-02 2020-01-01 00:06:05 2020-01-02 03:34:26 2020-01-01 00:06:05.000 2020-01-02 03:34:26.000 365 99266 49815.5 4981550 365 99266 49815.5 4981550 -32204 32731 5251.02 525102 -124 127 3.02 302 +366 2 10356 99267 1.09909 298.09909 149.59909 14959.9099 1.09909 298.0991 149.5991 14959.91003 1.09909 298.09909 149.59908999999985 14959.90900 2020-01-01 2020-01-02 2020-01-01 00:06:06 2020-01-02 03:34:27 2020-01-01 00:06:06.000 2020-01-02 03:34:27.000 366 99267 49816.5 4981650 366 99267 49816.5 4981650 -32203 32732 5252.02 525202 -128 127 1.46 146 +367 2 10357 99268 1.1021 298.1021 149.6021 14960.21021 1.1021 298.1021 149.6021 14960.21015 1.10210 298.10210 149.60210000000004 14960.21000 2020-01-01 2020-01-02 2020-01-01 00:06:07 2020-01-02 03:34:28 2020-01-01 00:06:07.000 2020-01-02 03:34:28.000 367 99268 49817.5 4981750 367 99268 49817.5 4981750 -32202 32733 5253.02 525302 -128 123 -0.1 -10 +368 2 10358 99269 1.1051 298.1051 149.6051 14960.51051 1.1051 298.1051 149.6051 14960.51031 1.10510 298.10510 149.6051000000001 14960.51000 2020-01-01 2020-01-02 2020-01-01 00:06:08 2020-01-02 03:34:29 2020-01-01 00:06:08.000 2020-01-02 03:34:29.000 368 99269 49818.5 4981850 368 99269 49818.5 4981850 -32201 32734 5254.02 525402 -127 124 0.9 90 +369 2 10359 99270 1.1081 298.1081 149.6081 14960.81081 1.1081 298.1081 149.6081 14960.81062 1.10810 298.10810 149.60809999999995 14960.81000 2020-01-01 2020-01-02 2020-01-01 00:06:09 2020-01-02 03:34:30 2020-01-01 00:06:09.000 2020-01-02 03:34:30.000 369 99270 49819.5 4981950 369 99270 49819.5 4981950 -32200 32735 5255.02 525502 -126 125 1.9 190 +37 2 10027 99937 0.11111 300.11111 150.11111 15161.22222 0.11111 300.1111 150.11111 15161.22248 0.11111 300.11111 150.11110999999988 15161.22211 2020-01-01 2020-01-02 2020-01-01 00:00:37 2020-01-02 03:45:37 2020-01-01 00:00:37.000 2020-01-02 03:45:37.000 37 99937 49987 5048687 37 99937 49987 5048687 -32532 32403 4566.009900990099 461167 -127 124 -1.4356435643564356 -145 +370 2 10360 99271 1.11111 298.11111 149.61111 14961.11111 1.11111 298.1111 149.61111 14961.11137 1.11111 298.11111 149.6111099999999 14961.11100 2020-01-01 2020-01-02 2020-01-01 00:06:10 2020-01-02 03:34:31 2020-01-01 00:06:10.000 2020-01-02 03:34:31.000 370 99271 49820.5 4982050 370 99271 49820.5 4982050 -32199 32736 5256.02 525602 -125 126 2.9 290 +371 2 10361 99272 1.11411 298.11411 149.61411 14961.41141 1.11411 298.1141 149.61411 14961.4115 1.11411 298.11411 149.61411000000024 14961.41100 2020-01-01 2020-01-02 2020-01-01 00:06:11 2020-01-02 03:34:32 2020-01-01 00:06:11.000 2020-01-02 03:34:32.000 371 99272 49821.5 4982150 371 99272 49821.5 4982150 -32198 32737 5257.02 525702 -124 127 3.9 390 +372 2 10362 99273 1.11711 298.11711 149.61711 14961.71171 1.11711 298.11713 149.61711 14961.71165 1.11711 298.11711 149.61710999999985 14961.71100 2020-01-01 2020-01-02 2020-01-01 00:06:12 2020-01-02 03:34:33 2020-01-01 00:06:12.000 2020-01-02 03:34:33.000 372 99273 49822.5 4982250 372 99273 49822.5 4982250 -32197 32738 5258.02 525802 -128 127 2.34 234 +373 2 10363 99274 1.12012 298.12012 149.62012 14962.01201 1.12012 298.12012 149.62011 14962.01179 1.12012 298.12012 149.6201199999998 14962.01200 2020-01-01 2020-01-02 2020-01-01 00:06:13 2020-01-02 03:34:34 2020-01-01 00:06:13.000 2020-01-02 03:34:34.000 373 99274 49823.5 4982350 373 99274 49823.5 4982350 -32196 32739 5259.02 525902 -128 123 0.78 78 +374 2 10364 99275 1.12312 298.12312 149.62312 14962.31231 1.12312 298.1231 149.62312 14962.31208 1.12312 298.12312 149.62312000000014 14962.31200 2020-01-01 2020-01-02 2020-01-01 00:06:14 2020-01-02 03:34:35 2020-01-01 00:06:14.000 2020-01-02 03:34:35.000 374 99275 49824.5 4982450 374 99275 49824.5 4982450 -32195 32740 5260.02 526002 -127 124 1.78 178 +375 2 10365 99276 1.12612 298.12612 149.62612 14962.61261 1.12612 298.12613 149.62612 14962.61283 1.12612 298.12612 149.62612000000016 14962.61200 2020-01-01 2020-01-02 2020-01-01 00:06:15 2020-01-02 03:34:36 2020-01-01 00:06:15.000 2020-01-02 03:34:36.000 375 99276 49825.5 4982550 375 99276 49825.5 4982550 -32194 32741 5261.02 526102 -126 125 2.78 278 +376 2 10366 99277 1.12912 298.12912 149.62912 14962.91291 1.12912 298.12912 149.62912 14962.91297 1.12912 298.12912 149.6291199999998 14962.91200 2020-01-01 2020-01-02 2020-01-01 00:06:16 2020-01-02 03:34:37 2020-01-01 00:06:16.000 2020-01-02 03:34:37.000 376 99277 49826.5 4982650 376 99277 49826.5 4982650 -32193 32742 5262.02 526202 -125 126 3.78 378 377 2 10367 99278 1.13213 298.13213 149.63213 14963.21321 1.13213 298.13214 149.63213 14963.21312 1.13213 298.13213 149.63213 14963.21300 2020-01-01 2020-01-02 2020-01-01 00:06:17 2020-01-02 03:34:38 2020-01-01 00:06:17.000 2020-01-02 03:34:38.000 377 99278 49827.5 4982750 377 99278 49827.5 4982750 -32192 32743 5263.02 526302 -124 127 4.78 478 -378 2 10368 99279 1.13513 298.13513 149.63513 14963.51351 1.13513 298.13513 149.63513 14963.51326 1.13513 298.13513 149.63513 14963.51300 2020-01-01 2020-01-02 2020-01-01 00:06:18 2020-01-02 03:34:39 2020-01-01 00:06:18.000 2020-01-02 03:34:39.000 378 99279 49828.5 4982850 378 99279 49828.5 4982850 -32191 32744 5264.02 526402 -128 127 3.22 322 -379 2 10369 99280 1.13813 298.13813 149.63813 14963.81381 1.13813 298.13815 149.63814 14963.81401 1.13813 298.13813 149.63813 14963.81300 2020-01-01 2020-01-02 2020-01-01 00:06:19 2020-01-02 03:34:40 2020-01-01 00:06:19.000 2020-01-02 03:34:40.000 379 99280 49829.5 4982950 379 99280 49829.5 4982950 -32190 32745 5265.02 526502 -128 127 1.66 166 -38 2 10028 99938 0.11411 300.11411 150.11411 15161.52552 0.11411 300.1141 150.11411 15161.52562 0.11411 300.11411 150.11411 15161.52511 2020-01-01 2020-01-02 2020-01-01 00:00:38 2020-01-02 03:45:38 2020-01-01 00:00:38.000 2020-01-02 03:45:38.000 38 99938 49988 5048788 38 99938 49988 5048788 -32531 32404 4567.009900990099 461268 -126 125 -0.43564356435643564 -44 -380 2 10370 99281 1.14114 298.14114 149.64114 14964.11411 1.14114 298.14114 149.64114 14964.11431 1.14114 298.14114 149.64114 14964.11400 2020-01-01 2020-01-02 2020-01-01 00:06:20 2020-01-02 03:34:41 2020-01-01 00:06:20.000 2020-01-02 03:34:41.000 380 99281 49830.5 4983050 380 99281 49830.5 4983050 -32189 32746 5266.02 526602 -128 124 0.1 10 -381 2 10371 99282 1.14414 298.14414 149.64414 14964.41441 1.14414 298.14413 149.64414 14964.41448 1.14414 298.14414 149.64414 14964.41400 2020-01-01 2020-01-02 2020-01-01 00:06:21 2020-01-02 03:34:42 2020-01-01 00:06:21.000 2020-01-02 03:34:42.000 381 99282 49831.5 4983150 381 99282 49831.5 4983150 -32188 32747 5267.02 526702 -127 125 1.1 110 -382 2 10372 99283 1.14714 298.14714 149.64714 14964.71471 1.14714 298.14716 149.64714 14964.71459 1.14714 298.14714 149.64714 14964.71400 2020-01-01 2020-01-02 2020-01-01 00:06:22 2020-01-02 03:34:43 2020-01-01 00:06:22.000 2020-01-02 03:34:43.000 382 99283 49832.5 4983250 382 99283 49832.5 4983250 -32187 32748 5268.02 526802 -126 126 2.1 210 +378 2 10368 99279 1.13513 298.13513 149.63513 14963.51351 1.13513 298.13513 149.63513 14963.51326 1.13513 298.13513 149.6351300000001 14963.51300 2020-01-01 2020-01-02 2020-01-01 00:06:18 2020-01-02 03:34:39 2020-01-01 00:06:18.000 2020-01-02 03:34:39.000 378 99279 49828.5 4982850 378 99279 49828.5 4982850 -32191 32744 5264.02 526402 -128 127 3.22 322 +379 2 10369 99280 1.13813 298.13813 149.63813 14963.81381 1.13813 298.13815 149.63814 14963.81401 1.13813 298.13813 149.63812999999993 14963.81300 2020-01-01 2020-01-02 2020-01-01 00:06:19 2020-01-02 03:34:40 2020-01-01 00:06:19.000 2020-01-02 03:34:40.000 379 99280 49829.5 4982950 379 99280 49829.5 4982950 -32190 32745 5265.02 526502 -128 127 1.66 166 +38 2 10028 99938 0.11411 300.11411 150.11411 15161.52552 0.11411 300.1141 150.11411 15161.52562 0.11411 300.11411 150.11411000000024 15161.52511 2020-01-01 2020-01-02 2020-01-01 00:00:38 2020-01-02 03:45:38 2020-01-01 00:00:38.000 2020-01-02 03:45:38.000 38 99938 49988 5048788 38 99938 49988 5048788 -32531 32404 4567.009900990099 461268 -126 125 -0.43564356435643564 -44 +380 2 10370 99281 1.14114 298.14114 149.64114 14964.11411 1.14114 298.14114 149.64114 14964.11431 1.14114 298.14114 149.64113999999987 14964.11400 2020-01-01 2020-01-02 2020-01-01 00:06:20 2020-01-02 03:34:41 2020-01-01 00:06:20.000 2020-01-02 03:34:41.000 380 99281 49830.5 4983050 380 99281 49830.5 4983050 -32189 32746 5266.02 526602 -128 124 0.1 10 +381 2 10371 99282 1.14414 298.14414 149.64414 14964.41441 1.14414 298.14413 149.64414 14964.41448 1.14414 298.14414 149.64414000000016 14964.41400 2020-01-01 2020-01-02 2020-01-01 00:06:21 2020-01-02 03:34:42 2020-01-01 00:06:21.000 2020-01-02 03:34:42.000 381 99282 49831.5 4983150 381 99282 49831.5 4983150 -32188 32747 5267.02 526702 -127 125 1.1 110 +382 2 10372 99283 1.14714 298.14714 149.64714 14964.71471 1.14714 298.14716 149.64714 14964.71459 1.14714 298.14714 149.64713999999978 14964.71400 2020-01-01 2020-01-02 2020-01-01 00:06:22 2020-01-02 03:34:43 2020-01-01 00:06:22.000 2020-01-02 03:34:43.000 382 99283 49832.5 4983250 382 99283 49832.5 4983250 -32187 32748 5268.02 526802 -126 126 2.1 210 383 2 10373 99284 1.15015 298.15015 149.65015 14965.01501 1.15015 298.15015 149.65014 14965.01472 1.15015 298.15015 149.65015 14965.01500 2020-01-01 2020-01-02 2020-01-01 00:06:23 2020-01-02 03:34:44 2020-01-01 00:06:23.000 2020-01-02 03:34:44.000 383 99284 49833.5 4983350 383 99284 49833.5 4983350 -32186 32749 5269.02 526902 -125 127 3.1 310 -384 2 10374 99285 1.15315 298.15315 149.65315 14965.31531 1.15315 298.15317 149.65315 14965.31547 1.15315 298.15315 149.65315 14965.31500 2020-01-01 2020-01-02 2020-01-01 00:06:24 2020-01-02 03:34:45 2020-01-01 00:06:24.000 2020-01-02 03:34:45.000 384 99285 49834.5 4983450 384 99285 49834.5 4983450 -32185 32750 5270.02 527002 -128 127 1.54 154 -385 2 10375 99286 1.15615 298.15615 149.65615 14965.61561 1.15615 298.15616 149.65615 14965.61578 1.15615 298.15615 149.65615 14965.61500 2020-01-01 2020-01-02 2020-01-01 00:06:25 2020-01-02 03:34:46 2020-01-01 00:06:25.000 2020-01-02 03:34:46.000 385 99286 49835.5 4983550 385 99286 49835.5 4983550 -32184 32751 5271.02 527102 -128 127 -0.02 -2 -386 2 10376 99287 1.15915 298.15915 149.65915 14965.91591 1.15915 298.15915 149.65915 14965.91594 1.15915 298.15915 149.65915 14965.91500 2020-01-01 2020-01-02 2020-01-01 00:06:26 2020-01-02 03:34:47 2020-01-01 00:06:26.000 2020-01-02 03:34:47.000 386 99287 49836.5 4983650 386 99287 49836.5 4983650 -32183 32752 5272.02 527202 -128 123 -1.58 -158 -387 2 10377 99288 1.16216 298.16216 149.66216 14966.21621 1.16216 298.16217 149.66216 14966.21606 1.16216 298.16216 149.66216 14966.21600 2020-01-01 2020-01-02 2020-01-01 00:06:27 2020-01-02 03:34:48 2020-01-01 00:06:27.000 2020-01-02 03:34:48.000 387 99288 49837.5 4983750 387 99288 49837.5 4983750 -32182 32753 5273.02 527302 -127 124 -0.58 -58 +384 2 10374 99285 1.15315 298.15315 149.65315 14965.31531 1.15315 298.15317 149.65315 14965.31547 1.15315 298.15315 149.65315000000007 14965.31500 2020-01-01 2020-01-02 2020-01-01 00:06:24 2020-01-02 03:34:45 2020-01-01 00:06:24.000 2020-01-02 03:34:45.000 384 99285 49834.5 4983450 384 99285 49834.5 4983450 -32185 32750 5270.02 527002 -128 127 1.54 154 +385 2 10375 99286 1.15615 298.15615 149.65615 14965.61561 1.15615 298.15616 149.65615 14965.61578 1.15615 298.15615 149.65615000000008 14965.61500 2020-01-01 2020-01-02 2020-01-01 00:06:25 2020-01-02 03:34:46 2020-01-01 00:06:25.000 2020-01-02 03:34:46.000 385 99286 49835.5 4983550 385 99286 49835.5 4983550 -32184 32751 5271.02 527102 -128 127 -0.02 -2 +386 2 10376 99287 1.15915 298.15915 149.65915 14965.91591 1.15915 298.15915 149.65915 14965.91594 1.15915 298.15915 149.65914999999998 14965.91500 2020-01-01 2020-01-02 2020-01-01 00:06:26 2020-01-02 03:34:47 2020-01-01 00:06:26.000 2020-01-02 03:34:47.000 386 99287 49836.5 4983650 386 99287 49836.5 4983650 -32183 32752 5272.02 527202 -128 123 -1.58 -158 +387 2 10377 99288 1.16216 298.16216 149.66216 14966.21621 1.16216 298.16217 149.66216 14966.21606 1.16216 298.16216 149.6621599999999 14966.21600 2020-01-01 2020-01-02 2020-01-01 00:06:27 2020-01-02 03:34:48 2020-01-01 00:06:27.000 2020-01-02 03:34:48.000 387 99288 49837.5 4983750 387 99288 49837.5 4983750 -32182 32753 5273.02 527302 -127 124 -0.58 -58 388 2 10378 99289 1.16516 298.16516 149.66516 14966.51651 1.16516 298.16516 149.66516 14966.51636 1.16516 298.16516 149.66516 14966.51600 2020-01-01 2020-01-02 2020-01-01 00:06:28 2020-01-02 03:34:49 2020-01-01 00:06:28.000 2020-01-02 03:34:49.000 388 99289 49838.5 4983850 388 99289 49838.5 4983850 -32181 32754 5274.02 527402 -126 125 0.42 42 -389 2 10379 99290 1.16816 298.16816 149.66816 14966.81681 1.16816 298.16818 149.66816 14966.81695 1.16816 298.16816 149.66816 14966.81600 2020-01-01 2020-01-02 2020-01-01 00:06:29 2020-01-02 03:34:50 2020-01-01 00:06:29.000 2020-01-02 03:34:50.000 389 99290 49839.5 4983950 389 99290 49839.5 4983950 -32180 32755 5275.02 527502 -125 126 1.42 142 -39 2 10029 99939 0.11711 300.11711 150.11711 15161.82882 0.11711 300.11713 150.11711 15161.82876 0.11711 300.11711 150.11711 15161.82811 2020-01-01 2020-01-02 2020-01-01 00:00:39 2020-01-02 03:45:39 2020-01-01 00:00:39.000 2020-01-02 03:45:39.000 39 99939 49989 5048889 39 99939 49989 5048889 -32530 32405 4568.009900990099 461369 -125 126 0.5643564356435643 57 -390 2 10380 99291 1.17117 298.17117 149.67117 14967.11711 1.17117 298.17117 149.67117 14967.11725 1.17117 298.17117 149.67117 14967.11700 2020-01-01 2020-01-02 2020-01-01 00:06:30 2020-01-02 03:34:51 2020-01-01 00:06:30.000 2020-01-02 03:34:51.000 390 99291 49840.5 4984050 390 99291 49840.5 4984050 -32179 32756 5276.02 527602 -124 127 2.42 242 -391 2 10381 99292 1.17417 298.17417 149.67417 14967.41741 1.17417 298.17416 149.67417 14967.41741 1.17417 298.17417 149.67417 14967.41700 2020-01-01 2020-01-02 2020-01-01 00:06:31 2020-01-02 03:34:52 2020-01-01 00:06:31.000 2020-01-02 03:34:52.000 391 99292 49841.5 4984150 391 99292 49841.5 4984150 -32178 32757 5277.02 527702 -128 127 0.86 86 -392 2 10382 99293 1.17717 298.17717 149.67717 14967.71771 1.17717 298.1772 149.67717 14967.71753 1.17717 298.17717 149.67717 14967.71700 2020-01-01 2020-01-02 2020-01-01 00:06:32 2020-01-02 03:34:53 2020-01-01 00:06:32.000 2020-01-02 03:34:53.000 392 99293 49842.5 4984250 392 99293 49842.5 4984250 -32177 32758 5278.02 527802 -128 123 -0.7 -70 -393 2 10383 99294 1.18018 298.18018 149.68018 14968.01801 1.18018 298.18018 149.68017 14968.01782 1.18018 298.18018 149.68018 14968.01800 2020-01-01 2020-01-02 2020-01-01 00:06:33 2020-01-02 03:34:54 2020-01-01 00:06:33.000 2020-01-02 03:34:54.000 393 99294 49843.5 4984350 393 99294 49843.5 4984350 -32176 32759 5279.02 527902 -127 124 0.3 30 +389 2 10379 99290 1.16816 298.16816 149.66816 14966.81681 1.16816 298.16818 149.66816 14966.81695 1.16816 298.16816 149.6681599999999 14966.81600 2020-01-01 2020-01-02 2020-01-01 00:06:29 2020-01-02 03:34:50 2020-01-01 00:06:29.000 2020-01-02 03:34:50.000 389 99290 49839.5 4983950 389 99290 49839.5 4983950 -32180 32755 5275.02 527502 -125 126 1.42 142 +39 2 10029 99939 0.11711 300.11711 150.11711 15161.82882 0.11711 300.11713 150.11711 15161.82876 0.11711 300.11711 150.1171099999999 15161.82811 2020-01-01 2020-01-02 2020-01-01 00:00:39 2020-01-02 03:45:39 2020-01-01 00:00:39.000 2020-01-02 03:45:39.000 39 99939 49989 5048889 39 99939 49989 5048889 -32530 32405 4568.009900990099 461369 -125 126 0.5643564356435643 57 +390 2 10380 99291 1.17117 298.17117 149.67117 14967.11711 1.17117 298.17117 149.67117 14967.11725 1.17117 298.17117 149.67116999999982 14967.11700 2020-01-01 2020-01-02 2020-01-01 00:06:30 2020-01-02 03:34:51 2020-01-01 00:06:30.000 2020-01-02 03:34:51.000 390 99291 49840.5 4984050 390 99291 49840.5 4984050 -32179 32756 5276.02 527602 -124 127 2.42 242 +391 2 10381 99292 1.17417 298.17417 149.67417 14967.41741 1.17417 298.17416 149.67417 14967.41741 1.17417 298.17417 149.67417000000015 14967.41700 2020-01-01 2020-01-02 2020-01-01 00:06:31 2020-01-02 03:34:52 2020-01-01 00:06:31.000 2020-01-02 03:34:52.000 391 99292 49841.5 4984150 391 99292 49841.5 4984150 -32178 32757 5277.02 527702 -128 127 0.86 86 +392 2 10382 99293 1.17717 298.17717 149.67717 14967.71771 1.17717 298.1772 149.67717 14967.71753 1.17717 298.17717 149.67717000000016 14967.71700 2020-01-01 2020-01-02 2020-01-01 00:06:32 2020-01-02 03:34:53 2020-01-01 00:06:32.000 2020-01-02 03:34:53.000 392 99293 49842.5 4984250 392 99293 49842.5 4984250 -32177 32758 5278.02 527802 -128 123 -0.7 -70 +393 2 10383 99294 1.18018 298.18018 149.68018 14968.01801 1.18018 298.18018 149.68017 14968.01782 1.18018 298.18018 149.68017999999995 14968.01800 2020-01-01 2020-01-02 2020-01-01 00:06:33 2020-01-02 03:34:54 2020-01-01 00:06:33.000 2020-01-02 03:34:54.000 393 99294 49843.5 4984350 393 99294 49843.5 4984350 -32176 32759 5279.02 527902 -127 124 0.3 30 394 2 10384 99295 1.18318 298.18318 149.68318 14968.31831 1.18318 298.1832 149.68318 14968.31842 1.18318 298.18318 149.68318 14968.31800 2020-01-01 2020-01-02 2020-01-01 00:06:34 2020-01-02 03:34:55 2020-01-01 00:06:34.000 2020-01-02 03:34:55.000 394 99295 49844.5 4984450 394 99295 49844.5 4984450 -32175 32760 5280.02 528002 -126 125 1.3 130 -395 2 10385 99296 1.18618 298.18618 149.68618 14968.61861 1.18618 298.1862 149.68618 14968.61875 1.18618 298.18618 149.68618 14968.61800 2020-01-01 2020-01-02 2020-01-01 00:06:35 2020-01-02 03:34:56 2020-01-01 00:06:35.000 2020-01-02 03:34:56.000 395 99296 49845.5 4984550 395 99296 49845.5 4984550 -32174 32761 5281.02 528102 -125 126 2.3 230 -396 2 10386 99297 1.18918 298.18918 149.68918 14968.91891 1.18918 298.18918 149.68918 14968.91889 1.18918 298.18918 149.68918 14968.91800 2020-01-01 2020-01-02 2020-01-01 00:06:36 2020-01-02 03:34:57 2020-01-01 00:06:36.000 2020-01-02 03:34:57.000 396 99297 49846.5 4984650 396 99297 49846.5 4984650 -32173 32762 5282.02 528202 -124 127 3.3 330 -397 2 10387 99298 1.19219 298.19219 149.69219 14969.21921 1.19219 298.1922 149.69219 14969.21964 1.19219 298.19219 149.69219 14969.21900 2020-01-01 2020-01-02 2020-01-01 00:06:37 2020-01-02 03:34:58 2020-01-01 00:06:37.000 2020-01-02 03:34:58.000 397 99298 49847.5 4984750 397 99298 49847.5 4984750 -32172 32763 5283.02 528302 -128 127 1.74 174 -398 2 10388 99299 1.19519 298.19519 149.69519 14969.51951 1.19519 298.1952 149.69519 14969.51929 1.19519 298.19519 149.69519 14969.51900 2020-01-01 2020-01-02 2020-01-01 00:06:38 2020-01-02 03:34:59 2020-01-01 00:06:38.000 2020-01-02 03:34:59.000 398 99299 49848.5 4984850 398 99299 49848.5 4984850 -32171 32764 5284.02 528402 -128 123 0.18 18 -399 2 10389 99300 1.19819 298.19819 149.69819 14969.81981 1.19819 298.1982 149.69819 14969.81989 1.19819 298.19819 149.69819 14969.81900 2020-01-01 2020-01-02 2020-01-01 00:06:39 2020-01-02 03:35:00 2020-01-01 00:06:39.000 2020-01-02 03:35:00.000 399 99300 49849.5 4984950 399 99300 49849.5 4984950 -32170 32765 5285.02 528502 -127 124 1.18 118 -4 2 1003 9994 0.01201 300.01201 150.01201 15151.21321 0.01201 300.01202 150.01201 15151.21318 0.01201 300.01201 150.01201 15151.21301 2020-01-01 2020-01-02 2020-01-01 00:00:04 2020-01-02 03:45:04 2020-01-01 00:00:04.000 2020-01-02 03:45:04.000 4 99904 49954 5045354 4 99904 49954 5045354 -32565 32370 4533.009900990099 457834 -128 127 -1.4851485148514851 -150 -40 2 10030 99940 0.12012 300.12012 150.12012 15162.13213 0.12012 300.12012 150.12011 15162.13191 0.12012 300.12012 150.12012 15162.13212 2020-01-01 2020-01-02 2020-01-01 00:00:40 2020-01-02 03:45:40 2020-01-01 00:00:40.000 2020-01-02 03:45:40.000 40 99940 49990 5048990 40 99940 49990 5048990 -32529 32406 4569.009900990099 461470 -124 127 1.5643564356435644 158 -400 2 10390 99301 1.2012 298.2012 149.7012 14970.12012 1.2012 298.2012 149.7012 14970.12022 1.20120 298.20120 149.70120 14970.12000 2020-01-01 2020-01-02 2020-01-01 00:06:40 2020-01-02 03:35:01 2020-01-01 00:06:40.000 2020-01-02 03:35:01.000 400 99301 49850.5 4985050 400 99301 49850.5 4985050 -32169 32766 5286.02 528602 -126 125 2.18 218 -401 2 10391 99302 1.2042 298.2042 149.7042 14970.42042 1.2042 298.2042 149.7042 14970.42035 1.20420 298.20420 149.70420 14970.42000 2020-01-01 2020-01-02 2020-01-01 00:06:41 2020-01-02 03:35:02 2020-01-01 00:06:41.000 2020-01-02 03:35:02.000 401 99302 49851.5 4985150 401 99302 49851.5 4985150 -32168 32767 5287.02 528702 -125 126 3.18 318 -402 2 10392 99303 1.2072 298.2072 149.7072 14970.72072 1.2072 298.2072 149.70721 14970.72111 1.20720 298.20720 149.70720 14970.72000 2020-01-01 2020-01-02 2020-01-01 00:06:42 2020-01-02 03:35:03 2020-01-01 00:06:42.000 2020-01-02 03:35:03.000 402 99303 49852.5 4985250 402 99303 49852.5 4985250 -32768 32370 4632.66 463266 -124 127 4.18 418 -403 2 10393 99304 1.21021 298.21021 149.71021 14971.02102 1.21021 298.2102 149.7102 14971.02077 1.21021 298.21021 149.71021 14971.02100 2020-01-01 2020-01-02 2020-01-01 00:06:43 2020-01-02 03:35:04 2020-01-01 00:06:43.000 2020-01-02 03:35:04.000 403 99304 49853.5 4985350 403 99304 49853.5 4985350 -32767 32371 4633.66 463366 -128 127 2.62 262 -404 2 10394 99305 1.21321 298.21321 149.71321 14971.32132 1.21321 298.21323 149.71321 14971.32139 1.21321 298.21321 149.71321 14971.32100 2020-01-01 2020-01-02 2020-01-01 00:06:44 2020-01-02 03:35:05 2020-01-01 00:06:44.000 2020-01-02 03:35:05.000 404 99305 49854.5 4985450 404 99305 49854.5 4985450 -32766 32372 4634.66 463466 -128 127 1.06 106 +395 2 10385 99296 1.18618 298.18618 149.68618 14968.61861 1.18618 298.1862 149.68618 14968.61875 1.18618 298.18618 149.68618000000006 14968.61800 2020-01-01 2020-01-02 2020-01-01 00:06:35 2020-01-02 03:34:56 2020-01-01 00:06:35.000 2020-01-02 03:34:56.000 395 99296 49845.5 4984550 395 99296 49845.5 4984550 -32174 32761 5281.02 528102 -125 126 2.3 230 +396 2 10386 99297 1.18918 298.18918 149.68918 14968.91891 1.18918 298.18918 149.68918 14968.91889 1.18918 298.18918 149.68917999999994 14968.91800 2020-01-01 2020-01-02 2020-01-01 00:06:36 2020-01-02 03:34:57 2020-01-01 00:06:36.000 2020-01-02 03:34:57.000 396 99297 49846.5 4984650 396 99297 49846.5 4984650 -32173 32762 5282.02 528202 -124 127 3.3 330 +397 2 10387 99298 1.19219 298.19219 149.69219 14969.21921 1.19219 298.1922 149.69219 14969.21964 1.19219 298.19219 149.6921899999999 14969.21900 2020-01-01 2020-01-02 2020-01-01 00:06:37 2020-01-02 03:34:58 2020-01-01 00:06:37.000 2020-01-02 03:34:58.000 397 99298 49847.5 4984750 397 99298 49847.5 4984750 -32172 32763 5283.02 528302 -128 127 1.74 174 +398 2 10388 99299 1.19519 298.19519 149.69519 14969.51951 1.19519 298.1952 149.69519 14969.51929 1.19519 298.19519 149.6951900000002 14969.51900 2020-01-01 2020-01-02 2020-01-01 00:06:38 2020-01-02 03:34:59 2020-01-01 00:06:38.000 2020-01-02 03:34:59.000 398 99299 49848.5 4984850 398 99299 49848.5 4984850 -32171 32764 5284.02 528402 -128 123 0.18 18 +399 2 10389 99300 1.19819 298.19819 149.69819 14969.81981 1.19819 298.1982 149.69819 14969.81989 1.19819 298.19819 149.69818999999987 14969.81900 2020-01-01 2020-01-02 2020-01-01 00:06:39 2020-01-02 03:35:00 2020-01-01 00:06:39.000 2020-01-02 03:35:00.000 399 99300 49849.5 4984950 399 99300 49849.5 4984950 -32170 32765 5285.02 528502 -127 124 1.18 118 +4 2 1003 9994 0.01201 300.01201 150.01201 15151.21321 0.01201 300.01202 150.01201 15151.21318 0.01201 300.01201 150.01201000000017 15151.21301 2020-01-01 2020-01-02 2020-01-01 00:00:04 2020-01-02 03:45:04 2020-01-01 00:00:04.000 2020-01-02 03:45:04.000 4 99904 49954 5045354 4 99904 49954 5045354 -32565 32370 4533.009900990099 457834 -128 127 -1.4851485148514851 -150 +40 2 10030 99940 0.12012 300.12012 150.12012 15162.13213 0.12012 300.12012 150.12011 15162.13191 0.12012 300.12012 150.1201199999998 15162.13212 2020-01-01 2020-01-02 2020-01-01 00:00:40 2020-01-02 03:45:40 2020-01-01 00:00:40.000 2020-01-02 03:45:40.000 40 99940 49990 5048990 40 99940 49990 5048990 -32529 32406 4569.009900990099 461470 -124 127 1.5643564356435644 158 +400 2 10390 99301 1.2012 298.2012 149.7012 14970.12012 1.2012 298.2012 149.7012 14970.12022 1.20120 298.20120 149.70120000000003 14970.12000 2020-01-01 2020-01-02 2020-01-01 00:06:40 2020-01-02 03:35:01 2020-01-01 00:06:40.000 2020-01-02 03:35:01.000 400 99301 49850.5 4985050 400 99301 49850.5 4985050 -32169 32766 5286.02 528602 -126 125 2.18 218 +401 2 10391 99302 1.2042 298.2042 149.7042 14970.42042 1.2042 298.2042 149.7042 14970.42035 1.20420 298.20420 149.7042000000001 14970.42000 2020-01-01 2020-01-02 2020-01-01 00:06:41 2020-01-02 03:35:02 2020-01-01 00:06:41.000 2020-01-02 03:35:02.000 401 99302 49851.5 4985150 401 99302 49851.5 4985150 -32168 32767 5287.02 528702 -125 126 3.18 318 +402 2 10392 99303 1.2072 298.2072 149.7072 14970.72072 1.2072 298.2072 149.70721 14970.72111 1.20720 298.20720 149.70720000000006 14970.72000 2020-01-01 2020-01-02 2020-01-01 00:06:42 2020-01-02 03:35:03 2020-01-01 00:06:42.000 2020-01-02 03:35:03.000 402 99303 49852.5 4985250 402 99303 49852.5 4985250 -32768 32370 4632.66 463266 -124 127 4.18 418 +403 2 10393 99304 1.21021 298.21021 149.71021 14971.02102 1.21021 298.2102 149.7102 14971.02077 1.21021 298.21021 149.71020999999993 14971.02100 2020-01-01 2020-01-02 2020-01-01 00:06:43 2020-01-02 03:35:04 2020-01-01 00:06:43.000 2020-01-02 03:35:04.000 403 99304 49853.5 4985350 403 99304 49853.5 4985350 -32767 32371 4633.66 463366 -128 127 2.62 262 +404 2 10394 99305 1.21321 298.21321 149.71321 14971.32132 1.21321 298.21323 149.71321 14971.32139 1.21321 298.21321 149.71320999999992 14971.32100 2020-01-01 2020-01-02 2020-01-01 00:06:44 2020-01-02 03:35:05 2020-01-01 00:06:44.000 2020-01-02 03:35:05.000 404 99305 49854.5 4985450 404 99305 49854.5 4985450 -32766 32372 4634.66 463466 -128 127 1.06 106 405 2 10395 99306 1.21621 298.21621 149.71621 14971.62162 1.21621 298.21622 149.71621 14971.62169 1.21621 298.21621 149.71621 14971.62100 2020-01-01 2020-01-02 2020-01-01 00:06:45 2020-01-02 03:35:06 2020-01-01 00:06:45.000 2020-01-02 03:35:06.000 405 99306 49855.5 4985550 405 99306 49855.5 4985550 -32765 32373 4635.66 463566 -128 124 -0.5 -50 -406 2 10396 99307 1.21921 298.21921 149.71921 14971.92192 1.21921 298.2192 149.71921 14971.92199 1.21921 298.21921 149.71921 14971.92100 2020-01-01 2020-01-02 2020-01-01 00:06:46 2020-01-02 03:35:07 2020-01-01 00:06:46.000 2020-01-02 03:35:07.000 406 99307 49856.5 4985650 406 99307 49856.5 4985650 -32764 32374 4636.66 463666 -127 125 0.5 50 -407 2 10397 99308 1.22222 298.22222 149.72222 14972.22222 1.22222 298.22223 149.72222 14972.22257 1.22222 298.22222 149.72222 14972.22200 2020-01-01 2020-01-02 2020-01-01 00:06:47 2020-01-02 03:35:08 2020-01-01 00:06:47.000 2020-01-02 03:35:08.000 407 99308 49857.5 4985750 407 99308 49857.5 4985750 -32763 32375 4637.66 463766 -126 126 1.5 150 -408 2 10398 99309 1.22522 298.22522 149.72522 14972.52252 1.22522 298.22522 149.72522 14972.52224 1.22522 298.22522 149.72522 14972.52200 2020-01-01 2020-01-02 2020-01-01 00:06:48 2020-01-02 03:35:09 2020-01-01 00:06:48.000 2020-01-02 03:35:09.000 408 99309 49858.5 4985850 408 99309 49858.5 4985850 -32762 32376 4638.66 463866 -125 127 2.5 250 -409 2 10399 99310 1.22822 298.22822 149.72822 14972.82282 1.22822 298.22824 149.72822 14972.82286 1.22822 298.22822 149.72822 14972.82200 2020-01-01 2020-01-02 2020-01-01 00:06:49 2020-01-02 03:35:10 2020-01-01 00:06:49.000 2020-01-02 03:35:10.000 409 99310 49859.5 4985950 409 99310 49859.5 4985950 -32761 32377 4639.66 463966 -128 127 0.94 94 -41 2 10031 99941 0.12312 300.12312 150.12312 15162.43543 0.12312 300.1231 150.12312 15162.43521 0.12312 300.12312 150.12312 15162.43512 2020-01-01 2020-01-02 2020-01-01 00:00:41 2020-01-02 03:45:41 2020-01-01 00:00:41.000 2020-01-02 03:45:41.000 41 99941 49991 5049091 41 99941 49991 5049091 -32528 32407 4570.009900990099 461571 -128 127 0.0297029702970297 3 -410 2 10400 99311 1.23123 298.23123 149.73123 14973.12312 1.23123 298.23123 149.73123 14973.12316 1.23123 298.23123 149.73123 14973.12300 2020-01-01 2020-01-02 2020-01-01 00:06:50 2020-01-02 03:35:11 2020-01-01 00:06:50.000 2020-01-02 03:35:11.000 410 99311 49860.5 4986050 410 99311 49860.5 4986050 -32760 32378 4640.66 464066 -128 127 -0.62 -62 +406 2 10396 99307 1.21921 298.21921 149.71921 14971.92192 1.21921 298.2192 149.71921 14971.92199 1.21921 298.21921 149.71920999999992 14971.92100 2020-01-01 2020-01-02 2020-01-01 00:06:46 2020-01-02 03:35:07 2020-01-01 00:06:46.000 2020-01-02 03:35:07.000 406 99307 49856.5 4985650 406 99307 49856.5 4985650 -32764 32374 4636.66 463666 -127 125 0.5 50 +407 2 10397 99308 1.22222 298.22222 149.72222 14972.22222 1.22222 298.22223 149.72222 14972.22257 1.22222 298.22222 149.72221999999982 14972.22200 2020-01-01 2020-01-02 2020-01-01 00:06:47 2020-01-02 03:35:08 2020-01-01 00:06:47.000 2020-01-02 03:35:08.000 407 99308 49857.5 4985750 407 99308 49857.5 4985750 -32763 32375 4637.66 463766 -126 126 1.5 150 +408 2 10398 99309 1.22522 298.22522 149.72522 14972.52252 1.22522 298.22522 149.72522 14972.52224 1.22522 298.22522 149.7252200000001 14972.52200 2020-01-01 2020-01-02 2020-01-01 00:06:48 2020-01-02 03:35:09 2020-01-01 00:06:48.000 2020-01-02 03:35:09.000 408 99309 49858.5 4985850 408 99309 49858.5 4985850 -32762 32376 4638.66 463866 -125 127 2.5 250 +409 2 10399 99310 1.22822 298.22822 149.72822 14972.82282 1.22822 298.22824 149.72822 14972.82286 1.22822 298.22822 149.7282200000002 14972.82200 2020-01-01 2020-01-02 2020-01-01 00:06:49 2020-01-02 03:35:10 2020-01-01 00:06:49.000 2020-01-02 03:35:10.000 409 99310 49859.5 4985950 409 99310 49859.5 4985950 -32761 32377 4639.66 463966 -128 127 0.94 94 +41 2 10031 99941 0.12312 300.12312 150.12312 15162.43543 0.12312 300.1231 150.12312 15162.43521 0.12312 300.12312 150.12312000000014 15162.43512 2020-01-01 2020-01-02 2020-01-01 00:00:41 2020-01-02 03:45:41 2020-01-01 00:00:41.000 2020-01-02 03:45:41.000 41 99941 49991 5049091 41 99941 49991 5049091 -32528 32407 4570.009900990099 461571 -128 127 0.0297029702970297 3 +410 2 10400 99311 1.23123 298.23123 149.73123 14973.12312 1.23123 298.23123 149.73123 14973.12316 1.23123 298.23123 149.73122999999995 14973.12300 2020-01-01 2020-01-02 2020-01-01 00:06:50 2020-01-02 03:35:11 2020-01-01 00:06:50.000 2020-01-02 03:35:11.000 410 99311 49860.5 4986050 410 99311 49860.5 4986050 -32760 32378 4640.66 464066 -128 127 -0.62 -62 411 2 10401 99312 1.23423 298.23423 149.73423 14973.42342 1.23423 298.23422 149.73423 14973.42345 1.23423 298.23423 149.73423 14973.42300 2020-01-01 2020-01-02 2020-01-01 00:06:51 2020-01-02 03:35:12 2020-01-01 00:06:51.000 2020-01-02 03:35:12.000 411 99312 49861.5 4986150 411 99312 49861.5 4986150 -32759 32379 4641.66 464166 -128 123 -2.18 -218 -412 2 10402 99313 1.23723 298.23723 149.73723 14973.72372 1.23723 298.23724 149.73724 14973.72405 1.23723 298.23723 149.73723 14973.72300 2020-01-01 2020-01-02 2020-01-01 00:06:52 2020-01-02 03:35:13 2020-01-01 00:06:52.000 2020-01-02 03:35:13.000 412 99313 49862.5 4986250 412 99313 49862.5 4986250 -32758 32380 4642.66 464266 -127 124 -1.18 -118 -413 2 10403 99314 1.24024 298.24024 149.74024 14974.02402 1.24024 298.24023 149.74023 14974.02374 1.24024 298.24024 149.74024 14974.02400 2020-01-01 2020-01-02 2020-01-01 00:06:53 2020-01-02 03:35:14 2020-01-01 00:06:53.000 2020-01-02 03:35:14.000 413 99314 49863.5 4986350 413 99314 49863.5 4986350 -32757 32381 4643.66 464366 -126 125 -0.18 -18 -414 2 10404 99315 1.24324 298.24324 149.74324 14974.32432 1.24324 298.24326 149.74324 14974.32433 1.24324 298.24324 149.74324 14974.32400 2020-01-01 2020-01-02 2020-01-01 00:06:54 2020-01-02 03:35:15 2020-01-01 00:06:54.000 2020-01-02 03:35:15.000 414 99315 49864.5 4986450 414 99315 49864.5 4986450 -32756 32382 4644.66 464466 -125 126 0.82 82 -415 2 10405 99316 1.24624 298.24624 149.74624 14974.62462 1.24624 298.24625 149.74624 14974.62463 1.24624 298.24624 149.74624 14974.62400 2020-01-01 2020-01-02 2020-01-01 00:06:55 2020-01-02 03:35:16 2020-01-01 00:06:55.000 2020-01-02 03:35:16.000 415 99316 49865.5 4986550 415 99316 49865.5 4986550 -32755 32383 4645.66 464566 -124 127 1.82 182 -416 2 10406 99317 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924 14974.92400 2020-01-01 2020-01-02 2020-01-01 00:06:56 2020-01-02 03:35:17 2020-01-01 00:06:56.000 2020-01-02 03:35:17.000 416 99317 49866.5 4986650 416 99317 49866.5 4986650 -32754 32384 4646.66 464666 -128 127 0.26 26 -417 2 10407 99318 1.25225 298.25225 149.75225 14975.22522 1.25225 298.25226 149.75225 14975.22552 1.25225 298.25225 149.75225 14975.22500 2020-01-01 2020-01-02 2020-01-01 00:06:57 2020-01-02 03:35:18 2020-01-01 00:06:57.000 2020-01-02 03:35:18.000 417 99318 49867.5 4986750 417 99318 49867.5 4986750 -32753 32385 4647.66 464766 -128 123 -1.3 -130 -418 2 10408 99319 1.25525 298.25525 149.75525 14975.52552 1.25525 298.25525 149.75525 14975.52521 1.25525 298.25525 149.75525 14975.52500 2020-01-01 2020-01-02 2020-01-01 00:06:58 2020-01-02 03:35:19 2020-01-01 00:06:58.000 2020-01-02 03:35:19.000 418 99319 49868.5 4986850 418 99319 49868.5 4986850 -32752 32386 4648.66 464866 -127 124 -0.3 -30 -419 2 10409 99320 1.25825 298.25825 149.75825 14975.82582 1.25825 298.25827 149.75825 14975.8258 1.25825 298.25825 149.75825 14975.82500 2020-01-01 2020-01-02 2020-01-01 00:06:59 2020-01-02 03:35:20 2020-01-01 00:06:59.000 2020-01-02 03:35:20.000 419 99320 49869.5 4986950 419 99320 49869.5 4986950 -32751 32387 4649.66 464966 -126 125 0.7 70 -42 2 10032 99942 0.12612 300.12612 150.12612 15162.73873 0.12612 300.12613 150.12612 15162.73896 0.12612 300.12612 150.12612 15162.73812 2020-01-01 2020-01-02 2020-01-01 00:00:42 2020-01-02 03:45:42 2020-01-01 00:00:42.000 2020-01-02 03:45:42.000 42 99942 49992 5049192 42 99942 49992 5049192 -32527 32408 4571.009900990099 461672 -128 127 -1.504950495049505 -152 -420 2 10410 99321 1.26126 298.26126 149.76126 14976.12612 1.26126 298.26126 149.76126 14976.12609 1.26126 298.26126 149.76126 14976.12600 2020-01-01 2020-01-02 2020-01-01 00:07:00 2020-01-02 03:35:21 2020-01-01 00:07:00.000 2020-01-02 03:35:21.000 420 99321 49870.5 4987050 420 99321 49870.5 4987050 -32750 32388 4650.66 465066 -125 126 1.7 170 -421 2 10411 99322 1.26426 298.26426 149.76426 14976.42642 1.26426 298.26425 149.76426 14976.4264 1.26426 298.26426 149.76426 14976.42600 2020-01-01 2020-01-02 2020-01-01 00:07:01 2020-01-02 03:35:22 2020-01-01 00:07:01.000 2020-01-02 03:35:22.000 421 99322 49871.5 4987150 421 99322 49871.5 4987150 -32749 32389 4651.66 465166 -124 127 2.7 270 +412 2 10402 99313 1.23723 298.23723 149.73723 14973.72372 1.23723 298.23724 149.73724 14973.72405 1.23723 298.23723 149.73723000000007 14973.72300 2020-01-01 2020-01-02 2020-01-01 00:06:52 2020-01-02 03:35:13 2020-01-01 00:06:52.000 2020-01-02 03:35:13.000 412 99313 49862.5 4986250 412 99313 49862.5 4986250 -32758 32380 4642.66 464266 -127 124 -1.18 -118 +413 2 10403 99314 1.24024 298.24024 149.74024 14974.02402 1.24024 298.24023 149.74023 14974.02374 1.24024 298.24024 149.74023999999983 14974.02400 2020-01-01 2020-01-02 2020-01-01 00:06:53 2020-01-02 03:35:14 2020-01-01 00:06:53.000 2020-01-02 03:35:14.000 413 99314 49863.5 4986350 413 99314 49863.5 4986350 -32757 32381 4643.66 464366 -126 125 -0.18 -18 +414 2 10404 99315 1.24324 298.24324 149.74324 14974.32432 1.24324 298.24326 149.74324 14974.32433 1.24324 298.24324 149.74323999999984 14974.32400 2020-01-01 2020-01-02 2020-01-01 00:06:54 2020-01-02 03:35:15 2020-01-01 00:06:54.000 2020-01-02 03:35:15.000 414 99315 49864.5 4986450 414 99315 49864.5 4986450 -32756 32382 4644.66 464466 -125 126 0.82 82 +415 2 10405 99316 1.24624 298.24624 149.74624 14974.62462 1.24624 298.24625 149.74624 14974.62463 1.24624 298.24624 149.74624000000023 14974.62400 2020-01-01 2020-01-02 2020-01-01 00:06:55 2020-01-02 03:35:16 2020-01-01 00:06:55.000 2020-01-02 03:35:16.000 415 99316 49865.5 4986550 415 99316 49865.5 4986550 -32755 32383 4645.66 464566 -124 127 1.82 182 +416 2 10406 99317 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74923999999987 14974.92400 2020-01-01 2020-01-02 2020-01-01 00:06:56 2020-01-02 03:35:17 2020-01-01 00:06:56.000 2020-01-02 03:35:17.000 416 99317 49866.5 4986650 416 99317 49866.5 4986650 -32754 32384 4646.66 464666 -128 127 0.26 26 +417 2 10407 99318 1.25225 298.25225 149.75225 14975.22522 1.25225 298.25226 149.75225 14975.22552 1.25225 298.25225 149.75224999999975 14975.22500 2020-01-01 2020-01-02 2020-01-01 00:06:57 2020-01-02 03:35:18 2020-01-01 00:06:57.000 2020-01-02 03:35:18.000 417 99318 49867.5 4986750 417 99318 49867.5 4986750 -32753 32385 4647.66 464766 -128 123 -1.3 -130 +418 2 10408 99319 1.25525 298.25525 149.75525 14975.52552 1.25525 298.25525 149.75525 14975.52521 1.25525 298.25525 149.7552500000001 14975.52500 2020-01-01 2020-01-02 2020-01-01 00:06:58 2020-01-02 03:35:19 2020-01-01 00:06:58.000 2020-01-02 03:35:19.000 418 99319 49868.5 4986850 418 99319 49868.5 4986850 -32752 32386 4648.66 464866 -127 124 -0.3 -30 +419 2 10409 99320 1.25825 298.25825 149.75825 14975.82582 1.25825 298.25827 149.75825 14975.8258 1.25825 298.25825 149.75825000000012 14975.82500 2020-01-01 2020-01-02 2020-01-01 00:06:59 2020-01-02 03:35:20 2020-01-01 00:06:59.000 2020-01-02 03:35:20.000 419 99320 49869.5 4986950 419 99320 49869.5 4986950 -32751 32387 4649.66 464966 -126 125 0.7 70 +42 2 10032 99942 0.12612 300.12612 150.12612 15162.73873 0.12612 300.12613 150.12612 15162.73896 0.12612 300.12612 150.12612000000007 15162.73812 2020-01-01 2020-01-02 2020-01-01 00:00:42 2020-01-02 03:45:42 2020-01-01 00:00:42.000 2020-01-02 03:45:42.000 42 99942 49992 5049192 42 99942 49992 5049192 -32527 32408 4571.009900990099 461672 -128 127 -1.504950495049505 -152 +420 2 10410 99321 1.26126 298.26126 149.76126 14976.12612 1.26126 298.26126 149.76126 14976.12609 1.26126 298.26126 149.76125999999994 14976.12600 2020-01-01 2020-01-02 2020-01-01 00:07:00 2020-01-02 03:35:21 2020-01-01 00:07:00.000 2020-01-02 03:35:21.000 420 99321 49870.5 4987050 420 99321 49870.5 4987050 -32750 32388 4650.66 465066 -125 126 1.7 170 +421 2 10411 99322 1.26426 298.26426 149.76426 14976.42642 1.26426 298.26425 149.76426 14976.4264 1.26426 298.26426 149.76425999999995 14976.42600 2020-01-01 2020-01-02 2020-01-01 00:07:01 2020-01-02 03:35:22 2020-01-01 00:07:01.000 2020-01-02 03:35:22.000 421 99322 49871.5 4987150 421 99322 49871.5 4987150 -32749 32389 4651.66 465166 -124 127 2.7 270 422 2 10412 99323 1.26726 298.26726 149.76726 14976.72672 1.26726 298.26727 149.76727 14976.72702 1.26726 298.26726 149.76726 14976.72600 2020-01-01 2020-01-02 2020-01-01 00:07:02 2020-01-02 03:35:23 2020-01-01 00:07:02.000 2020-01-02 03:35:23.000 422 99323 49872.5 4987250 422 99323 49872.5 4987250 -32748 32390 4652.66 465266 -128 127 1.14 114 -423 2 10413 99324 1.27027 298.27027 149.77027 14977.02702 1.27027 298.27026 149.77026 14977.02667 1.27027 298.27027 149.77027 14977.02700 2020-01-01 2020-01-02 2020-01-01 00:07:03 2020-01-02 03:35:24 2020-01-01 00:07:03.000 2020-01-02 03:35:24.000 423 99324 49873.5 4987350 423 99324 49873.5 4987350 -32747 32391 4653.66 465366 -128 123 -0.42 -42 -424 2 10414 99325 1.27327 298.27327 149.77327 14977.32732 1.27327 298.2733 149.77327 14977.32727 1.27327 298.27327 149.77327 14977.32700 2020-01-01 2020-01-02 2020-01-01 00:07:04 2020-01-02 03:35:25 2020-01-01 00:07:04.000 2020-01-02 03:35:25.000 424 99325 49874.5 4987450 424 99325 49874.5 4987450 -32746 32392 4654.66 465466 -127 124 0.58 58 -425 2 10415 99326 1.27627 298.27627 149.77627 14977.62762 1.27627 298.27628 149.77627 14977.62756 1.27627 298.27627 149.77627 14977.62700 2020-01-01 2020-01-02 2020-01-01 00:07:05 2020-01-02 03:35:26 2020-01-01 00:07:05.000 2020-01-02 03:35:26.000 425 99326 49875.5 4987550 425 99326 49875.5 4987550 -32745 32393 4655.66 465566 -126 125 1.58 158 -426 2 10416 99327 1.27927 298.27927 149.77927 14977.92792 1.27927 298.27927 149.77927 14977.92787 1.27927 298.27927 149.77927 14977.92700 2020-01-01 2020-01-02 2020-01-01 00:07:06 2020-01-02 03:35:27 2020-01-01 00:07:06.000 2020-01-02 03:35:27.000 426 99327 49876.5 4987650 426 99327 49876.5 4987650 -32744 32394 4656.66 465666 -125 126 2.58 258 -427 2 10417 99328 1.28228 298.28228 149.78228 14978.22822 1.28228 298.2823 149.78228 14978.22849 1.28228 298.28228 149.78228 14978.22800 2020-01-01 2020-01-02 2020-01-01 00:07:07 2020-01-02 03:35:28 2020-01-01 00:07:07.000 2020-01-02 03:35:28.000 427 99328 49877.5 4987750 427 99328 49877.5 4987750 -32743 32395 4657.66 465766 -124 127 3.58 358 -428 2 10418 99329 1.28528 298.28528 149.78528 14978.52852 1.28528 298.28528 149.78528 14978.52815 1.28528 298.28528 149.78528 14978.52800 2020-01-01 2020-01-02 2020-01-01 00:07:08 2020-01-02 03:35:29 2020-01-01 00:07:08.000 2020-01-02 03:35:29.000 428 99329 49878.5 4987850 428 99329 49878.5 4987850 -32742 32396 4658.66 465866 -128 127 2.02 202 -429 2 10419 99330 1.28828 298.28828 149.78828 14978.82882 1.28828 298.2883 149.78828 14978.8289 1.28828 298.28828 149.78828 14978.82800 2020-01-01 2020-01-02 2020-01-01 00:07:09 2020-01-02 03:35:30 2020-01-01 00:07:09.000 2020-01-02 03:35:30.000 429 99330 49879.5 4987950 429 99330 49879.5 4987950 -32741 32397 4659.66 465966 -128 127 0.46 46 -43 2 10033 99943 0.12912 300.12912 150.12912 15163.04204 0.12912 300.12912 150.12912 15163.04211 0.12912 300.12912 150.12912 15163.04112 2020-01-01 2020-01-02 2020-01-01 00:00:43 2020-01-02 03:45:43 2020-01-01 00:00:43.000 2020-01-02 03:45:43.000 43 99943 49993 5049293 43 99943 49993 5049293 -32526 32409 4572.009900990099 461773 -128 124 -3.0396039603960396 -307 -430 2 10420 99331 1.29129 298.29129 149.79129 14979.12912 1.29129 298.2913 149.79129 14979.12904 1.29129 298.29129 149.79129 14979.12900 2020-01-01 2020-01-02 2020-01-01 00:07:10 2020-01-02 03:35:31 2020-01-01 00:07:10.000 2020-01-02 03:35:31.000 430 99331 49880.5 4988050 430 99331 49880.5 4988050 -32740 32398 4660.66 466066 -128 124 -1.1 -110 -431 2 10421 99332 1.29429 298.29429 149.79429 14979.42942 1.29429 298.29428 149.79429 14979.42933 1.29429 298.29429 149.79429 14979.42900 2020-01-01 2020-01-02 2020-01-01 00:07:11 2020-01-02 03:35:32 2020-01-01 00:07:11.000 2020-01-02 03:35:32.000 431 99332 49881.5 4988150 431 99332 49881.5 4988150 -32739 32399 4661.66 466166 -127 125 -0.1 -10 -432 2 10422 99333 1.29729 298.29729 149.79729 14979.72972 1.29729 298.2973 149.79729 14979.72996 1.29729 298.29729 149.79729 14979.72900 2020-01-01 2020-01-02 2020-01-01 00:07:12 2020-01-02 03:35:33 2020-01-01 00:07:12.000 2020-01-02 03:35:33.000 432 99333 49882.5 4988250 432 99333 49882.5 4988250 -32738 32400 4662.66 466266 -126 126 0.9 90 -433 2 10423 99334 1.3003 298.3003 149.8003 14980.03003 1.3003 298.3003 149.80029 14980.02962 1.30030 298.30030 149.80030 14980.03000 2020-01-01 2020-01-02 2020-01-01 00:07:13 2020-01-02 03:35:34 2020-01-01 00:07:13.000 2020-01-02 03:35:34.000 433 99334 49883.5 4988350 433 99334 49883.5 4988350 -32737 32401 4663.66 466366 -125 127 1.9 190 -434 2 10424 99335 1.3033 298.3033 149.8033 14980.33033 1.3033 298.3033 149.8033 14980.33037 1.30330 298.30330 149.80330 14980.33000 2020-01-01 2020-01-02 2020-01-01 00:07:14 2020-01-02 03:35:35 2020-01-01 00:07:14.000 2020-01-02 03:35:35.000 434 99335 49884.5 4988450 434 99335 49884.5 4988450 -32736 32402 4664.66 466466 -128 127 0.34 34 -435 2 10425 99336 1.3063 298.3063 149.8063 14980.63063 1.3063 298.3063 149.8063 14980.63051 1.30630 298.30630 149.80630 14980.63000 2020-01-01 2020-01-02 2020-01-01 00:07:15 2020-01-02 03:35:36 2020-01-01 00:07:15.000 2020-01-02 03:35:36.000 435 99336 49885.5 4988550 435 99336 49885.5 4988550 -32735 32403 4665.66 466566 -128 127 -1.22 -122 -436 2 10426 99337 1.3093 298.3093 149.8093 14980.93093 1.3093 298.3093 149.8093 14980.93084 1.30930 298.30930 149.80930 14980.93000 2020-01-01 2020-01-02 2020-01-01 00:07:16 2020-01-02 03:35:37 2020-01-01 00:07:16.000 2020-01-02 03:35:37.000 436 99337 49886.5 4988650 436 99337 49886.5 4988650 -32734 32404 4666.66 466666 -128 123 -2.78 -278 -437 2 10427 99338 1.31231 298.31231 149.81231 14981.23123 1.31231 298.31232 149.81231 14981.23143 1.31231 298.31231 149.81231 14981.23100 2020-01-01 2020-01-02 2020-01-01 00:07:17 2020-01-02 03:35:38 2020-01-01 00:07:17.000 2020-01-02 03:35:38.000 437 99338 49887.5 4988750 437 99338 49887.5 4988750 -32733 32405 4667.66 466766 -127 124 -1.78 -178 -438 2 10428 99339 1.31531 298.31531 149.81531 14981.53153 1.31531 298.3153 149.81531 14981.53173 1.31531 298.31531 149.81531 14981.53100 2020-01-01 2020-01-02 2020-01-01 00:07:18 2020-01-02 03:35:39 2020-01-01 00:07:18.000 2020-01-02 03:35:39.000 438 99339 49888.5 4988850 438 99339 49888.5 4988850 -32732 32406 4668.66 466866 -126 125 -0.78 -78 -439 2 10429 99340 1.31831 298.31831 149.81831 14981.83183 1.31831 298.31833 149.81831 14981.83184 1.31831 298.31831 149.81831 14981.83100 2020-01-01 2020-01-02 2020-01-01 00:07:19 2020-01-02 03:35:40 2020-01-01 00:07:19.000 2020-01-02 03:35:40.000 439 99340 49889.5 4988950 439 99340 49889.5 4988950 -32731 32407 4669.66 466966 -125 126 0.22 22 +423 2 10413 99324 1.27027 298.27027 149.77027 14977.02702 1.27027 298.27026 149.77026 14977.02667 1.27027 298.27027 149.7702700000002 14977.02700 2020-01-01 2020-01-02 2020-01-01 00:07:03 2020-01-02 03:35:24 2020-01-01 00:07:03.000 2020-01-02 03:35:24.000 423 99324 49873.5 4987350 423 99324 49873.5 4987350 -32747 32391 4653.66 465366 -128 123 -0.42 -42 +424 2 10414 99325 1.27327 298.27327 149.77327 14977.32732 1.27327 298.2733 149.77327 14977.32727 1.27327 298.27327 149.77326999999985 14977.32700 2020-01-01 2020-01-02 2020-01-01 00:07:04 2020-01-02 03:35:25 2020-01-01 00:07:04.000 2020-01-02 03:35:25.000 424 99325 49874.5 4987450 424 99325 49874.5 4987450 -32746 32392 4654.66 465466 -127 124 0.58 58 +425 2 10415 99326 1.27627 298.27627 149.77627 14977.62762 1.27627 298.27628 149.77627 14977.62756 1.27627 298.27627 149.77627000000012 14977.62700 2020-01-01 2020-01-02 2020-01-01 00:07:05 2020-01-02 03:35:26 2020-01-01 00:07:05.000 2020-01-02 03:35:26.000 425 99326 49875.5 4987550 425 99326 49875.5 4987550 -32745 32393 4655.66 465566 -126 125 1.58 158 +426 2 10416 99327 1.27927 298.27927 149.77927 14977.92792 1.27927 298.27927 149.77927 14977.92787 1.27927 298.27927 149.7792699999999 14977.92700 2020-01-01 2020-01-02 2020-01-01 00:07:06 2020-01-02 03:35:27 2020-01-01 00:07:06.000 2020-01-02 03:35:27.000 426 99327 49876.5 4987650 426 99327 49876.5 4987650 -32744 32394 4656.66 465666 -125 126 2.58 258 +427 2 10417 99328 1.28228 298.28228 149.78228 14978.22822 1.28228 298.2823 149.78228 14978.22849 1.28228 298.28228 149.78227999999996 14978.22800 2020-01-01 2020-01-02 2020-01-01 00:07:07 2020-01-02 03:35:28 2020-01-01 00:07:07.000 2020-01-02 03:35:28.000 427 99328 49877.5 4987750 427 99328 49877.5 4987750 -32743 32395 4657.66 465766 -124 127 3.58 358 +428 2 10418 99329 1.28528 298.28528 149.78528 14978.52852 1.28528 298.28528 149.78528 14978.52815 1.28528 298.28528 149.78528000000003 14978.52800 2020-01-01 2020-01-02 2020-01-01 00:07:08 2020-01-02 03:35:29 2020-01-01 00:07:08.000 2020-01-02 03:35:29.000 428 99329 49878.5 4987850 428 99329 49878.5 4987850 -32742 32396 4658.66 465866 -128 127 2.02 202 +429 2 10419 99330 1.28828 298.28828 149.78828 14978.82882 1.28828 298.2883 149.78828 14978.8289 1.28828 298.28828 149.7882800000001 14978.82800 2020-01-01 2020-01-02 2020-01-01 00:07:09 2020-01-02 03:35:30 2020-01-01 00:07:09.000 2020-01-02 03:35:30.000 429 99330 49879.5 4987950 429 99330 49879.5 4987950 -32741 32397 4659.66 465966 -128 127 0.46 46 +43 2 10033 99943 0.12912 300.12912 150.12912 15163.04204 0.12912 300.12912 150.12912 15163.04211 0.12912 300.12912 150.1291199999998 15163.04112 2020-01-01 2020-01-02 2020-01-01 00:00:43 2020-01-02 03:45:43 2020-01-01 00:00:43.000 2020-01-02 03:45:43.000 43 99943 49993 5049293 43 99943 49993 5049293 -32526 32409 4572.009900990099 461773 -128 124 -3.0396039603960396 -307 +430 2 10420 99331 1.29129 298.29129 149.79129 14979.12912 1.29129 298.2913 149.79129 14979.12904 1.29129 298.29129 149.7912899999999 14979.12900 2020-01-01 2020-01-02 2020-01-01 00:07:10 2020-01-02 03:35:31 2020-01-01 00:07:10.000 2020-01-02 03:35:31.000 430 99331 49880.5 4988050 430 99331 49880.5 4988050 -32740 32398 4660.66 466066 -128 124 -1.1 -110 +431 2 10421 99332 1.29429 298.29429 149.79429 14979.42942 1.29429 298.29428 149.79429 14979.42933 1.29429 298.29429 149.79428999999988 14979.42900 2020-01-01 2020-01-02 2020-01-01 00:07:11 2020-01-02 03:35:32 2020-01-01 00:07:11.000 2020-01-02 03:35:32.000 431 99332 49881.5 4988150 431 99332 49881.5 4988150 -32739 32399 4661.66 466166 -127 125 -0.1 -10 +432 2 10422 99333 1.29729 298.29729 149.79729 14979.72972 1.29729 298.2973 149.79729 14979.72996 1.29729 298.29729 149.79729000000023 14979.72900 2020-01-01 2020-01-02 2020-01-01 00:07:12 2020-01-02 03:35:33 2020-01-01 00:07:12.000 2020-01-02 03:35:33.000 432 99333 49882.5 4988250 432 99333 49882.5 4988250 -32738 32400 4662.66 466266 -126 126 0.9 90 +433 2 10423 99334 1.3003 298.3003 149.8003 14980.03003 1.3003 298.3003 149.80029 14980.02962 1.30030 298.30030 149.8003000000001 14980.03000 2020-01-01 2020-01-02 2020-01-01 00:07:13 2020-01-02 03:35:34 2020-01-01 00:07:13.000 2020-01-02 03:35:34.000 433 99334 49883.5 4988350 433 99334 49883.5 4988350 -32737 32401 4663.66 466366 -125 127 1.9 190 +434 2 10424 99335 1.3033 298.3033 149.8033 14980.33033 1.3033 298.3033 149.8033 14980.33037 1.30330 298.30330 149.80329999999978 14980.33000 2020-01-01 2020-01-02 2020-01-01 00:07:14 2020-01-02 03:35:35 2020-01-01 00:07:14.000 2020-01-02 03:35:35.000 434 99335 49884.5 4988450 434 99335 49884.5 4988450 -32736 32402 4664.66 466466 -128 127 0.34 34 +435 2 10425 99336 1.3063 298.3063 149.8063 14980.63063 1.3063 298.3063 149.8063 14980.63051 1.30630 298.30630 149.80630000000014 14980.63000 2020-01-01 2020-01-02 2020-01-01 00:07:15 2020-01-02 03:35:36 2020-01-01 00:07:15.000 2020-01-02 03:35:36.000 435 99336 49885.5 4988550 435 99336 49885.5 4988550 -32735 32403 4665.66 466566 -128 127 -1.22 -122 +436 2 10426 99337 1.3093 298.3093 149.8093 14980.93093 1.3093 298.3093 149.8093 14980.93084 1.30930 298.30930 149.80930000000018 14980.93000 2020-01-01 2020-01-02 2020-01-01 00:07:16 2020-01-02 03:35:37 2020-01-01 00:07:16.000 2020-01-02 03:35:37.000 436 99337 49886.5 4988650 436 99337 49886.5 4988650 -32734 32404 4666.66 466666 -128 123 -2.78 -278 +437 2 10427 99338 1.31231 298.31231 149.81231 14981.23123 1.31231 298.31232 149.81231 14981.23143 1.31231 298.31231 149.8123099999999 14981.23100 2020-01-01 2020-01-02 2020-01-01 00:07:17 2020-01-02 03:35:38 2020-01-01 00:07:17.000 2020-01-02 03:35:38.000 437 99338 49887.5 4988750 437 99338 49887.5 4988750 -32733 32405 4667.66 466766 -127 124 -1.78 -178 +438 2 10428 99339 1.31531 298.31531 149.81531 14981.53153 1.31531 298.3153 149.81531 14981.53173 1.31531 298.31531 149.81530999999998 14981.53100 2020-01-01 2020-01-02 2020-01-01 00:07:18 2020-01-02 03:35:39 2020-01-01 00:07:18.000 2020-01-02 03:35:39.000 438 99339 49888.5 4988850 438 99339 49888.5 4988850 -32732 32406 4668.66 466866 -126 125 -0.78 -78 +439 2 10429 99340 1.31831 298.31831 149.81831 14981.83183 1.31831 298.31833 149.81831 14981.83184 1.31831 298.31831 149.81831000000005 14981.83100 2020-01-01 2020-01-02 2020-01-01 00:07:19 2020-01-02 03:35:40 2020-01-01 00:07:19.000 2020-01-02 03:35:40.000 439 99340 49889.5 4988950 439 99340 49889.5 4988950 -32731 32407 4669.66 466966 -125 126 0.22 22 44 2 10034 99944 0.13213 300.13213 150.13213 15163.34534 0.13213 300.13214 150.13213 15163.34525 0.13213 300.13213 150.13213 15163.34513 2020-01-01 2020-01-02 2020-01-01 00:00:44 2020-01-02 03:45:44 2020-01-01 00:00:44.000 2020-01-02 03:45:44.000 44 99944 49994 5049394 44 99944 49994 5049394 -32525 32410 4573.009900990099 461874 -127 125 -2.0396039603960396 -206 -440 2 10430 99341 1.32132 298.32132 149.82132 14982.13213 1.32132 298.32132 149.82131 14982.13197 1.32132 298.32132 149.82132 14982.13200 2020-01-01 2020-01-02 2020-01-01 00:07:20 2020-01-02 03:35:41 2020-01-01 00:07:20.000 2020-01-02 03:35:41.000 440 99341 49890.5 4989050 440 99341 49890.5 4989050 -32730 32408 4670.66 467066 -124 127 1.22 122 -441 2 10431 99342 1.32432 298.32432 149.82432 14982.43243 1.32432 298.3243 149.82432 14982.4323 1.32432 298.32432 149.82432 14982.43200 2020-01-01 2020-01-02 2020-01-01 00:07:21 2020-01-02 03:35:42 2020-01-01 00:07:21.000 2020-01-02 03:35:42.000 441 99342 49891.5 4989150 441 99342 49891.5 4989150 -32729 32409 4671.66 467166 -128 127 -0.34 -34 -442 2 10432 99343 1.32732 298.32732 149.82732 14982.73273 1.32732 298.32733 149.82732 14982.7329 1.32732 298.32732 149.82732 14982.73200 2020-01-01 2020-01-02 2020-01-01 00:07:22 2020-01-02 03:35:43 2020-01-01 00:07:22.000 2020-01-02 03:35:43.000 442 99343 49892.5 4989250 442 99343 49892.5 4989250 -32728 32410 4672.66 467266 -128 123 -1.9 -190 -443 2 10433 99344 1.33033 298.33033 149.83033 14983.03303 1.33033 298.33032 149.83033 14983.03319 1.33033 298.33033 149.83033 14983.03300 2020-01-01 2020-01-02 2020-01-01 00:07:23 2020-01-02 03:35:44 2020-01-01 00:07:23.000 2020-01-02 03:35:44.000 443 99344 49893.5 4989350 443 99344 49893.5 4989350 -32727 32411 4673.66 467366 -127 124 -0.9 -90 +440 2 10430 99341 1.32132 298.32132 149.82132 14982.13213 1.32132 298.32132 149.82131 14982.13197 1.32132 298.32132 149.82131999999984 14982.13200 2020-01-01 2020-01-02 2020-01-01 00:07:20 2020-01-02 03:35:41 2020-01-01 00:07:20.000 2020-01-02 03:35:41.000 440 99341 49890.5 4989050 440 99341 49890.5 4989050 -32730 32408 4670.66 467066 -124 127 1.22 122 +441 2 10431 99342 1.32432 298.32432 149.82432 14982.43243 1.32432 298.3243 149.82432 14982.4323 1.32432 298.32432 149.8243199999999 14982.43200 2020-01-01 2020-01-02 2020-01-01 00:07:21 2020-01-02 03:35:42 2020-01-01 00:07:21.000 2020-01-02 03:35:42.000 441 99342 49891.5 4989150 441 99342 49891.5 4989150 -32729 32409 4671.66 467166 -128 127 -0.34 -34 +442 2 10432 99343 1.32732 298.32732 149.82732 14982.73273 1.32732 298.32733 149.82732 14982.7329 1.32732 298.32732 149.82732000000016 14982.73200 2020-01-01 2020-01-02 2020-01-01 00:07:22 2020-01-02 03:35:43 2020-01-01 00:07:22.000 2020-01-02 03:35:43.000 442 99343 49892.5 4989250 442 99343 49892.5 4989250 -32728 32410 4672.66 467266 -128 123 -1.9 -190 +443 2 10433 99344 1.33033 298.33033 149.83033 14983.03303 1.33033 298.33032 149.83033 14983.03319 1.33033 298.33033 149.83033000000006 14983.03300 2020-01-01 2020-01-02 2020-01-01 00:07:23 2020-01-02 03:35:44 2020-01-01 00:07:23.000 2020-01-02 03:35:44.000 443 99344 49893.5 4989350 443 99344 49893.5 4989350 -32727 32411 4673.66 467366 -127 124 -0.9 -90 444 2 10434 99345 1.33333 298.33333 149.83333 14983.33333 1.33333 298.33334 149.83333 14983.33331 1.33333 298.33333 149.83333 14983.33300 2020-01-01 2020-01-02 2020-01-01 00:07:24 2020-01-02 03:35:45 2020-01-01 00:07:24.000 2020-01-02 03:35:45.000 444 99345 49894.5 4989450 444 99345 49894.5 4989450 -32726 32412 4674.66 467466 -126 125 0.1 10 -445 2 10435 99346 1.33633 298.33633 149.83633 14983.63363 1.33633 298.33633 149.83633 14983.63348 1.33633 298.33633 149.83633 14983.63300 2020-01-01 2020-01-02 2020-01-01 00:07:25 2020-01-02 03:35:46 2020-01-01 00:07:25.000 2020-01-02 03:35:46.000 445 99346 49895.5 4989550 445 99346 49895.5 4989550 -32725 32413 4675.66 467566 -125 126 1.1 110 -446 2 10436 99347 1.33933 298.33933 149.83933 14983.93393 1.33933 298.33932 149.83933 14983.93378 1.33933 298.33933 149.83933 14983.93300 2020-01-01 2020-01-02 2020-01-01 00:07:26 2020-01-02 03:35:47 2020-01-01 00:07:26.000 2020-01-02 03:35:47.000 446 99347 49896.5 4989650 446 99347 49896.5 4989650 -32724 32414 4676.66 467666 -124 127 2.1 210 -447 2 10437 99348 1.34234 298.34234 149.84234 14984.23423 1.34234 298.34235 149.84234 14984.23437 1.34234 298.34234 149.84234 14984.23400 2020-01-01 2020-01-02 2020-01-01 00:07:27 2020-01-02 03:35:48 2020-01-01 00:07:27.000 2020-01-02 03:35:48.000 447 99348 49897.5 4989750 447 99348 49897.5 4989750 -32723 32415 4677.66 467766 -128 127 0.54 54 -448 2 10438 99349 1.34534 298.34534 149.84534 14984.53453 1.34534 298.34534 149.84534 14984.53466 1.34534 298.34534 149.84534 14984.53400 2020-01-01 2020-01-02 2020-01-01 00:07:28 2020-01-02 03:35:49 2020-01-01 00:07:28.000 2020-01-02 03:35:49.000 448 99349 49898.5 4989850 448 99349 49898.5 4989850 -32722 32416 4678.66 467866 -128 123 -1.02 -102 -449 2 10439 99350 1.34834 298.34834 149.84834 14984.83483 1.34834 298.34836 149.84834 14984.83478 1.34834 298.34834 149.84834 14984.83400 2020-01-01 2020-01-02 2020-01-01 00:07:29 2020-01-02 03:35:50 2020-01-01 00:07:29.000 2020-01-02 03:35:50.000 449 99350 49899.5 4989950 449 99350 49899.5 4989950 -32721 32417 4679.66 467966 -127 124 -0.02 -2 -45 2 10035 99945 0.13513 300.13513 150.13513 15163.64864 0.13513 300.13513 150.13513 15163.64839 0.13513 300.13513 150.13513 15163.64813 2020-01-01 2020-01-02 2020-01-01 00:00:45 2020-01-02 03:45:45 2020-01-01 00:00:45.000 2020-01-02 03:45:45.000 45 99945 49995 5049495 45 99945 49995 5049495 -32524 32411 4574.009900990099 461975 -126 126 -1.0396039603960396 -105 -450 2 10440 99351 1.35135 298.35135 149.85135 14985.13513 1.35135 298.35135 149.85134 14985.13495 1.35135 298.35135 149.85135 14985.13500 2020-01-01 2020-01-02 2020-01-01 00:07:30 2020-01-02 03:35:51 2020-01-01 00:07:30.000 2020-01-02 03:35:51.000 450 99351 49900.5 4990050 450 99351 49900.5 4990050 -32720 32418 4680.66 468066 -126 125 0.98 98 -451 2 10441 99352 1.35435 298.35435 149.85435 14985.43543 1.35435 298.35434 149.85435 14985.43525 1.35435 298.35435 149.85435 14985.43500 2020-01-01 2020-01-02 2020-01-01 00:07:31 2020-01-02 03:35:52 2020-01-01 00:07:31.000 2020-01-02 03:35:52.000 451 99352 49901.5 4990150 451 99352 49901.5 4990150 -32719 32419 4681.66 468166 -125 126 1.98 198 -452 2 10442 99353 1.35735 298.35735 149.85735 14985.73573 1.35735 298.35736 149.85736 14985.736 1.35735 298.35735 149.85735 14985.73500 2020-01-01 2020-01-02 2020-01-01 00:07:32 2020-01-02 03:35:53 2020-01-01 00:07:32.000 2020-01-02 03:35:53.000 452 99353 49902.5 4990250 452 99353 49902.5 4990250 -32718 32420 4682.66 468266 -124 127 2.98 298 -453 2 10443 99354 1.36036 298.36036 149.86036 14986.03603 1.36036 298.36035 149.86036 14986.03614 1.36036 298.36036 149.86036 14986.03600 2020-01-01 2020-01-02 2020-01-01 00:07:33 2020-01-02 03:35:54 2020-01-01 00:07:33.000 2020-01-02 03:35:54.000 453 99354 49903.5 4990350 453 99354 49903.5 4990350 -32717 32421 4683.66 468366 -128 127 1.42 142 -454 2 10444 99355 1.36336 298.36336 149.86336 14986.33633 1.36336 298.36337 149.86336 14986.33629 1.36336 298.36336 149.86336 14986.33600 2020-01-01 2020-01-02 2020-01-01 00:07:34 2020-01-02 03:35:55 2020-01-01 00:07:34.000 2020-01-02 03:35:55.000 454 99355 49904.5 4990450 454 99355 49904.5 4990450 -32716 32422 4684.66 468466 -128 127 -0.14 -14 +445 2 10435 99346 1.33633 298.33633 149.83633 14983.63363 1.33633 298.33633 149.83633 14983.63348 1.33633 298.33633 149.83633000000006 14983.63300 2020-01-01 2020-01-02 2020-01-01 00:07:25 2020-01-02 03:35:46 2020-01-01 00:07:25.000 2020-01-02 03:35:46.000 445 99346 49895.5 4989550 445 99346 49895.5 4989550 -32725 32413 4675.66 467566 -125 126 1.1 110 +446 2 10436 99347 1.33933 298.33933 149.83933 14983.93393 1.33933 298.33932 149.83933 14983.93378 1.33933 298.33933 149.83933000000007 14983.93300 2020-01-01 2020-01-02 2020-01-01 00:07:26 2020-01-02 03:35:47 2020-01-01 00:07:26.000 2020-01-02 03:35:47.000 446 99347 49896.5 4989650 446 99347 49896.5 4989650 -32724 32414 4676.66 467666 -124 127 2.1 210 +447 2 10437 99348 1.34234 298.34234 149.84234 14984.23423 1.34234 298.34235 149.84234 14984.23437 1.34234 298.34234 149.84233999999992 14984.23400 2020-01-01 2020-01-02 2020-01-01 00:07:27 2020-01-02 03:35:48 2020-01-01 00:07:27.000 2020-01-02 03:35:48.000 447 99348 49897.5 4989750 447 99348 49897.5 4989750 -32723 32415 4677.66 467766 -128 127 0.54 54 +448 2 10438 99349 1.34534 298.34534 149.84534 14984.53453 1.34534 298.34534 149.84534 14984.53466 1.34534 298.34534 149.8453399999999 14984.53400 2020-01-01 2020-01-02 2020-01-01 00:07:28 2020-01-02 03:35:49 2020-01-01 00:07:28.000 2020-01-02 03:35:49.000 448 99349 49898.5 4989850 448 99349 49898.5 4989850 -32722 32416 4678.66 467866 -128 123 -1.02 -102 +449 2 10439 99350 1.34834 298.34834 149.84834 14984.83483 1.34834 298.34836 149.84834 14984.83478 1.34834 298.34834 149.8483400000001 14984.83400 2020-01-01 2020-01-02 2020-01-01 00:07:29 2020-01-02 03:35:50 2020-01-01 00:07:29.000 2020-01-02 03:35:50.000 449 99350 49899.5 4989950 449 99350 49899.5 4989950 -32721 32417 4679.66 467966 -127 124 -0.02 -2 +45 2 10035 99945 0.13513 300.13513 150.13513 15163.64864 0.13513 300.13513 150.13513 15163.64839 0.13513 300.13513 150.13513000000006 15163.64813 2020-01-01 2020-01-02 2020-01-01 00:00:45 2020-01-02 03:45:45 2020-01-01 00:00:45.000 2020-01-02 03:45:45.000 45 99945 49995 5049495 45 99945 49995 5049495 -32524 32411 4574.009900990099 461975 -126 126 -1.0396039603960396 -105 +450 2 10440 99351 1.35135 298.35135 149.85135 14985.13513 1.35135 298.35135 149.85134 14985.13495 1.35135 298.35135 149.85135000000014 14985.13500 2020-01-01 2020-01-02 2020-01-01 00:07:30 2020-01-02 03:35:51 2020-01-01 00:07:30.000 2020-01-02 03:35:51.000 450 99351 49900.5 4990050 450 99351 49900.5 4990050 -32720 32418 4680.66 468066 -126 125 0.98 98 +451 2 10441 99352 1.35435 298.35435 149.85435 14985.43543 1.35435 298.35434 149.85435 14985.43525 1.35435 298.35435 149.85434999999978 14985.43500 2020-01-01 2020-01-02 2020-01-01 00:07:31 2020-01-02 03:35:52 2020-01-01 00:07:31.000 2020-01-02 03:35:52.000 451 99352 49901.5 4990150 451 99352 49901.5 4990150 -32719 32419 4681.66 468166 -125 126 1.98 198 +452 2 10442 99353 1.35735 298.35735 149.85735 14985.73573 1.35735 298.35736 149.85736 14985.736 1.35735 298.35735 149.85735000000017 14985.73500 2020-01-01 2020-01-02 2020-01-01 00:07:32 2020-01-02 03:35:53 2020-01-01 00:07:32.000 2020-01-02 03:35:53.000 452 99353 49902.5 4990250 452 99353 49902.5 4990250 -32718 32420 4682.66 468266 -124 127 2.98 298 +453 2 10443 99354 1.36036 298.36036 149.86036 14986.03603 1.36036 298.36035 149.86036 14986.03614 1.36036 298.36036 149.86036000000007 14986.03600 2020-01-01 2020-01-02 2020-01-01 00:07:33 2020-01-02 03:35:54 2020-01-01 00:07:33.000 2020-01-02 03:35:54.000 453 99354 49903.5 4990350 453 99354 49903.5 4990350 -32717 32421 4683.66 468366 -128 127 1.42 142 +454 2 10444 99355 1.36336 298.36336 149.86336 14986.33633 1.36336 298.36337 149.86336 14986.33629 1.36336 298.36336 149.8633599999999 14986.33600 2020-01-01 2020-01-02 2020-01-01 00:07:34 2020-01-02 03:35:55 2020-01-01 00:07:34.000 2020-01-02 03:35:55.000 454 99355 49904.5 4990450 454 99355 49904.5 4990450 -32716 32422 4684.66 468466 -128 127 -0.14 -14 455 2 10445 99356 1.36636 298.36636 149.86636 14986.63663 1.36636 298.36636 149.86636 14986.63641 1.36636 298.36636 149.86636 14986.63600 2020-01-01 2020-01-02 2020-01-01 00:07:35 2020-01-02 03:35:56 2020-01-01 00:07:35.000 2020-01-02 03:35:56.000 455 99356 49905.5 4990550 455 99356 49905.5 4990550 -32715 32423 4685.66 468566 -128 124 -1.7 -170 -456 2 10446 99357 1.36936 298.36936 149.86936 14986.93693 1.36936 298.36935 149.86936 14986.93672 1.36936 298.36936 149.86936 14986.93600 2020-01-01 2020-01-02 2020-01-01 00:07:36 2020-01-02 03:35:57 2020-01-01 00:07:36.000 2020-01-02 03:35:57.000 456 99357 49906.5 4990650 456 99357 49906.5 4990650 -32714 32424 4686.66 468666 -127 125 -0.7 -70 -457 2 10447 99358 1.37237 298.37237 149.87237 14987.23723 1.37237 298.37238 149.87237 14987.23747 1.37237 298.37237 149.87237 14987.23700 2020-01-01 2020-01-02 2020-01-01 00:07:37 2020-01-02 03:35:58 2020-01-01 00:07:37.000 2020-01-02 03:35:58.000 457 99358 49907.5 4990750 457 99358 49907.5 4990750 -32713 32425 4687.66 468766 -126 126 0.3 30 -458 2 10448 99359 1.37537 298.37537 149.87537 14987.53753 1.37537 298.37537 149.87537 14987.5376 1.37537 298.37537 149.87537 14987.53700 2020-01-01 2020-01-02 2020-01-01 00:07:38 2020-01-02 03:35:59 2020-01-01 00:07:38.000 2020-01-02 03:35:59.000 458 99359 49908.5 4990850 458 99359 49908.5 4990850 -32712 32426 4688.66 468866 -125 127 1.3 130 -459 2 10449 99360 1.37837 298.37837 149.87837 14987.83783 1.37837 298.3784 149.87837 14987.83775 1.37837 298.37837 149.87837 14987.83700 2020-01-01 2020-01-02 2020-01-01 00:07:39 2020-01-02 03:36:00 2020-01-01 00:07:39.000 2020-01-02 03:36:00.000 459 99360 49909.5 4990950 459 99360 49909.5 4990950 -32711 32427 4689.66 468966 -128 127 -0.26 -26 -46 2 10036 99946 0.13813 300.13813 150.13813 15163.95195 0.13813 300.13815 150.13814 15163.95214 0.13813 300.13813 150.13813 15163.95113 2020-01-01 2020-01-02 2020-01-01 00:00:46 2020-01-02 03:45:46 2020-01-01 00:00:46.000 2020-01-02 03:45:46.000 46 99946 49996 5049596 46 99946 49996 5049596 -32523 32412 4575.009900990099 462076 -125 127 -0.039603960396039604 -4 -460 2 10450 99361 1.38138 298.38138 149.88138 14988.13813 1.38138 298.38138 149.88137 14988.13789 1.38138 298.38138 149.88138 14988.13800 2020-01-01 2020-01-02 2020-01-01 00:07:40 2020-01-02 03:36:01 2020-01-01 00:07:40.000 2020-01-02 03:36:01.000 460 99361 49910.5 4991050 460 99361 49910.5 4991050 -32710 32428 4690.66 469066 -128 127 -1.82 -182 -461 2 10451 99362 1.38438 298.38438 149.88438 14988.43843 1.38438 298.3844 149.88438 14988.43864 1.38438 298.38438 149.88438 14988.43800 2020-01-01 2020-01-02 2020-01-01 00:07:41 2020-01-02 03:36:02 2020-01-01 00:07:41.000 2020-01-02 03:36:02.000 461 99362 49911.5 4991150 461 99362 49911.5 4991150 -32709 32429 4691.66 469166 -128 123 -3.38 -338 -462 2 10452 99363 1.38738 298.38738 149.88738 14988.73873 1.38738 298.3874 149.88738 14988.73894 1.38738 298.38738 149.88738 14988.73800 2020-01-01 2020-01-02 2020-01-01 00:07:42 2020-01-02 03:36:03 2020-01-01 00:07:42.000 2020-01-02 03:36:03.000 462 99363 49912.5 4991250 462 99363 49912.5 4991250 -32708 32430 4692.66 469266 -127 124 -2.38 -238 +456 2 10446 99357 1.36936 298.36936 149.86936 14986.93693 1.36936 298.36935 149.86936 14986.93672 1.36936 298.36936 149.86936000000009 14986.93600 2020-01-01 2020-01-02 2020-01-01 00:07:36 2020-01-02 03:35:57 2020-01-01 00:07:36.000 2020-01-02 03:35:57.000 456 99357 49906.5 4990650 456 99357 49906.5 4990650 -32714 32424 4686.66 468666 -127 125 -0.7 -70 +457 2 10447 99358 1.37237 298.37237 149.87237 14987.23723 1.37237 298.37238 149.87237 14987.23747 1.37237 298.37237 149.87236999999976 14987.23700 2020-01-01 2020-01-02 2020-01-01 00:07:37 2020-01-02 03:35:58 2020-01-01 00:07:37.000 2020-01-02 03:35:58.000 457 99358 49907.5 4990750 457 99358 49907.5 4990750 -32713 32425 4687.66 468766 -126 126 0.3 30 +458 2 10448 99359 1.37537 298.37537 149.87537 14987.53753 1.37537 298.37537 149.87537 14987.5376 1.37537 298.37537 149.87536999999986 14987.53700 2020-01-01 2020-01-02 2020-01-01 00:07:38 2020-01-02 03:35:59 2020-01-01 00:07:38.000 2020-01-02 03:35:59.000 458 99359 49908.5 4990850 458 99359 49908.5 4990850 -32712 32426 4688.66 468866 -125 127 1.3 130 +459 2 10449 99360 1.37837 298.37837 149.87837 14987.83783 1.37837 298.3784 149.87837 14987.83775 1.37837 298.37837 149.87837000000016 14987.83700 2020-01-01 2020-01-02 2020-01-01 00:07:39 2020-01-02 03:36:00 2020-01-01 00:07:39.000 2020-01-02 03:36:00.000 459 99360 49909.5 4990950 459 99360 49909.5 4990950 -32711 32427 4689.66 468966 -128 127 -0.26 -26 +46 2 10036 99946 0.13813 300.13813 150.13813 15163.95195 0.13813 300.13815 150.13814 15163.95214 0.13813 300.13813 150.13812999999993 15163.95113 2020-01-01 2020-01-02 2020-01-01 00:00:46 2020-01-02 03:45:46 2020-01-01 00:00:46.000 2020-01-02 03:45:46.000 46 99946 49996 5049596 46 99946 49996 5049596 -32523 32412 4575.009900990099 462076 -125 127 -0.039603960396039604 -4 +460 2 10450 99361 1.38138 298.38138 149.88138 14988.13813 1.38138 298.38138 149.88137 14988.13789 1.38138 298.38138 149.8813800000001 14988.13800 2020-01-01 2020-01-02 2020-01-01 00:07:40 2020-01-02 03:36:01 2020-01-01 00:07:40.000 2020-01-02 03:36:01.000 460 99361 49910.5 4991050 460 99361 49910.5 4991050 -32710 32428 4690.66 469066 -128 127 -1.82 -182 +461 2 10451 99362 1.38438 298.38438 149.88438 14988.43843 1.38438 298.3844 149.88438 14988.43864 1.38438 298.38438 149.88438000000002 14988.43800 2020-01-01 2020-01-02 2020-01-01 00:07:41 2020-01-02 03:36:02 2020-01-01 00:07:41.000 2020-01-02 03:36:02.000 461 99362 49911.5 4991150 461 99362 49911.5 4991150 -32709 32429 4691.66 469166 -128 123 -3.38 -338 +462 2 10452 99363 1.38738 298.38738 149.88738 14988.73873 1.38738 298.3874 149.88738 14988.73894 1.38738 298.38738 149.8873800000001 14988.73800 2020-01-01 2020-01-02 2020-01-01 00:07:42 2020-01-02 03:36:03 2020-01-01 00:07:42.000 2020-01-02 03:36:03.000 462 99363 49912.5 4991250 462 99363 49912.5 4991250 -32708 32430 4692.66 469266 -127 124 -2.38 -238 463 2 10453 99364 1.39039 298.39039 149.89039 14989.03903 1.39039 298.39038 149.89039 14989.03907 1.39039 298.39039 149.89039 14989.03900 2020-01-01 2020-01-02 2020-01-01 00:07:43 2020-01-02 03:36:04 2020-01-01 00:07:43.000 2020-01-02 03:36:04.000 463 99364 49913.5 4991350 463 99364 49913.5 4991350 -32707 32431 4693.66 469366 -126 125 -1.38 -138 -464 2 10454 99365 1.39339 298.39339 149.89339 14989.33933 1.39339 298.3934 149.89339 14989.33922 1.39339 298.39339 149.89339 14989.33900 2020-01-01 2020-01-02 2020-01-01 00:07:44 2020-01-02 03:36:05 2020-01-01 00:07:44.000 2020-01-02 03:36:05.000 464 99365 49914.5 4991450 464 99365 49914.5 4991450 -32706 32432 4694.66 469466 -125 126 -0.38 -38 -465 2 10455 99366 1.39639 298.39639 149.89639 14989.63963 1.39639 298.3964 149.89639 14989.63936 1.39639 298.39639 149.89639 14989.63900 2020-01-01 2020-01-02 2020-01-01 00:07:45 2020-01-02 03:36:06 2020-01-01 00:07:45.000 2020-01-02 03:36:06.000 465 99366 49915.5 4991550 465 99366 49915.5 4991550 -32705 32433 4695.66 469566 -124 127 0.62 62 +464 2 10454 99365 1.39339 298.39339 149.89339 14989.33933 1.39339 298.3934 149.89339 14989.33922 1.39339 298.39339 149.8933899999999 14989.33900 2020-01-01 2020-01-02 2020-01-01 00:07:44 2020-01-02 03:36:05 2020-01-01 00:07:44.000 2020-01-02 03:36:05.000 464 99365 49914.5 4991450 464 99365 49914.5 4991450 -32706 32432 4694.66 469466 -125 126 -0.38 -38 +465 2 10455 99366 1.39639 298.39639 149.89639 14989.63963 1.39639 298.3964 149.89639 14989.63936 1.39639 298.39639 149.8963899999999 14989.63900 2020-01-01 2020-01-02 2020-01-01 00:07:45 2020-01-02 03:36:06 2020-01-01 00:07:45.000 2020-01-02 03:36:06.000 465 99366 49915.5 4991550 465 99366 49915.5 4991550 -32705 32433 4695.66 469566 -124 127 0.62 62 466 2 10456 99367 1.39939 298.39939 149.89939 14989.93993 1.39939 298.3994 149.8994 14989.94011 1.39939 298.39939 149.89939 14989.93900 2020-01-01 2020-01-02 2020-01-01 00:07:46 2020-01-02 03:36:07 2020-01-01 00:07:46.000 2020-01-02 03:36:07.000 466 99367 49916.5 4991650 466 99367 49916.5 4991650 -32704 32434 4696.66 469666 -128 127 -0.94 -94 -467 2 10457 99368 1.4024 298.4024 149.9024 14990.24024 1.4024 298.4024 149.9024 14990.24041 1.40240 298.40240 149.90240 14990.24000 2020-01-01 2020-01-02 2020-01-01 00:07:47 2020-01-02 03:36:08 2020-01-01 00:07:47.000 2020-01-02 03:36:08.000 467 99368 49917.5 4991750 467 99368 49917.5 4991750 -32703 32435 4697.66 469766 -128 123 -2.5 -250 -468 2 10458 99369 1.4054 298.4054 149.9054 14990.54054 1.4054 298.4054 149.9054 14990.54058 1.40540 298.40540 149.90540 14990.54000 2020-01-01 2020-01-02 2020-01-01 00:07:48 2020-01-02 03:36:09 2020-01-01 00:07:48.000 2020-01-02 03:36:09.000 468 99369 49918.5 4991850 468 99369 49918.5 4991850 -32702 32436 4698.66 469866 -127 124 -1.5 -150 -469 2 10459 99370 1.4084 298.4084 149.9084 14990.84084 1.4084 298.40842 149.9084 14990.8407 1.40840 298.40840 149.90840 14990.84000 2020-01-01 2020-01-02 2020-01-01 00:07:49 2020-01-02 03:36:10 2020-01-01 00:07:49.000 2020-01-02 03:36:10.000 469 99370 49919.5 4991950 469 99370 49919.5 4991950 -32701 32437 4699.66 469966 -126 125 -0.5 -50 -47 2 10037 99947 0.14114 300.14114 150.14114 15164.25525 0.14114 300.14114 150.14114 15164.25545 0.14114 300.14114 150.14114 15164.25514 2020-01-01 2020-01-02 2020-01-01 00:00:47 2020-01-02 03:45:47 2020-01-01 00:00:47.000 2020-01-02 03:45:47.000 47 99947 49997 5049697 47 99947 49997 5049697 -32522 32413 4576.009900990099 462177 -128 127 -1.5742574257425743 -159 -470 2 10460 99371 1.41141 298.41141 149.91141 14991.14114 1.41141 298.4114 149.9114 14991.14099 1.41141 298.41141 149.91141 14991.14100 2020-01-01 2020-01-02 2020-01-01 00:07:50 2020-01-02 03:36:11 2020-01-01 00:07:50.000 2020-01-02 03:36:11.000 470 99371 49920.5 4992050 470 99371 49920.5 4992050 -32700 32438 4700.66 470066 -125 126 0.5 50 -471 2 10461 99372 1.41441 298.41441 149.91441 14991.44144 1.41441 298.41443 149.91441 14991.44159 1.41441 298.41441 149.91441 14991.44100 2020-01-01 2020-01-02 2020-01-01 00:07:51 2020-01-02 03:36:12 2020-01-01 00:07:51.000 2020-01-02 03:36:12.000 471 99372 49921.5 4992150 471 99372 49921.5 4992150 -32699 32439 4701.66 470166 -124 127 1.5 150 +467 2 10457 99368 1.4024 298.4024 149.9024 14990.24024 1.4024 298.4024 149.9024 14990.24041 1.40240 298.40240 149.90240000000017 14990.24000 2020-01-01 2020-01-02 2020-01-01 00:07:47 2020-01-02 03:36:08 2020-01-01 00:07:47.000 2020-01-02 03:36:08.000 467 99368 49917.5 4991750 467 99368 49917.5 4991750 -32703 32435 4697.66 469766 -128 123 -2.5 -250 +468 2 10458 99369 1.4054 298.4054 149.9054 14990.54054 1.4054 298.4054 149.9054 14990.54058 1.40540 298.40540 149.90539999999984 14990.54000 2020-01-01 2020-01-02 2020-01-01 00:07:48 2020-01-02 03:36:09 2020-01-01 00:07:48.000 2020-01-02 03:36:09.000 468 99369 49918.5 4991850 468 99369 49918.5 4991850 -32702 32436 4698.66 469866 -127 124 -1.5 -150 +469 2 10459 99370 1.4084 298.4084 149.9084 14990.84084 1.4084 298.40842 149.9084 14990.8407 1.40840 298.40840 149.90840000000009 14990.84000 2020-01-01 2020-01-02 2020-01-01 00:07:49 2020-01-02 03:36:10 2020-01-01 00:07:49.000 2020-01-02 03:36:10.000 469 99370 49919.5 4991950 469 99370 49919.5 4991950 -32701 32437 4699.66 469966 -126 125 -0.5 -50 +47 2 10037 99947 0.14114 300.14114 150.14114 15164.25525 0.14114 300.14114 150.14114 15164.25545 0.14114 300.14114 150.1411399999999 15164.25514 2020-01-01 2020-01-02 2020-01-01 00:00:47 2020-01-02 03:45:47 2020-01-01 00:00:47.000 2020-01-02 03:45:47.000 47 99947 49997 5049697 47 99947 49997 5049697 -32522 32413 4576.009900990099 462177 -128 127 -1.5742574257425743 -159 +470 2 10460 99371 1.41141 298.41141 149.91141 14991.14114 1.41141 298.4114 149.9114 14991.14099 1.41141 298.41141 149.91141000000007 14991.14100 2020-01-01 2020-01-02 2020-01-01 00:07:50 2020-01-02 03:36:11 2020-01-01 00:07:50.000 2020-01-02 03:36:11.000 470 99371 49920.5 4992050 470 99371 49920.5 4992050 -32700 32438 4700.66 470066 -125 126 0.5 50 +471 2 10461 99372 1.41441 298.41441 149.91441 14991.44144 1.41441 298.41443 149.91441 14991.44159 1.41441 298.41441 149.91440999999992 14991.44100 2020-01-01 2020-01-02 2020-01-01 00:07:51 2020-01-02 03:36:12 2020-01-01 00:07:51.000 2020-01-02 03:36:12.000 471 99372 49921.5 4992150 471 99372 49921.5 4992150 -32699 32439 4701.66 470166 -124 127 1.5 150 472 2 10462 99373 1.41741 298.41741 149.91741 14991.74174 1.41741 298.41742 149.91741 14991.74188 1.41741 298.41741 149.91741 14991.74100 2020-01-01 2020-01-02 2020-01-01 00:07:52 2020-01-02 03:36:13 2020-01-01 00:07:52.000 2020-01-02 03:36:13.000 472 99373 49922.5 4992250 472 99373 49922.5 4992250 -32698 32440 4702.66 470266 -128 127 -0.06 -6 -473 2 10463 99374 1.42042 298.42042 149.92042 14992.04204 1.42042 298.4204 149.92042 14992.04204 1.42042 298.42042 149.92042 14992.04200 2020-01-01 2020-01-02 2020-01-01 00:07:53 2020-01-02 03:36:14 2020-01-01 00:07:53.000 2020-01-02 03:36:14.000 473 99374 49923.5 4992350 473 99374 49923.5 4992350 -32697 32441 4703.66 470366 -128 123 -1.62 -162 -474 2 10464 99375 1.42342 298.42342 149.92342 14992.34234 1.42342 298.42343 149.92342 14992.34216 1.42342 298.42342 149.92342 14992.34200 2020-01-01 2020-01-02 2020-01-01 00:07:54 2020-01-02 03:36:15 2020-01-01 00:07:54.000 2020-01-02 03:36:15.000 474 99375 49924.5 4992450 474 99375 49924.5 4992450 -32696 32442 4704.66 470466 -127 124 -0.62 -62 -475 2 10465 99376 1.42642 298.42642 149.92642 14992.64264 1.42642 298.42642 149.92642 14992.64246 1.42642 298.42642 149.92642 14992.64200 2020-01-01 2020-01-02 2020-01-01 00:07:55 2020-01-02 03:36:16 2020-01-01 00:07:55.000 2020-01-02 03:36:16.000 475 99376 49925.5 4992550 475 99376 49925.5 4992550 -32695 32443 4705.66 470566 -126 125 0.38 38 -476 2 10466 99377 1.42942 298.42942 149.92942 14992.94294 1.42942 298.42944 149.92943 14992.94305 1.42942 298.42942 149.92942 14992.94200 2020-01-01 2020-01-02 2020-01-01 00:07:56 2020-01-02 03:36:17 2020-01-01 00:07:56.000 2020-01-02 03:36:17.000 476 99377 49926.5 4992650 476 99377 49926.5 4992650 -32694 32444 4706.66 470666 -125 126 1.38 138 -477 2 10467 99378 1.43243 298.43243 149.93243 14993.24324 1.43243 298.43243 149.93243 14993.24338 1.43243 298.43243 149.93243 14993.24300 2020-01-01 2020-01-02 2020-01-01 00:07:57 2020-01-02 03:36:18 2020-01-01 00:07:57.000 2020-01-02 03:36:18.000 477 99378 49927.5 4992750 477 99378 49927.5 4992750 -32693 32445 4707.66 470766 -124 127 2.38 238 -478 2 10468 99379 1.43543 298.43543 149.93543 14993.54354 1.43543 298.43542 149.93543 14993.54352 1.43543 298.43543 149.93543 14993.54300 2020-01-01 2020-01-02 2020-01-01 00:07:58 2020-01-02 03:36:19 2020-01-01 00:07:58.000 2020-01-02 03:36:19.000 478 99379 49928.5 4992850 478 99379 49928.5 4992850 -32692 32446 4708.66 470866 -128 127 0.82 82 -479 2 10469 99380 1.43843 298.43843 149.93843 14993.84384 1.43843 298.43845 149.93844 14993.84427 1.43843 298.43843 149.93843 14993.84300 2020-01-01 2020-01-02 2020-01-01 00:07:59 2020-01-02 03:36:20 2020-01-01 00:07:59.000 2020-01-02 03:36:20.000 479 99380 49929.5 4992950 479 99380 49929.5 4992950 -32691 32447 4709.66 470966 -128 127 -0.74 -74 -48 2 10038 99948 0.14414 300.14414 150.14414 15164.55855 0.14414 300.14413 150.14414 15164.55863 0.14414 300.14414 150.14414 15164.55814 2020-01-01 2020-01-02 2020-01-01 00:00:48 2020-01-02 03:45:48 2020-01-01 00:00:48.000 2020-01-02 03:45:48.000 48 99948 49998 5049798 48 99948 49998 5049798 -32521 32414 4577.009900990099 462278 -128 127 -3.108910891089109 -314 +473 2 10463 99374 1.42042 298.42042 149.92042 14992.04204 1.42042 298.4204 149.92042 14992.04204 1.42042 298.42042 149.92042000000018 14992.04200 2020-01-01 2020-01-02 2020-01-01 00:07:53 2020-01-02 03:36:14 2020-01-01 00:07:53.000 2020-01-02 03:36:14.000 473 99374 49923.5 4992350 473 99374 49923.5 4992350 -32697 32441 4703.66 470366 -128 123 -1.62 -162 +474 2 10464 99375 1.42342 298.42342 149.92342 14992.34234 1.42342 298.42343 149.92342 14992.34216 1.42342 298.42342 149.92341999999982 14992.34200 2020-01-01 2020-01-02 2020-01-01 00:07:54 2020-01-02 03:36:15 2020-01-01 00:07:54.000 2020-01-02 03:36:15.000 474 99375 49924.5 4992450 474 99375 49924.5 4992450 -32696 32442 4704.66 470466 -127 124 -0.62 -62 +475 2 10465 99376 1.42642 298.42642 149.92642 14992.64264 1.42642 298.42642 149.92642 14992.64246 1.42642 298.42642 149.92641999999987 14992.64200 2020-01-01 2020-01-02 2020-01-01 00:07:55 2020-01-02 03:36:16 2020-01-01 00:07:55.000 2020-01-02 03:36:16.000 475 99376 49925.5 4992550 475 99376 49925.5 4992550 -32695 32443 4705.66 470566 -126 125 0.38 38 +476 2 10466 99377 1.42942 298.42942 149.92942 14992.94294 1.42942 298.42944 149.92943 14992.94305 1.42942 298.42942 149.9294200000002 14992.94200 2020-01-01 2020-01-02 2020-01-01 00:07:56 2020-01-02 03:36:17 2020-01-01 00:07:56.000 2020-01-02 03:36:17.000 476 99377 49926.5 4992650 476 99377 49926.5 4992650 -32694 32444 4706.66 470666 -125 126 1.38 138 +477 2 10467 99378 1.43243 298.43243 149.93243 14993.24324 1.43243 298.43243 149.93243 14993.24338 1.43243 298.43243 149.9324300000001 14993.24300 2020-01-01 2020-01-02 2020-01-01 00:07:57 2020-01-02 03:36:18 2020-01-01 00:07:57.000 2020-01-02 03:36:18.000 477 99378 49927.5 4992750 477 99378 49927.5 4992750 -32693 32445 4707.66 470766 -124 127 2.38 238 +478 2 10468 99379 1.43543 298.43543 149.93543 14993.54354 1.43543 298.43542 149.93543 14993.54352 1.43543 298.43543 149.9354299999998 14993.54300 2020-01-01 2020-01-02 2020-01-01 00:07:58 2020-01-02 03:36:19 2020-01-01 00:07:58.000 2020-01-02 03:36:19.000 478 99379 49928.5 4992850 478 99379 49928.5 4992850 -32692 32446 4708.66 470866 -128 127 0.82 82 +479 2 10469 99380 1.43843 298.43843 149.93843 14993.84384 1.43843 298.43845 149.93844 14993.84427 1.43843 298.43843 149.9384300000001 14993.84300 2020-01-01 2020-01-02 2020-01-01 00:07:59 2020-01-02 03:36:20 2020-01-01 00:07:59.000 2020-01-02 03:36:20.000 479 99380 49929.5 4992950 479 99380 49929.5 4992950 -32691 32447 4709.66 470966 -128 127 -0.74 -74 +48 2 10038 99948 0.14414 300.14414 150.14414 15164.55855 0.14414 300.14413 150.14414 15164.55863 0.14414 300.14414 150.14414000000016 15164.55814 2020-01-01 2020-01-02 2020-01-01 00:00:48 2020-01-02 03:45:48 2020-01-01 00:00:48.000 2020-01-02 03:45:48.000 48 99948 49998 5049798 48 99948 49998 5049798 -32521 32414 4577.009900990099 462278 -128 127 -3.108910891089109 -314 480 2 10470 99381 1.44144 298.44144 149.94144 14994.14414 1.44144 298.44144 149.94143 14994.14392 1.44144 298.44144 149.94144 14994.14400 2020-01-01 2020-01-02 2020-01-01 00:08:00 2020-01-02 03:36:21 2020-01-01 00:08:00.000 2020-01-02 03:36:21.000 480 99381 49930.5 4993050 480 99381 49930.5 4993050 -32690 32448 4710.66 471066 -128 124 -2.3 -230 -481 2 10471 99382 1.44444 298.44444 149.94444 14994.44444 1.44444 298.44446 149.94444 14994.44452 1.44444 298.44444 149.94444 14994.44400 2020-01-01 2020-01-02 2020-01-01 00:08:01 2020-01-02 03:36:22 2020-01-01 00:08:01.000 2020-01-02 03:36:22.000 481 99382 49931.5 4993150 481 99382 49931.5 4993150 -32689 32449 4711.66 471166 -127 125 -1.3 -130 -482 2 10472 99383 1.44744 298.44744 149.94744 14994.74474 1.44744 298.44745 149.94744 14994.74485 1.44744 298.44744 149.94744 14994.74400 2020-01-01 2020-01-02 2020-01-01 00:08:02 2020-01-02 03:36:23 2020-01-01 00:08:02.000 2020-01-02 03:36:23.000 482 99383 49932.5 4993250 482 99383 49932.5 4993250 -32688 32450 4712.66 471266 -126 126 -0.3 -30 -483 2 10473 99384 1.45045 298.45045 149.95045 14995.04504 1.45045 298.45044 149.95044 14995.04499 1.45045 298.45045 149.95045 14995.04500 2020-01-01 2020-01-02 2020-01-01 00:08:03 2020-01-02 03:36:24 2020-01-01 00:08:03.000 2020-01-02 03:36:24.000 483 99384 49933.5 4993350 483 99384 49933.5 4993350 -32687 32451 4713.66 471366 -125 127 0.7 70 -484 2 10474 99385 1.45345 298.45345 149.95345 14995.34534 1.45345 298.45346 149.95345 14995.34574 1.45345 298.45345 149.95345 14995.34500 2020-01-01 2020-01-02 2020-01-01 00:08:04 2020-01-02 03:36:25 2020-01-01 00:08:04.000 2020-01-02 03:36:25.000 484 99385 49934.5 4993450 484 99385 49934.5 4993450 -32686 32452 4714.66 471466 -128 127 -0.86 -86 -485 2 10475 99386 1.45645 298.45645 149.95645 14995.64564 1.45645 298.45645 149.95645 14995.6454 1.45645 298.45645 149.95645 14995.64500 2020-01-01 2020-01-02 2020-01-01 00:08:05 2020-01-02 03:36:26 2020-01-01 00:08:05.000 2020-01-02 03:36:26.000 485 99386 49935.5 4993550 485 99386 49935.5 4993550 -32685 32453 4715.66 471566 -128 127 -2.42 -242 -486 2 10476 99387 1.45945 298.45945 149.95945 14995.94594 1.45945 298.45947 149.95946 14995.94602 1.45945 298.45945 149.95945 14995.94500 2020-01-01 2020-01-02 2020-01-01 00:08:06 2020-01-02 03:36:27 2020-01-01 00:08:06.000 2020-01-02 03:36:27.000 486 99387 49936.5 4993650 486 99387 49936.5 4993650 -32684 32454 4716.66 471666 -128 123 -3.98 -398 -487 2 10477 99388 1.46246 298.46246 149.96246 14996.24624 1.46246 298.46246 149.96246 14996.24633 1.46246 298.46246 149.96246 14996.24600 2020-01-01 2020-01-02 2020-01-01 00:08:07 2020-01-02 03:36:28 2020-01-01 00:08:07.000 2020-01-02 03:36:28.000 487 99388 49937.5 4993750 487 99388 49937.5 4993750 -32683 32455 4717.66 471766 -127 124 -2.98 -298 -488 2 10478 99389 1.46546 298.46546 149.96546 14996.54654 1.46546 298.46545 149.96546 14996.54645 1.46546 298.46546 149.96546 14996.54600 2020-01-01 2020-01-02 2020-01-01 00:08:08 2020-01-02 03:36:29 2020-01-01 00:08:08.000 2020-01-02 03:36:29.000 488 99389 49938.5 4993850 488 99389 49938.5 4993850 -32682 32456 4718.66 471866 -126 125 -1.98 -198 -489 2 10479 99390 1.46846 298.46846 149.96846 14996.84684 1.46846 298.46848 149.96847 14996.84721 1.46846 298.46846 149.96846 14996.84600 2020-01-01 2020-01-02 2020-01-01 00:08:09 2020-01-02 03:36:30 2020-01-01 00:08:09.000 2020-01-02 03:36:30.000 489 99390 49939.5 4993950 489 99390 49939.5 4993950 -32681 32457 4719.66 471966 -125 126 -0.98 -98 -49 2 10039 99949 0.14714 300.14714 150.14714 15164.86186 0.14714 300.14716 150.14714 15164.86173 0.14714 300.14714 150.14714 15164.86114 2020-01-01 2020-01-02 2020-01-01 00:00:49 2020-01-02 03:45:49 2020-01-01 00:00:49.000 2020-01-02 03:45:49.000 49 99949 49999 5049899 49 99949 49999 5049899 -32520 32415 4578.009900990099 462379 -128 123 -4.643564356435643 -469 -490 2 10480 99391 1.47147 298.47147 149.97147 14997.14714 1.47147 298.47147 149.97146 14997.14687 1.47147 298.47147 149.97147 14997.14700 2020-01-01 2020-01-02 2020-01-01 00:08:10 2020-01-02 03:36:31 2020-01-01 00:08:10.000 2020-01-02 03:36:31.000 490 99391 49940.5 4994050 490 99391 49940.5 4994050 -32680 32458 4720.66 472066 -124 127 0.02 2 -491 2 10481 99392 1.47447 298.47447 149.97447 14997.44744 1.47447 298.4745 149.97447 14997.44749 1.47447 298.47447 149.97447 14997.44700 2020-01-01 2020-01-02 2020-01-01 00:08:11 2020-01-02 03:36:32 2020-01-01 00:08:11.000 2020-01-02 03:36:32.000 491 99392 49941.5 4994150 491 99392 49941.5 4994150 -32679 32459 4721.66 472166 -128 127 -1.54 -154 -492 2 10482 99393 1.47747 298.47747 149.97747 14997.74774 1.47747 298.47748 149.97747 14997.74779 1.47747 298.47747 149.97747 14997.74700 2020-01-01 2020-01-02 2020-01-01 00:08:12 2020-01-02 03:36:33 2020-01-01 00:08:12.000 2020-01-02 03:36:33.000 492 99393 49942.5 4994250 492 99393 49942.5 4994250 -32678 32460 4722.66 472266 -128 123 -3.1 -310 -493 2 10483 99394 1.48048 298.48048 149.98048 14998.04804 1.48048 298.48047 149.98048 14998.04809 1.48048 298.48048 149.98048 14998.04800 2020-01-01 2020-01-02 2020-01-01 00:08:13 2020-01-02 03:36:34 2020-01-01 00:08:13.000 2020-01-02 03:36:34.000 493 99394 49943.5 4994350 493 99394 49943.5 4994350 -32677 32461 4723.66 472366 -127 124 -2.1 -210 -494 2 10484 99395 1.48348 298.48348 149.98348 14998.34834 1.48348 298.4835 149.98348 14998.34868 1.48348 298.48348 149.98348 14998.34800 2020-01-01 2020-01-02 2020-01-01 00:08:14 2020-01-02 03:36:35 2020-01-01 00:08:14.000 2020-01-02 03:36:35.000 494 99395 49944.5 4994450 494 99395 49944.5 4994450 -32676 32462 4724.66 472466 -126 125 -1.1 -110 -495 2 10485 99396 1.48648 298.48648 149.98648 14998.64864 1.48648 298.48648 149.98648 14998.64837 1.48648 298.48648 149.98648 14998.64800 2020-01-01 2020-01-02 2020-01-01 00:08:15 2020-01-02 03:36:36 2020-01-01 00:08:15.000 2020-01-02 03:36:36.000 495 99396 49945.5 4994550 495 99396 49945.5 4994550 -32675 32463 4725.66 472566 -125 126 -0.1 -10 -496 2 10486 99397 1.48948 298.48948 149.98948 14998.94894 1.48948 298.4895 149.98948 14998.94896 1.48948 298.48948 149.98948 14998.94800 2020-01-01 2020-01-02 2020-01-01 00:08:16 2020-01-02 03:36:37 2020-01-01 00:08:16.000 2020-01-02 03:36:37.000 496 99397 49946.5 4994650 496 99397 49946.5 4994650 -32674 32464 4726.66 472666 -124 127 0.9 90 -497 2 10487 99398 1.49249 298.49249 149.99249 14999.24924 1.49249 298.4925 149.99249 14999.24926 1.49249 298.49249 149.99249 14999.24900 2020-01-01 2020-01-02 2020-01-01 00:08:17 2020-01-02 03:36:38 2020-01-01 00:08:17.000 2020-01-02 03:36:38.000 497 99398 49947.5 4994750 497 99398 49947.5 4994750 -32673 32465 4727.66 472766 -128 127 -0.66 -66 -498 2 10488 99399 1.49549 298.49549 149.99549 14999.54954 1.49549 298.49548 149.99549 14999.54956 1.49549 298.49549 149.99549 14999.54900 2020-01-01 2020-01-02 2020-01-01 00:08:18 2020-01-02 03:36:39 2020-01-01 00:08:18.000 2020-01-02 03:36:39.000 498 99399 49948.5 4994850 498 99399 49948.5 4994850 -32672 32466 4728.66 472866 -128 123 -2.22 -222 -499 2 10489 99400 1.49849 298.49849 149.99849 14999.84984 1.49849 298.4985 149.9985 14999.85015 1.49849 298.49849 149.99849 14999.84900 2020-01-01 2020-01-02 2020-01-01 00:08:19 2020-01-02 03:36:40 2020-01-01 00:08:19.000 2020-01-02 03:36:40.000 499 99400 49949.5 4994950 499 99400 49949.5 4994950 -32671 32467 4729.66 472966 -127 124 -1.22 -122 -5 2 1004 9995 0.01501 300.01501 150.01501 15151.51651 0.01501 300.015 150.01501 15151.51648 0.01501 300.01501 150.01501 15151.51601 2020-01-01 2020-01-02 2020-01-01 00:00:05 2020-01-02 03:45:05 2020-01-01 00:00:05.000 2020-01-02 03:45:05.000 5 99905 49955 5045455 5 99905 49955 5045455 -32564 32371 4534.009900990099 457935 -128 123 -3.01980198019802 -305 +481 2 10471 99382 1.44444 298.44444 149.94444 14994.44444 1.44444 298.44446 149.94444 14994.44452 1.44444 298.44444 149.94443999999993 14994.44400 2020-01-01 2020-01-02 2020-01-01 00:08:01 2020-01-02 03:36:22 2020-01-01 00:08:01.000 2020-01-02 03:36:22.000 481 99382 49931.5 4993150 481 99382 49931.5 4993150 -32689 32449 4711.66 471166 -127 125 -1.3 -130 +482 2 10472 99383 1.44744 298.44744 149.94744 14994.74474 1.44744 298.44745 149.94744 14994.74485 1.44744 298.44744 149.94743999999994 14994.74400 2020-01-01 2020-01-02 2020-01-01 00:08:02 2020-01-02 03:36:23 2020-01-01 00:08:02.000 2020-01-02 03:36:23.000 482 99383 49932.5 4993250 482 99383 49932.5 4993250 -32688 32450 4712.66 471266 -126 126 -0.3 -30 +483 2 10473 99384 1.45045 298.45045 149.95045 14995.04504 1.45045 298.45044 149.95044 14995.04499 1.45045 298.45045 149.95045000000013 14995.04500 2020-01-01 2020-01-02 2020-01-01 00:08:03 2020-01-02 03:36:24 2020-01-01 00:08:03.000 2020-01-02 03:36:24.000 483 99384 49933.5 4993350 483 99384 49933.5 4993350 -32687 32451 4713.66 471366 -125 127 0.7 70 +484 2 10474 99385 1.45345 298.45345 149.95345 14995.34534 1.45345 298.45346 149.95345 14995.34574 1.45345 298.45345 149.9534500000002 14995.34500 2020-01-01 2020-01-02 2020-01-01 00:08:04 2020-01-02 03:36:25 2020-01-01 00:08:04.000 2020-01-02 03:36:25.000 484 99385 49934.5 4993450 484 99385 49934.5 4993450 -32686 32452 4714.66 471466 -128 127 -0.86 -86 +485 2 10475 99386 1.45645 298.45645 149.95645 14995.64564 1.45645 298.45645 149.95645 14995.6454 1.45645 298.45645 149.95644999999985 14995.64500 2020-01-01 2020-01-02 2020-01-01 00:08:05 2020-01-02 03:36:26 2020-01-01 00:08:05.000 2020-01-02 03:36:26.000 485 99386 49935.5 4993550 485 99386 49935.5 4993550 -32685 32453 4715.66 471566 -128 127 -2.42 -242 +486 2 10476 99387 1.45945 298.45945 149.95945 14995.94594 1.45945 298.45947 149.95946 14995.94602 1.45945 298.45945 149.95945000000012 14995.94500 2020-01-01 2020-01-02 2020-01-01 00:08:06 2020-01-02 03:36:27 2020-01-01 00:08:06.000 2020-01-02 03:36:27.000 486 99387 49936.5 4993650 486 99387 49936.5 4993650 -32684 32454 4716.66 471666 -128 123 -3.98 -398 +487 2 10477 99388 1.46246 298.46246 149.96246 14996.24624 1.46246 298.46246 149.96246 14996.24633 1.46246 298.46246 149.96246000000014 14996.24600 2020-01-01 2020-01-02 2020-01-01 00:08:07 2020-01-02 03:36:28 2020-01-01 00:08:07.000 2020-01-02 03:36:28.000 487 99388 49937.5 4993750 487 99388 49937.5 4993750 -32683 32455 4717.66 471766 -127 124 -2.98 -298 +488 2 10478 99389 1.46546 298.46546 149.96546 14996.54654 1.46546 298.46545 149.96546 14996.54645 1.46546 298.46546 149.96545999999998 14996.54600 2020-01-01 2020-01-02 2020-01-01 00:08:08 2020-01-02 03:36:29 2020-01-01 00:08:08.000 2020-01-02 03:36:29.000 488 99389 49938.5 4993850 488 99389 49938.5 4993850 -32682 32456 4718.66 471866 -126 125 -1.98 -198 +489 2 10479 99390 1.46846 298.46846 149.96846 14996.84684 1.46846 298.46848 149.96847 14996.84721 1.46846 298.46846 149.96846000000002 14996.84600 2020-01-01 2020-01-02 2020-01-01 00:08:09 2020-01-02 03:36:30 2020-01-01 00:08:09.000 2020-01-02 03:36:30.000 489 99390 49939.5 4993950 489 99390 49939.5 4993950 -32681 32457 4719.66 471966 -125 126 -0.98 -98 +49 2 10039 99949 0.14714 300.14714 150.14714 15164.86186 0.14714 300.14716 150.14714 15164.86173 0.14714 300.14714 150.14713999999984 15164.86114 2020-01-01 2020-01-02 2020-01-01 00:00:49 2020-01-02 03:45:49 2020-01-01 00:00:49.000 2020-01-02 03:45:49.000 49 99949 49999 5049899 49 99949 49999 5049899 -32520 32415 4578.009900990099 462379 -128 123 -4.643564356435643 -469 +490 2 10480 99391 1.47147 298.47147 149.97147 14997.14714 1.47147 298.47147 149.97146 14997.14687 1.47147 298.47147 149.9714700000002 14997.14700 2020-01-01 2020-01-02 2020-01-01 00:08:10 2020-01-02 03:36:31 2020-01-01 00:08:10.000 2020-01-02 03:36:31.000 490 99391 49940.5 4994050 490 99391 49940.5 4994050 -32680 32458 4720.66 472066 -124 127 0.02 2 +491 2 10481 99392 1.47447 298.47447 149.97447 14997.44744 1.47447 298.4745 149.97447 14997.44749 1.47447 298.47447 149.97446999999985 14997.44700 2020-01-01 2020-01-02 2020-01-01 00:08:11 2020-01-02 03:36:32 2020-01-01 00:08:11.000 2020-01-02 03:36:32.000 491 99392 49941.5 4994150 491 99392 49941.5 4994150 -32679 32459 4721.66 472166 -128 127 -1.54 -154 +492 2 10482 99393 1.47747 298.47747 149.97747 14997.74774 1.47747 298.47748 149.97747 14997.74779 1.47747 298.47747 149.97746999999987 14997.74700 2020-01-01 2020-01-02 2020-01-01 00:08:12 2020-01-02 03:36:33 2020-01-01 00:08:12.000 2020-01-02 03:36:33.000 492 99393 49942.5 4994250 492 99393 49942.5 4994250 -32678 32460 4722.66 472266 -128 123 -3.1 -310 +493 2 10483 99394 1.48048 298.48048 149.98048 14998.04804 1.48048 298.48047 149.98048 14998.04809 1.48048 298.48048 149.9804800000001 14998.04800 2020-01-01 2020-01-02 2020-01-01 00:08:13 2020-01-02 03:36:34 2020-01-01 00:08:13.000 2020-01-02 03:36:34.000 493 99394 49943.5 4994350 493 99394 49943.5 4994350 -32677 32461 4723.66 472366 -127 124 -2.1 -210 +494 2 10484 99395 1.48348 298.48348 149.98348 14998.34834 1.48348 298.4835 149.98348 14998.34868 1.48348 298.48348 149.9834800000001 14998.34800 2020-01-01 2020-01-02 2020-01-01 00:08:14 2020-01-02 03:36:35 2020-01-01 00:08:14.000 2020-01-02 03:36:35.000 494 99395 49944.5 4994450 494 99395 49944.5 4994450 -32676 32462 4724.66 472466 -126 125 -1.1 -110 +495 2 10485 99396 1.48648 298.48648 149.98648 14998.64864 1.48648 298.48648 149.98648 14998.64837 1.48648 298.48648 149.98647999999977 14998.64800 2020-01-01 2020-01-02 2020-01-01 00:08:15 2020-01-02 03:36:36 2020-01-01 00:08:15.000 2020-01-02 03:36:36.000 495 99396 49945.5 4994550 495 99396 49945.5 4994550 -32675 32463 4725.66 472566 -125 126 -0.1 -10 +496 2 10486 99397 1.48948 298.48948 149.98948 14998.94894 1.48948 298.4895 149.98948 14998.94896 1.48948 298.48948 149.98948000000013 14998.94800 2020-01-01 2020-01-02 2020-01-01 00:08:16 2020-01-02 03:36:37 2020-01-01 00:08:16.000 2020-01-02 03:36:37.000 496 99397 49946.5 4994650 496 99397 49946.5 4994650 -32674 32464 4726.66 472666 -124 127 0.9 90 +497 2 10487 99398 1.49249 298.49249 149.99249 14999.24924 1.49249 298.4925 149.99249 14999.24926 1.49249 298.49249 149.99249000000003 14999.24900 2020-01-01 2020-01-02 2020-01-01 00:08:17 2020-01-02 03:36:38 2020-01-01 00:08:17.000 2020-01-02 03:36:38.000 497 99398 49947.5 4994750 497 99398 49947.5 4994750 -32673 32465 4727.66 472766 -128 127 -0.66 -66 +498 2 10488 99399 1.49549 298.49549 149.99549 14999.54954 1.49549 298.49548 149.99549 14999.54956 1.49549 298.49549 149.9954899999999 14999.54900 2020-01-01 2020-01-02 2020-01-01 00:08:18 2020-01-02 03:36:39 2020-01-01 00:08:18.000 2020-01-02 03:36:39.000 498 99399 49948.5 4994850 498 99399 49948.5 4994850 -32672 32466 4728.66 472866 -128 123 -2.22 -222 +499 2 10489 99400 1.49849 298.49849 149.99849 14999.84984 1.49849 298.4985 149.9985 14999.85015 1.49849 298.49849 149.99848999999998 14999.84900 2020-01-01 2020-01-02 2020-01-01 00:08:19 2020-01-02 03:36:40 2020-01-01 00:08:19.000 2020-01-02 03:36:40.000 499 99400 49949.5 4994950 499 99400 49949.5 4994950 -32671 32467 4729.66 472966 -127 124 -1.22 -122 +5 2 1004 9995 0.01501 300.01501 150.01501 15151.51651 0.01501 300.015 150.01501 15151.51648 0.01501 300.01501 150.01500999999988 15151.51601 2020-01-01 2020-01-02 2020-01-01 00:00:05 2020-01-02 03:45:05 2020-01-01 00:00:05.000 2020-01-02 03:45:05.000 5 99905 49955 5045455 5 99905 49955 5045455 -32564 32371 4534.009900990099 457935 -128 123 -3.01980198019802 -305 50 2 10040 99950 0.15015 300.15015 150.15015 15165.16516 0.15015 300.15015 150.15014 15165.16487 0.15015 300.15015 150.15015 15165.16515 2020-01-01 2020-01-02 2020-01-01 00:00:50 2020-01-02 03:45:50 2020-01-01 00:00:50.000 2020-01-02 03:45:50.000 50 99950 50000 5050000 50 99950 50000 5050000 -32519 32416 4579.009900990099 462480 -127 124 -3.6435643564356437 -368 -500 2 10490 99401 1.5015 298.5015 150.0015 15000.15015 1.5015 298.5015 150.00149 15000.14984 1.50150 298.50150 150.00150 15000.15000 2020-01-01 2020-01-02 2020-01-01 00:08:20 2020-01-02 03:36:41 2020-01-01 00:08:20.000 2020-01-02 03:36:41.000 500 99401 49950.5 4995050 500 99401 49950.5 4995050 -32670 32468 4730.66 473066 -126 125 -0.22 -22 -501 2 10491 99402 1.5045 298.5045 150.0045 15000.45045 1.5045 298.50452 150.0045 15000.45043 1.50450 298.50450 150.00450 15000.45000 2020-01-01 2020-01-02 2020-01-01 00:08:21 2020-01-02 03:36:42 2020-01-01 00:08:21.000 2020-01-02 03:36:42.000 501 99402 49951.5 4995150 501 99402 49951.5 4995150 -32669 32469 4731.66 473166 -125 126 0.78 78 -502 2 10492 99403 1.5075 298.5075 150.0075 15000.75075 1.5075 298.5075 150.0075 15000.75073 1.50750 298.50750 150.00750 15000.75000 2020-01-01 2020-01-02 2020-01-01 00:08:22 2020-01-02 03:36:43 2020-01-01 00:08:22.000 2020-01-02 03:36:43.000 502 99403 49952.5 4995250 502 99403 49952.5 4995250 -32668 32470 4732.66 473266 -124 127 1.78 178 -503 2 10493 99404 1.51051 298.51051 150.01051 15001.05105 1.51051 298.5105 150.01051 15001.05103 1.51051 298.51051 150.01051 15001.05100 2020-01-01 2020-01-02 2020-01-01 00:08:23 2020-01-02 03:36:44 2020-01-01 00:08:23.000 2020-01-02 03:36:44.000 503 99404 49953.5 4995350 503 99404 49953.5 4995350 -32667 32471 4733.66 473366 -128 127 0.22 22 -504 2 10494 99405 1.51351 298.51351 150.01351 15001.35135 1.51351 298.51352 150.01351 15001.35162 1.51351 298.51351 150.01351 15001.35100 2020-01-01 2020-01-02 2020-01-01 00:08:24 2020-01-02 03:36:45 2020-01-01 00:08:24.000 2020-01-02 03:36:45.000 504 99405 49954.5 4995450 504 99405 49954.5 4995450 -32666 32472 4734.66 473466 -128 127 -1.34 -134 -505 2 10495 99406 1.51651 298.51651 150.01651 15001.65165 1.51651 298.5165 150.01651 15001.65131 1.51651 298.51651 150.01651 15001.65100 2020-01-01 2020-01-02 2020-01-01 00:08:25 2020-01-02 03:36:46 2020-01-01 00:08:25.000 2020-01-02 03:36:46.000 505 99406 49955.5 4995550 505 99406 49955.5 4995550 -32665 32473 4735.66 473566 -128 124 -2.9 -290 -506 2 10496 99407 1.51951 298.51951 150.01951 15001.95195 1.51951 298.51953 150.01951 15001.9519 1.51951 298.51951 150.01951 15001.95100 2020-01-01 2020-01-02 2020-01-01 00:08:26 2020-01-02 03:36:47 2020-01-01 00:08:26.000 2020-01-02 03:36:47.000 506 99407 49956.5 4995650 506 99407 49956.5 4995650 -32664 32474 4736.66 473666 -127 125 -1.9 -190 -507 2 10497 99408 1.52252 298.52252 150.02252 15002.25225 1.52252 298.52252 150.02252 15002.2522 1.52252 298.52252 150.02252 15002.25200 2020-01-01 2020-01-02 2020-01-01 00:08:27 2020-01-02 03:36:48 2020-01-01 00:08:27.000 2020-01-02 03:36:48.000 507 99408 49957.5 4995750 507 99408 49957.5 4995750 -32663 32475 4737.66 473766 -126 126 -0.9 -90 -508 2 10498 99409 1.52552 298.52552 150.02552 15002.55255 1.52552 298.5255 150.02552 15002.5525 1.52552 298.52552 150.02552 15002.55200 2020-01-01 2020-01-02 2020-01-01 00:08:28 2020-01-02 03:36:49 2020-01-01 00:08:28.000 2020-01-02 03:36:49.000 508 99409 49958.5 4995850 508 99409 49958.5 4995850 -32662 32476 4738.66 473866 -125 127 0.1 10 -509 2 10499 99410 1.52852 298.52852 150.02852 15002.85285 1.52852 298.52853 150.02853 15002.85312 1.52852 298.52852 150.02852 15002.85200 2020-01-01 2020-01-02 2020-01-01 00:08:29 2020-01-02 03:36:50 2020-01-01 00:08:29.000 2020-01-02 03:36:50.000 509 99410 49959.5 4995950 509 99410 49959.5 4995950 -32661 32477 4739.66 473966 -128 127 -1.46 -146 -51 2 10041 99951 0.15315 300.15315 150.15315 15165.46846 0.15315 300.15317 150.15315 15165.46863 0.15315 300.15315 150.15315 15165.46815 2020-01-01 2020-01-02 2020-01-01 00:00:51 2020-01-02 03:45:51 2020-01-01 00:00:51.000 2020-01-02 03:45:51.000 51 99951 50001 5050101 51 99951 50001 5050101 -32518 32417 4580.009900990099 462581 -126 125 -2.6435643564356437 -267 -510 2 10500 99411 1.53153 298.53153 150.03153 15003.15315 1.53153 298.53152 150.03152 15003.15278 1.53153 298.53153 150.03153 15003.15300 2020-01-01 2020-01-02 2020-01-01 00:08:30 2020-01-02 03:36:51 2020-01-01 00:08:30.000 2020-01-02 03:36:51.000 510 99411 49960.5 4996050 510 99411 49960.5 4996050 -32660 32478 4740.66 474066 -128 127 -3.02 -302 -511 2 10501 99412 1.53453 298.53453 150.03453 15003.45345 1.53453 298.53455 150.03453 15003.45354 1.53453 298.53453 150.03453 15003.45300 2020-01-01 2020-01-02 2020-01-01 00:08:31 2020-01-02 03:36:52 2020-01-01 00:08:31.000 2020-01-02 03:36:52.000 511 99412 49961.5 4996150 511 99412 49961.5 4996150 -32659 32479 4741.66 474166 -128 123 -4.58 -458 -512 2 10502 99413 1.53753 298.53753 150.03753 15003.75375 1.53753 298.53754 150.03753 15003.75366 1.53753 298.53753 150.03753 15003.75300 2020-01-01 2020-01-02 2020-01-01 00:08:32 2020-01-02 03:36:53 2020-01-01 00:08:32.000 2020-01-02 03:36:53.000 512 99413 49962.5 4996250 512 99413 49962.5 4996250 -32658 32480 4742.66 474266 -127 124 -3.58 -358 +500 2 10490 99401 1.5015 298.5015 150.0015 15000.15015 1.5015 298.5015 150.00149 15000.14984 1.50150 298.50150 150.00150000000014 15000.15000 2020-01-01 2020-01-02 2020-01-01 00:08:20 2020-01-02 03:36:41 2020-01-01 00:08:20.000 2020-01-02 03:36:41.000 500 99401 49950.5 4995050 500 99401 49950.5 4995050 -32670 32468 4730.66 473066 -126 125 -0.22 -22 +501 2 10491 99402 1.5045 298.5045 150.0045 15000.45045 1.5045 298.50452 150.0045 15000.45043 1.50450 298.50450 150.0044999999998 15000.45000 2020-01-01 2020-01-02 2020-01-01 00:08:21 2020-01-02 03:36:42 2020-01-01 00:08:21.000 2020-01-02 03:36:42.000 501 99402 49951.5 4995150 501 99402 49951.5 4995150 -32669 32469 4731.66 473166 -125 126 0.78 78 +502 2 10492 99403 1.5075 298.5075 150.0075 15000.75075 1.5075 298.5075 150.0075 15000.75073 1.50750 298.50750 150.00749999999988 15000.75000 2020-01-01 2020-01-02 2020-01-01 00:08:22 2020-01-02 03:36:43 2020-01-01 00:08:22.000 2020-01-02 03:36:43.000 502 99403 49952.5 4995250 502 99403 49952.5 4995250 -32668 32470 4732.66 473266 -124 127 1.78 178 +503 2 10493 99404 1.51051 298.51051 150.01051 15001.05105 1.51051 298.5105 150.01051 15001.05103 1.51051 298.51051 150.01051000000004 15001.05100 2020-01-01 2020-01-02 2020-01-01 00:08:23 2020-01-02 03:36:44 2020-01-01 00:08:23.000 2020-01-02 03:36:44.000 503 99404 49953.5 4995350 503 99404 49953.5 4995350 -32667 32471 4733.66 473366 -128 127 0.22 22 +504 2 10494 99405 1.51351 298.51351 150.01351 15001.35135 1.51351 298.51352 150.01351 15001.35162 1.51351 298.51351 150.01351000000005 15001.35100 2020-01-01 2020-01-02 2020-01-01 00:08:24 2020-01-02 03:36:45 2020-01-01 00:08:24.000 2020-01-02 03:36:45.000 504 99405 49954.5 4995450 504 99405 49954.5 4995450 -32666 32472 4734.66 473466 -128 127 -1.34 -134 +505 2 10495 99406 1.51651 298.51651 150.01651 15001.65165 1.51651 298.5165 150.01651 15001.65131 1.51651 298.51651 150.01650999999998 15001.65100 2020-01-01 2020-01-02 2020-01-01 00:08:25 2020-01-02 03:36:46 2020-01-01 00:08:25.000 2020-01-02 03:36:46.000 505 99406 49955.5 4995550 505 99406 49955.5 4995550 -32665 32473 4735.66 473566 -128 124 -2.9 -290 +506 2 10496 99407 1.51951 298.51951 150.01951 15001.95195 1.51951 298.51953 150.01951 15001.9519 1.51951 298.51951 150.01951000000005 15001.95100 2020-01-01 2020-01-02 2020-01-01 00:08:26 2020-01-02 03:36:47 2020-01-01 00:08:26.000 2020-01-02 03:36:47.000 506 99407 49956.5 4995650 506 99407 49956.5 4995650 -32664 32474 4736.66 473666 -127 125 -1.9 -190 +507 2 10497 99408 1.52252 298.52252 150.02252 15002.25225 1.52252 298.52252 150.02252 15002.2522 1.52252 298.52252 150.0225200000002 15002.25200 2020-01-01 2020-01-02 2020-01-01 00:08:27 2020-01-02 03:36:48 2020-01-01 00:08:27.000 2020-01-02 03:36:48.000 507 99408 49957.5 4995750 507 99408 49957.5 4995750 -32663 32475 4737.66 473766 -126 126 -0.9 -90 +508 2 10498 99409 1.52552 298.52552 150.02552 15002.55255 1.52552 298.5255 150.02552 15002.5525 1.52552 298.52552 150.02551999999991 15002.55200 2020-01-01 2020-01-02 2020-01-01 00:08:28 2020-01-02 03:36:49 2020-01-01 00:08:28.000 2020-01-02 03:36:49.000 508 99409 49958.5 4995850 508 99409 49958.5 4995850 -32662 32476 4738.66 473866 -125 127 0.1 10 +509 2 10499 99410 1.52852 298.52852 150.02852 15002.85285 1.52852 298.52853 150.02853 15002.85312 1.52852 298.52852 150.0285199999999 15002.85200 2020-01-01 2020-01-02 2020-01-01 00:08:29 2020-01-02 03:36:50 2020-01-01 00:08:29.000 2020-01-02 03:36:50.000 509 99410 49959.5 4995950 509 99410 49959.5 4995950 -32661 32477 4739.66 473966 -128 127 -1.46 -146 +51 2 10041 99951 0.15315 300.15315 150.15315 15165.46846 0.15315 300.15317 150.15315 15165.46863 0.15315 300.15315 150.15315000000007 15165.46815 2020-01-01 2020-01-02 2020-01-01 00:00:51 2020-01-02 03:45:51 2020-01-01 00:00:51.000 2020-01-02 03:45:51.000 51 99951 50001 5050101 51 99951 50001 5050101 -32518 32417 4580.009900990099 462581 -126 125 -2.6435643564356437 -267 +510 2 10500 99411 1.53153 298.53153 150.03153 15003.15315 1.53153 298.53152 150.03152 15003.15278 1.53153 298.53153 150.03153000000012 15003.15300 2020-01-01 2020-01-02 2020-01-01 00:08:30 2020-01-02 03:36:51 2020-01-01 00:08:30.000 2020-01-02 03:36:51.000 510 99411 49960.5 4996050 510 99411 49960.5 4996050 -32660 32478 4740.66 474066 -128 127 -3.02 -302 +511 2 10501 99412 1.53453 298.53453 150.03453 15003.45345 1.53453 298.53455 150.03453 15003.45354 1.53453 298.53453 150.03453000000013 15003.45300 2020-01-01 2020-01-02 2020-01-01 00:08:31 2020-01-02 03:36:52 2020-01-01 00:08:31.000 2020-01-02 03:36:52.000 511 99412 49961.5 4996150 511 99412 49961.5 4996150 -32659 32479 4741.66 474166 -128 123 -4.58 -458 +512 2 10502 99413 1.53753 298.53753 150.03753 15003.75375 1.53753 298.53754 150.03753 15003.75366 1.53753 298.53753 150.0375299999998 15003.75300 2020-01-01 2020-01-02 2020-01-01 00:08:32 2020-01-02 03:36:53 2020-01-01 00:08:32.000 2020-01-02 03:36:53.000 512 99413 49962.5 4996250 512 99413 49962.5 4996250 -32658 32480 4742.66 474266 -127 124 -3.58 -358 513 2 10503 99414 1.54054 298.54054 150.04054 15004.05405 1.54054 298.54053 150.04053 15004.05397 1.54054 298.54054 150.04054 15004.05400 2020-01-01 2020-01-02 2020-01-01 00:08:33 2020-01-02 03:36:54 2020-01-01 00:08:33.000 2020-01-02 03:36:54.000 513 99414 49963.5 4996350 513 99414 49963.5 4996350 -32657 32481 4743.66 474366 -126 125 -2.58 -258 -514 2 10504 99415 1.54354 298.54354 150.04354 15004.35435 1.54354 298.54355 150.04354 15004.35459 1.54354 298.54354 150.04354 15004.35400 2020-01-01 2020-01-02 2020-01-01 00:08:34 2020-01-02 03:36:55 2020-01-01 00:08:34.000 2020-01-02 03:36:55.000 514 99415 49964.5 4996450 514 99415 49964.5 4996450 -32656 32482 4744.66 474466 -125 126 -1.58 -158 -515 2 10505 99416 1.54654 298.54654 150.04654 15004.65465 1.54654 298.54654 150.04654 15004.65425 1.54654 298.54654 150.04654 15004.65400 2020-01-01 2020-01-02 2020-01-01 00:08:35 2020-01-02 03:36:56 2020-01-01 00:08:35.000 2020-01-02 03:36:56.000 515 99416 49965.5 4996550 515 99416 49965.5 4996550 -32655 32483 4745.66 474566 -124 127 -0.58 -58 +514 2 10504 99415 1.54354 298.54354 150.04354 15004.35435 1.54354 298.54355 150.04354 15004.35459 1.54354 298.54354 150.04354000000004 15004.35400 2020-01-01 2020-01-02 2020-01-01 00:08:34 2020-01-02 03:36:55 2020-01-01 00:08:34.000 2020-01-02 03:36:55.000 514 99415 49964.5 4996450 514 99415 49964.5 4996450 -32656 32482 4744.66 474466 -125 126 -1.58 -158 +515 2 10505 99416 1.54654 298.54654 150.04654 15004.65465 1.54654 298.54654 150.04654 15004.65425 1.54654 298.54654 150.0465399999999 15004.65400 2020-01-01 2020-01-02 2020-01-01 00:08:35 2020-01-02 03:36:56 2020-01-01 00:08:35.000 2020-01-02 03:36:56.000 515 99416 49965.5 4996550 515 99416 49965.5 4996550 -32655 32483 4745.66 474566 -124 127 -0.58 -58 516 2 10506 99417 1.54954 298.54954 150.04954 15004.95495 1.54954 298.54956 150.04955 15004.955 1.54954 298.54954 150.04954 15004.95400 2020-01-01 2020-01-02 2020-01-01 00:08:36 2020-01-02 03:36:57 2020-01-01 00:08:36.000 2020-01-02 03:36:57.000 516 99417 49966.5 4996650 516 99417 49966.5 4996650 -32654 32484 4746.66 474666 -128 127 -2.14 -214 -517 2 10507 99418 1.55255 298.55255 150.05255 15005.25525 1.55255 298.55255 150.05255 15005.25514 1.55255 298.55255 150.05255 15005.25500 2020-01-01 2020-01-02 2020-01-01 00:08:37 2020-01-02 03:36:58 2020-01-01 00:08:37.000 2020-01-02 03:36:58.000 517 99418 49967.5 4996750 517 99418 49967.5 4996750 -32653 32485 4747.66 474766 -128 123 -3.7 -370 -518 2 10508 99419 1.55555 298.55555 150.05555 15005.55555 1.55555 298.55554 150.05555 15005.55547 1.55555 298.55555 150.05555 15005.55500 2020-01-01 2020-01-02 2020-01-01 00:08:38 2020-01-02 03:36:59 2020-01-01 00:08:38.000 2020-01-02 03:36:59.000 518 99419 49968.5 4996850 518 99419 49968.5 4996850 -32652 32486 4748.66 474866 -127 124 -2.7 -270 -519 2 10509 99420 1.55855 298.55855 150.05855 15005.85585 1.55855 298.55856 150.05856 15005.85607 1.55855 298.55855 150.05855 15005.85500 2020-01-01 2020-01-02 2020-01-01 00:08:39 2020-01-02 03:37:00 2020-01-01 00:08:39.000 2020-01-02 03:37:00.000 519 99420 49969.5 4996950 519 99420 49969.5 4996950 -32651 32487 4749.66 474966 -126 125 -1.7 -170 -52 2 10042 99952 0.15615 300.15615 150.15615 15165.77177 0.15615 300.15616 150.15615 15165.77193 0.15615 300.15615 150.15615 15165.77115 2020-01-01 2020-01-02 2020-01-01 00:00:52 2020-01-02 03:45:52 2020-01-01 00:00:52.000 2020-01-02 03:45:52.000 52 99952 50002 5050202 52 99952 50002 5050202 -32517 32418 4581.009900990099 462682 -125 126 -1.6435643564356435 -166 -520 2 10510 99421 1.56156 298.56156 150.06156 15006.15615 1.56156 298.56155 150.06155 15006.15572 1.56156 298.56156 150.06156 15006.15600 2020-01-01 2020-01-02 2020-01-01 00:08:40 2020-01-02 03:37:01 2020-01-01 00:08:40.000 2020-01-02 03:37:01.000 520 99421 49970.5 4997050 520 99421 49970.5 4997050 -32650 32488 4750.66 475066 -125 126 -0.7 -70 -521 2 10511 99422 1.56456 298.56456 150.06456 15006.45645 1.56456 298.56458 150.06456 15006.45647 1.56456 298.56456 150.06456 15006.45600 2020-01-01 2020-01-02 2020-01-01 00:08:41 2020-01-02 03:37:02 2020-01-01 00:08:41.000 2020-01-02 03:37:02.000 521 99422 49971.5 4997150 521 99422 49971.5 4997150 -32649 32489 4751.66 475166 -124 127 0.3 30 -522 2 10512 99423 1.56756 298.56756 150.06756 15006.75675 1.56756 298.56757 150.06756 15006.75661 1.56756 298.56756 150.06756 15006.75600 2020-01-01 2020-01-02 2020-01-01 00:08:42 2020-01-02 03:37:03 2020-01-01 00:08:42.000 2020-01-02 03:37:03.000 522 99423 49972.5 4997250 522 99423 49972.5 4997250 -32648 32490 4752.66 475266 -128 127 -1.26 -126 -523 2 10513 99424 1.57057 298.57057 150.07057 15007.05705 1.57057 298.57056 150.07056 15007.05694 1.57057 298.57057 150.07057 15007.05700 2020-01-01 2020-01-02 2020-01-01 00:08:43 2020-01-02 03:37:04 2020-01-01 00:08:43.000 2020-01-02 03:37:04.000 523 99424 49973.5 4997350 523 99424 49973.5 4997350 -32647 32491 4753.66 475366 -128 123 -2.82 -282 -524 2 10514 99425 1.57357 298.57357 150.07357 15007.35735 1.57357 298.57358 150.07357 15007.35753 1.57357 298.57357 150.07357 15007.35700 2020-01-01 2020-01-02 2020-01-01 00:08:44 2020-01-02 03:37:05 2020-01-01 00:08:44.000 2020-01-02 03:37:05.000 524 99425 49974.5 4997450 524 99425 49974.5 4997450 -32646 32492 4754.66 475466 -127 124 -1.82 -182 -525 2 10515 99426 1.57657 298.57657 150.07657 15007.65765 1.57657 298.57657 150.07657 15007.65783 1.57657 298.57657 150.07657 15007.65700 2020-01-01 2020-01-02 2020-01-01 00:08:45 2020-01-02 03:37:06 2020-01-01 00:08:45.000 2020-01-02 03:37:06.000 525 99426 49975.5 4997550 525 99426 49975.5 4997550 -32645 32493 4755.66 475566 -126 125 -0.82 -82 -526 2 10516 99427 1.57957 298.57957 150.07957 15007.95795 1.57957 298.5796 150.07957 15007.95795 1.57957 298.57957 150.07957 15007.95700 2020-01-01 2020-01-02 2020-01-01 00:08:46 2020-01-02 03:37:07 2020-01-01 00:08:46.000 2020-01-02 03:37:07.000 526 99427 49976.5 4997650 526 99427 49976.5 4997650 -32644 32494 4756.66 475666 -125 126 0.18 18 -527 2 10517 99428 1.58258 298.58258 150.08258 15008.25825 1.58258 298.58258 150.08258 15008.25811 1.58258 298.58258 150.08258 15008.25800 2020-01-01 2020-01-02 2020-01-01 00:08:47 2020-01-02 03:37:08 2020-01-01 00:08:47.000 2020-01-02 03:37:08.000 527 99428 49977.5 4997750 527 99428 49977.5 4997750 -32643 32495 4757.66 475766 -124 127 1.18 118 -528 2 10518 99429 1.58558 298.58558 150.08558 15008.55855 1.58558 298.58557 150.08558 15008.5584 1.58558 298.58558 150.08558 15008.55800 2020-01-01 2020-01-02 2020-01-01 00:08:48 2020-01-02 03:37:09 2020-01-01 00:08:48.000 2020-01-02 03:37:09.000 528 99429 49978.5 4997850 528 99429 49978.5 4997850 -32642 32496 4758.66 475866 -128 127 -0.38 -38 -529 2 10519 99430 1.58858 298.58858 150.08858 15008.85885 1.58858 298.5886 150.08859 15008.859 1.58858 298.58858 150.08858 15008.85800 2020-01-01 2020-01-02 2020-01-01 00:08:49 2020-01-02 03:37:10 2020-01-01 00:08:49.000 2020-01-02 03:37:10.000 529 99430 49979.5 4997950 529 99430 49979.5 4997950 -32641 32497 4759.66 475966 -128 127 -1.94 -194 -53 2 10043 99953 0.15915 300.15915 150.15915 15166.07507 0.15915 300.15915 150.15915 15166.07511 0.15915 300.15915 150.15915 15166.07415 2020-01-01 2020-01-02 2020-01-01 00:00:53 2020-01-02 03:45:53 2020-01-01 00:00:53.000 2020-01-02 03:45:53.000 53 99953 50003 5050303 53 99953 50003 5050303 -32516 32419 4582.009900990099 462783 -124 127 -0.6435643564356436 -65 +517 2 10507 99418 1.55255 298.55255 150.05255 15005.25525 1.55255 298.55255 150.05255 15005.25514 1.55255 298.55255 150.05255000000017 15005.25500 2020-01-01 2020-01-02 2020-01-01 00:08:37 2020-01-02 03:36:58 2020-01-01 00:08:37.000 2020-01-02 03:36:58.000 517 99418 49967.5 4996750 517 99418 49967.5 4996750 -32653 32485 4747.66 474766 -128 123 -3.7 -370 +518 2 10508 99419 1.55555 298.55555 150.05555 15005.55555 1.55555 298.55554 150.05555 15005.55547 1.55555 298.55555 150.0555499999998 15005.55500 2020-01-01 2020-01-02 2020-01-01 00:08:38 2020-01-02 03:36:59 2020-01-01 00:08:38.000 2020-01-02 03:36:59.000 518 99419 49968.5 4996850 518 99419 49968.5 4996850 -32652 32486 4748.66 474866 -127 124 -2.7 -270 +519 2 10509 99420 1.55855 298.55855 150.05855 15005.85585 1.55855 298.55856 150.05856 15005.85607 1.55855 298.55855 150.0585499999999 15005.85500 2020-01-01 2020-01-02 2020-01-01 00:08:39 2020-01-02 03:37:00 2020-01-01 00:08:39.000 2020-01-02 03:37:00.000 519 99420 49969.5 4996950 519 99420 49969.5 4996950 -32651 32487 4749.66 474966 -126 125 -1.7 -170 +52 2 10042 99952 0.15615 300.15615 150.15615 15165.77177 0.15615 300.15616 150.15615 15165.77193 0.15615 300.15615 150.15615000000003 15165.77115 2020-01-01 2020-01-02 2020-01-01 00:00:52 2020-01-02 03:45:52 2020-01-01 00:00:52.000 2020-01-02 03:45:52.000 52 99952 50002 5050202 52 99952 50002 5050202 -32517 32418 4581.009900990099 462682 -125 126 -1.6435643564356435 -166 +520 2 10510 99421 1.56156 298.56156 150.06156 15006.15615 1.56156 298.56155 150.06155 15006.15572 1.56156 298.56156 150.06156000000007 15006.15600 2020-01-01 2020-01-02 2020-01-01 00:08:40 2020-01-02 03:37:01 2020-01-01 00:08:40.000 2020-01-02 03:37:01.000 520 99421 49970.5 4997050 520 99421 49970.5 4997050 -32650 32488 4750.66 475066 -125 126 -0.7 -70 +521 2 10511 99422 1.56456 298.56456 150.06456 15006.45645 1.56456 298.56458 150.06456 15006.45647 1.56456 298.56456 150.06456000000009 15006.45600 2020-01-01 2020-01-02 2020-01-01 00:08:41 2020-01-02 03:37:02 2020-01-01 00:08:41.000 2020-01-02 03:37:02.000 521 99422 49971.5 4997150 521 99422 49971.5 4997150 -32649 32489 4751.66 475166 -124 127 0.3 30 +522 2 10512 99423 1.56756 298.56756 150.06756 15006.75675 1.56756 298.56757 150.06756 15006.75661 1.56756 298.56756 150.06756000000004 15006.75600 2020-01-01 2020-01-02 2020-01-01 00:08:42 2020-01-02 03:37:03 2020-01-01 00:08:42.000 2020-01-02 03:37:03.000 522 99423 49972.5 4997250 522 99423 49972.5 4997250 -32648 32490 4752.66 475266 -128 127 -1.26 -126 +523 2 10513 99424 1.57057 298.57057 150.07057 15007.05705 1.57057 298.57056 150.07056 15007.05694 1.57057 298.57057 150.0705699999999 15007.05700 2020-01-01 2020-01-02 2020-01-01 00:08:43 2020-01-02 03:37:04 2020-01-01 00:08:43.000 2020-01-02 03:37:04.000 523 99424 49973.5 4997350 523 99424 49973.5 4997350 -32647 32491 4753.66 475366 -128 123 -2.82 -282 +524 2 10514 99425 1.57357 298.57357 150.07357 15007.35735 1.57357 298.57358 150.07357 15007.35753 1.57357 298.57357 150.07356999999996 15007.35700 2020-01-01 2020-01-02 2020-01-01 00:08:44 2020-01-02 03:37:05 2020-01-01 00:08:44.000 2020-01-02 03:37:05.000 524 99425 49974.5 4997450 524 99425 49974.5 4997450 -32646 32492 4754.66 475466 -127 124 -1.82 -182 +525 2 10515 99426 1.57657 298.57657 150.07657 15007.65765 1.57657 298.57657 150.07657 15007.65783 1.57657 298.57657 150.07656999999992 15007.65700 2020-01-01 2020-01-02 2020-01-01 00:08:45 2020-01-02 03:37:06 2020-01-01 00:08:45.000 2020-01-02 03:37:06.000 525 99426 49975.5 4997550 525 99426 49975.5 4997550 -32645 32493 4755.66 475566 -126 125 -0.82 -82 +526 2 10516 99427 1.57957 298.57957 150.07957 15007.95795 1.57957 298.5796 150.07957 15007.95795 1.57957 298.57957 150.0795699999999 15007.95700 2020-01-01 2020-01-02 2020-01-01 00:08:46 2020-01-02 03:37:07 2020-01-01 00:08:46.000 2020-01-02 03:37:07.000 526 99427 49976.5 4997650 526 99427 49976.5 4997650 -32644 32494 4756.66 475666 -125 126 0.18 18 +527 2 10517 99428 1.58258 298.58258 150.08258 15008.25825 1.58258 298.58258 150.08258 15008.25811 1.58258 298.58258 150.08258000000012 15008.25800 2020-01-01 2020-01-02 2020-01-01 00:08:47 2020-01-02 03:37:08 2020-01-01 00:08:47.000 2020-01-02 03:37:08.000 527 99428 49977.5 4997750 527 99428 49977.5 4997750 -32643 32495 4757.66 475766 -124 127 1.18 118 +528 2 10518 99429 1.58558 298.58558 150.08558 15008.55855 1.58558 298.58557 150.08558 15008.5584 1.58558 298.58558 150.08558000000014 15008.55800 2020-01-01 2020-01-02 2020-01-01 00:08:48 2020-01-02 03:37:09 2020-01-01 00:08:48.000 2020-01-02 03:37:09.000 528 99429 49978.5 4997850 528 99429 49978.5 4997850 -32642 32496 4758.66 475866 -128 127 -0.38 -38 +529 2 10519 99430 1.58858 298.58858 150.08858 15008.85885 1.58858 298.5886 150.08859 15008.859 1.58858 298.58858 150.08857999999984 15008.85800 2020-01-01 2020-01-02 2020-01-01 00:08:49 2020-01-02 03:37:10 2020-01-01 00:08:49.000 2020-01-02 03:37:10.000 529 99430 49979.5 4997950 529 99430 49979.5 4997950 -32641 32497 4759.66 475966 -128 127 -1.94 -194 +53 2 10043 99953 0.15915 300.15915 150.15915 15166.07507 0.15915 300.15915 150.15915 15166.07511 0.15915 300.15915 150.15914999999998 15166.07415 2020-01-01 2020-01-02 2020-01-01 00:00:53 2020-01-02 03:45:53 2020-01-01 00:00:53.000 2020-01-02 03:45:53.000 53 99953 50003 5050303 53 99953 50003 5050303 -32516 32419 4582.009900990099 462783 -124 127 -0.6435643564356436 -65 530 2 10520 99431 1.59159 298.59159 150.09159 15009.15915 1.59159 298.59158 150.09159 15009.15929 1.59159 298.59159 150.09159 15009.15900 2020-01-01 2020-01-02 2020-01-01 00:08:50 2020-01-02 03:37:11 2020-01-01 00:08:50.000 2020-01-02 03:37:11.000 530 99431 49980.5 4998050 530 99431 49980.5 4998050 -32640 32498 4760.66 476066 -128 124 -3.5 -350 -531 2 10521 99432 1.59459 298.59459 150.09459 15009.45945 1.59459 298.5946 150.09459 15009.45941 1.59459 298.59459 150.09459 15009.45900 2020-01-01 2020-01-02 2020-01-01 00:08:51 2020-01-02 03:37:12 2020-01-01 00:08:51.000 2020-01-02 03:37:12.000 531 99432 49981.5 4998150 531 99432 49981.5 4998150 -32639 32499 4761.66 476166 -127 125 -2.5 -250 -532 2 10522 99433 1.59759 298.59759 150.09759 15009.75975 1.59759 298.5976 150.09759 15009.75958 1.59759 298.59759 150.09759 15009.75900 2020-01-01 2020-01-02 2020-01-01 00:08:52 2020-01-02 03:37:13 2020-01-01 00:08:52.000 2020-01-02 03:37:13.000 532 99433 49982.5 4998250 532 99433 49982.5 4998250 -32638 32500 4762.66 476266 -126 126 -1.5 -150 -533 2 10523 99434 1.6006 298.6006 150.1006 15010.06006 1.6006 298.6006 150.10059 15010.05988 1.60060 298.60060 150.10060 15010.06000 2020-01-01 2020-01-02 2020-01-01 00:08:53 2020-01-02 03:37:14 2020-01-01 00:08:53.000 2020-01-02 03:37:14.000 533 99434 49983.5 4998350 533 99434 49983.5 4998350 -32637 32501 4763.66 476366 -125 127 -0.5 -50 -534 2 10524 99435 1.6036 298.6036 150.1036 15010.36036 1.6036 298.6036 150.1036 15010.36063 1.60360 298.60360 150.10360 15010.36000 2020-01-01 2020-01-02 2020-01-01 00:08:54 2020-01-02 03:37:15 2020-01-01 00:08:54.000 2020-01-02 03:37:15.000 534 99435 49984.5 4998450 534 99435 49984.5 4998450 -32636 32502 4764.66 476466 -128 127 -2.06 -206 -535 2 10525 99436 1.6066 298.6066 150.1066 15010.66066 1.6066 298.6066 150.1066 15010.66077 1.60660 298.60660 150.10660 15010.66000 2020-01-01 2020-01-02 2020-01-01 00:08:55 2020-01-02 03:37:16 2020-01-01 00:08:55.000 2020-01-02 03:37:16.000 535 99436 49985.5 4998550 535 99436 49985.5 4998550 -32635 32503 4765.66 476566 -128 127 -3.62 -362 -536 2 10526 99437 1.6096 298.6096 150.1096 15010.96096 1.6096 298.60962 150.1096 15010.96092 1.60960 298.60960 150.10960 15010.96000 2020-01-01 2020-01-02 2020-01-01 00:08:56 2020-01-02 03:37:17 2020-01-01 00:08:56.000 2020-01-02 03:37:17.000 536 99437 49986.5 4998650 536 99437 49986.5 4998650 -32634 32504 4766.66 476666 -128 123 -5.18 -518 -537 2 10527 99438 1.61261 298.61261 150.11261 15011.26126 1.61261 298.6126 150.11261 15011.26105 1.61261 298.61261 150.11261 15011.26100 2020-01-01 2020-01-02 2020-01-01 00:08:57 2020-01-02 03:37:18 2020-01-01 00:08:57.000 2020-01-02 03:37:18.000 537 99438 49987.5 4998750 537 99438 49987.5 4998750 -32633 32505 4767.66 476766 -127 124 -4.18 -418 -538 2 10528 99439 1.61561 298.61561 150.11561 15011.56156 1.61561 298.6156 150.11561 15011.56135 1.61561 298.61561 150.11561 15011.56100 2020-01-01 2020-01-02 2020-01-01 00:08:58 2020-01-02 03:37:19 2020-01-01 00:08:58.000 2020-01-02 03:37:19.000 538 99439 49988.5 4998850 538 99439 49988.5 4998850 -32632 32506 4768.66 476866 -126 125 -3.18 -318 -539 2 10529 99440 1.61861 298.61861 150.11861 15011.86186 1.61861 298.61862 150.11862 15011.8621 1.61861 298.61861 150.11861 15011.86100 2020-01-01 2020-01-02 2020-01-01 00:08:59 2020-01-02 03:37:20 2020-01-01 00:08:59.000 2020-01-02 03:37:20.000 539 99440 49989.5 4998950 539 99440 49989.5 4998950 -32631 32507 4769.66 476966 -125 126 -2.18 -218 -54 2 10044 99954 0.16216 300.16216 150.16216 15166.37837 0.16216 300.16217 150.16216 15166.37822 0.16216 300.16216 150.16216 15166.37816 2020-01-01 2020-01-02 2020-01-01 00:00:54 2020-01-02 03:45:54 2020-01-01 00:00:54.000 2020-01-02 03:45:54.000 54 99954 50004 5050404 54 99954 50004 5050404 -32515 32420 4583.009900990099 462884 -128 127 -2.1782178217821784 -220 -540 2 10530 99441 1.62162 298.62162 150.12162 15012.16216 1.62162 298.6216 150.12162 15012.16224 1.62162 298.62162 150.12162 15012.16200 2020-01-01 2020-01-02 2020-01-01 00:09:00 2020-01-02 03:37:21 2020-01-01 00:09:00.000 2020-01-02 03:37:21.000 540 99441 49990.5 4999050 540 99441 49990.5 4999050 -32630 32508 4770.66 477066 -124 127 -1.18 -118 +531 2 10521 99432 1.59459 298.59459 150.09459 15009.45945 1.59459 298.5946 150.09459 15009.45941 1.59459 298.59459 150.09459000000007 15009.45900 2020-01-01 2020-01-02 2020-01-01 00:08:51 2020-01-02 03:37:12 2020-01-01 00:08:51.000 2020-01-02 03:37:12.000 531 99432 49981.5 4998150 531 99432 49981.5 4998150 -32639 32499 4761.66 476166 -127 125 -2.5 -250 +532 2 10522 99433 1.59759 298.59759 150.09759 15009.75975 1.59759 298.5976 150.09759 15009.75958 1.59759 298.59759 150.09758999999994 15009.75900 2020-01-01 2020-01-02 2020-01-01 00:08:52 2020-01-02 03:37:13 2020-01-01 00:08:52.000 2020-01-02 03:37:13.000 532 99433 49982.5 4998250 532 99433 49982.5 4998250 -32638 32500 4762.66 476266 -126 126 -1.5 -150 +533 2 10523 99434 1.6006 298.6006 150.1006 15010.06006 1.6006 298.6006 150.10059 15010.05988 1.60060 298.60060 150.10059999999984 15010.06000 2020-01-01 2020-01-02 2020-01-01 00:08:53 2020-01-02 03:37:14 2020-01-01 00:08:53.000 2020-01-02 03:37:14.000 533 99434 49983.5 4998350 533 99434 49983.5 4998350 -32637 32501 4763.66 476366 -125 127 -0.5 -50 +534 2 10524 99435 1.6036 298.6036 150.1036 15010.36036 1.6036 298.6036 150.1036 15010.36063 1.60360 298.60360 150.10360000000017 15010.36000 2020-01-01 2020-01-02 2020-01-01 00:08:54 2020-01-02 03:37:15 2020-01-01 00:08:54.000 2020-01-02 03:37:15.000 534 99435 49984.5 4998450 534 99435 49984.5 4998450 -32636 32502 4764.66 476466 -128 127 -2.06 -206 +535 2 10525 99436 1.6066 298.6066 150.1066 15010.66066 1.6066 298.6066 150.1066 15010.66077 1.60660 298.60660 150.10659999999987 15010.66000 2020-01-01 2020-01-02 2020-01-01 00:08:55 2020-01-02 03:37:16 2020-01-01 00:08:55.000 2020-01-02 03:37:16.000 535 99436 49985.5 4998550 535 99436 49985.5 4998550 -32635 32503 4765.66 476566 -128 127 -3.62 -362 +536 2 10526 99437 1.6096 298.6096 150.1096 15010.96096 1.6096 298.60962 150.1096 15010.96092 1.60960 298.60960 150.10959999999986 15010.96000 2020-01-01 2020-01-02 2020-01-01 00:08:56 2020-01-02 03:37:17 2020-01-01 00:08:56.000 2020-01-02 03:37:17.000 536 99437 49986.5 4998650 536 99437 49986.5 4998650 -32634 32504 4766.66 476666 -128 123 -5.18 -518 +537 2 10527 99438 1.61261 298.61261 150.11261 15011.26126 1.61261 298.6126 150.11261 15011.26105 1.61261 298.61261 150.1126100000001 15011.26100 2020-01-01 2020-01-02 2020-01-01 00:08:57 2020-01-02 03:37:18 2020-01-01 00:08:57.000 2020-01-02 03:37:18.000 537 99438 49987.5 4998750 537 99438 49987.5 4998750 -32633 32505 4767.66 476766 -127 124 -4.18 -418 +538 2 10528 99439 1.61561 298.61561 150.11561 15011.56156 1.61561 298.6156 150.11561 15011.56135 1.61561 298.61561 150.11561000000012 15011.56100 2020-01-01 2020-01-02 2020-01-01 00:08:58 2020-01-02 03:37:19 2020-01-01 00:08:58.000 2020-01-02 03:37:19.000 538 99439 49988.5 4998850 538 99439 49988.5 4998850 -32632 32506 4768.66 476866 -126 125 -3.18 -318 +539 2 10529 99440 1.61861 298.61861 150.11861 15011.86186 1.61861 298.61862 150.11862 15011.8621 1.61861 298.61861 150.1186099999999 15011.86100 2020-01-01 2020-01-02 2020-01-01 00:08:59 2020-01-02 03:37:20 2020-01-01 00:08:59.000 2020-01-02 03:37:20.000 539 99440 49989.5 4998950 539 99440 49989.5 4998950 -32631 32507 4769.66 476966 -125 126 -2.18 -218 +54 2 10044 99954 0.16216 300.16216 150.16216 15166.37837 0.16216 300.16217 150.16216 15166.37822 0.16216 300.16216 150.1621599999999 15166.37816 2020-01-01 2020-01-02 2020-01-01 00:00:54 2020-01-02 03:45:54 2020-01-01 00:00:54.000 2020-01-02 03:45:54.000 54 99954 50004 5050404 54 99954 50004 5050404 -32515 32420 4583.009900990099 462884 -128 127 -2.1782178217821784 -220 +540 2 10530 99441 1.62162 298.62162 150.12162 15012.16216 1.62162 298.6216 150.12162 15012.16224 1.62162 298.62162 150.12161999999992 15012.16200 2020-01-01 2020-01-02 2020-01-01 00:09:00 2020-01-02 03:37:21 2020-01-01 00:09:00.000 2020-01-02 03:37:21.000 540 99441 49990.5 4999050 540 99441 49990.5 4999050 -32630 32508 4770.66 477066 -124 127 -1.18 -118 541 2 10531 99442 1.62462 298.62462 150.12462 15012.46246 1.62462 298.62463 150.12462 15012.46239 1.62462 298.62462 150.12462 15012.46200 2020-01-01 2020-01-02 2020-01-01 00:09:01 2020-01-02 03:37:22 2020-01-01 00:09:01.000 2020-01-02 03:37:22.000 541 99442 49991.5 4999150 541 99442 49991.5 4999150 -32629 32509 4771.66 477166 -128 127 -2.74 -274 -542 2 10532 99443 1.62762 298.62762 150.12762 15012.76276 1.62762 298.62762 150.12762 15012.76252 1.62762 298.62762 150.12762 15012.76200 2020-01-01 2020-01-02 2020-01-01 00:09:02 2020-01-02 03:37:23 2020-01-01 00:09:02.000 2020-01-02 03:37:23.000 542 99443 49992.5 4999250 542 99443 49992.5 4999250 -32628 32510 4772.66 477266 -128 123 -4.3 -430 -543 2 10533 99444 1.63063 298.63063 150.13063 15013.06306 1.63063 298.63065 150.13063 15013.06327 1.63063 298.63063 150.13063 15013.06300 2020-01-01 2020-01-02 2020-01-01 00:09:03 2020-01-02 03:37:24 2020-01-01 00:09:03.000 2020-01-02 03:37:24.000 543 99444 49993.5 4999350 543 99444 49993.5 4999350 -32627 32511 4773.66 477366 -127 124 -3.3 -330 -544 2 10534 99445 1.63363 298.63363 150.13363 15013.36336 1.63363 298.63364 150.13363 15013.36358 1.63363 298.63363 150.13363 15013.36300 2020-01-01 2020-01-02 2020-01-01 00:09:04 2020-01-02 03:37:25 2020-01-01 00:09:04.000 2020-01-02 03:37:25.000 544 99445 49994.5 4999450 544 99445 49994.5 4999450 -32626 32512 4774.66 477466 -126 125 -2.3 -230 -545 2 10535 99446 1.63663 298.63663 150.13663 15013.66366 1.63663 298.63663 150.13663 15013.6637 1.63663 298.63663 150.13663 15013.66300 2020-01-01 2020-01-02 2020-01-01 00:09:05 2020-01-02 03:37:26 2020-01-01 00:09:05.000 2020-01-02 03:37:26.000 545 99446 49995.5 4999550 545 99446 49995.5 4999550 -32625 32513 4775.66 477566 -125 126 -1.3 -130 -546 2 10536 99447 1.63963 298.63963 150.13963 15013.96396 1.63963 298.63965 150.13963 15013.96385 1.63963 298.63963 150.13963 15013.96300 2020-01-01 2020-01-02 2020-01-01 00:09:06 2020-01-02 03:37:27 2020-01-01 00:09:06.000 2020-01-02 03:37:27.000 546 99447 49996.5 4999650 546 99447 49996.5 4999650 -32624 32514 4776.66 477666 -124 127 -0.3 -30 +542 2 10532 99443 1.62762 298.62762 150.12762 15012.76276 1.62762 298.62762 150.12762 15012.76252 1.62762 298.62762 150.12761999999998 15012.76200 2020-01-01 2020-01-02 2020-01-01 00:09:02 2020-01-02 03:37:23 2020-01-01 00:09:02.000 2020-01-02 03:37:23.000 542 99443 49992.5 4999250 542 99443 49992.5 4999250 -32628 32510 4772.66 477266 -128 123 -4.3 -430 +543 2 10533 99444 1.63063 298.63063 150.13063 15013.06306 1.63063 298.63065 150.13063 15013.06327 1.63063 298.63063 150.13062999999985 15013.06300 2020-01-01 2020-01-02 2020-01-01 00:09:03 2020-01-02 03:37:24 2020-01-01 00:09:03.000 2020-01-02 03:37:24.000 543 99444 49993.5 4999350 543 99444 49993.5 4999350 -32627 32511 4773.66 477366 -127 124 -3.3 -330 +544 2 10534 99445 1.63363 298.63363 150.13363 15013.36336 1.63363 298.63364 150.13363 15013.36358 1.63363 298.63363 150.13363000000012 15013.36300 2020-01-01 2020-01-02 2020-01-01 00:09:04 2020-01-02 03:37:25 2020-01-01 00:09:04.000 2020-01-02 03:37:25.000 544 99445 49994.5 4999450 544 99445 49994.5 4999450 -32626 32512 4774.66 477466 -126 125 -2.3 -230 +545 2 10535 99446 1.63663 298.63663 150.13663 15013.66366 1.63663 298.63663 150.13663 15013.6637 1.63663 298.63663 150.1366300000002 15013.66300 2020-01-01 2020-01-02 2020-01-01 00:09:05 2020-01-02 03:37:26 2020-01-01 00:09:05.000 2020-01-02 03:37:26.000 545 99446 49995.5 4999550 545 99446 49995.5 4999550 -32625 32513 4775.66 477566 -125 126 -1.3 -130 +546 2 10536 99447 1.63963 298.63963 150.13963 15013.96396 1.63963 298.63965 150.13963 15013.96385 1.63963 298.63963 150.13962999999984 15013.96300 2020-01-01 2020-01-02 2020-01-01 00:09:06 2020-01-02 03:37:27 2020-01-01 00:09:06.000 2020-01-02 03:37:27.000 546 99447 49996.5 4999650 546 99447 49996.5 4999650 -32624 32514 4776.66 477666 -124 127 -0.3 -30 547 2 10537 99448 1.64264 298.64264 150.14264 15014.26426 1.64264 298.64264 150.14263 15014.26399 1.64264 298.64264 150.14264 15014.26400 2020-01-01 2020-01-02 2020-01-01 00:09:07 2020-01-02 03:37:28 2020-01-01 00:09:07.000 2020-01-02 03:37:28.000 547 99448 49997.5 4999750 547 99448 49997.5 4999750 -32623 32515 4777.66 477766 -128 127 -1.86 -186 -548 2 10538 99449 1.64564 298.64564 150.14564 15014.56456 1.64564 298.64566 150.14564 15014.56474 1.64564 298.64564 150.14564 15014.56400 2020-01-01 2020-01-02 2020-01-01 00:09:08 2020-01-02 03:37:29 2020-01-01 00:09:08.000 2020-01-02 03:37:29.000 548 99449 49998.5 4999850 548 99449 49998.5 4999850 -32622 32516 4778.66 477866 -128 123 -3.42 -342 -549 2 10539 99450 1.64864 298.64864 150.14864 15014.86486 1.64864 298.64865 150.14865 15014.86504 1.64864 298.64864 150.14864 15014.86400 2020-01-01 2020-01-02 2020-01-01 00:09:09 2020-01-02 03:37:30 2020-01-01 00:09:09.000 2020-01-02 03:37:30.000 549 99450 49999.5 4999950 549 99450 49999.5 4999950 -32621 32517 4779.66 477966 -127 124 -2.42 -242 -55 2 10045 99955 0.16516 300.16516 150.16516 15166.68168 0.16516 300.16516 150.16516 15166.68151 0.16516 300.16516 150.16516 15166.68116 2020-01-01 2020-01-02 2020-01-01 00:00:55 2020-01-02 03:45:55 2020-01-01 00:00:55.000 2020-01-02 03:45:55.000 55 99955 50005 5050505 55 99955 50005 5050505 -32514 32421 4584.009900990099 462985 -128 123 -3.712871287128713 -375 -550 2 10540 99451 1.65165 298.65165 150.15165 15015.16516 1.65165 298.65164 150.15165 15015.16521 1.65165 298.65165 150.15165 15015.16500 2020-01-01 2020-01-02 2020-01-01 00:09:10 2020-01-02 03:37:31 2020-01-01 00:09:10.000 2020-01-02 03:37:31.000 550 99451 50000.5 5000050 550 99451 50000.5 5000050 -32620 32518 4780.66 478066 -126 125 -1.42 -142 -551 2 10541 99452 1.65465 298.65465 150.15465 15015.46546 1.65465 298.65466 150.15465 15015.46533 1.65465 298.65465 150.15465 15015.46500 2020-01-01 2020-01-02 2020-01-01 00:09:11 2020-01-02 03:37:32 2020-01-01 00:09:11.000 2020-01-02 03:37:32.000 551 99452 50001.5 5000150 551 99452 50001.5 5000150 -32619 32519 4781.66 478166 -125 126 -0.42 -42 -552 2 10542 99453 1.65765 298.65765 150.15765 15015.76576 1.65765 298.65765 150.15765 15015.76562 1.65765 298.65765 150.15765 15015.76500 2020-01-01 2020-01-02 2020-01-01 00:09:12 2020-01-02 03:37:33 2020-01-01 00:09:12.000 2020-01-02 03:37:33.000 552 99453 50002.5 5000250 552 99453 50002.5 5000250 -32618 32520 4782.66 478266 -124 127 0.58 58 -553 2 10543 99454 1.66066 298.66066 150.16066 15016.06606 1.66066 298.66068 150.16066 15016.06621 1.66066 298.66066 150.16066 15016.06600 2020-01-01 2020-01-02 2020-01-01 00:09:13 2020-01-02 03:37:34 2020-01-01 00:09:13.000 2020-01-02 03:37:34.000 553 99454 50003.5 5000350 553 99454 50003.5 5000350 -32617 32521 4783.66 478366 -128 127 -0.98 -98 -554 2 10544 99455 1.66366 298.66366 150.16366 15016.36636 1.66366 298.66367 150.16366 15016.36651 1.66366 298.66366 150.16366 15016.36600 2020-01-01 2020-01-02 2020-01-01 00:09:14 2020-01-02 03:37:35 2020-01-01 00:09:14.000 2020-01-02 03:37:35.000 554 99455 50004.5 5000450 554 99455 50004.5 5000450 -32616 32522 4784.66 478466 -128 127 -2.54 -254 -555 2 10545 99456 1.66666 298.66666 150.16666 15016.66666 1.66666 298.66666 150.16666 15016.66668 1.66666 298.66666 150.16666 15016.66600 2020-01-01 2020-01-02 2020-01-01 00:09:15 2020-01-02 03:37:36 2020-01-01 00:09:15.000 2020-01-02 03:37:36.000 555 99456 50005.5 5000550 555 99456 50005.5 5000550 -32615 32523 4785.66 478566 -128 124 -4.1 -410 -556 2 10546 99457 1.66966 298.66966 150.16966 15016.96696 1.66966 298.66968 150.16966 15016.9668 1.66966 298.66966 150.16966 15016.96600 2020-01-01 2020-01-02 2020-01-01 00:09:16 2020-01-02 03:37:37 2020-01-01 00:09:16.000 2020-01-02 03:37:37.000 556 99457 50006.5 5000650 556 99457 50006.5 5000650 -32614 32524 4786.66 478666 -127 125 -3.1 -310 -557 2 10547 99458 1.67267 298.67267 150.17267 15017.26726 1.67267 298.67267 150.17267 15017.26709 1.67267 298.67267 150.17267 15017.26700 2020-01-01 2020-01-02 2020-01-01 00:09:17 2020-01-02 03:37:38 2020-01-01 00:09:17.000 2020-01-02 03:37:38.000 557 99458 50007.5 5000750 557 99458 50007.5 5000750 -32613 32525 4787.66 478766 -126 126 -2.1 -210 +548 2 10538 99449 1.64564 298.64564 150.14564 15014.56456 1.64564 298.64566 150.14564 15014.56474 1.64564 298.64564 150.1456400000001 15014.56400 2020-01-01 2020-01-02 2020-01-01 00:09:08 2020-01-02 03:37:29 2020-01-01 00:09:08.000 2020-01-02 03:37:29.000 548 99449 49998.5 4999850 548 99449 49998.5 4999850 -32622 32516 4778.66 477866 -128 123 -3.42 -342 +549 2 10539 99450 1.64864 298.64864 150.14864 15014.86486 1.64864 298.64865 150.14865 15014.86504 1.64864 298.64864 150.14863999999997 15014.86400 2020-01-01 2020-01-02 2020-01-01 00:09:09 2020-01-02 03:37:30 2020-01-01 00:09:09.000 2020-01-02 03:37:30.000 549 99450 49999.5 4999950 549 99450 49999.5 4999950 -32621 32517 4779.66 477966 -127 124 -2.42 -242 +55 2 10045 99955 0.16516 300.16516 150.16516 15166.68168 0.16516 300.16516 150.16516 15166.68151 0.16516 300.16516 150.16515999999996 15166.68116 2020-01-01 2020-01-02 2020-01-01 00:00:55 2020-01-02 03:45:55 2020-01-01 00:00:55.000 2020-01-02 03:45:55.000 55 99955 50005 5050505 55 99955 50005 5050505 -32514 32421 4584.009900990099 462985 -128 123 -3.712871287128713 -375 +550 2 10540 99451 1.65165 298.65165 150.15165 15015.16516 1.65165 298.65164 150.15165 15015.16521 1.65165 298.65165 150.15164999999985 15015.16500 2020-01-01 2020-01-02 2020-01-01 00:09:10 2020-01-02 03:37:31 2020-01-01 00:09:10.000 2020-01-02 03:37:31.000 550 99451 50000.5 5000050 550 99451 50000.5 5000050 -32620 32518 4780.66 478066 -126 125 -1.42 -142 +551 2 10541 99452 1.65465 298.65465 150.15465 15015.46546 1.65465 298.65466 150.15465 15015.46533 1.65465 298.65465 150.1546500000002 15015.46500 2020-01-01 2020-01-02 2020-01-01 00:09:11 2020-01-02 03:37:32 2020-01-01 00:09:11.000 2020-01-02 03:37:32.000 551 99452 50001.5 5000150 551 99452 50001.5 5000150 -32619 32519 4781.66 478166 -125 126 -0.42 -42 +552 2 10542 99453 1.65765 298.65765 150.15765 15015.76576 1.65765 298.65765 150.15765 15015.76562 1.65765 298.65765 150.15764999999985 15015.76500 2020-01-01 2020-01-02 2020-01-01 00:09:12 2020-01-02 03:37:33 2020-01-01 00:09:12.000 2020-01-02 03:37:33.000 552 99453 50002.5 5000250 552 99453 50002.5 5000250 -32618 32520 4782.66 478266 -124 127 0.58 58 +553 2 10543 99454 1.66066 298.66066 150.16066 15016.06606 1.66066 298.66068 150.16066 15016.06621 1.66066 298.66066 150.16065999999978 15016.06600 2020-01-01 2020-01-02 2020-01-01 00:09:13 2020-01-02 03:37:34 2020-01-01 00:09:13.000 2020-01-02 03:37:34.000 553 99454 50003.5 5000350 553 99454 50003.5 5000350 -32617 32521 4783.66 478366 -128 127 -0.98 -98 +554 2 10544 99455 1.66366 298.66366 150.16366 15016.36636 1.66366 298.66367 150.16366 15016.36651 1.66366 298.66366 150.1636600000001 15016.36600 2020-01-01 2020-01-02 2020-01-01 00:09:14 2020-01-02 03:37:35 2020-01-01 00:09:14.000 2020-01-02 03:37:35.000 554 99455 50004.5 5000450 554 99455 50004.5 5000450 -32616 32522 4784.66 478466 -128 127 -2.54 -254 +555 2 10545 99456 1.66666 298.66666 150.16666 15016.66666 1.66666 298.66666 150.16666 15016.66668 1.66666 298.66666 150.16666000000012 15016.66600 2020-01-01 2020-01-02 2020-01-01 00:09:15 2020-01-02 03:37:36 2020-01-01 00:09:15.000 2020-01-02 03:37:36.000 555 99456 50005.5 5000550 555 99456 50005.5 5000550 -32615 32523 4785.66 478566 -128 124 -4.1 -410 +556 2 10546 99457 1.66966 298.66966 150.16966 15016.96696 1.66966 298.66968 150.16966 15016.9668 1.66966 298.66966 150.1696599999998 15016.96600 2020-01-01 2020-01-02 2020-01-01 00:09:16 2020-01-02 03:37:37 2020-01-01 00:09:16.000 2020-01-02 03:37:37.000 556 99457 50006.5 5000650 556 99457 50006.5 5000650 -32614 32524 4786.66 478666 -127 125 -3.1 -310 +557 2 10547 99458 1.67267 298.67267 150.17267 15017.26726 1.67267 298.67267 150.17267 15017.26709 1.67267 298.67267 150.17266999999995 15017.26700 2020-01-01 2020-01-02 2020-01-01 00:09:17 2020-01-02 03:37:38 2020-01-01 00:09:17.000 2020-01-02 03:37:38.000 557 99458 50007.5 5000750 557 99458 50007.5 5000750 -32613 32525 4787.66 478766 -126 126 -2.1 -210 558 2 10548 99459 1.67567 298.67567 150.17567 15017.56756 1.67567 298.6757 150.17567 15017.56769 1.67567 298.67567 150.17567 15017.56700 2020-01-01 2020-01-02 2020-01-01 00:09:18 2020-01-02 03:37:39 2020-01-01 00:09:18.000 2020-01-02 03:37:39.000 558 99459 50008.5 5000850 558 99459 50008.5 5000850 -32612 32526 4788.66 478866 -125 127 -1.1 -110 -559 2 10549 99460 1.67867 298.67867 150.17867 15017.86786 1.67867 298.67868 150.17868 15017.86802 1.67867 298.67867 150.17867 15017.86700 2020-01-01 2020-01-02 2020-01-01 00:09:19 2020-01-02 03:37:40 2020-01-01 00:09:19.000 2020-01-02 03:37:40.000 559 99460 50009.5 5000950 559 99460 50009.5 5000950 -32611 32527 4789.66 478966 -128 127 -2.66 -266 -56 2 10046 99956 0.16816 300.16816 150.16816 15166.98498 0.16816 300.16818 150.16816 15166.98512 0.16816 300.16816 150.16816 15166.98416 2020-01-01 2020-01-02 2020-01-01 00:00:56 2020-01-02 03:45:56 2020-01-01 00:00:56.000 2020-01-02 03:45:56.000 56 99956 50006 5050606 56 99956 50006 5050606 -32513 32422 4585.009900990099 463086 -127 124 -2.712871287128713 -274 -560 2 10550 99461 1.68168 298.68168 150.18168 15018.16816 1.68168 298.68167 150.18168 15018.16815 1.68168 298.68168 150.18168 15018.16800 2020-01-01 2020-01-02 2020-01-01 00:09:20 2020-01-02 03:37:41 2020-01-01 00:09:20.000 2020-01-02 03:37:41.000 560 99461 50010.5 5001050 560 99461 50010.5 5001050 -32610 32528 4790.66 479066 -128 127 -4.22 -422 -561 2 10551 99462 1.68468 298.68468 150.18468 15018.46846 1.68468 298.6847 150.18468 15018.46826 1.68468 298.68468 150.18468 15018.46800 2020-01-01 2020-01-02 2020-01-01 00:09:21 2020-01-02 03:37:42 2020-01-01 00:09:21.000 2020-01-02 03:37:42.000 561 99462 50011.5 5001150 561 99462 50011.5 5001150 -32609 32529 4791.66 479166 -128 123 -5.78 -578 -562 2 10552 99463 1.68768 298.68768 150.18768 15018.76876 1.68768 298.68768 150.18768 15018.76856 1.68768 298.68768 150.18768 15018.76800 2020-01-01 2020-01-02 2020-01-01 00:09:22 2020-01-02 03:37:43 2020-01-01 00:09:22.000 2020-01-02 03:37:43.000 562 99463 50012.5 5001250 562 99463 50012.5 5001250 -32608 32530 4792.66 479266 -127 124 -4.78 -478 +559 2 10549 99460 1.67867 298.67867 150.17867 15017.86786 1.67867 298.67868 150.17868 15017.86802 1.67867 298.67867 150.17866999999995 15017.86700 2020-01-01 2020-01-02 2020-01-01 00:09:19 2020-01-02 03:37:40 2020-01-01 00:09:19.000 2020-01-02 03:37:40.000 559 99460 50009.5 5000950 559 99460 50009.5 5000950 -32611 32527 4789.66 478966 -128 127 -2.66 -266 +56 2 10046 99956 0.16816 300.16816 150.16816 15166.98498 0.16816 300.16818 150.16816 15166.98512 0.16816 300.16816 150.1681599999999 15166.98416 2020-01-01 2020-01-02 2020-01-01 00:00:56 2020-01-02 03:45:56 2020-01-01 00:00:56.000 2020-01-02 03:45:56.000 56 99956 50006 5050606 56 99956 50006 5050606 -32513 32422 4585.009900990099 463086 -127 124 -2.712871287128713 -274 +560 2 10550 99461 1.68168 298.68168 150.18168 15018.16816 1.68168 298.68167 150.18168 15018.16815 1.68168 298.68168 150.18167999999986 15018.16800 2020-01-01 2020-01-02 2020-01-01 00:09:20 2020-01-02 03:37:41 2020-01-01 00:09:20.000 2020-01-02 03:37:41.000 560 99461 50010.5 5001050 560 99461 50010.5 5001050 -32610 32528 4790.66 479066 -128 127 -4.22 -422 +561 2 10551 99462 1.68468 298.68468 150.18468 15018.46846 1.68468 298.6847 150.18468 15018.46826 1.68468 298.68468 150.18468000000013 15018.46800 2020-01-01 2020-01-02 2020-01-01 00:09:21 2020-01-02 03:37:42 2020-01-01 00:09:21.000 2020-01-02 03:37:42.000 561 99462 50011.5 5001150 561 99462 50011.5 5001150 -32609 32529 4791.66 479166 -128 123 -5.78 -578 +562 2 10552 99463 1.68768 298.68768 150.18768 15018.76876 1.68768 298.68768 150.18768 15018.76856 1.68768 298.68768 150.1876799999998 15018.76800 2020-01-01 2020-01-02 2020-01-01 00:09:22 2020-01-02 03:37:43 2020-01-01 00:09:22.000 2020-01-02 03:37:43.000 562 99463 50012.5 5001250 562 99463 50012.5 5001250 -32608 32530 4792.66 479266 -127 124 -4.78 -478 563 2 10553 99464 1.69069 298.69069 150.19069 15019.06906 1.69069 298.6907 150.19069 15019.06915 1.69069 298.69069 150.19069 15019.06900 2020-01-01 2020-01-02 2020-01-01 00:09:23 2020-01-02 03:37:44 2020-01-01 00:09:23.000 2020-01-02 03:37:44.000 563 99464 50013.5 5001350 563 99464 50013.5 5001350 -32607 32531 4793.66 479366 -126 125 -3.78 -378 -564 2 10554 99465 1.69369 298.69369 150.19369 15019.36936 1.69369 298.6937 150.19369 15019.36948 1.69369 298.69369 150.19369 15019.36900 2020-01-01 2020-01-02 2020-01-01 00:09:24 2020-01-02 03:37:45 2020-01-01 00:09:24.000 2020-01-02 03:37:45.000 564 99465 50014.5 5001450 564 99465 50014.5 5001450 -32606 32532 4794.66 479466 -125 126 -2.78 -278 -565 2 10555 99466 1.69669 298.69669 150.19669 15019.66966 1.69669 298.6967 150.19669 15019.66962 1.69669 298.69669 150.19669 15019.66900 2020-01-01 2020-01-02 2020-01-01 00:09:25 2020-01-02 03:37:46 2020-01-01 00:09:25.000 2020-01-02 03:37:46.000 565 99466 50015.5 5001550 565 99466 50015.5 5001550 -32605 32533 4795.66 479566 -124 127 -1.78 -178 +564 2 10554 99465 1.69369 298.69369 150.19369 15019.36936 1.69369 298.6937 150.19369 15019.36948 1.69369 298.69369 150.19369000000003 15019.36900 2020-01-01 2020-01-02 2020-01-01 00:09:24 2020-01-02 03:37:45 2020-01-01 00:09:24.000 2020-01-02 03:37:45.000 564 99465 50014.5 5001450 564 99465 50014.5 5001450 -32606 32532 4794.66 479466 -125 126 -2.78 -278 +565 2 10555 99466 1.69669 298.69669 150.19669 15019.66966 1.69669 298.6967 150.19669 15019.66962 1.69669 298.69669 150.1966900000001 15019.66900 2020-01-01 2020-01-02 2020-01-01 00:09:25 2020-01-02 03:37:46 2020-01-01 00:09:25.000 2020-01-02 03:37:46.000 565 99466 50015.5 5001550 565 99466 50015.5 5001550 -32605 32533 4795.66 479566 -124 127 -1.78 -178 566 2 10556 99467 1.69969 298.69969 150.19969 15019.96996 1.69969 298.6997 150.1997 15019.97037 1.69969 298.69969 150.19969 15019.96900 2020-01-01 2020-01-02 2020-01-01 00:09:26 2020-01-02 03:37:47 2020-01-01 00:09:26.000 2020-01-02 03:37:47.000 566 99467 50016.5 5001650 566 99467 50016.5 5001650 -32604 32534 4796.66 479666 -128 127 -3.34 -334 -567 2 10557 99468 1.7027 298.7027 150.2027 15020.27027 1.7027 298.7027 150.2027 15020.27003 1.70270 298.70270 150.20270 15020.27000 2020-01-01 2020-01-02 2020-01-01 00:09:27 2020-01-02 03:37:48 2020-01-01 00:09:27.000 2020-01-02 03:37:48.000 567 99468 50017.5 5001750 567 99468 50017.5 5001750 -32603 32535 4797.66 479766 -128 123 -4.9 -490 -568 2 10558 99469 1.7057 298.7057 150.2057 15020.57057 1.7057 298.70572 150.2057 15020.57066 1.70570 298.70570 150.20570 15020.57000 2020-01-01 2020-01-02 2020-01-01 00:09:28 2020-01-02 03:37:49 2020-01-01 00:09:28.000 2020-01-02 03:37:49.000 568 99469 50018.5 5001850 568 99469 50018.5 5001850 -32602 32536 4798.66 479866 -127 124 -3.9 -390 -569 2 10559 99470 1.7087 298.7087 150.2087 15020.87087 1.7087 298.7087 150.2087 15020.87095 1.70870 298.70870 150.20870 15020.87000 2020-01-01 2020-01-02 2020-01-01 00:09:29 2020-01-02 03:37:50 2020-01-01 00:09:29.000 2020-01-02 03:37:50.000 569 99470 50019.5 5001950 569 99470 50019.5 5001950 -32601 32537 4799.66 479966 -126 125 -2.9 -290 -57 2 10047 99957 0.17117 300.17117 150.17117 15167.28828 0.17117 300.17117 150.17117 15167.28841 0.17117 300.17117 150.17117 15167.28817 2020-01-01 2020-01-02 2020-01-01 00:00:57 2020-01-02 03:45:57 2020-01-01 00:00:57.000 2020-01-02 03:45:57.000 57 99957 50007 5050707 57 99957 50007 5050707 -32512 32423 4586.009900990099 463187 -126 125 -1.7128712871287128 -173 -570 2 10560 99471 1.71171 298.71171 150.21171 15021.17117 1.71171 298.7117 150.21171 15021.17109 1.71171 298.71171 150.21171 15021.17100 2020-01-01 2020-01-02 2020-01-01 00:09:30 2020-01-02 03:37:51 2020-01-01 00:09:30.000 2020-01-02 03:37:51.000 570 99471 50020.5 5002050 570 99471 50020.5 5002050 -32600 32538 4800.66 480066 -125 126 -1.9 -190 -571 2 10561 99472 1.71471 298.71471 150.21471 15021.47147 1.71471 298.71472 150.21471 15021.47184 1.71471 298.71471 150.21471 15021.47100 2020-01-01 2020-01-02 2020-01-01 00:09:31 2020-01-02 03:37:52 2020-01-01 00:09:31.000 2020-01-02 03:37:52.000 571 99472 50021.5 5002150 571 99472 50021.5 5002150 -32599 32539 4801.66 480166 -124 127 -0.9 -90 -572 2 10562 99473 1.71771 298.71771 150.21771 15021.77177 1.71771 298.7177 150.21771 15021.7715 1.71771 298.71771 150.21771 15021.77100 2020-01-01 2020-01-02 2020-01-01 00:09:32 2020-01-02 03:37:53 2020-01-01 00:09:32.000 2020-01-02 03:37:53.000 572 99473 50022.5 5002250 572 99473 50022.5 5002250 -32598 32540 4802.66 480266 -128 127 -2.46 -246 -573 2 10563 99474 1.72072 298.72072 150.22072 15022.07207 1.72072 298.72073 150.22072 15022.07212 1.72072 298.72072 150.22072 15022.07200 2020-01-01 2020-01-02 2020-01-01 00:09:33 2020-01-02 03:37:54 2020-01-01 00:09:33.000 2020-01-02 03:37:54.000 573 99474 50023.5 5002350 573 99474 50023.5 5002350 -32597 32541 4803.66 480366 -128 123 -4.02 -402 +567 2 10557 99468 1.7027 298.7027 150.2027 15020.27027 1.7027 298.7027 150.2027 15020.27003 1.70270 298.70270 150.20269999999988 15020.27000 2020-01-01 2020-01-02 2020-01-01 00:09:27 2020-01-02 03:37:48 2020-01-01 00:09:27.000 2020-01-02 03:37:48.000 567 99468 50017.5 5001750 567 99468 50017.5 5001750 -32603 32535 4797.66 479766 -128 123 -4.9 -490 +568 2 10558 99469 1.7057 298.7057 150.2057 15020.57057 1.7057 298.70572 150.2057 15020.57066 1.70570 298.70570 150.2057000000002 15020.57000 2020-01-01 2020-01-02 2020-01-01 00:09:28 2020-01-02 03:37:49 2020-01-01 00:09:28.000 2020-01-02 03:37:49.000 568 99469 50018.5 5001850 568 99469 50018.5 5001850 -32602 32536 4798.66 479866 -127 124 -3.9 -390 +569 2 10559 99470 1.7087 298.7087 150.2087 15020.87087 1.7087 298.7087 150.2087 15020.87095 1.70870 298.70870 150.20869999999988 15020.87000 2020-01-01 2020-01-02 2020-01-01 00:09:29 2020-01-02 03:37:50 2020-01-01 00:09:29.000 2020-01-02 03:37:50.000 569 99470 50019.5 5001950 569 99470 50019.5 5001950 -32601 32537 4799.66 479966 -126 125 -2.9 -290 +57 2 10047 99957 0.17117 300.17117 150.17117 15167.28828 0.17117 300.17117 150.17117 15167.28841 0.17117 300.17117 150.1711699999998 15167.28817 2020-01-01 2020-01-02 2020-01-01 00:00:57 2020-01-02 03:45:57 2020-01-01 00:00:57.000 2020-01-02 03:45:57.000 57 99957 50007 5050707 57 99957 50007 5050707 -32512 32423 4586.009900990099 463187 -126 125 -1.7128712871287128 -173 +570 2 10560 99471 1.71171 298.71171 150.21171 15021.17117 1.71171 298.7117 150.21171 15021.17109 1.71171 298.71171 150.2117099999998 15021.17100 2020-01-01 2020-01-02 2020-01-01 00:09:30 2020-01-02 03:37:51 2020-01-01 00:09:30.000 2020-01-02 03:37:51.000 570 99471 50020.5 5002050 570 99471 50020.5 5002050 -32600 32538 4800.66 480066 -125 126 -1.9 -190 +571 2 10561 99472 1.71471 298.71471 150.21471 15021.47147 1.71471 298.71472 150.21471 15021.47184 1.71471 298.71471 150.21471000000014 15021.47100 2020-01-01 2020-01-02 2020-01-01 00:09:31 2020-01-02 03:37:52 2020-01-01 00:09:31.000 2020-01-02 03:37:52.000 571 99472 50021.5 5002150 571 99472 50021.5 5002150 -32599 32539 4801.66 480166 -124 127 -0.9 -90 +572 2 10562 99473 1.71771 298.71771 150.21771 15021.77177 1.71771 298.7177 150.21771 15021.7715 1.71771 298.71771 150.21771000000012 15021.77100 2020-01-01 2020-01-02 2020-01-01 00:09:32 2020-01-02 03:37:53 2020-01-01 00:09:32.000 2020-01-02 03:37:53.000 572 99473 50022.5 5002250 572 99473 50022.5 5002250 -32598 32540 4802.66 480266 -128 127 -2.46 -246 +573 2 10563 99474 1.72072 298.72072 150.22072 15022.07207 1.72072 298.72073 150.22072 15022.07212 1.72072 298.72072 150.2207199999999 15022.07200 2020-01-01 2020-01-02 2020-01-01 00:09:33 2020-01-02 03:37:54 2020-01-01 00:09:33.000 2020-01-02 03:37:54.000 573 99474 50023.5 5002350 573 99474 50023.5 5002350 -32597 32541 4803.66 480366 -128 123 -4.02 -402 574 2 10564 99475 1.72372 298.72372 150.22372 15022.37237 1.72372 298.72372 150.22372 15022.37243 1.72372 298.72372 150.22372 15022.37200 2020-01-01 2020-01-02 2020-01-01 00:09:34 2020-01-02 03:37:55 2020-01-01 00:09:34.000 2020-01-02 03:37:55.000 574 99475 50024.5 5002450 574 99475 50024.5 5002450 -32596 32542 4804.66 480466 -127 124 -3.02 -302 -575 2 10565 99476 1.72672 298.72672 150.22672 15022.67267 1.72672 298.7267 150.22672 15022.67272 1.72672 298.72672 150.22672 15022.67200 2020-01-01 2020-01-02 2020-01-01 00:09:35 2020-01-02 03:37:56 2020-01-01 00:09:35.000 2020-01-02 03:37:56.000 575 99476 50025.5 5002550 575 99476 50025.5 5002550 -32595 32543 4805.66 480566 -126 125 -2.02 -202 -576 2 10566 99477 1.72972 298.72972 150.22972 15022.97297 1.72972 298.72974 150.22973 15022.97332 1.72972 298.72972 150.22972 15022.97200 2020-01-01 2020-01-02 2020-01-01 00:09:36 2020-01-02 03:37:57 2020-01-01 00:09:36.000 2020-01-02 03:37:57.000 576 99477 50026.5 5002650 576 99477 50026.5 5002650 -32594 32544 4806.66 480666 -125 126 -1.02 -102 -577 2 10567 99478 1.73273 298.73273 150.23273 15023.27327 1.73273 298.73273 150.23272 15023.27297 1.73273 298.73273 150.23273 15023.27300 2020-01-01 2020-01-02 2020-01-01 00:09:37 2020-01-02 03:37:58 2020-01-01 00:09:37.000 2020-01-02 03:37:58.000 577 99478 50027.5 5002750 577 99478 50027.5 5002750 -32593 32545 4807.66 480766 -124 127 -0.02 -2 -578 2 10568 99479 1.73573 298.73573 150.23573 15023.57357 1.73573 298.73575 150.23573 15023.57359 1.73573 298.73573 150.23573 15023.57300 2020-01-01 2020-01-02 2020-01-01 00:09:38 2020-01-02 03:37:59 2020-01-01 00:09:38.000 2020-01-02 03:37:59.000 578 99479 50028.5 5002850 578 99479 50028.5 5002850 -32592 32546 4808.66 480866 -128 127 -1.58 -158 -579 2 10569 99480 1.73873 298.73873 150.23873 15023.87387 1.73873 298.73874 150.23873 15023.8739 1.73873 298.73873 150.23873 15023.87300 2020-01-01 2020-01-02 2020-01-01 00:09:39 2020-01-02 03:38:00 2020-01-01 00:09:39.000 2020-01-02 03:38:00.000 579 99480 50029.5 5002950 579 99480 50029.5 5002950 -32591 32547 4809.66 480966 -128 123 -3.14 -314 -58 2 10048 99958 0.17417 300.17417 150.17417 15167.59159 0.17417 300.17416 150.17417 15167.59159 0.17417 300.17417 150.17417 15167.59117 2020-01-01 2020-01-02 2020-01-01 00:00:58 2020-01-02 03:45:58 2020-01-01 00:00:58.000 2020-01-02 03:45:58.000 58 99958 50008 5050808 58 99958 50008 5050808 -32511 32424 4587.009900990099 463288 -125 126 -0.7128712871287128 -72 -580 2 10570 99481 1.74174 298.74174 150.24174 15024.17417 1.74174 298.74173 150.24174 15024.17419 1.74174 298.74174 150.24174 15024.17400 2020-01-01 2020-01-02 2020-01-01 00:09:40 2020-01-02 03:38:01 2020-01-01 00:09:40.000 2020-01-02 03:38:01.000 580 99481 50030.5 5003050 580 99481 50030.5 5003050 -32590 32548 4810.66 481066 -127 124 -2.14 -214 -581 2 10571 99482 1.74474 298.74474 150.24474 15024.47447 1.74474 298.74475 150.24474 15024.47478 1.74474 298.74474 150.24474 15024.47400 2020-01-01 2020-01-02 2020-01-01 00:09:41 2020-01-02 03:38:02 2020-01-01 00:09:41.000 2020-01-02 03:38:02.000 581 99482 50031.5 5003150 581 99482 50031.5 5003150 -32589 32549 4811.66 481166 -126 125 -1.14 -114 -582 2 10572 99483 1.74774 298.74774 150.24774 15024.77477 1.74774 298.74774 150.24774 15024.77447 1.74774 298.74774 150.24774 15024.77400 2020-01-01 2020-01-02 2020-01-01 00:09:42 2020-01-02 03:38:03 2020-01-01 00:09:42.000 2020-01-02 03:38:03.000 582 99483 50032.5 5003250 582 99483 50032.5 5003250 -32588 32550 4812.66 481266 -125 126 -0.14 -14 -583 2 10573 99484 1.75075 298.75075 150.25075 15025.07507 1.75075 298.75076 150.25075 15025.07507 1.75075 298.75075 150.25075 15025.07500 2020-01-01 2020-01-02 2020-01-01 00:09:43 2020-01-02 03:38:04 2020-01-01 00:09:43.000 2020-01-02 03:38:04.000 583 99484 50033.5 5003350 583 99484 50033.5 5003350 -32587 32551 4813.66 481366 -124 127 0.86 86 -584 2 10574 99485 1.75375 298.75375 150.25375 15025.37537 1.75375 298.75375 150.25375 15025.37536 1.75375 298.75375 150.25375 15025.37500 2020-01-01 2020-01-02 2020-01-01 00:09:44 2020-01-02 03:38:05 2020-01-01 00:09:44.000 2020-01-02 03:38:05.000 584 99485 50034.5 5003450 584 99485 50034.5 5003450 -32586 32552 4814.66 481466 -128 127 -0.7 -70 -585 2 10575 99486 1.75675 298.75675 150.25675 15025.67567 1.75675 298.75674 150.25675 15025.67566 1.75675 298.75675 150.25675 15025.67500 2020-01-01 2020-01-02 2020-01-01 00:09:45 2020-01-02 03:38:06 2020-01-01 00:09:45.000 2020-01-02 03:38:06.000 585 99486 50035.5 5003550 585 99486 50035.5 5003550 -32585 32553 4815.66 481566 -128 127 -2.26 -226 -586 2 10576 99487 1.75975 298.75975 150.25975 15025.97597 1.75975 298.75977 150.25976 15025.97625 1.75975 298.75975 150.25975 15025.97500 2020-01-01 2020-01-02 2020-01-01 00:09:46 2020-01-02 03:38:07 2020-01-01 00:09:46.000 2020-01-02 03:38:07.000 586 99487 50036.5 5003650 586 99487 50036.5 5003650 -32584 32554 4816.66 481666 -128 123 -3.82 -382 -587 2 10577 99488 1.76276 298.76276 150.26276 15026.27627 1.76276 298.76276 150.26275 15026.27594 1.76276 298.76276 150.26276 15026.27600 2020-01-01 2020-01-02 2020-01-01 00:09:47 2020-01-02 03:38:08 2020-01-01 00:09:47.000 2020-01-02 03:38:08.000 587 99488 50037.5 5003750 587 99488 50037.5 5003750 -32583 32555 4817.66 481766 -127 124 -2.82 -282 -588 2 10578 99489 1.76576 298.76576 150.26576 15026.57657 1.76576 298.76578 150.26576 15026.57654 1.76576 298.76576 150.26576 15026.57600 2020-01-01 2020-01-02 2020-01-01 00:09:48 2020-01-02 03:38:09 2020-01-01 00:09:48.000 2020-01-02 03:38:09.000 588 99489 50038.5 5003850 588 99489 50038.5 5003850 -32582 32556 4818.66 481866 -126 125 -1.82 -182 -589 2 10579 99490 1.76876 298.76876 150.26876 15026.87687 1.76876 298.76877 150.26876 15026.87683 1.76876 298.76876 150.26876 15026.87600 2020-01-01 2020-01-02 2020-01-01 00:09:49 2020-01-02 03:38:10 2020-01-01 00:09:49.000 2020-01-02 03:38:10.000 589 99490 50039.5 5003950 589 99490 50039.5 5003950 -32581 32557 4819.66 481966 -125 126 -0.82 -82 -59 2 10049 99959 0.17717 300.17717 150.17717 15167.89489 0.17717 300.1772 150.17717 15167.8947 0.17717 300.17717 150.17717 15167.89417 2020-01-01 2020-01-02 2020-01-01 00:00:59 2020-01-02 03:45:59 2020-01-01 00:00:59.000 2020-01-02 03:45:59.000 59 99959 50009 5050909 59 99959 50009 5050909 -32510 32425 4588.009900990099 463389 -124 127 0.2871287128712871 29 -590 2 10580 99491 1.77177 298.77177 150.27177 15027.17717 1.77177 298.77176 150.27177 15027.17713 1.77177 298.77177 150.27177 15027.17700 2020-01-01 2020-01-02 2020-01-01 00:09:50 2020-01-02 03:38:11 2020-01-01 00:09:50.000 2020-01-02 03:38:11.000 590 99491 50040.5 5004050 590 99491 50040.5 5004050 -32580 32558 4820.66 482066 -124 127 0.18 18 +575 2 10565 99476 1.72672 298.72672 150.22672 15022.67267 1.72672 298.7267 150.22672 15022.67272 1.72672 298.72672 150.22672000000006 15022.67200 2020-01-01 2020-01-02 2020-01-01 00:09:35 2020-01-02 03:37:56 2020-01-01 00:09:35.000 2020-01-02 03:37:56.000 575 99476 50025.5 5002550 575 99476 50025.5 5002550 -32595 32543 4805.66 480566 -126 125 -2.02 -202 +576 2 10566 99477 1.72972 298.72972 150.22972 15022.97297 1.72972 298.72974 150.22973 15022.97332 1.72972 298.72972 150.22971999999993 15022.97200 2020-01-01 2020-01-02 2020-01-01 00:09:36 2020-01-02 03:37:57 2020-01-01 00:09:36.000 2020-01-02 03:37:57.000 576 99477 50026.5 5002650 576 99477 50026.5 5002650 -32594 32544 4806.66 480666 -125 126 -1.02 -102 +577 2 10567 99478 1.73273 298.73273 150.23273 15023.27327 1.73273 298.73273 150.23272 15023.27297 1.73273 298.73273 150.2327299999999 15023.27300 2020-01-01 2020-01-02 2020-01-01 00:09:37 2020-01-02 03:37:58 2020-01-01 00:09:37.000 2020-01-02 03:37:58.000 577 99478 50027.5 5002750 577 99478 50027.5 5002750 -32593 32545 4807.66 480766 -124 127 -0.02 -2 +578 2 10568 99479 1.73573 298.73573 150.23573 15023.57357 1.73573 298.73575 150.23573 15023.57359 1.73573 298.73573 150.23573000000016 15023.57300 2020-01-01 2020-01-02 2020-01-01 00:09:38 2020-01-02 03:37:59 2020-01-01 00:09:38.000 2020-01-02 03:37:59.000 578 99479 50028.5 5002850 578 99479 50028.5 5002850 -32592 32546 4808.66 480866 -128 127 -1.58 -158 +579 2 10569 99480 1.73873 298.73873 150.23873 15023.87387 1.73873 298.73874 150.23873 15023.8739 1.73873 298.73873 150.23872999999983 15023.87300 2020-01-01 2020-01-02 2020-01-01 00:09:39 2020-01-02 03:38:00 2020-01-01 00:09:39.000 2020-01-02 03:38:00.000 579 99480 50029.5 5002950 579 99480 50029.5 5002950 -32591 32547 4809.66 480966 -128 123 -3.14 -314 +58 2 10048 99958 0.17417 300.17417 150.17417 15167.59159 0.17417 300.17416 150.17417 15167.59159 0.17417 300.17417 150.17417000000012 15167.59117 2020-01-01 2020-01-02 2020-01-01 00:00:58 2020-01-02 03:45:58 2020-01-01 00:00:58.000 2020-01-02 03:45:58.000 58 99958 50008 5050808 58 99958 50008 5050808 -32511 32424 4587.009900990099 463288 -125 126 -0.7128712871287128 -72 +580 2 10570 99481 1.74174 298.74174 150.24174 15024.17417 1.74174 298.74173 150.24174 15024.17419 1.74174 298.74174 150.24174000000002 15024.17400 2020-01-01 2020-01-02 2020-01-01 00:09:40 2020-01-02 03:38:01 2020-01-01 00:09:40.000 2020-01-02 03:38:01.000 580 99481 50030.5 5003050 580 99481 50030.5 5003050 -32590 32548 4810.66 481066 -127 124 -2.14 -214 +581 2 10571 99482 1.74474 298.74474 150.24474 15024.47447 1.74474 298.74475 150.24474 15024.47478 1.74474 298.74474 150.24474000000006 15024.47400 2020-01-01 2020-01-02 2020-01-01 00:09:41 2020-01-02 03:38:02 2020-01-01 00:09:41.000 2020-01-02 03:38:02.000 581 99482 50031.5 5003150 581 99482 50031.5 5003150 -32589 32549 4811.66 481166 -126 125 -1.14 -114 +582 2 10572 99483 1.74774 298.74774 150.24774 15024.77477 1.74774 298.74774 150.24774 15024.77447 1.74774 298.74774 150.24774000000005 15024.77400 2020-01-01 2020-01-02 2020-01-01 00:09:42 2020-01-02 03:38:03 2020-01-01 00:09:42.000 2020-01-02 03:38:03.000 582 99483 50032.5 5003250 582 99483 50032.5 5003250 -32588 32550 4812.66 481266 -125 126 -0.14 -14 +583 2 10573 99484 1.75075 298.75075 150.25075 15025.07507 1.75075 298.75076 150.25075 15025.07507 1.75075 298.75075 150.25074999999993 15025.07500 2020-01-01 2020-01-02 2020-01-01 00:09:43 2020-01-02 03:38:04 2020-01-01 00:09:43.000 2020-01-02 03:38:04.000 583 99484 50033.5 5003350 583 99484 50033.5 5003350 -32587 32551 4813.66 481366 -124 127 0.86 86 +584 2 10574 99485 1.75375 298.75375 150.25375 15025.37537 1.75375 298.75375 150.25375 15025.37536 1.75375 298.75375 150.2537499999999 15025.37500 2020-01-01 2020-01-02 2020-01-01 00:09:44 2020-01-02 03:38:05 2020-01-01 00:09:44.000 2020-01-02 03:38:05.000 584 99485 50034.5 5003450 584 99485 50034.5 5003450 -32586 32552 4814.66 481466 -128 127 -0.7 -70 +585 2 10575 99486 1.75675 298.75675 150.25675 15025.67567 1.75675 298.75674 150.25675 15025.67566 1.75675 298.75675 150.2567500000001 15025.67500 2020-01-01 2020-01-02 2020-01-01 00:09:45 2020-01-02 03:38:06 2020-01-01 00:09:45.000 2020-01-02 03:38:06.000 585 99486 50035.5 5003550 585 99486 50035.5 5003550 -32585 32553 4815.66 481566 -128 127 -2.26 -226 +586 2 10576 99487 1.75975 298.75975 150.25975 15025.97597 1.75975 298.75977 150.25976 15025.97625 1.75975 298.75975 150.25974999999988 15025.97500 2020-01-01 2020-01-02 2020-01-01 00:09:46 2020-01-02 03:38:07 2020-01-01 00:09:46.000 2020-01-02 03:38:07.000 586 99487 50036.5 5003650 586 99487 50036.5 5003650 -32584 32554 4816.66 481666 -128 123 -3.82 -382 +587 2 10577 99488 1.76276 298.76276 150.26276 15026.27627 1.76276 298.76276 150.26275 15026.27594 1.76276 298.76276 150.26275999999982 15026.27600 2020-01-01 2020-01-02 2020-01-01 00:09:47 2020-01-02 03:38:08 2020-01-01 00:09:47.000 2020-01-02 03:38:08.000 587 99488 50037.5 5003750 587 99488 50037.5 5003750 -32583 32555 4817.66 481766 -127 124 -2.82 -282 +588 2 10578 99489 1.76576 298.76576 150.26576 15026.57657 1.76576 298.76578 150.26576 15026.57654 1.76576 298.76576 150.26576000000014 15026.57600 2020-01-01 2020-01-02 2020-01-01 00:09:48 2020-01-02 03:38:09 2020-01-01 00:09:48.000 2020-01-02 03:38:09.000 588 99489 50038.5 5003850 588 99489 50038.5 5003850 -32582 32556 4818.66 481866 -126 125 -1.82 -182 +589 2 10579 99490 1.76876 298.76876 150.26876 15026.87687 1.76876 298.76877 150.26876 15026.87683 1.76876 298.76876 150.26876000000013 15026.87600 2020-01-01 2020-01-02 2020-01-01 00:09:49 2020-01-02 03:38:10 2020-01-01 00:09:49.000 2020-01-02 03:38:10.000 589 99490 50039.5 5003950 589 99490 50039.5 5003950 -32581 32557 4819.66 481966 -125 126 -0.82 -82 +59 2 10049 99959 0.17717 300.17717 150.17717 15167.89489 0.17717 300.1772 150.17717 15167.8947 0.17717 300.17717 150.17717000000013 15167.89417 2020-01-01 2020-01-02 2020-01-01 00:00:59 2020-01-02 03:45:59 2020-01-01 00:00:59.000 2020-01-02 03:45:59.000 59 99959 50009 5050909 59 99959 50009 5050909 -32510 32425 4588.009900990099 463389 -124 127 0.2871287128712871 29 +590 2 10580 99491 1.77177 298.77177 150.27177 15027.17717 1.77177 298.77176 150.27177 15027.17713 1.77177 298.77177 150.27176999999992 15027.17700 2020-01-01 2020-01-02 2020-01-01 00:09:50 2020-01-02 03:38:11 2020-01-01 00:09:50.000 2020-01-02 03:38:11.000 590 99491 50040.5 5004050 590 99491 50040.5 5004050 -32580 32558 4820.66 482066 -124 127 0.18 18 591 2 10581 99492 1.77477 298.77477 150.27477 15027.47747 1.77477 298.77478 150.27477 15027.47775 1.77477 298.77477 150.27477 15027.47700 2020-01-01 2020-01-02 2020-01-01 00:09:51 2020-01-02 03:38:12 2020-01-01 00:09:51.000 2020-01-02 03:38:12.000 591 99492 50041.5 5004150 591 99492 50041.5 5004150 -32579 32559 4821.66 482166 -128 127 -1.38 -138 -592 2 10582 99493 1.77777 298.77777 150.27777 15027.77777 1.77777 298.77777 150.27777 15027.77742 1.77777 298.77777 150.27777 15027.77700 2020-01-01 2020-01-02 2020-01-01 00:09:52 2020-01-02 03:38:13 2020-01-01 00:09:52.000 2020-01-02 03:38:13.000 592 99493 50042.5 5004250 592 99493 50042.5 5004250 -32578 32560 4822.66 482266 -128 123 -2.94 -294 -593 2 10583 99494 1.78078 298.78078 150.28078 15028.07807 1.78078 298.7808 150.28078 15028.078 1.78078 298.78078 150.28078 15028.07800 2020-01-01 2020-01-02 2020-01-01 00:09:53 2020-01-02 03:38:14 2020-01-01 00:09:53.000 2020-01-02 03:38:14.000 593 99494 50043.5 5004350 593 99494 50043.5 5004350 -32577 32561 4823.66 482366 -127 124 -1.94 -194 -594 2 10584 99495 1.78378 298.78378 150.28378 15028.37837 1.78378 298.78378 150.28378 15028.3783 1.78378 298.78378 150.28378 15028.37800 2020-01-01 2020-01-02 2020-01-01 00:09:54 2020-01-02 03:38:15 2020-01-01 00:09:54.000 2020-01-02 03:38:15.000 594 99495 50044.5 5004450 594 99495 50044.5 5004450 -32576 32562 4824.66 482466 -126 125 -0.94 -94 -595 2 10585 99496 1.78678 298.78678 150.28678 15028.67867 1.78678 298.78677 150.28678 15028.6786 1.78678 298.78678 150.28678 15028.67800 2020-01-01 2020-01-02 2020-01-01 00:09:55 2020-01-02 03:38:16 2020-01-01 00:09:55.000 2020-01-02 03:38:16.000 595 99496 50045.5 5004550 595 99496 50045.5 5004550 -32575 32563 4825.66 482566 -125 126 0.06 6 -596 2 10586 99497 1.78978 298.78978 150.28978 15028.97897 1.78978 298.7898 150.28979 15028.97922 1.78978 298.78978 150.28978 15028.97800 2020-01-01 2020-01-02 2020-01-01 00:09:56 2020-01-02 03:38:17 2020-01-01 00:09:56.000 2020-01-02 03:38:17.000 596 99497 50046.5 5004650 596 99497 50046.5 5004650 -32574 32564 4826.66 482666 -124 127 1.06 106 -597 2 10587 99498 1.79279 298.79279 150.29279 15029.27927 1.79279 298.7928 150.29278 15029.27888 1.79279 298.79279 150.29279 15029.27900 2020-01-01 2020-01-02 2020-01-01 00:09:57 2020-01-02 03:38:18 2020-01-01 00:09:57.000 2020-01-02 03:38:18.000 597 99498 50047.5 5004750 597 99498 50047.5 5004750 -32573 32565 4827.66 482766 -128 127 -0.5 -50 -598 2 10588 99499 1.79579 298.79579 150.29579 15029.57957 1.79579 298.7958 150.29579 15029.57964 1.79579 298.79579 150.29579 15029.57900 2020-01-01 2020-01-02 2020-01-01 00:09:58 2020-01-02 03:38:19 2020-01-01 00:09:58.000 2020-01-02 03:38:19.000 598 99499 50048.5 5004850 598 99499 50048.5 5004850 -32572 32566 4828.66 482866 -128 123 -2.06 -206 -599 2 10589 99500 1.79879 298.79879 150.29879 15029.87987 1.79879 298.7988 150.29879 15029.87977 1.79879 298.79879 150.29879 15029.87900 2020-01-01 2020-01-02 2020-01-01 00:09:59 2020-01-02 03:38:20 2020-01-01 00:09:59.000 2020-01-02 03:38:20.000 599 99500 50049.5 5004950 599 99500 50049.5 5004950 -32571 32567 4829.66 482966 -127 124 -1.06 -106 -6 2 1005 9996 0.01801 300.01801 150.01801 15151.81981 0.01801 300.018 150.01801 15151.81978 0.01801 300.01801 150.01801 15151.81901 2020-01-01 2020-01-02 2020-01-01 00:00:06 2020-01-02 03:45:06 2020-01-01 00:00:06.000 2020-01-02 03:45:06.000 6 99906 49956 5045556 6 99906 49956 5045556 -32563 32372 4535.009900990099 458036 -127 124 -2.01980198019802 -204 -60 2 10050 99960 0.18018 300.18018 150.18018 15168.19819 0.18018 300.18018 150.18017 15168.198 0.18018 300.18018 150.18018 15168.19818 2020-01-01 2020-01-02 2020-01-01 00:01:00 2020-01-02 03:46:00 2020-01-01 00:01:00.000 2020-01-02 03:46:00.000 60 99960 50010 5051010 60 99960 50010 5051010 -32509 32426 4589.009900990099 463490 -128 127 -1.2475247524752475 -126 -600 2 10590 99501 1.8018 298.8018 150.3018 15030.18018 1.8018 298.8018 150.3018 15030.1801 1.80180 298.80180 150.30180 15030.18000 2020-01-01 2020-01-02 2020-01-01 00:10:00 2020-01-02 03:38:21 2020-01-01 00:10:00.000 2020-01-02 03:38:21.000 600 99501 50050.5 5005050 600 99501 50050.5 5005050 -32570 32568 4830.66 483066 -126 125 -0.06 -6 -601 2 10591 99502 1.8048 298.8048 150.3048 15030.48048 1.8048 298.8048 150.3048 15030.4807 1.80480 298.80480 150.30480 15030.48000 2020-01-01 2020-01-02 2020-01-01 00:10:01 2020-01-02 03:38:22 2020-01-01 00:10:01.000 2020-01-02 03:38:22.000 601 99502 50051.5 5005150 601 99502 50051.5 5005150 -32569 32569 4831.66 483166 -125 126 0.94 94 -602 2 10592 99503 1.8078 298.8078 150.3078 15030.78078 1.8078 298.8078 150.3078 15030.78035 1.80780 298.80780 150.30780 15030.78000 2020-01-01 2020-01-02 2020-01-01 00:10:02 2020-01-02 03:38:23 2020-01-01 00:10:02.000 2020-01-02 03:38:23.000 602 99503 50052.5 5005250 602 99503 50052.5 5005250 -32568 32570 4832.66 483266 -124 127 1.94 194 -603 2 10593 99504 1.81081 298.81081 150.31081 15031.08108 1.81081 298.81082 150.31081 15031.0811 1.81081 298.81081 150.31081 15031.08100 2020-01-01 2020-01-02 2020-01-01 00:10:03 2020-01-02 03:38:24 2020-01-01 00:10:03.000 2020-01-02 03:38:24.000 603 99504 50053.5 5005350 603 99504 50053.5 5005350 -32567 32571 4833.66 483366 -128 127 0.38 38 -604 2 10594 99505 1.81381 298.81381 150.31381 15031.38138 1.81381 298.8138 150.31381 15031.38124 1.81381 298.81381 150.31381 15031.38100 2020-01-01 2020-01-02 2020-01-01 00:10:04 2020-01-02 03:38:25 2020-01-01 00:10:04.000 2020-01-02 03:38:25.000 604 99505 50054.5 5005450 604 99505 50054.5 5005450 -32566 32572 4834.66 483466 -128 123 -1.18 -118 -605 2 10595 99506 1.81681 298.81681 150.31681 15031.68168 1.81681 298.8168 150.31681 15031.68157 1.81681 298.81681 150.31681 15031.68100 2020-01-01 2020-01-02 2020-01-01 00:10:05 2020-01-02 03:38:26 2020-01-01 00:10:05.000 2020-01-02 03:38:26.000 605 99506 50055.5 5005550 605 99506 50055.5 5005550 -32565 32573 4835.66 483566 -127 124 -0.18 -18 -606 2 10596 99507 1.81981 298.81981 150.31981 15031.98198 1.81981 298.81982 150.31982 15031.98217 1.81981 298.81981 150.31981 15031.98100 2020-01-01 2020-01-02 2020-01-01 00:10:06 2020-01-02 03:38:27 2020-01-01 00:10:06.000 2020-01-02 03:38:27.000 606 99507 50056.5 5005650 606 99507 50056.5 5005650 -32564 32574 4836.66 483666 -126 125 0.82 82 -607 2 10597 99508 1.82282 298.82282 150.32282 15032.28228 1.82282 298.8228 150.32282 15032.28246 1.82282 298.82282 150.32282 15032.28200 2020-01-01 2020-01-02 2020-01-01 00:10:07 2020-01-02 03:38:28 2020-01-01 00:10:07.000 2020-01-02 03:38:28.000 607 99508 50057.5 5005750 607 99508 50057.5 5005750 -32563 32575 4837.66 483766 -125 126 1.82 182 -608 2 10598 99509 1.82582 298.82582 150.32582 15032.58258 1.82582 298.82584 150.32582 15032.58258 1.82582 298.82582 150.32582 15032.58200 2020-01-01 2020-01-02 2020-01-01 00:10:08 2020-01-02 03:38:29 2020-01-01 00:10:08.000 2020-01-02 03:38:29.000 608 99509 50058.5 5005850 608 99509 50058.5 5005850 -32562 32576 4838.66 483866 -124 127 2.82 282 -609 2 10599 99510 1.82882 298.82882 150.32882 15032.88288 1.82882 298.82883 150.32882 15032.88274 1.82882 298.82882 150.32882 15032.88200 2020-01-01 2020-01-02 2020-01-01 00:10:09 2020-01-02 03:38:30 2020-01-01 00:10:09.000 2020-01-02 03:38:30.000 609 99510 50059.5 5005950 609 99510 50059.5 5005950 -32561 32577 4839.66 483966 -128 127 1.26 126 +592 2 10582 99493 1.77777 298.77777 150.27777 15027.77777 1.77777 298.77777 150.27777 15027.77742 1.77777 298.77777 150.27777000000006 15027.77700 2020-01-01 2020-01-02 2020-01-01 00:09:52 2020-01-02 03:38:13 2020-01-01 00:09:52.000 2020-01-02 03:38:13.000 592 99493 50042.5 5004250 592 99493 50042.5 5004250 -32578 32560 4822.66 482266 -128 123 -2.94 -294 +593 2 10583 99494 1.78078 298.78078 150.28078 15028.07807 1.78078 298.7808 150.28078 15028.078 1.78078 298.78078 150.28077999999985 15028.07800 2020-01-01 2020-01-02 2020-01-01 00:09:53 2020-01-02 03:38:14 2020-01-01 00:09:53.000 2020-01-02 03:38:14.000 593 99494 50043.5 5004350 593 99494 50043.5 5004350 -32577 32561 4823.66 482366 -127 124 -1.94 -194 +594 2 10584 99495 1.78378 298.78378 150.28378 15028.37837 1.78378 298.78378 150.28378 15028.3783 1.78378 298.78378 150.2837799999999 15028.37800 2020-01-01 2020-01-02 2020-01-01 00:09:54 2020-01-02 03:38:15 2020-01-01 00:09:54.000 2020-01-02 03:38:15.000 594 99495 50044.5 5004450 594 99495 50044.5 5004450 -32576 32562 4824.66 482466 -126 125 -0.94 -94 +595 2 10585 99496 1.78678 298.78678 150.28678 15028.67867 1.78678 298.78677 150.28678 15028.6786 1.78678 298.78678 150.28678000000016 15028.67800 2020-01-01 2020-01-02 2020-01-01 00:09:55 2020-01-02 03:38:16 2020-01-01 00:09:55.000 2020-01-02 03:38:16.000 595 99496 50045.5 5004550 595 99496 50045.5 5004550 -32575 32563 4825.66 482566 -125 126 0.06 6 +596 2 10586 99497 1.78978 298.78978 150.28978 15028.97897 1.78978 298.7898 150.28979 15028.97922 1.78978 298.78978 150.28977999999984 15028.97800 2020-01-01 2020-01-02 2020-01-01 00:09:56 2020-01-02 03:38:17 2020-01-01 00:09:56.000 2020-01-02 03:38:17.000 596 99497 50046.5 5004650 596 99497 50046.5 5004650 -32574 32564 4826.66 482666 -124 127 1.06 106 +597 2 10587 99498 1.79279 298.79279 150.29279 15029.27927 1.79279 298.7928 150.29278 15029.27888 1.79279 298.79279 150.29279000000005 15029.27900 2020-01-01 2020-01-02 2020-01-01 00:09:57 2020-01-02 03:38:18 2020-01-01 00:09:57.000 2020-01-02 03:38:18.000 597 99498 50047.5 5004750 597 99498 50047.5 5004750 -32573 32565 4827.66 482766 -128 127 -0.5 -50 +598 2 10588 99499 1.79579 298.79579 150.29579 15029.57957 1.79579 298.7958 150.29579 15029.57964 1.79579 298.79579 150.2957900000001 15029.57900 2020-01-01 2020-01-02 2020-01-01 00:09:58 2020-01-02 03:38:19 2020-01-01 00:09:58.000 2020-01-02 03:38:19.000 598 99499 50048.5 5004850 598 99499 50048.5 5004850 -32572 32566 4828.66 482866 -128 123 -2.06 -206 +599 2 10589 99500 1.79879 298.79879 150.29879 15029.87987 1.79879 298.7988 150.29879 15029.87977 1.79879 298.79879 150.2987900000001 15029.87900 2020-01-01 2020-01-02 2020-01-01 00:09:59 2020-01-02 03:38:20 2020-01-01 00:09:59.000 2020-01-02 03:38:20.000 599 99500 50049.5 5004950 599 99500 50049.5 5004950 -32571 32567 4829.66 482966 -127 124 -1.06 -106 +6 2 1005 9996 0.01801 300.01801 150.01801 15151.81981 0.01801 300.018 150.01801 15151.81978 0.01801 300.01801 150.01800999999986 15151.81901 2020-01-01 2020-01-02 2020-01-01 00:00:06 2020-01-02 03:45:06 2020-01-01 00:00:06.000 2020-01-02 03:45:06.000 6 99906 49956 5045556 6 99906 49956 5045556 -32563 32372 4535.009900990099 458036 -127 124 -2.01980198019802 -204 +60 2 10050 99960 0.18018 300.18018 150.18018 15168.19819 0.18018 300.18018 150.18017 15168.198 0.18018 300.18018 150.1801799999999 15168.19818 2020-01-01 2020-01-02 2020-01-01 00:01:00 2020-01-02 03:46:00 2020-01-01 00:01:00.000 2020-01-02 03:46:00.000 60 99960 50010 5051010 60 99960 50010 5051010 -32509 32426 4589.009900990099 463490 -128 127 -1.2475247524752475 -126 +600 2 10590 99501 1.8018 298.8018 150.3018 15030.18018 1.8018 298.8018 150.3018 15030.1801 1.80180 298.80180 150.3017999999999 15030.18000 2020-01-01 2020-01-02 2020-01-01 00:10:00 2020-01-02 03:38:21 2020-01-01 00:10:00.000 2020-01-02 03:38:21.000 600 99501 50050.5 5005050 600 99501 50050.5 5005050 -32570 32568 4830.66 483066 -126 125 -0.06 -6 +601 2 10591 99502 1.8048 298.8048 150.3048 15030.48048 1.8048 298.8048 150.3048 15030.4807 1.80480 298.80480 150.30479999999991 15030.48000 2020-01-01 2020-01-02 2020-01-01 00:10:01 2020-01-02 03:38:22 2020-01-01 00:10:01.000 2020-01-02 03:38:22.000 601 99502 50051.5 5005150 601 99502 50051.5 5005150 -32569 32569 4831.66 483166 -125 126 0.94 94 +602 2 10592 99503 1.8078 298.8078 150.3078 15030.78078 1.8078 298.8078 150.3078 15030.78035 1.80780 298.80780 150.3078 15030.78000 2020-01-01 2020-01-02 2020-01-01 00:10:02 2020-01-02 03:38:23 2020-01-01 00:10:02.000 2020-01-02 03:38:23.000 602 99503 50052.5 5005250 602 99503 50052.5 5005250 -32568 32570 4832.66 483266 -124 127 1.94 194 +603 2 10593 99504 1.81081 298.81081 150.31081 15031.08108 1.81081 298.81082 150.31081 15031.0811 1.81081 298.81081 150.31081000000017 15031.08100 2020-01-01 2020-01-02 2020-01-01 00:10:03 2020-01-02 03:38:24 2020-01-01 00:10:03.000 2020-01-02 03:38:24.000 603 99504 50053.5 5005350 603 99504 50053.5 5005350 -32567 32571 4833.66 483366 -128 127 0.38 38 +604 2 10594 99505 1.81381 298.81381 150.31381 15031.38138 1.81381 298.8138 150.31381 15031.38124 1.81381 298.81381 150.31380999999985 15031.38100 2020-01-01 2020-01-02 2020-01-01 00:10:04 2020-01-02 03:38:25 2020-01-01 00:10:04.000 2020-01-02 03:38:25.000 604 99505 50054.5 5005450 604 99505 50054.5 5005450 -32566 32572 4834.66 483466 -128 123 -1.18 -118 +605 2 10595 99506 1.81681 298.81681 150.31681 15031.68168 1.81681 298.8168 150.31681 15031.68157 1.81681 298.81681 150.3168100000001 15031.68100 2020-01-01 2020-01-02 2020-01-01 00:10:05 2020-01-02 03:38:26 2020-01-01 00:10:05.000 2020-01-02 03:38:26.000 605 99506 50055.5 5005550 605 99506 50055.5 5005550 -32565 32573 4835.66 483566 -127 124 -0.18 -18 +606 2 10596 99507 1.81981 298.81981 150.31981 15031.98198 1.81981 298.81982 150.31982 15031.98217 1.81981 298.81981 150.3198100000002 15031.98100 2020-01-01 2020-01-02 2020-01-01 00:10:06 2020-01-02 03:38:27 2020-01-01 00:10:06.000 2020-01-02 03:38:27.000 606 99507 50056.5 5005650 606 99507 50056.5 5005650 -32564 32574 4836.66 483666 -126 125 0.82 82 +607 2 10597 99508 1.82282 298.82282 150.32282 15032.28228 1.82282 298.8228 150.32282 15032.28246 1.82282 298.82282 150.32281999999995 15032.28200 2020-01-01 2020-01-02 2020-01-01 00:10:07 2020-01-02 03:38:28 2020-01-01 00:10:07.000 2020-01-02 03:38:28.000 607 99508 50057.5 5005750 607 99508 50057.5 5005750 -32563 32575 4837.66 483766 -125 126 1.82 182 +608 2 10598 99509 1.82582 298.82582 150.32582 15032.58258 1.82582 298.82584 150.32582 15032.58258 1.82582 298.82582 150.32582000000002 15032.58200 2020-01-01 2020-01-02 2020-01-01 00:10:08 2020-01-02 03:38:29 2020-01-01 00:10:08.000 2020-01-02 03:38:29.000 608 99509 50058.5 5005850 608 99509 50058.5 5005850 -32562 32576 4838.66 483866 -124 127 2.82 282 +609 2 10599 99510 1.82882 298.82882 150.32882 15032.88288 1.82882 298.82883 150.32882 15032.88274 1.82882 298.82882 150.3288200000001 15032.88200 2020-01-01 2020-01-02 2020-01-01 00:10:09 2020-01-02 03:38:30 2020-01-01 00:10:09.000 2020-01-02 03:38:30.000 609 99510 50059.5 5005950 609 99510 50059.5 5005950 -32561 32577 4839.66 483966 -128 127 1.26 126 61 2 10051 99961 0.18318 300.18318 150.18318 15168.5015 0.18318 300.1832 150.18318 15168.5016 0.18318 300.18318 150.18318 15168.50118 2020-01-01 2020-01-02 2020-01-01 00:01:01 2020-01-02 03:46:01 2020-01-01 00:01:01.000 2020-01-02 03:46:01.000 61 99961 50011 5051111 61 99961 50011 5051111 -32508 32427 4590.009900990099 463591 -128 123 -2.782178217821782 -281 -610 2 10600 99511 1.83183 298.83183 150.33183 15033.18318 1.83183 298.83182 150.33183 15033.18304 1.83183 298.83183 150.33183 15033.18300 2020-01-01 2020-01-02 2020-01-01 00:10:10 2020-01-02 03:38:31 2020-01-01 00:10:10.000 2020-01-02 03:38:31.000 610 99511 50060.5 5006050 610 99511 50060.5 5006050 -32560 32578 4840.66 484066 -128 127 -0.3 -30 -611 2 10601 99512 1.83483 298.83483 150.33483 15033.48348 1.83483 298.83484 150.33483 15033.48363 1.83483 298.83483 150.33483 15033.48300 2020-01-01 2020-01-02 2020-01-01 00:10:11 2020-01-02 03:38:32 2020-01-01 00:10:11.000 2020-01-02 03:38:32.000 611 99512 50061.5 5006150 611 99512 50061.5 5006150 -32559 32579 4841.66 484166 -128 123 -1.86 -186 -612 2 10602 99513 1.83783 298.83783 150.33783 15033.78378 1.83783 298.83783 150.33783 15033.78393 1.83783 298.83783 150.33783 15033.78300 2020-01-01 2020-01-02 2020-01-01 00:10:12 2020-01-02 03:38:33 2020-01-01 00:10:12.000 2020-01-02 03:38:33.000 612 99513 50062.5 5006250 612 99513 50062.5 5006250 -32558 32580 4842.66 484266 -127 124 -0.86 -86 -613 2 10603 99514 1.84084 298.84084 150.34084 15034.08408 1.84084 298.84085 150.34084 15034.08405 1.84084 298.84084 150.34084 15034.08400 2020-01-01 2020-01-02 2020-01-01 00:10:13 2020-01-02 03:38:34 2020-01-01 00:10:13.000 2020-01-02 03:38:34.000 613 99514 50063.5 5006350 613 99514 50063.5 5006350 -32557 32581 4843.66 484366 -126 125 0.14 14 -614 2 10604 99515 1.84384 298.84384 150.34384 15034.38438 1.84384 298.84384 150.34384 15034.38421 1.84384 298.84384 150.34384 15034.38400 2020-01-01 2020-01-02 2020-01-01 00:10:14 2020-01-02 03:38:35 2020-01-01 00:10:14.000 2020-01-02 03:38:35.000 614 99515 50064.5 5006450 614 99515 50064.5 5006450 -32556 32582 4844.66 484466 -125 126 1.14 114 -615 2 10605 99516 1.84684 298.84684 150.34684 15034.68468 1.84684 298.84683 150.34684 15034.68452 1.84684 298.84684 150.34684 15034.68400 2020-01-01 2020-01-02 2020-01-01 00:10:15 2020-01-02 03:38:36 2020-01-01 00:10:15.000 2020-01-02 03:38:36.000 615 99516 50065.5 5006550 615 99516 50065.5 5006550 -32555 32583 4845.66 484566 -124 127 2.14 214 -616 2 10606 99517 1.84984 298.84984 150.34984 15034.98498 1.84984 298.84985 150.34985 15034.98527 1.84984 298.84984 150.34984 15034.98400 2020-01-01 2020-01-02 2020-01-01 00:10:16 2020-01-02 03:38:37 2020-01-01 00:10:16.000 2020-01-02 03:38:37.000 616 99517 50066.5 5006650 616 99517 50066.5 5006650 -32554 32584 4846.66 484666 -128 127 0.58 58 -617 2 10607 99518 1.85285 298.85285 150.35285 15035.28528 1.85285 298.85284 150.35285 15035.2854 1.85285 298.85285 150.35285 15035.28500 2020-01-01 2020-01-02 2020-01-01 00:10:17 2020-01-02 03:38:38 2020-01-01 00:10:17.000 2020-01-02 03:38:38.000 617 99518 50067.5 5006750 617 99518 50067.5 5006750 -32553 32585 4847.66 484766 -128 123 -0.98 -98 -618 2 10608 99519 1.85585 298.85585 150.35585 15035.58558 1.85585 298.85587 150.35585 15035.58551 1.85585 298.85585 150.35585 15035.58500 2020-01-01 2020-01-02 2020-01-01 00:10:18 2020-01-02 03:38:39 2020-01-01 00:10:18.000 2020-01-02 03:38:39.000 618 99519 50068.5 5006850 618 99519 50068.5 5006850 -32552 32586 4848.66 484866 -127 124 0.02 2 -619 2 10609 99520 1.85885 298.85885 150.35885 15035.88588 1.85885 298.85886 150.35885 15035.88568 1.85885 298.85885 150.35885 15035.88500 2020-01-01 2020-01-02 2020-01-01 00:10:19 2020-01-02 03:38:40 2020-01-01 00:10:19.000 2020-01-02 03:38:40.000 619 99520 50069.5 5006950 619 99520 50069.5 5006950 -32551 32587 4849.66 484966 -126 125 1.02 102 -62 2 10052 99962 0.18618 300.18618 150.18618 15168.8048 0.18618 300.1862 150.18618 15168.80494 0.18618 300.18618 150.18618 15168.80418 2020-01-01 2020-01-02 2020-01-01 00:01:02 2020-01-02 03:46:02 2020-01-01 00:01:02.000 2020-01-02 03:46:02.000 62 99962 50012 5051212 62 99962 50012 5051212 -32507 32428 4591.009900990099 463692 -127 124 -1.7821782178217822 -180 -620 2 10610 99521 1.86186 298.86186 150.36186 15036.18618 1.86186 298.86185 150.36185 15036.18598 1.86186 298.86186 150.36186 15036.18600 2020-01-01 2020-01-02 2020-01-01 00:10:20 2020-01-02 03:38:41 2020-01-01 00:10:20.000 2020-01-02 03:38:41.000 620 99521 50070.5 5007050 620 99521 50070.5 5007050 -32550 32588 4850.66 485066 -125 126 2.02 202 -621 2 10611 99522 1.86486 298.86486 150.36486 15036.48648 1.86486 298.86487 150.36486 15036.48673 1.86486 298.86486 150.36486 15036.48600 2020-01-01 2020-01-02 2020-01-01 00:10:21 2020-01-02 03:38:42 2020-01-01 00:10:21.000 2020-01-02 03:38:42.000 621 99522 50071.5 5007150 621 99522 50071.5 5007150 -32549 32589 4851.66 485166 -124 127 3.02 302 -622 2 10612 99523 1.86786 298.86786 150.36786 15036.78678 1.86786 298.86786 150.36786 15036.78687 1.86786 298.86786 150.36786 15036.78600 2020-01-01 2020-01-02 2020-01-01 00:10:22 2020-01-02 03:38:43 2020-01-01 00:10:22.000 2020-01-02 03:38:43.000 622 99523 50072.5 5007250 622 99523 50072.5 5007250 -32548 32590 4852.66 485266 -128 127 1.46 146 -623 2 10613 99524 1.87087 298.87087 150.37087 15037.08708 1.87087 298.87088 150.37087 15037.08702 1.87087 298.87087 150.37087 15037.08700 2020-01-01 2020-01-02 2020-01-01 00:10:23 2020-01-02 03:38:44 2020-01-01 00:10:23.000 2020-01-02 03:38:44.000 623 99524 50073.5 5007350 623 99524 50073.5 5007350 -32547 32591 4853.66 485366 -128 123 -0.1 -10 -624 2 10614 99525 1.87387 298.87387 150.37387 15037.38738 1.87387 298.87387 150.37387 15037.38716 1.87387 298.87387 150.37387 15037.38700 2020-01-01 2020-01-02 2020-01-01 00:10:24 2020-01-02 03:38:45 2020-01-01 00:10:24.000 2020-01-02 03:38:45.000 624 99525 50074.5 5007450 624 99525 50074.5 5007450 -32546 32592 4854.66 485466 -127 124 0.9 90 -625 2 10615 99526 1.87687 298.87687 150.37687 15037.68768 1.87687 298.8769 150.37687 15037.68791 1.87687 298.87687 150.37687 15037.68700 2020-01-01 2020-01-02 2020-01-01 00:10:25 2020-01-02 03:38:46 2020-01-01 00:10:25.000 2020-01-02 03:38:46.000 625 99526 50075.5 5007550 625 99526 50075.5 5007550 -32545 32593 4855.66 485566 -126 125 1.9 190 -626 2 10616 99527 1.87987 298.87987 150.37987 15037.98798 1.87987 298.87988 150.37988 15037.9882 1.87987 298.87987 150.37987 15037.98700 2020-01-01 2020-01-02 2020-01-01 00:10:26 2020-01-02 03:38:47 2020-01-01 00:10:26.000 2020-01-02 03:38:47.000 626 99527 50076.5 5007650 626 99527 50076.5 5007650 -32544 32594 4856.66 485666 -125 126 2.9 290 -627 2 10617 99528 1.88288 298.88288 150.38288 15038.28828 1.88288 298.88287 150.38288 15038.28834 1.88288 298.88288 150.38288 15038.28800 2020-01-01 2020-01-02 2020-01-01 00:10:27 2020-01-02 03:38:48 2020-01-01 00:10:27.000 2020-01-02 03:38:48.000 627 99528 50077.5 5007750 627 99528 50077.5 5007750 -32543 32595 4857.66 485766 -124 127 3.9 390 -628 2 10618 99529 1.88588 298.88588 150.38588 15038.58858 1.88588 298.8859 150.38588 15038.58849 1.88588 298.88588 150.38588 15038.58800 2020-01-01 2020-01-02 2020-01-01 00:10:28 2020-01-02 03:38:49 2020-01-01 00:10:28.000 2020-01-02 03:38:49.000 628 99529 50078.5 5007850 628 99529 50078.5 5007850 -32542 32596 4858.66 485866 -128 127 2.34 234 -629 2 10619 99530 1.88888 298.88888 150.38888 15038.88888 1.88888 298.8889 150.38888 15038.88862 1.88888 298.88888 150.38888 15038.88800 2020-01-01 2020-01-02 2020-01-01 00:10:29 2020-01-02 03:38:50 2020-01-01 00:10:29.000 2020-01-02 03:38:50.000 629 99530 50079.5 5007950 629 99530 50079.5 5007950 -32541 32597 4859.66 485966 -128 123 0.78 78 -63 2 10053 99963 0.18918 300.18918 150.18918 15169.1081 0.18918 300.18918 150.18918 15169.10808 0.18918 300.18918 150.18918 15169.10718 2020-01-01 2020-01-02 2020-01-01 00:01:03 2020-01-02 03:46:03 2020-01-01 00:01:03.000 2020-01-02 03:46:03.000 63 99963 50013 5051313 63 99963 50013 5051313 -32506 32429 4592.009900990099 463793 -126 125 -0.7821782178217822 -79 -630 2 10620 99531 1.89189 298.89189 150.39189 15039.18918 1.89189 298.8919 150.39189 15039.18937 1.89189 298.89189 150.39189 15039.18900 2020-01-01 2020-01-02 2020-01-01 00:10:30 2020-01-02 03:38:51 2020-01-01 00:10:30.000 2020-01-02 03:38:51.000 630 99531 50080.5 5008050 630 99531 50080.5 5008050 -32540 32598 4860.66 486066 -127 124 1.78 178 -631 2 10621 99532 1.89489 298.89489 150.39489 15039.48948 1.89489 298.8949 150.39489 15039.48968 1.89489 298.89489 150.39489 15039.48900 2020-01-01 2020-01-02 2020-01-01 00:10:31 2020-01-02 03:38:52 2020-01-01 00:10:31.000 2020-01-02 03:38:52.000 631 99532 50081.5 5008150 631 99532 50081.5 5008150 -32539 32599 4861.66 486166 -126 125 2.78 278 -632 2 10622 99533 1.89789 298.89789 150.39789 15039.78978 1.89789 298.8979 150.39789 15039.78984 1.89789 298.89789 150.39789 15039.78900 2020-01-01 2020-01-02 2020-01-01 00:10:32 2020-01-02 03:38:53 2020-01-01 00:10:32.000 2020-01-02 03:38:53.000 632 99533 50082.5 5008250 632 99533 50082.5 5008250 -32538 32600 4862.66 486266 -125 126 3.78 378 -633 2 10623 99534 1.9009 298.9009 150.4009 15040.09009 1.9009 298.9009 150.40089 15040.08996 1.90090 298.90090 150.40090 15040.09000 2020-01-01 2020-01-02 2020-01-01 00:10:33 2020-01-02 03:38:54 2020-01-01 00:10:33.000 2020-01-02 03:38:54.000 633 99534 50083.5 5008350 633 99534 50083.5 5008350 -32537 32601 4863.66 486366 -124 127 4.78 478 -634 2 10624 99535 1.9039 298.9039 150.4039 15040.39039 1.9039 298.9039 150.4039 15040.39009 1.90390 298.90390 150.40390 15040.39000 2020-01-01 2020-01-02 2020-01-01 00:10:34 2020-01-02 03:38:55 2020-01-01 00:10:34.000 2020-01-02 03:38:55.000 634 99535 50084.5 5008450 634 99535 50084.5 5008450 -32536 32602 4864.66 486466 -128 127 3.22 322 -635 2 10625 99536 1.9069 298.9069 150.4069 15040.69069 1.9069 298.90692 150.4069 15040.69084 1.90690 298.90690 150.40690 15040.69000 2020-01-01 2020-01-02 2020-01-01 00:10:35 2020-01-02 03:38:56 2020-01-01 00:10:35.000 2020-01-02 03:38:56.000 635 99536 50085.5 5008550 635 99536 50085.5 5008550 -32535 32603 4865.66 486566 -128 127 1.66 166 -636 2 10626 99537 1.9099 298.9099 150.4099 15040.99099 1.90991 298.9099 150.40991 15040.99115 1.90990 298.90990 150.40990 15040.99000 2020-01-01 2020-01-02 2020-01-01 00:10:36 2020-01-02 03:38:57 2020-01-01 00:10:36.000 2020-01-02 03:38:57.000 636 99537 50086.5 5008650 636 99537 50086.5 5008650 -32534 32604 4866.66 486666 -128 124 0.1 10 -637 2 10627 99538 1.91291 298.91291 150.41291 15041.29129 1.91291 298.9129 150.41291 15041.29131 1.91291 298.91291 150.41291 15041.29100 2020-01-01 2020-01-02 2020-01-01 00:10:37 2020-01-02 03:38:58 2020-01-01 00:10:37.000 2020-01-02 03:38:58.000 637 99538 50087.5 5008750 637 99538 50087.5 5008750 -32533 32605 4867.66 486766 -127 125 1.1 110 -638 2 10628 99539 1.91591 298.91591 150.41591 15041.59159 1.91591 298.91592 150.41591 15041.59143 1.91591 298.91591 150.41591 15041.59100 2020-01-01 2020-01-02 2020-01-01 00:10:38 2020-01-02 03:38:59 2020-01-01 00:10:38.000 2020-01-02 03:38:59.000 638 99539 50088.5 5008850 638 99539 50088.5 5008850 -32532 32606 4868.66 486866 -126 126 2.1 210 -639 2 10629 99540 1.91891 298.91891 150.41891 15041.89189 1.91891 298.9189 150.41891 15041.89172 1.91891 298.91891 150.41891 15041.89100 2020-01-01 2020-01-02 2020-01-01 00:10:39 2020-01-02 03:39:00 2020-01-01 00:10:39.000 2020-01-02 03:39:00.000 639 99540 50089.5 5008950 639 99540 50089.5 5008950 -32531 32607 4869.66 486966 -125 127 3.1 310 -64 2 10054 99964 0.19219 300.19219 150.19219 15169.41141 0.19219 300.1922 150.19219 15169.41184 0.19219 300.19219 150.19219 15169.41119 2020-01-01 2020-01-02 2020-01-01 00:01:04 2020-01-02 03:46:04 2020-01-01 00:01:04.000 2020-01-02 03:46:04.000 64 99964 50014 5051414 64 99964 50014 5051414 -32505 32430 4593.009900990099 463894 -125 126 0.21782178217821782 22 -640 2 10630 99541 1.92192 298.92192 150.42192 15042.19219 1.92192 298.92194 150.42192 15042.19232 1.92192 298.92192 150.42192 15042.19200 2020-01-01 2020-01-02 2020-01-01 00:10:40 2020-01-02 03:39:01 2020-01-01 00:10:40.000 2020-01-02 03:39:01.000 640 99541 50090.5 5009050 640 99541 50090.5 5009050 -32530 32608 4870.66 487066 -128 127 1.54 154 -641 2 10631 99542 1.92492 298.92492 150.42492 15042.49249 1.92492 298.92493 150.42492 15042.49265 1.92492 298.92492 150.42492 15042.49200 2020-01-01 2020-01-02 2020-01-01 00:10:41 2020-01-02 03:39:02 2020-01-01 00:10:41.000 2020-01-02 03:39:02.000 641 99542 50091.5 5009150 641 99542 50091.5 5009150 -32529 32609 4871.66 487166 -128 127 -0.02 -2 -642 2 10632 99543 1.92792 298.92792 150.42792 15042.79279 1.92792 298.92792 150.42792 15042.79278 1.92792 298.92792 150.42792 15042.79200 2020-01-01 2020-01-02 2020-01-01 00:10:42 2020-01-02 03:39:03 2020-01-01 00:10:42.000 2020-01-02 03:39:03.000 642 99543 50092.5 5009250 642 99543 50092.5 5009250 -32528 32610 4872.66 487266 -128 123 -1.58 -158 -643 2 10633 99544 1.93093 298.93093 150.43093 15043.09309 1.93093 298.93094 150.43092 15043.0929 1.93093 298.93093 150.43093 15043.09300 2020-01-01 2020-01-02 2020-01-01 00:10:43 2020-01-02 03:39:04 2020-01-01 00:10:43.000 2020-01-02 03:39:04.000 643 99544 50093.5 5009350 643 99544 50093.5 5009350 -32527 32611 4873.66 487366 -127 124 -0.58 -58 -644 2 10634 99545 1.93393 298.93393 150.43393 15043.39339 1.93393 298.93393 150.43393 15043.39319 1.93393 298.93393 150.43393 15043.39300 2020-01-01 2020-01-02 2020-01-01 00:10:44 2020-01-02 03:39:05 2020-01-01 00:10:44.000 2020-01-02 03:39:05.000 644 99545 50094.5 5009450 644 99545 50094.5 5009450 -32526 32612 4874.66 487466 -126 125 0.42 42 -645 2 10635 99546 1.93693 298.93693 150.43693 15043.69369 1.93693 298.93695 150.43693 15043.69379 1.93693 298.93693 150.43693 15043.69300 2020-01-01 2020-01-02 2020-01-01 00:10:45 2020-01-02 03:39:06 2020-01-01 00:10:45.000 2020-01-02 03:39:06.000 645 99546 50095.5 5009550 645 99546 50095.5 5009550 -32525 32613 4875.66 487566 -125 126 1.42 142 -646 2 10636 99547 1.93993 298.93993 150.43993 15043.99399 1.93994 298.93994 150.43994 15043.99412 1.93993 298.93993 150.43993 15043.99300 2020-01-01 2020-01-02 2020-01-01 00:10:46 2020-01-02 03:39:07 2020-01-01 00:10:46.000 2020-01-02 03:39:07.000 646 99547 50096.5 5009650 646 99547 50096.5 5009650 -32524 32614 4876.66 487666 -124 127 2.42 242 -647 2 10637 99548 1.94294 298.94294 150.44294 15044.29429 1.94294 298.94293 150.44294 15044.29425 1.94294 298.94294 150.44294 15044.29400 2020-01-01 2020-01-02 2020-01-01 00:10:47 2020-01-02 03:39:08 2020-01-01 00:10:47.000 2020-01-02 03:39:08.000 647 99548 50097.5 5009750 647 99548 50097.5 5009750 -32523 32615 4877.66 487766 -128 127 0.86 86 -648 2 10638 99549 1.94594 298.94594 150.44594 15044.59459 1.94594 298.94595 150.44595 15044.595 1.94594 298.94594 150.44594 15044.59400 2020-01-01 2020-01-02 2020-01-01 00:10:48 2020-01-02 03:39:09 2020-01-01 00:10:48.000 2020-01-02 03:39:09.000 648 99549 50098.5 5009850 648 99549 50098.5 5009850 -32522 32616 4878.66 487866 -128 123 -0.7 -70 -649 2 10639 99550 1.94894 298.94894 150.44894 15044.89489 1.94894 298.94894 150.44894 15044.89467 1.94894 298.94894 150.44894 15044.89400 2020-01-01 2020-01-02 2020-01-01 00:10:49 2020-01-02 03:39:10 2020-01-01 00:10:49.000 2020-01-02 03:39:10.000 649 99550 50099.5 5009950 649 99550 50099.5 5009950 -32521 32617 4879.66 487966 -127 124 0.3 30 -65 2 10055 99965 0.19519 300.19519 150.19519 15169.71471 0.19519 300.1952 150.19519 15169.71448 0.19519 300.19519 150.19519 15169.71419 2020-01-01 2020-01-02 2020-01-01 00:01:05 2020-01-02 03:46:05 2020-01-01 00:01:05.000 2020-01-02 03:46:05.000 65 99965 50015 5051515 65 99965 50015 5051515 -32504 32431 4594.009900990099 463995 -124 127 1.2178217821782178 123 -650 2 10640 99551 1.95195 298.95195 150.45195 15045.19519 1.95195 298.95197 150.45195 15045.19525 1.95195 298.95195 150.45195 15045.19500 2020-01-01 2020-01-02 2020-01-01 00:10:50 2020-01-02 03:39:11 2020-01-01 00:10:50.000 2020-01-02 03:39:11.000 650 99551 50100.5 5010050 650 99551 50100.5 5010050 -32520 32618 4880.66 488066 -126 125 1.3 130 -651 2 10641 99552 1.95495 298.95495 150.45495 15045.49549 1.95495 298.95496 150.45495 15045.49558 1.95495 298.95495 150.45495 15045.49500 2020-01-01 2020-01-02 2020-01-01 00:10:51 2020-01-02 03:39:12 2020-01-01 00:10:51.000 2020-01-02 03:39:12.000 651 99552 50101.5 5010150 651 99552 50101.5 5010150 -32519 32619 4881.66 488166 -125 126 2.3 230 +610 2 10600 99511 1.83183 298.83183 150.33183 15033.18318 1.83183 298.83182 150.33183 15033.18304 1.83183 298.83183 150.33182999999985 15033.18300 2020-01-01 2020-01-02 2020-01-01 00:10:10 2020-01-02 03:38:31 2020-01-01 00:10:10.000 2020-01-02 03:38:31.000 610 99511 50060.5 5006050 610 99511 50060.5 5006050 -32560 32578 4840.66 484066 -128 127 -0.3 -30 +611 2 10601 99512 1.83483 298.83483 150.33483 15033.48348 1.83483 298.83484 150.33483 15033.48363 1.83483 298.83483 150.33482999999984 15033.48300 2020-01-01 2020-01-02 2020-01-01 00:10:11 2020-01-02 03:38:32 2020-01-01 00:10:11.000 2020-01-02 03:38:32.000 611 99512 50061.5 5006150 611 99512 50061.5 5006150 -32559 32579 4841.66 484166 -128 123 -1.86 -186 +612 2 10602 99513 1.83783 298.83783 150.33783 15033.78378 1.83783 298.83783 150.33783 15033.78393 1.83783 298.83783 150.33783000000022 15033.78300 2020-01-01 2020-01-02 2020-01-01 00:10:12 2020-01-02 03:38:33 2020-01-01 00:10:12.000 2020-01-02 03:38:33.000 612 99513 50062.5 5006250 612 99513 50062.5 5006250 -32558 32580 4842.66 484266 -127 124 -0.86 -86 +613 2 10603 99514 1.84084 298.84084 150.34084 15034.08408 1.84084 298.84085 150.34084 15034.08405 1.84084 298.84084 150.34084000000007 15034.08400 2020-01-01 2020-01-02 2020-01-01 00:10:13 2020-01-02 03:38:34 2020-01-01 00:10:13.000 2020-01-02 03:38:34.000 613 99514 50063.5 5006350 613 99514 50063.5 5006350 -32557 32581 4843.66 484366 -126 125 0.14 14 +614 2 10604 99515 1.84384 298.84384 150.34384 15034.38438 1.84384 298.84384 150.34384 15034.38421 1.84384 298.84384 150.34383999999977 15034.38400 2020-01-01 2020-01-02 2020-01-01 00:10:14 2020-01-02 03:38:35 2020-01-01 00:10:14.000 2020-01-02 03:38:35.000 614 99515 50064.5 5006450 614 99515 50064.5 5006450 -32556 32582 4844.66 484466 -125 126 1.14 114 +615 2 10605 99516 1.84684 298.84684 150.34684 15034.68468 1.84684 298.84683 150.34684 15034.68452 1.84684 298.84684 150.34684000000007 15034.68400 2020-01-01 2020-01-02 2020-01-01 00:10:15 2020-01-02 03:38:36 2020-01-01 00:10:15.000 2020-01-02 03:38:36.000 615 99516 50065.5 5006550 615 99516 50065.5 5006550 -32555 32583 4845.66 484566 -124 127 2.14 214 +616 2 10606 99517 1.84984 298.84984 150.34984 15034.98498 1.84984 298.84985 150.34985 15034.98527 1.84984 298.84984 150.34984000000014 15034.98400 2020-01-01 2020-01-02 2020-01-01 00:10:16 2020-01-02 03:38:37 2020-01-01 00:10:16.000 2020-01-02 03:38:37.000 616 99517 50066.5 5006650 616 99517 50066.5 5006650 -32554 32584 4846.66 484666 -128 127 0.58 58 +617 2 10607 99518 1.85285 298.85285 150.35285 15035.28528 1.85285 298.85284 150.35285 15035.2854 1.85285 298.85285 150.35284999999993 15035.28500 2020-01-01 2020-01-02 2020-01-01 00:10:17 2020-01-02 03:38:38 2020-01-01 00:10:17.000 2020-01-02 03:38:38.000 617 99518 50067.5 5006750 617 99518 50067.5 5006750 -32553 32585 4847.66 484766 -128 123 -0.98 -98 +618 2 10608 99519 1.85585 298.85585 150.35585 15035.58558 1.85585 298.85587 150.35585 15035.58551 1.85585 298.85585 150.35584999999995 15035.58500 2020-01-01 2020-01-02 2020-01-01 00:10:18 2020-01-02 03:38:39 2020-01-01 00:10:18.000 2020-01-02 03:38:39.000 618 99519 50068.5 5006850 618 99519 50068.5 5006850 -32552 32586 4848.66 484866 -127 124 0.02 2 +619 2 10609 99520 1.85885 298.85885 150.35885 15035.88588 1.85885 298.85886 150.35885 15035.88568 1.85885 298.85885 150.35885000000002 15035.88500 2020-01-01 2020-01-02 2020-01-01 00:10:19 2020-01-02 03:38:40 2020-01-01 00:10:19.000 2020-01-02 03:38:40.000 619 99520 50069.5 5006950 619 99520 50069.5 5006950 -32551 32587 4849.66 484966 -126 125 1.02 102 +62 2 10052 99962 0.18618 300.18618 150.18618 15168.8048 0.18618 300.1862 150.18618 15168.80494 0.18618 300.18618 150.18618000000006 15168.80418 2020-01-01 2020-01-02 2020-01-01 00:01:02 2020-01-02 03:46:02 2020-01-01 00:01:02.000 2020-01-02 03:46:02.000 62 99962 50012 5051212 62 99962 50012 5051212 -32507 32428 4591.009900990099 463692 -127 124 -1.7821782178217822 -180 +620 2 10610 99521 1.86186 298.86186 150.36186 15036.18618 1.86186 298.86185 150.36185 15036.18598 1.86186 298.86186 150.36186000000018 15036.18600 2020-01-01 2020-01-02 2020-01-01 00:10:20 2020-01-02 03:38:41 2020-01-01 00:10:20.000 2020-01-02 03:38:41.000 620 99521 50070.5 5007050 620 99521 50070.5 5007050 -32550 32588 4850.66 485066 -125 126 2.02 202 +621 2 10611 99522 1.86486 298.86486 150.36486 15036.48648 1.86486 298.86487 150.36486 15036.48673 1.86486 298.86486 150.36485999999985 15036.48600 2020-01-01 2020-01-02 2020-01-01 00:10:21 2020-01-02 03:38:42 2020-01-01 00:10:21.000 2020-01-02 03:38:42.000 621 99522 50071.5 5007150 621 99522 50071.5 5007150 -32549 32589 4851.66 485166 -124 127 3.02 302 +622 2 10612 99523 1.86786 298.86786 150.36786 15036.78678 1.86786 298.86786 150.36786 15036.78687 1.86786 298.86786 150.36786000000012 15036.78600 2020-01-01 2020-01-02 2020-01-01 00:10:22 2020-01-02 03:38:43 2020-01-01 00:10:22.000 2020-01-02 03:38:43.000 622 99523 50072.5 5007250 622 99523 50072.5 5007250 -32548 32590 4852.66 485266 -128 127 1.46 146 +623 2 10613 99524 1.87087 298.87087 150.37087 15037.08708 1.87087 298.87088 150.37087 15037.08702 1.87087 298.87087 150.3708700000001 15037.08700 2020-01-01 2020-01-02 2020-01-01 00:10:23 2020-01-02 03:38:44 2020-01-01 00:10:23.000 2020-01-02 03:38:44.000 623 99524 50073.5 5007350 623 99524 50073.5 5007350 -32547 32591 4853.66 485366 -128 123 -0.1 -10 +624 2 10614 99525 1.87387 298.87387 150.37387 15037.38738 1.87387 298.87387 150.37387 15037.38716 1.87387 298.87387 150.37386999999998 15037.38700 2020-01-01 2020-01-02 2020-01-01 00:10:24 2020-01-02 03:38:45 2020-01-01 00:10:24.000 2020-01-02 03:38:45.000 624 99525 50074.5 5007450 624 99525 50074.5 5007450 -32546 32592 4854.66 485466 -127 124 0.9 90 +625 2 10615 99526 1.87687 298.87687 150.37687 15037.68768 1.87687 298.8769 150.37687 15037.68791 1.87687 298.87687 150.37687000000003 15037.68700 2020-01-01 2020-01-02 2020-01-01 00:10:25 2020-01-02 03:38:46 2020-01-01 00:10:25.000 2020-01-02 03:38:46.000 625 99526 50075.5 5007550 625 99526 50075.5 5007550 -32545 32593 4855.66 485566 -126 125 1.9 190 +626 2 10616 99527 1.87987 298.87987 150.37987 15037.98798 1.87987 298.87988 150.37988 15037.9882 1.87987 298.87987 150.3798700000001 15037.98700 2020-01-01 2020-01-02 2020-01-01 00:10:26 2020-01-02 03:38:47 2020-01-01 00:10:26.000 2020-01-02 03:38:47.000 626 99527 50076.5 5007650 626 99527 50076.5 5007650 -32544 32594 4856.66 485666 -125 126 2.9 290 +627 2 10617 99528 1.88288 298.88288 150.38288 15038.28828 1.88288 298.88287 150.38288 15038.28834 1.88288 298.88288 150.38287999999991 15038.28800 2020-01-01 2020-01-02 2020-01-01 00:10:27 2020-01-02 03:38:48 2020-01-01 00:10:27.000 2020-01-02 03:38:48.000 627 99528 50077.5 5007750 627 99528 50077.5 5007750 -32543 32595 4857.66 485766 -124 127 3.9 390 +628 2 10618 99529 1.88588 298.88588 150.38588 15038.58858 1.88588 298.8859 150.38588 15038.58849 1.88588 298.88588 150.38587999999987 15038.58800 2020-01-01 2020-01-02 2020-01-01 00:10:28 2020-01-02 03:38:49 2020-01-01 00:10:28.000 2020-01-02 03:38:49.000 628 99529 50078.5 5007850 628 99529 50078.5 5007850 -32542 32596 4858.66 485866 -128 127 2.34 234 +629 2 10619 99530 1.88888 298.88888 150.38888 15038.88888 1.88888 298.8889 150.38888 15038.88862 1.88888 298.88888 150.3888800000002 15038.88800 2020-01-01 2020-01-02 2020-01-01 00:10:29 2020-01-02 03:38:50 2020-01-01 00:10:29.000 2020-01-02 03:38:50.000 629 99530 50079.5 5007950 629 99530 50079.5 5007950 -32541 32597 4859.66 485966 -128 123 0.78 78 +63 2 10053 99963 0.18918 300.18918 150.18918 15169.1081 0.18918 300.18918 150.18918 15169.10808 0.18918 300.18918 150.1891799999999 15169.10718 2020-01-01 2020-01-02 2020-01-01 00:01:03 2020-01-02 03:46:03 2020-01-01 00:01:03.000 2020-01-02 03:46:03.000 63 99963 50013 5051313 63 99963 50013 5051313 -32506 32429 4592.009900990099 463793 -126 125 -0.7821782178217822 -79 +630 2 10620 99531 1.89189 298.89189 150.39189 15039.18918 1.89189 298.8919 150.39189 15039.18937 1.89189 298.89189 150.3918900000001 15039.18900 2020-01-01 2020-01-02 2020-01-01 00:10:30 2020-01-02 03:38:51 2020-01-01 00:10:30.000 2020-01-02 03:38:51.000 630 99531 50080.5 5008050 630 99531 50080.5 5008050 -32540 32598 4860.66 486066 -127 124 1.78 178 +631 2 10621 99532 1.89489 298.89489 150.39489 15039.48948 1.89489 298.8949 150.39489 15039.48968 1.89489 298.89489 150.39488999999978 15039.48900 2020-01-01 2020-01-02 2020-01-01 00:10:31 2020-01-02 03:38:52 2020-01-01 00:10:31.000 2020-01-02 03:38:52.000 631 99532 50081.5 5008150 631 99532 50081.5 5008150 -32539 32599 4861.66 486166 -126 125 2.78 278 +632 2 10622 99533 1.89789 298.89789 150.39789 15039.78978 1.89789 298.8979 150.39789 15039.78984 1.89789 298.89789 150.39789000000013 15039.78900 2020-01-01 2020-01-02 2020-01-01 00:10:32 2020-01-02 03:38:53 2020-01-01 00:10:32.000 2020-01-02 03:38:53.000 632 99533 50082.5 5008250 632 99533 50082.5 5008250 -32538 32600 4862.66 486266 -125 126 3.78 378 +633 2 10623 99534 1.9009 298.9009 150.4009 15040.09009 1.9009 298.9009 150.40089 15040.08996 1.90090 298.90090 150.40090000000004 15040.09000 2020-01-01 2020-01-02 2020-01-01 00:10:33 2020-01-02 03:38:54 2020-01-01 00:10:33.000 2020-01-02 03:38:54.000 633 99534 50083.5 5008350 633 99534 50083.5 5008350 -32537 32601 4863.66 486366 -124 127 4.78 478 +634 2 10624 99535 1.9039 298.9039 150.4039 15040.39039 1.9039 298.9039 150.4039 15040.39009 1.90390 298.90390 150.4038999999999 15040.39000 2020-01-01 2020-01-02 2020-01-01 00:10:34 2020-01-02 03:38:55 2020-01-01 00:10:34.000 2020-01-02 03:38:55.000 634 99535 50084.5 5008450 634 99535 50084.5 5008450 -32536 32602 4864.66 486466 -128 127 3.22 322 +635 2 10625 99536 1.9069 298.9069 150.4069 15040.69069 1.9069 298.90692 150.4069 15040.69084 1.90690 298.90690 150.40689999999998 15040.69000 2020-01-01 2020-01-02 2020-01-01 00:10:35 2020-01-02 03:38:56 2020-01-01 00:10:35.000 2020-01-02 03:38:56.000 635 99536 50085.5 5008550 635 99536 50085.5 5008550 -32535 32603 4865.66 486566 -128 127 1.66 166 +636 2 10626 99537 1.9099 298.9099 150.4099 15040.99099 1.90991 298.9099 150.40991 15040.99115 1.90990 298.90990 150.40990000000005 15040.99000 2020-01-01 2020-01-02 2020-01-01 00:10:36 2020-01-02 03:38:57 2020-01-01 00:10:36.000 2020-01-02 03:38:57.000 636 99537 50086.5 5008650 636 99537 50086.5 5008650 -32534 32604 4866.66 486666 -128 124 0.1 10 +637 2 10627 99538 1.91291 298.91291 150.41291 15041.29129 1.91291 298.9129 150.41291 15041.29131 1.91291 298.91291 150.4129099999998 15041.29100 2020-01-01 2020-01-02 2020-01-01 00:10:37 2020-01-02 03:38:58 2020-01-01 00:10:37.000 2020-01-02 03:38:58.000 637 99538 50087.5 5008750 637 99538 50087.5 5008750 -32533 32605 4867.66 486766 -127 125 1.1 110 +638 2 10628 99539 1.91591 298.91591 150.41591 15041.59159 1.91591 298.91592 150.41591 15041.59143 1.91591 298.91591 150.41590999999988 15041.59100 2020-01-01 2020-01-02 2020-01-01 00:10:38 2020-01-02 03:38:59 2020-01-01 00:10:38.000 2020-01-02 03:38:59.000 638 99539 50088.5 5008850 638 99539 50088.5 5008850 -32532 32606 4868.66 486866 -126 126 2.1 210 +639 2 10629 99540 1.91891 298.91891 150.41891 15041.89189 1.91891 298.9189 150.41891 15041.89172 1.91891 298.91891 150.41891000000015 15041.89100 2020-01-01 2020-01-02 2020-01-01 00:10:39 2020-01-02 03:39:00 2020-01-01 00:10:39.000 2020-01-02 03:39:00.000 639 99540 50089.5 5008950 639 99540 50089.5 5008950 -32531 32607 4869.66 486966 -125 127 3.1 310 +64 2 10054 99964 0.19219 300.19219 150.19219 15169.41141 0.19219 300.1922 150.19219 15169.41184 0.19219 300.19219 150.1921899999999 15169.41119 2020-01-01 2020-01-02 2020-01-01 00:01:04 2020-01-02 03:46:04 2020-01-01 00:01:04.000 2020-01-02 03:46:04.000 64 99964 50014 5051414 64 99964 50014 5051414 -32505 32430 4593.009900990099 463894 -125 126 0.21782178217821782 22 +640 2 10630 99541 1.92192 298.92192 150.42192 15042.19219 1.92192 298.92194 150.42192 15042.19232 1.92192 298.92192 150.42192000000006 15042.19200 2020-01-01 2020-01-02 2020-01-01 00:10:40 2020-01-02 03:39:01 2020-01-01 00:10:40.000 2020-01-02 03:39:01.000 640 99541 50090.5 5009050 640 99541 50090.5 5009050 -32530 32608 4870.66 487066 -128 127 1.54 154 +641 2 10631 99542 1.92492 298.92492 150.42492 15042.49249 1.92492 298.92493 150.42492 15042.49265 1.92492 298.92492 150.42491999999996 15042.49200 2020-01-01 2020-01-02 2020-01-01 00:10:41 2020-01-02 03:39:02 2020-01-01 00:10:41.000 2020-01-02 03:39:02.000 641 99542 50091.5 5009150 641 99542 50091.5 5009150 -32529 32609 4871.66 487166 -128 127 -0.02 -2 +642 2 10632 99543 1.92792 298.92792 150.42792 15042.79279 1.92792 298.92792 150.42792 15042.79278 1.92792 298.92792 150.42792000000006 15042.79200 2020-01-01 2020-01-02 2020-01-01 00:10:42 2020-01-02 03:39:03 2020-01-01 00:10:42.000 2020-01-02 03:39:03.000 642 99543 50092.5 5009250 642 99543 50092.5 5009250 -32528 32610 4872.66 487266 -128 123 -1.58 -158 +643 2 10633 99544 1.93093 298.93093 150.43093 15043.09309 1.93093 298.93094 150.43092 15043.0929 1.93093 298.93093 150.43093000000022 15043.09300 2020-01-01 2020-01-02 2020-01-01 00:10:43 2020-01-02 03:39:04 2020-01-01 00:10:43.000 2020-01-02 03:39:04.000 643 99544 50093.5 5009350 643 99544 50093.5 5009350 -32527 32611 4873.66 487366 -127 124 -0.58 -58 +644 2 10634 99545 1.93393 298.93393 150.43393 15043.39339 1.93393 298.93393 150.43393 15043.39319 1.93393 298.93393 150.4339299999999 15043.39300 2020-01-01 2020-01-02 2020-01-01 00:10:44 2020-01-02 03:39:05 2020-01-01 00:10:44.000 2020-01-02 03:39:05.000 644 99545 50094.5 5009450 644 99545 50094.5 5009450 -32526 32612 4874.66 487466 -126 125 0.42 42 +645 2 10635 99546 1.93693 298.93693 150.43693 15043.69369 1.93693 298.93695 150.43693 15043.69379 1.93693 298.93693 150.4369299999999 15043.69300 2020-01-01 2020-01-02 2020-01-01 00:10:45 2020-01-02 03:39:06 2020-01-01 00:10:45.000 2020-01-02 03:39:06.000 645 99546 50095.5 5009550 645 99546 50095.5 5009550 -32525 32613 4875.66 487566 -125 126 1.42 142 +646 2 10636 99547 1.93993 298.93993 150.43993 15043.99399 1.93994 298.93994 150.43994 15043.99412 1.93993 298.93993 150.4399300000002 15043.99300 2020-01-01 2020-01-02 2020-01-01 00:10:46 2020-01-02 03:39:07 2020-01-01 00:10:46.000 2020-01-02 03:39:07.000 646 99547 50096.5 5009650 646 99547 50096.5 5009650 -32524 32614 4876.66 487666 -124 127 2.42 242 +647 2 10637 99548 1.94294 298.94294 150.44294 15044.29429 1.94294 298.94293 150.44294 15044.29425 1.94294 298.94294 150.44294000000014 15044.29400 2020-01-01 2020-01-02 2020-01-01 00:10:47 2020-01-02 03:39:08 2020-01-01 00:10:47.000 2020-01-02 03:39:08.000 647 99548 50097.5 5009750 647 99548 50097.5 5009750 -32523 32615 4877.66 487766 -128 127 0.86 86 +648 2 10638 99549 1.94594 298.94594 150.44594 15044.59459 1.94594 298.94595 150.44595 15044.595 1.94594 298.94594 150.44593999999978 15044.59400 2020-01-01 2020-01-02 2020-01-01 00:10:48 2020-01-02 03:39:09 2020-01-01 00:10:48.000 2020-01-02 03:39:09.000 648 99549 50098.5 5009850 648 99549 50098.5 5009850 -32522 32616 4878.66 487866 -128 123 -0.7 -70 +649 2 10639 99550 1.94894 298.94894 150.44894 15044.89489 1.94894 298.94894 150.44894 15044.89467 1.94894 298.94894 150.44894000000016 15044.89400 2020-01-01 2020-01-02 2020-01-01 00:10:49 2020-01-02 03:39:10 2020-01-01 00:10:49.000 2020-01-02 03:39:10.000 649 99550 50099.5 5009950 649 99550 50099.5 5009950 -32521 32617 4879.66 487966 -127 124 0.3 30 +65 2 10055 99965 0.19519 300.19519 150.19519 15169.71471 0.19519 300.1952 150.19519 15169.71448 0.19519 300.19519 150.1951900000002 15169.71419 2020-01-01 2020-01-02 2020-01-01 00:01:05 2020-01-02 03:46:05 2020-01-01 00:01:05.000 2020-01-02 03:46:05.000 65 99965 50015 5051515 65 99965 50015 5051515 -32504 32431 4594.009900990099 463995 -124 127 1.2178217821782178 123 +650 2 10640 99551 1.95195 298.95195 150.45195 15045.19519 1.95195 298.95197 150.45195 15045.19525 1.95195 298.95195 150.4519500000001 15045.19500 2020-01-01 2020-01-02 2020-01-01 00:10:50 2020-01-02 03:39:11 2020-01-01 00:10:50.000 2020-01-02 03:39:11.000 650 99551 50100.5 5010050 650 99551 50100.5 5010050 -32520 32618 4880.66 488066 -126 125 1.3 130 +651 2 10641 99552 1.95495 298.95495 150.45495 15045.49549 1.95495 298.95496 150.45495 15045.49558 1.95495 298.95495 150.4549499999999 15045.49500 2020-01-01 2020-01-02 2020-01-01 00:10:51 2020-01-02 03:39:12 2020-01-01 00:10:51.000 2020-01-02 03:39:12.000 651 99552 50101.5 5010150 651 99552 50101.5 5010150 -32519 32619 4881.66 488166 -125 126 2.3 230 652 2 10642 99553 1.95795 298.95795 150.45795 15045.79579 1.95795 298.95795 150.45795 15045.79572 1.95795 298.95795 150.45795 15045.79500 2020-01-01 2020-01-02 2020-01-01 00:10:52 2020-01-02 03:39:13 2020-01-01 00:10:52.000 2020-01-02 03:39:13.000 652 99553 50102.5 5010250 652 99553 50102.5 5010250 -32518 32620 4882.66 488266 -124 127 3.3 330 -653 2 10643 99554 1.96096 298.96096 150.46096 15046.09609 1.96096 298.96097 150.46096 15046.09647 1.96096 298.96096 150.46096 15046.09600 2020-01-01 2020-01-02 2020-01-01 00:10:53 2020-01-02 03:39:14 2020-01-01 00:10:53.000 2020-01-02 03:39:14.000 653 99554 50103.5 5010350 653 99554 50103.5 5010350 -32517 32621 4883.66 488366 -128 127 1.74 174 -654 2 10644 99555 1.96396 298.96396 150.46396 15046.39639 1.96396 298.96396 150.46396 15046.39613 1.96396 298.96396 150.46396 15046.39600 2020-01-01 2020-01-02 2020-01-01 00:10:54 2020-01-02 03:39:15 2020-01-01 00:10:54.000 2020-01-02 03:39:15.000 654 99555 50104.5 5010450 654 99555 50104.5 5010450 -32516 32622 4884.66 488466 -128 123 0.18 18 -655 2 10645 99556 1.96696 298.96696 150.46696 15046.69669 1.96696 298.96698 150.46696 15046.69676 1.96696 298.96696 150.46696 15046.69600 2020-01-01 2020-01-02 2020-01-01 00:10:55 2020-01-02 03:39:16 2020-01-01 00:10:55.000 2020-01-02 03:39:16.000 655 99556 50105.5 5010550 655 99556 50105.5 5010550 -32515 32623 4885.66 488566 -127 124 1.18 118 -656 2 10646 99557 1.96996 298.96996 150.46996 15046.99699 1.96997 298.96997 150.46997 15046.99706 1.96996 298.96996 150.46996 15046.99600 2020-01-01 2020-01-02 2020-01-01 00:10:56 2020-01-02 03:39:17 2020-01-01 00:10:56.000 2020-01-02 03:39:17.000 656 99557 50106.5 5010650 656 99557 50106.5 5010650 -32514 32624 4886.66 488666 -126 125 2.18 218 -657 2 10647 99558 1.97297 298.97297 150.47297 15047.29729 1.97297 298.97296 150.47297 15047.29735 1.97297 298.97297 150.47297 15047.29700 2020-01-01 2020-01-02 2020-01-01 00:10:57 2020-01-02 03:39:18 2020-01-01 00:10:57.000 2020-01-02 03:39:18.000 657 99558 50107.5 5010750 657 99558 50107.5 5010750 -32513 32625 4887.66 488766 -125 126 3.18 318 +653 2 10643 99554 1.96096 298.96096 150.46096 15046.09609 1.96096 298.96097 150.46096 15046.09647 1.96096 298.96096 150.46096000000014 15046.09600 2020-01-01 2020-01-02 2020-01-01 00:10:53 2020-01-02 03:39:14 2020-01-01 00:10:53.000 2020-01-02 03:39:14.000 653 99554 50103.5 5010350 653 99554 50103.5 5010350 -32517 32621 4883.66 488366 -128 127 1.74 174 +654 2 10644 99555 1.96396 298.96396 150.46396 15046.39639 1.96396 298.96396 150.46396 15046.39613 1.96396 298.96396 150.46395999999982 15046.39600 2020-01-01 2020-01-02 2020-01-01 00:10:54 2020-01-02 03:39:15 2020-01-01 00:10:54.000 2020-01-02 03:39:15.000 654 99555 50104.5 5010450 654 99555 50104.5 5010450 -32516 32622 4884.66 488466 -128 123 0.18 18 +655 2 10645 99556 1.96696 298.96696 150.46696 15046.69669 1.96696 298.96698 150.46696 15046.69676 1.96696 298.96696 150.46695999999986 15046.69600 2020-01-01 2020-01-02 2020-01-01 00:10:55 2020-01-02 03:39:16 2020-01-01 00:10:55.000 2020-01-02 03:39:16.000 655 99556 50105.5 5010550 655 99556 50105.5 5010550 -32515 32623 4885.66 488566 -127 124 1.18 118 +656 2 10646 99557 1.96996 298.96996 150.46996 15046.99699 1.96997 298.96997 150.46997 15046.99706 1.96996 298.96996 150.46996000000016 15046.99600 2020-01-01 2020-01-02 2020-01-01 00:10:56 2020-01-02 03:39:17 2020-01-01 00:10:56.000 2020-01-02 03:39:17.000 656 99557 50106.5 5010650 656 99557 50106.5 5010650 -32514 32624 4886.66 488666 -126 125 2.18 218 +657 2 10647 99558 1.97297 298.97297 150.47297 15047.29729 1.97297 298.97296 150.47297 15047.29735 1.97297 298.97297 150.47297000000006 15047.29700 2020-01-01 2020-01-02 2020-01-01 00:10:57 2020-01-02 03:39:18 2020-01-01 00:10:57.000 2020-01-02 03:39:18.000 657 99558 50107.5 5010750 657 99558 50107.5 5010750 -32513 32625 4887.66 488766 -125 126 3.18 318 658 2 10648 99559 1.97597 298.97597 150.47597 15047.59759 1.97597 298.97598 150.47597 15047.59794 1.97597 298.97597 150.47597 15047.59700 2020-01-01 2020-01-02 2020-01-01 00:10:58 2020-01-02 03:39:19 2020-01-01 00:10:58.000 2020-01-02 03:39:19.000 658 99559 50108.5 5010850 658 99559 50108.5 5010850 -32512 32626 4888.66 488866 -124 127 4.18 418 -659 2 10649 99560 1.97897 298.97897 150.47897 15047.89789 1.97897 298.97897 150.47897 15047.8976 1.97897 298.97897 150.47897 15047.89700 2020-01-01 2020-01-02 2020-01-01 00:10:59 2020-01-02 03:39:20 2020-01-01 00:10:59.000 2020-01-02 03:39:20.000 659 99560 50109.5 5010950 659 99560 50109.5 5010950 -32511 32627 4889.66 488966 -128 127 2.62 262 -66 2 10056 99966 0.19819 300.19819 150.19819 15170.01801 0.19819 300.1982 150.19819 15170.01808 0.19819 300.19819 150.19819 15170.01719 2020-01-01 2020-01-02 2020-01-01 00:01:06 2020-01-02 03:46:06 2020-01-01 00:01:06.000 2020-01-02 03:46:06.000 66 99966 50016 5051616 66 99966 50016 5051616 -32503 32432 4595.009900990099 464096 -128 127 -0.31683168316831684 -32 -660 2 10650 99561 1.98198 298.98198 150.48198 15048.19819 1.98198 298.982 150.48198 15048.19822 1.98198 298.98198 150.48198 15048.19800 2020-01-01 2020-01-02 2020-01-01 00:11:00 2020-01-02 03:39:21 2020-01-01 00:11:00.000 2020-01-02 03:39:21.000 660 99561 50110.5 5011050 660 99561 50110.5 5011050 -32510 32628 4890.66 489066 -128 127 1.06 106 -661 2 10651 99562 1.98498 298.98498 150.48498 15048.49849 1.98498 298.985 150.48498 15048.49853 1.98498 298.98498 150.48498 15048.49800 2020-01-01 2020-01-02 2020-01-01 00:11:01 2020-01-02 03:39:22 2020-01-01 00:11:01.000 2020-01-02 03:39:22.000 661 99562 50111.5 5011150 661 99562 50111.5 5011150 -32509 32629 4891.66 489166 -128 124 -0.5 -50 -662 2 10652 99563 1.98798 298.98798 150.48798 15048.79879 1.98798 298.98798 150.48798 15048.79882 1.98798 298.98798 150.48798 15048.79800 2020-01-01 2020-01-02 2020-01-01 00:11:02 2020-01-02 03:39:23 2020-01-01 00:11:02.000 2020-01-02 03:39:23.000 662 99563 50112.5 5011250 662 99563 50112.5 5011250 -32508 32630 4892.66 489266 -127 125 0.5 50 -663 2 10653 99564 1.99099 298.99099 150.49099 15049.09909 1.99099 298.991 150.49099 15049.09942 1.99099 298.99099 150.49099 15049.09900 2020-01-01 2020-01-02 2020-01-01 00:11:03 2020-01-02 03:39:24 2020-01-01 00:11:03.000 2020-01-02 03:39:24.000 663 99564 50113.5 5011350 663 99564 50113.5 5011350 -32507 32631 4893.66 489366 -126 126 1.5 150 -664 2 10654 99565 1.99399 298.99399 150.49399 15049.39939 1.99399 298.994 150.49399 15049.39911 1.99399 298.99399 150.49399 15049.39900 2020-01-01 2020-01-02 2020-01-01 00:11:04 2020-01-02 03:39:25 2020-01-01 00:11:04.000 2020-01-02 03:39:25.000 664 99565 50114.5 5011450 664 99565 50114.5 5011450 -32506 32632 4894.66 489466 -125 127 2.5 250 -665 2 10655 99566 1.99699 298.99699 150.49699 15049.69969 1.99699 298.997 150.49699 15049.6997 1.99699 298.99699 150.49699 15049.69900 2020-01-01 2020-01-02 2020-01-01 00:11:05 2020-01-02 03:39:26 2020-01-01 00:11:05.000 2020-01-02 03:39:26.000 665 99566 50115.5 5011550 665 99566 50115.5 5011550 -32505 32633 4895.66 489566 -128 127 0.94 94 -666 2 10656 99567 2 299 150.5 15050 2 299 150.5 15050 2.00000 299.00000 150.50000 15050.00000 2020-01-01 2020-01-02 2020-01-01 00:11:06 2020-01-02 03:39:27 2020-01-01 00:11:06.000 2020-01-02 03:39:27.000 666 99567 50116.5 5011650 666 99567 50116.5 5011650 -32504 32634 4896.66 489666 -128 127 -0.62 -62 -667 2 10657 99568 2.003 299.003 150.503 15050.3003 2.003 299.003 150.503 15050.30029 2.00300 299.00300 150.50300 15050.30000 2020-01-01 2020-01-02 2020-01-01 00:11:07 2020-01-02 03:39:28 2020-01-01 00:11:07.000 2020-01-02 03:39:28.000 667 99568 50117.5 5011750 667 99568 50117.5 5011750 -32503 32635 4897.66 489766 -128 123 -2.18 -218 -668 2 10658 99569 2.006 299.006 150.506 15050.6006 2.006 299.006 150.506 15050.60089 2.00600 299.00600 150.50600 15050.60000 2020-01-01 2020-01-02 2020-01-01 00:11:08 2020-01-02 03:39:29 2020-01-01 00:11:08.000 2020-01-02 03:39:29.000 668 99569 50118.5 5011850 668 99569 50118.5 5011850 -32502 32636 4898.66 489866 -127 124 -1.18 -118 -669 2 10659 99570 2.009 299.009 150.509 15050.9009 2.009 299.009 150.509 15050.90057 2.00900 299.00900 150.50900 15050.90000 2020-01-01 2020-01-02 2020-01-01 00:11:09 2020-01-02 03:39:30 2020-01-01 00:11:09.000 2020-01-02 03:39:30.000 669 99570 50119.5 5011950 669 99570 50119.5 5011950 -32501 32637 4899.66 489966 -126 125 -0.18 -18 -67 2 10057 99967 0.2012 300.2012 150.2012 15170.32132 0.2012 300.2012 150.2012 15170.32142 0.20120 300.20120 150.20120 15170.32120 2020-01-01 2020-01-02 2020-01-01 00:01:07 2020-01-02 03:46:07 2020-01-01 00:01:07.000 2020-01-02 03:46:07.000 67 99967 50017 5051717 67 99967 50017 5051717 -32502 32433 4596.009900990099 464197 -128 127 -1.8514851485148516 -187 -670 2 10660 99571 2.01201 299.01201 150.51201 15051.2012 2.01201 299.01202 150.51201 15051.20117 2.01201 299.01201 150.51201 15051.20100 2020-01-01 2020-01-02 2020-01-01 00:11:10 2020-01-02 03:39:31 2020-01-01 00:11:10.000 2020-01-02 03:39:31.000 670 99571 50120.5 5012050 670 99571 50120.5 5012050 -32500 32638 4900.66 490066 -125 126 0.82 82 -671 2 10661 99572 2.01501 299.01501 150.51501 15051.5015 2.01501 299.015 150.51501 15051.50146 2.01501 299.01501 150.51501 15051.50100 2020-01-01 2020-01-02 2020-01-01 00:11:11 2020-01-02 03:39:32 2020-01-01 00:11:11.000 2020-01-02 03:39:32.000 671 99572 50121.5 5012150 671 99572 50121.5 5012150 -32499 32639 4901.66 490166 -124 127 1.82 182 -672 2 10662 99573 2.01801 299.01801 150.51801 15051.8018 2.01801 299.018 150.51801 15051.80176 2.01801 299.01801 150.51801 15051.80100 2020-01-01 2020-01-02 2020-01-01 00:11:12 2020-01-02 03:39:33 2020-01-01 00:11:12.000 2020-01-02 03:39:33.000 672 99573 50122.5 5012250 672 99573 50122.5 5012250 -32498 32640 4902.66 490266 -128 127 0.26 26 -673 2 10663 99574 2.02102 299.02102 150.52102 15052.1021 2.02102 299.02103 150.52102 15052.1024 2.02102 299.02102 150.52102 15052.10200 2020-01-01 2020-01-02 2020-01-01 00:11:13 2020-01-02 03:39:34 2020-01-01 00:11:13.000 2020-01-02 03:39:34.000 673 99574 50123.5 5012350 673 99574 50123.5 5012350 -32497 32641 4903.66 490366 -128 123 -1.3 -130 -674 2 10664 99575 2.02402 299.02402 150.52402 15052.4024 2.02402 299.02402 150.52402 15052.40204 2.02402 299.02402 150.52402 15052.40200 2020-01-01 2020-01-02 2020-01-01 00:11:14 2020-01-02 03:39:35 2020-01-01 00:11:14.000 2020-01-02 03:39:35.000 674 99575 50124.5 5012450 674 99575 50124.5 5012450 -32496 32642 4904.66 490466 -127 124 -0.3 -30 -675 2 10665 99576 2.02702 299.02702 150.52702 15052.7027 2.02702 299.02704 150.52702 15052.70264 2.02702 299.02702 150.52702 15052.70200 2020-01-01 2020-01-02 2020-01-01 00:11:15 2020-01-02 03:39:36 2020-01-01 00:11:15.000 2020-01-02 03:39:36.000 675 99576 50125.5 5012550 675 99576 50125.5 5012550 -32495 32643 4905.66 490566 -126 125 0.7 70 -676 2 10666 99577 2.03003 299.03003 150.53003 15053.003 2.03003 299.03003 150.53002 15053.00293 2.03003 299.03003 150.53003 15053.00300 2020-01-01 2020-01-02 2020-01-01 00:11:16 2020-01-02 03:39:37 2020-01-01 00:11:16.000 2020-01-02 03:39:37.000 676 99577 50126.5 5012650 676 99577 50126.5 5012650 -32494 32644 4906.66 490666 -125 126 1.7 170 +659 2 10649 99560 1.97897 298.97897 150.47897 15047.89789 1.97897 298.97897 150.47897 15047.8976 1.97897 298.97897 150.4789700000001 15047.89700 2020-01-01 2020-01-02 2020-01-01 00:10:59 2020-01-02 03:39:20 2020-01-01 00:10:59.000 2020-01-02 03:39:20.000 659 99560 50109.5 5010950 659 99560 50109.5 5010950 -32511 32627 4889.66 488966 -128 127 2.62 262 +66 2 10056 99966 0.19819 300.19819 150.19819 15170.01801 0.19819 300.1982 150.19819 15170.01808 0.19819 300.19819 150.19818999999987 15170.01719 2020-01-01 2020-01-02 2020-01-01 00:01:06 2020-01-02 03:46:06 2020-01-01 00:01:06.000 2020-01-02 03:46:06.000 66 99966 50016 5051616 66 99966 50016 5051616 -32503 32432 4595.009900990099 464096 -128 127 -0.31683168316831684 -32 +660 2 10650 99561 1.98198 298.98198 150.48198 15048.19819 1.98198 298.982 150.48198 15048.19822 1.98198 298.98198 150.48197999999996 15048.19800 2020-01-01 2020-01-02 2020-01-01 00:11:00 2020-01-02 03:39:21 2020-01-01 00:11:00.000 2020-01-02 03:39:21.000 660 99561 50110.5 5011050 660 99561 50110.5 5011050 -32510 32628 4890.66 489066 -128 127 1.06 106 +661 2 10651 99562 1.98498 298.98498 150.48498 15048.49849 1.98498 298.985 150.48498 15048.49853 1.98498 298.98498 150.4849799999999 15048.49800 2020-01-01 2020-01-02 2020-01-01 00:11:01 2020-01-02 03:39:22 2020-01-01 00:11:01.000 2020-01-02 03:39:22.000 661 99562 50111.5 5011150 661 99562 50111.5 5011150 -32509 32629 4891.66 489166 -128 124 -0.5 -50 +662 2 10652 99563 1.98798 298.98798 150.48798 15048.79879 1.98798 298.98798 150.48798 15048.79882 1.98798 298.98798 150.4879799999999 15048.79800 2020-01-01 2020-01-02 2020-01-01 00:11:02 2020-01-02 03:39:23 2020-01-01 00:11:02.000 2020-01-02 03:39:23.000 662 99563 50112.5 5011250 662 99563 50112.5 5011250 -32508 32630 4892.66 489266 -127 125 0.5 50 +663 2 10653 99564 1.99099 298.99099 150.49099 15049.09909 1.99099 298.991 150.49099 15049.09942 1.99099 298.99099 150.49099000000015 15049.09900 2020-01-01 2020-01-02 2020-01-01 00:11:03 2020-01-02 03:39:24 2020-01-01 00:11:03.000 2020-01-02 03:39:24.000 663 99564 50113.5 5011350 663 99564 50113.5 5011350 -32507 32631 4893.66 489366 -126 126 1.5 150 +664 2 10654 99565 1.99399 298.99399 150.49399 15049.39939 1.99399 298.994 150.49399 15049.39911 1.99399 298.99399 150.49399000000014 15049.39900 2020-01-01 2020-01-02 2020-01-01 00:11:04 2020-01-02 03:39:25 2020-01-01 00:11:04.000 2020-01-02 03:39:25.000 664 99565 50114.5 5011450 664 99565 50114.5 5011450 -32506 32632 4894.66 489466 -125 127 2.5 250 +665 2 10655 99566 1.99699 298.99699 150.49699 15049.69969 1.99699 298.997 150.49699 15049.6997 1.99699 298.99699 150.4969899999998 15049.69900 2020-01-01 2020-01-02 2020-01-01 00:11:05 2020-01-02 03:39:26 2020-01-01 00:11:05.000 2020-01-02 03:39:26.000 665 99566 50115.5 5011550 665 99566 50115.5 5011550 -32505 32633 4895.66 489566 -128 127 0.94 94 +666 2 10656 99567 2 299 150.5 15050 2 299 150.5 15050 2.00000 299.00000 150.5 15050.00000 2020-01-01 2020-01-02 2020-01-01 00:11:06 2020-01-02 03:39:27 2020-01-01 00:11:06.000 2020-01-02 03:39:27.000 666 99567 50116.5 5011650 666 99567 50116.5 5011650 -32504 32634 4896.66 489666 -128 127 -0.62 -62 +667 2 10657 99568 2.003 299.003 150.503 15050.3003 2.003 299.003 150.503 15050.30029 2.00300 299.00300 150.50300000000004 15050.30000 2020-01-01 2020-01-02 2020-01-01 00:11:07 2020-01-02 03:39:28 2020-01-01 00:11:07.000 2020-01-02 03:39:28.000 667 99568 50117.5 5011750 667 99568 50117.5 5011750 -32503 32635 4897.66 489766 -128 123 -2.18 -218 +668 2 10658 99569 2.006 299.006 150.506 15050.6006 2.006 299.006 150.506 15050.60089 2.00600 299.00600 150.50599999999991 15050.60000 2020-01-01 2020-01-02 2020-01-01 00:11:08 2020-01-02 03:39:29 2020-01-01 00:11:08.000 2020-01-02 03:39:29.000 668 99569 50118.5 5011850 668 99569 50118.5 5011850 -32502 32636 4898.66 489866 -127 124 -1.18 -118 +669 2 10659 99570 2.009 299.009 150.509 15050.9009 2.009 299.009 150.509 15050.90057 2.00900 299.00900 150.509 15050.90000 2020-01-01 2020-01-02 2020-01-01 00:11:09 2020-01-02 03:39:30 2020-01-01 00:11:09.000 2020-01-02 03:39:30.000 669 99570 50119.5 5011950 669 99570 50119.5 5011950 -32501 32637 4899.66 489966 -126 125 -0.18 -18 +67 2 10057 99967 0.2012 300.2012 150.2012 15170.32132 0.2012 300.2012 150.2012 15170.32142 0.20120 300.20120 150.20120000000003 15170.32120 2020-01-01 2020-01-02 2020-01-01 00:01:07 2020-01-02 03:46:07 2020-01-01 00:01:07.000 2020-01-02 03:46:07.000 67 99967 50017 5051717 67 99967 50017 5051717 -32502 32433 4596.009900990099 464197 -128 127 -1.8514851485148516 -187 +670 2 10660 99571 2.01201 299.01201 150.51201 15051.2012 2.01201 299.01202 150.51201 15051.20117 2.01201 299.01201 150.51201000000015 15051.20100 2020-01-01 2020-01-02 2020-01-01 00:11:10 2020-01-02 03:39:31 2020-01-01 00:11:10.000 2020-01-02 03:39:31.000 670 99571 50120.5 5012050 670 99571 50120.5 5012050 -32500 32638 4900.66 490066 -125 126 0.82 82 +671 2 10661 99572 2.01501 299.01501 150.51501 15051.5015 2.01501 299.015 150.51501 15051.50146 2.01501 299.01501 150.51500999999985 15051.50100 2020-01-01 2020-01-02 2020-01-01 00:11:11 2020-01-02 03:39:32 2020-01-01 00:11:11.000 2020-01-02 03:39:32.000 671 99572 50121.5 5012150 671 99572 50121.5 5012150 -32499 32639 4901.66 490166 -124 127 1.82 182 +672 2 10662 99573 2.01801 299.01801 150.51801 15051.8018 2.01801 299.018 150.51801 15051.80176 2.01801 299.01801 150.51800999999986 15051.80100 2020-01-01 2020-01-02 2020-01-01 00:11:12 2020-01-02 03:39:33 2020-01-01 00:11:12.000 2020-01-02 03:39:33.000 672 99573 50122.5 5012250 672 99573 50122.5 5012250 -32498 32640 4902.66 490266 -128 127 0.26 26 +673 2 10663 99574 2.02102 299.02102 150.52102 15052.1021 2.02102 299.02103 150.52102 15052.1024 2.02102 299.02102 150.5210200000001 15052.10200 2020-01-01 2020-01-02 2020-01-01 00:11:13 2020-01-02 03:39:34 2020-01-01 00:11:13.000 2020-01-02 03:39:34.000 673 99574 50123.5 5012350 673 99574 50123.5 5012350 -32497 32641 4903.66 490366 -128 123 -1.3 -130 +674 2 10664 99575 2.02402 299.02402 150.52402 15052.4024 2.02402 299.02402 150.52402 15052.40204 2.02402 299.02402 150.52402000000006 15052.40200 2020-01-01 2020-01-02 2020-01-01 00:11:14 2020-01-02 03:39:35 2020-01-01 00:11:14.000 2020-01-02 03:39:35.000 674 99575 50124.5 5012450 674 99575 50124.5 5012450 -32496 32642 4904.66 490466 -127 124 -0.3 -30 +675 2 10665 99576 2.02702 299.02702 150.52702 15052.7027 2.02702 299.02704 150.52702 15052.70264 2.02702 299.02702 150.5270199999999 15052.70200 2020-01-01 2020-01-02 2020-01-01 00:11:15 2020-01-02 03:39:36 2020-01-01 00:11:15.000 2020-01-02 03:39:36.000 675 99576 50125.5 5012550 675 99576 50125.5 5012550 -32495 32643 4905.66 490566 -126 125 0.7 70 +676 2 10666 99577 2.03003 299.03003 150.53003 15053.003 2.03003 299.03003 150.53002 15053.00293 2.03003 299.03003 150.53002999999993 15053.00300 2020-01-01 2020-01-02 2020-01-01 00:11:16 2020-01-02 03:39:37 2020-01-01 00:11:16.000 2020-01-02 03:39:37.000 676 99577 50126.5 5012650 676 99577 50126.5 5012650 -32494 32644 4906.66 490666 -125 126 1.7 170 677 2 10667 99578 2.03303 299.03303 150.53303 15053.3033 2.03303 299.03302 150.53303 15053.30323 2.03303 299.03303 150.53303 15053.30300 2020-01-01 2020-01-02 2020-01-01 00:11:17 2020-01-02 03:39:38 2020-01-01 00:11:17.000 2020-01-02 03:39:38.000 677 99578 50127.5 5012750 677 99578 50127.5 5012750 -32493 32645 4907.66 490766 -124 127 2.7 270 -678 2 10668 99579 2.03603 299.03603 150.53603 15053.6036 2.03603 299.03604 150.53603 15053.60387 2.03603 299.03603 150.53603 15053.60300 2020-01-01 2020-01-02 2020-01-01 00:11:18 2020-01-02 03:39:39 2020-01-01 00:11:18.000 2020-01-02 03:39:39.000 678 99579 50128.5 5012850 678 99579 50128.5 5012850 -32492 32646 4908.66 490866 -128 127 1.14 114 -679 2 10669 99580 2.03903 299.03903 150.53903 15053.9039 2.03903 299.03903 150.53903 15053.90351 2.03903 299.03903 150.53903 15053.90300 2020-01-01 2020-01-02 2020-01-01 00:11:19 2020-01-02 03:39:40 2020-01-01 00:11:19.000 2020-01-02 03:39:40.000 679 99580 50129.5 5012950 679 99580 50129.5 5012950 -32491 32647 4909.66 490966 -128 123 -0.42 -42 -68 2 10058 99968 0.2042 300.2042 150.2042 15170.62462 0.2042 300.2042 150.2042 15170.62457 0.20420 300.20420 150.20420 15170.62420 2020-01-01 2020-01-02 2020-01-01 00:01:08 2020-01-02 03:46:08 2020-01-01 00:01:08.000 2020-01-02 03:46:08.000 68 99968 50018 5051818 68 99968 50018 5051818 -32501 32434 4597.009900990099 464298 -128 124 -3.386138613861386 -342 -680 2 10670 99581 2.04204 299.04204 150.54204 15054.2042 2.04204 299.04205 150.54204 15054.20426 2.04204 299.04204 150.54204 15054.20400 2020-01-01 2020-01-02 2020-01-01 00:11:20 2020-01-02 03:39:41 2020-01-01 00:11:20.000 2020-01-02 03:39:41.000 680 99581 50130.5 5013050 680 99581 50130.5 5013050 -32490 32648 4910.66 491066 -127 124 0.58 58 -681 2 10671 99582 2.04504 299.04504 150.54504 15054.5045 2.04504 299.04504 150.54504 15054.5044 2.04504 299.04504 150.54504 15054.50400 2020-01-01 2020-01-02 2020-01-01 00:11:21 2020-01-02 03:39:42 2020-01-01 00:11:21.000 2020-01-02 03:39:42.000 681 99582 50131.5 5013150 681 99582 50131.5 5013150 -32489 32649 4911.66 491166 -126 125 1.58 158 -682 2 10672 99583 2.04804 299.04804 150.54804 15054.8048 2.04804 299.04803 150.54804 15054.80474 2.04804 299.04804 150.54804 15054.80400 2020-01-01 2020-01-02 2020-01-01 00:11:22 2020-01-02 03:39:43 2020-01-01 00:11:22.000 2020-01-02 03:39:43.000 682 99583 50132.5 5013250 682 99583 50132.5 5013250 -32488 32650 4912.66 491266 -125 126 2.58 258 +678 2 10668 99579 2.03603 299.03603 150.53603 15053.6036 2.03603 299.03604 150.53603 15053.60387 2.03603 299.03603 150.53602999999993 15053.60300 2020-01-01 2020-01-02 2020-01-01 00:11:18 2020-01-02 03:39:39 2020-01-01 00:11:18.000 2020-01-02 03:39:39.000 678 99579 50128.5 5012850 678 99579 50128.5 5012850 -32492 32646 4908.66 490866 -128 127 1.14 114 +679 2 10669 99580 2.03903 299.03903 150.53903 15053.9039 2.03903 299.03903 150.53903 15053.90351 2.03903 299.03903 150.53902999999994 15053.90300 2020-01-01 2020-01-02 2020-01-01 00:11:19 2020-01-02 03:39:40 2020-01-01 00:11:19.000 2020-01-02 03:39:40.000 679 99580 50129.5 5012950 679 99580 50129.5 5012950 -32491 32647 4909.66 490966 -128 123 -0.42 -42 +68 2 10058 99968 0.2042 300.2042 150.2042 15170.62462 0.2042 300.2042 150.2042 15170.62457 0.20420 300.20420 150.2042000000001 15170.62420 2020-01-01 2020-01-02 2020-01-01 00:01:08 2020-01-02 03:46:08 2020-01-01 00:01:08.000 2020-01-02 03:46:08.000 68 99968 50018 5051818 68 99968 50018 5051818 -32501 32434 4597.009900990099 464298 -128 124 -3.386138613861386 -342 +680 2 10670 99581 2.04204 299.04204 150.54204 15054.2042 2.04204 299.04205 150.54204 15054.20426 2.04204 299.04204 150.5420400000001 15054.20400 2020-01-01 2020-01-02 2020-01-01 00:11:20 2020-01-02 03:39:41 2020-01-01 00:11:20.000 2020-01-02 03:39:41.000 680 99581 50130.5 5013050 680 99581 50130.5 5013050 -32490 32648 4910.66 491066 -127 124 0.58 58 +681 2 10671 99582 2.04504 299.04504 150.54504 15054.5045 2.04504 299.04504 150.54504 15054.5044 2.04504 299.04504 150.5450400000002 15054.50400 2020-01-01 2020-01-02 2020-01-01 00:11:21 2020-01-02 03:39:42 2020-01-01 00:11:21.000 2020-01-02 03:39:42.000 681 99582 50131.5 5013150 681 99582 50131.5 5013150 -32489 32649 4911.66 491166 -126 125 1.58 158 +682 2 10672 99583 2.04804 299.04804 150.54804 15054.8048 2.04804 299.04803 150.54804 15054.80474 2.04804 299.04804 150.54803999999984 15054.80400 2020-01-01 2020-01-02 2020-01-01 00:11:22 2020-01-02 03:39:43 2020-01-01 00:11:22.000 2020-01-02 03:39:43.000 682 99583 50132.5 5013250 682 99583 50132.5 5013250 -32488 32650 4912.66 491266 -125 126 2.58 258 683 2 10673 99584 2.05105 299.05105 150.55105 15055.1051 2.05105 299.05106 150.55105 15055.10533 2.05105 299.05105 150.55105 15055.10500 2020-01-01 2020-01-02 2020-01-01 00:11:23 2020-01-02 03:39:44 2020-01-01 00:11:23.000 2020-01-02 03:39:44.000 683 99584 50133.5 5013350 683 99584 50133.5 5013350 -32487 32651 4913.66 491366 -124 127 3.58 358 -684 2 10674 99585 2.05405 299.05405 150.55405 15055.4054 2.05405 299.05405 150.55404 15055.40498 2.05405 299.05405 150.55405 15055.40500 2020-01-01 2020-01-02 2020-01-01 00:11:24 2020-01-02 03:39:45 2020-01-01 00:11:24.000 2020-01-02 03:39:45.000 684 99585 50134.5 5013450 684 99585 50134.5 5013450 -32486 32652 4914.66 491466 -128 127 2.02 202 -685 2 10675 99586 2.05705 299.05705 150.55705 15055.7057 2.05705 299.05707 150.55705 15055.70573 2.05705 299.05705 150.55705 15055.70500 2020-01-01 2020-01-02 2020-01-01 00:11:25 2020-01-02 03:39:46 2020-01-01 00:11:25.000 2020-01-02 03:39:46.000 685 99586 50135.5 5013550 685 99586 50135.5 5013550 -32485 32653 4915.66 491566 -128 127 0.46 46 -686 2 10676 99587 2.06006 299.06006 150.56006 15056.006 2.06006 299.06006 150.56005 15056.00587 2.06006 299.06006 150.56006 15056.00600 2020-01-01 2020-01-02 2020-01-01 00:11:26 2020-01-02 03:39:47 2020-01-01 00:11:26.000 2020-01-02 03:39:47.000 686 99587 50136.5 5013650 686 99587 50136.5 5013650 -32484 32654 4916.66 491666 -128 124 -1.1 -110 -687 2 10677 99588 2.06306 299.06306 150.56306 15056.3063 2.06306 299.06305 150.56306 15056.30621 2.06306 299.06306 150.56306 15056.30600 2020-01-01 2020-01-02 2020-01-01 00:11:27 2020-01-02 03:39:48 2020-01-01 00:11:27.000 2020-01-02 03:39:48.000 687 99588 50137.5 5013750 687 99588 50137.5 5013750 -32483 32655 4917.66 491766 -127 125 -0.1 -10 -688 2 10678 99589 2.06606 299.06606 150.56606 15056.6066 2.06606 299.06607 150.56606 15056.60681 2.06606 299.06606 150.56606 15056.60600 2020-01-01 2020-01-02 2020-01-01 00:11:28 2020-01-02 03:39:49 2020-01-01 00:11:28.000 2020-01-02 03:39:49.000 688 99589 50138.5 5013850 688 99589 50138.5 5013850 -32482 32656 4918.66 491866 -126 126 0.9 90 -689 2 10679 99590 2.06906 299.06906 150.56906 15056.9069 2.06906 299.06906 150.56907 15056.9071 2.06906 299.06906 150.56906 15056.90600 2020-01-01 2020-01-02 2020-01-01 00:11:29 2020-01-02 03:39:50 2020-01-01 00:11:29.000 2020-01-02 03:39:50.000 689 99590 50139.5 5013950 689 99590 50139.5 5013950 -32481 32657 4919.66 491966 -125 127 1.9 190 -69 2 10059 99969 0.2072 300.2072 150.2072 15170.92792 0.2072 300.2072 150.20721 15170.92832 0.20720 300.20720 150.20720 15170.92720 2020-01-01 2020-01-02 2020-01-01 00:01:09 2020-01-02 03:46:09 2020-01-01 00:01:09.000 2020-01-02 03:46:09.000 69 99969 50019 5051919 69 99969 50019 5051919 -32500 32435 4598.009900990099 464399 -127 125 -2.386138613861386 -241 -690 2 10680 99591 2.07207 299.07207 150.57207 15057.2072 2.07207 299.07208 150.57207 15057.2072 2.07207 299.07207 150.57207 15057.20700 2020-01-01 2020-01-02 2020-01-01 00:11:30 2020-01-02 03:39:51 2020-01-01 00:11:30.000 2020-01-02 03:39:51.000 690 99591 50140.5 5014050 690 99591 50140.5 5014050 -32480 32658 4920.66 492066 -128 127 0.34 34 -691 2 10681 99592 2.07507 299.07507 150.57507 15057.5075 2.07507 299.07507 150.57507 15057.50734 2.07507 299.07507 150.57507 15057.50700 2020-01-01 2020-01-02 2020-01-01 00:11:31 2020-01-02 03:39:52 2020-01-01 00:11:31.000 2020-01-02 03:39:52.000 691 99592 50141.5 5014150 691 99592 50141.5 5014150 -32479 32659 4921.66 492166 -128 127 -1.22 -122 -692 2 10682 99593 2.07807 299.07807 150.57807 15057.8078 2.07807 299.07806 150.57807 15057.80767 2.07807 299.07807 150.57807 15057.80700 2020-01-01 2020-01-02 2020-01-01 00:11:32 2020-01-02 03:39:53 2020-01-01 00:11:32.000 2020-01-02 03:39:53.000 692 99593 50142.5 5014250 692 99593 50142.5 5014250 -32478 32660 4922.66 492266 -128 123 -2.78 -278 -693 2 10683 99594 2.08108 299.08108 150.58108 15058.1081 2.08108 299.0811 150.58108 15058.10827 2.08108 299.08108 150.58108 15058.10800 2020-01-01 2020-01-02 2020-01-01 00:11:33 2020-01-02 03:39:54 2020-01-01 00:11:33.000 2020-01-02 03:39:54.000 693 99594 50143.5 5014350 693 99594 50143.5 5014350 -32477 32661 4923.66 492366 -127 124 -1.78 -178 -694 2 10684 99595 2.08408 299.08408 150.58408 15058.4084 2.08408 299.08408 150.58408 15058.40857 2.08408 299.08408 150.58408 15058.40800 2020-01-01 2020-01-02 2020-01-01 00:11:34 2020-01-02 03:39:55 2020-01-01 00:11:34.000 2020-01-02 03:39:55.000 694 99595 50144.5 5014450 694 99595 50144.5 5014450 -32476 32662 4924.66 492466 -126 125 -0.78 -78 -695 2 10685 99596 2.08708 299.08708 150.58708 15058.7087 2.08708 299.0871 150.58708 15058.70867 2.08708 299.08708 150.58708 15058.70800 2020-01-01 2020-01-02 2020-01-01 00:11:35 2020-01-02 03:39:56 2020-01-01 00:11:35.000 2020-01-02 03:39:56.000 695 99596 50145.5 5014550 695 99596 50145.5 5014550 -32475 32663 4925.66 492566 -125 126 0.22 22 -696 2 10686 99597 2.09009 299.09009 150.59009 15059.009 2.09009 299.0901 150.59008 15059.00885 2.09009 299.09009 150.59009 15059.00900 2020-01-01 2020-01-02 2020-01-01 00:11:36 2020-01-02 03:39:57 2020-01-01 00:11:36.000 2020-01-02 03:39:57.000 696 99597 50146.5 5014650 696 99597 50146.5 5014650 -32474 32664 4926.66 492666 -124 127 1.22 122 -697 2 10687 99598 2.09309 299.09309 150.59309 15059.3093 2.09309 299.09308 150.59309 15059.30915 2.09309 299.09309 150.59309 15059.30900 2020-01-01 2020-01-02 2020-01-01 00:11:37 2020-01-02 03:39:58 2020-01-01 00:11:37.000 2020-01-02 03:39:58.000 697 99598 50147.5 5014750 697 99598 50147.5 5014750 -32473 32665 4927.66 492766 -128 127 -0.34 -34 -698 2 10688 99599 2.09609 299.09609 150.59609 15059.6096 2.09609 299.0961 150.59609 15059.6099 2.09609 299.09609 150.59609 15059.60900 2020-01-01 2020-01-02 2020-01-01 00:11:38 2020-01-02 03:39:59 2020-01-01 00:11:38.000 2020-01-02 03:39:59.000 698 99599 50148.5 5014850 698 99599 50148.5 5014850 -32472 32666 4928.66 492866 -128 123 -1.9 -190 -699 2 10689 99600 2.09909 299.09909 150.59909 15059.9099 2.09909 299.0991 150.5991 15059.91003 2.09909 299.09909 150.59909 15059.90900 2020-01-01 2020-01-02 2020-01-01 00:11:39 2020-01-02 03:40:00 2020-01-01 00:11:39.000 2020-01-02 03:40:00.000 699 99600 50149.5 5014950 699 99600 50149.5 5014950 -32471 32667 4929.66 492966 -127 124 -0.9 -90 -7 2 1006 9997 0.02102 300.02102 150.02102 15152.12312 0.02102 300.02103 150.02102 15152.12342 0.02102 300.02102 150.02102 15152.12302 2020-01-01 2020-01-02 2020-01-01 00:00:07 2020-01-02 03:45:07 2020-01-01 00:00:07.000 2020-01-02 03:45:07.000 7 99907 49957 5045657 7 99907 49957 5045657 -32562 32373 4536.009900990099 458137 -126 125 -1.0198019801980198 -103 -70 2 10060 99970 0.21021 300.21021 150.21021 15171.23123 0.21021 300.2102 150.2102 15171.23097 0.21021 300.21021 150.21021 15171.23121 2020-01-01 2020-01-02 2020-01-01 00:01:10 2020-01-02 03:46:10 2020-01-01 00:01:10.000 2020-01-02 03:46:10.000 70 99970 50020 5052020 70 99970 50020 5052020 -32499 32436 4599.009900990099 464500 -126 126 -1.386138613861386 -140 -700 2 10690 99601 2.1021 299.1021 150.6021 15060.21021 2.1021 299.1021 150.6021 15060.21014 2.10210 299.10210 150.60210 15060.21000 2020-01-01 2020-01-02 2020-01-01 00:11:40 2020-01-02 03:40:01 2020-01-01 00:11:40.000 2020-01-02 03:40:01.000 700 99601 50150.5 5015050 700 99601 50150.5 5015050 -32470 32668 4930.66 493066 -126 125 0.1 10 -701 2 10691 99602 2.1051 299.1051 150.6051 15060.51051 2.1051 299.1051 150.6051 15060.51031 2.10510 299.10510 150.60510 15060.51000 2020-01-01 2020-01-02 2020-01-01 00:11:41 2020-01-02 03:40:02 2020-01-01 00:11:41.000 2020-01-02 03:40:02.000 701 99602 50151.5 5015150 701 99602 50151.5 5015150 -32469 32669 4931.66 493166 -125 126 1.1 110 -702 2 10692 99603 2.1081 299.1081 150.6081 15060.81081 2.1081 299.1081 150.6081 15060.81062 2.10810 299.10810 150.60810 15060.81000 2020-01-01 2020-01-02 2020-01-01 00:11:42 2020-01-02 03:40:03 2020-01-01 00:11:42.000 2020-01-02 03:40:03.000 702 99603 50152.5 5015250 702 99603 50152.5 5015250 -32468 32670 4932.66 493266 -124 127 2.1 210 -703 2 10693 99604 2.11111 299.11111 150.61111 15061.11111 2.11111 299.1111 150.61111 15061.11137 2.11111 299.11111 150.61111 15061.11100 2020-01-01 2020-01-02 2020-01-01 00:11:43 2020-01-02 03:40:04 2020-01-01 00:11:43.000 2020-01-02 03:40:04.000 703 99604 50153.5 5015350 703 99604 50153.5 5015350 -32467 32671 4933.66 493366 -128 127 0.54 54 -704 2 10694 99605 2.11411 299.11411 150.61411 15061.41141 2.11411 299.1141 150.61411 15061.41151 2.11411 299.11411 150.61411 15061.41100 2020-01-01 2020-01-02 2020-01-01 00:11:44 2020-01-02 03:40:05 2020-01-01 00:11:44.000 2020-01-02 03:40:05.000 704 99605 50154.5 5015450 704 99605 50154.5 5015450 -32466 32672 4934.66 493466 -128 123 -1.02 -102 -705 2 10695 99606 2.11711 299.11711 150.61711 15061.71171 2.11711 299.11713 150.61711 15061.71165 2.11711 299.11711 150.61711 15061.71100 2020-01-01 2020-01-02 2020-01-01 00:11:45 2020-01-02 03:40:06 2020-01-01 00:11:45.000 2020-01-02 03:40:06.000 705 99606 50155.5 5015550 705 99606 50155.5 5015550 -32465 32673 4935.66 493566 -127 124 -0.02 -2 -706 2 10696 99607 2.12012 299.12012 150.62012 15062.01201 2.12012 299.12012 150.62011 15062.01179 2.12012 299.12012 150.62012 15062.01200 2020-01-01 2020-01-02 2020-01-01 00:11:46 2020-01-02 03:40:07 2020-01-01 00:11:46.000 2020-01-02 03:40:07.000 706 99607 50156.5 5015650 706 99607 50156.5 5015650 -32464 32674 4936.66 493666 -126 125 0.98 98 -707 2 10697 99608 2.12312 299.12312 150.62312 15062.31231 2.12312 299.1231 150.62312 15062.31208 2.12312 299.12312 150.62312 15062.31200 2020-01-01 2020-01-02 2020-01-01 00:11:47 2020-01-02 03:40:08 2020-01-01 00:11:47.000 2020-01-02 03:40:08.000 707 99608 50157.5 5015750 707 99608 50157.5 5015750 -32463 32675 4937.66 493766 -125 126 1.98 198 -708 2 10698 99609 2.12612 299.12612 150.62612 15062.61261 2.12612 299.12613 150.62612 15062.61283 2.12612 299.12612 150.62612 15062.61200 2020-01-01 2020-01-02 2020-01-01 00:11:48 2020-01-02 03:40:09 2020-01-01 00:11:48.000 2020-01-02 03:40:09.000 708 99609 50158.5 5015850 708 99609 50158.5 5015850 -32462 32676 4938.66 493866 -124 127 2.98 298 -709 2 10699 99610 2.12912 299.12912 150.62912 15062.91291 2.12912 299.12912 150.62912 15062.91298 2.12912 299.12912 150.62912 15062.91200 2020-01-01 2020-01-02 2020-01-01 00:11:49 2020-01-02 03:40:10 2020-01-01 00:11:49.000 2020-01-02 03:40:10.000 709 99610 50159.5 5015950 709 99610 50159.5 5015950 -32461 32677 4939.66 493966 -128 127 1.42 142 -71 2 10061 99971 0.21321 300.21321 150.21321 15171.53453 0.21321 300.21323 150.21321 15171.5346 0.21321 300.21321 150.21321 15171.53421 2020-01-01 2020-01-02 2020-01-01 00:01:11 2020-01-02 03:46:11 2020-01-01 00:01:11.000 2020-01-02 03:46:11.000 71 99971 50021 5052121 71 99971 50021 5052121 -32498 32437 4600.009900990099 464601 -125 127 -0.38613861386138615 -39 +684 2 10674 99585 2.05405 299.05405 150.55405 15055.4054 2.05405 299.05405 150.55404 15055.40498 2.05405 299.05405 150.55405000000013 15055.40500 2020-01-01 2020-01-02 2020-01-01 00:11:24 2020-01-02 03:39:45 2020-01-01 00:11:24.000 2020-01-02 03:39:45.000 684 99585 50134.5 5013450 684 99585 50134.5 5013450 -32486 32652 4914.66 491466 -128 127 2.02 202 +685 2 10675 99586 2.05705 299.05705 150.55705 15055.7057 2.05705 299.05707 150.55705 15055.70573 2.05705 299.05705 150.55704999999992 15055.70500 2020-01-01 2020-01-02 2020-01-01 00:11:25 2020-01-02 03:39:46 2020-01-01 00:11:25.000 2020-01-02 03:39:46.000 685 99586 50135.5 5013550 685 99586 50135.5 5013550 -32485 32653 4915.66 491566 -128 127 0.46 46 +686 2 10676 99587 2.06006 299.06006 150.56006 15056.006 2.06006 299.06006 150.56005 15056.00587 2.06006 299.06006 150.56005999999988 15056.00600 2020-01-01 2020-01-02 2020-01-01 00:11:26 2020-01-02 03:39:47 2020-01-01 00:11:26.000 2020-01-02 03:39:47.000 686 99587 50136.5 5013650 686 99587 50136.5 5013650 -32484 32654 4916.66 491666 -128 124 -1.1 -110 +687 2 10677 99588 2.06306 299.06306 150.56306 15056.3063 2.06306 299.06305 150.56306 15056.30621 2.06306 299.06306 150.5630600000002 15056.30600 2020-01-01 2020-01-02 2020-01-01 00:11:27 2020-01-02 03:39:48 2020-01-01 00:11:27.000 2020-01-02 03:39:48.000 687 99588 50137.5 5013750 687 99588 50137.5 5013750 -32483 32655 4917.66 491766 -127 125 -0.1 -10 +688 2 10678 99589 2.06606 299.06606 150.56606 15056.6066 2.06606 299.06607 150.56606 15056.60681 2.06606 299.06606 150.56605999999988 15056.60600 2020-01-01 2020-01-02 2020-01-01 00:11:28 2020-01-02 03:39:49 2020-01-01 00:11:28.000 2020-01-02 03:39:49.000 688 99589 50138.5 5013850 688 99589 50138.5 5013850 -32482 32656 4918.66 491866 -126 126 0.9 90 +689 2 10679 99590 2.06906 299.06906 150.56906 15056.9069 2.06906 299.06906 150.56907 15056.9071 2.06906 299.06906 150.56905999999987 15056.90600 2020-01-01 2020-01-02 2020-01-01 00:11:29 2020-01-02 03:39:50 2020-01-01 00:11:29.000 2020-01-02 03:39:50.000 689 99590 50139.5 5013950 689 99590 50139.5 5013950 -32481 32657 4919.66 491966 -125 127 1.9 190 +69 2 10059 99969 0.2072 300.2072 150.2072 15170.92792 0.2072 300.2072 150.20721 15170.92832 0.20720 300.20720 150.20720000000009 15170.92720 2020-01-01 2020-01-02 2020-01-01 00:01:09 2020-01-02 03:46:09 2020-01-01 00:01:09.000 2020-01-02 03:46:09.000 69 99969 50019 5051919 69 99969 50019 5051919 -32500 32435 4598.009900990099 464399 -127 125 -2.386138613861386 -241 +690 2 10680 99591 2.07207 299.07207 150.57207 15057.2072 2.07207 299.07208 150.57207 15057.2072 2.07207 299.07207 150.5720700000001 15057.20700 2020-01-01 2020-01-02 2020-01-01 00:11:30 2020-01-02 03:39:51 2020-01-01 00:11:30.000 2020-01-02 03:39:51.000 690 99591 50140.5 5014050 690 99591 50140.5 5014050 -32480 32658 4920.66 492066 -128 127 0.34 34 +691 2 10681 99592 2.07507 299.07507 150.57507 15057.5075 2.07507 299.07507 150.57507 15057.50734 2.07507 299.07507 150.57507000000012 15057.50700 2020-01-01 2020-01-02 2020-01-01 00:11:31 2020-01-02 03:39:52 2020-01-01 00:11:31.000 2020-01-02 03:39:52.000 691 99592 50141.5 5014150 691 99592 50141.5 5014150 -32479 32659 4921.66 492166 -128 127 -1.22 -122 +692 2 10682 99593 2.07807 299.07807 150.57807 15057.8078 2.07807 299.07806 150.57807 15057.80767 2.07807 299.07807 150.5780699999998 15057.80700 2020-01-01 2020-01-02 2020-01-01 00:11:32 2020-01-02 03:39:53 2020-01-01 00:11:32.000 2020-01-02 03:39:53.000 692 99593 50142.5 5014250 692 99593 50142.5 5014250 -32478 32660 4922.66 492266 -128 123 -2.78 -278 +693 2 10683 99594 2.08108 299.08108 150.58108 15058.1081 2.08108 299.0811 150.58108 15058.10827 2.08108 299.08108 150.58107999999996 15058.10800 2020-01-01 2020-01-02 2020-01-01 00:11:33 2020-01-02 03:39:54 2020-01-01 00:11:33.000 2020-01-02 03:39:54.000 693 99594 50143.5 5014350 693 99594 50143.5 5014350 -32477 32661 4923.66 492366 -127 124 -1.78 -178 +694 2 10684 99595 2.08408 299.08408 150.58408 15058.4084 2.08408 299.08408 150.58408 15058.40857 2.08408 299.08408 150.58408000000003 15058.40800 2020-01-01 2020-01-02 2020-01-01 00:11:34 2020-01-02 03:39:55 2020-01-01 00:11:34.000 2020-01-02 03:39:55.000 694 99595 50144.5 5014450 694 99595 50144.5 5014450 -32476 32662 4924.66 492466 -126 125 -0.78 -78 +695 2 10685 99596 2.08708 299.08708 150.58708 15058.7087 2.08708 299.0871 150.58708 15058.70867 2.08708 299.08708 150.5870799999999 15058.70800 2020-01-01 2020-01-02 2020-01-01 00:11:35 2020-01-02 03:39:56 2020-01-01 00:11:35.000 2020-01-02 03:39:56.000 695 99596 50145.5 5014550 695 99596 50145.5 5014550 -32475 32663 4925.66 492566 -125 126 0.22 22 +696 2 10686 99597 2.09009 299.09009 150.59009 15059.009 2.09009 299.0901 150.59008 15059.00885 2.09009 299.09009 150.59008999999986 15059.00900 2020-01-01 2020-01-02 2020-01-01 00:11:36 2020-01-02 03:39:57 2020-01-01 00:11:36.000 2020-01-02 03:39:57.000 696 99597 50146.5 5014650 696 99597 50146.5 5014650 -32474 32664 4926.66 492666 -124 127 1.22 122 +697 2 10687 99598 2.09309 299.09309 150.59309 15059.3093 2.09309 299.09308 150.59309 15059.30915 2.09309 299.09309 150.59309000000013 15059.30900 2020-01-01 2020-01-02 2020-01-01 00:11:37 2020-01-02 03:39:58 2020-01-01 00:11:37.000 2020-01-02 03:39:58.000 697 99598 50147.5 5014750 697 99598 50147.5 5014750 -32473 32665 4927.66 492766 -128 127 -0.34 -34 +698 2 10688 99599 2.09609 299.09609 150.59609 15059.6096 2.09609 299.0961 150.59609 15059.6099 2.09609 299.09609 150.5960899999998 15059.60900 2020-01-01 2020-01-02 2020-01-01 00:11:38 2020-01-02 03:39:59 2020-01-01 00:11:38.000 2020-01-02 03:39:59.000 698 99599 50148.5 5014850 698 99599 50148.5 5014850 -32472 32666 4928.66 492866 -128 123 -1.9 -190 +699 2 10689 99600 2.09909 299.09909 150.59909 15059.9099 2.09909 299.0991 150.5991 15059.91003 2.09909 299.09909 150.59908999999988 15059.90900 2020-01-01 2020-01-02 2020-01-01 00:11:39 2020-01-02 03:40:00 2020-01-01 00:11:39.000 2020-01-02 03:40:00.000 699 99600 50149.5 5014950 699 99600 50149.5 5014950 -32471 32667 4929.66 492966 -127 124 -0.9 -90 +7 2 1006 9997 0.02102 300.02102 150.02102 15152.12312 0.02102 300.02103 150.02102 15152.12342 0.02102 300.02102 150.0210200000001 15152.12302 2020-01-01 2020-01-02 2020-01-01 00:00:07 2020-01-02 03:45:07 2020-01-01 00:00:07.000 2020-01-02 03:45:07.000 7 99907 49957 5045657 7 99907 49957 5045657 -32562 32373 4536.009900990099 458137 -126 125 -1.0198019801980198 -103 +70 2 10060 99970 0.21021 300.21021 150.21021 15171.23123 0.21021 300.2102 150.2102 15171.23097 0.21021 300.21021 150.2102099999999 15171.23121 2020-01-01 2020-01-02 2020-01-01 00:01:10 2020-01-02 03:46:10 2020-01-01 00:01:10.000 2020-01-02 03:46:10.000 70 99970 50020 5052020 70 99970 50020 5052020 -32499 32436 4599.009900990099 464500 -126 126 -1.386138613861386 -140 +700 2 10690 99601 2.1021 299.1021 150.6021 15060.21021 2.1021 299.1021 150.6021 15060.21014 2.10210 299.10210 150.60210000000004 15060.21000 2020-01-01 2020-01-02 2020-01-01 00:11:40 2020-01-02 03:40:01 2020-01-01 00:11:40.000 2020-01-02 03:40:01.000 700 99601 50150.5 5015050 700 99601 50150.5 5015050 -32470 32668 4930.66 493066 -126 125 0.1 10 +701 2 10691 99602 2.1051 299.1051 150.6051 15060.51051 2.1051 299.1051 150.6051 15060.51031 2.10510 299.10510 150.60510000000005 15060.51000 2020-01-01 2020-01-02 2020-01-01 00:11:41 2020-01-02 03:40:02 2020-01-01 00:11:41.000 2020-01-02 03:40:02.000 701 99602 50151.5 5015150 701 99602 50151.5 5015150 -32469 32669 4931.66 493166 -125 126 1.1 110 +702 2 10692 99603 2.1081 299.1081 150.6081 15060.81081 2.1081 299.1081 150.6081 15060.81062 2.10810 299.10810 150.60809999999998 15060.81000 2020-01-01 2020-01-02 2020-01-01 00:11:42 2020-01-02 03:40:03 2020-01-01 00:11:42.000 2020-01-02 03:40:03.000 702 99603 50152.5 5015250 702 99603 50152.5 5015250 -32468 32670 4932.66 493266 -124 127 2.1 210 +703 2 10693 99604 2.11111 299.11111 150.61111 15061.11111 2.11111 299.1111 150.61111 15061.11137 2.11111 299.11111 150.61110999999988 15061.11100 2020-01-01 2020-01-02 2020-01-01 00:11:43 2020-01-02 03:40:04 2020-01-01 00:11:43.000 2020-01-02 03:40:04.000 703 99604 50153.5 5015350 703 99604 50153.5 5015350 -32467 32671 4933.66 493366 -128 127 0.54 54 +704 2 10694 99605 2.11411 299.11411 150.61411 15061.41141 2.11411 299.1141 150.61411 15061.41151 2.11411 299.11411 150.6141100000002 15061.41100 2020-01-01 2020-01-02 2020-01-01 00:11:44 2020-01-02 03:40:05 2020-01-01 00:11:44.000 2020-01-02 03:40:05.000 704 99605 50154.5 5015450 704 99605 50154.5 5015450 -32466 32672 4934.66 493466 -128 123 -1.02 -102 +705 2 10695 99606 2.11711 299.11711 150.61711 15061.71171 2.11711 299.11713 150.61711 15061.71165 2.11711 299.11711 150.6171099999999 15061.71100 2020-01-01 2020-01-02 2020-01-01 00:11:45 2020-01-02 03:40:06 2020-01-01 00:11:45.000 2020-01-02 03:40:06.000 705 99606 50155.5 5015550 705 99606 50155.5 5015550 -32465 32673 4935.66 493566 -127 124 -0.02 -2 +706 2 10696 99607 2.12012 299.12012 150.62012 15062.01201 2.12012 299.12012 150.62011 15062.01179 2.12012 299.12012 150.6201199999998 15062.01200 2020-01-01 2020-01-02 2020-01-01 00:11:46 2020-01-02 03:40:07 2020-01-01 00:11:46.000 2020-01-02 03:40:07.000 706 99607 50156.5 5015650 706 99607 50156.5 5015650 -32464 32674 4936.66 493666 -126 125 0.98 98 +707 2 10697 99608 2.12312 299.12312 150.62312 15062.31231 2.12312 299.1231 150.62312 15062.31208 2.12312 299.12312 150.62312000000014 15062.31200 2020-01-01 2020-01-02 2020-01-01 00:11:47 2020-01-02 03:40:08 2020-01-01 00:11:47.000 2020-01-02 03:40:08.000 707 99608 50157.5 5015750 707 99608 50157.5 5015750 -32463 32675 4937.66 493766 -125 126 1.98 198 +708 2 10698 99609 2.12612 299.12612 150.62612 15062.61261 2.12612 299.12613 150.62612 15062.61283 2.12612 299.12612 150.62612000000013 15062.61200 2020-01-01 2020-01-02 2020-01-01 00:11:48 2020-01-02 03:40:09 2020-01-01 00:11:48.000 2020-01-02 03:40:09.000 708 99609 50158.5 5015850 708 99609 50158.5 5015850 -32462 32676 4938.66 493866 -124 127 2.98 298 +709 2 10699 99610 2.12912 299.12912 150.62912 15062.91291 2.12912 299.12912 150.62912 15062.91298 2.12912 299.12912 150.6291199999998 15062.91200 2020-01-01 2020-01-02 2020-01-01 00:11:49 2020-01-02 03:40:10 2020-01-01 00:11:49.000 2020-01-02 03:40:10.000 709 99610 50159.5 5015950 709 99610 50159.5 5015950 -32461 32677 4939.66 493966 -128 127 1.42 142 +71 2 10061 99971 0.21321 300.21321 150.21321 15171.53453 0.21321 300.21323 150.21321 15171.5346 0.21321 300.21321 150.21320999999992 15171.53421 2020-01-01 2020-01-02 2020-01-01 00:01:11 2020-01-02 03:46:11 2020-01-01 00:01:11.000 2020-01-02 03:46:11.000 71 99971 50021 5052121 71 99971 50021 5052121 -32498 32437 4600.009900990099 464601 -125 127 -0.38613861386138615 -39 710 2 10700 99611 2.13213 299.13213 150.63213 15063.21321 2.13213 299.13214 150.63213 15063.21311 2.13213 299.13213 150.63213 15063.21300 2020-01-01 2020-01-02 2020-01-01 00:11:50 2020-01-02 03:40:11 2020-01-01 00:11:50.000 2020-01-02 03:40:11.000 710 99611 50160.5 5016050 710 99611 50160.5 5016050 -32460 32678 4940.66 494066 -128 127 -0.14 -14 -711 2 10701 99612 2.13513 299.13513 150.63513 15063.51351 2.13513 299.13513 150.63513 15063.51325 2.13513 299.13513 150.63513 15063.51300 2020-01-01 2020-01-02 2020-01-01 00:11:51 2020-01-02 03:40:12 2020-01-01 00:11:51.000 2020-01-02 03:40:12.000 711 99612 50161.5 5016150 711 99612 50161.5 5016150 -32459 32679 4941.66 494166 -128 124 -1.7 -170 -712 2 10702 99613 2.13813 299.13813 150.63813 15063.81381 2.13813 299.13815 150.63814 15063.81401 2.13813 299.13813 150.63813 15063.81300 2020-01-01 2020-01-02 2020-01-01 00:11:52 2020-01-02 03:40:13 2020-01-01 00:11:52.000 2020-01-02 03:40:13.000 712 99613 50162.5 5016250 712 99613 50162.5 5016250 -32458 32680 4942.66 494266 -127 125 -0.7 -70 -713 2 10703 99614 2.14114 299.14114 150.64114 15064.11411 2.14114 299.14114 150.64114 15064.11431 2.14114 299.14114 150.64114 15064.11400 2020-01-01 2020-01-02 2020-01-01 00:11:53 2020-01-02 03:40:14 2020-01-01 00:11:53.000 2020-01-02 03:40:14.000 713 99614 50163.5 5016350 713 99614 50163.5 5016350 -32457 32681 4943.66 494366 -126 126 0.3 30 -714 2 10704 99615 2.14414 299.14414 150.64414 15064.41441 2.14414 299.14413 150.64414 15064.41448 2.14414 299.14414 150.64414 15064.41400 2020-01-01 2020-01-02 2020-01-01 00:11:54 2020-01-02 03:40:15 2020-01-01 00:11:54.000 2020-01-02 03:40:15.000 714 99615 50164.5 5016450 714 99615 50164.5 5016450 -32456 32682 4944.66 494466 -125 127 1.3 130 -715 2 10705 99616 2.14714 299.14714 150.64714 15064.71471 2.14714 299.14716 150.64714 15064.71458 2.14714 299.14714 150.64714 15064.71400 2020-01-01 2020-01-02 2020-01-01 00:11:55 2020-01-02 03:40:16 2020-01-01 00:11:55.000 2020-01-02 03:40:16.000 715 99616 50165.5 5016550 715 99616 50165.5 5016550 -32455 32683 4945.66 494566 -128 127 -0.26 -26 -716 2 10706 99617 2.15015 299.15015 150.65015 15065.01501 2.15015 299.15015 150.65014 15065.01472 2.15015 299.15015 150.65015 15065.01500 2020-01-01 2020-01-02 2020-01-01 00:11:56 2020-01-02 03:40:17 2020-01-01 00:11:56.000 2020-01-02 03:40:17.000 716 99617 50166.5 5016650 716 99617 50166.5 5016650 -32454 32684 4946.66 494666 -128 127 -1.82 -182 -717 2 10707 99618 2.15315 299.15315 150.65315 15065.31531 2.15315 299.15317 150.65315 15065.31547 2.15315 299.15315 150.65315 15065.31500 2020-01-01 2020-01-02 2020-01-01 00:11:57 2020-01-02 03:40:18 2020-01-01 00:11:57.000 2020-01-02 03:40:18.000 717 99618 50167.5 5016750 717 99618 50167.5 5016750 -32453 32685 4947.66 494766 -128 123 -3.38 -338 -718 2 10708 99619 2.15615 299.15615 150.65615 15065.61561 2.15615 299.15616 150.65615 15065.61578 2.15615 299.15615 150.65615 15065.61500 2020-01-01 2020-01-02 2020-01-01 00:11:58 2020-01-02 03:40:19 2020-01-01 00:11:58.000 2020-01-02 03:40:19.000 718 99619 50168.5 5016850 718 99619 50168.5 5016850 -32452 32686 4948.66 494866 -127 124 -2.38 -238 +711 2 10701 99612 2.13513 299.13513 150.63513 15063.51351 2.13513 299.13513 150.63513 15063.51325 2.13513 299.13513 150.63513000000006 15063.51300 2020-01-01 2020-01-02 2020-01-01 00:11:51 2020-01-02 03:40:12 2020-01-01 00:11:51.000 2020-01-02 03:40:12.000 711 99612 50161.5 5016150 711 99612 50161.5 5016150 -32459 32679 4941.66 494166 -128 124 -1.7 -170 +712 2 10702 99613 2.13813 299.13813 150.63813 15063.81381 2.13813 299.13815 150.63814 15063.81401 2.13813 299.13813 150.6381299999999 15063.81300 2020-01-01 2020-01-02 2020-01-01 00:11:52 2020-01-02 03:40:13 2020-01-01 00:11:52.000 2020-01-02 03:40:13.000 712 99613 50162.5 5016250 712 99613 50162.5 5016250 -32458 32680 4942.66 494266 -127 125 -0.7 -70 +713 2 10703 99614 2.14114 299.14114 150.64114 15064.11411 2.14114 299.14114 150.64114 15064.11431 2.14114 299.14114 150.6411399999999 15064.11400 2020-01-01 2020-01-02 2020-01-01 00:11:53 2020-01-02 03:40:14 2020-01-01 00:11:53.000 2020-01-02 03:40:14.000 713 99614 50163.5 5016350 713 99614 50163.5 5016350 -32457 32681 4943.66 494366 -126 126 0.3 30 +714 2 10704 99615 2.14414 299.14414 150.64414 15064.41441 2.14414 299.14413 150.64414 15064.41448 2.14414 299.14414 150.64414000000014 15064.41400 2020-01-01 2020-01-02 2020-01-01 00:11:54 2020-01-02 03:40:15 2020-01-01 00:11:54.000 2020-01-02 03:40:15.000 714 99615 50164.5 5016450 714 99615 50164.5 5016450 -32456 32682 4944.66 494466 -125 127 1.3 130 +715 2 10705 99616 2.14714 299.14714 150.64714 15064.71471 2.14714 299.14716 150.64714 15064.71458 2.14714 299.14714 150.6471399999998 15064.71400 2020-01-01 2020-01-02 2020-01-01 00:11:55 2020-01-02 03:40:16 2020-01-01 00:11:55.000 2020-01-02 03:40:16.000 715 99616 50165.5 5016550 715 99616 50165.5 5016550 -32455 32683 4945.66 494566 -128 127 -0.26 -26 +716 2 10706 99617 2.15015 299.15015 150.65015 15065.01501 2.15015 299.15015 150.65014 15065.01472 2.15015 299.15015 150.65015000000002 15065.01500 2020-01-01 2020-01-02 2020-01-01 00:11:56 2020-01-02 03:40:17 2020-01-01 00:11:56.000 2020-01-02 03:40:17.000 716 99617 50166.5 5016650 716 99617 50166.5 5016650 -32454 32684 4946.66 494666 -128 127 -1.82 -182 +717 2 10707 99618 2.15315 299.15315 150.65315 15065.31531 2.15315 299.15317 150.65315 15065.31547 2.15315 299.15315 150.65315000000007 15065.31500 2020-01-01 2020-01-02 2020-01-01 00:11:57 2020-01-02 03:40:18 2020-01-01 00:11:57.000 2020-01-02 03:40:18.000 717 99618 50167.5 5016750 717 99618 50167.5 5016750 -32453 32685 4947.66 494766 -128 123 -3.38 -338 +718 2 10708 99619 2.15615 299.15615 150.65615 15065.61561 2.15615 299.15616 150.65615 15065.61578 2.15615 299.15615 150.65615000000005 15065.61500 2020-01-01 2020-01-02 2020-01-01 00:11:58 2020-01-02 03:40:19 2020-01-01 00:11:58.000 2020-01-02 03:40:19.000 718 99619 50168.5 5016850 718 99619 50168.5 5016850 -32452 32686 4948.66 494866 -127 124 -2.38 -238 719 2 10709 99620 2.15915 299.15915 150.65915 15065.91591 2.15915 299.15915 150.65915 15065.91595 2.15915 299.15915 150.65915 15065.91500 2020-01-01 2020-01-02 2020-01-01 00:11:59 2020-01-02 03:40:20 2020-01-01 00:11:59.000 2020-01-02 03:40:20.000 719 99620 50169.5 5016950 719 99620 50169.5 5016950 -32451 32687 4949.66 494966 -126 125 -1.38 -138 72 2 10062 99972 0.21621 300.21621 150.21621 15171.83783 0.21621 300.21622 150.21621 15171.83791 0.21621 300.21621 150.21621 15171.83721 2020-01-01 2020-01-02 2020-01-01 00:01:12 2020-01-02 03:46:12 2020-01-01 00:01:12.000 2020-01-02 03:46:12.000 72 99972 50022 5052222 72 99972 50022 5052222 -32497 32438 4601.009900990099 464702 -128 127 -1.9207920792079207 -194 -720 2 10710 99621 2.16216 299.16216 150.66216 15066.21621 2.16216 299.16217 150.66216 15066.21606 2.16216 299.16216 150.66216 15066.21600 2020-01-01 2020-01-02 2020-01-01 00:12:00 2020-01-02 03:40:21 2020-01-01 00:12:00.000 2020-01-02 03:40:21.000 720 99621 50170.5 5017050 720 99621 50170.5 5017050 -32450 32688 4950.66 495066 -125 126 -0.38 -38 -721 2 10711 99622 2.16516 299.16516 150.66516 15066.51651 2.16516 299.16516 150.66516 15066.51635 2.16516 299.16516 150.66516 15066.51600 2020-01-01 2020-01-02 2020-01-01 00:12:01 2020-01-02 03:40:22 2020-01-01 00:12:01.000 2020-01-02 03:40:22.000 721 99622 50171.5 5017150 721 99622 50171.5 5017150 -32449 32689 4951.66 495166 -124 127 0.62 62 -722 2 10712 99623 2.16816 299.16816 150.66816 15066.81681 2.16816 299.16818 150.66816 15066.81695 2.16816 299.16816 150.66816 15066.81600 2020-01-01 2020-01-02 2020-01-01 00:12:02 2020-01-02 03:40:23 2020-01-01 00:12:02.000 2020-01-02 03:40:23.000 722 99623 50172.5 5017250 722 99623 50172.5 5017250 -32448 32690 4952.66 495266 -128 127 -0.94 -94 -723 2 10713 99624 2.17117 299.17117 150.67117 15067.11711 2.17117 299.17117 150.67117 15067.11724 2.17117 299.17117 150.67117 15067.11700 2020-01-01 2020-01-02 2020-01-01 00:12:03 2020-01-02 03:40:24 2020-01-01 00:12:03.000 2020-01-02 03:40:24.000 723 99624 50173.5 5017350 723 99624 50173.5 5017350 -32447 32691 4953.66 495366 -128 123 -2.5 -250 -724 2 10714 99625 2.17417 299.17417 150.67417 15067.41741 2.17417 299.17416 150.67417 15067.41742 2.17417 299.17417 150.67417 15067.41700 2020-01-01 2020-01-02 2020-01-01 00:12:04 2020-01-02 03:40:25 2020-01-01 00:12:04.000 2020-01-02 03:40:25.000 724 99625 50174.5 5017450 724 99625 50174.5 5017450 -32446 32692 4954.66 495466 -127 124 -1.5 -150 -725 2 10715 99626 2.17717 299.17717 150.67717 15067.71771 2.17717 299.1772 150.67717 15067.71752 2.17717 299.17717 150.67717 15067.71700 2020-01-01 2020-01-02 2020-01-01 00:12:05 2020-01-02 03:40:26 2020-01-01 00:12:05.000 2020-01-02 03:40:26.000 725 99626 50175.5 5017550 725 99626 50175.5 5017550 -32445 32693 4955.66 495566 -126 125 -0.5 -50 -726 2 10716 99627 2.18018 299.18018 150.68018 15068.01801 2.18018 299.18018 150.68017 15068.01782 2.18018 299.18018 150.68018 15068.01800 2020-01-01 2020-01-02 2020-01-01 00:12:06 2020-01-02 03:40:27 2020-01-01 00:12:06.000 2020-01-02 03:40:27.000 726 99627 50176.5 5017650 726 99627 50176.5 5017650 -32444 32694 4956.66 495666 -125 126 0.5 50 +720 2 10710 99621 2.16216 299.16216 150.66216 15066.21621 2.16216 299.16217 150.66216 15066.21606 2.16216 299.16216 150.6621599999999 15066.21600 2020-01-01 2020-01-02 2020-01-01 00:12:00 2020-01-02 03:40:21 2020-01-01 00:12:00.000 2020-01-02 03:40:21.000 720 99621 50170.5 5017050 720 99621 50170.5 5017050 -32450 32688 4950.66 495066 -125 126 -0.38 -38 +721 2 10711 99622 2.16516 299.16516 150.66516 15066.51651 2.16516 299.16516 150.66516 15066.51635 2.16516 299.16516 150.66515999999996 15066.51600 2020-01-01 2020-01-02 2020-01-01 00:12:01 2020-01-02 03:40:22 2020-01-01 00:12:01.000 2020-01-02 03:40:22.000 721 99622 50171.5 5017150 721 99622 50171.5 5017150 -32449 32689 4951.66 495166 -124 127 0.62 62 +722 2 10712 99623 2.16816 299.16816 150.66816 15066.81681 2.16816 299.16818 150.66816 15066.81695 2.16816 299.16816 150.66815999999992 15066.81600 2020-01-01 2020-01-02 2020-01-01 00:12:02 2020-01-02 03:40:23 2020-01-01 00:12:02.000 2020-01-02 03:40:23.000 722 99623 50172.5 5017250 722 99623 50172.5 5017250 -32448 32690 4952.66 495266 -128 127 -0.94 -94 +723 2 10713 99624 2.17117 299.17117 150.67117 15067.11711 2.17117 299.17117 150.67117 15067.11724 2.17117 299.17117 150.6711699999998 15067.11700 2020-01-01 2020-01-02 2020-01-01 00:12:03 2020-01-02 03:40:24 2020-01-01 00:12:03.000 2020-01-02 03:40:24.000 723 99624 50173.5 5017350 723 99624 50173.5 5017350 -32447 32691 4953.66 495366 -128 123 -2.5 -250 +724 2 10714 99625 2.17417 299.17417 150.67417 15067.41741 2.17417 299.17416 150.67417 15067.41742 2.17417 299.17417 150.67417000000015 15067.41700 2020-01-01 2020-01-02 2020-01-01 00:12:04 2020-01-02 03:40:25 2020-01-01 00:12:04.000 2020-01-02 03:40:25.000 724 99625 50174.5 5017450 724 99625 50174.5 5017450 -32446 32692 4954.66 495466 -127 124 -1.5 -150 +725 2 10715 99626 2.17717 299.17717 150.67717 15067.71771 2.17717 299.1772 150.67717 15067.71752 2.17717 299.17717 150.67717000000016 15067.71700 2020-01-01 2020-01-02 2020-01-01 00:12:05 2020-01-02 03:40:26 2020-01-01 00:12:05.000 2020-01-02 03:40:26.000 725 99626 50175.5 5017550 725 99626 50175.5 5017550 -32445 32693 4955.66 495566 -126 125 -0.5 -50 +726 2 10716 99627 2.18018 299.18018 150.68018 15068.01801 2.18018 299.18018 150.68017 15068.01782 2.18018 299.18018 150.68017999999992 15068.01800 2020-01-01 2020-01-02 2020-01-01 00:12:06 2020-01-02 03:40:27 2020-01-01 00:12:06.000 2020-01-02 03:40:27.000 726 99627 50176.5 5017650 726 99627 50176.5 5017650 -32444 32694 4956.66 495666 -125 126 0.5 50 727 2 10717 99628 2.18318 299.18318 150.68318 15068.31831 2.18318 299.1832 150.68318 15068.31842 2.18318 299.18318 150.68318 15068.31800 2020-01-01 2020-01-02 2020-01-01 00:12:07 2020-01-02 03:40:28 2020-01-01 00:12:07.000 2020-01-02 03:40:28.000 727 99628 50177.5 5017750 727 99628 50177.5 5017750 -32443 32695 4957.66 495766 -124 127 1.5 150 -728 2 10718 99629 2.18618 299.18618 150.68618 15068.61861 2.18618 299.1862 150.68618 15068.61875 2.18618 299.18618 150.68618 15068.61800 2020-01-01 2020-01-02 2020-01-01 00:12:08 2020-01-02 03:40:29 2020-01-01 00:12:08.000 2020-01-02 03:40:29.000 728 99629 50178.5 5017850 728 99629 50178.5 5017850 -32442 32696 4958.66 495866 -128 127 -0.06 -6 -729 2 10719 99630 2.18918 299.18918 150.68918 15068.91891 2.18918 299.18918 150.68918 15068.91889 2.18918 299.18918 150.68918 15068.91800 2020-01-01 2020-01-02 2020-01-01 00:12:09 2020-01-02 03:40:30 2020-01-01 00:12:09.000 2020-01-02 03:40:30.000 729 99630 50179.5 5017950 729 99630 50179.5 5017950 -32441 32697 4959.66 495966 -128 123 -1.62 -162 -73 2 10063 99973 0.21921 300.21921 150.21921 15172.14114 0.21921 300.2192 150.21921 15172.14121 0.21921 300.21921 150.21921 15172.14021 2020-01-01 2020-01-02 2020-01-01 00:01:13 2020-01-02 03:46:13 2020-01-01 00:01:13.000 2020-01-02 03:46:13.000 73 99973 50023 5052323 73 99973 50023 5052323 -32496 32439 4602.009900990099 464803 -128 127 -3.4554455445544554 -349 -730 2 10720 99631 2.19219 299.19219 150.69219 15069.21921 2.19219 299.1922 150.69219 15069.21965 2.19219 299.19219 150.69219 15069.21900 2020-01-01 2020-01-02 2020-01-01 00:12:10 2020-01-02 03:40:31 2020-01-01 00:12:10.000 2020-01-02 03:40:31.000 730 99631 50180.5 5018050 730 99631 50180.5 5018050 -32440 32698 4960.66 496066 -127 124 -0.62 -62 -731 2 10721 99632 2.19519 299.19519 150.69519 15069.51951 2.19519 299.1952 150.69519 15069.51928 2.19519 299.19519 150.69519 15069.51900 2020-01-01 2020-01-02 2020-01-01 00:12:11 2020-01-02 03:40:32 2020-01-01 00:12:11.000 2020-01-02 03:40:32.000 731 99632 50181.5 5018150 731 99632 50181.5 5018150 -32439 32699 4961.66 496166 -126 125 0.38 38 -732 2 10722 99633 2.19819 299.19819 150.69819 15069.81981 2.19819 299.1982 150.69819 15069.81988 2.19819 299.19819 150.69819 15069.81900 2020-01-01 2020-01-02 2020-01-01 00:12:12 2020-01-02 03:40:33 2020-01-01 00:12:12.000 2020-01-02 03:40:33.000 732 99633 50182.5 5018250 732 99633 50182.5 5018250 -32438 32700 4962.66 496266 -125 126 1.38 138 -733 2 10723 99634 2.2012 299.2012 150.7012 15070.12012 2.2012 299.2012 150.7012 15070.12022 2.20120 299.20120 150.70120 15070.12000 2020-01-01 2020-01-02 2020-01-01 00:12:13 2020-01-02 03:40:34 2020-01-01 00:12:13.000 2020-01-02 03:40:34.000 733 99634 50183.5 5018350 733 99634 50183.5 5018350 -32437 32701 4963.66 496366 -124 127 2.38 238 -734 2 10724 99635 2.2042 299.2042 150.7042 15070.42042 2.2042 299.2042 150.7042 15070.42036 2.20420 299.20420 150.70420 15070.42000 2020-01-01 2020-01-02 2020-01-01 00:12:14 2020-01-02 03:40:35 2020-01-01 00:12:14.000 2020-01-02 03:40:35.000 734 99635 50184.5 5018450 734 99635 50184.5 5018450 -32436 32702 4964.66 496466 -128 127 0.82 82 -735 2 10725 99636 2.2072 299.2072 150.7072 15070.72072 2.2072 299.2072 150.70721 15070.72111 2.20720 299.20720 150.70720 15070.72000 2020-01-01 2020-01-02 2020-01-01 00:12:15 2020-01-02 03:40:36 2020-01-01 00:12:15.000 2020-01-02 03:40:36.000 735 99636 50185.5 5018550 735 99636 50185.5 5018550 -32435 32703 4965.66 496566 -128 127 -0.74 -74 -736 2 10726 99637 2.21021 299.21021 150.71021 15071.02102 2.21021 299.2102 150.7102 15071.02076 2.21021 299.21021 150.71021 15071.02100 2020-01-01 2020-01-02 2020-01-01 00:12:16 2020-01-02 03:40:37 2020-01-01 00:12:16.000 2020-01-02 03:40:37.000 736 99637 50186.5 5018650 736 99637 50186.5 5018650 -32434 32704 4966.66 496666 -128 124 -2.3 -230 -737 2 10727 99638 2.21321 299.21321 150.71321 15071.32132 2.21321 299.21323 150.71321 15071.32139 2.21321 299.21321 150.71321 15071.32100 2020-01-01 2020-01-02 2020-01-01 00:12:17 2020-01-02 03:40:38 2020-01-01 00:12:17.000 2020-01-02 03:40:38.000 737 99638 50187.5 5018750 737 99638 50187.5 5018750 -32433 32705 4967.66 496766 -127 125 -1.3 -130 +728 2 10718 99629 2.18618 299.18618 150.68618 15068.61861 2.18618 299.1862 150.68618 15068.61875 2.18618 299.18618 150.6861800000001 15068.61800 2020-01-01 2020-01-02 2020-01-01 00:12:08 2020-01-02 03:40:29 2020-01-01 00:12:08.000 2020-01-02 03:40:29.000 728 99629 50178.5 5017850 728 99629 50178.5 5017850 -32442 32696 4958.66 495866 -128 127 -0.06 -6 +729 2 10719 99630 2.18918 299.18918 150.68918 15068.91891 2.18918 299.18918 150.68918 15068.91889 2.18918 299.18918 150.6891799999999 15068.91800 2020-01-01 2020-01-02 2020-01-01 00:12:09 2020-01-02 03:40:30 2020-01-01 00:12:09.000 2020-01-02 03:40:30.000 729 99630 50179.5 5017950 729 99630 50179.5 5017950 -32441 32697 4959.66 495966 -128 123 -1.62 -162 +73 2 10063 99973 0.21921 300.21921 150.21921 15172.14114 0.21921 300.2192 150.21921 15172.14121 0.21921 300.21921 150.21920999999992 15172.14021 2020-01-01 2020-01-02 2020-01-01 00:01:13 2020-01-02 03:46:13 2020-01-01 00:01:13.000 2020-01-02 03:46:13.000 73 99973 50023 5052323 73 99973 50023 5052323 -32496 32439 4602.009900990099 464803 -128 127 -3.4554455445544554 -349 +730 2 10720 99631 2.19219 299.19219 150.69219 15069.21921 2.19219 299.1922 150.69219 15069.21965 2.19219 299.19219 150.6921899999999 15069.21900 2020-01-01 2020-01-02 2020-01-01 00:12:10 2020-01-02 03:40:31 2020-01-01 00:12:10.000 2020-01-02 03:40:31.000 730 99631 50180.5 5018050 730 99631 50180.5 5018050 -32440 32698 4960.66 496066 -127 124 -0.62 -62 +731 2 10721 99632 2.19519 299.19519 150.69519 15069.51951 2.19519 299.1952 150.69519 15069.51928 2.19519 299.19519 150.6951900000002 15069.51900 2020-01-01 2020-01-02 2020-01-01 00:12:11 2020-01-02 03:40:32 2020-01-01 00:12:11.000 2020-01-02 03:40:32.000 731 99632 50181.5 5018150 731 99632 50181.5 5018150 -32439 32699 4961.66 496166 -126 125 0.38 38 +732 2 10722 99633 2.19819 299.19819 150.69819 15069.81981 2.19819 299.1982 150.69819 15069.81988 2.19819 299.19819 150.69818999999984 15069.81900 2020-01-01 2020-01-02 2020-01-01 00:12:12 2020-01-02 03:40:33 2020-01-01 00:12:12.000 2020-01-02 03:40:33.000 732 99633 50182.5 5018250 732 99633 50182.5 5018250 -32438 32700 4962.66 496266 -125 126 1.38 138 +733 2 10723 99634 2.2012 299.2012 150.7012 15070.12012 2.2012 299.2012 150.7012 15070.12022 2.20120 299.20120 150.70120000000003 15070.12000 2020-01-01 2020-01-02 2020-01-01 00:12:13 2020-01-02 03:40:34 2020-01-01 00:12:13.000 2020-01-02 03:40:34.000 733 99634 50183.5 5018350 733 99634 50183.5 5018350 -32437 32701 4963.66 496366 -124 127 2.38 238 +734 2 10724 99635 2.2042 299.2042 150.7042 15070.42042 2.2042 299.2042 150.7042 15070.42036 2.20420 299.20420 150.7042000000001 15070.42000 2020-01-01 2020-01-02 2020-01-01 00:12:14 2020-01-02 03:40:35 2020-01-01 00:12:14.000 2020-01-02 03:40:35.000 734 99635 50184.5 5018450 734 99635 50184.5 5018450 -32436 32702 4964.66 496466 -128 127 0.82 82 +735 2 10725 99636 2.2072 299.2072 150.7072 15070.72072 2.2072 299.2072 150.70721 15070.72111 2.20720 299.20720 150.70720000000009 15070.72000 2020-01-01 2020-01-02 2020-01-01 00:12:15 2020-01-02 03:40:36 2020-01-01 00:12:15.000 2020-01-02 03:40:36.000 735 99636 50185.5 5018550 735 99636 50185.5 5018550 -32435 32703 4965.66 496566 -128 127 -0.74 -74 +736 2 10726 99637 2.21021 299.21021 150.71021 15071.02102 2.21021 299.2102 150.7102 15071.02076 2.21021 299.21021 150.71020999999996 15071.02100 2020-01-01 2020-01-02 2020-01-01 00:12:16 2020-01-02 03:40:37 2020-01-01 00:12:16.000 2020-01-02 03:40:37.000 736 99637 50186.5 5018650 736 99637 50186.5 5018650 -32434 32704 4966.66 496666 -128 124 -2.3 -230 +737 2 10727 99638 2.21321 299.21321 150.71321 15071.32132 2.21321 299.21323 150.71321 15071.32139 2.21321 299.21321 150.71320999999992 15071.32100 2020-01-01 2020-01-02 2020-01-01 00:12:17 2020-01-02 03:40:38 2020-01-01 00:12:17.000 2020-01-02 03:40:38.000 737 99638 50187.5 5018750 737 99638 50187.5 5018750 -32433 32705 4967.66 496766 -127 125 -1.3 -130 738 2 10728 99639 2.21621 299.21621 150.71621 15071.62162 2.21621 299.21622 150.71621 15071.62169 2.21621 299.21621 150.71621 15071.62100 2020-01-01 2020-01-02 2020-01-01 00:12:18 2020-01-02 03:40:39 2020-01-01 00:12:18.000 2020-01-02 03:40:39.000 738 99639 50188.5 5018850 738 99639 50188.5 5018850 -32432 32706 4968.66 496866 -126 126 -0.3 -30 -739 2 10729 99640 2.21921 299.21921 150.71921 15071.92192 2.21921 299.2192 150.71921 15071.92199 2.21921 299.21921 150.71921 15071.92100 2020-01-01 2020-01-02 2020-01-01 00:12:19 2020-01-02 03:40:40 2020-01-01 00:12:19.000 2020-01-02 03:40:40.000 739 99640 50189.5 5018950 739 99640 50189.5 5018950 -32431 32707 4969.66 496966 -125 127 0.7 70 -74 2 10064 99974 0.22222 300.22222 150.22222 15172.44444 0.22222 300.22223 150.22222 15172.4448 0.22222 300.22222 150.22222 15172.44422 2020-01-01 2020-01-02 2020-01-01 00:01:14 2020-01-02 03:46:14 2020-01-01 00:01:14.000 2020-01-02 03:46:14.000 74 99974 50024 5052424 74 99974 50024 5052424 -32495 32440 4603.009900990099 464904 -128 123 -4.99009900990099 -504 -740 2 10730 99641 2.22222 299.22222 150.72222 15072.22222 2.22222 299.22223 150.72222 15072.22258 2.22222 299.22222 150.72222 15072.22200 2020-01-01 2020-01-02 2020-01-01 00:12:20 2020-01-02 03:40:41 2020-01-01 00:12:20.000 2020-01-02 03:40:41.000 740 99641 50190.5 5019050 740 99641 50190.5 5019050 -32430 32708 4970.66 497066 -128 127 -0.86 -86 -741 2 10731 99642 2.22522 299.22522 150.72522 15072.52252 2.22522 299.22522 150.72522 15072.52223 2.22522 299.22522 150.72522 15072.52200 2020-01-01 2020-01-02 2020-01-01 00:12:21 2020-01-02 03:40:42 2020-01-01 00:12:21.000 2020-01-02 03:40:42.000 741 99642 50191.5 5019150 741 99642 50191.5 5019150 -32429 32709 4971.66 497166 -128 127 -2.42 -242 -742 2 10732 99643 2.22822 299.22822 150.72822 15072.82282 2.22822 299.22824 150.72822 15072.82286 2.22822 299.22822 150.72822 15072.82200 2020-01-01 2020-01-02 2020-01-01 00:12:22 2020-01-02 03:40:43 2020-01-01 00:12:22.000 2020-01-02 03:40:43.000 742 99643 50192.5 5019250 742 99643 50192.5 5019250 -32428 32710 4972.66 497266 -128 123 -3.98 -398 -743 2 10733 99644 2.23123 299.23123 150.73123 15073.12312 2.23123 299.23123 150.73123 15073.12316 2.23123 299.23123 150.73123 15073.12300 2020-01-01 2020-01-02 2020-01-01 00:12:23 2020-01-02 03:40:44 2020-01-01 00:12:23.000 2020-01-02 03:40:44.000 743 99644 50193.5 5019350 743 99644 50193.5 5019350 -32427 32711 4973.66 497366 -127 124 -2.98 -298 +739 2 10729 99640 2.21921 299.21921 150.71921 15071.92192 2.21921 299.2192 150.71921 15071.92199 2.21921 299.21921 150.71920999999995 15071.92100 2020-01-01 2020-01-02 2020-01-01 00:12:19 2020-01-02 03:40:40 2020-01-01 00:12:19.000 2020-01-02 03:40:40.000 739 99640 50189.5 5018950 739 99640 50189.5 5018950 -32431 32707 4969.66 496966 -125 127 0.7 70 +74 2 10064 99974 0.22222 300.22222 150.22222 15172.44444 0.22222 300.22223 150.22222 15172.4448 0.22222 300.22222 150.22221999999985 15172.44422 2020-01-01 2020-01-02 2020-01-01 00:01:14 2020-01-02 03:46:14 2020-01-01 00:01:14.000 2020-01-02 03:46:14.000 74 99974 50024 5052424 74 99974 50024 5052424 -32495 32440 4603.009900990099 464904 -128 123 -4.99009900990099 -504 +740 2 10730 99641 2.22222 299.22222 150.72222 15072.22222 2.22222 299.22223 150.72222 15072.22258 2.22222 299.22222 150.72221999999985 15072.22200 2020-01-01 2020-01-02 2020-01-01 00:12:20 2020-01-02 03:40:41 2020-01-01 00:12:20.000 2020-01-02 03:40:41.000 740 99641 50190.5 5019050 740 99641 50190.5 5019050 -32430 32708 4970.66 497066 -128 127 -0.86 -86 +741 2 10731 99642 2.22522 299.22522 150.72522 15072.52252 2.22522 299.22522 150.72522 15072.52223 2.22522 299.22522 150.7252200000001 15072.52200 2020-01-01 2020-01-02 2020-01-01 00:12:21 2020-01-02 03:40:42 2020-01-01 00:12:21.000 2020-01-02 03:40:42.000 741 99642 50191.5 5019150 741 99642 50191.5 5019150 -32429 32709 4971.66 497166 -128 127 -2.42 -242 +742 2 10732 99643 2.22822 299.22822 150.72822 15072.82282 2.22822 299.22824 150.72822 15072.82286 2.22822 299.22822 150.7282200000002 15072.82200 2020-01-01 2020-01-02 2020-01-01 00:12:22 2020-01-02 03:40:43 2020-01-01 00:12:22.000 2020-01-02 03:40:43.000 742 99643 50192.5 5019250 742 99643 50192.5 5019250 -32428 32710 4972.66 497266 -128 123 -3.98 -398 +743 2 10733 99644 2.23123 299.23123 150.73123 15073.12312 2.23123 299.23123 150.73123 15073.12316 2.23123 299.23123 150.73122999999993 15073.12300 2020-01-01 2020-01-02 2020-01-01 00:12:23 2020-01-02 03:40:44 2020-01-01 00:12:23.000 2020-01-02 03:40:44.000 743 99644 50193.5 5019350 743 99644 50193.5 5019350 -32427 32711 4973.66 497366 -127 124 -2.98 -298 744 2 10734 99645 2.23423 299.23423 150.73423 15073.42342 2.23423 299.23422 150.73423 15073.42345 2.23423 299.23423 150.73423 15073.42300 2020-01-01 2020-01-02 2020-01-01 00:12:24 2020-01-02 03:40:45 2020-01-01 00:12:24.000 2020-01-02 03:40:45.000 744 99645 50194.5 5019450 744 99645 50194.5 5019450 -32426 32712 4974.66 497466 -126 125 -1.98 -198 -745 2 10735 99646 2.23723 299.23723 150.73723 15073.72372 2.23723 299.23724 150.73724 15073.72405 2.23723 299.23723 150.73723 15073.72300 2020-01-01 2020-01-02 2020-01-01 00:12:25 2020-01-02 03:40:46 2020-01-01 00:12:25.000 2020-01-02 03:40:46.000 745 99646 50195.5 5019550 745 99646 50195.5 5019550 -32425 32713 4975.66 497566 -125 126 -0.98 -98 -746 2 10736 99647 2.24024 299.24024 150.74024 15074.02402 2.24024 299.24023 150.74023 15074.02373 2.24024 299.24024 150.74024 15074.02400 2020-01-01 2020-01-02 2020-01-01 00:12:26 2020-01-02 03:40:47 2020-01-01 00:12:26.000 2020-01-02 03:40:47.000 746 99647 50196.5 5019650 746 99647 50196.5 5019650 -32424 32714 4976.66 497666 -124 127 0.02 2 -747 2 10737 99648 2.24324 299.24324 150.74324 15074.32432 2.24324 299.24326 150.74324 15074.32433 2.24324 299.24324 150.74324 15074.32400 2020-01-01 2020-01-02 2020-01-01 00:12:27 2020-01-02 03:40:48 2020-01-01 00:12:27.000 2020-01-02 03:40:48.000 747 99648 50197.5 5019750 747 99648 50197.5 5019750 -32423 32715 4977.66 497766 -128 127 -1.54 -154 -748 2 10738 99649 2.24624 299.24624 150.74624 15074.62462 2.24624 299.24625 150.74624 15074.62463 2.24624 299.24624 150.74624 15074.62400 2020-01-01 2020-01-02 2020-01-01 00:12:28 2020-01-02 03:40:49 2020-01-01 00:12:28.000 2020-01-02 03:40:49.000 748 99649 50198.5 5019850 748 99649 50198.5 5019850 -32422 32716 4978.66 497866 -128 123 -3.1 -310 -749 2 10739 99650 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924 15074.92400 2020-01-01 2020-01-02 2020-01-01 00:12:29 2020-01-02 03:40:50 2020-01-01 00:12:29.000 2020-01-02 03:40:50.000 749 99650 50199.5 5019950 749 99650 50199.5 5019950 -32421 32717 4979.66 497966 -127 124 -2.1 -210 -75 2 10065 99975 0.22522 300.22522 150.22522 15172.74774 0.22522 300.22522 150.22522 15172.74745 0.22522 300.22522 150.22522 15172.74722 2020-01-01 2020-01-02 2020-01-01 00:01:15 2020-01-02 03:46:15 2020-01-01 00:01:15.000 2020-01-02 03:46:15.000 75 99975 50025 5052525 75 99975 50025 5052525 -32494 32441 4604.009900990099 465005 -127 124 -3.99009900990099 -403 -750 2 10740 99651 2.25225 299.25225 150.75225 15075.22522 2.25225 299.25226 150.75225 15075.22552 2.25225 299.25225 150.75225 15075.22500 2020-01-01 2020-01-02 2020-01-01 00:12:30 2020-01-02 03:40:51 2020-01-01 00:12:30.000 2020-01-02 03:40:51.000 750 99651 50200.5 5020050 750 99651 50200.5 5020050 -32420 32718 4980.66 498066 -126 125 -1.1 -110 -751 2 10741 99652 2.25525 299.25525 150.75525 15075.52552 2.25525 299.25525 150.75525 15075.5252 2.25525 299.25525 150.75525 15075.52500 2020-01-01 2020-01-02 2020-01-01 00:12:31 2020-01-02 03:40:52 2020-01-01 00:12:31.000 2020-01-02 03:40:52.000 751 99652 50201.5 5020150 751 99652 50201.5 5020150 -32419 32719 4981.66 498166 -125 126 -0.1 -10 -752 2 10742 99653 2.25825 299.25825 150.75825 15075.82582 2.25825 299.25827 150.75825 15075.8258 2.25825 299.25825 150.75825 15075.82500 2020-01-01 2020-01-02 2020-01-01 00:12:32 2020-01-02 03:40:53 2020-01-01 00:12:32.000 2020-01-02 03:40:53.000 752 99653 50202.5 5020250 752 99653 50202.5 5020250 -32418 32720 4982.66 498266 -124 127 0.9 90 -753 2 10743 99654 2.26126 299.26126 150.76126 15076.12612 2.26126 299.26126 150.76126 15076.12609 2.26126 299.26126 150.76126 15076.12600 2020-01-01 2020-01-02 2020-01-01 00:12:33 2020-01-02 03:40:54 2020-01-01 00:12:33.000 2020-01-02 03:40:54.000 753 99654 50203.5 5020350 753 99654 50203.5 5020350 -32417 32721 4983.66 498366 -128 127 -0.66 -66 -754 2 10744 99655 2.26426 299.26426 150.76426 15076.42642 2.26426 299.26425 150.76426 15076.4264 2.26426 299.26426 150.76426 15076.42600 2020-01-01 2020-01-02 2020-01-01 00:12:34 2020-01-02 03:40:55 2020-01-01 00:12:34.000 2020-01-02 03:40:55.000 754 99655 50204.5 5020450 754 99655 50204.5 5020450 -32416 32722 4984.66 498466 -128 123 -2.22 -222 +745 2 10735 99646 2.23723 299.23723 150.73723 15073.72372 2.23723 299.23724 150.73724 15073.72405 2.23723 299.23723 150.73723000000012 15073.72300 2020-01-01 2020-01-02 2020-01-01 00:12:25 2020-01-02 03:40:46 2020-01-01 00:12:25.000 2020-01-02 03:40:46.000 745 99646 50195.5 5019550 745 99646 50195.5 5019550 -32425 32713 4975.66 497566 -125 126 -0.98 -98 +746 2 10736 99647 2.24024 299.24024 150.74024 15074.02402 2.24024 299.24023 150.74023 15074.02373 2.24024 299.24024 150.7402399999999 15074.02400 2020-01-01 2020-01-02 2020-01-01 00:12:26 2020-01-02 03:40:47 2020-01-01 00:12:26.000 2020-01-02 03:40:47.000 746 99647 50196.5 5019650 746 99647 50196.5 5019650 -32424 32714 4976.66 497666 -124 127 0.02 2 +747 2 10737 99648 2.24324 299.24324 150.74324 15074.32432 2.24324 299.24326 150.74324 15074.32433 2.24324 299.24324 150.74323999999984 15074.32400 2020-01-01 2020-01-02 2020-01-01 00:12:27 2020-01-02 03:40:48 2020-01-01 00:12:27.000 2020-01-02 03:40:48.000 747 99648 50197.5 5019750 747 99648 50197.5 5019750 -32423 32715 4977.66 497766 -128 127 -1.54 -154 +748 2 10738 99649 2.24624 299.24624 150.74624 15074.62462 2.24624 299.24625 150.74624 15074.62463 2.24624 299.24624 150.74624000000017 15074.62400 2020-01-01 2020-01-02 2020-01-01 00:12:28 2020-01-02 03:40:49 2020-01-01 00:12:28.000 2020-01-02 03:40:49.000 748 99649 50198.5 5019850 748 99649 50198.5 5019850 -32422 32716 4978.66 497866 -128 123 -3.1 -310 +749 2 10739 99650 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74923999999987 15074.92400 2020-01-01 2020-01-02 2020-01-01 00:12:29 2020-01-02 03:40:50 2020-01-01 00:12:29.000 2020-01-02 03:40:50.000 749 99650 50199.5 5019950 749 99650 50199.5 5019950 -32421 32717 4979.66 497966 -127 124 -2.1 -210 +75 2 10065 99975 0.22522 300.22522 150.22522 15172.74774 0.22522 300.22522 150.22522 15172.74745 0.22522 300.22522 150.2252200000001 15172.74722 2020-01-01 2020-01-02 2020-01-01 00:01:15 2020-01-02 03:46:15 2020-01-01 00:01:15.000 2020-01-02 03:46:15.000 75 99975 50025 5052525 75 99975 50025 5052525 -32494 32441 4604.009900990099 465005 -127 124 -3.99009900990099 -403 +750 2 10740 99651 2.25225 299.25225 150.75225 15075.22522 2.25225 299.25226 150.75225 15075.22552 2.25225 299.25225 150.75224999999978 15075.22500 2020-01-01 2020-01-02 2020-01-01 00:12:30 2020-01-02 03:40:51 2020-01-01 00:12:30.000 2020-01-02 03:40:51.000 750 99651 50200.5 5020050 750 99651 50200.5 5020050 -32420 32718 4980.66 498066 -126 125 -1.1 -110 +751 2 10741 99652 2.25525 299.25525 150.75525 15075.52552 2.25525 299.25525 150.75525 15075.5252 2.25525 299.25525 150.7552500000001 15075.52500 2020-01-01 2020-01-02 2020-01-01 00:12:31 2020-01-02 03:40:52 2020-01-01 00:12:31.000 2020-01-02 03:40:52.000 751 99652 50201.5 5020150 751 99652 50201.5 5020150 -32419 32719 4981.66 498166 -125 126 -0.1 -10 +752 2 10742 99653 2.25825 299.25825 150.75825 15075.82582 2.25825 299.25827 150.75825 15075.8258 2.25825 299.25825 150.75825000000015 15075.82500 2020-01-01 2020-01-02 2020-01-01 00:12:32 2020-01-02 03:40:53 2020-01-01 00:12:32.000 2020-01-02 03:40:53.000 752 99653 50202.5 5020250 752 99653 50202.5 5020250 -32418 32720 4982.66 498266 -124 127 0.9 90 +753 2 10743 99654 2.26126 299.26126 150.76126 15076.12612 2.26126 299.26126 150.76126 15076.12609 2.26126 299.26126 150.76125999999994 15076.12600 2020-01-01 2020-01-02 2020-01-01 00:12:33 2020-01-02 03:40:54 2020-01-01 00:12:33.000 2020-01-02 03:40:54.000 753 99654 50203.5 5020350 753 99654 50203.5 5020350 -32417 32721 4983.66 498366 -128 127 -0.66 -66 +754 2 10744 99655 2.26426 299.26426 150.76426 15076.42642 2.26426 299.26425 150.76426 15076.4264 2.26426 299.26426 150.76425999999995 15076.42600 2020-01-01 2020-01-02 2020-01-01 00:12:34 2020-01-02 03:40:55 2020-01-01 00:12:34.000 2020-01-02 03:40:55.000 754 99655 50204.5 5020450 754 99655 50204.5 5020450 -32416 32722 4984.66 498466 -128 123 -2.22 -222 755 2 10745 99656 2.26726 299.26726 150.76726 15076.72672 2.26726 299.26727 150.76727 15076.72703 2.26726 299.26726 150.76726 15076.72600 2020-01-01 2020-01-02 2020-01-01 00:12:35 2020-01-02 03:40:56 2020-01-01 00:12:35.000 2020-01-02 03:40:56.000 755 99656 50205.5 5020550 755 99656 50205.5 5020550 -32415 32723 4985.66 498566 -127 124 -1.22 -122 -756 2 10746 99657 2.27027 299.27027 150.77027 15077.02702 2.27027 299.27026 150.77026 15077.02667 2.27027 299.27027 150.77027 15077.02700 2020-01-01 2020-01-02 2020-01-01 00:12:36 2020-01-02 03:40:57 2020-01-01 00:12:36.000 2020-01-02 03:40:57.000 756 99657 50206.5 5020650 756 99657 50206.5 5020650 -32414 32724 4986.66 498666 -126 125 -0.22 -22 -757 2 10747 99658 2.27327 299.27327 150.77327 15077.32732 2.27327 299.2733 150.77327 15077.32727 2.27327 299.27327 150.77327 15077.32700 2020-01-01 2020-01-02 2020-01-01 00:12:37 2020-01-02 03:40:58 2020-01-01 00:12:37.000 2020-01-02 03:40:58.000 757 99658 50207.5 5020750 757 99658 50207.5 5020750 -32413 32725 4987.66 498766 -125 126 0.78 78 -758 2 10748 99659 2.27627 299.27627 150.77627 15077.62762 2.27627 299.27628 150.77627 15077.62756 2.27627 299.27627 150.77627 15077.62700 2020-01-01 2020-01-02 2020-01-01 00:12:38 2020-01-02 03:40:59 2020-01-01 00:12:38.000 2020-01-02 03:40:59.000 758 99659 50208.5 5020850 758 99659 50208.5 5020850 -32412 32726 4988.66 498866 -124 127 1.78 178 -759 2 10749 99660 2.27927 299.27927 150.77927 15077.92792 2.27927 299.27927 150.77927 15077.92787 2.27927 299.27927 150.77927 15077.92700 2020-01-01 2020-01-02 2020-01-01 00:12:39 2020-01-02 03:41:00 2020-01-01 00:12:39.000 2020-01-02 03:41:00.000 759 99660 50209.5 5020950 759 99660 50209.5 5020950 -32411 32727 4989.66 498966 -128 127 0.22 22 -76 2 10066 99976 0.22822 300.22822 150.22822 15173.05105 0.22822 300.22824 150.22822 15173.05109 0.22822 300.22822 150.22822 15173.05022 2020-01-01 2020-01-02 2020-01-01 00:01:16 2020-01-02 03:46:16 2020-01-01 00:01:16.000 2020-01-02 03:46:16.000 76 99976 50026 5052626 76 99976 50026 5052626 -32493 32442 4605.009900990099 465106 -126 125 -2.99009900990099 -302 -760 2 10750 99661 2.28228 299.28228 150.78228 15078.22822 2.28228 299.2823 150.78228 15078.2285 2.28228 299.28228 150.78228 15078.22800 2020-01-01 2020-01-02 2020-01-01 00:12:40 2020-01-02 03:41:01 2020-01-01 00:12:40.000 2020-01-02 03:41:01.000 760 99661 50210.5 5021050 760 99661 50210.5 5021050 -32410 32728 4990.66 499066 -128 127 -1.34 -134 -761 2 10751 99662 2.28528 299.28528 150.78528 15078.52852 2.28528 299.28528 150.78528 15078.52814 2.28528 299.28528 150.78528 15078.52800 2020-01-01 2020-01-02 2020-01-01 00:12:41 2020-01-02 03:41:02 2020-01-01 00:12:41.000 2020-01-02 03:41:02.000 761 99662 50211.5 5021150 761 99662 50211.5 5021150 -32409 32729 4991.66 499166 -128 124 -2.9 -290 -762 2 10752 99663 2.28828 299.28828 150.78828 15078.82882 2.28828 299.2883 150.78828 15078.82889 2.28828 299.28828 150.78828 15078.82800 2020-01-01 2020-01-02 2020-01-01 00:12:42 2020-01-02 03:41:03 2020-01-01 00:12:42.000 2020-01-02 03:41:03.000 762 99663 50212.5 5021250 762 99663 50212.5 5021250 -32408 32730 4992.66 499266 -127 125 -1.9 -190 -763 2 10753 99664 2.29129 299.29129 150.79129 15079.12912 2.29129 299.2913 150.79129 15079.12904 2.29129 299.29129 150.79129 15079.12900 2020-01-01 2020-01-02 2020-01-01 00:12:43 2020-01-02 03:41:04 2020-01-01 00:12:43.000 2020-01-02 03:41:04.000 763 99664 50213.5 5021350 763 99664 50213.5 5021350 -32407 32731 4993.66 499366 -126 126 -0.9 -90 -764 2 10754 99665 2.29429 299.29429 150.79429 15079.42942 2.29429 299.29428 150.79429 15079.42933 2.29429 299.29429 150.79429 15079.42900 2020-01-01 2020-01-02 2020-01-01 00:12:44 2020-01-02 03:41:05 2020-01-01 00:12:44.000 2020-01-02 03:41:05.000 764 99665 50214.5 5021450 764 99665 50214.5 5021450 -32406 32732 4994.66 499466 -125 127 0.1 10 -765 2 10755 99666 2.29729 299.29729 150.79729 15079.72972 2.29729 299.2973 150.79729 15079.72996 2.29729 299.29729 150.79729 15079.72900 2020-01-01 2020-01-02 2020-01-01 00:12:45 2020-01-02 03:41:06 2020-01-01 00:12:45.000 2020-01-02 03:41:06.000 765 99666 50215.5 5021550 765 99666 50215.5 5021550 -32405 32733 4995.66 499566 -128 127 -1.46 -146 -766 2 10756 99667 2.3003 299.3003 150.8003 15080.03003 2.3003 299.3003 150.80029 15080.02961 2.30030 299.30030 150.80030 15080.03000 2020-01-01 2020-01-02 2020-01-01 00:12:46 2020-01-02 03:41:07 2020-01-01 00:12:46.000 2020-01-02 03:41:07.000 766 99667 50216.5 5021650 766 99667 50216.5 5021650 -32404 32734 4996.66 499666 -128 127 -3.02 -302 -767 2 10757 99668 2.3033 299.3033 150.8033 15080.33033 2.3033 299.3033 150.8033 15080.33036 2.30330 299.30330 150.80330 15080.33000 2020-01-01 2020-01-02 2020-01-01 00:12:47 2020-01-02 03:41:08 2020-01-01 00:12:47.000 2020-01-02 03:41:08.000 767 99668 50217.5 5021750 767 99668 50217.5 5021750 -32403 32735 4997.66 499766 -128 123 -4.58 -458 -768 2 10758 99669 2.3063 299.3063 150.8063 15080.63063 2.3063 299.3063 150.8063 15080.6305 2.30630 299.30630 150.80630 15080.63000 2020-01-01 2020-01-02 2020-01-01 00:12:48 2020-01-02 03:41:09 2020-01-01 00:12:48.000 2020-01-02 03:41:09.000 768 99669 50218.5 5021850 768 99669 50218.5 5021850 -32402 32736 4998.66 499866 -127 124 -3.58 -358 -769 2 10759 99670 2.3093 299.3093 150.8093 15080.93093 2.3093 299.3093 150.8093 15080.93084 2.30930 299.30930 150.80930 15080.93000 2020-01-01 2020-01-02 2020-01-01 00:12:49 2020-01-02 03:41:10 2020-01-01 00:12:49.000 2020-01-02 03:41:10.000 769 99670 50219.5 5021950 769 99670 50219.5 5021950 -32401 32737 4999.66 499966 -126 125 -2.58 -258 -77 2 10067 99977 0.23123 300.23123 150.23123 15173.35435 0.23123 300.23123 150.23123 15173.35439 0.23123 300.23123 150.23123 15173.35423 2020-01-01 2020-01-02 2020-01-01 00:01:17 2020-01-02 03:46:17 2020-01-01 00:01:17.000 2020-01-02 03:46:17.000 77 99977 50027 5052727 77 99977 50027 5052727 -32492 32443 4606.009900990099 465207 -125 126 -1.99009900990099 -201 -770 2 10760 99671 2.31231 299.31231 150.81231 15081.23123 2.31231 299.31232 150.81231 15081.23144 2.31231 299.31231 150.81231 15081.23100 2020-01-01 2020-01-02 2020-01-01 00:12:50 2020-01-02 03:41:11 2020-01-01 00:12:50.000 2020-01-02 03:41:11.000 770 99671 50220.5 5022050 770 99671 50220.5 5022050 -32400 32738 5000.66 500066 -125 126 -1.58 -158 -771 2 10761 99672 2.31531 299.31531 150.81531 15081.53153 2.31531 299.3153 150.81531 15081.53173 2.31531 299.31531 150.81531 15081.53100 2020-01-01 2020-01-02 2020-01-01 00:12:51 2020-01-02 03:41:12 2020-01-01 00:12:51.000 2020-01-02 03:41:12.000 771 99672 50221.5 5022150 771 99672 50221.5 5022150 -32399 32739 5001.66 500166 -124 127 -0.58 -58 -772 2 10762 99673 2.31831 299.31831 150.81831 15081.83183 2.31831 299.31833 150.81831 15081.83183 2.31831 299.31831 150.81831 15081.83100 2020-01-01 2020-01-02 2020-01-01 00:12:52 2020-01-02 03:41:13 2020-01-01 00:12:52.000 2020-01-02 03:41:13.000 772 99673 50222.5 5022250 772 99673 50222.5 5022250 -32398 32740 5002.66 500266 -128 127 -2.14 -214 -773 2 10763 99674 2.32132 299.32132 150.82132 15082.13213 2.32132 299.32132 150.82131 15082.13197 2.32132 299.32132 150.82132 15082.13200 2020-01-01 2020-01-02 2020-01-01 00:12:53 2020-01-02 03:41:14 2020-01-01 00:12:53.000 2020-01-02 03:41:14.000 773 99674 50223.5 5022350 773 99674 50223.5 5022350 -32397 32741 5003.66 500366 -128 123 -3.7 -370 -774 2 10764 99675 2.32432 299.32432 150.82432 15082.43243 2.32432 299.3243 150.82432 15082.43231 2.32432 299.32432 150.82432 15082.43200 2020-01-01 2020-01-02 2020-01-01 00:12:54 2020-01-02 03:41:15 2020-01-01 00:12:54.000 2020-01-02 03:41:15.000 774 99675 50224.5 5022450 774 99675 50224.5 5022450 -32396 32742 5004.66 500466 -127 124 -2.7 -270 -775 2 10765 99676 2.32732 299.32732 150.82732 15082.73273 2.32732 299.32733 150.82732 15082.73291 2.32732 299.32732 150.82732 15082.73200 2020-01-01 2020-01-02 2020-01-01 00:12:55 2020-01-02 03:41:16 2020-01-01 00:12:55.000 2020-01-02 03:41:16.000 775 99676 50225.5 5022550 775 99676 50225.5 5022550 -32395 32743 5005.66 500566 -126 125 -1.7 -170 -776 2 10766 99677 2.33033 299.33033 150.83033 15083.03303 2.33033 299.33032 150.83033 15083.0332 2.33033 299.33033 150.83033 15083.03300 2020-01-01 2020-01-02 2020-01-01 00:12:56 2020-01-02 03:41:17 2020-01-01 00:12:56.000 2020-01-02 03:41:17.000 776 99677 50226.5 5022650 776 99677 50226.5 5022650 -32394 32744 5006.66 500666 -125 126 -0.7 -70 -777 2 10767 99678 2.33333 299.33333 150.83333 15083.33333 2.33333 299.33334 150.83333 15083.3333 2.33333 299.33333 150.83333 15083.33300 2020-01-01 2020-01-02 2020-01-01 00:12:57 2020-01-02 03:41:18 2020-01-01 00:12:57.000 2020-01-02 03:41:18.000 777 99678 50227.5 5022750 777 99678 50227.5 5022750 -32393 32745 5007.66 500766 -124 127 0.3 30 -778 2 10768 99679 2.33633 299.33633 150.83633 15083.63363 2.33633 299.33633 150.83633 15083.63348 2.33633 299.33633 150.83633 15083.63300 2020-01-01 2020-01-02 2020-01-01 00:12:58 2020-01-02 03:41:19 2020-01-01 00:12:58.000 2020-01-02 03:41:19.000 778 99679 50228.5 5022850 778 99679 50228.5 5022850 -32392 32746 5008.66 500866 -128 127 -1.26 -126 -779 2 10769 99680 2.33933 299.33933 150.83933 15083.93393 2.33933 299.33932 150.83933 15083.93378 2.33933 299.33933 150.83933 15083.93300 2020-01-01 2020-01-02 2020-01-01 00:12:59 2020-01-02 03:41:20 2020-01-01 00:12:59.000 2020-01-02 03:41:20.000 779 99680 50229.5 5022950 779 99680 50229.5 5022950 -32391 32747 5009.66 500966 -128 123 -2.82 -282 -78 2 10068 99978 0.23423 300.23423 150.23423 15173.65765 0.23423 300.23422 150.23423 15173.65769 0.23423 300.23423 150.23423 15173.65723 2020-01-01 2020-01-02 2020-01-01 00:01:18 2020-01-02 03:46:18 2020-01-01 00:01:18.000 2020-01-02 03:46:18.000 78 99978 50028 5052828 78 99978 50028 5052828 -32491 32444 4607.009900990099 465308 -124 127 -0.9900990099009901 -100 -780 2 10770 99681 2.34234 299.34234 150.84234 15084.23423 2.34234 299.34235 150.84234 15084.23437 2.34234 299.34234 150.84234 15084.23400 2020-01-01 2020-01-02 2020-01-01 00:13:00 2020-01-02 03:41:21 2020-01-01 00:13:00.000 2020-01-02 03:41:21.000 780 99681 50230.5 5023050 780 99681 50230.5 5023050 -32390 32748 5010.66 501066 -127 124 -1.82 -182 -781 2 10771 99682 2.34534 299.34534 150.84534 15084.53453 2.34534 299.34534 150.84534 15084.53467 2.34534 299.34534 150.84534 15084.53400 2020-01-01 2020-01-02 2020-01-01 00:13:01 2020-01-02 03:41:22 2020-01-01 00:13:01.000 2020-01-02 03:41:22.000 781 99682 50231.5 5023150 781 99682 50231.5 5023150 -32389 32749 5011.66 501166 -126 125 -0.82 -82 -782 2 10772 99683 2.34834 299.34834 150.84834 15084.83483 2.34834 299.34836 150.84834 15084.83477 2.34834 299.34834 150.84834 15084.83400 2020-01-01 2020-01-02 2020-01-01 00:13:02 2020-01-02 03:41:23 2020-01-01 00:13:02.000 2020-01-02 03:41:23.000 782 99683 50232.5 5023250 782 99683 50232.5 5023250 -32388 32750 5012.66 501266 -125 126 0.18 18 -783 2 10773 99684 2.35135 299.35135 150.85135 15085.13513 2.35135 299.35135 150.85134 15085.13495 2.35135 299.35135 150.85135 15085.13500 2020-01-01 2020-01-02 2020-01-01 00:13:03 2020-01-02 03:41:24 2020-01-01 00:13:03.000 2020-01-02 03:41:24.000 783 99684 50233.5 5023350 783 99684 50233.5 5023350 -32387 32751 5013.66 501366 -124 127 1.18 118 -784 2 10774 99685 2.35435 299.35435 150.85435 15085.43543 2.35435 299.35434 150.85435 15085.43525 2.35435 299.35435 150.85435 15085.43500 2020-01-01 2020-01-02 2020-01-01 00:13:04 2020-01-02 03:41:25 2020-01-01 00:13:04.000 2020-01-02 03:41:25.000 784 99685 50234.5 5023450 784 99685 50234.5 5023450 -32386 32752 5014.66 501466 -128 127 -0.38 -38 -785 2 10775 99686 2.35735 299.35735 150.85735 15085.73573 2.35735 299.35736 150.85736 15085.736 2.35735 299.35735 150.85735 15085.73500 2020-01-01 2020-01-02 2020-01-01 00:13:05 2020-01-02 03:41:26 2020-01-01 00:13:05.000 2020-01-02 03:41:26.000 785 99686 50235.5 5023550 785 99686 50235.5 5023550 -32385 32753 5015.66 501566 -128 127 -1.94 -194 -786 2 10776 99687 2.36036 299.36036 150.86036 15086.03603 2.36036 299.36035 150.86036 15086.03614 2.36036 299.36036 150.86036 15086.03600 2020-01-01 2020-01-02 2020-01-01 00:13:06 2020-01-02 03:41:27 2020-01-01 00:13:06.000 2020-01-02 03:41:27.000 786 99687 50236.5 5023650 786 99687 50236.5 5023650 -32384 32754 5016.66 501666 -128 124 -3.5 -350 -787 2 10777 99688 2.36336 299.36336 150.86336 15086.33633 2.36336 299.36337 150.86336 15086.33628 2.36336 299.36336 150.86336 15086.33600 2020-01-01 2020-01-02 2020-01-01 00:13:07 2020-01-02 03:41:28 2020-01-01 00:13:07.000 2020-01-02 03:41:28.000 787 99688 50237.5 5023750 787 99688 50237.5 5023750 -32383 32755 5017.66 501766 -127 125 -2.5 -250 +756 2 10746 99657 2.27027 299.27027 150.77027 15077.02702 2.27027 299.27026 150.77026 15077.02667 2.27027 299.27027 150.7702700000002 15077.02700 2020-01-01 2020-01-02 2020-01-01 00:12:36 2020-01-02 03:40:57 2020-01-01 00:12:36.000 2020-01-02 03:40:57.000 756 99657 50206.5 5020650 756 99657 50206.5 5020650 -32414 32724 4986.66 498666 -126 125 -0.22 -22 +757 2 10747 99658 2.27327 299.27327 150.77327 15077.32732 2.27327 299.2733 150.77327 15077.32727 2.27327 299.27327 150.77326999999985 15077.32700 2020-01-01 2020-01-02 2020-01-01 00:12:37 2020-01-02 03:40:58 2020-01-01 00:12:37.000 2020-01-02 03:40:58.000 757 99658 50207.5 5020750 757 99658 50207.5 5020750 -32413 32725 4987.66 498766 -125 126 0.78 78 +758 2 10748 99659 2.27627 299.27627 150.77627 15077.62762 2.27627 299.27628 150.77627 15077.62756 2.27627 299.27627 150.77627000000012 15077.62700 2020-01-01 2020-01-02 2020-01-01 00:12:38 2020-01-02 03:40:59 2020-01-01 00:12:38.000 2020-01-02 03:40:59.000 758 99659 50208.5 5020850 758 99659 50208.5 5020850 -32412 32726 4988.66 498866 -124 127 1.78 178 +759 2 10749 99660 2.27927 299.27927 150.77927 15077.92792 2.27927 299.27927 150.77927 15077.92787 2.27927 299.27927 150.7792699999999 15077.92700 2020-01-01 2020-01-02 2020-01-01 00:12:39 2020-01-02 03:41:00 2020-01-01 00:12:39.000 2020-01-02 03:41:00.000 759 99660 50209.5 5020950 759 99660 50209.5 5020950 -32411 32727 4989.66 498966 -128 127 0.22 22 +76 2 10066 99976 0.22822 300.22822 150.22822 15173.05105 0.22822 300.22824 150.22822 15173.05109 0.22822 300.22822 150.2282200000002 15173.05022 2020-01-01 2020-01-02 2020-01-01 00:01:16 2020-01-02 03:46:16 2020-01-01 00:01:16.000 2020-01-02 03:46:16.000 76 99976 50026 5052626 76 99976 50026 5052626 -32493 32442 4605.009900990099 465106 -126 125 -2.99009900990099 -302 +760 2 10750 99661 2.28228 299.28228 150.78228 15078.22822 2.28228 299.2823 150.78228 15078.2285 2.28228 299.28228 150.78228000000001 15078.22800 2020-01-01 2020-01-02 2020-01-01 00:12:40 2020-01-02 03:41:01 2020-01-01 00:12:40.000 2020-01-02 03:41:01.000 760 99661 50210.5 5021050 760 99661 50210.5 5021050 -32410 32728 4990.66 499066 -128 127 -1.34 -134 +761 2 10751 99662 2.28528 299.28528 150.78528 15078.52852 2.28528 299.28528 150.78528 15078.52814 2.28528 299.28528 150.78528000000003 15078.52800 2020-01-01 2020-01-02 2020-01-01 00:12:41 2020-01-02 03:41:02 2020-01-01 00:12:41.000 2020-01-02 03:41:02.000 761 99662 50211.5 5021150 761 99662 50211.5 5021150 -32409 32729 4991.66 499166 -128 124 -2.9 -290 +762 2 10752 99663 2.28828 299.28828 150.78828 15078.82882 2.28828 299.2883 150.78828 15078.82889 2.28828 299.28828 150.78828000000007 15078.82800 2020-01-01 2020-01-02 2020-01-01 00:12:42 2020-01-02 03:41:03 2020-01-01 00:12:42.000 2020-01-02 03:41:03.000 762 99663 50212.5 5021250 762 99663 50212.5 5021250 -32408 32730 4992.66 499266 -127 125 -1.9 -190 +763 2 10753 99664 2.29129 299.29129 150.79129 15079.12912 2.29129 299.2913 150.79129 15079.12904 2.29129 299.29129 150.79128999999992 15079.12900 2020-01-01 2020-01-02 2020-01-01 00:12:43 2020-01-02 03:41:04 2020-01-01 00:12:43.000 2020-01-02 03:41:04.000 763 99664 50213.5 5021350 763 99664 50213.5 5021350 -32407 32731 4993.66 499366 -126 126 -0.9 -90 +764 2 10754 99665 2.29429 299.29429 150.79429 15079.42942 2.29429 299.29428 150.79429 15079.42933 2.29429 299.29429 150.79428999999988 15079.42900 2020-01-01 2020-01-02 2020-01-01 00:12:44 2020-01-02 03:41:05 2020-01-01 00:12:44.000 2020-01-02 03:41:05.000 764 99665 50214.5 5021450 764 99665 50214.5 5021450 -32406 32732 4994.66 499466 -125 127 0.1 10 +765 2 10755 99666 2.29729 299.29729 150.79729 15079.72972 2.29729 299.2973 150.79729 15079.72996 2.29729 299.29729 150.7972900000002 15079.72900 2020-01-01 2020-01-02 2020-01-01 00:12:45 2020-01-02 03:41:06 2020-01-01 00:12:45.000 2020-01-02 03:41:06.000 765 99666 50215.5 5021550 765 99666 50215.5 5021550 -32405 32733 4995.66 499566 -128 127 -1.46 -146 +766 2 10756 99667 2.3003 299.3003 150.8003 15080.03003 2.3003 299.3003 150.80029 15080.02961 2.30030 299.30030 150.8003000000001 15080.03000 2020-01-01 2020-01-02 2020-01-01 00:12:46 2020-01-02 03:41:07 2020-01-01 00:12:46.000 2020-01-02 03:41:07.000 766 99667 50216.5 5021650 766 99667 50216.5 5021650 -32404 32734 4996.66 499666 -128 127 -3.02 -302 +767 2 10757 99668 2.3033 299.3033 150.8033 15080.33033 2.3033 299.3033 150.8033 15080.33036 2.30330 299.30330 150.80329999999978 15080.33000 2020-01-01 2020-01-02 2020-01-01 00:12:47 2020-01-02 03:41:08 2020-01-01 00:12:47.000 2020-01-02 03:41:08.000 767 99668 50217.5 5021750 767 99668 50217.5 5021750 -32403 32735 4997.66 499766 -128 123 -4.58 -458 +768 2 10758 99669 2.3063 299.3063 150.8063 15080.63063 2.3063 299.3063 150.8063 15080.6305 2.30630 299.30630 150.80630000000014 15080.63000 2020-01-01 2020-01-02 2020-01-01 00:12:48 2020-01-02 03:41:09 2020-01-01 00:12:48.000 2020-01-02 03:41:09.000 768 99669 50218.5 5021850 768 99669 50218.5 5021850 -32402 32736 4998.66 499866 -127 124 -3.58 -358 +769 2 10759 99670 2.3093 299.3093 150.8093 15080.93093 2.3093 299.3093 150.8093 15080.93084 2.30930 299.30930 150.80930000000012 15080.93000 2020-01-01 2020-01-02 2020-01-01 00:12:49 2020-01-02 03:41:10 2020-01-01 00:12:49.000 2020-01-02 03:41:10.000 769 99670 50219.5 5021950 769 99670 50219.5 5021950 -32401 32737 4999.66 499966 -126 125 -2.58 -258 +77 2 10067 99977 0.23123 300.23123 150.23123 15173.35435 0.23123 300.23123 150.23123 15173.35439 0.23123 300.23123 150.23122999999993 15173.35423 2020-01-01 2020-01-02 2020-01-01 00:01:17 2020-01-02 03:46:17 2020-01-01 00:01:17.000 2020-01-02 03:46:17.000 77 99977 50027 5052727 77 99977 50027 5052727 -32492 32443 4606.009900990099 465207 -125 126 -1.99009900990099 -201 +770 2 10760 99671 2.31231 299.31231 150.81231 15081.23123 2.31231 299.31232 150.81231 15081.23144 2.31231 299.31231 150.8123099999999 15081.23100 2020-01-01 2020-01-02 2020-01-01 00:12:50 2020-01-02 03:41:11 2020-01-01 00:12:50.000 2020-01-02 03:41:11.000 770 99671 50220.5 5022050 770 99671 50220.5 5022050 -32400 32738 5000.66 500066 -125 126 -1.58 -158 +771 2 10761 99672 2.31531 299.31531 150.81531 15081.53153 2.31531 299.3153 150.81531 15081.53173 2.31531 299.31531 150.81530999999998 15081.53100 2020-01-01 2020-01-02 2020-01-01 00:12:51 2020-01-02 03:41:12 2020-01-01 00:12:51.000 2020-01-02 03:41:12.000 771 99672 50221.5 5022150 771 99672 50221.5 5022150 -32399 32739 5001.66 500166 -124 127 -0.58 -58 +772 2 10762 99673 2.31831 299.31831 150.81831 15081.83183 2.31831 299.31833 150.81831 15081.83183 2.31831 299.31831 150.81831000000003 15081.83100 2020-01-01 2020-01-02 2020-01-01 00:12:52 2020-01-02 03:41:13 2020-01-01 00:12:52.000 2020-01-02 03:41:13.000 772 99673 50222.5 5022250 772 99673 50222.5 5022250 -32398 32740 5002.66 500266 -128 127 -2.14 -214 +773 2 10763 99674 2.32132 299.32132 150.82132 15082.13213 2.32132 299.32132 150.82131 15082.13197 2.32132 299.32132 150.8213199999998 15082.13200 2020-01-01 2020-01-02 2020-01-01 00:12:53 2020-01-02 03:41:14 2020-01-01 00:12:53.000 2020-01-02 03:41:14.000 773 99674 50223.5 5022350 773 99674 50223.5 5022350 -32397 32741 5003.66 500366 -128 123 -3.7 -370 +774 2 10764 99675 2.32432 299.32432 150.82432 15082.43243 2.32432 299.3243 150.82432 15082.43231 2.32432 299.32432 150.8243199999999 15082.43200 2020-01-01 2020-01-02 2020-01-01 00:12:54 2020-01-02 03:41:15 2020-01-01 00:12:54.000 2020-01-02 03:41:15.000 774 99675 50224.5 5022450 774 99675 50224.5 5022450 -32396 32742 5004.66 500466 -127 124 -2.7 -270 +775 2 10765 99676 2.32732 299.32732 150.82732 15082.73273 2.32732 299.32733 150.82732 15082.73291 2.32732 299.32732 150.82732000000016 15082.73200 2020-01-01 2020-01-02 2020-01-01 00:12:55 2020-01-02 03:41:16 2020-01-01 00:12:55.000 2020-01-02 03:41:16.000 775 99676 50225.5 5022550 775 99676 50225.5 5022550 -32395 32743 5005.66 500566 -126 125 -1.7 -170 +776 2 10766 99677 2.33033 299.33033 150.83033 15083.03303 2.33033 299.33032 150.83033 15083.0332 2.33033 299.33033 150.83033000000003 15083.03300 2020-01-01 2020-01-02 2020-01-01 00:12:56 2020-01-02 03:41:17 2020-01-01 00:12:56.000 2020-01-02 03:41:17.000 776 99677 50226.5 5022650 776 99677 50226.5 5022650 -32394 32744 5006.66 500666 -125 126 -0.7 -70 +777 2 10767 99678 2.33333 299.33333 150.83333 15083.33333 2.33333 299.33334 150.83333 15083.3333 2.33333 299.33333 150.83333000000002 15083.33300 2020-01-01 2020-01-02 2020-01-01 00:12:57 2020-01-02 03:41:18 2020-01-01 00:12:57.000 2020-01-02 03:41:18.000 777 99678 50227.5 5022750 777 99678 50227.5 5022750 -32393 32745 5007.66 500766 -124 127 0.3 30 +778 2 10768 99679 2.33633 299.33633 150.83633 15083.63363 2.33633 299.33633 150.83633 15083.63348 2.33633 299.33633 150.83633000000006 15083.63300 2020-01-01 2020-01-02 2020-01-01 00:12:58 2020-01-02 03:41:19 2020-01-01 00:12:58.000 2020-01-02 03:41:19.000 778 99679 50228.5 5022850 778 99679 50228.5 5022850 -32392 32746 5008.66 500866 -128 127 -1.26 -126 +779 2 10769 99680 2.33933 299.33933 150.83933 15083.93393 2.33933 299.33932 150.83933 15083.93378 2.33933 299.33933 150.83933000000002 15083.93300 2020-01-01 2020-01-02 2020-01-01 00:12:59 2020-01-02 03:41:20 2020-01-01 00:12:59.000 2020-01-02 03:41:20.000 779 99680 50229.5 5022950 779 99680 50229.5 5022950 -32391 32747 5009.66 500966 -128 123 -2.82 -282 +78 2 10068 99978 0.23423 300.23423 150.23423 15173.65765 0.23423 300.23422 150.23423 15173.65769 0.23423 300.23423 150.23423000000003 15173.65723 2020-01-01 2020-01-02 2020-01-01 00:01:18 2020-01-02 03:46:18 2020-01-01 00:01:18.000 2020-01-02 03:46:18.000 78 99978 50028 5052828 78 99978 50028 5052828 -32491 32444 4607.009900990099 465308 -124 127 -0.9900990099009901 -100 +780 2 10770 99681 2.34234 299.34234 150.84234 15084.23423 2.34234 299.34235 150.84234 15084.23437 2.34234 299.34234 150.8423399999999 15084.23400 2020-01-01 2020-01-02 2020-01-01 00:13:00 2020-01-02 03:41:21 2020-01-01 00:13:00.000 2020-01-02 03:41:21.000 780 99681 50230.5 5023050 780 99681 50230.5 5023050 -32390 32748 5010.66 501066 -127 124 -1.82 -182 +781 2 10771 99682 2.34534 299.34534 150.84534 15084.53453 2.34534 299.34534 150.84534 15084.53467 2.34534 299.34534 150.8453399999999 15084.53400 2020-01-01 2020-01-02 2020-01-01 00:13:01 2020-01-02 03:41:22 2020-01-01 00:13:01.000 2020-01-02 03:41:22.000 781 99682 50231.5 5023150 781 99682 50231.5 5023150 -32389 32749 5011.66 501166 -126 125 -0.82 -82 +782 2 10772 99683 2.34834 299.34834 150.84834 15084.83483 2.34834 299.34836 150.84834 15084.83477 2.34834 299.34834 150.84834000000006 15084.83400 2020-01-01 2020-01-02 2020-01-01 00:13:02 2020-01-02 03:41:23 2020-01-01 00:13:02.000 2020-01-02 03:41:23.000 782 99683 50232.5 5023250 782 99683 50232.5 5023250 -32388 32750 5012.66 501266 -125 126 0.18 18 +783 2 10773 99684 2.35135 299.35135 150.85135 15085.13513 2.35135 299.35135 150.85134 15085.13495 2.35135 299.35135 150.85135000000014 15085.13500 2020-01-01 2020-01-02 2020-01-01 00:13:03 2020-01-02 03:41:24 2020-01-01 00:13:03.000 2020-01-02 03:41:24.000 783 99684 50233.5 5023350 783 99684 50233.5 5023350 -32387 32751 5013.66 501366 -124 127 1.18 118 +784 2 10774 99685 2.35435 299.35435 150.85435 15085.43543 2.35435 299.35434 150.85435 15085.43525 2.35435 299.35435 150.85434999999978 15085.43500 2020-01-01 2020-01-02 2020-01-01 00:13:04 2020-01-02 03:41:25 2020-01-01 00:13:04.000 2020-01-02 03:41:25.000 784 99685 50234.5 5023450 784 99685 50234.5 5023450 -32386 32752 5014.66 501466 -128 127 -0.38 -38 +785 2 10775 99686 2.35735 299.35735 150.85735 15085.73573 2.35735 299.35736 150.85736 15085.736 2.35735 299.35735 150.85735000000017 15085.73500 2020-01-01 2020-01-02 2020-01-01 00:13:05 2020-01-02 03:41:26 2020-01-01 00:13:05.000 2020-01-02 03:41:26.000 785 99686 50235.5 5023550 785 99686 50235.5 5023550 -32385 32753 5015.66 501566 -128 127 -1.94 -194 +786 2 10776 99687 2.36036 299.36036 150.86036 15086.03603 2.36036 299.36035 150.86036 15086.03614 2.36036 299.36036 150.86036000000004 15086.03600 2020-01-01 2020-01-02 2020-01-01 00:13:06 2020-01-02 03:41:27 2020-01-01 00:13:06.000 2020-01-02 03:41:27.000 786 99687 50236.5 5023650 786 99687 50236.5 5023650 -32384 32754 5016.66 501666 -128 124 -3.5 -350 +787 2 10777 99688 2.36336 299.36336 150.86336 15086.33633 2.36336 299.36337 150.86336 15086.33628 2.36336 299.36336 150.86335999999991 15086.33600 2020-01-01 2020-01-02 2020-01-01 00:13:07 2020-01-02 03:41:28 2020-01-01 00:13:07.000 2020-01-02 03:41:28.000 787 99688 50237.5 5023750 787 99688 50237.5 5023750 -32383 32755 5017.66 501766 -127 125 -2.5 -250 788 2 10778 99689 2.36636 299.36636 150.86636 15086.63663 2.36636 299.36636 150.86636 15086.63641 2.36636 299.36636 150.86636 15086.63600 2020-01-01 2020-01-02 2020-01-01 00:13:08 2020-01-02 03:41:29 2020-01-01 00:13:08.000 2020-01-02 03:41:29.000 788 99689 50238.5 5023850 788 99689 50238.5 5023850 -32382 32756 5018.66 501866 -126 126 -1.5 -150 -789 2 10779 99690 2.36936 299.36936 150.86936 15086.93693 2.36936 299.36935 150.86936 15086.93672 2.36936 299.36936 150.86936 15086.93600 2020-01-01 2020-01-02 2020-01-01 00:13:09 2020-01-02 03:41:30 2020-01-01 00:13:09.000 2020-01-02 03:41:30.000 789 99690 50239.5 5023950 789 99690 50239.5 5023950 -32381 32757 5019.66 501966 -125 127 -0.5 -50 -79 2 10069 99979 0.23723 300.23723 150.23723 15173.96096 0.23723 300.23724 150.23724 15173.96129 0.23723 300.23723 150.23723 15173.96023 2020-01-01 2020-01-02 2020-01-01 00:01:19 2020-01-02 03:46:19 2020-01-01 00:01:19.000 2020-01-02 03:46:19.000 79 99979 50029 5052929 79 99979 50029 5052929 -32490 32445 4608.009900990099 465409 -128 127 -2.5247524752475248 -255 -790 2 10780 99691 2.37237 299.37237 150.87237 15087.23723 2.37237 299.37238 150.87237 15087.23747 2.37237 299.37237 150.87237 15087.23700 2020-01-01 2020-01-02 2020-01-01 00:13:10 2020-01-02 03:41:31 2020-01-01 00:13:10.000 2020-01-02 03:41:31.000 790 99691 50240.5 5024050 790 99691 50240.5 5024050 -32380 32758 5020.66 502066 -128 127 -2.06 -206 -791 2 10781 99692 2.37537 299.37537 150.87537 15087.53753 2.37537 299.37537 150.87537 15087.53761 2.37537 299.37537 150.87537 15087.53700 2020-01-01 2020-01-02 2020-01-01 00:13:11 2020-01-02 03:41:32 2020-01-01 00:13:11.000 2020-01-02 03:41:32.000 791 99692 50241.5 5024150 791 99692 50241.5 5024150 -32379 32759 5021.66 502166 -128 127 -3.62 -362 -792 2 10782 99693 2.37837 299.37837 150.87837 15087.83783 2.37837 299.3784 150.87837 15087.83775 2.37837 299.37837 150.87837 15087.83700 2020-01-01 2020-01-02 2020-01-01 00:13:12 2020-01-02 03:41:33 2020-01-01 00:13:12.000 2020-01-02 03:41:33.000 792 99693 50242.5 5024250 792 99693 50242.5 5024250 -32378 32760 5022.66 502266 -128 123 -5.18 -518 -793 2 10783 99694 2.38138 299.38138 150.88138 15088.13813 2.38138 299.38138 150.88137 15088.13789 2.38138 299.38138 150.88138 15088.13800 2020-01-01 2020-01-02 2020-01-01 00:13:13 2020-01-02 03:41:34 2020-01-01 00:13:13.000 2020-01-02 03:41:34.000 793 99694 50243.5 5024350 793 99694 50243.5 5024350 -32377 32761 5023.66 502366 -127 124 -4.18 -418 -794 2 10784 99695 2.38438 299.38438 150.88438 15088.43843 2.38438 299.3844 150.88438 15088.43864 2.38438 299.38438 150.88438 15088.43800 2020-01-01 2020-01-02 2020-01-01 00:13:14 2020-01-02 03:41:35 2020-01-01 00:13:14.000 2020-01-02 03:41:35.000 794 99695 50244.5 5024450 794 99695 50244.5 5024450 -32376 32762 5024.66 502466 -126 125 -3.18 -318 -795 2 10785 99696 2.38738 299.38738 150.88738 15088.73873 2.38738 299.3874 150.88738 15088.73894 2.38738 299.38738 150.88738 15088.73800 2020-01-01 2020-01-02 2020-01-01 00:13:15 2020-01-02 03:41:36 2020-01-01 00:13:15.000 2020-01-02 03:41:36.000 795 99696 50245.5 5024550 795 99696 50245.5 5024550 -32375 32763 5025.66 502566 -125 126 -2.18 -218 -796 2 10786 99697 2.39039 299.39039 150.89039 15089.03903 2.39039 299.39038 150.89039 15089.03908 2.39039 299.39039 150.89039 15089.03900 2020-01-01 2020-01-02 2020-01-01 00:13:16 2020-01-02 03:41:37 2020-01-01 00:13:16.000 2020-01-02 03:41:37.000 796 99697 50246.5 5024650 796 99697 50246.5 5024650 -32374 32764 5026.66 502666 -124 127 -1.18 -118 -797 2 10787 99698 2.39339 299.39339 150.89339 15089.33933 2.39339 299.3934 150.89339 15089.33921 2.39339 299.39339 150.89339 15089.33900 2020-01-01 2020-01-02 2020-01-01 00:13:17 2020-01-02 03:41:38 2020-01-01 00:13:17.000 2020-01-02 03:41:38.000 797 99698 50247.5 5024750 797 99698 50247.5 5024750 -32373 32765 5027.66 502766 -128 127 -2.74 -274 -798 2 10788 99699 2.39639 299.39639 150.89639 15089.63963 2.39639 299.3964 150.89639 15089.63936 2.39639 299.39639 150.89639 15089.63900 2020-01-01 2020-01-02 2020-01-01 00:13:18 2020-01-02 03:41:39 2020-01-01 00:13:18.000 2020-01-02 03:41:39.000 798 99699 50248.5 5024850 798 99699 50248.5 5024850 -32372 32766 5028.66 502866 -128 123 -4.3 -430 -799 2 10789 99700 2.39939 299.39939 150.89939 15089.93993 2.39939 299.3994 150.8994 15089.94011 2.39939 299.39939 150.89939 15089.93900 2020-01-01 2020-01-02 2020-01-01 00:13:19 2020-01-02 03:41:40 2020-01-01 00:13:19.000 2020-01-02 03:41:40.000 799 99700 50249.5 5024950 799 99700 50249.5 5024950 -32371 32767 5029.66 502966 -127 124 -3.3 -330 -8 2 1007 9998 0.02402 300.02402 150.02402 15152.42642 0.02402 300.02402 150.02402 15152.42607 0.02402 300.02402 150.02402 15152.42602 2020-01-01 2020-01-02 2020-01-01 00:00:08 2020-01-02 03:45:08 2020-01-01 00:00:08.000 2020-01-02 03:45:08.000 8 99908 49958 5045758 8 99908 49958 5045758 -32561 32374 4537.009900990099 458238 -125 126 -0.019801980198019802 -2 -80 2 10070 99980 0.24024 300.24024 150.24024 15174.26426 0.24024 300.24023 150.24023 15174.26397 0.24024 300.24024 150.24024 15174.26424 2020-01-01 2020-01-02 2020-01-01 00:01:20 2020-01-02 03:46:20 2020-01-01 00:01:20.000 2020-01-02 03:46:20.000 80 99980 50030 5053030 80 99980 50030 5053030 -32489 32446 4609.009900990099 465510 -128 123 -4.0594059405940595 -410 -800 2 10790 99701 2.4024 299.4024 150.9024 15090.24024 2.4024 299.4024 150.9024 15090.24041 2.40240 299.40240 150.90240 15090.24000 2020-01-01 2020-01-02 2020-01-01 00:13:20 2020-01-02 03:41:41 2020-01-01 00:13:20.000 2020-01-02 03:41:41.000 800 99701 50250.5 5025050 800 99701 50250.5 5025050 -32768 32167 4375.3 437530 -126 125 -2.3 -230 -801 2 10791 99702 2.4054 299.4054 150.9054 15090.54054 2.4054 299.4054 150.9054 15090.54058 2.40540 299.40540 150.90540 15090.54000 2020-01-01 2020-01-02 2020-01-01 00:13:21 2020-01-02 03:41:42 2020-01-01 00:13:21.000 2020-01-02 03:41:42.000 801 99702 50251.5 5025150 801 99702 50251.5 5025150 -32767 32168 4376.3 437630 -125 126 -1.3 -130 -802 2 10792 99703 2.4084 299.4084 150.9084 15090.84084 2.4084 299.40842 150.9084 15090.84069 2.40840 299.40840 150.90840 15090.84000 2020-01-01 2020-01-02 2020-01-01 00:13:22 2020-01-02 03:41:43 2020-01-01 00:13:22.000 2020-01-02 03:41:43.000 802 99703 50252.5 5025250 802 99703 50252.5 5025250 -32766 32169 4377.3 437730 -124 127 -0.3 -30 -803 2 10793 99704 2.41141 299.41141 150.91141 15091.14114 2.41141 299.4114 150.9114 15091.14098 2.41141 299.41141 150.91141 15091.14100 2020-01-01 2020-01-02 2020-01-01 00:13:23 2020-01-02 03:41:44 2020-01-01 00:13:23.000 2020-01-02 03:41:44.000 803 99704 50253.5 5025350 803 99704 50253.5 5025350 -32765 32170 4378.3 437830 -128 127 -1.86 -186 -804 2 10794 99705 2.41441 299.41441 150.91441 15091.44144 2.41441 299.41443 150.91441 15091.44158 2.41441 299.41441 150.91441 15091.44100 2020-01-01 2020-01-02 2020-01-01 00:13:24 2020-01-02 03:41:45 2020-01-01 00:13:24.000 2020-01-02 03:41:45.000 804 99705 50254.5 5025450 804 99705 50254.5 5025450 -32764 32171 4379.3 437930 -128 123 -3.42 -342 +789 2 10779 99690 2.36936 299.36936 150.86936 15086.93693 2.36936 299.36935 150.86936 15086.93672 2.36936 299.36936 150.86936000000006 15086.93600 2020-01-01 2020-01-02 2020-01-01 00:13:09 2020-01-02 03:41:30 2020-01-01 00:13:09.000 2020-01-02 03:41:30.000 789 99690 50239.5 5023950 789 99690 50239.5 5023950 -32381 32757 5019.66 501966 -125 127 -0.5 -50 +79 2 10069 99979 0.23723 300.23723 150.23723 15173.96096 0.23723 300.23724 150.23724 15173.96129 0.23723 300.23723 150.23723000000015 15173.96023 2020-01-01 2020-01-02 2020-01-01 00:01:19 2020-01-02 03:46:19 2020-01-01 00:01:19.000 2020-01-02 03:46:19.000 79 99979 50029 5052929 79 99979 50029 5052929 -32490 32445 4608.009900990099 465409 -128 127 -2.5247524752475248 -255 +790 2 10780 99691 2.37237 299.37237 150.87237 15087.23723 2.37237 299.37238 150.87237 15087.23747 2.37237 299.37237 150.87236999999982 15087.23700 2020-01-01 2020-01-02 2020-01-01 00:13:10 2020-01-02 03:41:31 2020-01-01 00:13:10.000 2020-01-02 03:41:31.000 790 99691 50240.5 5024050 790 99691 50240.5 5024050 -32380 32758 5020.66 502066 -128 127 -2.06 -206 +791 2 10781 99692 2.37537 299.37537 150.87537 15087.53753 2.37537 299.37537 150.87537 15087.53761 2.37537 299.37537 150.87536999999986 15087.53700 2020-01-01 2020-01-02 2020-01-01 00:13:11 2020-01-02 03:41:32 2020-01-01 00:13:11.000 2020-01-02 03:41:32.000 791 99692 50241.5 5024150 791 99692 50241.5 5024150 -32379 32759 5021.66 502166 -128 127 -3.62 -362 +792 2 10782 99693 2.37837 299.37837 150.87837 15087.83783 2.37837 299.3784 150.87837 15087.83775 2.37837 299.37837 150.87837000000016 15087.83700 2020-01-01 2020-01-02 2020-01-01 00:13:12 2020-01-02 03:41:33 2020-01-01 00:13:12.000 2020-01-02 03:41:33.000 792 99693 50242.5 5024250 792 99693 50242.5 5024250 -32378 32760 5022.66 502266 -128 123 -5.18 -518 +793 2 10783 99694 2.38138 299.38138 150.88138 15088.13813 2.38138 299.38138 150.88137 15088.13789 2.38138 299.38138 150.8813800000001 15088.13800 2020-01-01 2020-01-02 2020-01-01 00:13:13 2020-01-02 03:41:34 2020-01-01 00:13:13.000 2020-01-02 03:41:34.000 793 99694 50243.5 5024350 793 99694 50243.5 5024350 -32377 32761 5023.66 502366 -127 124 -4.18 -418 +794 2 10784 99695 2.38438 299.38438 150.88438 15088.43843 2.38438 299.3844 150.88438 15088.43864 2.38438 299.38438 150.88438000000005 15088.43800 2020-01-01 2020-01-02 2020-01-01 00:13:14 2020-01-02 03:41:35 2020-01-01 00:13:14.000 2020-01-02 03:41:35.000 794 99695 50244.5 5024450 794 99695 50244.5 5024450 -32376 32762 5024.66 502466 -126 125 -3.18 -318 +795 2 10785 99696 2.38738 299.38738 150.88738 15088.73873 2.38738 299.3874 150.88738 15088.73894 2.38738 299.38738 150.8873800000001 15088.73800 2020-01-01 2020-01-02 2020-01-01 00:13:15 2020-01-02 03:41:36 2020-01-01 00:13:15.000 2020-01-02 03:41:36.000 795 99696 50245.5 5024550 795 99696 50245.5 5024550 -32375 32763 5025.66 502566 -125 126 -2.18 -218 +796 2 10786 99697 2.39039 299.39039 150.89039 15089.03903 2.39039 299.39038 150.89039 15089.03908 2.39039 299.39039 150.89038999999997 15089.03900 2020-01-01 2020-01-02 2020-01-01 00:13:16 2020-01-02 03:41:37 2020-01-01 00:13:16.000 2020-01-02 03:41:37.000 796 99697 50246.5 5024650 796 99697 50246.5 5024650 -32374 32764 5026.66 502666 -124 127 -1.18 -118 +797 2 10787 99698 2.39339 299.39339 150.89339 15089.33933 2.39339 299.3934 150.89339 15089.33921 2.39339 299.39339 150.8933899999999 15089.33900 2020-01-01 2020-01-02 2020-01-01 00:13:17 2020-01-02 03:41:38 2020-01-01 00:13:17.000 2020-01-02 03:41:38.000 797 99698 50247.5 5024750 797 99698 50247.5 5024750 -32373 32765 5027.66 502766 -128 127 -2.74 -274 +798 2 10788 99699 2.39639 299.39639 150.89639 15089.63963 2.39639 299.3964 150.89639 15089.63936 2.39639 299.39639 150.8963899999999 15089.63900 2020-01-01 2020-01-02 2020-01-01 00:13:18 2020-01-02 03:41:39 2020-01-01 00:13:18.000 2020-01-02 03:41:39.000 798 99699 50248.5 5024850 798 99699 50248.5 5024850 -32372 32766 5028.66 502866 -128 123 -4.3 -430 +799 2 10789 99700 2.39939 299.39939 150.89939 15089.93993 2.39939 299.3994 150.8994 15089.94011 2.39939 299.39939 150.89938999999998 15089.93900 2020-01-01 2020-01-02 2020-01-01 00:13:19 2020-01-02 03:41:40 2020-01-01 00:13:19.000 2020-01-02 03:41:40.000 799 99700 50249.5 5024950 799 99700 50249.5 5024950 -32371 32767 5029.66 502966 -127 124 -3.3 -330 +8 2 1007 9998 0.02402 300.02402 150.02402 15152.42642 0.02402 300.02402 150.02402 15152.42607 0.02402 300.02402 150.02402000000012 15152.42602 2020-01-01 2020-01-02 2020-01-01 00:00:08 2020-01-02 03:45:08 2020-01-01 00:00:08.000 2020-01-02 03:45:08.000 8 99908 49958 5045758 8 99908 49958 5045758 -32561 32374 4537.009900990099 458238 -125 126 -0.019801980198019802 -2 +80 2 10070 99980 0.24024 300.24024 150.24024 15174.26426 0.24024 300.24023 150.24023 15174.26397 0.24024 300.24024 150.2402399999998 15174.26424 2020-01-01 2020-01-02 2020-01-01 00:01:20 2020-01-02 03:46:20 2020-01-01 00:01:20.000 2020-01-02 03:46:20.000 80 99980 50030 5053030 80 99980 50030 5053030 -32489 32446 4609.009900990099 465510 -128 123 -4.0594059405940595 -410 +800 2 10790 99701 2.4024 299.4024 150.9024 15090.24024 2.4024 299.4024 150.9024 15090.24041 2.40240 299.40240 150.90240000000017 15090.24000 2020-01-01 2020-01-02 2020-01-01 00:13:20 2020-01-02 03:41:41 2020-01-01 00:13:20.000 2020-01-02 03:41:41.000 800 99701 50250.5 5025050 800 99701 50250.5 5025050 -32768 32167 4375.3 437530 -126 125 -2.3 -230 +801 2 10791 99702 2.4054 299.4054 150.9054 15090.54054 2.4054 299.4054 150.9054 15090.54058 2.40540 299.40540 150.90539999999984 15090.54000 2020-01-01 2020-01-02 2020-01-01 00:13:21 2020-01-02 03:41:42 2020-01-01 00:13:21.000 2020-01-02 03:41:42.000 801 99702 50251.5 5025150 801 99702 50251.5 5025150 -32767 32168 4376.3 437630 -125 126 -1.3 -130 +802 2 10792 99703 2.4084 299.4084 150.9084 15090.84084 2.4084 299.40842 150.9084 15090.84069 2.40840 299.40840 150.90840000000009 15090.84000 2020-01-01 2020-01-02 2020-01-01 00:13:22 2020-01-02 03:41:43 2020-01-01 00:13:22.000 2020-01-02 03:41:43.000 802 99703 50252.5 5025250 802 99703 50252.5 5025250 -32766 32169 4377.3 437730 -124 127 -0.3 -30 +803 2 10793 99704 2.41141 299.41141 150.91141 15091.14114 2.41141 299.4114 150.9114 15091.14098 2.41141 299.41141 150.91141000000007 15091.14100 2020-01-01 2020-01-02 2020-01-01 00:13:23 2020-01-02 03:41:44 2020-01-01 00:13:23.000 2020-01-02 03:41:44.000 803 99704 50253.5 5025350 803 99704 50253.5 5025350 -32765 32170 4378.3 437830 -128 127 -1.86 -186 +804 2 10794 99705 2.41441 299.41441 150.91441 15091.44144 2.41441 299.41443 150.91441 15091.44158 2.41441 299.41441 150.91440999999995 15091.44100 2020-01-01 2020-01-02 2020-01-01 00:13:24 2020-01-02 03:41:45 2020-01-01 00:13:24.000 2020-01-02 03:41:45.000 804 99705 50254.5 5025450 804 99705 50254.5 5025450 -32764 32171 4379.3 437930 -128 123 -3.42 -342 805 2 10795 99706 2.41741 299.41741 150.91741 15091.74174 2.41741 299.41742 150.91741 15091.74188 2.41741 299.41741 150.91741 15091.74100 2020-01-01 2020-01-02 2020-01-01 00:13:25 2020-01-02 03:41:46 2020-01-01 00:13:25.000 2020-01-02 03:41:46.000 805 99706 50255.5 5025550 805 99706 50255.5 5025550 -32763 32172 4380.3 438030 -127 124 -2.42 -242 -806 2 10796 99707 2.42042 299.42042 150.92042 15092.04204 2.42042 299.4204 150.92042 15092.04205 2.42042 299.42042 150.92042 15092.04200 2020-01-01 2020-01-02 2020-01-01 00:13:26 2020-01-02 03:41:47 2020-01-01 00:13:26.000 2020-01-02 03:41:47.000 806 99707 50256.5 5025650 806 99707 50256.5 5025650 -32762 32173 4381.3 438130 -126 125 -1.42 -142 -807 2 10797 99708 2.42342 299.42342 150.92342 15092.34234 2.42342 299.42343 150.92342 15092.34216 2.42342 299.42342 150.92342 15092.34200 2020-01-01 2020-01-02 2020-01-01 00:13:27 2020-01-02 03:41:48 2020-01-01 00:13:27.000 2020-01-02 03:41:48.000 807 99708 50257.5 5025750 807 99708 50257.5 5025750 -32761 32174 4382.3 438230 -125 126 -0.42 -42 -808 2 10798 99709 2.42642 299.42642 150.92642 15092.64264 2.42642 299.42642 150.92642 15092.64245 2.42642 299.42642 150.92642 15092.64200 2020-01-01 2020-01-02 2020-01-01 00:13:28 2020-01-02 03:41:49 2020-01-01 00:13:28.000 2020-01-02 03:41:49.000 808 99709 50258.5 5025850 808 99709 50258.5 5025850 -32760 32175 4383.3 438330 -124 127 0.58 58 -809 2 10799 99710 2.42942 299.42942 150.92942 15092.94294 2.42942 299.42944 150.92943 15092.94305 2.42942 299.42942 150.92942 15092.94200 2020-01-01 2020-01-02 2020-01-01 00:13:29 2020-01-02 03:41:50 2020-01-01 00:13:29.000 2020-01-02 03:41:50.000 809 99710 50259.5 5025950 809 99710 50259.5 5025950 -32759 32176 4384.3 438430 -128 127 -0.98 -98 -81 2 10071 99981 0.24324 300.24324 150.24324 15174.56756 0.24324 300.24326 150.24324 15174.56758 0.24324 300.24324 150.24324 15174.56724 2020-01-01 2020-01-02 2020-01-01 00:01:21 2020-01-02 03:46:21 2020-01-01 00:01:21.000 2020-01-02 03:46:21.000 81 99981 50031 5053131 81 99981 50031 5053131 -32488 32447 4610.009900990099 465611 -127 124 -3.0594059405940595 -309 -810 2 10800 99711 2.43243 299.43243 150.93243 15093.24324 2.43243 299.43243 150.93243 15093.24338 2.43243 299.43243 150.93243 15093.24300 2020-01-01 2020-01-02 2020-01-01 00:13:30 2020-01-02 03:41:51 2020-01-01 00:13:30.000 2020-01-02 03:41:51.000 810 99711 50260.5 5026050 810 99711 50260.5 5026050 -32758 32177 4385.3 438530 -128 127 -2.54 -254 -811 2 10801 99712 2.43543 299.43543 150.93543 15093.54354 2.43543 299.43542 150.93543 15093.54353 2.43543 299.43543 150.93543 15093.54300 2020-01-01 2020-01-02 2020-01-01 00:13:31 2020-01-02 03:41:52 2020-01-01 00:13:31.000 2020-01-02 03:41:52.000 811 99712 50261.5 5026150 811 99712 50261.5 5026150 -32757 32178 4386.3 438630 -128 124 -4.1 -410 -812 2 10802 99713 2.43843 299.43843 150.93843 15093.84384 2.43843 299.43845 150.93844 15093.84428 2.43843 299.43843 150.93843 15093.84300 2020-01-01 2020-01-02 2020-01-01 00:13:32 2020-01-02 03:41:53 2020-01-01 00:13:32.000 2020-01-02 03:41:53.000 812 99713 50262.5 5026250 812 99713 50262.5 5026250 -32756 32179 4387.3 438730 -127 125 -3.1 -310 +806 2 10796 99707 2.42042 299.42042 150.92042 15092.04204 2.42042 299.4204 150.92042 15092.04205 2.42042 299.42042 150.92042000000018 15092.04200 2020-01-01 2020-01-02 2020-01-01 00:13:26 2020-01-02 03:41:47 2020-01-01 00:13:26.000 2020-01-02 03:41:47.000 806 99707 50256.5 5025650 806 99707 50256.5 5025650 -32762 32173 4381.3 438130 -126 125 -1.42 -142 +807 2 10797 99708 2.42342 299.42342 150.92342 15092.34234 2.42342 299.42343 150.92342 15092.34216 2.42342 299.42342 150.92341999999985 15092.34200 2020-01-01 2020-01-02 2020-01-01 00:13:27 2020-01-02 03:41:48 2020-01-01 00:13:27.000 2020-01-02 03:41:48.000 807 99708 50257.5 5025750 807 99708 50257.5 5025750 -32761 32174 4382.3 438230 -125 126 -0.42 -42 +808 2 10798 99709 2.42642 299.42642 150.92642 15092.64264 2.42642 299.42642 150.92642 15092.64245 2.42642 299.42642 150.92641999999987 15092.64200 2020-01-01 2020-01-02 2020-01-01 00:13:28 2020-01-02 03:41:49 2020-01-01 00:13:28.000 2020-01-02 03:41:49.000 808 99709 50258.5 5025850 808 99709 50258.5 5025850 -32760 32175 4383.3 438330 -124 127 0.58 58 +809 2 10799 99710 2.42942 299.42942 150.92942 15092.94294 2.42942 299.42944 150.92943 15092.94305 2.42942 299.42942 150.9294200000002 15092.94200 2020-01-01 2020-01-02 2020-01-01 00:13:29 2020-01-02 03:41:50 2020-01-01 00:13:29.000 2020-01-02 03:41:50.000 809 99710 50259.5 5025950 809 99710 50259.5 5025950 -32759 32176 4384.3 438430 -128 127 -0.98 -98 +81 2 10071 99981 0.24324 300.24324 150.24324 15174.56756 0.24324 300.24326 150.24324 15174.56758 0.24324 300.24324 150.24323999999987 15174.56724 2020-01-01 2020-01-02 2020-01-01 00:01:21 2020-01-02 03:46:21 2020-01-01 00:01:21.000 2020-01-02 03:46:21.000 81 99981 50031 5053131 81 99981 50031 5053131 -32488 32447 4610.009900990099 465611 -127 124 -3.0594059405940595 -309 +810 2 10800 99711 2.43243 299.43243 150.93243 15093.24324 2.43243 299.43243 150.93243 15093.24338 2.43243 299.43243 150.93243000000012 15093.24300 2020-01-01 2020-01-02 2020-01-01 00:13:30 2020-01-02 03:41:51 2020-01-01 00:13:30.000 2020-01-02 03:41:51.000 810 99711 50260.5 5026050 810 99711 50260.5 5026050 -32758 32177 4385.3 438530 -128 127 -2.54 -254 +811 2 10801 99712 2.43543 299.43543 150.93543 15093.54354 2.43543 299.43542 150.93543 15093.54353 2.43543 299.43543 150.9354299999998 15093.54300 2020-01-01 2020-01-02 2020-01-01 00:13:31 2020-01-02 03:41:52 2020-01-01 00:13:31.000 2020-01-02 03:41:52.000 811 99712 50261.5 5026150 811 99712 50261.5 5026150 -32757 32178 4386.3 438630 -128 124 -4.1 -410 +812 2 10802 99713 2.43843 299.43843 150.93843 15093.84384 2.43843 299.43845 150.93844 15093.84428 2.43843 299.43843 150.9384300000001 15093.84300 2020-01-01 2020-01-02 2020-01-01 00:13:32 2020-01-02 03:41:53 2020-01-01 00:13:32.000 2020-01-02 03:41:53.000 812 99713 50262.5 5026250 812 99713 50262.5 5026250 -32756 32179 4387.3 438730 -127 125 -3.1 -310 813 2 10803 99714 2.44144 299.44144 150.94144 15094.14414 2.44144 299.44144 150.94143 15094.14392 2.44144 299.44144 150.94144 15094.14400 2020-01-01 2020-01-02 2020-01-01 00:13:33 2020-01-02 03:41:54 2020-01-01 00:13:33.000 2020-01-02 03:41:54.000 813 99714 50263.5 5026350 813 99714 50263.5 5026350 -32755 32180 4388.3 438830 -126 126 -2.1 -210 -814 2 10804 99715 2.44444 299.44444 150.94444 15094.44444 2.44444 299.44446 150.94444 15094.44452 2.44444 299.44444 150.94444 15094.44400 2020-01-01 2020-01-02 2020-01-01 00:13:34 2020-01-02 03:41:55 2020-01-01 00:13:34.000 2020-01-02 03:41:55.000 814 99715 50264.5 5026450 814 99715 50264.5 5026450 -32754 32181 4389.3 438930 -125 127 -1.1 -110 -815 2 10805 99716 2.44744 299.44744 150.94744 15094.74474 2.44744 299.44745 150.94744 15094.74485 2.44744 299.44744 150.94744 15094.74400 2020-01-01 2020-01-02 2020-01-01 00:13:35 2020-01-02 03:41:56 2020-01-01 00:13:35.000 2020-01-02 03:41:56.000 815 99716 50265.5 5026550 815 99716 50265.5 5026550 -32753 32182 4390.3 439030 -128 127 -2.66 -266 -816 2 10806 99717 2.45045 299.45045 150.95045 15095.04504 2.45045 299.45044 150.95045 15095.045 2.45045 299.45045 150.95045 15095.04500 2020-01-01 2020-01-02 2020-01-01 00:13:36 2020-01-02 03:41:57 2020-01-01 00:13:36.000 2020-01-02 03:41:57.000 816 99717 50266.5 5026650 816 99717 50266.5 5026650 -32752 32183 4391.3 439130 -128 127 -4.22 -422 -817 2 10807 99718 2.45345 299.45345 150.95345 15095.34534 2.45345 299.45346 150.95345 15095.34574 2.45345 299.45345 150.95345 15095.34500 2020-01-01 2020-01-02 2020-01-01 00:13:37 2020-01-02 03:41:58 2020-01-01 00:13:37.000 2020-01-02 03:41:58.000 817 99718 50267.5 5026750 817 99718 50267.5 5026750 -32751 32184 4392.3 439230 -128 123 -5.78 -578 -818 2 10808 99719 2.45645 299.45645 150.95645 15095.64564 2.45645 299.45645 150.95645 15095.64539 2.45645 299.45645 150.95645 15095.64500 2020-01-01 2020-01-02 2020-01-01 00:13:38 2020-01-02 03:41:59 2020-01-01 00:13:38.000 2020-01-02 03:41:59.000 818 99719 50268.5 5026850 818 99719 50268.5 5026850 -32750 32185 4393.3 439330 -127 124 -4.78 -478 -819 2 10809 99720 2.45945 299.45945 150.95945 15095.94594 2.45945 299.45947 150.95946 15095.94602 2.45945 299.45945 150.95945 15095.94500 2020-01-01 2020-01-02 2020-01-01 00:13:39 2020-01-02 03:42:00 2020-01-01 00:13:39.000 2020-01-02 03:42:00.000 819 99720 50269.5 5026950 819 99720 50269.5 5026950 -32749 32186 4394.3 439430 -126 125 -3.78 -378 -82 2 10072 99982 0.24624 300.24624 150.24624 15174.87087 0.24624 300.24625 150.24624 15174.87088 0.24624 300.24624 150.24624 15174.87024 2020-01-01 2020-01-02 2020-01-01 00:01:22 2020-01-02 03:46:22 2020-01-01 00:01:22.000 2020-01-02 03:46:22.000 82 99982 50032 5053232 82 99982 50032 5053232 -32487 32448 4611.009900990099 465712 -126 125 -2.0594059405940595 -208 -820 2 10810 99721 2.46246 299.46246 150.96246 15096.24624 2.46246 299.46246 150.96246 15096.24633 2.46246 299.46246 150.96246 15096.24600 2020-01-01 2020-01-02 2020-01-01 00:13:40 2020-01-02 03:42:01 2020-01-01 00:13:40.000 2020-01-02 03:42:01.000 820 99721 50270.5 5027050 820 99721 50270.5 5027050 -32748 32187 4395.3 439530 -125 126 -2.78 -278 -821 2 10811 99722 2.46546 299.46546 150.96546 15096.54654 2.46546 299.46545 150.96546 15096.54646 2.46546 299.46546 150.96546 15096.54600 2020-01-01 2020-01-02 2020-01-01 00:13:41 2020-01-02 03:42:02 2020-01-01 00:13:41.000 2020-01-02 03:42:02.000 821 99722 50271.5 5027150 821 99722 50271.5 5027150 -32747 32188 4396.3 439630 -124 127 -1.78 -178 -822 2 10812 99723 2.46846 299.46846 150.96846 15096.84684 2.46846 299.46848 150.96847 15096.84721 2.46846 299.46846 150.96846 15096.84600 2020-01-01 2020-01-02 2020-01-01 00:13:42 2020-01-02 03:42:03 2020-01-01 00:13:42.000 2020-01-02 03:42:03.000 822 99723 50272.5 5027250 822 99723 50272.5 5027250 -32746 32189 4397.3 439730 -128 127 -3.34 -334 -823 2 10813 99724 2.47147 299.47147 150.97147 15097.14714 2.47147 299.47147 150.97146 15097.14686 2.47147 299.47147 150.97147 15097.14700 2020-01-01 2020-01-02 2020-01-01 00:13:43 2020-01-02 03:42:04 2020-01-01 00:13:43.000 2020-01-02 03:42:04.000 823 99724 50273.5 5027350 823 99724 50273.5 5027350 -32745 32190 4398.3 439830 -128 123 -4.9 -490 -824 2 10814 99725 2.47447 299.47447 150.97447 15097.44744 2.47447 299.4745 150.97447 15097.44749 2.47447 299.47447 150.97447 15097.44700 2020-01-01 2020-01-02 2020-01-01 00:13:44 2020-01-02 03:42:05 2020-01-01 00:13:44.000 2020-01-02 03:42:05.000 824 99725 50274.5 5027450 824 99725 50274.5 5027450 -32744 32191 4399.3 439930 -127 124 -3.9 -390 -825 2 10815 99726 2.47747 299.47747 150.97747 15097.74774 2.47747 299.47748 150.97747 15097.74779 2.47747 299.47747 150.97747 15097.74700 2020-01-01 2020-01-02 2020-01-01 00:13:45 2020-01-02 03:42:06 2020-01-01 00:13:45.000 2020-01-02 03:42:06.000 825 99726 50275.5 5027550 825 99726 50275.5 5027550 -32743 32192 4400.3 440030 -126 125 -2.9 -290 -826 2 10816 99727 2.48048 299.48048 150.98048 15098.04804 2.48048 299.48047 150.98048 15098.04809 2.48048 299.48048 150.98048 15098.04800 2020-01-01 2020-01-02 2020-01-01 00:13:46 2020-01-02 03:42:07 2020-01-01 00:13:46.000 2020-01-02 03:42:07.000 826 99727 50276.5 5027650 826 99727 50276.5 5027650 -32742 32193 4401.3 440130 -125 126 -1.9 -190 -827 2 10817 99728 2.48348 299.48348 150.98348 15098.34834 2.48348 299.4835 150.98348 15098.34869 2.48348 299.48348 150.98348 15098.34800 2020-01-01 2020-01-02 2020-01-01 00:13:47 2020-01-02 03:42:08 2020-01-01 00:13:47.000 2020-01-02 03:42:08.000 827 99728 50277.5 5027750 827 99728 50277.5 5027750 -32741 32194 4402.3 440230 -124 127 -0.9 -90 -828 2 10818 99729 2.48648 299.48648 150.98648 15098.64864 2.48648 299.48648 150.98648 15098.64837 2.48648 299.48648 150.98648 15098.64800 2020-01-01 2020-01-02 2020-01-01 00:13:48 2020-01-02 03:42:09 2020-01-01 00:13:48.000 2020-01-02 03:42:09.000 828 99729 50278.5 5027850 828 99729 50278.5 5027850 -32740 32195 4403.3 440330 -128 127 -2.46 -246 -829 2 10819 99730 2.48948 299.48948 150.98948 15098.94894 2.48948 299.4895 150.98948 15098.94896 2.48948 299.48948 150.98948 15098.94800 2020-01-01 2020-01-02 2020-01-01 00:13:49 2020-01-02 03:42:10 2020-01-01 00:13:49.000 2020-01-02 03:42:10.000 829 99730 50279.5 5027950 829 99730 50279.5 5027950 -32739 32196 4404.3 440430 -128 123 -4.02 -402 -83 2 10073 99983 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924 15175.17324 2020-01-01 2020-01-02 2020-01-01 00:01:23 2020-01-02 03:46:23 2020-01-01 00:01:23.000 2020-01-02 03:46:23.000 83 99983 50033 5053333 83 99983 50033 5053333 -32486 32449 4612.009900990099 465813 -125 126 -1.0594059405940595 -107 -830 2 10820 99731 2.49249 299.49249 150.99249 15099.24924 2.49249 299.4925 150.99249 15099.24926 2.49249 299.49249 150.99249 15099.24900 2020-01-01 2020-01-02 2020-01-01 00:13:50 2020-01-02 03:42:11 2020-01-01 00:13:50.000 2020-01-02 03:42:11.000 830 99731 50280.5 5028050 830 99731 50280.5 5028050 -32738 32197 4405.3 440530 -127 124 -3.02 -302 -831 2 10821 99732 2.49549 299.49549 150.99549 15099.54954 2.49549 299.49548 150.99549 15099.54956 2.49549 299.49549 150.99549 15099.54900 2020-01-01 2020-01-02 2020-01-01 00:13:51 2020-01-02 03:42:12 2020-01-01 00:13:51.000 2020-01-02 03:42:12.000 831 99732 50281.5 5028150 831 99732 50281.5 5028150 -32737 32198 4406.3 440630 -126 125 -2.02 -202 -832 2 10822 99733 2.49849 299.49849 150.99849 15099.84984 2.49849 299.4985 150.9985 15099.85016 2.49849 299.49849 150.99849 15099.84900 2020-01-01 2020-01-02 2020-01-01 00:13:52 2020-01-02 03:42:13 2020-01-01 00:13:52.000 2020-01-02 03:42:13.000 832 99733 50282.5 5028250 832 99733 50282.5 5028250 -32736 32199 4407.3 440730 -125 126 -1.02 -102 -833 2 10823 99734 2.5015 299.5015 151.0015 15100.15015 2.5015 299.5015 151.00149 15100.14983 2.50150 299.50150 151.00150 15100.15000 2020-01-01 2020-01-02 2020-01-01 00:13:53 2020-01-02 03:42:14 2020-01-01 00:13:53.000 2020-01-02 03:42:14.000 833 99734 50283.5 5028350 833 99734 50283.5 5028350 -32735 32200 4408.3 440830 -124 127 -0.02 -2 -834 2 10824 99735 2.5045 299.5045 151.0045 15100.45045 2.5045 299.50452 151.0045 15100.45043 2.50450 299.50450 151.00450 15100.45000 2020-01-01 2020-01-02 2020-01-01 00:13:54 2020-01-02 03:42:15 2020-01-01 00:13:54.000 2020-01-02 03:42:15.000 834 99735 50284.5 5028450 834 99735 50284.5 5028450 -32734 32201 4409.3 440930 -128 127 -1.58 -158 -835 2 10825 99736 2.5075 299.5075 151.0075 15100.75075 2.5075 299.5075 151.0075 15100.75073 2.50750 299.50750 151.00750 15100.75000 2020-01-01 2020-01-02 2020-01-01 00:13:55 2020-01-02 03:42:16 2020-01-01 00:13:55.000 2020-01-02 03:42:16.000 835 99736 50285.5 5028550 835 99736 50285.5 5028550 -32733 32202 4410.3 441030 -128 123 -3.14 -314 -836 2 10826 99737 2.51051 299.51051 151.01051 15101.05105 2.51051 299.5105 151.01051 15101.05103 2.51051 299.51051 151.01051 15101.05100 2020-01-01 2020-01-02 2020-01-01 00:13:56 2020-01-02 03:42:17 2020-01-01 00:13:56.000 2020-01-02 03:42:17.000 836 99737 50286.5 5028650 836 99737 50286.5 5028650 -32732 32203 4411.3 441130 -127 124 -2.14 -214 -837 2 10827 99738 2.51351 299.51351 151.01351 15101.35135 2.51351 299.51352 151.01351 15101.35162 2.51351 299.51351 151.01351 15101.35100 2020-01-01 2020-01-02 2020-01-01 00:13:57 2020-01-02 03:42:18 2020-01-01 00:13:57.000 2020-01-02 03:42:18.000 837 99738 50287.5 5028750 837 99738 50287.5 5028750 -32731 32204 4412.3 441230 -126 125 -1.14 -114 -838 2 10828 99739 2.51651 299.51651 151.01651 15101.65165 2.51651 299.5165 151.01651 15101.6513 2.51651 299.51651 151.01651 15101.65100 2020-01-01 2020-01-02 2020-01-01 00:13:58 2020-01-02 03:42:19 2020-01-01 00:13:58.000 2020-01-02 03:42:19.000 838 99739 50288.5 5028850 838 99739 50288.5 5028850 -32730 32205 4413.3 441330 -125 126 -0.14 -14 -839 2 10829 99740 2.51951 299.51951 151.01951 15101.95195 2.51951 299.51953 151.01951 15101.9519 2.51951 299.51951 151.01951 15101.95100 2020-01-01 2020-01-02 2020-01-01 00:13:59 2020-01-02 03:42:20 2020-01-01 00:13:59.000 2020-01-02 03:42:20.000 839 99740 50289.5 5028950 839 99740 50289.5 5028950 -32729 32206 4414.3 441430 -124 127 0.86 86 -84 2 10074 99984 0.25225 300.25225 150.25225 15175.47747 0.25225 300.25226 150.25225 15175.47778 0.25225 300.25225 150.25225 15175.47725 2020-01-01 2020-01-02 2020-01-01 00:01:24 2020-01-02 03:46:24 2020-01-01 00:01:24.000 2020-01-02 03:46:24.000 84 99984 50034 5053434 84 99984 50034 5053434 -32485 32450 4613.009900990099 465914 -124 127 -0.0594059405940594 -6 -840 2 10830 99741 2.52252 299.52252 151.02252 15102.25225 2.52252 299.52252 151.02252 15102.2522 2.52252 299.52252 151.02252 15102.25200 2020-01-01 2020-01-02 2020-01-01 00:14:00 2020-01-02 03:42:21 2020-01-01 00:14:00.000 2020-01-02 03:42:21.000 840 99741 50290.5 5029050 840 99741 50290.5 5029050 -32728 32207 4415.3 441530 -128 127 -0.7 -70 -841 2 10831 99742 2.52552 299.52552 151.02552 15102.55255 2.52552 299.5255 151.02552 15102.5525 2.52552 299.52552 151.02552 15102.55200 2020-01-01 2020-01-02 2020-01-01 00:14:01 2020-01-02 03:42:22 2020-01-01 00:14:01.000 2020-01-02 03:42:22.000 841 99742 50291.5 5029150 841 99742 50291.5 5029150 -32727 32208 4416.3 441630 -128 127 -2.26 -226 -842 2 10832 99743 2.52852 299.52852 151.02852 15102.85285 2.52852 299.52853 151.02853 15102.85313 2.52852 299.52852 151.02852 15102.85200 2020-01-01 2020-01-02 2020-01-01 00:14:02 2020-01-02 03:42:23 2020-01-01 00:14:02.000 2020-01-02 03:42:23.000 842 99743 50292.5 5029250 842 99743 50292.5 5029250 -32726 32209 4417.3 441730 -128 123 -3.82 -382 -843 2 10833 99744 2.53153 299.53153 151.03153 15103.15315 2.53153 299.53152 151.03152 15103.15278 2.53153 299.53153 151.03153 15103.15300 2020-01-01 2020-01-02 2020-01-01 00:14:03 2020-01-02 03:42:24 2020-01-01 00:14:03.000 2020-01-02 03:42:24.000 843 99744 50293.5 5029350 843 99744 50293.5 5029350 -32725 32210 4418.3 441830 -127 124 -2.82 -282 -844 2 10834 99745 2.53453 299.53453 151.03453 15103.45345 2.53453 299.53455 151.03453 15103.45353 2.53453 299.53453 151.03453 15103.45300 2020-01-01 2020-01-02 2020-01-01 00:14:04 2020-01-02 03:42:25 2020-01-01 00:14:04.000 2020-01-02 03:42:25.000 844 99745 50294.5 5029450 844 99745 50294.5 5029450 -32724 32211 4419.3 441930 -126 125 -1.82 -182 -845 2 10835 99746 2.53753 299.53753 151.03753 15103.75375 2.53753 299.53754 151.03753 15103.75366 2.53753 299.53753 151.03753 15103.75300 2020-01-01 2020-01-02 2020-01-01 00:14:05 2020-01-02 03:42:26 2020-01-01 00:14:05.000 2020-01-02 03:42:26.000 845 99746 50295.5 5029550 845 99746 50295.5 5029550 -32723 32212 4420.3 442030 -125 126 -0.82 -82 +814 2 10804 99715 2.44444 299.44444 150.94444 15094.44444 2.44444 299.44446 150.94444 15094.44452 2.44444 299.44444 150.94443999999996 15094.44400 2020-01-01 2020-01-02 2020-01-01 00:13:34 2020-01-02 03:41:55 2020-01-01 00:13:34.000 2020-01-02 03:41:55.000 814 99715 50264.5 5026450 814 99715 50264.5 5026450 -32754 32181 4389.3 438930 -125 127 -1.1 -110 +815 2 10805 99716 2.44744 299.44744 150.94744 15094.74474 2.44744 299.44745 150.94744 15094.74485 2.44744 299.44744 150.94743999999994 15094.74400 2020-01-01 2020-01-02 2020-01-01 00:13:35 2020-01-02 03:41:56 2020-01-01 00:13:35.000 2020-01-02 03:41:56.000 815 99716 50265.5 5026550 815 99716 50265.5 5026550 -32753 32182 4390.3 439030 -128 127 -2.66 -266 +816 2 10806 99717 2.45045 299.45045 150.95045 15095.04504 2.45045 299.45044 150.95045 15095.045 2.45045 299.45045 150.9504500000001 15095.04500 2020-01-01 2020-01-02 2020-01-01 00:13:36 2020-01-02 03:41:57 2020-01-01 00:13:36.000 2020-01-02 03:41:57.000 816 99717 50266.5 5026650 816 99717 50266.5 5026650 -32752 32183 4391.3 439130 -128 127 -4.22 -422 +817 2 10807 99718 2.45345 299.45345 150.95345 15095.34534 2.45345 299.45346 150.95345 15095.34574 2.45345 299.45345 150.9534500000002 15095.34500 2020-01-01 2020-01-02 2020-01-01 00:13:37 2020-01-02 03:41:58 2020-01-01 00:13:37.000 2020-01-02 03:41:58.000 817 99718 50267.5 5026750 817 99718 50267.5 5026750 -32751 32184 4392.3 439230 -128 123 -5.78 -578 +818 2 10808 99719 2.45645 299.45645 150.95645 15095.64564 2.45645 299.45645 150.95645 15095.64539 2.45645 299.45645 150.95644999999985 15095.64500 2020-01-01 2020-01-02 2020-01-01 00:13:38 2020-01-02 03:41:59 2020-01-01 00:13:38.000 2020-01-02 03:41:59.000 818 99719 50268.5 5026850 818 99719 50268.5 5026850 -32750 32185 4393.3 439330 -127 124 -4.78 -478 +819 2 10809 99720 2.45945 299.45945 150.95945 15095.94594 2.45945 299.45947 150.95946 15095.94602 2.45945 299.45945 150.95945000000012 15095.94500 2020-01-01 2020-01-02 2020-01-01 00:13:39 2020-01-02 03:42:00 2020-01-01 00:13:39.000 2020-01-02 03:42:00.000 819 99720 50269.5 5026950 819 99720 50269.5 5026950 -32749 32186 4394.3 439430 -126 125 -3.78 -378 +82 2 10072 99982 0.24624 300.24624 150.24624 15174.87087 0.24624 300.24625 150.24624 15174.87088 0.24624 300.24624 150.2462400000002 15174.87024 2020-01-01 2020-01-02 2020-01-01 00:01:22 2020-01-02 03:46:22 2020-01-01 00:01:22.000 2020-01-02 03:46:22.000 82 99982 50032 5053232 82 99982 50032 5053232 -32487 32448 4611.009900990099 465712 -126 125 -2.0594059405940595 -208 +820 2 10810 99721 2.46246 299.46246 150.96246 15096.24624 2.46246 299.46246 150.96246 15096.24633 2.46246 299.46246 150.9624600000001 15096.24600 2020-01-01 2020-01-02 2020-01-01 00:13:40 2020-01-02 03:42:01 2020-01-01 00:13:40.000 2020-01-02 03:42:01.000 820 99721 50270.5 5027050 820 99721 50270.5 5027050 -32748 32187 4395.3 439530 -125 126 -2.78 -278 +821 2 10811 99722 2.46546 299.46546 150.96546 15096.54654 2.46546 299.46545 150.96546 15096.54646 2.46546 299.46546 150.96545999999995 15096.54600 2020-01-01 2020-01-02 2020-01-01 00:13:41 2020-01-02 03:42:02 2020-01-01 00:13:41.000 2020-01-02 03:42:02.000 821 99722 50271.5 5027150 821 99722 50271.5 5027150 -32747 32188 4396.3 439630 -124 127 -1.78 -178 +822 2 10812 99723 2.46846 299.46846 150.96846 15096.84684 2.46846 299.46848 150.96847 15096.84721 2.46846 299.46846 150.96846000000002 15096.84600 2020-01-01 2020-01-02 2020-01-01 00:13:42 2020-01-02 03:42:03 2020-01-01 00:13:42.000 2020-01-02 03:42:03.000 822 99723 50272.5 5027250 822 99723 50272.5 5027250 -32746 32189 4397.3 439730 -128 127 -3.34 -334 +823 2 10813 99724 2.47147 299.47147 150.97147 15097.14714 2.47147 299.47147 150.97146 15097.14686 2.47147 299.47147 150.97147000000024 15097.14700 2020-01-01 2020-01-02 2020-01-01 00:13:43 2020-01-02 03:42:04 2020-01-01 00:13:43.000 2020-01-02 03:42:04.000 823 99724 50273.5 5027350 823 99724 50273.5 5027350 -32745 32190 4398.3 439830 -128 123 -4.9 -490 +824 2 10814 99725 2.47447 299.47447 150.97447 15097.44744 2.47447 299.4745 150.97447 15097.44749 2.47447 299.47447 150.97446999999985 15097.44700 2020-01-01 2020-01-02 2020-01-01 00:13:44 2020-01-02 03:42:05 2020-01-01 00:13:44.000 2020-01-02 03:42:05.000 824 99725 50274.5 5027450 824 99725 50274.5 5027450 -32744 32191 4399.3 439930 -127 124 -3.9 -390 +825 2 10815 99726 2.47747 299.47747 150.97747 15097.74774 2.47747 299.47748 150.97747 15097.74779 2.47747 299.47747 150.97746999999987 15097.74700 2020-01-01 2020-01-02 2020-01-01 00:13:45 2020-01-02 03:42:06 2020-01-01 00:13:45.000 2020-01-02 03:42:06.000 825 99726 50275.5 5027550 825 99726 50275.5 5027550 -32743 32192 4400.3 440030 -126 125 -2.9 -290 +826 2 10816 99727 2.48048 299.48048 150.98048 15098.04804 2.48048 299.48047 150.98048 15098.04809 2.48048 299.48048 150.9804800000001 15098.04800 2020-01-01 2020-01-02 2020-01-01 00:13:46 2020-01-02 03:42:07 2020-01-01 00:13:46.000 2020-01-02 03:42:07.000 826 99727 50276.5 5027650 826 99727 50276.5 5027650 -32742 32193 4401.3 440130 -125 126 -1.9 -190 +827 2 10817 99728 2.48348 299.48348 150.98348 15098.34834 2.48348 299.4835 150.98348 15098.34869 2.48348 299.48348 150.98348000000013 15098.34800 2020-01-01 2020-01-02 2020-01-01 00:13:47 2020-01-02 03:42:08 2020-01-01 00:13:47.000 2020-01-02 03:42:08.000 827 99728 50277.5 5027750 827 99728 50277.5 5027750 -32741 32194 4402.3 440230 -124 127 -0.9 -90 +828 2 10818 99729 2.48648 299.48648 150.98648 15098.64864 2.48648 299.48648 150.98648 15098.64837 2.48648 299.48648 150.98647999999977 15098.64800 2020-01-01 2020-01-02 2020-01-01 00:13:48 2020-01-02 03:42:09 2020-01-01 00:13:48.000 2020-01-02 03:42:09.000 828 99729 50278.5 5027850 828 99729 50278.5 5027850 -32740 32195 4403.3 440330 -128 127 -2.46 -246 +829 2 10819 99730 2.48948 299.48948 150.98948 15098.94894 2.48948 299.4895 150.98948 15098.94896 2.48948 299.48948 150.98948000000013 15098.94800 2020-01-01 2020-01-02 2020-01-01 00:13:49 2020-01-02 03:42:10 2020-01-01 00:13:49.000 2020-01-02 03:42:10.000 829 99730 50279.5 5027950 829 99730 50279.5 5027950 -32739 32196 4404.3 440430 -128 123 -4.02 -402 +83 2 10073 99983 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24923999999987 15175.17324 2020-01-01 2020-01-02 2020-01-01 00:01:23 2020-01-02 03:46:23 2020-01-01 00:01:23.000 2020-01-02 03:46:23.000 83 99983 50033 5053333 83 99983 50033 5053333 -32486 32449 4612.009900990099 465813 -125 126 -1.0594059405940595 -107 +830 2 10820 99731 2.49249 299.49249 150.99249 15099.24924 2.49249 299.4925 150.99249 15099.24926 2.49249 299.49249 150.99249000000003 15099.24900 2020-01-01 2020-01-02 2020-01-01 00:13:50 2020-01-02 03:42:11 2020-01-01 00:13:50.000 2020-01-02 03:42:11.000 830 99731 50280.5 5028050 830 99731 50280.5 5028050 -32738 32197 4405.3 440530 -127 124 -3.02 -302 +831 2 10821 99732 2.49549 299.49549 150.99549 15099.54954 2.49549 299.49548 150.99549 15099.54956 2.49549 299.49549 150.9954899999999 15099.54900 2020-01-01 2020-01-02 2020-01-01 00:13:51 2020-01-02 03:42:12 2020-01-01 00:13:51.000 2020-01-02 03:42:12.000 831 99732 50281.5 5028150 831 99732 50281.5 5028150 -32737 32198 4406.3 440630 -126 125 -2.02 -202 +832 2 10822 99733 2.49849 299.49849 150.99849 15099.84984 2.49849 299.4985 150.9985 15099.85016 2.49849 299.49849 150.99848999999998 15099.84900 2020-01-01 2020-01-02 2020-01-01 00:13:52 2020-01-02 03:42:13 2020-01-01 00:13:52.000 2020-01-02 03:42:13.000 832 99733 50282.5 5028250 832 99733 50282.5 5028250 -32736 32199 4407.3 440730 -125 126 -1.02 -102 +833 2 10823 99734 2.5015 299.5015 151.0015 15100.15015 2.5015 299.5015 151.00149 15100.14983 2.50150 299.50150 151.00150000000014 15100.15000 2020-01-01 2020-01-02 2020-01-01 00:13:53 2020-01-02 03:42:14 2020-01-01 00:13:53.000 2020-01-02 03:42:14.000 833 99734 50283.5 5028350 833 99734 50283.5 5028350 -32735 32200 4408.3 440830 -124 127 -0.02 -2 +834 2 10824 99735 2.5045 299.5045 151.0045 15100.45045 2.5045 299.50452 151.0045 15100.45043 2.50450 299.50450 151.00449999999984 15100.45000 2020-01-01 2020-01-02 2020-01-01 00:13:54 2020-01-02 03:42:15 2020-01-01 00:13:54.000 2020-01-02 03:42:15.000 834 99735 50284.5 5028450 834 99735 50284.5 5028450 -32734 32201 4409.3 440930 -128 127 -1.58 -158 +835 2 10825 99736 2.5075 299.5075 151.0075 15100.75075 2.5075 299.5075 151.0075 15100.75073 2.50750 299.50750 151.00749999999988 15100.75000 2020-01-01 2020-01-02 2020-01-01 00:13:55 2020-01-02 03:42:16 2020-01-01 00:13:55.000 2020-01-02 03:42:16.000 835 99736 50285.5 5028550 835 99736 50285.5 5028550 -32733 32202 4410.3 441030 -128 123 -3.14 -314 +836 2 10826 99737 2.51051 299.51051 151.01051 15101.05105 2.51051 299.5105 151.01051 15101.05103 2.51051 299.51051 151.01051000000004 15101.05100 2020-01-01 2020-01-02 2020-01-01 00:13:56 2020-01-02 03:42:17 2020-01-01 00:13:56.000 2020-01-02 03:42:17.000 836 99737 50286.5 5028650 836 99737 50286.5 5028650 -32732 32203 4411.3 441130 -127 124 -2.14 -214 +837 2 10827 99738 2.51351 299.51351 151.01351 15101.35135 2.51351 299.51352 151.01351 15101.35162 2.51351 299.51351 151.01351000000005 15101.35100 2020-01-01 2020-01-02 2020-01-01 00:13:57 2020-01-02 03:42:18 2020-01-01 00:13:57.000 2020-01-02 03:42:18.000 837 99738 50287.5 5028750 837 99738 50287.5 5028750 -32731 32204 4412.3 441230 -126 125 -1.14 -114 +838 2 10828 99739 2.51651 299.51651 151.01651 15101.65165 2.51651 299.5165 151.01651 15101.6513 2.51651 299.51651 151.01650999999995 15101.65100 2020-01-01 2020-01-02 2020-01-01 00:13:58 2020-01-02 03:42:19 2020-01-01 00:13:58.000 2020-01-02 03:42:19.000 838 99739 50288.5 5028850 838 99739 50288.5 5028850 -32730 32205 4413.3 441330 -125 126 -0.14 -14 +839 2 10829 99740 2.51951 299.51951 151.01951 15101.95195 2.51951 299.51953 151.01951 15101.9519 2.51951 299.51951 151.01951000000005 15101.95100 2020-01-01 2020-01-02 2020-01-01 00:13:59 2020-01-02 03:42:20 2020-01-01 00:13:59.000 2020-01-02 03:42:20.000 839 99740 50289.5 5028950 839 99740 50289.5 5028950 -32729 32206 4414.3 441430 -124 127 0.86 86 +84 2 10074 99984 0.25225 300.25225 150.25225 15175.47747 0.25225 300.25226 150.25225 15175.47778 0.25225 300.25225 150.25224999999975 15175.47725 2020-01-01 2020-01-02 2020-01-01 00:01:24 2020-01-02 03:46:24 2020-01-01 00:01:24.000 2020-01-02 03:46:24.000 84 99984 50034 5053434 84 99984 50034 5053434 -32485 32450 4613.009900990099 465914 -124 127 -0.0594059405940594 -6 +840 2 10830 99741 2.52252 299.52252 151.02252 15102.25225 2.52252 299.52252 151.02252 15102.2522 2.52252 299.52252 151.0225200000002 15102.25200 2020-01-01 2020-01-02 2020-01-01 00:14:00 2020-01-02 03:42:21 2020-01-01 00:14:00.000 2020-01-02 03:42:21.000 840 99741 50290.5 5029050 840 99741 50290.5 5029050 -32728 32207 4415.3 441530 -128 127 -0.7 -70 +841 2 10831 99742 2.52552 299.52552 151.02552 15102.55255 2.52552 299.5255 151.02552 15102.5525 2.52552 299.52552 151.02551999999991 15102.55200 2020-01-01 2020-01-02 2020-01-01 00:14:01 2020-01-02 03:42:22 2020-01-01 00:14:01.000 2020-01-02 03:42:22.000 841 99742 50291.5 5029150 841 99742 50291.5 5029150 -32727 32208 4416.3 441630 -128 127 -2.26 -226 +842 2 10832 99743 2.52852 299.52852 151.02852 15102.85285 2.52852 299.52853 151.02853 15102.85313 2.52852 299.52852 151.0285199999999 15102.85200 2020-01-01 2020-01-02 2020-01-01 00:14:02 2020-01-02 03:42:23 2020-01-01 00:14:02.000 2020-01-02 03:42:23.000 842 99743 50292.5 5029250 842 99743 50292.5 5029250 -32726 32209 4417.3 441730 -128 123 -3.82 -382 +843 2 10833 99744 2.53153 299.53153 151.03153 15103.15315 2.53153 299.53152 151.03152 15103.15278 2.53153 299.53153 151.03153000000015 15103.15300 2020-01-01 2020-01-02 2020-01-01 00:14:03 2020-01-02 03:42:24 2020-01-01 00:14:03.000 2020-01-02 03:42:24.000 843 99744 50293.5 5029350 843 99744 50293.5 5029350 -32725 32210 4418.3 441830 -127 124 -2.82 -282 +844 2 10834 99745 2.53453 299.53453 151.03453 15103.45345 2.53453 299.53455 151.03453 15103.45353 2.53453 299.53453 151.03453000000013 15103.45300 2020-01-01 2020-01-02 2020-01-01 00:14:04 2020-01-02 03:42:25 2020-01-01 00:14:04.000 2020-01-02 03:42:25.000 844 99745 50294.5 5029450 844 99745 50294.5 5029450 -32724 32211 4419.3 441930 -126 125 -1.82 -182 +845 2 10835 99746 2.53753 299.53753 151.03753 15103.75375 2.53753 299.53754 151.03753 15103.75366 2.53753 299.53753 151.0375299999998 15103.75300 2020-01-01 2020-01-02 2020-01-01 00:14:05 2020-01-02 03:42:26 2020-01-01 00:14:05.000 2020-01-02 03:42:26.000 845 99746 50295.5 5029550 845 99746 50295.5 5029550 -32723 32212 4420.3 442030 -125 126 -0.82 -82 846 2 10836 99747 2.54054 299.54054 151.04054 15104.05405 2.54054 299.54053 151.04053 15104.05397 2.54054 299.54054 151.04054 15104.05400 2020-01-01 2020-01-02 2020-01-01 00:14:06 2020-01-02 03:42:27 2020-01-01 00:14:06.000 2020-01-02 03:42:27.000 846 99747 50296.5 5029650 846 99747 50296.5 5029650 -32722 32213 4421.3 442130 -124 127 0.18 18 -847 2 10837 99748 2.54354 299.54354 151.04354 15104.35435 2.54354 299.54355 151.04354 15104.3546 2.54354 299.54354 151.04354 15104.35400 2020-01-01 2020-01-02 2020-01-01 00:14:07 2020-01-02 03:42:28 2020-01-01 00:14:07.000 2020-01-02 03:42:28.000 847 99748 50297.5 5029750 847 99748 50297.5 5029750 -32721 32214 4422.3 442230 -128 127 -1.38 -138 -848 2 10838 99749 2.54654 299.54654 151.04654 15104.65465 2.54654 299.54654 151.04654 15104.65425 2.54654 299.54654 151.04654 15104.65400 2020-01-01 2020-01-02 2020-01-01 00:14:08 2020-01-02 03:42:29 2020-01-01 00:14:08.000 2020-01-02 03:42:29.000 848 99749 50298.5 5029850 848 99749 50298.5 5029850 -32720 32215 4423.3 442330 -128 123 -2.94 -294 -849 2 10839 99750 2.54954 299.54954 151.04954 15104.95495 2.54954 299.54956 151.04954 15104.95499 2.54954 299.54954 151.04954 15104.95400 2020-01-01 2020-01-02 2020-01-01 00:14:09 2020-01-02 03:42:30 2020-01-01 00:14:09.000 2020-01-02 03:42:30.000 849 99750 50299.5 5029950 849 99750 50299.5 5029950 -32719 32216 4424.3 442430 -127 124 -1.94 -194 -85 2 10075 99985 0.25525 300.25525 150.25525 15175.78078 0.25525 300.25525 150.25525 15175.78046 0.25525 300.25525 150.25525 15175.78025 2020-01-01 2020-01-02 2020-01-01 00:01:25 2020-01-02 03:46:25 2020-01-01 00:01:25.000 2020-01-02 03:46:25.000 85 99985 50035 5053535 85 99985 50035 5053535 -32484 32451 4614.009900990099 466015 -128 127 -1.5940594059405941 -161 -850 2 10840 99751 2.55255 299.55255 151.05255 15105.25525 2.55255 299.55255 151.05255 15105.25514 2.55255 299.55255 151.05255 15105.25500 2020-01-01 2020-01-02 2020-01-01 00:14:10 2020-01-02 03:42:31 2020-01-01 00:14:10.000 2020-01-02 03:42:31.000 850 99751 50300.5 5030050 850 99751 50300.5 5030050 -32718 32217 4425.3 442530 -126 125 -0.94 -94 -851 2 10841 99752 2.55555 299.55555 151.05555 15105.55555 2.55555 299.55554 151.05555 15105.55547 2.55555 299.55555 151.05555 15105.55500 2020-01-01 2020-01-02 2020-01-01 00:14:11 2020-01-02 03:42:32 2020-01-01 00:14:11.000 2020-01-02 03:42:32.000 851 99752 50301.5 5030150 851 99752 50301.5 5030150 -32717 32218 4426.3 442630 -125 126 0.06 6 -852 2 10842 99753 2.55855 299.55855 151.05855 15105.85585 2.55855 299.55856 151.05856 15105.85607 2.55855 299.55855 151.05855 15105.85500 2020-01-01 2020-01-02 2020-01-01 00:14:12 2020-01-02 03:42:33 2020-01-01 00:14:12.000 2020-01-02 03:42:33.000 852 99753 50302.5 5030250 852 99753 50302.5 5030250 -32716 32219 4427.3 442730 -124 127 1.06 106 -853 2 10843 99754 2.56156 299.56156 151.06156 15106.15615 2.56156 299.56155 151.06155 15106.15571 2.56156 299.56156 151.06156 15106.15600 2020-01-01 2020-01-02 2020-01-01 00:14:13 2020-01-02 03:42:34 2020-01-01 00:14:13.000 2020-01-02 03:42:34.000 853 99754 50303.5 5030350 853 99754 50303.5 5030350 -32715 32220 4428.3 442830 -128 127 -0.5 -50 -854 2 10844 99755 2.56456 299.56456 151.06456 15106.45645 2.56456 299.56458 151.06456 15106.45646 2.56456 299.56456 151.06456 15106.45600 2020-01-01 2020-01-02 2020-01-01 00:14:14 2020-01-02 03:42:35 2020-01-01 00:14:14.000 2020-01-02 03:42:35.000 854 99755 50304.5 5030450 854 99755 50304.5 5030450 -32714 32221 4429.3 442930 -128 123 -2.06 -206 -855 2 10845 99756 2.56756 299.56756 151.06756 15106.75675 2.56756 299.56757 151.06756 15106.75661 2.56756 299.56756 151.06756 15106.75600 2020-01-01 2020-01-02 2020-01-01 00:14:15 2020-01-02 03:42:36 2020-01-01 00:14:15.000 2020-01-02 03:42:36.000 855 99756 50305.5 5030550 855 99756 50305.5 5030550 -32713 32222 4430.3 443030 -127 124 -1.06 -106 -856 2 10846 99757 2.57057 299.57057 151.07057 15107.05705 2.57057 299.57056 151.07056 15107.05694 2.57057 299.57057 151.07057 15107.05700 2020-01-01 2020-01-02 2020-01-01 00:14:16 2020-01-02 03:42:37 2020-01-01 00:14:16.000 2020-01-02 03:42:37.000 856 99757 50306.5 5030650 856 99757 50306.5 5030650 -32712 32223 4431.3 443130 -126 125 -0.06 -6 +847 2 10837 99748 2.54354 299.54354 151.04354 15104.35435 2.54354 299.54355 151.04354 15104.3546 2.54354 299.54354 151.04354000000006 15104.35400 2020-01-01 2020-01-02 2020-01-01 00:14:07 2020-01-02 03:42:28 2020-01-01 00:14:07.000 2020-01-02 03:42:28.000 847 99748 50297.5 5029750 847 99748 50297.5 5029750 -32721 32214 4422.3 442230 -128 127 -1.38 -138 +848 2 10838 99749 2.54654 299.54654 151.04654 15104.65465 2.54654 299.54654 151.04654 15104.65425 2.54654 299.54654 151.0465399999999 15104.65400 2020-01-01 2020-01-02 2020-01-01 00:14:08 2020-01-02 03:42:29 2020-01-01 00:14:08.000 2020-01-02 03:42:29.000 848 99749 50298.5 5029850 848 99749 50298.5 5029850 -32720 32215 4423.3 442330 -128 123 -2.94 -294 +849 2 10839 99750 2.54954 299.54954 151.04954 15104.95495 2.54954 299.54956 151.04954 15104.95499 2.54954 299.54954 151.04953999999998 15104.95400 2020-01-01 2020-01-02 2020-01-01 00:14:09 2020-01-02 03:42:30 2020-01-01 00:14:09.000 2020-01-02 03:42:30.000 849 99750 50299.5 5029950 849 99750 50299.5 5029950 -32719 32216 4424.3 442430 -127 124 -1.94 -194 +85 2 10075 99985 0.25525 300.25525 150.25525 15175.78078 0.25525 300.25525 150.25525 15175.78046 0.25525 300.25525 150.2552500000001 15175.78025 2020-01-01 2020-01-02 2020-01-01 00:01:25 2020-01-02 03:46:25 2020-01-01 00:01:25.000 2020-01-02 03:46:25.000 85 99985 50035 5053535 85 99985 50035 5053535 -32484 32451 4614.009900990099 466015 -128 127 -1.5940594059405941 -161 +850 2 10840 99751 2.55255 299.55255 151.05255 15105.25525 2.55255 299.55255 151.05255 15105.25514 2.55255 299.55255 151.05255000000014 15105.25500 2020-01-01 2020-01-02 2020-01-01 00:14:10 2020-01-02 03:42:31 2020-01-01 00:14:10.000 2020-01-02 03:42:31.000 850 99751 50300.5 5030050 850 99751 50300.5 5030050 -32718 32217 4425.3 442530 -126 125 -0.94 -94 +851 2 10841 99752 2.55555 299.55555 151.05555 15105.55555 2.55555 299.55554 151.05555 15105.55547 2.55555 299.55555 151.0555499999998 15105.55500 2020-01-01 2020-01-02 2020-01-01 00:14:11 2020-01-02 03:42:32 2020-01-01 00:14:11.000 2020-01-02 03:42:32.000 851 99752 50301.5 5030150 851 99752 50301.5 5030150 -32717 32218 4426.3 442630 -125 126 0.06 6 +852 2 10842 99753 2.55855 299.55855 151.05855 15105.85585 2.55855 299.55856 151.05856 15105.85607 2.55855 299.55855 151.0585499999999 15105.85500 2020-01-01 2020-01-02 2020-01-01 00:14:12 2020-01-02 03:42:33 2020-01-01 00:14:12.000 2020-01-02 03:42:33.000 852 99753 50302.5 5030250 852 99753 50302.5 5030250 -32716 32219 4427.3 442730 -124 127 1.06 106 +853 2 10843 99754 2.56156 299.56156 151.06156 15106.15615 2.56156 299.56155 151.06155 15106.15571 2.56156 299.56156 151.06156000000007 15106.15600 2020-01-01 2020-01-02 2020-01-01 00:14:13 2020-01-02 03:42:34 2020-01-01 00:14:13.000 2020-01-02 03:42:34.000 853 99754 50303.5 5030350 853 99754 50303.5 5030350 -32715 32220 4428.3 442830 -128 127 -0.5 -50 +854 2 10844 99755 2.56456 299.56456 151.06456 15106.45645 2.56456 299.56458 151.06456 15106.45646 2.56456 299.56456 151.06456000000009 15106.45600 2020-01-01 2020-01-02 2020-01-01 00:14:14 2020-01-02 03:42:35 2020-01-01 00:14:14.000 2020-01-02 03:42:35.000 854 99755 50304.5 5030450 854 99755 50304.5 5030450 -32714 32221 4429.3 442930 -128 123 -2.06 -206 +855 2 10845 99756 2.56756 299.56756 151.06756 15106.75675 2.56756 299.56757 151.06756 15106.75661 2.56756 299.56756 151.06756000000001 15106.75600 2020-01-01 2020-01-02 2020-01-01 00:14:15 2020-01-02 03:42:36 2020-01-01 00:14:15.000 2020-01-02 03:42:36.000 855 99756 50305.5 5030550 855 99756 50305.5 5030550 -32713 32222 4430.3 443030 -127 124 -1.06 -106 +856 2 10846 99757 2.57057 299.57057 151.07057 15107.05705 2.57057 299.57056 151.07056 15107.05694 2.57057 299.57057 151.0705699999999 15107.05700 2020-01-01 2020-01-02 2020-01-01 00:14:16 2020-01-02 03:42:37 2020-01-01 00:14:16.000 2020-01-02 03:42:37.000 856 99757 50306.5 5030650 856 99757 50306.5 5030650 -32712 32223 4431.3 443130 -126 125 -0.06 -6 857 2 10847 99758 2.57357 299.57357 151.07357 15107.35735 2.57357 299.57358 151.07357 15107.35754 2.57357 299.57357 151.07357 15107.35700 2020-01-01 2020-01-02 2020-01-01 00:14:17 2020-01-02 03:42:38 2020-01-01 00:14:17.000 2020-01-02 03:42:38.000 857 99758 50307.5 5030750 857 99758 50307.5 5030750 -32711 32224 4432.3 443230 -125 126 0.94 94 -858 2 10848 99759 2.57657 299.57657 151.07657 15107.65765 2.57657 299.57657 151.07657 15107.65783 2.57657 299.57657 151.07657 15107.65700 2020-01-01 2020-01-02 2020-01-01 00:14:18 2020-01-02 03:42:39 2020-01-01 00:14:18.000 2020-01-02 03:42:39.000 858 99759 50308.5 5030850 858 99759 50308.5 5030850 -32710 32225 4433.3 443330 -124 127 1.94 194 -859 2 10849 99760 2.57957 299.57957 151.07957 15107.95795 2.57957 299.5796 151.07957 15107.95794 2.57957 299.57957 151.07957 15107.95700 2020-01-01 2020-01-02 2020-01-01 00:14:19 2020-01-02 03:42:40 2020-01-01 00:14:19.000 2020-01-02 03:42:40.000 859 99760 50309.5 5030950 859 99760 50309.5 5030950 -32709 32226 4434.3 443430 -128 127 0.38 38 -86 2 10076 99986 0.25825 300.25825 150.25825 15176.08408 0.25825 300.25827 150.25825 15176.08406 0.25825 300.25825 150.25825 15176.08325 2020-01-01 2020-01-02 2020-01-01 00:01:26 2020-01-02 03:46:26 2020-01-01 00:01:26.000 2020-01-02 03:46:26.000 86 99986 50036 5053636 86 99986 50036 5053636 -32483 32452 4615.009900990099 466116 -128 123 -3.128712871287129 -316 -860 2 10850 99761 2.58258 299.58258 151.08258 15108.25825 2.58258 299.58258 151.08258 15108.25811 2.58258 299.58258 151.08258 15108.25800 2020-01-01 2020-01-02 2020-01-01 00:14:20 2020-01-02 03:42:41 2020-01-01 00:14:20.000 2020-01-02 03:42:41.000 860 99761 50310.5 5031050 860 99761 50310.5 5031050 -32708 32227 4435.3 443530 -128 123 -1.18 -118 -861 2 10851 99762 2.58558 299.58558 151.08558 15108.55855 2.58558 299.58557 151.08558 15108.55841 2.58558 299.58558 151.08558 15108.55800 2020-01-01 2020-01-02 2020-01-01 00:14:21 2020-01-02 03:42:42 2020-01-01 00:14:21.000 2020-01-02 03:42:42.000 861 99762 50311.5 5031150 861 99762 50311.5 5031150 -32707 32228 4436.3 443630 -127 124 -0.18 -18 -862 2 10852 99763 2.58858 299.58858 151.08858 15108.85885 2.58858 299.5886 151.08859 15108.85901 2.58858 299.58858 151.08858 15108.85800 2020-01-01 2020-01-02 2020-01-01 00:14:22 2020-01-02 03:42:43 2020-01-01 00:14:22.000 2020-01-02 03:42:43.000 862 99763 50312.5 5031250 862 99763 50312.5 5031250 -32706 32229 4437.3 443730 -126 125 0.82 82 +858 2 10848 99759 2.57657 299.57657 151.07657 15107.65765 2.57657 299.57657 151.07657 15107.65783 2.57657 299.57657 151.07656999999995 15107.65700 2020-01-01 2020-01-02 2020-01-01 00:14:18 2020-01-02 03:42:39 2020-01-01 00:14:18.000 2020-01-02 03:42:39.000 858 99759 50308.5 5030850 858 99759 50308.5 5030850 -32710 32225 4433.3 443330 -124 127 1.94 194 +859 2 10849 99760 2.57957 299.57957 151.07957 15107.95795 2.57957 299.5796 151.07957 15107.95794 2.57957 299.57957 151.0795699999999 15107.95700 2020-01-01 2020-01-02 2020-01-01 00:14:19 2020-01-02 03:42:40 2020-01-01 00:14:19.000 2020-01-02 03:42:40.000 859 99760 50309.5 5030950 859 99760 50309.5 5030950 -32709 32226 4434.3 443430 -128 127 0.38 38 +86 2 10076 99986 0.25825 300.25825 150.25825 15176.08408 0.25825 300.25827 150.25825 15176.08406 0.25825 300.25825 150.25825000000012 15176.08325 2020-01-01 2020-01-02 2020-01-01 00:01:26 2020-01-02 03:46:26 2020-01-01 00:01:26.000 2020-01-02 03:46:26.000 86 99986 50036 5053636 86 99986 50036 5053636 -32483 32452 4615.009900990099 466116 -128 123 -3.128712871287129 -316 +860 2 10850 99761 2.58258 299.58258 151.08258 15108.25825 2.58258 299.58258 151.08258 15108.25811 2.58258 299.58258 151.08258000000012 15108.25800 2020-01-01 2020-01-02 2020-01-01 00:14:20 2020-01-02 03:42:41 2020-01-01 00:14:20.000 2020-01-02 03:42:41.000 860 99761 50310.5 5031050 860 99761 50310.5 5031050 -32708 32227 4435.3 443530 -128 123 -1.18 -118 +861 2 10851 99762 2.58558 299.58558 151.08558 15108.55855 2.58558 299.58557 151.08558 15108.55841 2.58558 299.58558 151.08558000000016 15108.55800 2020-01-01 2020-01-02 2020-01-01 00:14:21 2020-01-02 03:42:42 2020-01-01 00:14:21.000 2020-01-02 03:42:42.000 861 99762 50311.5 5031150 861 99762 50311.5 5031150 -32707 32228 4436.3 443630 -127 124 -0.18 -18 +862 2 10852 99763 2.58858 299.58858 151.08858 15108.85885 2.58858 299.5886 151.08859 15108.85901 2.58858 299.58858 151.08857999999984 15108.85800 2020-01-01 2020-01-02 2020-01-01 00:14:22 2020-01-02 03:42:43 2020-01-01 00:14:22.000 2020-01-02 03:42:43.000 862 99763 50312.5 5031250 862 99763 50312.5 5031250 -32706 32229 4437.3 443730 -126 125 0.82 82 863 2 10853 99764 2.59159 299.59159 151.09159 15109.15915 2.59159 299.59158 151.09159 15109.1593 2.59159 299.59159 151.09159 15109.15900 2020-01-01 2020-01-02 2020-01-01 00:14:23 2020-01-02 03:42:44 2020-01-01 00:14:23.000 2020-01-02 03:42:44.000 863 99764 50313.5 5031350 863 99764 50313.5 5031350 -32705 32230 4438.3 443830 -125 126 1.82 182 -864 2 10854 99765 2.59459 299.59459 151.09459 15109.45945 2.59459 299.5946 151.09459 15109.45941 2.59459 299.59459 151.09459 15109.45900 2020-01-01 2020-01-02 2020-01-01 00:14:24 2020-01-02 03:42:45 2020-01-01 00:14:24.000 2020-01-02 03:42:45.000 864 99765 50314.5 5031450 864 99765 50314.5 5031450 -32704 32231 4439.3 443930 -124 127 2.82 282 -865 2 10855 99766 2.59759 299.59759 151.09759 15109.75975 2.59759 299.5976 151.09759 15109.75958 2.59759 299.59759 151.09759 15109.75900 2020-01-01 2020-01-02 2020-01-01 00:14:25 2020-01-02 03:42:46 2020-01-01 00:14:25.000 2020-01-02 03:42:46.000 865 99766 50315.5 5031550 865 99766 50315.5 5031550 -32703 32232 4440.3 444030 -128 127 1.26 126 -866 2 10856 99767 2.6006 299.6006 151.1006 15110.06006 2.6006 299.6006 151.10059 15110.05988 2.60060 299.60060 151.10060 15110.06000 2020-01-01 2020-01-02 2020-01-01 00:14:26 2020-01-02 03:42:47 2020-01-01 00:14:26.000 2020-01-02 03:42:47.000 866 99767 50316.5 5031650 866 99767 50316.5 5031650 -32702 32233 4441.3 444130 -128 127 -0.3 -30 -867 2 10857 99768 2.6036 299.6036 151.1036 15110.36036 2.6036 299.6036 151.1036 15110.36063 2.60360 299.60360 151.10360 15110.36000 2020-01-01 2020-01-02 2020-01-01 00:14:27 2020-01-02 03:42:48 2020-01-01 00:14:27.000 2020-01-02 03:42:48.000 867 99768 50317.5 5031750 867 99768 50317.5 5031750 -32701 32234 4442.3 444230 -128 123 -1.86 -186 -868 2 10858 99769 2.6066 299.6066 151.1066 15110.66066 2.6066 299.6066 151.1066 15110.66078 2.60660 299.60660 151.10660 15110.66000 2020-01-01 2020-01-02 2020-01-01 00:14:28 2020-01-02 03:42:49 2020-01-01 00:14:28.000 2020-01-02 03:42:49.000 868 99769 50318.5 5031850 868 99769 50318.5 5031850 -32700 32235 4443.3 444330 -127 124 -0.86 -86 -869 2 10859 99770 2.6096 299.6096 151.1096 15110.96096 2.6096 299.60962 151.1096 15110.96091 2.60960 299.60960 151.10960 15110.96000 2020-01-01 2020-01-02 2020-01-01 00:14:29 2020-01-02 03:42:50 2020-01-01 00:14:29.000 2020-01-02 03:42:50.000 869 99770 50319.5 5031950 869 99770 50319.5 5031950 -32699 32236 4444.3 444430 -126 125 0.14 14 -87 2 10077 99987 0.26126 300.26126 150.26126 15176.38738 0.26126 300.26126 150.26126 15176.38736 0.26126 300.26126 150.26126 15176.38726 2020-01-01 2020-01-02 2020-01-01 00:01:27 2020-01-02 03:46:27 2020-01-01 00:01:27.000 2020-01-02 03:46:27.000 87 99987 50037 5053737 87 99987 50037 5053737 -32482 32453 4616.009900990099 466217 -127 124 -2.128712871287129 -215 -870 2 10860 99771 2.61261 299.61261 151.11261 15111.26126 2.61261 299.6126 151.11261 15111.26105 2.61261 299.61261 151.11261 15111.26100 2020-01-01 2020-01-02 2020-01-01 00:14:30 2020-01-02 03:42:51 2020-01-01 00:14:30.000 2020-01-02 03:42:51.000 870 99771 50320.5 5032050 870 99771 50320.5 5032050 -32698 32237 4445.3 444530 -125 126 1.14 114 -871 2 10861 99772 2.61561 299.61561 151.11561 15111.56156 2.61561 299.6156 151.11561 15111.56135 2.61561 299.61561 151.11561 15111.56100 2020-01-01 2020-01-02 2020-01-01 00:14:31 2020-01-02 03:42:52 2020-01-01 00:14:31.000 2020-01-02 03:42:52.000 871 99772 50321.5 5032150 871 99772 50321.5 5032150 -32697 32238 4446.3 444630 -124 127 2.14 214 -872 2 10862 99773 2.61861 299.61861 151.11861 15111.86186 2.61861 299.61862 151.11862 15111.8621 2.61861 299.61861 151.11861 15111.86100 2020-01-01 2020-01-02 2020-01-01 00:14:32 2020-01-02 03:42:53 2020-01-01 00:14:32.000 2020-01-02 03:42:53.000 872 99773 50322.5 5032250 872 99773 50322.5 5032250 -32696 32239 4447.3 444730 -128 127 0.58 58 -873 2 10863 99774 2.62162 299.62162 151.12162 15112.16216 2.62162 299.6216 151.12162 15112.16224 2.62162 299.62162 151.12162 15112.16200 2020-01-01 2020-01-02 2020-01-01 00:14:33 2020-01-02 03:42:54 2020-01-01 00:14:33.000 2020-01-02 03:42:54.000 873 99774 50323.5 5032350 873 99774 50323.5 5032350 -32695 32240 4448.3 444830 -128 123 -0.98 -98 +864 2 10854 99765 2.59459 299.59459 151.09459 15109.45945 2.59459 299.5946 151.09459 15109.45941 2.59459 299.59459 151.09459000000007 15109.45900 2020-01-01 2020-01-02 2020-01-01 00:14:24 2020-01-02 03:42:45 2020-01-01 00:14:24.000 2020-01-02 03:42:45.000 864 99765 50314.5 5031450 864 99765 50314.5 5031450 -32704 32231 4439.3 443930 -124 127 2.82 282 +865 2 10855 99766 2.59759 299.59759 151.09759 15109.75975 2.59759 299.5976 151.09759 15109.75958 2.59759 299.59759 151.0975899999999 15109.75900 2020-01-01 2020-01-02 2020-01-01 00:14:25 2020-01-02 03:42:46 2020-01-01 00:14:25.000 2020-01-02 03:42:46.000 865 99766 50315.5 5031550 865 99766 50315.5 5031550 -32703 32232 4440.3 444030 -128 127 1.26 126 +866 2 10856 99767 2.6006 299.6006 151.1006 15110.06006 2.6006 299.6006 151.10059 15110.05988 2.60060 299.60060 151.10059999999984 15110.06000 2020-01-01 2020-01-02 2020-01-01 00:14:26 2020-01-02 03:42:47 2020-01-01 00:14:26.000 2020-01-02 03:42:47.000 866 99767 50316.5 5031650 866 99767 50316.5 5031650 -32702 32233 4441.3 444130 -128 127 -0.3 -30 +867 2 10857 99768 2.6036 299.6036 151.1036 15110.36036 2.6036 299.6036 151.1036 15110.36063 2.60360 299.60360 151.10360000000017 15110.36000 2020-01-01 2020-01-02 2020-01-01 00:14:27 2020-01-02 03:42:48 2020-01-01 00:14:27.000 2020-01-02 03:42:48.000 867 99768 50317.5 5031750 867 99768 50317.5 5031750 -32701 32234 4442.3 444230 -128 123 -1.86 -186 +868 2 10858 99769 2.6066 299.6066 151.1066 15110.66066 2.6066 299.6066 151.1066 15110.66078 2.60660 299.60660 151.10659999999984 15110.66000 2020-01-01 2020-01-02 2020-01-01 00:14:28 2020-01-02 03:42:49 2020-01-01 00:14:28.000 2020-01-02 03:42:49.000 868 99769 50318.5 5031850 868 99769 50318.5 5031850 -32700 32235 4443.3 444330 -127 124 -0.86 -86 +869 2 10859 99770 2.6096 299.6096 151.1096 15110.96096 2.6096 299.60962 151.1096 15110.96091 2.60960 299.60960 151.10959999999983 15110.96000 2020-01-01 2020-01-02 2020-01-01 00:14:29 2020-01-02 03:42:50 2020-01-01 00:14:29.000 2020-01-02 03:42:50.000 869 99770 50319.5 5031950 869 99770 50319.5 5031950 -32699 32236 4444.3 444430 -126 125 0.14 14 +87 2 10077 99987 0.26126 300.26126 150.26126 15176.38738 0.26126 300.26126 150.26126 15176.38736 0.26126 300.26126 150.26125999999994 15176.38726 2020-01-01 2020-01-02 2020-01-01 00:01:27 2020-01-02 03:46:27 2020-01-01 00:01:27.000 2020-01-02 03:46:27.000 87 99987 50037 5053737 87 99987 50037 5053737 -32482 32453 4616.009900990099 466217 -127 124 -2.128712871287129 -215 +870 2 10860 99771 2.61261 299.61261 151.11261 15111.26126 2.61261 299.6126 151.11261 15111.26105 2.61261 299.61261 151.1126100000001 15111.26100 2020-01-01 2020-01-02 2020-01-01 00:14:30 2020-01-02 03:42:51 2020-01-01 00:14:30.000 2020-01-02 03:42:51.000 870 99771 50320.5 5032050 870 99771 50320.5 5032050 -32698 32237 4445.3 444530 -125 126 1.14 114 +871 2 10861 99772 2.61561 299.61561 151.11561 15111.56156 2.61561 299.6156 151.11561 15111.56135 2.61561 299.61561 151.11561000000012 15111.56100 2020-01-01 2020-01-02 2020-01-01 00:14:31 2020-01-02 03:42:52 2020-01-01 00:14:31.000 2020-01-02 03:42:52.000 871 99772 50321.5 5032150 871 99772 50321.5 5032150 -32697 32238 4446.3 444630 -124 127 2.14 214 +872 2 10862 99773 2.61861 299.61861 151.11861 15111.86186 2.61861 299.61862 151.11862 15111.8621 2.61861 299.61861 151.1186099999999 15111.86100 2020-01-01 2020-01-02 2020-01-01 00:14:32 2020-01-02 03:42:53 2020-01-01 00:14:32.000 2020-01-02 03:42:53.000 872 99773 50322.5 5032250 872 99773 50322.5 5032250 -32696 32239 4447.3 444730 -128 127 0.58 58 +873 2 10863 99774 2.62162 299.62162 151.12162 15112.16216 2.62162 299.6216 151.12162 15112.16224 2.62162 299.62162 151.12161999999992 15112.16200 2020-01-01 2020-01-02 2020-01-01 00:14:33 2020-01-02 03:42:54 2020-01-01 00:14:33.000 2020-01-02 03:42:54.000 873 99774 50323.5 5032350 873 99774 50323.5 5032350 -32695 32240 4448.3 444830 -128 123 -0.98 -98 874 2 10864 99775 2.62462 299.62462 151.12462 15112.46246 2.62462 299.62463 151.12462 15112.46238 2.62462 299.62462 151.12462 15112.46200 2020-01-01 2020-01-02 2020-01-01 00:14:34 2020-01-02 03:42:55 2020-01-01 00:14:34.000 2020-01-02 03:42:55.000 874 99775 50324.5 5032450 874 99775 50324.5 5032450 -32694 32241 4449.3 444930 -127 124 0.02 2 -875 2 10865 99776 2.62762 299.62762 151.12762 15112.76276 2.62762 299.62762 151.12762 15112.76252 2.62762 299.62762 151.12762 15112.76200 2020-01-01 2020-01-02 2020-01-01 00:14:35 2020-01-02 03:42:56 2020-01-01 00:14:35.000 2020-01-02 03:42:56.000 875 99776 50325.5 5032550 875 99776 50325.5 5032550 -32693 32242 4450.3 445030 -126 125 1.02 102 -876 2 10866 99777 2.63063 299.63063 151.13063 15113.06306 2.63063 299.63065 151.13063 15113.06327 2.63063 299.63063 151.13063 15113.06300 2020-01-01 2020-01-02 2020-01-01 00:14:36 2020-01-02 03:42:57 2020-01-01 00:14:36.000 2020-01-02 03:42:57.000 876 99777 50326.5 5032650 876 99777 50326.5 5032650 -32692 32243 4451.3 445130 -125 126 2.02 202 -877 2 10867 99778 2.63363 299.63363 151.13363 15113.36336 2.63363 299.63364 151.13363 15113.36358 2.63363 299.63363 151.13363 15113.36300 2020-01-01 2020-01-02 2020-01-01 00:14:37 2020-01-02 03:42:58 2020-01-01 00:14:37.000 2020-01-02 03:42:58.000 877 99778 50327.5 5032750 877 99778 50327.5 5032750 -32691 32244 4452.3 445230 -124 127 3.02 302 -878 2 10868 99779 2.63663 299.63663 151.13663 15113.66366 2.63663 299.63663 151.13663 15113.66371 2.63663 299.63663 151.13663 15113.66300 2020-01-01 2020-01-02 2020-01-01 00:14:38 2020-01-02 03:42:59 2020-01-01 00:14:38.000 2020-01-02 03:42:59.000 878 99779 50328.5 5032850 878 99779 50328.5 5032850 -32690 32245 4453.3 445330 -128 127 1.46 146 -879 2 10869 99780 2.63963 299.63963 151.13963 15113.96396 2.63963 299.63965 151.13963 15113.96385 2.63963 299.63963 151.13963 15113.96300 2020-01-01 2020-01-02 2020-01-01 00:14:39 2020-01-02 03:43:00 2020-01-01 00:14:39.000 2020-01-02 03:43:00.000 879 99780 50329.5 5032950 879 99780 50329.5 5032950 -32689 32246 4454.3 445430 -128 123 -0.1 -10 -88 2 10078 99988 0.26426 300.26426 150.26426 15176.69069 0.26426 300.26425 150.26426 15176.69066 0.26426 300.26426 150.26426 15176.69026 2020-01-01 2020-01-02 2020-01-01 00:01:28 2020-01-02 03:46:28 2020-01-01 00:01:28.000 2020-01-02 03:46:28.000 88 99988 50038 5053838 88 99988 50038 5053838 -32481 32454 4617.009900990099 466318 -126 125 -1.1287128712871286 -114 +875 2 10865 99776 2.62762 299.62762 151.12762 15112.76276 2.62762 299.62762 151.12762 15112.76252 2.62762 299.62762 151.12761999999992 15112.76200 2020-01-01 2020-01-02 2020-01-01 00:14:35 2020-01-02 03:42:56 2020-01-01 00:14:35.000 2020-01-02 03:42:56.000 875 99776 50325.5 5032550 875 99776 50325.5 5032550 -32693 32242 4450.3 445030 -126 125 1.02 102 +876 2 10866 99777 2.63063 299.63063 151.13063 15113.06306 2.63063 299.63065 151.13063 15113.06327 2.63063 299.63063 151.13062999999985 15113.06300 2020-01-01 2020-01-02 2020-01-01 00:14:36 2020-01-02 03:42:57 2020-01-01 00:14:36.000 2020-01-02 03:42:57.000 876 99777 50326.5 5032650 876 99777 50326.5 5032650 -32692 32243 4451.3 445130 -125 126 2.02 202 +877 2 10867 99778 2.63363 299.63363 151.13363 15113.36336 2.63363 299.63364 151.13363 15113.36358 2.63363 299.63363 151.1336300000001 15113.36300 2020-01-01 2020-01-02 2020-01-01 00:14:37 2020-01-02 03:42:58 2020-01-01 00:14:37.000 2020-01-02 03:42:58.000 877 99778 50327.5 5032750 877 99778 50327.5 5032750 -32691 32244 4452.3 445230 -124 127 3.02 302 +878 2 10868 99779 2.63663 299.63663 151.13663 15113.66366 2.63663 299.63663 151.13663 15113.66371 2.63663 299.63663 151.13663000000017 15113.66300 2020-01-01 2020-01-02 2020-01-01 00:14:38 2020-01-02 03:42:59 2020-01-01 00:14:38.000 2020-01-02 03:42:59.000 878 99779 50328.5 5032850 878 99779 50328.5 5032850 -32690 32245 4453.3 445330 -128 127 1.46 146 +879 2 10869 99780 2.63963 299.63963 151.13963 15113.96396 2.63963 299.63965 151.13963 15113.96385 2.63963 299.63963 151.13962999999984 15113.96300 2020-01-01 2020-01-02 2020-01-01 00:14:39 2020-01-02 03:43:00 2020-01-01 00:14:39.000 2020-01-02 03:43:00.000 879 99780 50329.5 5032950 879 99780 50329.5 5032950 -32689 32246 4454.3 445430 -128 123 -0.1 -10 +88 2 10078 99988 0.26426 300.26426 150.26426 15176.69069 0.26426 300.26425 150.26426 15176.69066 0.26426 300.26426 150.26425999999995 15176.69026 2020-01-01 2020-01-02 2020-01-01 00:01:28 2020-01-02 03:46:28 2020-01-01 00:01:28.000 2020-01-02 03:46:28.000 88 99988 50038 5053838 88 99988 50038 5053838 -32481 32454 4617.009900990099 466318 -126 125 -1.1287128712871286 -114 880 2 10870 99781 2.64264 299.64264 151.14264 15114.26426 2.64264 299.64264 151.14263 15114.26399 2.64264 299.64264 151.14264 15114.26400 2020-01-01 2020-01-02 2020-01-01 00:14:40 2020-01-02 03:43:01 2020-01-01 00:14:40.000 2020-01-02 03:43:01.000 880 99781 50330.5 5033050 880 99781 50330.5 5033050 -32688 32247 4455.3 445530 -127 124 0.9 90 -881 2 10871 99782 2.64564 299.64564 151.14564 15114.56456 2.64564 299.64566 151.14564 15114.56474 2.64564 299.64564 151.14564 15114.56400 2020-01-01 2020-01-02 2020-01-01 00:14:41 2020-01-02 03:43:02 2020-01-01 00:14:41.000 2020-01-02 03:43:02.000 881 99782 50331.5 5033150 881 99782 50331.5 5033150 -32687 32248 4456.3 445630 -126 125 1.9 190 -882 2 10872 99783 2.64864 299.64864 151.14864 15114.86486 2.64864 299.64865 151.14865 15114.86504 2.64864 299.64864 151.14864 15114.86400 2020-01-01 2020-01-02 2020-01-01 00:14:42 2020-01-02 03:43:03 2020-01-01 00:14:42.000 2020-01-02 03:43:03.000 882 99783 50332.5 5033250 882 99783 50332.5 5033250 -32686 32249 4457.3 445730 -125 126 2.9 290 -883 2 10873 99784 2.65165 299.65165 151.15165 15115.16516 2.65165 299.65164 151.15165 15115.16522 2.65165 299.65165 151.15165 15115.16500 2020-01-01 2020-01-02 2020-01-01 00:14:43 2020-01-02 03:43:04 2020-01-01 00:14:43.000 2020-01-02 03:43:04.000 883 99784 50333.5 5033350 883 99784 50333.5 5033350 -32685 32250 4458.3 445830 -124 127 3.9 390 -884 2 10874 99785 2.65465 299.65465 151.15465 15115.46546 2.65465 299.65466 151.15465 15115.46532 2.65465 299.65465 151.15465 15115.46500 2020-01-01 2020-01-02 2020-01-01 00:14:44 2020-01-02 03:43:05 2020-01-01 00:14:44.000 2020-01-02 03:43:05.000 884 99785 50334.5 5033450 884 99785 50334.5 5033450 -32684 32251 4459.3 445930 -128 127 2.34 234 -885 2 10875 99786 2.65765 299.65765 151.15765 15115.76576 2.65765 299.65765 151.15765 15115.76562 2.65765 299.65765 151.15765 15115.76500 2020-01-01 2020-01-02 2020-01-01 00:14:45 2020-01-02 03:43:06 2020-01-01 00:14:45.000 2020-01-02 03:43:06.000 885 99786 50335.5 5033550 885 99786 50335.5 5033550 -32683 32252 4460.3 446030 -128 123 0.78 78 -886 2 10876 99787 2.66066 299.66066 151.16066 15116.06606 2.66066 299.66068 151.16066 15116.06621 2.66066 299.66066 151.16066 15116.06600 2020-01-01 2020-01-02 2020-01-01 00:14:46 2020-01-02 03:43:07 2020-01-01 00:14:46.000 2020-01-02 03:43:07.000 886 99787 50336.5 5033650 886 99787 50336.5 5033650 -32682 32253 4461.3 446130 -127 124 1.78 178 -887 2 10877 99788 2.66366 299.66366 151.16366 15116.36636 2.66366 299.66367 151.16366 15116.36651 2.66366 299.66366 151.16366 15116.36600 2020-01-01 2020-01-02 2020-01-01 00:14:47 2020-01-02 03:43:08 2020-01-01 00:14:47.000 2020-01-02 03:43:08.000 887 99788 50337.5 5033750 887 99788 50337.5 5033750 -32681 32254 4462.3 446230 -126 125 2.78 278 -888 2 10878 99789 2.66666 299.66666 151.16666 15116.66666 2.66666 299.66666 151.16666 15116.66669 2.66666 299.66666 151.16666 15116.66600 2020-01-01 2020-01-02 2020-01-01 00:14:48 2020-01-02 03:43:09 2020-01-01 00:14:48.000 2020-01-02 03:43:09.000 888 99789 50338.5 5033850 888 99789 50338.5 5033850 -32680 32255 4463.3 446330 -125 126 3.78 378 -889 2 10879 99790 2.66966 299.66966 151.16966 15116.96696 2.66966 299.66968 151.16966 15116.96679 2.66966 299.66966 151.16966 15116.96600 2020-01-01 2020-01-02 2020-01-01 00:14:49 2020-01-02 03:43:10 2020-01-01 00:14:49.000 2020-01-02 03:43:10.000 889 99790 50339.5 5033950 889 99790 50339.5 5033950 -32679 32256 4464.3 446430 -124 127 4.78 478 -89 2 10079 99989 0.26726 300.26726 150.26726 15176.99399 0.26726 300.26727 150.26727 15176.9943 0.26726 300.26726 150.26726 15176.99326 2020-01-01 2020-01-02 2020-01-01 00:01:29 2020-01-02 03:46:29 2020-01-01 00:01:29.000 2020-01-02 03:46:29.000 89 99989 50039 5053939 89 99989 50039 5053939 -32480 32455 4618.009900990099 466419 -125 126 -0.12871287128712872 -13 -890 2 10880 99791 2.67267 299.67267 151.17267 15117.26726 2.67267 299.67267 151.17267 15117.26708 2.67267 299.67267 151.17267 15117.26700 2020-01-01 2020-01-02 2020-01-01 00:14:50 2020-01-02 03:43:11 2020-01-01 00:14:50.000 2020-01-02 03:43:11.000 890 99791 50340.5 5034050 890 99791 50340.5 5034050 -32678 32257 4465.3 446530 -128 127 3.22 322 +881 2 10871 99782 2.64564 299.64564 151.14564 15114.56456 2.64564 299.64566 151.14564 15114.56474 2.64564 299.64564 151.1456400000001 15114.56400 2020-01-01 2020-01-02 2020-01-01 00:14:41 2020-01-02 03:43:02 2020-01-01 00:14:41.000 2020-01-02 03:43:02.000 881 99782 50331.5 5033150 881 99782 50331.5 5033150 -32687 32248 4456.3 445630 -126 125 1.9 190 +882 2 10872 99783 2.64864 299.64864 151.14864 15114.86486 2.64864 299.64865 151.14865 15114.86504 2.64864 299.64864 151.14863999999992 15114.86400 2020-01-01 2020-01-02 2020-01-01 00:14:42 2020-01-02 03:43:03 2020-01-01 00:14:42.000 2020-01-02 03:43:03.000 882 99783 50332.5 5033250 882 99783 50332.5 5033250 -32686 32249 4457.3 445730 -125 126 2.9 290 +883 2 10873 99784 2.65165 299.65165 151.15165 15115.16516 2.65165 299.65164 151.15165 15115.16522 2.65165 299.65165 151.15164999999988 15115.16500 2020-01-01 2020-01-02 2020-01-01 00:14:43 2020-01-02 03:43:04 2020-01-01 00:14:43.000 2020-01-02 03:43:04.000 883 99784 50333.5 5033350 883 99784 50333.5 5033350 -32685 32250 4458.3 445830 -124 127 3.9 390 +884 2 10874 99785 2.65465 299.65465 151.15465 15115.46546 2.65465 299.65466 151.15465 15115.46532 2.65465 299.65465 151.1546500000002 15115.46500 2020-01-01 2020-01-02 2020-01-01 00:14:44 2020-01-02 03:43:05 2020-01-01 00:14:44.000 2020-01-02 03:43:05.000 884 99785 50334.5 5033450 884 99785 50334.5 5033450 -32684 32251 4459.3 445930 -128 127 2.34 234 +885 2 10875 99786 2.65765 299.65765 151.15765 15115.76576 2.65765 299.65765 151.15765 15115.76562 2.65765 299.65765 151.15764999999988 15115.76500 2020-01-01 2020-01-02 2020-01-01 00:14:45 2020-01-02 03:43:06 2020-01-01 00:14:45.000 2020-01-02 03:43:06.000 885 99786 50335.5 5033550 885 99786 50335.5 5033550 -32683 32252 4460.3 446030 -128 123 0.78 78 +886 2 10876 99787 2.66066 299.66066 151.16066 15116.06606 2.66066 299.66068 151.16066 15116.06621 2.66066 299.66066 151.16065999999978 15116.06600 2020-01-01 2020-01-02 2020-01-01 00:14:46 2020-01-02 03:43:07 2020-01-01 00:14:46.000 2020-01-02 03:43:07.000 886 99787 50336.5 5033650 886 99787 50336.5 5033650 -32682 32253 4461.3 446130 -127 124 1.78 178 +887 2 10877 99788 2.66366 299.66366 151.16366 15116.36636 2.66366 299.66367 151.16366 15116.36651 2.66366 299.66366 151.1636600000001 15116.36600 2020-01-01 2020-01-02 2020-01-01 00:14:47 2020-01-02 03:43:08 2020-01-01 00:14:47.000 2020-01-02 03:43:08.000 887 99788 50337.5 5033750 887 99788 50337.5 5033750 -32681 32254 4462.3 446230 -126 125 2.78 278 +888 2 10878 99789 2.66666 299.66666 151.16666 15116.66666 2.66666 299.66666 151.16666 15116.66669 2.66666 299.66666 151.1666600000001 15116.66600 2020-01-01 2020-01-02 2020-01-01 00:14:48 2020-01-02 03:43:09 2020-01-01 00:14:48.000 2020-01-02 03:43:09.000 888 99789 50338.5 5033850 888 99789 50338.5 5033850 -32680 32255 4463.3 446330 -125 126 3.78 378 +889 2 10879 99790 2.66966 299.66966 151.16966 15116.96696 2.66966 299.66968 151.16966 15116.96679 2.66966 299.66966 151.1696599999998 15116.96600 2020-01-01 2020-01-02 2020-01-01 00:14:49 2020-01-02 03:43:10 2020-01-01 00:14:49.000 2020-01-02 03:43:10.000 889 99790 50339.5 5033950 889 99790 50339.5 5033950 -32679 32256 4464.3 446430 -124 127 4.78 478 +89 2 10079 99989 0.26726 300.26726 150.26726 15176.99399 0.26726 300.26727 150.26727 15176.9943 0.26726 300.26726 150.26726000000005 15176.99326 2020-01-01 2020-01-02 2020-01-01 00:01:29 2020-01-02 03:46:29 2020-01-01 00:01:29.000 2020-01-02 03:46:29.000 89 99989 50039 5053939 89 99989 50039 5053939 -32480 32455 4618.009900990099 466419 -125 126 -0.12871287128712872 -13 +890 2 10880 99791 2.67267 299.67267 151.17267 15117.26726 2.67267 299.67267 151.17267 15117.26708 2.67267 299.67267 151.17266999999995 15117.26700 2020-01-01 2020-01-02 2020-01-01 00:14:50 2020-01-02 03:43:11 2020-01-01 00:14:50.000 2020-01-02 03:43:11.000 890 99791 50340.5 5034050 890 99791 50340.5 5034050 -32678 32257 4465.3 446530 -128 127 3.22 322 891 2 10881 99792 2.67567 299.67567 151.17567 15117.56756 2.67567 299.6757 151.17567 15117.56768 2.67567 299.67567 151.17567 15117.56700 2020-01-01 2020-01-02 2020-01-01 00:14:51 2020-01-02 03:43:12 2020-01-01 00:14:51.000 2020-01-02 03:43:12.000 891 99792 50341.5 5034150 891 99792 50341.5 5034150 -32677 32258 4466.3 446630 -128 127 1.66 166 -892 2 10882 99793 2.67867 299.67867 151.17867 15117.86786 2.67867 299.67868 151.17868 15117.86802 2.67867 299.67867 151.17867 15117.86700 2020-01-01 2020-01-02 2020-01-01 00:14:52 2020-01-02 03:43:13 2020-01-01 00:14:52.000 2020-01-02 03:43:13.000 892 99793 50342.5 5034250 892 99793 50342.5 5034250 -32676 32259 4467.3 446730 -128 124 0.1 10 -893 2 10883 99794 2.68168 299.68168 151.18168 15118.16816 2.68168 299.68167 151.18168 15118.16816 2.68168 299.68168 151.18168 15118.16800 2020-01-01 2020-01-02 2020-01-01 00:14:53 2020-01-02 03:43:14 2020-01-01 00:14:53.000 2020-01-02 03:43:14.000 893 99794 50343.5 5034350 893 99794 50343.5 5034350 -32675 32260 4468.3 446830 -127 125 1.1 110 -894 2 10884 99795 2.68468 299.68468 151.18468 15118.46846 2.68468 299.6847 151.18468 15118.46826 2.68468 299.68468 151.18468 15118.46800 2020-01-01 2020-01-02 2020-01-01 00:14:54 2020-01-02 03:43:15 2020-01-01 00:14:54.000 2020-01-02 03:43:15.000 894 99795 50344.5 5034450 894 99795 50344.5 5034450 -32674 32261 4469.3 446930 -126 126 2.1 210 -895 2 10885 99796 2.68768 299.68768 151.18768 15118.76876 2.68768 299.68768 151.18768 15118.76855 2.68768 299.68768 151.18768 15118.76800 2020-01-01 2020-01-02 2020-01-01 00:14:55 2020-01-02 03:43:16 2020-01-01 00:14:55.000 2020-01-02 03:43:16.000 895 99796 50345.5 5034550 895 99796 50345.5 5034550 -32673 32262 4470.3 447030 -125 127 3.1 310 -896 2 10886 99797 2.69069 299.69069 151.19069 15119.06906 2.69069 299.6907 151.19069 15119.06915 2.69069 299.69069 151.19069 15119.06900 2020-01-01 2020-01-02 2020-01-01 00:14:56 2020-01-02 03:43:17 2020-01-01 00:14:56.000 2020-01-02 03:43:17.000 896 99797 50346.5 5034650 896 99797 50346.5 5034650 -32672 32263 4471.3 447130 -128 127 1.54 154 -897 2 10887 99798 2.69369 299.69369 151.19369 15119.36936 2.69369 299.6937 151.19369 15119.36949 2.69369 299.69369 151.19369 15119.36900 2020-01-01 2020-01-02 2020-01-01 00:14:57 2020-01-02 03:43:18 2020-01-01 00:14:57.000 2020-01-02 03:43:18.000 897 99798 50347.5 5034750 897 99798 50347.5 5034750 -32671 32264 4472.3 447230 -128 127 -0.02 -2 -898 2 10888 99799 2.69669 299.69669 151.19669 15119.66966 2.69669 299.6967 151.19669 15119.66963 2.69669 299.69669 151.19669 15119.66900 2020-01-01 2020-01-02 2020-01-01 00:14:58 2020-01-02 03:43:19 2020-01-01 00:14:58.000 2020-01-02 03:43:19.000 898 99799 50348.5 5034850 898 99799 50348.5 5034850 -32670 32265 4473.3 447330 -128 123 -1.58 -158 -899 2 10889 99800 2.69969 299.69969 151.19969 15119.96996 2.69969 299.6997 151.1997 15119.97038 2.69969 299.69969 151.19969 15119.96900 2020-01-01 2020-01-02 2020-01-01 00:14:59 2020-01-02 03:43:20 2020-01-01 00:14:59.000 2020-01-02 03:43:20.000 899 99800 50349.5 5034950 899 99800 50349.5 5034950 -32669 32266 4474.3 447430 -127 124 -0.58 -58 -9 2 1008 9999 0.02702 300.02702 150.02702 15152.72972 0.02702 300.02704 150.02702 15152.72966 0.02702 300.02702 150.02702 15152.72902 2020-01-01 2020-01-02 2020-01-01 00:00:09 2020-01-02 03:45:09 2020-01-01 00:00:09.000 2020-01-02 03:45:09.000 9 99909 49959 5045859 9 99909 49959 5045859 -32560 32375 4538.009900990099 458339 -124 127 0.9801980198019802 99 -90 2 10080 99990 0.27027 300.27027 150.27027 15177.29729 0.27027 300.27026 150.27026 15177.29694 0.27027 300.27027 150.27027 15177.29727 2020-01-01 2020-01-02 2020-01-01 00:01:30 2020-01-02 03:46:30 2020-01-01 00:01:30.000 2020-01-02 03:46:30.000 90 99990 50040 5054040 90 99990 50040 5054040 -32479 32456 4619.009900990099 466520 -124 127 0.8712871287128713 88 -900 2 10890 99801 2.7027 299.7027 151.2027 15120.27027 2.7027 299.7027 151.2027 15120.27003 2.70270 299.70270 151.20270 15120.27000 2020-01-01 2020-01-02 2020-01-01 00:15:00 2020-01-02 03:43:21 2020-01-01 00:15:00.000 2020-01-02 03:43:21.000 900 99801 50350.5 5035050 900 99801 50350.5 5035050 -32668 32267 4475.3 447530 -126 125 0.42 42 -901 2 10891 99802 2.7057 299.7057 151.2057 15120.57057 2.7057 299.70572 151.2057 15120.57066 2.70570 299.70570 151.20570 15120.57000 2020-01-01 2020-01-02 2020-01-01 00:15:01 2020-01-02 03:43:22 2020-01-01 00:15:01.000 2020-01-02 03:43:22.000 901 99802 50351.5 5035150 901 99802 50351.5 5035150 -32667 32268 4476.3 447630 -125 126 1.42 142 -902 2 10892 99803 2.7087 299.7087 151.2087 15120.87087 2.7087 299.7087 151.2087 15120.87095 2.70870 299.70870 151.20870 15120.87000 2020-01-01 2020-01-02 2020-01-01 00:15:02 2020-01-02 03:43:23 2020-01-01 00:15:02.000 2020-01-02 03:43:23.000 902 99803 50352.5 5035250 902 99803 50352.5 5035250 -32666 32269 4477.3 447730 -124 127 2.42 242 -903 2 10893 99804 2.71171 299.71171 151.21171 15121.17117 2.71171 299.7117 151.21171 15121.1711 2.71171 299.71171 151.21171 15121.17100 2020-01-01 2020-01-02 2020-01-01 00:15:03 2020-01-02 03:43:24 2020-01-01 00:15:03.000 2020-01-02 03:43:24.000 903 99804 50353.5 5035350 903 99804 50353.5 5035350 -32665 32270 4478.3 447830 -128 127 0.86 86 -904 2 10894 99805 2.71471 299.71471 151.21471 15121.47147 2.71471 299.71472 151.21471 15121.47185 2.71471 299.71471 151.21471 15121.47100 2020-01-01 2020-01-02 2020-01-01 00:15:04 2020-01-02 03:43:25 2020-01-01 00:15:04.000 2020-01-02 03:43:25.000 904 99805 50354.5 5035450 904 99805 50354.5 5035450 -32664 32271 4479.3 447930 -128 123 -0.7 -70 -905 2 10895 99806 2.71771 299.71771 151.21771 15121.77177 2.71771 299.7177 151.21771 15121.77149 2.71771 299.71771 151.21771 15121.77100 2020-01-01 2020-01-02 2020-01-01 00:15:05 2020-01-02 03:43:26 2020-01-01 00:15:05.000 2020-01-02 03:43:26.000 905 99806 50355.5 5035550 905 99806 50355.5 5035550 -32663 32272 4480.3 448030 -127 124 0.3 30 -906 2 10896 99807 2.72072 299.72072 151.22072 15122.07207 2.72072 299.72073 151.22072 15122.07212 2.72072 299.72072 151.22072 15122.07200 2020-01-01 2020-01-02 2020-01-01 00:15:06 2020-01-02 03:43:27 2020-01-01 00:15:06.000 2020-01-02 03:43:27.000 906 99807 50356.5 5035650 906 99807 50356.5 5035650 -32662 32273 4481.3 448130 -126 125 1.3 130 +892 2 10882 99793 2.67867 299.67867 151.17867 15117.86786 2.67867 299.67868 151.17868 15117.86802 2.67867 299.67867 151.17866999999993 15117.86700 2020-01-01 2020-01-02 2020-01-01 00:14:52 2020-01-02 03:43:13 2020-01-01 00:14:52.000 2020-01-02 03:43:13.000 892 99793 50342.5 5034250 892 99793 50342.5 5034250 -32676 32259 4467.3 446730 -128 124 0.1 10 +893 2 10883 99794 2.68168 299.68168 151.18168 15118.16816 2.68168 299.68167 151.18168 15118.16816 2.68168 299.68168 151.18167999999986 15118.16800 2020-01-01 2020-01-02 2020-01-01 00:14:53 2020-01-02 03:43:14 2020-01-01 00:14:53.000 2020-01-02 03:43:14.000 893 99794 50343.5 5034350 893 99794 50343.5 5034350 -32675 32260 4468.3 446830 -127 125 1.1 110 +894 2 10884 99795 2.68468 299.68468 151.18468 15118.46846 2.68468 299.6847 151.18468 15118.46826 2.68468 299.68468 151.18468000000013 15118.46800 2020-01-01 2020-01-02 2020-01-01 00:14:54 2020-01-02 03:43:15 2020-01-01 00:14:54.000 2020-01-02 03:43:15.000 894 99795 50344.5 5034450 894 99795 50344.5 5034450 -32674 32261 4469.3 446930 -126 126 2.1 210 +895 2 10885 99796 2.68768 299.68768 151.18768 15118.76876 2.68768 299.68768 151.18768 15118.76855 2.68768 299.68768 151.1876799999998 15118.76800 2020-01-01 2020-01-02 2020-01-01 00:14:55 2020-01-02 03:43:16 2020-01-01 00:14:55.000 2020-01-02 03:43:16.000 895 99796 50345.5 5034550 895 99796 50345.5 5034550 -32673 32262 4470.3 447030 -125 127 3.1 310 +896 2 10886 99797 2.69069 299.69069 151.19069 15119.06906 2.69069 299.6907 151.19069 15119.06915 2.69069 299.69069 151.19068999999993 15119.06900 2020-01-01 2020-01-02 2020-01-01 00:14:56 2020-01-02 03:43:17 2020-01-01 00:14:56.000 2020-01-02 03:43:17.000 896 99797 50346.5 5034650 896 99797 50346.5 5034650 -32672 32263 4471.3 447130 -128 127 1.54 154 +897 2 10887 99798 2.69369 299.69369 151.19369 15119.36936 2.69369 299.6937 151.19369 15119.36949 2.69369 299.69369 151.19369000000003 15119.36900 2020-01-01 2020-01-02 2020-01-01 00:14:57 2020-01-02 03:43:18 2020-01-01 00:14:57.000 2020-01-02 03:43:18.000 897 99798 50347.5 5034750 897 99798 50347.5 5034750 -32671 32264 4472.3 447230 -128 127 -0.02 -2 +898 2 10888 99799 2.69669 299.69669 151.19669 15119.66966 2.69669 299.6967 151.19669 15119.66963 2.69669 299.69669 151.1966900000001 15119.66900 2020-01-01 2020-01-02 2020-01-01 00:14:58 2020-01-02 03:43:19 2020-01-01 00:14:58.000 2020-01-02 03:43:19.000 898 99799 50348.5 5034850 898 99799 50348.5 5034850 -32670 32265 4473.3 447330 -128 123 -1.58 -158 +899 2 10889 99800 2.69969 299.69969 151.19969 15119.96996 2.69969 299.6997 151.1997 15119.97038 2.69969 299.69969 151.19968999999998 15119.96900 2020-01-01 2020-01-02 2020-01-01 00:14:59 2020-01-02 03:43:20 2020-01-01 00:14:59.000 2020-01-02 03:43:20.000 899 99800 50349.5 5034950 899 99800 50349.5 5034950 -32669 32266 4474.3 447430 -127 124 -0.58 -58 +9 2 1008 9999 0.02702 300.02702 150.02702 15152.72972 0.02702 300.02704 150.02702 15152.72966 0.02702 300.02702 150.02701999999985 15152.72902 2020-01-01 2020-01-02 2020-01-01 00:00:09 2020-01-02 03:45:09 2020-01-01 00:00:09.000 2020-01-02 03:45:09.000 9 99909 49959 5045859 9 99909 49959 5045859 -32560 32375 4538.009900990099 458339 -124 127 0.9801980198019802 99 +90 2 10080 99990 0.27027 300.27027 150.27027 15177.29729 0.27027 300.27026 150.27026 15177.29694 0.27027 300.27027 150.27027000000024 15177.29727 2020-01-01 2020-01-02 2020-01-01 00:01:30 2020-01-02 03:46:30 2020-01-01 00:01:30.000 2020-01-02 03:46:30.000 90 99990 50040 5054040 90 99990 50040 5054040 -32479 32456 4619.009900990099 466520 -124 127 0.8712871287128713 88 +900 2 10890 99801 2.7027 299.7027 151.2027 15120.27027 2.7027 299.7027 151.2027 15120.27003 2.70270 299.70270 151.20269999999988 15120.27000 2020-01-01 2020-01-02 2020-01-01 00:15:00 2020-01-02 03:43:21 2020-01-01 00:15:00.000 2020-01-02 03:43:21.000 900 99801 50350.5 5035050 900 99801 50350.5 5035050 -32668 32267 4475.3 447530 -126 125 0.42 42 +901 2 10891 99802 2.7057 299.7057 151.2057 15120.57057 2.7057 299.70572 151.2057 15120.57066 2.70570 299.70570 151.2057000000002 15120.57000 2020-01-01 2020-01-02 2020-01-01 00:15:01 2020-01-02 03:43:22 2020-01-01 00:15:01.000 2020-01-02 03:43:22.000 901 99802 50351.5 5035150 901 99802 50351.5 5035150 -32667 32268 4476.3 447630 -125 126 1.42 142 +902 2 10892 99803 2.7087 299.7087 151.2087 15120.87087 2.7087 299.7087 151.2087 15120.87095 2.70870 299.70870 151.2086999999999 15120.87000 2020-01-01 2020-01-02 2020-01-01 00:15:02 2020-01-02 03:43:23 2020-01-01 00:15:02.000 2020-01-02 03:43:23.000 902 99803 50352.5 5035250 902 99803 50352.5 5035250 -32666 32269 4477.3 447730 -124 127 2.42 242 +903 2 10893 99804 2.71171 299.71171 151.21171 15121.17117 2.71171 299.7117 151.21171 15121.1711 2.71171 299.71171 151.21170999999978 15121.17100 2020-01-01 2020-01-02 2020-01-01 00:15:03 2020-01-02 03:43:24 2020-01-01 00:15:03.000 2020-01-02 03:43:24.000 903 99804 50353.5 5035350 903 99804 50353.5 5035350 -32665 32270 4478.3 447830 -128 127 0.86 86 +904 2 10894 99805 2.71471 299.71471 151.21471 15121.47147 2.71471 299.71472 151.21471 15121.47185 2.71471 299.71471 151.2147100000001 15121.47100 2020-01-01 2020-01-02 2020-01-01 00:15:04 2020-01-02 03:43:25 2020-01-01 00:15:04.000 2020-01-02 03:43:25.000 904 99805 50354.5 5035450 904 99805 50354.5 5035450 -32664 32271 4479.3 447930 -128 123 -0.7 -70 +905 2 10895 99806 2.71771 299.71771 151.21771 15121.77177 2.71771 299.7177 151.21771 15121.77149 2.71771 299.71771 151.21771000000012 15121.77100 2020-01-01 2020-01-02 2020-01-01 00:15:05 2020-01-02 03:43:26 2020-01-01 00:15:05.000 2020-01-02 03:43:26.000 905 99806 50355.5 5035550 905 99806 50355.5 5035550 -32663 32272 4480.3 448030 -127 124 0.3 30 +906 2 10896 99807 2.72072 299.72072 151.22072 15122.07207 2.72072 299.72073 151.22072 15122.07212 2.72072 299.72072 151.2207199999999 15122.07200 2020-01-01 2020-01-02 2020-01-01 00:15:06 2020-01-02 03:43:27 2020-01-01 00:15:06.000 2020-01-02 03:43:27.000 906 99807 50356.5 5035650 906 99807 50356.5 5035650 -32662 32273 4481.3 448130 -126 125 1.3 130 907 2 10897 99808 2.72372 299.72372 151.22372 15122.37237 2.72372 299.72372 151.22372 15122.37243 2.72372 299.72372 151.22372 15122.37200 2020-01-01 2020-01-02 2020-01-01 00:15:07 2020-01-02 03:43:28 2020-01-01 00:15:07.000 2020-01-02 03:43:28.000 907 99808 50357.5 5035750 907 99808 50357.5 5035750 -32661 32274 4482.3 448230 -125 126 2.3 230 -908 2 10898 99809 2.72672 299.72672 151.22672 15122.67267 2.72672 299.7267 151.22672 15122.67272 2.72672 299.72672 151.22672 15122.67200 2020-01-01 2020-01-02 2020-01-01 00:15:08 2020-01-02 03:43:29 2020-01-01 00:15:08.000 2020-01-02 03:43:29.000 908 99809 50358.5 5035850 908 99809 50358.5 5035850 -32660 32275 4483.3 448330 -124 127 3.3 330 -909 2 10899 99810 2.72972 299.72972 151.22972 15122.97297 2.72972 299.72974 151.22973 15122.97332 2.72972 299.72972 151.22972 15122.97200 2020-01-01 2020-01-02 2020-01-01 00:15:09 2020-01-02 03:43:30 2020-01-01 00:15:09.000 2020-01-02 03:43:30.000 909 99810 50359.5 5035950 909 99810 50359.5 5035950 -32659 32276 4484.3 448430 -128 127 1.74 174 -91 2 10081 99991 0.27327 300.27327 150.27327 15177.6006 0.27327 300.2733 150.27327 15177.60054 0.27327 300.27327 150.27327 15177.60027 2020-01-01 2020-01-02 2020-01-01 00:01:31 2020-01-02 03:46:31 2020-01-01 00:01:31.000 2020-01-02 03:46:31.000 91 99991 50041 5054141 91 99991 50041 5054141 -32478 32457 4620.009900990099 466621 -128 127 -0.6633663366336634 -67 -910 2 10900 99811 2.73273 299.73273 151.23273 15123.27327 2.73273 299.73273 151.23272 15123.27296 2.73273 299.73273 151.23273 15123.27300 2020-01-01 2020-01-02 2020-01-01 00:15:10 2020-01-02 03:43:31 2020-01-01 00:15:10.000 2020-01-02 03:43:31.000 910 99811 50360.5 5036050 910 99811 50360.5 5036050 -32658 32277 4485.3 448530 -128 123 0.18 18 -911 2 10901 99812 2.73573 299.73573 151.23573 15123.57357 2.73573 299.73575 151.23573 15123.57359 2.73573 299.73573 151.23573 15123.57300 2020-01-01 2020-01-02 2020-01-01 00:15:11 2020-01-02 03:43:32 2020-01-01 00:15:11.000 2020-01-02 03:43:32.000 911 99812 50361.5 5036150 911 99812 50361.5 5036150 -32657 32278 4486.3 448630 -127 124 1.18 118 -912 2 10902 99813 2.73873 299.73873 151.23873 15123.87387 2.73873 299.73874 151.23873 15123.8739 2.73873 299.73873 151.23873 15123.87300 2020-01-01 2020-01-02 2020-01-01 00:15:12 2020-01-02 03:43:33 2020-01-01 00:15:12.000 2020-01-02 03:43:33.000 912 99813 50362.5 5036250 912 99813 50362.5 5036250 -32656 32279 4487.3 448730 -126 125 2.18 218 +908 2 10898 99809 2.72672 299.72672 151.22672 15122.67267 2.72672 299.7267 151.22672 15122.67272 2.72672 299.72672 151.22672000000006 15122.67200 2020-01-01 2020-01-02 2020-01-01 00:15:08 2020-01-02 03:43:29 2020-01-01 00:15:08.000 2020-01-02 03:43:29.000 908 99809 50358.5 5035850 908 99809 50358.5 5035850 -32660 32275 4483.3 448330 -124 127 3.3 330 +909 2 10899 99810 2.72972 299.72972 151.22972 15122.97297 2.72972 299.72974 151.22973 15122.97332 2.72972 299.72972 151.22971999999987 15122.97200 2020-01-01 2020-01-02 2020-01-01 00:15:09 2020-01-02 03:43:30 2020-01-01 00:15:09.000 2020-01-02 03:43:30.000 909 99810 50359.5 5035950 909 99810 50359.5 5035950 -32659 32276 4484.3 448430 -128 127 1.74 174 +91 2 10081 99991 0.27327 300.27327 150.27327 15177.6006 0.27327 300.2733 150.27327 15177.60054 0.27327 300.27327 150.27326999999985 15177.60027 2020-01-01 2020-01-02 2020-01-01 00:01:31 2020-01-02 03:46:31 2020-01-01 00:01:31.000 2020-01-02 03:46:31.000 91 99991 50041 5054141 91 99991 50041 5054141 -32478 32457 4620.009900990099 466621 -128 127 -0.6633663366336634 -67 +910 2 10900 99811 2.73273 299.73273 151.23273 15123.27327 2.73273 299.73273 151.23272 15123.27296 2.73273 299.73273 151.2327299999999 15123.27300 2020-01-01 2020-01-02 2020-01-01 00:15:10 2020-01-02 03:43:31 2020-01-01 00:15:10.000 2020-01-02 03:43:31.000 910 99811 50360.5 5036050 910 99811 50360.5 5036050 -32658 32277 4485.3 448530 -128 123 0.18 18 +911 2 10901 99812 2.73573 299.73573 151.23573 15123.57357 2.73573 299.73575 151.23573 15123.57359 2.73573 299.73573 151.23573000000016 15123.57300 2020-01-01 2020-01-02 2020-01-01 00:15:11 2020-01-02 03:43:32 2020-01-01 00:15:11.000 2020-01-02 03:43:32.000 911 99812 50361.5 5036150 911 99812 50361.5 5036150 -32657 32278 4486.3 448630 -127 124 1.18 118 +912 2 10902 99813 2.73873 299.73873 151.23873 15123.87387 2.73873 299.73874 151.23873 15123.8739 2.73873 299.73873 151.2387299999998 15123.87300 2020-01-01 2020-01-02 2020-01-01 00:15:12 2020-01-02 03:43:33 2020-01-01 00:15:12.000 2020-01-02 03:43:33.000 912 99813 50362.5 5036250 912 99813 50362.5 5036250 -32656 32279 4487.3 448730 -126 125 2.18 218 913 2 10903 99814 2.74174 299.74174 151.24174 15124.17417 2.74174 299.74173 151.24174 15124.17419 2.74174 299.74174 151.24174 15124.17400 2020-01-01 2020-01-02 2020-01-01 00:15:13 2020-01-02 03:43:34 2020-01-01 00:15:13.000 2020-01-02 03:43:34.000 913 99814 50363.5 5036350 913 99814 50363.5 5036350 -32655 32280 4488.3 448830 -125 126 3.18 318 -914 2 10904 99815 2.74474 299.74474 151.24474 15124.47447 2.74474 299.74475 151.24474 15124.47479 2.74474 299.74474 151.24474 15124.47400 2020-01-01 2020-01-02 2020-01-01 00:15:14 2020-01-02 03:43:35 2020-01-01 00:15:14.000 2020-01-02 03:43:35.000 914 99815 50364.5 5036450 914 99815 50364.5 5036450 -32654 32281 4489.3 448930 -124 127 4.18 418 -915 2 10905 99816 2.74774 299.74774 151.24774 15124.77477 2.74774 299.74774 151.24774 15124.77447 2.74774 299.74774 151.24774 15124.77400 2020-01-01 2020-01-02 2020-01-01 00:15:15 2020-01-02 03:43:36 2020-01-01 00:15:15.000 2020-01-02 03:43:36.000 915 99816 50365.5 5036550 915 99816 50365.5 5036550 -32653 32282 4490.3 449030 -128 127 2.62 262 -916 2 10906 99817 2.75075 299.75075 151.25075 15125.07507 2.75075 299.75076 151.25075 15125.07507 2.75075 299.75075 151.25075 15125.07500 2020-01-01 2020-01-02 2020-01-01 00:15:16 2020-01-02 03:43:37 2020-01-01 00:15:16.000 2020-01-02 03:43:37.000 916 99817 50366.5 5036650 916 99817 50366.5 5036650 -32652 32283 4491.3 449130 -128 127 1.06 106 -917 2 10907 99818 2.75375 299.75375 151.25375 15125.37537 2.75375 299.75375 151.25375 15125.37536 2.75375 299.75375 151.25375 15125.37500 2020-01-01 2020-01-02 2020-01-01 00:15:17 2020-01-02 03:43:38 2020-01-01 00:15:17.000 2020-01-02 03:43:38.000 917 99818 50367.5 5036750 917 99818 50367.5 5036750 -32651 32284 4492.3 449230 -128 124 -0.5 -50 -918 2 10908 99819 2.75675 299.75675 151.25675 15125.67567 2.75675 299.75674 151.25675 15125.67566 2.75675 299.75675 151.25675 15125.67500 2020-01-01 2020-01-02 2020-01-01 00:15:18 2020-01-02 03:43:39 2020-01-01 00:15:18.000 2020-01-02 03:43:39.000 918 99819 50368.5 5036850 918 99819 50368.5 5036850 -32650 32285 4493.3 449330 -127 125 0.5 50 -919 2 10909 99820 2.75975 299.75975 151.25975 15125.97597 2.75975 299.75977 151.25976 15125.97626 2.75975 299.75975 151.25975 15125.97500 2020-01-01 2020-01-02 2020-01-01 00:15:19 2020-01-02 03:43:40 2020-01-01 00:15:19.000 2020-01-02 03:43:40.000 919 99820 50369.5 5036950 919 99820 50369.5 5036950 -32649 32286 4494.3 449430 -126 126 1.5 150 -92 2 10082 99992 0.27627 300.27627 150.27627 15177.9039 0.27627 300.27628 150.27627 15177.90384 0.27627 300.27627 150.27627 15177.90327 2020-01-01 2020-01-02 2020-01-01 00:01:32 2020-01-02 03:46:32 2020-01-01 00:01:32.000 2020-01-02 03:46:32.000 92 99992 50042 5054242 92 99992 50042 5054242 -32477 32458 4621.009900990099 466722 -128 123 -2.198019801980198 -222 -920 2 10910 99821 2.76276 299.76276 151.26276 15126.27627 2.76276 299.76276 151.26275 15126.27594 2.76276 299.76276 151.26276 15126.27600 2020-01-01 2020-01-02 2020-01-01 00:15:20 2020-01-02 03:43:41 2020-01-01 00:15:20.000 2020-01-02 03:43:41.000 920 99821 50370.5 5037050 920 99821 50370.5 5037050 -32648 32287 4495.3 449530 -125 127 2.5 250 -921 2 10911 99822 2.76576 299.76576 151.26576 15126.57657 2.76576 299.76578 151.26576 15126.57654 2.76576 299.76576 151.26576 15126.57600 2020-01-01 2020-01-02 2020-01-01 00:15:21 2020-01-02 03:43:42 2020-01-01 00:15:21.000 2020-01-02 03:43:42.000 921 99822 50371.5 5037150 921 99822 50371.5 5037150 -32647 32288 4496.3 449630 -128 127 0.94 94 -922 2 10912 99823 2.76876 299.76876 151.26876 15126.87687 2.76876 299.76877 151.26876 15126.87683 2.76876 299.76876 151.26876 15126.87600 2020-01-01 2020-01-02 2020-01-01 00:15:22 2020-01-02 03:43:43 2020-01-01 00:15:22.000 2020-01-02 03:43:43.000 922 99823 50372.5 5037250 922 99823 50372.5 5037250 -32646 32289 4497.3 449730 -128 127 -0.62 -62 -923 2 10913 99824 2.77177 299.77177 151.27177 15127.17717 2.77177 299.77176 151.27177 15127.17713 2.77177 299.77177 151.27177 15127.17700 2020-01-01 2020-01-02 2020-01-01 00:15:23 2020-01-02 03:43:44 2020-01-01 00:15:23.000 2020-01-02 03:43:44.000 923 99824 50373.5 5037350 923 99824 50373.5 5037350 -32645 32290 4498.3 449830 -128 123 -2.18 -218 +914 2 10904 99815 2.74474 299.74474 151.24474 15124.47447 2.74474 299.74475 151.24474 15124.47479 2.74474 299.74474 151.24474000000006 15124.47400 2020-01-01 2020-01-02 2020-01-01 00:15:14 2020-01-02 03:43:35 2020-01-01 00:15:14.000 2020-01-02 03:43:35.000 914 99815 50364.5 5036450 914 99815 50364.5 5036450 -32654 32281 4489.3 448930 -124 127 4.18 418 +915 2 10905 99816 2.74774 299.74774 151.24774 15124.77477 2.74774 299.74774 151.24774 15124.77447 2.74774 299.74774 151.24774000000008 15124.77400 2020-01-01 2020-01-02 2020-01-01 00:15:15 2020-01-02 03:43:36 2020-01-01 00:15:15.000 2020-01-02 03:43:36.000 915 99816 50365.5 5036550 915 99816 50365.5 5036550 -32653 32282 4490.3 449030 -128 127 2.62 262 +916 2 10906 99817 2.75075 299.75075 151.25075 15125.07507 2.75075 299.75076 151.25075 15125.07507 2.75075 299.75075 151.25074999999987 15125.07500 2020-01-01 2020-01-02 2020-01-01 00:15:16 2020-01-02 03:43:37 2020-01-01 00:15:16.000 2020-01-02 03:43:37.000 916 99817 50366.5 5036650 916 99817 50366.5 5036650 -32652 32283 4491.3 449130 -128 127 1.06 106 +917 2 10907 99818 2.75375 299.75375 151.25375 15125.37537 2.75375 299.75375 151.25375 15125.37536 2.75375 299.75375 151.2537499999999 15125.37500 2020-01-01 2020-01-02 2020-01-01 00:15:17 2020-01-02 03:43:38 2020-01-01 00:15:17.000 2020-01-02 03:43:38.000 917 99818 50367.5 5036750 917 99818 50367.5 5036750 -32651 32284 4492.3 449230 -128 124 -0.5 -50 +918 2 10908 99819 2.75675 299.75675 151.25675 15125.67567 2.75675 299.75674 151.25675 15125.67566 2.75675 299.75675 151.25675000000004 15125.67500 2020-01-01 2020-01-02 2020-01-01 00:15:18 2020-01-02 03:43:39 2020-01-01 00:15:18.000 2020-01-02 03:43:39.000 918 99819 50368.5 5036850 918 99819 50368.5 5036850 -32650 32285 4493.3 449330 -127 125 0.5 50 +919 2 10909 99820 2.75975 299.75975 151.25975 15125.97597 2.75975 299.75977 151.25976 15125.97626 2.75975 299.75975 151.25974999999994 15125.97500 2020-01-01 2020-01-02 2020-01-01 00:15:19 2020-01-02 03:43:40 2020-01-01 00:15:19.000 2020-01-02 03:43:40.000 919 99820 50369.5 5036950 919 99820 50369.5 5036950 -32649 32286 4494.3 449430 -126 126 1.5 150 +92 2 10082 99992 0.27627 300.27627 150.27627 15177.9039 0.27627 300.27628 150.27627 15177.90384 0.27627 300.27627 150.27627000000012 15177.90327 2020-01-01 2020-01-02 2020-01-01 00:01:32 2020-01-02 03:46:32 2020-01-01 00:01:32.000 2020-01-02 03:46:32.000 92 99992 50042 5054242 92 99992 50042 5054242 -32477 32458 4621.009900990099 466722 -128 123 -2.198019801980198 -222 +920 2 10910 99821 2.76276 299.76276 151.26276 15126.27627 2.76276 299.76276 151.26275 15126.27594 2.76276 299.76276 151.2627599999998 15126.27600 2020-01-01 2020-01-02 2020-01-01 00:15:20 2020-01-02 03:43:41 2020-01-01 00:15:20.000 2020-01-02 03:43:41.000 920 99821 50370.5 5037050 920 99821 50370.5 5037050 -32648 32287 4495.3 449530 -125 127 2.5 250 +921 2 10911 99822 2.76576 299.76576 151.26576 15126.57657 2.76576 299.76578 151.26576 15126.57654 2.76576 299.76576 151.26576000000017 15126.57600 2020-01-01 2020-01-02 2020-01-01 00:15:21 2020-01-02 03:43:42 2020-01-01 00:15:21.000 2020-01-02 03:43:42.000 921 99822 50371.5 5037150 921 99822 50371.5 5037150 -32647 32288 4496.3 449630 -128 127 0.94 94 +922 2 10912 99823 2.76876 299.76876 151.26876 15126.87687 2.76876 299.76877 151.26876 15126.87683 2.76876 299.76876 151.26876000000016 15126.87600 2020-01-01 2020-01-02 2020-01-01 00:15:22 2020-01-02 03:43:43 2020-01-01 00:15:22.000 2020-01-02 03:43:43.000 922 99823 50372.5 5037250 922 99823 50372.5 5037250 -32646 32289 4497.3 449730 -128 127 -0.62 -62 +923 2 10913 99824 2.77177 299.77177 151.27177 15127.17717 2.77177 299.77176 151.27177 15127.17713 2.77177 299.77177 151.27176999999992 15127.17700 2020-01-01 2020-01-02 2020-01-01 00:15:23 2020-01-02 03:43:44 2020-01-01 00:15:23.000 2020-01-02 03:43:44.000 923 99824 50373.5 5037350 923 99824 50373.5 5037350 -32645 32290 4498.3 449830 -128 123 -2.18 -218 924 2 10914 99825 2.77477 299.77477 151.27477 15127.47747 2.77477 299.77478 151.27477 15127.47776 2.77477 299.77477 151.27477 15127.47700 2020-01-01 2020-01-02 2020-01-01 00:15:24 2020-01-02 03:43:45 2020-01-01 00:15:24.000 2020-01-02 03:43:45.000 924 99825 50374.5 5037450 924 99825 50374.5 5037450 -32644 32291 4499.3 449930 -127 124 -1.18 -118 -925 2 10915 99826 2.77777 299.77777 151.27777 15127.77777 2.77777 299.77777 151.27777 15127.77741 2.77777 299.77777 151.27777 15127.77700 2020-01-01 2020-01-02 2020-01-01 00:15:25 2020-01-02 03:43:46 2020-01-01 00:15:25.000 2020-01-02 03:43:46.000 925 99826 50375.5 5037550 925 99826 50375.5 5037550 -32643 32292 4500.3 450030 -126 125 -0.18 -18 -926 2 10916 99827 2.78078 299.78078 151.28078 15128.07807 2.78078 299.7808 151.28078 15128.078 2.78078 299.78078 151.28078 15128.07800 2020-01-01 2020-01-02 2020-01-01 00:15:26 2020-01-02 03:43:47 2020-01-01 00:15:26.000 2020-01-02 03:43:47.000 926 99827 50376.5 5037650 926 99827 50376.5 5037650 -32642 32293 4501.3 450130 -125 126 0.82 82 -927 2 10917 99828 2.78378 299.78378 151.28378 15128.37837 2.78378 299.78378 151.28378 15128.3783 2.78378 299.78378 151.28378 15128.37800 2020-01-01 2020-01-02 2020-01-01 00:15:27 2020-01-02 03:43:48 2020-01-01 00:15:27.000 2020-01-02 03:43:48.000 927 99828 50377.5 5037750 927 99828 50377.5 5037750 -32641 32294 4502.3 450230 -124 127 1.82 182 -928 2 10918 99829 2.78678 299.78678 151.28678 15128.67867 2.78678 299.78677 151.28678 15128.6786 2.78678 299.78678 151.28678 15128.67800 2020-01-01 2020-01-02 2020-01-01 00:15:28 2020-01-02 03:43:49 2020-01-01 00:15:28.000 2020-01-02 03:43:49.000 928 99829 50378.5 5037850 928 99829 50378.5 5037850 -32640 32295 4503.3 450330 -128 127 0.26 26 -929 2 10919 99830 2.78978 299.78978 151.28978 15128.97897 2.78978 299.7898 151.28979 15128.97923 2.78978 299.78978 151.28978 15128.97800 2020-01-01 2020-01-02 2020-01-01 00:15:29 2020-01-02 03:43:50 2020-01-01 00:15:29.000 2020-01-02 03:43:50.000 929 99830 50379.5 5037950 929 99830 50379.5 5037950 -32639 32296 4504.3 450430 -128 123 -1.3 -130 -93 2 10083 99993 0.27927 300.27927 150.27927 15178.2072 0.27927 300.27927 150.27927 15178.20715 0.27927 300.27927 150.27927 15178.20627 2020-01-01 2020-01-02 2020-01-01 00:01:33 2020-01-02 03:46:33 2020-01-01 00:01:33.000 2020-01-02 03:46:33.000 93 99993 50043 5054343 93 99993 50043 5054343 -32476 32459 4622.009900990099 466823 -127 124 -1.198019801980198 -121 +925 2 10915 99826 2.77777 299.77777 151.27777 15127.77777 2.77777 299.77777 151.27777 15127.77741 2.77777 299.77777 151.2777700000001 15127.77700 2020-01-01 2020-01-02 2020-01-01 00:15:25 2020-01-02 03:43:46 2020-01-01 00:15:25.000 2020-01-02 03:43:46.000 925 99826 50375.5 5037550 925 99826 50375.5 5037550 -32643 32292 4500.3 450030 -126 125 -0.18 -18 +926 2 10916 99827 2.78078 299.78078 151.28078 15128.07807 2.78078 299.7808 151.28078 15128.078 2.78078 299.78078 151.28077999999982 15128.07800 2020-01-01 2020-01-02 2020-01-01 00:15:26 2020-01-02 03:43:47 2020-01-01 00:15:26.000 2020-01-02 03:43:47.000 926 99827 50376.5 5037650 926 99827 50376.5 5037650 -32642 32293 4501.3 450130 -125 126 0.82 82 +927 2 10917 99828 2.78378 299.78378 151.28378 15128.37837 2.78378 299.78378 151.28378 15128.3783 2.78378 299.78378 151.28377999999987 15128.37800 2020-01-01 2020-01-02 2020-01-01 00:15:27 2020-01-02 03:43:48 2020-01-01 00:15:27.000 2020-01-02 03:43:48.000 927 99828 50377.5 5037750 927 99828 50377.5 5037750 -32641 32294 4502.3 450230 -124 127 1.82 182 +928 2 10918 99829 2.78678 299.78678 151.28678 15128.67867 2.78678 299.78677 151.28678 15128.6786 2.78678 299.78678 151.2867800000002 15128.67800 2020-01-01 2020-01-02 2020-01-01 00:15:28 2020-01-02 03:43:49 2020-01-01 00:15:28.000 2020-01-02 03:43:49.000 928 99829 50378.5 5037850 928 99829 50378.5 5037850 -32640 32295 4503.3 450330 -128 127 0.26 26 +929 2 10919 99830 2.78978 299.78978 151.28978 15128.97897 2.78978 299.7898 151.28979 15128.97923 2.78978 299.78978 151.28977999999984 15128.97800 2020-01-01 2020-01-02 2020-01-01 00:15:29 2020-01-02 03:43:50 2020-01-01 00:15:29.000 2020-01-02 03:43:50.000 929 99830 50379.5 5037950 929 99830 50379.5 5037950 -32639 32296 4504.3 450430 -128 123 -1.3 -130 +93 2 10083 99993 0.27927 300.27927 150.27927 15178.2072 0.27927 300.27927 150.27927 15178.20715 0.27927 300.27927 150.27926999999983 15178.20627 2020-01-01 2020-01-02 2020-01-01 00:01:33 2020-01-02 03:46:33 2020-01-01 00:01:33.000 2020-01-02 03:46:33.000 93 99993 50043 5054343 93 99993 50043 5054343 -32476 32459 4622.009900990099 466823 -127 124 -1.198019801980198 -121 930 2 10920 99831 2.79279 299.79279 151.29279 15129.27927 2.79279 299.7928 151.29278 15129.27888 2.79279 299.79279 151.29279 15129.27900 2020-01-01 2020-01-02 2020-01-01 00:15:30 2020-01-02 03:43:51 2020-01-01 00:15:30.000 2020-01-02 03:43:51.000 930 99831 50380.5 5038050 930 99831 50380.5 5038050 -32638 32297 4505.3 450530 -127 124 -0.3 -30 -931 2 10921 99832 2.79579 299.79579 151.29579 15129.57957 2.79579 299.7958 151.29579 15129.57963 2.79579 299.79579 151.29579 15129.57900 2020-01-01 2020-01-02 2020-01-01 00:15:31 2020-01-02 03:43:52 2020-01-01 00:15:31.000 2020-01-02 03:43:52.000 931 99832 50381.5 5038150 931 99832 50381.5 5038150 -32637 32298 4506.3 450630 -126 125 0.7 70 -932 2 10922 99833 2.79879 299.79879 151.29879 15129.87987 2.79879 299.7988 151.29879 15129.87977 2.79879 299.79879 151.29879 15129.87900 2020-01-01 2020-01-02 2020-01-01 00:15:32 2020-01-02 03:43:53 2020-01-01 00:15:32.000 2020-01-02 03:43:53.000 932 99833 50382.5 5038250 932 99833 50382.5 5038250 -32636 32299 4507.3 450730 -125 126 1.7 170 -933 2 10923 99834 2.8018 299.8018 151.3018 15130.18018 2.8018 299.8018 151.3018 15130.18011 2.80180 299.80180 151.30180 15130.18000 2020-01-01 2020-01-02 2020-01-01 00:15:33 2020-01-02 03:43:54 2020-01-01 00:15:33.000 2020-01-02 03:43:54.000 933 99834 50383.5 5038350 933 99834 50383.5 5038350 -32635 32300 4508.3 450830 -124 127 2.7 270 -934 2 10924 99835 2.8048 299.8048 151.3048 15130.48048 2.8048 299.8048 151.3048 15130.48071 2.80480 299.80480 151.30480 15130.48000 2020-01-01 2020-01-02 2020-01-01 00:15:34 2020-01-02 03:43:55 2020-01-01 00:15:34.000 2020-01-02 03:43:55.000 934 99835 50384.5 5038450 934 99835 50384.5 5038450 -32634 32301 4509.3 450930 -128 127 1.14 114 -935 2 10925 99836 2.8078 299.8078 151.3078 15130.78078 2.8078 299.8078 151.3078 15130.78034 2.80780 299.80780 151.30780 15130.78000 2020-01-01 2020-01-02 2020-01-01 00:15:35 2020-01-02 03:43:56 2020-01-01 00:15:35.000 2020-01-02 03:43:56.000 935 99836 50385.5 5038550 935 99836 50385.5 5038550 -32633 32302 4510.3 451030 -128 123 -0.42 -42 -936 2 10926 99837 2.81081 299.81081 151.31081 15131.08108 2.81081 299.81082 151.31081 15131.0811 2.81081 299.81081 151.31081 15131.08100 2020-01-01 2020-01-02 2020-01-01 00:15:36 2020-01-02 03:43:57 2020-01-01 00:15:36.000 2020-01-02 03:43:57.000 936 99837 50386.5 5038650 936 99837 50386.5 5038650 -32632 32303 4511.3 451130 -127 124 0.58 58 -937 2 10927 99838 2.81381 299.81381 151.31381 15131.38138 2.81381 299.8138 151.31381 15131.38124 2.81381 299.81381 151.31381 15131.38100 2020-01-01 2020-01-02 2020-01-01 00:15:37 2020-01-02 03:43:58 2020-01-01 00:15:37.000 2020-01-02 03:43:58.000 937 99838 50387.5 5038750 937 99838 50387.5 5038750 -32631 32304 4512.3 451230 -126 125 1.58 158 -938 2 10928 99839 2.81681 299.81681 151.31681 15131.68168 2.81681 299.8168 151.31681 15131.68157 2.81681 299.81681 151.31681 15131.68100 2020-01-01 2020-01-02 2020-01-01 00:15:38 2020-01-02 03:43:59 2020-01-01 00:15:38.000 2020-01-02 03:43:59.000 938 99839 50388.5 5038850 938 99839 50388.5 5038850 -32630 32305 4513.3 451330 -125 126 2.58 258 -939 2 10929 99840 2.81981 299.81981 151.31981 15131.98198 2.81982 299.81982 151.31982 15131.98217 2.81981 299.81981 151.31981 15131.98100 2020-01-01 2020-01-02 2020-01-01 00:15:39 2020-01-02 03:44:00 2020-01-01 00:15:39.000 2020-01-02 03:44:00.000 939 99840 50389.5 5038950 939 99840 50389.5 5038950 -32629 32306 4514.3 451430 -124 127 3.58 358 -94 2 10084 99994 0.28228 300.28228 150.28228 15178.51051 0.28228 300.2823 150.28228 15178.51078 0.28228 300.28228 150.28228 15178.51028 2020-01-01 2020-01-02 2020-01-01 00:01:34 2020-01-02 03:46:34 2020-01-01 00:01:34.000 2020-01-02 03:46:34.000 94 99994 50044 5054444 94 99994 50044 5054444 -32475 32460 4623.009900990099 466924 -126 125 -0.19801980198019803 -20 -940 2 10930 99841 2.82282 299.82282 151.32282 15132.28228 2.82282 299.8228 151.32282 15132.28247 2.82282 299.82282 151.32282 15132.28200 2020-01-01 2020-01-02 2020-01-01 00:15:40 2020-01-02 03:44:01 2020-01-01 00:15:40.000 2020-01-02 03:44:01.000 940 99841 50390.5 5039050 940 99841 50390.5 5039050 -32628 32307 4515.3 451530 -128 127 2.02 202 +931 2 10921 99832 2.79579 299.79579 151.29579 15129.57957 2.79579 299.7958 151.29579 15129.57963 2.79579 299.79579 151.2957900000001 15129.57900 2020-01-01 2020-01-02 2020-01-01 00:15:31 2020-01-02 03:43:52 2020-01-01 00:15:31.000 2020-01-02 03:43:52.000 931 99832 50381.5 5038150 931 99832 50381.5 5038150 -32637 32298 4506.3 450630 -126 125 0.7 70 +932 2 10922 99833 2.79879 299.79879 151.29879 15129.87987 2.79879 299.7988 151.29879 15129.87977 2.79879 299.79879 151.2987900000001 15129.87900 2020-01-01 2020-01-02 2020-01-01 00:15:32 2020-01-02 03:43:53 2020-01-01 00:15:32.000 2020-01-02 03:43:53.000 932 99833 50382.5 5038250 932 99833 50382.5 5038250 -32636 32299 4507.3 450730 -125 126 1.7 170 +933 2 10923 99834 2.8018 299.8018 151.3018 15130.18018 2.8018 299.8018 151.3018 15130.18011 2.80180 299.80180 151.3017999999999 15130.18000 2020-01-01 2020-01-02 2020-01-01 00:15:33 2020-01-02 03:43:54 2020-01-01 00:15:33.000 2020-01-02 03:43:54.000 933 99834 50383.5 5038350 933 99834 50383.5 5038350 -32635 32300 4508.3 450830 -124 127 2.7 270 +934 2 10924 99835 2.8048 299.8048 151.3048 15130.48048 2.8048 299.8048 151.3048 15130.48071 2.80480 299.80480 151.30479999999991 15130.48000 2020-01-01 2020-01-02 2020-01-01 00:15:34 2020-01-02 03:43:55 2020-01-01 00:15:34.000 2020-01-02 03:43:55.000 934 99835 50384.5 5038450 934 99835 50384.5 5038450 -32634 32301 4509.3 450930 -128 127 1.14 114 +935 2 10925 99836 2.8078 299.8078 151.3078 15130.78078 2.8078 299.8078 151.3078 15130.78034 2.80780 299.80780 151.30780000000001 15130.78000 2020-01-01 2020-01-02 2020-01-01 00:15:35 2020-01-02 03:43:56 2020-01-01 00:15:35.000 2020-01-02 03:43:56.000 935 99836 50385.5 5038550 935 99836 50385.5 5038550 -32633 32302 4510.3 451030 -128 123 -0.42 -42 +936 2 10926 99837 2.81081 299.81081 151.31081 15131.08108 2.81081 299.81082 151.31081 15131.0811 2.81081 299.81081 151.31081000000017 15131.08100 2020-01-01 2020-01-02 2020-01-01 00:15:36 2020-01-02 03:43:57 2020-01-01 00:15:36.000 2020-01-02 03:43:57.000 936 99837 50386.5 5038650 936 99837 50386.5 5038650 -32632 32303 4511.3 451130 -127 124 0.58 58 +937 2 10927 99838 2.81381 299.81381 151.31381 15131.38138 2.81381 299.8138 151.31381 15131.38124 2.81381 299.81381 151.31380999999985 15131.38100 2020-01-01 2020-01-02 2020-01-01 00:15:37 2020-01-02 03:43:58 2020-01-01 00:15:37.000 2020-01-02 03:43:58.000 937 99838 50387.5 5038750 937 99838 50387.5 5038750 -32631 32304 4512.3 451230 -126 125 1.58 158 +938 2 10928 99839 2.81681 299.81681 151.31681 15131.68168 2.81681 299.8168 151.31681 15131.68157 2.81681 299.81681 151.3168100000001 15131.68100 2020-01-01 2020-01-02 2020-01-01 00:15:38 2020-01-02 03:43:59 2020-01-01 00:15:38.000 2020-01-02 03:43:59.000 938 99839 50388.5 5038850 938 99839 50388.5 5038850 -32630 32305 4513.3 451330 -125 126 2.58 258 +939 2 10929 99840 2.81981 299.81981 151.31981 15131.98198 2.81982 299.81982 151.31982 15131.98217 2.81981 299.81981 151.3198100000002 15131.98100 2020-01-01 2020-01-02 2020-01-01 00:15:39 2020-01-02 03:44:00 2020-01-01 00:15:39.000 2020-01-02 03:44:00.000 939 99840 50389.5 5038950 939 99840 50389.5 5038950 -32629 32306 4514.3 451430 -124 127 3.58 358 +94 2 10084 99994 0.28228 300.28228 150.28228 15178.51051 0.28228 300.2823 150.28228 15178.51078 0.28228 300.28228 150.28227999999996 15178.51028 2020-01-01 2020-01-02 2020-01-01 00:01:34 2020-01-02 03:46:34 2020-01-01 00:01:34.000 2020-01-02 03:46:34.000 94 99994 50044 5054444 94 99994 50044 5054444 -32475 32460 4623.009900990099 466924 -126 125 -0.19801980198019803 -20 +940 2 10930 99841 2.82282 299.82282 151.32282 15132.28228 2.82282 299.8228 151.32282 15132.28247 2.82282 299.82282 151.32281999999995 15132.28200 2020-01-01 2020-01-02 2020-01-01 00:15:40 2020-01-02 03:44:01 2020-01-01 00:15:40.000 2020-01-02 03:44:01.000 940 99841 50390.5 5039050 940 99841 50390.5 5039050 -32628 32307 4515.3 451530 -128 127 2.02 202 941 2 10931 99842 2.82582 299.82582 151.32582 15132.58258 2.82582 299.82584 151.32582 15132.58257 2.82582 299.82582 151.32582 15132.58200 2020-01-01 2020-01-02 2020-01-01 00:15:41 2020-01-02 03:44:02 2020-01-01 00:15:41.000 2020-01-02 03:44:02.000 941 99842 50391.5 5039150 941 99842 50391.5 5039150 -32627 32308 4516.3 451630 -128 127 0.46 46 -942 2 10932 99843 2.82882 299.82882 151.32882 15132.88288 2.82882 299.82883 151.32882 15132.88275 2.82882 299.82882 151.32882 15132.88200 2020-01-01 2020-01-02 2020-01-01 00:15:42 2020-01-02 03:44:03 2020-01-01 00:15:42.000 2020-01-02 03:44:03.000 942 99843 50392.5 5039250 942 99843 50392.5 5039250 -32626 32309 4517.3 451730 -128 124 -1.1 -110 -943 2 10933 99844 2.83183 299.83183 151.33183 15133.18318 2.83183 299.83182 151.33183 15133.18304 2.83183 299.83183 151.33183 15133.18300 2020-01-01 2020-01-02 2020-01-01 00:15:43 2020-01-02 03:44:04 2020-01-01 00:15:43.000 2020-01-02 03:44:04.000 943 99844 50393.5 5039350 943 99844 50393.5 5039350 -32625 32310 4518.3 451830 -127 125 -0.1 -10 -944 2 10934 99845 2.83483 299.83483 151.33483 15133.48348 2.83483 299.83484 151.33483 15133.48364 2.83483 299.83483 151.33483 15133.48300 2020-01-01 2020-01-02 2020-01-01 00:15:44 2020-01-02 03:44:05 2020-01-01 00:15:44.000 2020-01-02 03:44:05.000 944 99845 50394.5 5039450 944 99845 50394.5 5039450 -32624 32311 4519.3 451930 -126 126 0.9 90 -945 2 10935 99846 2.83783 299.83783 151.33783 15133.78378 2.83783 299.83783 151.33783 15133.78393 2.83783 299.83783 151.33783 15133.78300 2020-01-01 2020-01-02 2020-01-01 00:15:45 2020-01-02 03:44:06 2020-01-01 00:15:45.000 2020-01-02 03:44:06.000 945 99846 50395.5 5039550 945 99846 50395.5 5039550 -32623 32312 4520.3 452030 -125 127 1.9 190 -946 2 10936 99847 2.84084 299.84084 151.34084 15134.08408 2.84084 299.84085 151.34084 15134.08404 2.84084 299.84084 151.34084 15134.08400 2020-01-01 2020-01-02 2020-01-01 00:15:46 2020-01-02 03:44:07 2020-01-01 00:15:46.000 2020-01-02 03:44:07.000 946 99847 50396.5 5039650 946 99847 50396.5 5039650 -32622 32313 4521.3 452130 -128 127 0.34 34 -947 2 10937 99848 2.84384 299.84384 151.34384 15134.38438 2.84384 299.84384 151.34384 15134.38421 2.84384 299.84384 151.34384 15134.38400 2020-01-01 2020-01-02 2020-01-01 00:15:47 2020-01-02 03:44:08 2020-01-01 00:15:47.000 2020-01-02 03:44:08.000 947 99848 50397.5 5039750 947 99848 50397.5 5039750 -32621 32314 4522.3 452230 -128 127 -1.22 -122 -948 2 10938 99849 2.84684 299.84684 151.34684 15134.68468 2.84684 299.84683 151.34684 15134.68452 2.84684 299.84684 151.34684 15134.68400 2020-01-01 2020-01-02 2020-01-01 00:15:48 2020-01-02 03:44:09 2020-01-01 00:15:48.000 2020-01-02 03:44:09.000 948 99849 50398.5 5039850 948 99849 50398.5 5039850 -32620 32315 4523.3 452330 -128 123 -2.78 -278 -949 2 10939 99850 2.84984 299.84984 151.34984 15134.98498 2.84985 299.84985 151.34985 15134.98527 2.84984 299.84984 151.34984 15134.98400 2020-01-01 2020-01-02 2020-01-01 00:15:49 2020-01-02 03:44:10 2020-01-01 00:15:49.000 2020-01-02 03:44:10.000 949 99850 50399.5 5039950 949 99850 50399.5 5039950 -32619 32316 4524.3 452430 -127 124 -1.78 -178 -95 2 10085 99995 0.28528 300.28528 150.28528 15178.81381 0.28528 300.28528 150.28528 15178.81343 0.28528 300.28528 150.28528 15178.81328 2020-01-01 2020-01-02 2020-01-01 00:01:35 2020-01-02 03:46:35 2020-01-01 00:01:35.000 2020-01-02 03:46:35.000 95 99995 50045 5054545 95 99995 50045 5054545 -32474 32461 4624.009900990099 467025 -125 126 0.801980198019802 81 -950 2 10940 99851 2.85285 299.85285 151.35285 15135.28528 2.85285 299.85284 151.35285 15135.28541 2.85285 299.85285 151.35285 15135.28500 2020-01-01 2020-01-02 2020-01-01 00:15:50 2020-01-02 03:44:11 2020-01-01 00:15:50.000 2020-01-02 03:44:11.000 950 99851 50400.5 5040050 950 99851 50400.5 5040050 -32618 32317 4525.3 452530 -126 125 -0.78 -78 -951 2 10941 99852 2.85585 299.85585 151.35585 15135.58558 2.85585 299.85587 151.35585 15135.58551 2.85585 299.85585 151.35585 15135.58500 2020-01-01 2020-01-02 2020-01-01 00:15:51 2020-01-02 03:44:12 2020-01-01 00:15:51.000 2020-01-02 03:44:12.000 951 99852 50401.5 5040150 951 99852 50401.5 5040150 -32617 32318 4526.3 452630 -125 126 0.22 22 -952 2 10942 99853 2.85885 299.85885 151.35885 15135.88588 2.85885 299.85886 151.35885 15135.88568 2.85885 299.85885 151.35885 15135.88500 2020-01-01 2020-01-02 2020-01-01 00:15:52 2020-01-02 03:44:13 2020-01-01 00:15:52.000 2020-01-02 03:44:13.000 952 99853 50402.5 5040250 952 99853 50402.5 5040250 -32616 32319 4527.3 452730 -124 127 1.22 122 -953 2 10943 99854 2.86186 299.86186 151.36186 15136.18618 2.86186 299.86185 151.36185 15136.18598 2.86186 299.86186 151.36186 15136.18600 2020-01-01 2020-01-02 2020-01-01 00:15:53 2020-01-02 03:44:14 2020-01-01 00:15:53.000 2020-01-02 03:44:14.000 953 99854 50403.5 5040350 953 99854 50403.5 5040350 -32615 32320 4528.3 452830 -128 127 -0.34 -34 -954 2 10944 99855 2.86486 299.86486 151.36486 15136.48648 2.86486 299.86487 151.36486 15136.48674 2.86486 299.86486 151.36486 15136.48600 2020-01-01 2020-01-02 2020-01-01 00:15:54 2020-01-02 03:44:15 2020-01-01 00:15:54.000 2020-01-02 03:44:15.000 954 99855 50404.5 5040450 954 99855 50404.5 5040450 -32614 32321 4529.3 452930 -128 123 -1.9 -190 -955 2 10945 99856 2.86786 299.86786 151.36786 15136.78678 2.86786 299.86786 151.36786 15136.78688 2.86786 299.86786 151.36786 15136.78600 2020-01-01 2020-01-02 2020-01-01 00:15:55 2020-01-02 03:44:16 2020-01-01 00:15:55.000 2020-01-02 03:44:16.000 955 99856 50405.5 5040550 955 99856 50405.5 5040550 -32613 32322 4530.3 453030 -127 124 -0.9 -90 -956 2 10946 99857 2.87087 299.87087 151.37087 15137.08708 2.87087 299.87088 151.37087 15137.08701 2.87087 299.87087 151.37087 15137.08700 2020-01-01 2020-01-02 2020-01-01 00:15:56 2020-01-02 03:44:17 2020-01-01 00:15:56.000 2020-01-02 03:44:17.000 956 99857 50406.5 5040650 956 99857 50406.5 5040650 -32612 32323 4531.3 453130 -126 125 0.1 10 -957 2 10947 99858 2.87387 299.87387 151.37387 15137.38738 2.87387 299.87387 151.37387 15137.38716 2.87387 299.87387 151.37387 15137.38700 2020-01-01 2020-01-02 2020-01-01 00:15:57 2020-01-02 03:44:18 2020-01-01 00:15:57.000 2020-01-02 03:44:18.000 957 99858 50407.5 5040750 957 99858 50407.5 5040750 -32611 32324 4532.3 453230 -125 126 1.1 110 -958 2 10948 99859 2.87687 299.87687 151.37687 15137.68768 2.87687 299.8769 151.37687 15137.68791 2.87687 299.87687 151.37687 15137.68700 2020-01-01 2020-01-02 2020-01-01 00:15:58 2020-01-02 03:44:19 2020-01-01 00:15:58.000 2020-01-02 03:44:19.000 958 99859 50408.5 5040850 958 99859 50408.5 5040850 -32610 32325 4533.3 453330 -124 127 2.1 210 -959 2 10949 99860 2.87987 299.87987 151.37987 15137.98798 2.87988 299.87988 151.37988 15137.9882 2.87987 299.87987 151.37987 15137.98700 2020-01-01 2020-01-02 2020-01-01 00:15:59 2020-01-02 03:44:20 2020-01-01 00:15:59.000 2020-01-02 03:44:20.000 959 99860 50409.5 5040950 959 99860 50409.5 5040950 -32609 32326 4534.3 453430 -128 127 0.54 54 -96 2 10086 99996 0.28828 300.28828 150.28828 15179.11711 0.28828 300.2883 150.28828 15179.11718 0.28828 300.28828 150.28828 15179.11628 2020-01-01 2020-01-02 2020-01-01 00:01:36 2020-01-02 03:46:36 2020-01-01 00:01:36.000 2020-01-02 03:46:36.000 96 99996 50046 5054646 96 99996 50046 5054646 -32473 32462 4625.009900990099 467126 -124 127 1.801980198019802 182 -960 2 10950 99861 2.88288 299.88288 151.38288 15138.28828 2.88288 299.88287 151.38288 15138.28834 2.88288 299.88288 151.38288 15138.28800 2020-01-01 2020-01-02 2020-01-01 00:16:00 2020-01-02 03:44:21 2020-01-01 00:16:00.000 2020-01-02 03:44:21.000 960 99861 50410.5 5041050 960 99861 50410.5 5041050 -32608 32327 4535.3 453530 -128 123 -1.02 -102 -961 2 10951 99862 2.88588 299.88588 151.38588 15138.58858 2.88588 299.8859 151.38588 15138.58848 2.88588 299.88588 151.38588 15138.58800 2020-01-01 2020-01-02 2020-01-01 00:16:01 2020-01-02 03:44:22 2020-01-01 00:16:01.000 2020-01-02 03:44:22.000 961 99862 50411.5 5041150 961 99862 50411.5 5041150 -32607 32328 4536.3 453630 -127 124 -0.02 -2 -962 2 10952 99863 2.88888 299.88888 151.38888 15138.88888 2.88888 299.8889 151.38888 15138.88862 2.88888 299.88888 151.38888 15138.88800 2020-01-01 2020-01-02 2020-01-01 00:16:02 2020-01-02 03:44:23 2020-01-01 00:16:02.000 2020-01-02 03:44:23.000 962 99863 50412.5 5041250 962 99863 50412.5 5041250 -32606 32329 4537.3 453730 -126 125 0.98 98 -963 2 10953 99864 2.89189 299.89189 151.39189 15139.18918 2.89189 299.8919 151.39189 15139.18937 2.89189 299.89189 151.39189 15139.18900 2020-01-01 2020-01-02 2020-01-01 00:16:03 2020-01-02 03:44:24 2020-01-01 00:16:03.000 2020-01-02 03:44:24.000 963 99864 50413.5 5041350 963 99864 50413.5 5041350 -32605 32330 4538.3 453830 -125 126 1.98 198 -964 2 10954 99865 2.89489 299.89489 151.39489 15139.48948 2.89489 299.8949 151.39489 15139.48968 2.89489 299.89489 151.39489 15139.48900 2020-01-01 2020-01-02 2020-01-01 00:16:04 2020-01-02 03:44:25 2020-01-01 00:16:04.000 2020-01-02 03:44:25.000 964 99865 50414.5 5041450 964 99865 50414.5 5041450 -32604 32331 4539.3 453930 -124 127 2.98 298 -965 2 10955 99866 2.89789 299.89789 151.39789 15139.78978 2.89789 299.8979 151.39789 15139.78985 2.89789 299.89789 151.39789 15139.78900 2020-01-01 2020-01-02 2020-01-01 00:16:05 2020-01-02 03:44:26 2020-01-01 00:16:05.000 2020-01-02 03:44:26.000 965 99866 50415.5 5041550 965 99866 50415.5 5041550 -32603 32332 4540.3 454030 -128 127 1.42 142 -966 2 10956 99867 2.9009 299.9009 151.4009 15140.09009 2.9009 299.9009 151.40089 15140.08996 2.90090 299.90090 151.40090 15140.09000 2020-01-01 2020-01-02 2020-01-01 00:16:06 2020-01-02 03:44:27 2020-01-01 00:16:06.000 2020-01-02 03:44:27.000 966 99867 50416.5 5041650 966 99867 50416.5 5041650 -32602 32333 4541.3 454130 -128 127 -0.14 -14 -967 2 10957 99868 2.9039 299.9039 151.4039 15140.39039 2.9039 299.9039 151.4039 15140.39009 2.90390 299.90390 151.40390 15140.39000 2020-01-01 2020-01-02 2020-01-01 00:16:07 2020-01-02 03:44:28 2020-01-01 00:16:07.000 2020-01-02 03:44:28.000 967 99868 50417.5 5041750 967 99868 50417.5 5041750 -32601 32334 4542.3 454230 -128 124 -1.7 -170 -968 2 10958 99869 2.9069 299.9069 151.4069 15140.69069 2.9069 299.90692 151.4069 15140.69084 2.90690 299.90690 151.40690 15140.69000 2020-01-01 2020-01-02 2020-01-01 00:16:08 2020-01-02 03:44:29 2020-01-01 00:16:08.000 2020-01-02 03:44:29.000 968 99869 50418.5 5041850 968 99869 50418.5 5041850 -32600 32335 4543.3 454330 -127 125 -0.7 -70 -969 2 10959 99870 2.9099 299.9099 151.4099 15140.99099 2.90991 299.9099 151.40991 15140.99114 2.90990 299.90990 151.40990 15140.99000 2020-01-01 2020-01-02 2020-01-01 00:16:09 2020-01-02 03:44:30 2020-01-01 00:16:09.000 2020-01-02 03:44:30.000 969 99870 50419.5 5041950 969 99870 50419.5 5041950 -32599 32336 4544.3 454430 -126 126 0.3 30 -97 2 10087 99997 0.29129 300.29129 150.29129 15179.42042 0.29129 300.2913 150.29129 15179.42033 0.29129 300.29129 150.29129 15179.42029 2020-01-01 2020-01-02 2020-01-01 00:01:37 2020-01-02 03:46:37 2020-01-01 00:01:37.000 2020-01-02 03:46:37.000 97 99997 50047 5054747 97 99997 50047 5054747 -32472 32463 4626.009900990099 467227 -128 127 0.26732673267326734 27 -970 2 10960 99871 2.91291 299.91291 151.41291 15141.29129 2.91291 299.9129 151.41291 15141.29132 2.91291 299.91291 151.41291 15141.29100 2020-01-01 2020-01-02 2020-01-01 00:16:10 2020-01-02 03:44:31 2020-01-01 00:16:10.000 2020-01-02 03:44:31.000 970 99871 50420.5 5042050 970 99871 50420.5 5042050 -32598 32337 4545.3 454530 -125 127 1.3 130 -971 2 10961 99872 2.91591 299.91591 151.41591 15141.59159 2.91591 299.91592 151.41591 15141.59142 2.91591 299.91591 151.41591 15141.59100 2020-01-01 2020-01-02 2020-01-01 00:16:11 2020-01-02 03:44:32 2020-01-01 00:16:11.000 2020-01-02 03:44:32.000 971 99872 50421.5 5042150 971 99872 50421.5 5042150 -32597 32338 4546.3 454630 -128 127 -0.26 -26 -972 2 10962 99873 2.91891 299.91891 151.41891 15141.89189 2.91891 299.9189 151.41891 15141.89172 2.91891 299.91891 151.41891 15141.89100 2020-01-01 2020-01-02 2020-01-01 00:16:12 2020-01-02 03:44:33 2020-01-01 00:16:12.000 2020-01-02 03:44:33.000 972 99873 50422.5 5042250 972 99873 50422.5 5042250 -32596 32339 4547.3 454730 -128 127 -1.82 -182 -973 2 10963 99874 2.92192 299.92192 151.42192 15142.19219 2.92192 299.92194 151.42192 15142.19232 2.92192 299.92192 151.42192 15142.19200 2020-01-01 2020-01-02 2020-01-01 00:16:13 2020-01-02 03:44:34 2020-01-01 00:16:13.000 2020-01-02 03:44:34.000 973 99874 50423.5 5042350 973 99874 50423.5 5042350 -32595 32340 4548.3 454830 -128 123 -3.38 -338 +942 2 10932 99843 2.82882 299.82882 151.32882 15132.88288 2.82882 299.82883 151.32882 15132.88275 2.82882 299.82882 151.3288200000001 15132.88200 2020-01-01 2020-01-02 2020-01-01 00:15:42 2020-01-02 03:44:03 2020-01-01 00:15:42.000 2020-01-02 03:44:03.000 942 99843 50392.5 5039250 942 99843 50392.5 5039250 -32626 32309 4517.3 451730 -128 124 -1.1 -110 +943 2 10933 99844 2.83183 299.83183 151.33183 15133.18318 2.83183 299.83182 151.33183 15133.18304 2.83183 299.83183 151.33182999999985 15133.18300 2020-01-01 2020-01-02 2020-01-01 00:15:43 2020-01-02 03:44:04 2020-01-01 00:15:43.000 2020-01-02 03:44:04.000 943 99844 50393.5 5039350 943 99844 50393.5 5039350 -32625 32310 4518.3 451830 -127 125 -0.1 -10 +944 2 10934 99845 2.83483 299.83483 151.33483 15133.48348 2.83483 299.83484 151.33483 15133.48364 2.83483 299.83483 151.33482999999984 15133.48300 2020-01-01 2020-01-02 2020-01-01 00:15:44 2020-01-02 03:44:05 2020-01-01 00:15:44.000 2020-01-02 03:44:05.000 944 99845 50394.5 5039450 944 99845 50394.5 5039450 -32624 32311 4519.3 451930 -126 126 0.9 90 +945 2 10935 99846 2.83783 299.83783 151.33783 15133.78378 2.83783 299.83783 151.33783 15133.78393 2.83783 299.83783 151.3378300000002 15133.78300 2020-01-01 2020-01-02 2020-01-01 00:15:45 2020-01-02 03:44:06 2020-01-01 00:15:45.000 2020-01-02 03:44:06.000 945 99846 50395.5 5039550 945 99846 50395.5 5039550 -32623 32312 4520.3 452030 -125 127 1.9 190 +946 2 10936 99847 2.84084 299.84084 151.34084 15134.08408 2.84084 299.84085 151.34084 15134.08404 2.84084 299.84084 151.34084000000007 15134.08400 2020-01-01 2020-01-02 2020-01-01 00:15:46 2020-01-02 03:44:07 2020-01-01 00:15:46.000 2020-01-02 03:44:07.000 946 99847 50396.5 5039650 946 99847 50396.5 5039650 -32622 32313 4521.3 452130 -128 127 0.34 34 +947 2 10937 99848 2.84384 299.84384 151.34384 15134.38438 2.84384 299.84384 151.34384 15134.38421 2.84384 299.84384 151.34383999999977 15134.38400 2020-01-01 2020-01-02 2020-01-01 00:15:47 2020-01-02 03:44:08 2020-01-01 00:15:47.000 2020-01-02 03:44:08.000 947 99848 50397.5 5039750 947 99848 50397.5 5039750 -32621 32314 4522.3 452230 -128 127 -1.22 -122 +948 2 10938 99849 2.84684 299.84684 151.34684 15134.68468 2.84684 299.84683 151.34684 15134.68452 2.84684 299.84684 151.3468400000001 15134.68400 2020-01-01 2020-01-02 2020-01-01 00:15:48 2020-01-02 03:44:09 2020-01-01 00:15:48.000 2020-01-02 03:44:09.000 948 99849 50398.5 5039850 948 99849 50398.5 5039850 -32620 32315 4523.3 452330 -128 123 -2.78 -278 +949 2 10939 99850 2.84984 299.84984 151.34984 15134.98498 2.84985 299.84985 151.34985 15134.98527 2.84984 299.84984 151.34984000000009 15134.98400 2020-01-01 2020-01-02 2020-01-01 00:15:49 2020-01-02 03:44:10 2020-01-01 00:15:49.000 2020-01-02 03:44:10.000 949 99850 50399.5 5039950 949 99850 50399.5 5039950 -32619 32316 4524.3 452430 -127 124 -1.78 -178 +95 2 10085 99995 0.28528 300.28528 150.28528 15178.81381 0.28528 300.28528 150.28528 15178.81343 0.28528 300.28528 150.28528000000003 15178.81328 2020-01-01 2020-01-02 2020-01-01 00:01:35 2020-01-02 03:46:35 2020-01-01 00:01:35.000 2020-01-02 03:46:35.000 95 99995 50045 5054545 95 99995 50045 5054545 -32474 32461 4624.009900990099 467025 -125 126 0.801980198019802 81 +950 2 10940 99851 2.85285 299.85285 151.35285 15135.28528 2.85285 299.85284 151.35285 15135.28541 2.85285 299.85285 151.35284999999996 15135.28500 2020-01-01 2020-01-02 2020-01-01 00:15:50 2020-01-02 03:44:11 2020-01-01 00:15:50.000 2020-01-02 03:44:11.000 950 99851 50400.5 5040050 950 99851 50400.5 5040050 -32618 32317 4525.3 452530 -126 125 -0.78 -78 +951 2 10941 99852 2.85585 299.85585 151.35585 15135.58558 2.85585 299.85587 151.35585 15135.58551 2.85585 299.85585 151.35584999999995 15135.58500 2020-01-01 2020-01-02 2020-01-01 00:15:51 2020-01-02 03:44:12 2020-01-01 00:15:51.000 2020-01-02 03:44:12.000 951 99852 50401.5 5040150 951 99852 50401.5 5040150 -32617 32318 4526.3 452630 -125 126 0.22 22 +952 2 10942 99853 2.85885 299.85885 151.35885 15135.88588 2.85885 299.85886 151.35885 15135.88568 2.85885 299.85885 151.35885000000002 15135.88500 2020-01-01 2020-01-02 2020-01-01 00:15:52 2020-01-02 03:44:13 2020-01-01 00:15:52.000 2020-01-02 03:44:13.000 952 99853 50402.5 5040250 952 99853 50402.5 5040250 -32616 32319 4527.3 452730 -124 127 1.22 122 +953 2 10943 99854 2.86186 299.86186 151.36186 15136.18618 2.86186 299.86185 151.36185 15136.18598 2.86186 299.86186 151.36186000000018 15136.18600 2020-01-01 2020-01-02 2020-01-01 00:15:53 2020-01-02 03:44:14 2020-01-01 00:15:53.000 2020-01-02 03:44:14.000 953 99854 50403.5 5040350 953 99854 50403.5 5040350 -32615 32320 4528.3 452830 -128 127 -0.34 -34 +954 2 10944 99855 2.86486 299.86486 151.36486 15136.48648 2.86486 299.86487 151.36486 15136.48674 2.86486 299.86486 151.36485999999985 15136.48600 2020-01-01 2020-01-02 2020-01-01 00:15:54 2020-01-02 03:44:15 2020-01-01 00:15:54.000 2020-01-02 03:44:15.000 954 99855 50404.5 5040450 954 99855 50404.5 5040450 -32614 32321 4529.3 452930 -128 123 -1.9 -190 +955 2 10945 99856 2.86786 299.86786 151.36786 15136.78678 2.86786 299.86786 151.36786 15136.78688 2.86786 299.86786 151.36786000000012 15136.78600 2020-01-01 2020-01-02 2020-01-01 00:15:55 2020-01-02 03:44:16 2020-01-01 00:15:55.000 2020-01-02 03:44:16.000 955 99856 50405.5 5040550 955 99856 50405.5 5040550 -32613 32322 4530.3 453030 -127 124 -0.9 -90 +956 2 10946 99857 2.87087 299.87087 151.37087 15137.08708 2.87087 299.87088 151.37087 15137.08701 2.87087 299.87087 151.37087000000014 15137.08700 2020-01-01 2020-01-02 2020-01-01 00:15:56 2020-01-02 03:44:17 2020-01-01 00:15:56.000 2020-01-02 03:44:17.000 956 99857 50406.5 5040650 956 99857 50406.5 5040650 -32612 32323 4531.3 453130 -126 125 0.1 10 +957 2 10947 99858 2.87387 299.87387 151.37387 15137.38738 2.87387 299.87387 151.37387 15137.38716 2.87387 299.87387 151.37386999999995 15137.38700 2020-01-01 2020-01-02 2020-01-01 00:15:57 2020-01-02 03:44:18 2020-01-01 00:15:57.000 2020-01-02 03:44:18.000 957 99858 50407.5 5040750 957 99858 50407.5 5040750 -32611 32324 4532.3 453230 -125 126 1.1 110 +958 2 10948 99859 2.87687 299.87687 151.37687 15137.68768 2.87687 299.8769 151.37687 15137.68791 2.87687 299.87687 151.37687000000003 15137.68700 2020-01-01 2020-01-02 2020-01-01 00:15:58 2020-01-02 03:44:19 2020-01-01 00:15:58.000 2020-01-02 03:44:19.000 958 99859 50408.5 5040850 958 99859 50408.5 5040850 -32610 32325 4533.3 453330 -124 127 2.1 210 +959 2 10949 99860 2.87987 299.87987 151.37987 15137.98798 2.87988 299.87988 151.37988 15137.9882 2.87987 299.87987 151.3798700000001 15137.98700 2020-01-01 2020-01-02 2020-01-01 00:15:59 2020-01-02 03:44:20 2020-01-01 00:15:59.000 2020-01-02 03:44:20.000 959 99860 50409.5 5040950 959 99860 50409.5 5040950 -32609 32326 4534.3 453430 -128 127 0.54 54 +96 2 10086 99996 0.28828 300.28828 150.28828 15179.11711 0.28828 300.2883 150.28828 15179.11718 0.28828 300.28828 150.28828000000007 15179.11628 2020-01-01 2020-01-02 2020-01-01 00:01:36 2020-01-02 03:46:36 2020-01-01 00:01:36.000 2020-01-02 03:46:36.000 96 99996 50046 5054646 96 99996 50046 5054646 -32473 32462 4625.009900990099 467126 -124 127 1.801980198019802 182 +960 2 10950 99861 2.88288 299.88288 151.38288 15138.28828 2.88288 299.88287 151.38288 15138.28834 2.88288 299.88288 151.3828799999999 15138.28800 2020-01-01 2020-01-02 2020-01-01 00:16:00 2020-01-02 03:44:21 2020-01-01 00:16:00.000 2020-01-02 03:44:21.000 960 99861 50410.5 5041050 960 99861 50410.5 5041050 -32608 32327 4535.3 453530 -128 123 -1.02 -102 +961 2 10951 99862 2.88588 299.88588 151.38588 15138.58858 2.88588 299.8859 151.38588 15138.58848 2.88588 299.88588 151.38587999999987 15138.58800 2020-01-01 2020-01-02 2020-01-01 00:16:01 2020-01-02 03:44:22 2020-01-01 00:16:01.000 2020-01-02 03:44:22.000 961 99862 50411.5 5041150 961 99862 50411.5 5041150 -32607 32328 4536.3 453630 -127 124 -0.02 -2 +962 2 10952 99863 2.88888 299.88888 151.38888 15138.88888 2.88888 299.8889 151.38888 15138.88862 2.88888 299.88888 151.3888800000002 15138.88800 2020-01-01 2020-01-02 2020-01-01 00:16:02 2020-01-02 03:44:23 2020-01-01 00:16:02.000 2020-01-02 03:44:23.000 962 99863 50412.5 5041250 962 99863 50412.5 5041250 -32606 32329 4537.3 453730 -126 125 0.98 98 +963 2 10953 99864 2.89189 299.89189 151.39189 15139.18918 2.89189 299.8919 151.39189 15139.18937 2.89189 299.89189 151.39189000000013 15139.18900 2020-01-01 2020-01-02 2020-01-01 00:16:03 2020-01-02 03:44:24 2020-01-01 00:16:03.000 2020-01-02 03:44:24.000 963 99864 50413.5 5041350 963 99864 50413.5 5041350 -32605 32330 4538.3 453830 -125 126 1.98 198 +964 2 10954 99865 2.89489 299.89489 151.39489 15139.48948 2.89489 299.8949 151.39489 15139.48968 2.89489 299.89489 151.39488999999978 15139.48900 2020-01-01 2020-01-02 2020-01-01 00:16:04 2020-01-02 03:44:25 2020-01-01 00:16:04.000 2020-01-02 03:44:25.000 964 99865 50414.5 5041450 964 99865 50414.5 5041450 -32604 32331 4539.3 453930 -124 127 2.98 298 +965 2 10955 99866 2.89789 299.89789 151.39789 15139.78978 2.89789 299.8979 151.39789 15139.78985 2.89789 299.89789 151.39789000000013 15139.78900 2020-01-01 2020-01-02 2020-01-01 00:16:05 2020-01-02 03:44:26 2020-01-01 00:16:05.000 2020-01-02 03:44:26.000 965 99866 50415.5 5041550 965 99866 50415.5 5041550 -32603 32332 4540.3 454030 -128 127 1.42 142 +966 2 10956 99867 2.9009 299.9009 151.4009 15140.09009 2.9009 299.9009 151.40089 15140.08996 2.90090 299.90090 151.4009 15140.09000 2020-01-01 2020-01-02 2020-01-01 00:16:06 2020-01-02 03:44:27 2020-01-01 00:16:06.000 2020-01-02 03:44:27.000 966 99867 50416.5 5041650 966 99867 50416.5 5041650 -32602 32333 4541.3 454130 -128 127 -0.14 -14 +967 2 10957 99868 2.9039 299.9039 151.4039 15140.39039 2.9039 299.9039 151.4039 15140.39009 2.90390 299.90390 151.4038999999999 15140.39000 2020-01-01 2020-01-02 2020-01-01 00:16:07 2020-01-02 03:44:28 2020-01-01 00:16:07.000 2020-01-02 03:44:28.000 967 99868 50417.5 5041750 967 99868 50417.5 5041750 -32601 32334 4542.3 454230 -128 124 -1.7 -170 +968 2 10958 99869 2.9069 299.9069 151.4069 15140.69069 2.9069 299.90692 151.4069 15140.69084 2.90690 299.90690 151.40689999999998 15140.69000 2020-01-01 2020-01-02 2020-01-01 00:16:08 2020-01-02 03:44:29 2020-01-01 00:16:08.000 2020-01-02 03:44:29.000 968 99869 50418.5 5041850 968 99869 50418.5 5041850 -32600 32335 4543.3 454330 -127 125 -0.7 -70 +969 2 10959 99870 2.9099 299.9099 151.4099 15140.99099 2.90991 299.9099 151.40991 15140.99114 2.90990 299.90990 151.40990000000002 15140.99000 2020-01-01 2020-01-02 2020-01-01 00:16:09 2020-01-02 03:44:30 2020-01-01 00:16:09.000 2020-01-02 03:44:30.000 969 99870 50419.5 5041950 969 99870 50419.5 5041950 -32599 32336 4544.3 454430 -126 126 0.3 30 +97 2 10087 99997 0.29129 300.29129 150.29129 15179.42042 0.29129 300.2913 150.29129 15179.42033 0.29129 300.29129 150.29128999999986 15179.42029 2020-01-01 2020-01-02 2020-01-01 00:01:37 2020-01-02 03:46:37 2020-01-01 00:01:37.000 2020-01-02 03:46:37.000 97 99997 50047 5054747 97 99997 50047 5054747 -32472 32463 4626.009900990099 467227 -128 127 0.26732673267326734 27 +970 2 10960 99871 2.91291 299.91291 151.41291 15141.29129 2.91291 299.9129 151.41291 15141.29132 2.91291 299.91291 151.41290999999984 15141.29100 2020-01-01 2020-01-02 2020-01-01 00:16:10 2020-01-02 03:44:31 2020-01-01 00:16:10.000 2020-01-02 03:44:31.000 970 99871 50420.5 5042050 970 99871 50420.5 5042050 -32598 32337 4545.3 454530 -125 127 1.3 130 +971 2 10961 99872 2.91591 299.91591 151.41591 15141.59159 2.91591 299.91592 151.41591 15141.59142 2.91591 299.91591 151.41590999999988 15141.59100 2020-01-01 2020-01-02 2020-01-01 00:16:11 2020-01-02 03:44:32 2020-01-01 00:16:11.000 2020-01-02 03:44:32.000 971 99872 50421.5 5042150 971 99872 50421.5 5042150 -32597 32338 4546.3 454630 -128 127 -0.26 -26 +972 2 10962 99873 2.91891 299.91891 151.41891 15141.89189 2.91891 299.9189 151.41891 15141.89172 2.91891 299.91891 151.41891000000015 15141.89100 2020-01-01 2020-01-02 2020-01-01 00:16:12 2020-01-02 03:44:33 2020-01-01 00:16:12.000 2020-01-02 03:44:33.000 972 99873 50422.5 5042250 972 99873 50422.5 5042250 -32596 32339 4547.3 454730 -128 127 -1.82 -182 +973 2 10963 99874 2.92192 299.92192 151.42192 15142.19219 2.92192 299.92194 151.42192 15142.19232 2.92192 299.92192 151.42192000000006 15142.19200 2020-01-01 2020-01-02 2020-01-01 00:16:13 2020-01-02 03:44:34 2020-01-01 00:16:13.000 2020-01-02 03:44:34.000 973 99874 50423.5 5042350 973 99874 50423.5 5042350 -32595 32340 4548.3 454830 -128 123 -3.38 -338 974 2 10964 99875 2.92492 299.92492 151.42492 15142.49249 2.92492 299.92493 151.42492 15142.49265 2.92492 299.92492 151.42492 15142.49200 2020-01-01 2020-01-02 2020-01-01 00:16:14 2020-01-02 03:44:35 2020-01-01 00:16:14.000 2020-01-02 03:44:35.000 974 99875 50424.5 5042450 974 99875 50424.5 5042450 -32594 32341 4549.3 454930 -127 124 -2.38 -238 -975 2 10965 99876 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792 15142.79200 2020-01-01 2020-01-02 2020-01-01 00:16:15 2020-01-02 03:44:36 2020-01-01 00:16:15.000 2020-01-02 03:44:36.000 975 99876 50425.5 5042550 975 99876 50425.5 5042550 -32593 32342 4550.3 455030 -126 125 -1.38 -138 -976 2 10966 99877 2.93093 299.93093 151.43093 15143.09309 2.93093 299.93094 151.43092 15143.09289 2.93093 299.93093 151.43093 15143.09300 2020-01-01 2020-01-02 2020-01-01 00:16:16 2020-01-02 03:44:37 2020-01-01 00:16:16.000 2020-01-02 03:44:37.000 976 99877 50426.5 5042650 976 99877 50426.5 5042650 -32592 32343 4551.3 455130 -125 126 -0.38 -38 -977 2 10967 99878 2.93393 299.93393 151.43393 15143.39339 2.93393 299.93393 151.43393 15143.39318 2.93393 299.93393 151.43393 15143.39300 2020-01-01 2020-01-02 2020-01-01 00:16:17 2020-01-02 03:44:38 2020-01-01 00:16:17.000 2020-01-02 03:44:38.000 977 99878 50427.5 5042750 977 99878 50427.5 5042750 -32591 32344 4552.3 455230 -124 127 0.62 62 -978 2 10968 99879 2.93693 299.93693 151.43693 15143.69369 2.93693 299.93695 151.43693 15143.69378 2.93693 299.93693 151.43693 15143.69300 2020-01-01 2020-01-02 2020-01-01 00:16:18 2020-01-02 03:44:39 2020-01-01 00:16:18.000 2020-01-02 03:44:39.000 978 99879 50428.5 5042850 978 99879 50428.5 5042850 -32590 32345 4553.3 455330 -128 127 -0.94 -94 -979 2 10969 99880 2.93993 299.93993 151.43993 15143.99399 2.93994 299.93994 151.43994 15143.99412 2.93993 299.93993 151.43993 15143.99300 2020-01-01 2020-01-02 2020-01-01 00:16:19 2020-01-02 03:44:40 2020-01-01 00:16:19.000 2020-01-02 03:44:40.000 979 99880 50429.5 5042950 979 99880 50429.5 5042950 -32589 32346 4554.3 455430 -128 123 -2.5 -250 -98 2 10088 99998 0.29429 300.29429 150.29429 15179.72372 0.29429 300.29428 150.29429 15179.72363 0.29429 300.29429 150.29429 15179.72329 2020-01-01 2020-01-02 2020-01-01 00:01:38 2020-01-02 03:46:38 2020-01-01 00:01:38.000 2020-01-02 03:46:38.000 98 99998 50048 5054848 98 99998 50048 5054848 -32471 32464 4627.009900990099 467328 -128 127 -1.2673267326732673 -128 -980 2 10970 99881 2.94294 299.94294 151.44294 15144.29429 2.94294 299.94293 151.44294 15144.29426 2.94294 299.94294 151.44294 15144.29400 2020-01-01 2020-01-02 2020-01-01 00:16:20 2020-01-02 03:44:41 2020-01-01 00:16:20.000 2020-01-02 03:44:41.000 980 99881 50430.5 5043050 980 99881 50430.5 5043050 -32588 32347 4555.3 455530 -127 124 -1.5 -150 -981 2 10971 99882 2.94594 299.94594 151.44594 15144.59459 2.94594 299.94595 151.44595 15144.59501 2.94594 299.94594 151.44594 15144.59400 2020-01-01 2020-01-02 2020-01-01 00:16:21 2020-01-02 03:44:42 2020-01-01 00:16:21.000 2020-01-02 03:44:42.000 981 99882 50431.5 5043150 981 99882 50431.5 5043150 -32587 32348 4556.3 455630 -126 125 -0.5 -50 -982 2 10972 99883 2.94894 299.94894 151.44894 15144.89489 2.94894 299.94894 151.44894 15144.89466 2.94894 299.94894 151.44894 15144.89400 2020-01-01 2020-01-02 2020-01-01 00:16:22 2020-01-02 03:44:43 2020-01-01 00:16:22.000 2020-01-02 03:44:43.000 982 99883 50432.5 5043250 982 99883 50432.5 5043250 -32586 32349 4557.3 455730 -125 126 0.5 50 -983 2 10973 99884 2.95195 299.95195 151.45195 15145.19519 2.95195 299.95197 151.45195 15145.19525 2.95195 299.95195 151.45195 15145.19500 2020-01-01 2020-01-02 2020-01-01 00:16:23 2020-01-02 03:44:44 2020-01-01 00:16:23.000 2020-01-02 03:44:44.000 983 99884 50433.5 5043350 983 99884 50433.5 5043350 -32585 32350 4558.3 455830 -124 127 1.5 150 -984 2 10974 99885 2.95495 299.95495 151.45495 15145.49549 2.95495 299.95496 151.45495 15145.49559 2.95495 299.95495 151.45495 15145.49500 2020-01-01 2020-01-02 2020-01-01 00:16:24 2020-01-02 03:44:45 2020-01-01 00:16:24.000 2020-01-02 03:44:45.000 984 99885 50434.5 5043450 984 99885 50434.5 5043450 -32584 32351 4559.3 455930 -128 127 -0.06 -6 -985 2 10975 99886 2.95795 299.95795 151.45795 15145.79579 2.95795 299.95795 151.45795 15145.79573 2.95795 299.95795 151.45795 15145.79500 2020-01-01 2020-01-02 2020-01-01 00:16:25 2020-01-02 03:44:46 2020-01-01 00:16:25.000 2020-01-02 03:44:46.000 985 99886 50435.5 5043550 985 99886 50435.5 5043550 -32583 32352 4560.3 456030 -128 123 -1.62 -162 -986 2 10976 99887 2.96096 299.96096 151.46096 15146.09609 2.96096 299.96097 151.46096 15146.09648 2.96096 299.96096 151.46096 15146.09600 2020-01-01 2020-01-02 2020-01-01 00:16:26 2020-01-02 03:44:47 2020-01-01 00:16:26.000 2020-01-02 03:44:47.000 986 99887 50436.5 5043650 986 99887 50436.5 5043650 -32582 32353 4561.3 456130 -127 124 -0.62 -62 -987 2 10977 99888 2.96396 299.96396 151.46396 15146.39639 2.96396 299.96396 151.46396 15146.39612 2.96396 299.96396 151.46396 15146.39600 2020-01-01 2020-01-02 2020-01-01 00:16:27 2020-01-02 03:44:48 2020-01-01 00:16:27.000 2020-01-02 03:44:48.000 987 99888 50437.5 5043750 987 99888 50437.5 5043750 -32581 32354 4562.3 456230 -126 125 0.38 38 -988 2 10978 99889 2.96696 299.96696 151.46696 15146.69669 2.96696 299.96698 151.46696 15146.69676 2.96696 299.96696 151.46696 15146.69600 2020-01-01 2020-01-02 2020-01-01 00:16:28 2020-01-02 03:44:49 2020-01-01 00:16:28.000 2020-01-02 03:44:49.000 988 99889 50438.5 5043850 988 99889 50438.5 5043850 -32580 32355 4563.3 456330 -125 126 1.38 138 -989 2 10979 99890 2.96996 299.96996 151.46996 15146.99699 2.96997 299.96997 151.46997 15146.99706 2.96996 299.96996 151.46996 15146.99600 2020-01-01 2020-01-02 2020-01-01 00:16:29 2020-01-02 03:44:50 2020-01-01 00:16:29.000 2020-01-02 03:44:50.000 989 99890 50439.5 5043950 989 99890 50439.5 5043950 -32579 32356 4564.3 456430 -124 127 2.38 238 -99 2 10089 99999 0.29729 300.29729 150.29729 15180.02702 0.29729 300.2973 150.29729 15180.02726 0.29729 300.29729 150.29729 15180.02629 2020-01-01 2020-01-02 2020-01-01 00:01:39 2020-01-02 03:46:39 2020-01-01 00:01:39.000 2020-01-02 03:46:39.000 99 99999 50049 5054949 99 99999 50049 5054949 -32470 32465 4628.009900990099 467429 -128 123 -2.801980198019802 -283 -990 2 10980 99891 2.97297 299.97297 151.47297 15147.29729 2.97297 299.97296 151.47297 15147.29735 2.97297 299.97297 151.47297 15147.29700 2020-01-01 2020-01-02 2020-01-01 00:16:30 2020-01-02 03:44:51 2020-01-01 00:16:30.000 2020-01-02 03:44:51.000 990 99891 50440.5 5044050 990 99891 50440.5 5044050 -32578 32357 4565.3 456530 -128 127 0.82 82 +975 2 10965 99876 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792000000006 15142.79200 2020-01-01 2020-01-02 2020-01-01 00:16:15 2020-01-02 03:44:36 2020-01-01 00:16:15.000 2020-01-02 03:44:36.000 975 99876 50425.5 5042550 975 99876 50425.5 5042550 -32593 32342 4550.3 455030 -126 125 -1.38 -138 +976 2 10966 99877 2.93093 299.93093 151.43093 15143.09309 2.93093 299.93094 151.43092 15143.09289 2.93093 299.93093 151.43093000000022 15143.09300 2020-01-01 2020-01-02 2020-01-01 00:16:16 2020-01-02 03:44:37 2020-01-01 00:16:16.000 2020-01-02 03:44:37.000 976 99877 50426.5 5042650 976 99877 50426.5 5042650 -32592 32343 4551.3 455130 -125 126 -0.38 -38 +977 2 10967 99878 2.93393 299.93393 151.43393 15143.39339 2.93393 299.93393 151.43393 15143.39318 2.93393 299.93393 151.4339299999999 15143.39300 2020-01-01 2020-01-02 2020-01-01 00:16:17 2020-01-02 03:44:38 2020-01-01 00:16:17.000 2020-01-02 03:44:38.000 977 99878 50427.5 5042750 977 99878 50427.5 5042750 -32591 32344 4552.3 455230 -124 127 0.62 62 +978 2 10968 99879 2.93693 299.93693 151.43693 15143.69369 2.93693 299.93695 151.43693 15143.69378 2.93693 299.93693 151.4369299999999 15143.69300 2020-01-01 2020-01-02 2020-01-01 00:16:18 2020-01-02 03:44:39 2020-01-01 00:16:18.000 2020-01-02 03:44:39.000 978 99879 50428.5 5042850 978 99879 50428.5 5042850 -32590 32345 4553.3 455330 -128 127 -0.94 -94 +979 2 10969 99880 2.93993 299.93993 151.43993 15143.99399 2.93994 299.93994 151.43994 15143.99412 2.93993 299.93993 151.43993000000017 15143.99300 2020-01-01 2020-01-02 2020-01-01 00:16:19 2020-01-02 03:44:40 2020-01-01 00:16:19.000 2020-01-02 03:44:40.000 979 99880 50429.5 5042950 979 99880 50429.5 5042950 -32589 32346 4554.3 455430 -128 123 -2.5 -250 +98 2 10088 99998 0.29429 300.29429 150.29429 15179.72372 0.29429 300.29428 150.29429 15179.72363 0.29429 300.29429 150.29428999999988 15179.72329 2020-01-01 2020-01-02 2020-01-01 00:01:38 2020-01-02 03:46:38 2020-01-01 00:01:38.000 2020-01-02 03:46:38.000 98 99998 50048 5054848 98 99998 50048 5054848 -32471 32464 4627.009900990099 467328 -128 127 -1.2673267326732673 -128 +980 2 10970 99881 2.94294 299.94294 151.44294 15144.29429 2.94294 299.94293 151.44294 15144.29426 2.94294 299.94294 151.44294000000014 15144.29400 2020-01-01 2020-01-02 2020-01-01 00:16:20 2020-01-02 03:44:41 2020-01-01 00:16:20.000 2020-01-02 03:44:41.000 980 99881 50430.5 5043050 980 99881 50430.5 5043050 -32588 32347 4555.3 455530 -127 124 -1.5 -150 +981 2 10971 99882 2.94594 299.94594 151.44594 15144.59459 2.94594 299.94595 151.44595 15144.59501 2.94594 299.94594 151.4459399999998 15144.59400 2020-01-01 2020-01-02 2020-01-01 00:16:21 2020-01-02 03:44:42 2020-01-01 00:16:21.000 2020-01-02 03:44:42.000 981 99882 50431.5 5043150 981 99882 50431.5 5043150 -32587 32348 4556.3 455630 -126 125 -0.5 -50 +982 2 10972 99883 2.94894 299.94894 151.44894 15144.89489 2.94894 299.94894 151.44894 15144.89466 2.94894 299.94894 151.44894000000014 15144.89400 2020-01-01 2020-01-02 2020-01-01 00:16:22 2020-01-02 03:44:43 2020-01-01 00:16:22.000 2020-01-02 03:44:43.000 982 99883 50432.5 5043250 982 99883 50432.5 5043250 -32586 32349 4557.3 455730 -125 126 0.5 50 +983 2 10973 99884 2.95195 299.95195 151.45195 15145.19519 2.95195 299.95197 151.45195 15145.19525 2.95195 299.95195 151.45195000000004 15145.19500 2020-01-01 2020-01-02 2020-01-01 00:16:23 2020-01-02 03:44:44 2020-01-01 00:16:23.000 2020-01-02 03:44:44.000 983 99884 50433.5 5043350 983 99884 50433.5 5043350 -32585 32350 4558.3 455830 -124 127 1.5 150 +984 2 10974 99885 2.95495 299.95495 151.45495 15145.49549 2.95495 299.95496 151.45495 15145.49559 2.95495 299.95495 151.4549499999999 15145.49500 2020-01-01 2020-01-02 2020-01-01 00:16:24 2020-01-02 03:44:45 2020-01-01 00:16:24.000 2020-01-02 03:44:45.000 984 99885 50434.5 5043450 984 99885 50434.5 5043450 -32584 32351 4559.3 455930 -128 127 -0.06 -6 +985 2 10975 99886 2.95795 299.95795 151.45795 15145.79579 2.95795 299.95795 151.45795 15145.79573 2.95795 299.95795 151.45794999999998 15145.79500 2020-01-01 2020-01-02 2020-01-01 00:16:25 2020-01-02 03:44:46 2020-01-01 00:16:25.000 2020-01-02 03:44:46.000 985 99886 50435.5 5043550 985 99886 50435.5 5043550 -32583 32352 4560.3 456030 -128 123 -1.62 -162 +986 2 10976 99887 2.96096 299.96096 151.46096 15146.09609 2.96096 299.96097 151.46096 15146.09648 2.96096 299.96096 151.46096000000014 15146.09600 2020-01-01 2020-01-02 2020-01-01 00:16:26 2020-01-02 03:44:47 2020-01-01 00:16:26.000 2020-01-02 03:44:47.000 986 99887 50436.5 5043650 986 99887 50436.5 5043650 -32582 32353 4561.3 456130 -127 124 -0.62 -62 +987 2 10977 99888 2.96396 299.96396 151.46396 15146.39639 2.96396 299.96396 151.46396 15146.39612 2.96396 299.96396 151.46395999999982 15146.39600 2020-01-01 2020-01-02 2020-01-01 00:16:27 2020-01-02 03:44:48 2020-01-01 00:16:27.000 2020-01-02 03:44:48.000 987 99888 50437.5 5043750 987 99888 50437.5 5043750 -32581 32354 4562.3 456230 -126 125 0.38 38 +988 2 10978 99889 2.96696 299.96696 151.46696 15146.69669 2.96696 299.96698 151.46696 15146.69676 2.96696 299.96696 151.46695999999986 15146.69600 2020-01-01 2020-01-02 2020-01-01 00:16:28 2020-01-02 03:44:49 2020-01-01 00:16:28.000 2020-01-02 03:44:49.000 988 99889 50438.5 5043850 988 99889 50438.5 5043850 -32580 32355 4563.3 456330 -125 126 1.38 138 +989 2 10979 99890 2.96996 299.96996 151.46996 15146.99699 2.96997 299.96997 151.46997 15146.99706 2.96996 299.96996 151.46996000000016 15146.99600 2020-01-01 2020-01-02 2020-01-01 00:16:29 2020-01-02 03:44:50 2020-01-01 00:16:29.000 2020-01-02 03:44:50.000 989 99890 50439.5 5043950 989 99890 50439.5 5043950 -32579 32356 4564.3 456430 -124 127 2.38 238 +99 2 10089 99999 0.29729 300.29729 150.29729 15180.02702 0.29729 300.2973 150.29729 15180.02726 0.29729 300.29729 150.29729000000023 15180.02629 2020-01-01 2020-01-02 2020-01-01 00:01:39 2020-01-02 03:46:39 2020-01-01 00:01:39.000 2020-01-02 03:46:39.000 99 99999 50049 5054949 99 99999 50049 5054949 -32470 32465 4628.009900990099 467429 -128 123 -2.801980198019802 -283 +990 2 10980 99891 2.97297 299.97297 151.47297 15147.29729 2.97297 299.97296 151.47297 15147.29735 2.97297 299.97297 151.4729700000001 15147.29700 2020-01-01 2020-01-02 2020-01-01 00:16:30 2020-01-02 03:44:51 2020-01-01 00:16:30.000 2020-01-02 03:44:51.000 990 99891 50440.5 5044050 990 99891 50440.5 5044050 -32578 32357 4565.3 456530 -128 127 0.82 82 991 2 10981 99892 2.97597 299.97597 151.47597 15147.59759 2.97597 299.97598 151.47597 15147.59795 2.97597 299.97597 151.47597 15147.59700 2020-01-01 2020-01-02 2020-01-01 00:16:31 2020-01-02 03:44:52 2020-01-01 00:16:31.000 2020-01-02 03:44:52.000 991 99892 50441.5 5044150 991 99892 50441.5 5044150 -32577 32358 4566.3 456630 -128 127 -0.74 -74 -992 2 10982 99893 2.97897 299.97897 151.47897 15147.89789 2.97897 299.97897 151.47897 15147.89759 2.97897 299.97897 151.47897 15147.89700 2020-01-01 2020-01-02 2020-01-01 00:16:32 2020-01-02 03:44:53 2020-01-01 00:16:32.000 2020-01-02 03:44:53.000 992 99893 50442.5 5044250 992 99893 50442.5 5044250 -32576 32359 4567.3 456730 -128 124 -2.3 -230 +992 2 10982 99893 2.97897 299.97897 151.47897 15147.89789 2.97897 299.97897 151.47897 15147.89759 2.97897 299.97897 151.4789700000001 15147.89700 2020-01-01 2020-01-02 2020-01-01 00:16:32 2020-01-02 03:44:53 2020-01-01 00:16:32.000 2020-01-02 03:44:53.000 992 99893 50442.5 5044250 992 99893 50442.5 5044250 -32576 32359 4567.3 456730 -128 124 -2.3 -230 993 2 10983 99894 2.98198 299.98198 151.48198 15148.19819 2.98198 299.982 151.48198 15148.19823 2.98198 299.98198 151.48198 15148.19800 2020-01-01 2020-01-02 2020-01-01 00:16:33 2020-01-02 03:44:54 2020-01-01 00:16:33.000 2020-01-02 03:44:54.000 993 99894 50443.5 5044350 993 99894 50443.5 5044350 -32575 32360 4568.3 456830 -127 125 -1.3 -130 -994 2 10984 99895 2.98498 299.98498 151.48498 15148.49849 2.98498 299.985 151.48498 15148.49853 2.98498 299.98498 151.48498 15148.49800 2020-01-01 2020-01-02 2020-01-01 00:16:34 2020-01-02 03:44:55 2020-01-01 00:16:34.000 2020-01-02 03:44:55.000 994 99895 50444.5 5044450 994 99895 50444.5 5044450 -32574 32361 4569.3 456930 -126 126 -0.3 -30 -995 2 10985 99896 2.98798 299.98798 151.48798 15148.79879 2.98798 299.98798 151.48798 15148.79882 2.98798 299.98798 151.48798 15148.79800 2020-01-01 2020-01-02 2020-01-01 00:16:35 2020-01-02 03:44:56 2020-01-01 00:16:35.000 2020-01-02 03:44:56.000 995 99896 50445.5 5044550 995 99896 50445.5 5044550 -32573 32362 4570.3 457030 -125 127 0.7 70 -996 2 10986 99897 2.99099 299.99099 151.49099 15149.09909 2.99099 299.991 151.49099 15149.09942 2.99099 299.99099 151.49099 15149.09900 2020-01-01 2020-01-02 2020-01-01 00:16:36 2020-01-02 03:44:57 2020-01-01 00:16:36.000 2020-01-02 03:44:57.000 996 99897 50446.5 5044650 996 99897 50446.5 5044650 -32572 32363 4571.3 457130 -128 127 -0.86 -86 -997 2 10987 99898 2.99399 299.99399 151.49399 15149.39939 2.99399 299.994 151.49399 15149.3991 2.99399 299.99399 151.49399 15149.39900 2020-01-01 2020-01-02 2020-01-01 00:16:37 2020-01-02 03:44:58 2020-01-01 00:16:37.000 2020-01-02 03:44:58.000 997 99898 50447.5 5044750 997 99898 50447.5 5044750 -32571 32364 4572.3 457230 -128 127 -2.42 -242 -998 2 10988 99899 2.99699 299.99699 151.49699 15149.69969 2.99699 299.997 151.49699 15149.6997 2.99699 299.99699 151.49699 15149.69900 2020-01-01 2020-01-02 2020-01-01 00:16:38 2020-01-02 03:44:59 2020-01-01 00:16:38.000 2020-01-02 03:44:59.000 998 99899 50448.5 5044850 998 99899 50448.5 5044850 -32570 32365 4573.3 457330 -128 123 -3.98 -398 +994 2 10984 99895 2.98498 299.98498 151.48498 15148.49849 2.98498 299.985 151.48498 15148.49853 2.98498 299.98498 151.48497999999995 15148.49800 2020-01-01 2020-01-02 2020-01-01 00:16:34 2020-01-02 03:44:55 2020-01-01 00:16:34.000 2020-01-02 03:44:55.000 994 99895 50444.5 5044450 994 99895 50444.5 5044450 -32574 32361 4569.3 456930 -126 126 -0.3 -30 +995 2 10985 99896 2.98798 299.98798 151.48798 15148.79879 2.98798 299.98798 151.48798 15148.79882 2.98798 299.98798 151.4879799999999 15148.79800 2020-01-01 2020-01-02 2020-01-01 00:16:35 2020-01-02 03:44:56 2020-01-01 00:16:35.000 2020-01-02 03:44:56.000 995 99896 50445.5 5044550 995 99896 50445.5 5044550 -32573 32362 4570.3 457030 -125 127 0.7 70 +996 2 10986 99897 2.99099 299.99099 151.49099 15149.09909 2.99099 299.991 151.49099 15149.09942 2.99099 299.99099 151.49099000000015 15149.09900 2020-01-01 2020-01-02 2020-01-01 00:16:36 2020-01-02 03:44:57 2020-01-01 00:16:36.000 2020-01-02 03:44:57.000 996 99897 50446.5 5044650 996 99897 50446.5 5044650 -32572 32363 4571.3 457130 -128 127 -0.86 -86 +997 2 10987 99898 2.99399 299.99399 151.49399 15149.39939 2.99399 299.994 151.49399 15149.3991 2.99399 299.99399 151.49399000000017 15149.39900 2020-01-01 2020-01-02 2020-01-01 00:16:37 2020-01-02 03:44:58 2020-01-01 00:16:37.000 2020-01-02 03:44:58.000 997 99898 50447.5 5044750 997 99898 50447.5 5044750 -32571 32364 4572.3 457230 -128 127 -2.42 -242 +998 2 10988 99899 2.99699 299.99699 151.49699 15149.69969 2.99699 299.997 151.49699 15149.6997 2.99699 299.99699 151.49698999999984 15149.69900 2020-01-01 2020-01-02 2020-01-01 00:16:38 2020-01-02 03:44:59 2020-01-01 00:16:38.000 2020-01-02 03:44:59.000 998 99899 50448.5 5044850 998 99899 50448.5 5044850 -32570 32365 4573.3 457330 -128 123 -3.98 -398 ---- select row with nulls without states ---- -2 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N ---- select row with nulls with states ---- From 6fb7e944dbf0194cfd526a90e43d53e9acd683d8 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 30 Oct 2020 21:10:02 +0300 Subject: [PATCH 0083/1088] a test for file engine settings --- src/Storages/StorageFile.cpp | 28 +++++++++---------- .../01544_file_engine_settings.reference | 2 ++ .../0_stateless/01544_file_engine_settings.sh | 23 +++++++++++++++ 3 files changed, 39 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/01544_file_engine_settings.reference create mode 100755 tests/queries/0_stateless/01544_file_engine_settings.sh diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index d175736e8f8..517eeb1e671 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -167,18 +167,18 @@ StorageFile::StorageFile(int table_fd_, CommonArguments args) StorageFile::StorageFile(const std::string & table_path_, const std::string & user_files_path, CommonArguments args) : StorageFile(args) { - fmt::print(stderr, "Create storage File '{}' from file at \n{}\n", - args.table_id.getNameForLogs(), StackTrace().toString()); - - const auto & changes = args.context.getSettings().changes(); - for (const auto & change : changes) - { - fmt::print(stderr, "Changed setting '{}' to '{}'\n", - change.name, toString(change.value)); - } - - fmt::print(stderr, "delimiter = '{}'\n", - toString(args.context.getSettings().get("format_csv_delimiter"))); +// fmt::print(stderr, "Create storage File '{}' from file at \n{}\n", +// args.table_id.getNameForLogs(), StackTrace().toString()); +// +// const auto & changes = args.context.getSettings().changes(); +// for (const auto & change : changes) +// { +// fmt::print(stderr, "Changed setting '{}' to '{}'\n", +// change.name, toString(change.value)); +// } +// +// fmt::print(stderr, "delimiter = '{}'\n", +// toString(args.context.getSettings().get("format_csv_delimiter"))); is_db_table = false; @@ -204,8 +204,8 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArguments args) : StorageFile(args) { - fmt::print(stderr, "Create storage File '{}' from database at \n{}\n", - args.table_id.getNameForLogs(), StackTrace().toString()); +// fmt::print(stderr, "Create storage File '{}' from database at \n{}\n", +// args.table_id.getNameForLogs(), StackTrace().toString()); if (relative_table_dir_path.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); diff --git a/tests/queries/0_stateless/01544_file_engine_settings.reference b/tests/queries/0_stateless/01544_file_engine_settings.reference new file mode 100644 index 00000000000..d2afb8fc688 --- /dev/null +++ b/tests/queries/0_stateless/01544_file_engine_settings.reference @@ -0,0 +1,2 @@ +1|1 +1 1 diff --git a/tests/queries/0_stateless/01544_file_engine_settings.sh b/tests/queries/0_stateless/01544_file_engine_settings.sh new file mode 100755 index 00000000000..b13ec0f3db3 --- /dev/null +++ b/tests/queries/0_stateless/01544_file_engine_settings.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash +set -eu + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +the_file="$CLICKHOUSE_TMP/01544-t.csv" +rm -f -- "$the_file" + +# We are going to check that format settings work for File engine, +# by creating a table with a non-default delimiter, and reading from it. +${CLICKHOUSE_LOCAL} --query " + create table t(a int, b int) engine File(CSV, '$the_file') settings format_csv_delimiter = '|'; + insert into t select 1 a, 1 b; +" + +# See what's in the file +cat "$the_file" + +${CLICKHOUSE_LOCAL} --query " + create table t(a int, b int) engine File(CSV, '$the_file') settings format_csv_delimiter = '|'; + select * from t; +" From 19fcea88f216c62b2a32f4acece33399d969fb42 Mon Sep 17 00:00:00 2001 From: myrrc Date: Fri, 30 Oct 2020 21:13:57 +0300 Subject: [PATCH 0084/1088] fixing gcc build err --- src/DataTypes/IDataType.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 797b05d2ef3..c32aba721f4 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -469,7 +469,9 @@ struct WhichDataType constexpr WhichDataType(TypeIndex idx_ = TypeIndex::Nothing) : idx(idx_) {} constexpr WhichDataType(const IDataType & data_type) : idx(data_type.getTypeId()) {} constexpr WhichDataType(const IDataType * data_type) : idx(data_type->getTypeId()) {} - constexpr WhichDataType(const DataTypePtr & data_type) : idx(data_type->getTypeId()) {} + + // shared ptr -> is non-constexpr in gcc + WhichDataType(const DataTypePtr & data_type) : idx(data_type->getTypeId()) {} constexpr bool isUInt8() const { return idx == TypeIndex::UInt8; } constexpr bool isUInt16() const { return idx == TypeIndex::UInt16; } From 1fc763328965d7cab2890f15b073b3824ca5c068 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 30 Oct 2020 22:57:30 +0300 Subject: [PATCH 0085/1088] Heartbeats working --- src/Common/ZooKeeper/TestKeeperStorage.cpp | 107 ++++++++++++++++++++- src/Common/ZooKeeper/TestKeeperStorage.h | 4 + src/Common/ZooKeeper/ZooKeeperCommon.h | 2 +- src/Server/TestKeeperTCPHandler.h | 2 + 4 files changed, 112 insertions(+), 3 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Common/ZooKeeper/TestKeeperStorage.cpp index c238fa2620f..96593b291d6 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.cpp +++ b/src/Common/ZooKeeper/TestKeeperStorage.cpp @@ -1,6 +1,109 @@ #include +#include -namespace DB +namespace zkutil { - +using namespace DB; + +static String parentPath(const String & path) +{ + auto rslash_pos = path.rfind('/'); + if (rslash_pos > 0) + return path.substr(0, rslash_pos); + return "/"; +} + +using Undo = std::function; + +struct TestKeeperStorageRequest +{ + Coordination::ZooKeeperRequestPtr zk_request; + + TestKeeperStorageRequest(const Coordination::ZooKeeperRequestPtr & zk_request_) + : zk_request(zk_request_) + {} + virtual bool isMutable() const { return false; } + virtual std::pair process(TestKeeperStorage::Container & container, int64_t zxid) const = 0; + virtual ~TestKeeperStorageRequest() {} +}; + + +struct TestKeeperStorageCreateRequest final : public TestKeeperStorageRequest +{ + std::pair process(TestKeeperStorage::Container & container, int64_t zxid) const override + { + Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Undo undo; + Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); + Coordination::ZooKeeperCreateRequest & request = dynamic_cast(*zk_request); + + if (container.count(request.path)) + { + response.error = Coordination::Error::ZNODEEXISTS; + } + else + { + auto it = container.find(parentPath(request.path)); + + if (it == container.end()) + { + response.error = Coordination::Error::ZNONODE; + } + else if (it->second.is_ephemeral) + { + response.error = Coordination::Error::ZNOCHILDRENFOREPHEMERALS; + } + else + { + TestKeeperStorage::Node created_node; + created_node.seq_num = 0; + created_node.stat.czxid = zxid; + created_node.stat.mzxid = zxid; + created_node.stat.ctime = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1); + created_node.stat.mtime = created_node.stat.ctime; + created_node.stat.numChildren = 0; + created_node.stat.dataLength = request.data.length(); + created_node.data = request.data; + created_node.is_ephemeral = request.is_ephemeral; + created_node.is_sequental = request.is_sequential; + std::string path_created = request.path; + + if (request.is_sequential) + { + auto seq_num = it->second.seq_num; + ++it->second.seq_num; + + std::stringstream seq_num_str; + seq_num_str << std::setw(10) << std::setfill('0') << seq_num; + + path_created += seq_num_str.str(); + } + + response.path_created = path_created; + container.emplace(path_created, std::move(created_node)); + + undo = [&container, path_created, is_sequential = request.is_sequential, parent_path = it->first] + { + container.erase(path_created); + auto & undo_parent = container.at(parent_path); + --undo_parent.stat.cversion; + --undo_parent.stat.numChildren; + + if (is_sequential) + --undo_parent.seq_num; + }; + + ++it->second.stat.cversion; + ++it->second.stat.numChildren; + + response.error = Coordination::Error::ZOK; + } + } + + return { response_ptr, undo }; + } +}; + + + } diff --git a/src/Common/ZooKeeper/TestKeeperStorage.h b/src/Common/ZooKeeper/TestKeeperStorage.h index f0c8a942dff..893f9f2842c 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.h +++ b/src/Common/ZooKeeper/TestKeeperStorage.h @@ -12,6 +12,8 @@ using namespace DB; class TestKeeperStorage { + +public: struct TestKeeperRequest; using TestKeeperRequestPtr = std::shared_ptr; @@ -61,6 +63,8 @@ class TestKeeperStorage void processingThread(); + void writeResponse(const Coordination::ZooKeeperResponsePtr & response); + public: void putRequest(const Coordination::ZooKeeperRequestPtr & request, std::shared_ptr response_out); diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 6293cbb09fe..05886fc0468 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -133,7 +133,7 @@ struct ZooKeeperCloseResponse final : ZooKeeperResponse OpNum getOpNum() const override { return -11; } }; -struct ZooKeeperCreateRequest final : CreateRequest, ZooKeeperRequest +struct ZooKeeperCreateRequest final : public CreateRequest, ZooKeeperRequest { ZooKeeperCreateRequest() = default; explicit ZooKeeperCreateRequest(const CreateRequest & base) : CreateRequest(base) {} diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h index 967ea1d29e7..695f5f04382 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/TestKeeperTCPHandler.h @@ -43,6 +43,8 @@ private: void receiveHeartbeatRequest(); void sendHeartbeatResponse(); + + void receiveCreateRequest(); }; } From 382fff9009eb001487f1f4f88b390d52717e46a8 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 1 Nov 2020 13:54:07 +0000 Subject: [PATCH 0086/1088] fix --- .../IInterpreterUnionOrSelectQuery.h | 39 ++ src/Interpreters/InterpreterSelectQuery.cpp | 11 +- src/Interpreters/InterpreterSelectQuery.h | 29 +- .../InterpreterSelectWithUnionQuery.cpp | 413 ++++++++---------- .../InterpreterSelectWithUnionQuery.h | 53 +-- src/Interpreters/executeQuery.cpp | 2 - src/Parsers/ASTSelectWithUnionQuery.cpp | 11 +- src/Parsers/ASTSelectWithUnionQuery.h | 3 - src/Parsers/IAST.cpp | 5 - src/Parsers/IAST.h | 2 - src/Parsers/ParserSelectWithUnionQuery.cpp | 35 +- 11 files changed, 238 insertions(+), 365 deletions(-) create mode 100644 src/Interpreters/IInterpreterUnionOrSelectQuery.h diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h new file mode 100644 index 00000000000..67e3cf25e5a --- /dev/null +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ +class IInterpreterUnionOrSelectQuery : public IInterpreter +{ +public: + IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, const Context & context_, const SelectQueryOptions & options_) + : query_ptr(query_ptr_) + , context(std::make_shared(context_)) + , options(options_) + , max_streams(context->getSettingsRef().max_threads) + { + } + + virtual void buildQueryPlan(QueryPlan & query_plan) = 0; + + virtual void ignoreWithTotals() = 0; + + virtual ~IInterpreterUnionOrSelectQuery() override = default; + + Block getSampleBlock() { return result_header; } + + size_t getMaxStreams() const { return max_streams; } + +protected: + ASTPtr query_ptr; + std::shared_ptr context; + Block result_header; + SelectQueryOptions options; + size_t max_streams = 1; +}; +} + diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d9821be4e4e..0b86914b1fa 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -119,7 +119,6 @@ String InterpreterSelectQuery::generateFilterActions( ParserExpression expr_parser; expr_list->children.push_back(parseQuery(expr_parser, column_str, 0, context->getSettingsRef().max_parser_depth)); } - select_ast->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared()); auto tables = select_ast->tables(); auto tables_elem = std::make_shared(); @@ -215,10 +214,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( const SelectQueryOptions & options_, const Names & required_result_column_names, const StorageMetadataPtr & metadata_snapshot_) - : options(options_) /// NOTE: the query almost always should be cloned because it will be modified during analysis. - , query_ptr(options.modify_inplace ? query_ptr_ : query_ptr_->clone()) - , context(std::make_shared(context_)) + : IInterpreterUnionOrSelectQuery(options_.modify_inplace ? query_ptr_ : query_ptr_->clone(), context_, options_) , storage(storage_) , input(input_) , input_pipe(std::move(input_pipe_)) @@ -464,12 +461,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( sanitizeBlock(result_header, true); } - -Block InterpreterSelectQuery::getSampleBlock() -{ - return result_header; -} - void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) { executeImpl(query_plan, input, std::move(input_pipe)); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 455b1a1e623..6bbcb9a6bcd 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -3,16 +3,15 @@ #include #include -#include #include #include #include -#include -#include +#include +#include +#include +#include #include #include -#include -#include #include @@ -32,7 +31,7 @@ using TreeRewriterResultPtr = std::shared_ptr; /** Interprets the SELECT query. Returns the stream of blocks with the results of the query before `to_stage` stage. */ -class InterpreterSelectQuery : public IInterpreter +class InterpreterSelectQuery : public IInterpreterUnionOrSelectQuery { public: /** @@ -79,18 +78,12 @@ public: BlockIO execute() override; /// Builds QueryPlan for current query. - void buildQueryPlan(QueryPlan & query_plan); + virtual void buildQueryPlan(QueryPlan & query_plan) override; bool ignoreLimits() const override { return options.ignore_limits; } bool ignoreQuota() const override { return options.ignore_quota; } - Block getSampleBlock(); - - void ignoreWithTotals(); - - ASTPtr getQuery() const { return query_ptr; } - - size_t getMaxStreams() const { return max_streams; } + virtual void ignoreWithTotals() override; const SelectQueryInfo & getQueryInfo() const { return query_info; } @@ -158,9 +151,6 @@ private: */ void initSettings(); - SelectQueryOptions options; - ASTPtr query_ptr; - std::shared_ptr context; TreeRewriterResultPtr syntax_analyzer_result; std::unique_ptr query_analyzer; SelectQueryInfo query_info; @@ -172,15 +162,10 @@ private: QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; - /// How many streams we ask for storage to produce, and in how many threads we will do further processing. - size_t max_streams = 1; - /// List of columns to read to execute the query. Names required_columns; /// Structure of query source (table, subquery, etc). Block source_header; - /// Structure of query result. - Block result_header; /// The subquery interpreter, if the subquery std::unique_ptr interpreter_subquery; diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 92f88342241..43ff1c48167 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -10,8 +10,6 @@ #include #include -#include - namespace DB { @@ -22,206 +20,40 @@ namespace ErrorCodes extern const int EXPECTED_ALL_OR_DISTINCT; } -struct CustomizeUnionModeRewrite -{ - using TypeToVisit = ASTSelectWithUnionQuery; - - const UnionMode & default_union_mode; - - void visit(ASTSelectWithUnionQuery & union_select, ASTPtr &) - { - size_t num_selects = union_select.list_of_selects->children.size(); - if (!num_selects) - throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR); - if (num_selects > 1) - { - for (auto & mode : union_select.union_modes) - { - if (mode == ASTSelectWithUnionQuery::Mode::Unspecified) - { - if (default_union_mode == UnionMode::ALL) - mode = ASTSelectWithUnionQuery::Mode::ALL; - else if (default_union_mode == UnionMode::DISTINCT) - mode = ASTSelectWithUnionQuery::Mode::DISTINCT; - else - throw Exception( - "Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty", - DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); - } - } - /// Optimize: if there is UNION DISTINCT, all previous UNION DISTINCT can be rewritten to UNION ALL. - /// Therefore we have at most one UNION DISTINCT in a sequence. - for (auto rit = union_select.union_modes.rbegin(); rit != union_select.union_modes.rend(); ++rit) - { - if (*rit == ASTSelectWithUnionQuery::Mode::DISTINCT) - { - /// Number of streams need to do a DISTINCT transform after unite - for (auto mode_to_modify = ++rit; mode_to_modify != union_select.union_modes.rend(); ++mode_to_modify) - *mode_to_modify = ASTSelectWithUnionQuery::Mode::ALL; - break; - } - } - } - } -}; - -using CustomizeUnionQueryOptimizeVisitor = InDepthNodeVisitor, true>; - -QueryPlan NestedInterpreter::buildQueryPlan(const std::shared_ptr & context, const Block & header) -{ - QueryPlan res; - if (type == Type::LEAF) - { - if (interpreter) - { - interpreter->buildQueryPlan(res); - return res; - } - else - throw Exception("Interpreter is not initialized.", ErrorCodes::LOGICAL_ERROR); - } - - if (num_distinct_union == 0) - { - std::vector> plans(children.size()); - DataStreams data_streams(children.size()); - - for (size_t i = 0; i < children.size(); ++i) - { - plans[i] = std::make_unique(children[i]->buildQueryPlan(context, header)); - data_streams[i] = plans[i]->getCurrentDataStream(); - } - - auto max_threads = context->getSettingsRef().max_threads; - auto union_step = std::make_unique(std::move(data_streams), header, max_threads); - - res.unitePlans(std::move(union_step), std::move(plans)); - return res; - } - /// The first union_distinct_num UNION streams need to do a DISTINCT transform after unite - else - { - QueryPlan distinct_query_plan; - - std::vector> plans(num_distinct_union); - DataStreams data_streams(num_distinct_union); - - for (size_t i = 0; i < num_distinct_union; ++i) - { - plans[i] = std::make_unique(children[i]->buildQueryPlan(context, header)); - data_streams[i] = plans[i]->getCurrentDataStream(); - } - - auto max_threads = context->getSettingsRef().max_threads; - auto union_step = std::make_unique(std::move(data_streams), header, max_threads); - - distinct_query_plan.unitePlans(std::move(union_step), std::move(plans)); - - /// Add distinct transform - const Settings & settings = context->getSettingsRef(); - SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); - - auto distinct_step - = std::make_unique(distinct_query_plan.getCurrentDataStream(), limits, 0, header.getNames(), false); - - distinct_query_plan.addStep(std::move(distinct_step)); - - /// No other UNION streams after DISTINCT stream - if (num_distinct_union == children.size()) - { - return distinct_query_plan; - } - - /// Build final UNION step - std::vector> final_plans(children.size() - num_distinct_union + 1); - DataStreams final_data_streams(children.size() - num_distinct_union + 1); - - final_plans[0] = std::make_unique(std::move(distinct_query_plan)); - final_data_streams[0] = final_plans[0]->getCurrentDataStream(); - - for (size_t i = 1; i < children.size() - num_distinct_union + 1; ++i) - { - final_plans[i] = std::make_unique(children[num_distinct_union + i - 1]->buildQueryPlan(context, header)); - final_data_streams[i] = final_plans[i]->getCurrentDataStream(); - } - - auto final_union_step = std::make_unique(std::move(final_data_streams), header, max_threads); - res.unitePlans(std::move(final_union_step), std::move(final_plans)); - return res; - } -} - -void NestedInterpreter::ignoreWithTotals() -{ - if (type == Type::LEAF) - { - if (interpreter) - interpreter->ignoreWithTotals(); - else - { - throw Exception("Interpreter is not initialized.", ErrorCodes::LOGICAL_ERROR); - } - return; - } - for (auto & child : children) - { - child->ignoreWithTotals(); - } -} - - InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( - const ASTPtr & query_ptr_, - const Context & context_, - const SelectQueryOptions & options_, - const Names & required_result_column_names) - : options(options_), - query_ptr(query_ptr_), - context(std::make_shared(context_)), - max_streams(context->getSettingsRef().max_threads) + const ASTPtr & query_ptr_, const Context & context_, const SelectQueryOptions & options_, const Names & required_result_column_names) + : IInterpreterUnionOrSelectQuery(query_ptr_, context_, options_) { - std::cout << "\n\n In InterpreterSelectWithUnionQuery\n\n"; const auto & ast = query_ptr->as(); - std::cout << "\n\n before throw\n\n"; - if (!ast.flatten_nodes_list) - std::cout << "\n\n flatten_nodes_list is null\n\n"; - size_t total_num_selects = ast.flatten_nodes_list->children.size(); - std::cout << "\n\n after get num throw\n\n"; - if (!total_num_selects) - throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR); - std::cout << "\n\n after throw\n\n"; - /// Rewrite ast with settings.union_default_mode - const auto & settings = context->getSettingsRef(); - CustomizeUnionQueryOptimizeVisitor::Data data_union_mode{settings.union_default_mode}; - CustomizeUnionQueryOptimizeVisitor(data_union_mode).visit(query_ptr); + size_t num_children = ast.list_of_selects->children.size(); + if (!num_children) + throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR); /// We first build nested interpreters for each select query, then using this nested interpreters to build Tree Structured nested interpreter. /// Note that we pass 'required_result_column_names' to first SELECT. /// And for the rest, we pass names at the corresponding positions of 'required_result_column_names' in the result of first SELECT, /// because names could be different. - std::vector> interpreters; - interpreters.reserve(total_num_selects); - std::vector required_result_column_names_for_other_selects(total_num_selects); - if (!required_result_column_names.empty() && total_num_selects > 1) + nested_interpreters.reserve(num_children); + std::vector required_result_column_names_for_other_selects(num_children); + + if (!required_result_column_names.empty() && num_children > 1) { /// Result header if there are no filtering by 'required_result_column_names'. /// We use it to determine positions of 'required_result_column_names' in SELECT clause. - Block full_result_header - = InterpreterSelectQuery(ast.flatten_nodes_list->children.at(0), *context, options.copy().analyze().noModify()) - .getSampleBlock(); + Block full_result_header = getCurrentChildResultHeader(ast.list_of_selects->children.at(0), required_result_column_names); std::vector positions_of_required_result_columns(required_result_column_names.size()); + for (size_t required_result_num = 0, size = required_result_column_names.size(); required_result_num < size; ++required_result_num) positions_of_required_result_columns[required_result_num] = full_result_header.getPositionByName(required_result_column_names[required_result_num]); - for (size_t query_num = 1; query_num < total_num_selects; ++query_num) + for (size_t query_num = 1; query_num < num_children; ++query_num) { Block full_result_header_for_current_select - = InterpreterSelectQuery(ast.flatten_nodes_list->children.at(query_num), *context, options.copy().analyze().noModify()) - .getSampleBlock(); + = getCurrentChildResultHeader(ast.list_of_selects->children.at(query_num), required_result_column_names); if (full_result_header_for_current_select.columns() != full_result_header.columns()) throw Exception("Different number of columns in UNION ALL elements:\n" @@ -236,26 +68,26 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( } } - for (size_t query_num = 0; query_num < total_num_selects; ++query_num) + for (size_t query_num = 0; query_num < num_children; ++query_num) { const Names & current_required_result_column_names = query_num == 0 ? required_result_column_names : required_result_column_names_for_other_selects[query_num]; - interpreters.emplace_back(std::make_shared( - ast.flatten_nodes_list->children.at(query_num), *context, options, current_required_result_column_names)); + nested_interpreters.emplace_back( + buildCurrentChildInterpreter(ast.list_of_selects->children.at(query_num), current_required_result_column_names)); } /// Determine structure of the result. - if (total_num_selects == 1) + if (num_children == 1) { - result_header = interpreters.front()->getSampleBlock(); + result_header = nested_interpreters.front()->getSampleBlock(); } else { - Blocks headers(total_num_selects); - for (size_t query_num = 0; query_num < total_num_selects; ++query_num) - headers[query_num] = interpreters[query_num]->getSampleBlock(); + Blocks headers(num_children); + for (size_t query_num = 0; query_num < num_children; ++query_num) + headers[query_num] = nested_interpreters[query_num]->getSampleBlock(); result_header = getCommonHeaderForUnion(headers); } @@ -263,7 +95,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( /// InterpreterSelectWithUnionQuery ignores limits if all nested interpreters ignore limits. bool all_nested_ignore_limits = true; bool all_nested_ignore_quota = true; - for (auto & interpreter : interpreters) + for (auto & interpreter : nested_interpreters) { if (!interpreter->ignoreLimits()) all_nested_ignore_limits = false; @@ -273,44 +105,6 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( options.ignore_limits |= all_nested_ignore_limits; options.ignore_quota |= all_nested_ignore_quota; - int index = 0; - buildNestedTreeInterpreter(query_ptr, nested_interpreter, interpreters, index); -} - -/// We build a Tree Structured nested interpreters to build QueryPlan later -/// The structure of build nested interpreters is same as AST Tree -void InterpreterSelectWithUnionQuery::buildNestedTreeInterpreter( - const ASTPtr & ast_ptr, - std::shared_ptr nested_interpreter_, - std::vector> & interpreters, - int & index) -{ - std::cout << "\n\n in build \n\n"; - if (auto inner_union = ast_ptr->as()) - { - auto internal_intepreter = std::make_shared(); - const auto & union_modes = inner_union->union_modes; - - for (auto rit = union_modes.rbegin(); rit != union_modes.rend(); ++rit) - { - if (*rit == ASTSelectWithUnionQuery::Mode::DISTINCT) - { - internal_intepreter->num_distinct_union = union_modes.rend() - rit + 1; - break; - } - } - - nested_interpreter_->children.push_back(internal_intepreter); - - for (auto & child : inner_union->list_of_selects->children) - buildNestedTreeInterpreter(child, internal_intepreter, interpreters, index); - return; - } - - auto leaf_interpreter = std::make_shared(); - leaf_interpreter->type = NestedInterpreter::Type::LEAF; - leaf_interpreter->interpreter = interpreters[index++]; - nested_interpreter_->children.push_back(leaf_interpreter); } Block InterpreterSelectWithUnionQuery::getCommonHeaderForUnion(const Blocks & headers) @@ -343,33 +137,167 @@ Block InterpreterSelectWithUnionQuery::getCommonHeaderForUnion(const Blocks & he return common_header; } +Block InterpreterSelectWithUnionQuery::getCurrentChildResultHeader(const ASTPtr & ast_ptr_, const Names & required_result_column_names) +{ + if (const auto _ = ast_ptr_->as()) + return InterpreterSelectWithUnionQuery(ast_ptr_, *context, options.copy().analyze().noModify(), required_result_column_names) + .getSampleBlock(); + else + return InterpreterSelectQuery(ast_ptr_, *context, options.copy().analyze().noModify()).getSampleBlock(); +} + +std::unique_ptr +InterpreterSelectWithUnionQuery::buildCurrentChildInterpreter(const ASTPtr & ast_ptr_, const Names & current_required_result_column_names) +{ + if (const auto _ = ast_ptr_->as()) + return std::make_unique(ast_ptr_, *context, options, current_required_result_column_names); + else + return std::make_unique(ast_ptr_, *context, options, current_required_result_column_names); +} InterpreterSelectWithUnionQuery::~InterpreterSelectWithUnionQuery() = default; - -Block InterpreterSelectWithUnionQuery::getSampleBlock() +Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_, const Context & context_) { - return result_header; -} - -Block InterpreterSelectWithUnionQuery::getSampleBlock( - const ASTPtr & query_ptr, - const Context & context) -{ - auto & cache = context.getSampleBlockCache(); + auto & cache = context_.getSampleBlockCache(); /// Using query string because query_ptr changes for every internal SELECT - auto key = queryToString(query_ptr); + auto key = queryToString(query_ptr_); if (cache.find(key) != cache.end()) { return cache[key]; } - return cache[key] = InterpreterSelectWithUnionQuery(query_ptr, context, SelectQueryOptions().analyze()).getSampleBlock(); + return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock(); +} + +size_t InterpreterSelectWithUnionQuery::optimizeUnionList() +{ + auto union_distinct_num = 0; + + auto union_default_mode = context->getSettingsRef().union_default_mode; + auto & ast = query_ptr->as(); + size_t num_selects = ast.list_of_selects->children.size(); + + if (!num_selects) + throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR); + + if (num_selects > 1) + { + for (auto & mode : ast.union_modes) + { + if (mode == ASTSelectWithUnionQuery::Mode::Unspecified) + { + if (union_default_mode == UnionMode::ALL) + mode = ASTSelectWithUnionQuery::Mode::ALL; + else if (union_default_mode == UnionMode::DISTINCT) + mode = ASTSelectWithUnionQuery::Mode::DISTINCT; + else + throw Exception( + "Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty", + DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); + } + } + /// Optimize: if there is UNION DISTINCT, all previous UNION DISTINCT can be rewritten to UNION ALL. + /// Therefore we have at most one UNION DISTINCT in a sequence. + for (auto rit = ast.union_modes.rbegin(); rit != ast.union_modes.rend(); ++rit) + { + if (*rit == ASTSelectWithUnionQuery::Mode::DISTINCT) + { + /// Number of streams need to do a DISTINCT transform after unite + union_distinct_num = ast.union_modes.rend() - rit + 1; + for (auto mode_to_modify = ++rit; mode_to_modify != ast.union_modes.rend(); ++mode_to_modify) + *mode_to_modify = ASTSelectWithUnionQuery::Mode::ALL; + break; + } + } + } + return union_distinct_num; } void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) { - query_plan = nested_interpreter->buildQueryPlan(context, result_header); + auto num_distinct_union = optimizeUnionList(); + size_t num_plans = nested_interpreters.size(); + + /// Skip union for single interpreter. + if (num_plans == 1) + { + nested_interpreters.front()->buildQueryPlan(query_plan); + return; + } + + /// All UNION streams in the chain does not need to do DISTINCT transform + if (num_distinct_union == 0) + { + std::vector> plans(num_plans); + DataStreams data_streams(num_plans); + + for (size_t i = 0; i < num_plans; ++i) + { + plans[i] = std::make_unique(); + nested_interpreters[i]->buildQueryPlan(*plans[i]); + data_streams[i] = plans[i]->getCurrentDataStream(); + } + + auto max_threads = context->getSettingsRef().max_threads; + auto union_step = std::make_unique(std::move(data_streams), result_header, max_threads); + + query_plan.unitePlans(std::move(union_step), std::move(plans)); + } + + /// The first union_distinct_num UNION streams need to do a DISTINCT transform after unite + else + { + QueryPlan distinct_query_plan; + + std::vector> plans(num_distinct_union); + DataStreams data_streams(num_distinct_union); + + for (size_t i = 0; i < num_distinct_union; ++i) + { + plans[i] = std::make_unique(); + nested_interpreters[i]->buildQueryPlan(*plans[i]); + data_streams[i] = plans[i]->getCurrentDataStream(); + } + + auto max_threads = context->getSettingsRef().max_threads; + auto union_step = std::make_unique(std::move(data_streams), result_header, max_threads); + + distinct_query_plan.unitePlans(std::move(union_step), std::move(plans)); + + /// Add distinct transform + const Settings & settings = context->getSettingsRef(); + SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); + + auto distinct_step + = std::make_unique(distinct_query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false); + + distinct_query_plan.addStep(std::move(distinct_step)); + + /// No other UNION streams after DISTINCT stream + if (num_plans == num_distinct_union) + { + query_plan = std::move(distinct_query_plan); + return; + } + + /// Build final UNION step + std::vector> final_plans(num_plans - num_distinct_union + 1); + DataStreams final_data_streams(num_plans - num_distinct_union + 1); + + final_plans[0] = std::make_unique(std::move(distinct_query_plan)); + final_data_streams[0] = final_plans[0]->getCurrentDataStream(); + + for (size_t i = 1; i < num_plans - num_distinct_union + 1; ++i) + { + final_plans[i] = std::make_unique(); + nested_interpreters[num_distinct_union + i - 1]->buildQueryPlan(*final_plans[i]); + final_data_streams[i] = final_plans[i]->getCurrentDataStream(); + } + + auto final_union_step = std::make_unique(std::move(final_data_streams), result_header, max_threads); + query_plan.unitePlans(std::move(final_union_step), std::move(final_plans)); + } } BlockIO InterpreterSelectWithUnionQuery::execute() @@ -390,7 +318,8 @@ BlockIO InterpreterSelectWithUnionQuery::execute() void InterpreterSelectWithUnionQuery::ignoreWithTotals() { - nested_interpreter->ignoreWithTotals(); + for (auto & interpreter : nested_interpreters) + interpreter->ignoreWithTotals(); } } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index 4af73b3c723..3976b5f8b82 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -1,9 +1,7 @@ #pragma once #include -#include -#include -#include +#include #include namespace DB @@ -13,27 +11,13 @@ class Context; class InterpreterSelectQuery; class QueryPlan; -struct NestedInterpreter -{ - ~NestedInterpreter() { } - enum class Type - { - LEAF, - INTERNAL - }; - Type type = Type::INTERNAL; - std::vector> children; - std::shared_ptr interpreter; - size_t num_distinct_union = 0; - QueryPlan buildQueryPlan(const std::shared_ptr & context, const Block & header); - void ignoreWithTotals(); -}; - /** Interprets one or multiple SELECT queries inside UNION/UNION ALL/UNION DISTINCT chain. */ -class InterpreterSelectWithUnionQuery : public IInterpreter +class InterpreterSelectWithUnionQuery : public IInterpreterUnionOrSelectQuery { public: + using IInterpreterUnionOrSelectQuery::getSampleBlock; + InterpreterSelectWithUnionQuery( const ASTPtr & query_ptr_, const Context & context_, @@ -43,41 +27,30 @@ public: ~InterpreterSelectWithUnionQuery() override; /// Builds QueryPlan for current query. - void buildQueryPlan(QueryPlan & query_plan); + virtual void buildQueryPlan(QueryPlan & query_plan) override; BlockIO execute() override; bool ignoreLimits() const override { return options.ignore_limits; } bool ignoreQuota() const override { return options.ignore_quota; } - Block getSampleBlock(); - static Block getSampleBlock( const ASTPtr & query_ptr_, const Context & context_); - void ignoreWithTotals(); - - ASTPtr getQuery() const { return query_ptr; } + virtual void ignoreWithTotals() override; private: - SelectQueryOptions options; - ASTPtr query_ptr; - std::shared_ptr context; - - std::shared_ptr nested_interpreter; - - Block result_header; - - size_t max_streams = 1; + std::vector> nested_interpreters; static Block getCommonHeaderForUnion(const Blocks & headers); - static void buildNestedTreeInterpreter( - const ASTPtr & ast_ptr, - std::shared_ptr nested_interpreter_, - std::vector> & interpreters, - int & index); + Block getCurrentChildResultHeader(const ASTPtr & ast_ptr_, const Names & required_result_column_names); + + std::unique_ptr + buildCurrentChildInterpreter(const ASTPtr & ast_ptr_, const Names & current_required_result_column_names); + + size_t optimizeUnionList(); }; } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 8b4b35785c1..57c557c5658 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -70,12 +70,10 @@ namespace ErrorCodes static void checkASTSizeLimits(const IAST & ast, const Settings & settings) { - std::cout << "\n\n before check limits"; if (settings.max_ast_depth) ast.checkDepth(settings.max_ast_depth); if (settings.max_ast_elements) ast.checkSize(settings.max_ast_elements); - std::cout << "\n\n after check limits"; } diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 5deae6f653f..8748bf1ef85 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -15,8 +15,7 @@ ASTPtr ASTSelectWithUnionQuery::clone() const res->list_of_selects = list_of_selects->clone(); res->children.push_back(res->list_of_selects); - res->union_modes.insert(res->union_modes.begin(), union_modes.begin(), union_modes.end()); - res->flatten_nodes_list = flatten_nodes_list->clone(); + res->union_modes = union_modes; cloneOutputOptions(*res); return res; @@ -25,10 +24,8 @@ ASTPtr ASTSelectWithUnionQuery::clone() const void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - std::cout << "\n\nin format \n\n"; std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); -#if 0 auto mode_to_str = [&](auto mode) { if (mode == Mode::Unspecified) @@ -38,18 +35,16 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F else return "DISTINCT"; }; -#endif - for (ASTs::const_iterator it = flatten_nodes_list->children.begin(); it != flatten_nodes_list->children.end(); ++it) + for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it) { if (it != list_of_selects->children.begin()) settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION " - // << mode_to_str(union_modes[it - list_of_selects->children.begin() - 1]) << (settings.hilite ? hilite_none : "") + << mode_to_str(union_modes[it - list_of_selects->children.begin() - 1]) << (settings.hilite ? hilite_none : "") << settings.nl_or_ws; (*it)->formatImpl(settings, state, frame); } - std::cout << "\n\nafter format \n\n"; } } diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index 67ec21e246c..5600dd4b43a 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -28,9 +28,6 @@ public: Modes union_modes; ASTPtr list_of_selects; - - /// we need flatten_nodes to help build nested_interpreter - ASTPtr flatten_nodes_list; }; } diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index d9f0b3562bc..8ee4154541b 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -76,18 +76,13 @@ void IAST::updateTreeHashImpl(SipHash & hash_state) const size_t IAST::checkDepthImpl(size_t max_depth, size_t level) const { - std::cout << "\n\n in check depth impl\n\n"; - std::cout << "\nchildren.size = " << children.size() << "\n\n"; size_t res = level + 1; for (const auto & child : children) { - std::cout << "\n in for\n\n"; if (level >= max_depth) throw Exception("AST is too deep. Maximum: " + toString(max_depth), ErrorCodes::TOO_DEEP_AST); res = std::max(res, child->checkDepthImpl(max_depth, level + 1)); - std::cout << "\n after for\n\n"; } - std::cout << "\n\n after impl\n\n"; return res; } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index d1fca853592..c88c80021d6 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -11,7 +11,6 @@ #include #include -#include class SipHash; @@ -92,7 +91,6 @@ public: */ size_t checkDepth(size_t max_depth) const { - std::cout << "\n in check depth\n\n"; return checkDepthImpl(max_depth, 0); } diff --git a/src/Parsers/ParserSelectWithUnionQuery.cpp b/src/Parsers/ParserSelectWithUnionQuery.cpp index ee03da753e4..382d7a66669 100644 --- a/src/Parsers/ParserSelectWithUnionQuery.cpp +++ b/src/Parsers/ParserSelectWithUnionQuery.cpp @@ -1,5 +1,3 @@ -#include -#include #include #include #include @@ -10,24 +8,9 @@ namespace DB { -static void getSelectsFromUnionListNode(ASTPtr & ast_select, ASTs & selects) -{ - if (auto * inner_union = ast_select->as()) - { - for (auto & child : inner_union->list_of_selects->children) - { - getSelectsFromUnionListNode(child, selects); - } - - return; - } - - selects.push_back(std::move(ast_select)); -} bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - std::cout << "\n\n in ParserSelectWithUnionQuery\n\n"; ASTPtr list_node; ParserUnionList parser( @@ -42,27 +25,17 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & auto select_with_union_query = std::make_shared(); node = select_with_union_query; - select_with_union_query->list_of_selects = std::make_shared(); + select_with_union_query->list_of_selects = list_node; select_with_union_query->children.push_back(select_with_union_query->list_of_selects); - select_with_union_query->list_of_selects->children.insert( - select_with_union_query->list_of_selects->children.begin(), list_node->children.begin(), list_node->children.end()); select_with_union_query->union_modes = parser.getUnionModes(); - /// NOTE: We cann't simply flatten inner union query now, since we may have different union mode in query, + /// NOTE: We cann't flatten inner union query now, since we may have different union mode in query, /// so flatten may change it's semantics. For example: /// flatten `SELECT 1 UNION (SELECT 1 UNION ALL SELETC 1)` -> `SELECT 1 UNION SELECT 1 UNION ALL SELECT 1` - /// We can use a non-flatten AST to help build QueryPlan in InterpreterSelectWithUnionQuery - - select_with_union_query->flatten_nodes_list = std::make_shared(); - - for (auto & child : list_node->children) - { - getSelectsFromUnionListNode(child, select_with_union_query->flatten_nodes_list->children); - } - std::cout << "\n\n after ParserSelectWithUnionQuery\n\n"; - std::cout << "\n\n flatten_nodes.size =" << select_with_union_query->flatten_nodes_list->children.size() << "\n\n"; return true; } + + } From af45cddbc7a5033296f88259d4a2b79179cda30d Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 1 Nov 2020 14:43:29 +0000 Subject: [PATCH 0087/1088] fix --- src/Parsers/ASTSelectWithUnionQuery.cpp | 21 +- ...t_and_setting_union_default_mode.reference | 199 +++++++----------- ...istinct_and_setting_union_default_mode.sql | 22 ++ 3 files changed, 109 insertions(+), 133 deletions(-) diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 8748bf1ef85..57a3c00543c 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -1,9 +1,8 @@ -#include #include +#include +#include #include -#include - namespace DB { @@ -40,10 +39,18 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F { if (it != list_of_selects->children.begin()) settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION " - << mode_to_str(union_modes[it - list_of_selects->children.begin() - 1]) << (settings.hilite ? hilite_none : "") - << settings.nl_or_ws; - - (*it)->formatImpl(settings, state, frame); + << mode_to_str(union_modes[it - list_of_selects->children.begin() - 1]) << (settings.hilite ? hilite_none : ""); + if (auto _ = (*it)->as()) + { + auto sub_query = std::make_shared(); + sub_query->children.push_back(*it); + sub_query->formatImpl(settings, state, frame); + } + else + { + settings.ostr << settings.nl_or_ws; + (*it)->formatImpl(settings, state, frame); + } } } diff --git a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference index 6c4547333fe..868cadf1d81 100644 --- a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference +++ b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference @@ -1,140 +1,87 @@ -0 1 -2 -3 -4 -5 -6 -7 -8 -9 -0 1 -2 -3 -4 -5 -6 -7 -8 -9 -0 1 -2 -3 -4 -5 -6 -7 -8 -9 -0 1 -2 -3 -4 -5 -6 -7 -8 -9 -0 1 -2 -3 -4 -5 -6 -7 -8 -9 -0 1 -2 -3 -4 -5 -6 -7 -8 -9 -0 1 -2 -3 -4 -5 -6 -7 -8 -9 -0 1 -2 -3 -4 -5 -6 -7 -8 -9 -0 1 -2 -3 -4 -5 -6 -7 -8 -9 -0 1 -2 -3 -4 -5 -6 -7 -8 -9 -0 1 -2 -3 -4 -5 -6 -7 -8 -9 -0 1 -2 -3 -4 -5 -6 -7 -8 -9 -0 1 -2 -3 -4 -5 -6 -7 -8 -9 -0 1 -2 -3 -4 -5 -6 -7 -8 -9 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql index 12fe204591c..30ccfdc8c45 100644 --- a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql +++ b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql @@ -1,2 +1,24 @@ SELECT 1; (((((((SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1; +(((((((SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1; +SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION ALL SELECT 1; +SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1; +SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION ALL SELECT 1; +SELECT 1 UNION (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1); +SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1); +SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1))))))); +SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1))))))); +SELECT * FROM (SELECT 1 UNION ALL (SELECT 1 UNION SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1))); + +SET union_default_mode='ALL'; + +(((((((SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1; +(((((((SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1; +SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION ALL SELECT 1; +SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1; +SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION ALL SELECT 1; +SELECT 1 UNION (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1); +SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1); +SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1))))))); +SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL(SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1))))))); +SELECT * FROM (SELECT 1 UNION ALL (SELECT 1 UNION SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1))); From e82ce52e2f6a458a62fae6171920a57ddbec54a7 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 1 Nov 2020 14:57:41 +0000 Subject: [PATCH 0088/1088] fix --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + .../InterpreterSelectWithUnionQuery.cpp | 14 +++++++------- src/Interpreters/InterpreterSelectWithUnionQuery.h | 1 - src/Parsers/ExpressionListParsers.cpp | 2 -- 4 files changed, 8 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0b86914b1fa..73252894aa5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -119,6 +119,7 @@ String InterpreterSelectQuery::generateFilterActions( ParserExpression expr_parser; expr_list->children.push_back(parseQuery(expr_parser, column_str, 0, context->getSettingsRef().max_parser_depth)); } + select_ast->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared()); auto tables = select_ast->tables(); auto tables_elem = std::make_shared(); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 43ff1c48167..a3798e42f02 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -1,14 +1,15 @@ -#include -#include -#include -#include #include -#include +#include +#include +#include +#include +#include #include +#include #include #include #include -#include +#include namespace DB { @@ -30,7 +31,6 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( if (!num_children) throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR); - /// We first build nested interpreters for each select query, then using this nested interpreters to build Tree Structured nested interpreter. /// Note that we pass 'required_result_column_names' to first SELECT. /// And for the rest, we pass names at the corresponding positions of 'required_result_column_names' in the result of first SELECT, /// because names could be different. diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index 3976b5f8b82..06d31c92a67 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -2,7 +2,6 @@ #include #include -#include namespace DB { diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 220d304751e..1cc72f5fb8b 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -1,5 +1,3 @@ -#include -#include #include #include From 5f25f9c814a2314489e1f66c8fb52da5fff6ade7 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 1 Nov 2020 15:15:51 +0000 Subject: [PATCH 0089/1088] fix fix --- src/Interpreters/InterpreterSelectWithUnionQuery.cpp | 2 +- src/Parsers/ParserSelectWithUnionQuery.cpp | 5 ----- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index a3798e42f02..1fecd221aba 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -197,7 +197,7 @@ size_t InterpreterSelectWithUnionQuery::optimizeUnionList() DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); } } - /// Optimize: if there is UNION DISTINCT, all previous UNION DISTINCT can be rewritten to UNION ALL. + /// Optimize general cases: if there is UNION DISTINCT, all previous UNION DISTINCT can be rewritten to UNION ALL. /// Therefore we have at most one UNION DISTINCT in a sequence. for (auto rit = ast.union_modes.rbegin(); rit != ast.union_modes.rend(); ++rit) { diff --git a/src/Parsers/ParserSelectWithUnionQuery.cpp b/src/Parsers/ParserSelectWithUnionQuery.cpp index 382d7a66669..db586867db0 100644 --- a/src/Parsers/ParserSelectWithUnionQuery.cpp +++ b/src/Parsers/ParserSelectWithUnionQuery.cpp @@ -3,8 +3,6 @@ #include #include #include -#include - namespace DB { @@ -32,10 +30,7 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & /// NOTE: We cann't flatten inner union query now, since we may have different union mode in query, /// so flatten may change it's semantics. For example: /// flatten `SELECT 1 UNION (SELECT 1 UNION ALL SELETC 1)` -> `SELECT 1 UNION SELECT 1 UNION ALL SELECT 1` - return true; } - - } From f04d74172f794b76c393036ec2014a1d639d8b11 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 1 Nov 2020 15:41:31 +0000 Subject: [PATCH 0090/1088] fix --- ...9_union_distinct_and_setting_union_default_mode.reference | 5 +++++ .../01529_union_distinct_and_setting_union_default_mode.sql | 2 ++ 2 files changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference index 868cadf1d81..f9f3ee818e9 100644 --- a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference +++ b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference @@ -10,6 +10,8 @@ 1 1 1 +all +all 1 1 1 @@ -51,6 +53,9 @@ 1 1 1 +all +all +all 1 1 1 diff --git a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql index 30ccfdc8c45..6e45c150508 100644 --- a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql +++ b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql @@ -2,6 +2,7 @@ SELECT 1; (((((((SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1; (((((((SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1; SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION ALL SELECT 1; +SELECT 'all' UNION SELECT 'all' UNION ALL SELECT 'all'; SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1; SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION ALL SELECT 1; SELECT 1 UNION (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1); @@ -15,6 +16,7 @@ SET union_default_mode='ALL'; (((((((SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1; (((((((SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1; SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION ALL SELECT 1; +SELECT 'all' UNION SELECT 'all' UNION ALL SELECT 'all'; SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1; SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION ALL SELECT 1; SELECT 1 UNION (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1); From 3dce3c6a21ed0d16c19e401b66b9b9a27a2e5949 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 2 Nov 2020 05:28:37 +0000 Subject: [PATCH 0091/1088] fix --- src/Interpreters/InterpreterInsertQuery.cpp | 79 +++++++++++---------- src/Parsers/ASTSelectWithUnionQuery.cpp | 3 +- src/Storages/SelectQueryDescription.cpp | 23 ++++-- 3 files changed, 63 insertions(+), 42 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index aa8bcd74ea6..5ffd667bf40 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -140,34 +140,39 @@ Block InterpreterInsertQuery::getSampleBlock( /** A query that just reads all data without any complex computations or filetering. * If we just pipe the result to INSERT, we don't have to use too many threads for read. */ -static bool isTrivialSelect(const ASTSelectQuery & select_query) +static bool isTrivialSelect(const ASTPtr & select) { - const auto & tables = select_query.tables(); + if (auto select_query = select->as()) + { + const auto & tables = select_query->tables(); - if (!tables) - return false; + if (!tables) + return false; - const auto & tables_in_select_query = tables->as(); + const auto & tables_in_select_query = tables->as(); - if (tables_in_select_query.children.size() != 1) - return false; + if (tables_in_select_query.children.size() != 1) + return false; - const auto & child = tables_in_select_query.children.front(); - const auto & table_element = child->as(); - const auto & table_expr = table_element.table_expression->as(); + const auto & child = tables_in_select_query.children.front(); + const auto & table_element = child->as(); + const auto & table_expr = table_element.table_expression->as(); - if (table_expr.subquery) - return false; + if (table_expr.subquery) + return false; - /// Note: how to write it in more generic way? - return (!select_query.distinct - && !select_query.limit_with_ties - && !select_query.prewhere() - && !select_query.where() - && !select_query.groupBy() - && !select_query.having() - && !select_query.orderBy() - && !select_query.limitBy()); + /// Note: how to write it in more generic way? + return (!select_query->distinct + && !select_query->limit_with_ties + && !select_query->prewhere() + && !select_query->where() + && !select_query->groupBy() + && !select_query->having() + && !select_query->orderBy() + && !select_query->limitBy()); + } + /// This query is ASTSelectWithUnionQuery subquery + return false; }; @@ -196,23 +201,25 @@ BlockIO InterpreterInsertQuery::execute() auto new_query = std::dynamic_pointer_cast(query.clone()); if (select.list_of_selects->children.size() == 1) { - auto & select_query = select.list_of_selects->children.at(0)->as(); - JoinedTables joined_tables(Context(context), select_query); - - if (joined_tables.tablesCount() == 1) + if (auto select_query = select.list_of_selects->children.at(0)->as()) { - storage_src = std::dynamic_pointer_cast(joined_tables.getLeftTableStorage()); - if (storage_src) + JoinedTables joined_tables(Context(context), *select_query); + + if (joined_tables.tablesCount() == 1) { - const auto select_with_union_query = std::make_shared(); - select_with_union_query->list_of_selects = std::make_shared(); + storage_src = std::dynamic_pointer_cast(joined_tables.getLeftTableStorage()); + if (storage_src) + { + const auto select_with_union_query = std::make_shared(); + select_with_union_query->list_of_selects = std::make_shared(); - auto new_select_query = std::dynamic_pointer_cast(select_query.clone()); - select_with_union_query->list_of_selects->children.push_back(new_select_query); + auto new_select_query = std::dynamic_pointer_cast(select_query->clone()); + select_with_union_query->list_of_selects->children.push_back(new_select_query); - new_select_query->replaceDatabaseAndTable(storage_src->getRemoteDatabaseName(), storage_src->getRemoteTableName()); + new_select_query->replaceDatabaseAndTable(storage_src->getRemoteDatabaseName(), storage_src->getRemoteTableName()); - new_query->select = select_with_union_query; + new_query->select = select_with_union_query; + } } } } @@ -277,10 +284,8 @@ BlockIO InterpreterInsertQuery::execute() { const auto & selects = query.select->as().list_of_selects->children; - is_trivial_insert_select = std::all_of(selects.begin(), selects.end(), [](const ASTPtr & select) - { - return isTrivialSelect(select->as()); - }); + is_trivial_insert_select + = std::all_of(selects.begin(), selects.end(), [](const ASTPtr & select) { return isTrivialSelect(select); }); } if (is_trivial_insert_select) diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 57a3c00543c..29aa1b81acf 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -48,7 +48,8 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F } else { - settings.ostr << settings.nl_or_ws; + if (it != list_of_selects->children.begin()) + settings.ostr << settings.nl_or_ws; (*it)->formatImpl(settings, state, frame); } } diff --git a/src/Storages/SelectQueryDescription.cpp b/src/Storages/SelectQueryDescription.cpp index 0935a5be5ca..db886793f7c 100644 --- a/src/Storages/SelectQueryDescription.cpp +++ b/src/Storages/SelectQueryDescription.cpp @@ -98,20 +98,35 @@ void checkAllowedQueries(const ASTSelectQuery & query) } +/// check if only one single select query in SelectWithUnionQuery +static bool isSingleSelect(const ASTPtr & select, ASTPtr & res) +{ + auto new_select = select->as(); + if (new_select.list_of_selects->children.size() != 1) + return false; + auto & new_inner_query = new_select.list_of_selects->children.at(0); + if (auto _ = new_inner_query->as()) + { + res = new_inner_query; + return true; + } + else + return isSingleSelect(new_inner_query, res); +} + SelectQueryDescription SelectQueryDescription::getSelectQueryFromASTForMatView(const ASTPtr & select, const Context & context) { - auto & new_select = select->as(); + ASTPtr new_inner_query; - if (new_select.list_of_selects->children.size() != 1) + if (!isSingleSelect(select, new_inner_query)) throw Exception("UNION is not supported for MATERIALIZED VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW); - auto & new_inner_query = new_select.list_of_selects->children.at(0); auto & select_query = new_inner_query->as(); checkAllowedQueries(select_query); SelectQueryDescription result; result.select_table_id = extractDependentTableFromSelectQuery(select_query, context); - result.select_query = new_select.clone(); + result.select_query = select->as().clone(); result.inner_query = new_inner_query->clone(); return result; From 7159affda8e8b91e3e3da8ebcd7681c0ab305c6e Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 2 Nov 2020 06:53:09 +0000 Subject: [PATCH 0092/1088] fix --- src/Parsers/ASTSelectWithUnionQuery.cpp | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 29aa1b81acf..54ae310a23a 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -40,11 +40,22 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F if (it != list_of_selects->children.begin()) settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION " << mode_to_str(union_modes[it - list_of_selects->children.begin() - 1]) << (settings.hilite ? hilite_none : ""); - if (auto _ = (*it)->as()) + if (auto node = (*it)->as()) { - auto sub_query = std::make_shared(); - sub_query->children.push_back(*it); - sub_query->formatImpl(settings, state, frame); + // just one child in subquery, () is not need + if (node->list_of_selects->children.size() == 1) + { + if (it != list_of_selects->children.begin()) + settings.ostr << settings.nl_or_ws; + node->list_of_selects->children.at(0)->formatImpl(settings, state, frame); + } + // more than one child in subquery + else + { + auto sub_query = std::make_shared(); + sub_query->children.push_back(*it); + sub_query->formatImpl(settings, state, frame); + } } else { From 1a34abadbc25afe8e277273c64d743263ed45616 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 2 Nov 2020 08:02:35 +0000 Subject: [PATCH 0093/1088] fix fix fix --- src/Interpreters/InterpreterInsertQuery.cpp | 4 ++-- src/Interpreters/InterpreterSelectWithUnionQuery.cpp | 4 ++-- src/Parsers/ASTSelectWithUnionQuery.cpp | 2 +- src/Storages/SelectQueryDescription.cpp | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 5ffd667bf40..e2830322024 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -142,7 +142,7 @@ Block InterpreterInsertQuery::getSampleBlock( */ static bool isTrivialSelect(const ASTPtr & select) { - if (auto select_query = select->as()) + if (auto * select_query = select->as()) { const auto & tables = select_query->tables(); @@ -201,7 +201,7 @@ BlockIO InterpreterInsertQuery::execute() auto new_query = std::dynamic_pointer_cast(query.clone()); if (select.list_of_selects->children.size() == 1) { - if (auto select_query = select.list_of_selects->children.at(0)->as()) + if (auto * select_query = select.list_of_selects->children.at(0)->as()) { JoinedTables joined_tables(Context(context), *select_query); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 1fecd221aba..7cc0e890fc2 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -139,7 +139,7 @@ Block InterpreterSelectWithUnionQuery::getCommonHeaderForUnion(const Blocks & he Block InterpreterSelectWithUnionQuery::getCurrentChildResultHeader(const ASTPtr & ast_ptr_, const Names & required_result_column_names) { - if (const auto _ = ast_ptr_->as()) + if (ast_ptr_->as()) return InterpreterSelectWithUnionQuery(ast_ptr_, *context, options.copy().analyze().noModify(), required_result_column_names) .getSampleBlock(); else @@ -149,7 +149,7 @@ Block InterpreterSelectWithUnionQuery::getCurrentChildResultHeader(const ASTPtr std::unique_ptr InterpreterSelectWithUnionQuery::buildCurrentChildInterpreter(const ASTPtr & ast_ptr_, const Names & current_required_result_column_names) { - if (const auto _ = ast_ptr_->as()) + if (ast_ptr_->as()) return std::make_unique(ast_ptr_, *context, options, current_required_result_column_names); else return std::make_unique(ast_ptr_, *context, options, current_required_result_column_names); diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 54ae310a23a..639c8ec1b6e 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -40,7 +40,7 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F if (it != list_of_selects->children.begin()) settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION " << mode_to_str(union_modes[it - list_of_selects->children.begin() - 1]) << (settings.hilite ? hilite_none : ""); - if (auto node = (*it)->as()) + if (auto * node = (*it)->as()) { // just one child in subquery, () is not need if (node->list_of_selects->children.size() == 1) diff --git a/src/Storages/SelectQueryDescription.cpp b/src/Storages/SelectQueryDescription.cpp index db886793f7c..c11e6bd74f8 100644 --- a/src/Storages/SelectQueryDescription.cpp +++ b/src/Storages/SelectQueryDescription.cpp @@ -105,7 +105,7 @@ static bool isSingleSelect(const ASTPtr & select, ASTPtr & res) if (new_select.list_of_selects->children.size() != 1) return false; auto & new_inner_query = new_select.list_of_selects->children.at(0); - if (auto _ = new_inner_query->as()) + if (new_inner_query->as()) { res = new_inner_query; return true; From 5c77ae914f50dc00d960141a27f357d5388bd5c0 Mon Sep 17 00:00:00 2001 From: myrrc Date: Mon, 2 Nov 2020 14:47:59 +0300 Subject: [PATCH 0094/1088] fixed contexpr modifier build, updated tests --- .../AggregateFunctionAvg.cpp | 2 +- src/AggregateFunctions/AggregateFunctionAvg.h | 12 +- .../AggregateFunctionAvgWeighted.cpp | 2 +- ...01518_nullable_aggregate_states2.reference | 1220 ++++++++--------- 4 files changed, 614 insertions(+), 622 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAvg.cpp b/src/AggregateFunctions/AggregateFunctionAvg.cpp index 4d1b01b25fc..bec0a778c2d 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvg.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes namespace { -constexpr bool allowType(const DataTypePtr& type) noexcept +bool allowType(const DataTypePtr& type) noexcept { const WhichDataType t(type); return t.isInt() || t.isUInt() || t.isFloat() || t.isDecimal(); diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 2369ae835c3..7c2ad3d0984 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -17,16 +17,8 @@ struct RationalFraction Float64 numerator{0}; Denominator denominator{0}; - Float64 NO_SANITIZE_UNDEFINED result() const - { - if constexpr (std::numeric_limits::is_iec559) - return static_cast(numerator) / denominator; /// allow division by zero - - if (denominator == static_cast(0)) - return static_cast(0); - - return static_cast(numerator / denominator); - } + /// Allow division by zero as sometimes we need to return NaN. + Float64 NO_SANITIZE_UNDEFINED result() const { return numerator / denominator; } }; /** diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp index 6b677414d87..9045bff002a 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp @@ -15,7 +15,7 @@ namespace ErrorCodes namespace { -constexpr bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) noexcept +bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) noexcept { const WhichDataType l_dt(left), r_dt(right); diff --git a/tests/queries/0_stateless/01518_nullable_aggregate_states2.reference b/tests/queries/0_stateless/01518_nullable_aggregate_states2.reference index e068a35dbd1..e393fd798fb 100644 --- a/tests/queries/0_stateless/01518_nullable_aggregate_states2.reference +++ b/tests/queries/0_stateless/01518_nullable_aggregate_states2.reference @@ -288,723 +288,723 @@ 35 102 10025 99935 0.1051 300.1051 150.1051 15160.61561 0.1051 300.1051 150.1051 15160.61542 0.10510 300.10510 150.10510000000014 15160.61510 2020-01-01 2020-01-02 2020-01-01 00:00:35 2020-01-02 03:45:35 2020-01-01 00:00:35.000 2020-01-02 03:45:35.000 35 99935 49985 5048485 35 99935 49985 5048485 -32534 32401 4564.009900990099 460965 -128 127 -0.900990099009901 -91 350 101 10340 99251 1.05105 298.05105 149.55105 14955.1051 1.05105 298.05106 149.55105 14955.10532 1.05105 298.05105 149.55105 14955.10500 2020-01-01 2020-01-02 2020-01-01 00:05:50 2020-01-02 03:34:11 2020-01-01 00:05:50.000 2020-01-02 03:34:11.000 350 99251 49800.5 4980050 350 99251 49800.5 4980050 -32219 32716 5236.02 523602 -126 125 0.82 82 351 101 10341 99252 1.05405 298.05405 149.55405 14955.4054 1.05405 298.05405 149.55404 14955.40499 1.05405 298.05405 149.55405000000016 14955.40500 2020-01-01 2020-01-02 2020-01-01 00:05:51 2020-01-02 03:34:12 2020-01-01 00:05:51.000 2020-01-02 03:34:12.000 351 99252 49801.5 4980150 351 99252 49801.5 4980150 -32218 32717 5237.02 523702 -125 126 1.82 182 -352 101 10342 99253 1.05705 298.05705 149.55705 14955.7057 1.05705 298.05707 149.55705 14955.70574 1.05705 298.05705 149.55704999999992 14955.70500 2020-01-01 2020-01-02 2020-01-01 00:05:52 2020-01-02 03:34:13 2020-01-01 00:05:52.000 2020-01-02 03:34:13.000 352 99253 49802.5 4980250 352 99253 49802.5 4980250 -32217 32718 5238.02 523802 -124 127 2.82 282 -353 101 10343 99254 1.06006 298.06006 149.56006 14956.006 1.06006 298.06006 149.56005 14956.00587 1.06006 298.06006 149.56005999999982 14956.00600 2020-01-01 2020-01-02 2020-01-01 00:05:53 2020-01-02 03:34:14 2020-01-01 00:05:53.000 2020-01-02 03:34:14.000 353 99254 49803.5 4980350 353 99254 49803.5 4980350 -32216 32719 5239.02 523902 -128 127 1.26 126 -354 101 10344 99255 1.06306 298.06306 149.56306 14956.3063 1.06306 298.06305 149.56306 14956.3062 1.06306 298.06306 149.56306000000026 14956.30600 2020-01-01 2020-01-02 2020-01-01 00:05:54 2020-01-02 03:34:15 2020-01-01 00:05:54.000 2020-01-02 03:34:15.000 354 99255 49804.5 4980450 354 99255 49804.5 4980450 -32215 32720 5240.02 524002 -128 127 -0.3 -30 -355 101 10345 99256 1.06606 298.06606 149.56606 14956.6066 1.06606 298.06607 149.56606 14956.6068 1.06606 298.06606 149.56605999999982 14956.60600 2020-01-01 2020-01-02 2020-01-01 00:05:55 2020-01-02 03:34:16 2020-01-01 00:05:55.000 2020-01-02 03:34:16.000 355 99256 49805.5 4980550 355 99256 49805.5 4980550 -32214 32721 5241.02 524102 -128 123 -1.86 -186 -356 101 10346 99257 1.06906 298.06906 149.56906 14956.9069 1.06906 298.06906 149.56907 14956.90709 1.06906 298.06906 149.56905999999984 14956.90600 2020-01-01 2020-01-02 2020-01-01 00:05:56 2020-01-02 03:34:17 2020-01-01 00:05:56.000 2020-01-02 03:34:17.000 356 99257 49806.5 4980650 356 99257 49806.5 4980650 -32213 32722 5242.02 524202 -127 124 -0.86 -86 -357 101 10347 99258 1.07207 298.07207 149.57207 14957.2072 1.07207 298.07208 149.57207 14957.20721 1.07207 298.07207 149.57207000000014 14957.20700 2020-01-01 2020-01-02 2020-01-01 00:05:57 2020-01-02 03:34:18 2020-01-01 00:05:57.000 2020-01-02 03:34:18.000 357 99258 49807.5 4980750 357 99258 49807.5 4980750 -32212 32723 5243.02 524302 -126 125 0.14 14 -358 101 10348 99259 1.07507 298.07507 149.57507 14957.5075 1.07507 298.07507 149.57507 14957.50734 1.07507 298.07507 149.57507000000015 14957.50700 2020-01-01 2020-01-02 2020-01-01 00:05:58 2020-01-02 03:34:19 2020-01-01 00:05:58.000 2020-01-02 03:34:19.000 358 99259 49808.5 4980850 358 99259 49808.5 4980850 -32211 32724 5244.02 524402 -125 126 1.14 114 -359 101 10349 99260 1.07807 298.07807 149.57807 14957.8078 1.07807 298.07806 149.57807 14957.80767 1.07807 298.07807 149.5780699999997 14957.80700 2020-01-01 2020-01-02 2020-01-01 00:05:59 2020-01-02 03:34:20 2020-01-01 00:05:59.000 2020-01-02 03:34:20.000 359 99260 49809.5 4980950 359 99260 49809.5 4980950 -32210 32725 5245.02 524502 -124 127 2.14 214 +352 101 10342 99253 1.05705 298.05705 149.55705 14955.7057 1.05705 298.05707 149.55705 14955.70574 1.05705 298.05705 149.5570500000001 14955.70500 2020-01-01 2020-01-02 2020-01-01 00:05:52 2020-01-02 03:34:13 2020-01-01 00:05:52.000 2020-01-02 03:34:13.000 352 99253 49802.5 4980250 352 99253 49802.5 4980250 -32217 32718 5238.02 523802 -124 127 2.82 282 +353 101 10343 99254 1.06006 298.06006 149.56006 14956.006 1.06006 298.06006 149.56005 14956.00587 1.06006 298.06006 149.5600599999999 14956.00600 2020-01-01 2020-01-02 2020-01-01 00:05:53 2020-01-02 03:34:14 2020-01-01 00:05:53.000 2020-01-02 03:34:14.000 353 99254 49803.5 4980350 353 99254 49803.5 4980350 -32216 32719 5239.02 523902 -128 127 1.26 126 +354 101 10344 99255 1.06306 298.06306 149.56306 14956.3063 1.06306 298.06305 149.56306 14956.3062 1.06306 298.06306 149.56306000000012 14956.30600 2020-01-01 2020-01-02 2020-01-01 00:05:54 2020-01-02 03:34:15 2020-01-01 00:05:54.000 2020-01-02 03:34:15.000 354 99255 49804.5 4980450 354 99255 49804.5 4980450 -32215 32720 5240.02 524002 -128 127 -0.3 -30 +355 101 10345 99256 1.06606 298.06606 149.56606 14956.6066 1.06606 298.06607 149.56606 14956.6068 1.06606 298.06606 149.56605999999994 14956.60600 2020-01-01 2020-01-02 2020-01-01 00:05:55 2020-01-02 03:34:16 2020-01-01 00:05:55.000 2020-01-02 03:34:16.000 355 99256 49805.5 4980550 355 99256 49805.5 4980550 -32214 32721 5241.02 524102 -128 123 -1.86 -186 +356 101 10346 99257 1.06906 298.06906 149.56906 14956.9069 1.06906 298.06906 149.56907 14956.90709 1.06906 298.06906 149.56905999999992 14956.90600 2020-01-01 2020-01-02 2020-01-01 00:05:56 2020-01-02 03:34:17 2020-01-01 00:05:56.000 2020-01-02 03:34:17.000 356 99257 49806.5 4980650 356 99257 49806.5 4980650 -32213 32722 5242.02 524202 -127 124 -0.86 -86 +357 101 10347 99258 1.07207 298.07207 149.57207 14957.2072 1.07207 298.07208 149.57207 14957.20721 1.07207 298.07207 149.57207000000008 14957.20700 2020-01-01 2020-01-02 2020-01-01 00:05:57 2020-01-02 03:34:18 2020-01-01 00:05:57.000 2020-01-02 03:34:18.000 357 99258 49807.5 4980750 357 99258 49807.5 4980750 -32212 32723 5243.02 524302 -126 125 0.14 14 +358 101 10348 99259 1.07507 298.07507 149.57507 14957.5075 1.07507 298.07507 149.57507 14957.50734 1.07507 298.07507 149.57507000000004 14957.50700 2020-01-01 2020-01-02 2020-01-01 00:05:58 2020-01-02 03:34:19 2020-01-01 00:05:58.000 2020-01-02 03:34:19.000 358 99259 49808.5 4980850 358 99259 49808.5 4980850 -32211 32724 5244.02 524402 -125 126 1.14 114 +359 101 10349 99260 1.07807 298.07807 149.57807 14957.8078 1.07807 298.07806 149.57807 14957.80767 1.07807 298.07807 149.57806999999985 14957.80700 2020-01-01 2020-01-02 2020-01-01 00:05:59 2020-01-02 03:34:20 2020-01-01 00:05:59.000 2020-01-02 03:34:20.000 359 99260 49809.5 4980950 359 99260 49809.5 4980950 -32210 32725 5245.02 524502 -124 127 2.14 214 36 102 10026 99936 0.1081 300.1081 150.1081 15160.91891 0.1081 300.1081 150.1081 15160.91873 0.10810 300.10810 150.10809999999998 15160.91810 2020-01-01 2020-01-02 2020-01-01 00:00:36 2020-01-02 03:45:36 2020-01-01 00:00:36.000 2020-01-02 03:45:36.000 36 99936 49986 5048586 36 99936 49986 5048586 -32533 32402 4565.009900990099 461066 -128 123 -2.4356435643564356 -246 360 101 10350 99261 1.08108 298.08108 149.58108 14958.1081 1.08108 298.0811 149.58108 14958.10827 1.08108 298.08108 149.58107999999996 14958.10800 2020-01-01 2020-01-02 2020-01-01 00:06:00 2020-01-02 03:34:21 2020-01-01 00:06:00.000 2020-01-02 03:34:21.000 360 99261 49810.5 4981050 360 99261 49810.5 4981050 -32209 32726 5246.02 524602 -128 127 0.58 58 -361 101 10351 99262 1.08408 298.08408 149.58408 14958.4084 1.08408 298.08408 149.58408 14958.40856 1.08408 298.08408 149.58408000000006 14958.40800 2020-01-01 2020-01-02 2020-01-01 00:06:01 2020-01-02 03:34:22 2020-01-01 00:06:01.000 2020-01-02 03:34:22.000 361 99262 49811.5 4981150 361 99262 49811.5 4981150 -32208 32727 5247.02 524702 -128 123 -0.98 -98 -362 101 10352 99263 1.08708 298.08708 149.58708 14958.7087 1.08708 298.0871 149.58708 14958.70868 1.08708 298.08708 149.58707999999987 14958.70800 2020-01-01 2020-01-02 2020-01-01 00:06:02 2020-01-02 03:34:23 2020-01-01 00:06:02.000 2020-01-02 03:34:23.000 362 99263 49812.5 4981250 362 99263 49812.5 4981250 -32207 32728 5248.02 524802 -127 124 0.02 2 -363 101 10353 99264 1.09009 298.09009 149.59009 14959.009 1.09009 298.0901 149.59008 14959.00884 1.09009 298.09009 149.5900899999998 14959.00900 2020-01-01 2020-01-02 2020-01-01 00:06:03 2020-01-02 03:34:24 2020-01-01 00:06:03.000 2020-01-02 03:34:24.000 363 99264 49813.5 4981350 363 99264 49813.5 4981350 -32206 32729 5249.02 524902 -126 125 1.02 102 -364 101 10354 99265 1.09309 298.09309 149.59309 14959.3093 1.09309 298.09308 149.59309 14959.30915 1.09309 298.09309 149.5930900000002 14959.30900 2020-01-01 2020-01-02 2020-01-01 00:06:04 2020-01-02 03:34:25 2020-01-01 00:06:04.000 2020-01-02 03:34:25.000 364 99265 49814.5 4981450 364 99265 49814.5 4981450 -32205 32730 5250.02 525002 -125 126 2.02 202 -365 101 10355 99266 1.09609 298.09609 149.59609 14959.6096 1.09609 298.0961 149.59609 14959.6099 1.09609 298.09609 149.59608999999972 14959.60900 2020-01-01 2020-01-02 2020-01-01 00:06:05 2020-01-02 03:34:26 2020-01-01 00:06:05.000 2020-01-02 03:34:26.000 365 99266 49815.5 4981550 365 99266 49815.5 4981550 -32204 32731 5251.02 525102 -124 127 3.02 302 -366 101 10356 99267 1.09909 298.09909 149.59909 14959.9099 1.09909 298.0991 149.5991 14959.91003 1.09909 298.09909 149.59908999999982 14959.90900 2020-01-01 2020-01-02 2020-01-01 00:06:06 2020-01-02 03:34:27 2020-01-01 00:06:06.000 2020-01-02 03:34:27.000 366 99267 49816.5 4981650 366 99267 49816.5 4981650 -32203 32732 5252.02 525202 -128 127 1.46 146 +361 101 10351 99262 1.08408 298.08408 149.58408 14958.4084 1.08408 298.08408 149.58408 14958.40856 1.08408 298.08408 149.58408 14958.40800 2020-01-01 2020-01-02 2020-01-01 00:06:01 2020-01-02 03:34:22 2020-01-01 00:06:01.000 2020-01-02 03:34:22.000 361 99262 49811.5 4981150 361 99262 49811.5 4981150 -32208 32727 5247.02 524702 -128 123 -0.98 -98 +362 101 10352 99263 1.08708 298.08708 149.58708 14958.7087 1.08708 298.0871 149.58708 14958.70868 1.08708 298.08708 149.58707999999996 14958.70800 2020-01-01 2020-01-02 2020-01-01 00:06:02 2020-01-02 03:34:23 2020-01-01 00:06:02.000 2020-01-02 03:34:23.000 362 99263 49812.5 4981250 362 99263 49812.5 4981250 -32207 32728 5248.02 524802 -127 124 0.02 2 +363 101 10353 99264 1.09009 298.09009 149.59009 14959.009 1.09009 298.0901 149.59008 14959.00884 1.09009 298.09009 149.59008999999992 14959.00900 2020-01-01 2020-01-02 2020-01-01 00:06:03 2020-01-02 03:34:24 2020-01-01 00:06:03.000 2020-01-02 03:34:24.000 363 99264 49813.5 4981350 363 99264 49813.5 4981350 -32206 32729 5249.02 524902 -126 125 1.02 102 +364 101 10354 99265 1.09309 298.09309 149.59309 14959.3093 1.09309 298.09308 149.59309 14959.30915 1.09309 298.09309 149.59309000000007 14959.30900 2020-01-01 2020-01-02 2020-01-01 00:06:04 2020-01-02 03:34:25 2020-01-01 00:06:04.000 2020-01-02 03:34:25.000 364 99265 49814.5 4981450 364 99265 49814.5 4981450 -32205 32730 5250.02 525002 -125 126 2.02 202 +365 101 10355 99266 1.09609 298.09609 149.59609 14959.6096 1.09609 298.0961 149.59609 14959.6099 1.09609 298.09609 149.5960899999999 14959.60900 2020-01-01 2020-01-02 2020-01-01 00:06:05 2020-01-02 03:34:26 2020-01-01 00:06:05.000 2020-01-02 03:34:26.000 365 99266 49815.5 4981550 365 99266 49815.5 4981550 -32204 32731 5251.02 525102 -124 127 3.02 302 +366 101 10356 99267 1.09909 298.09909 149.59909 14959.9099 1.09909 298.0991 149.5991 14959.91003 1.09909 298.09909 149.59908999999993 14959.90900 2020-01-01 2020-01-02 2020-01-01 00:06:06 2020-01-02 03:34:27 2020-01-01 00:06:06.000 2020-01-02 03:34:27.000 366 99267 49816.5 4981650 366 99267 49816.5 4981650 -32203 32732 5252.02 525202 -128 127 1.46 146 367 101 10357 99268 1.1021 298.1021 149.6021 14960.21021 1.1021 298.1021 149.6021 14960.21015 1.10210 298.10210 149.60210000000004 14960.21000 2020-01-01 2020-01-02 2020-01-01 00:06:07 2020-01-02 03:34:28 2020-01-01 00:06:07.000 2020-01-02 03:34:28.000 367 99268 49817.5 4981750 367 99268 49817.5 4981750 -32202 32733 5253.02 525302 -128 123 -0.1 -10 -368 101 10358 99269 1.1051 298.1051 149.6051 14960.51051 1.1051 298.1051 149.6051 14960.51031 1.10510 298.10510 149.60510000000014 14960.51000 2020-01-01 2020-01-02 2020-01-01 00:06:08 2020-01-02 03:34:29 2020-01-01 00:06:08.000 2020-01-02 03:34:29.000 368 99269 49818.5 4981850 368 99269 49818.5 4981850 -32201 32734 5254.02 525402 -127 124 0.9 90 -369 101 10359 99270 1.1081 298.1081 149.6081 14960.81081 1.1081 298.1081 149.6081 14960.81062 1.10810 298.10810 149.60809999999995 14960.81000 2020-01-01 2020-01-02 2020-01-01 00:06:09 2020-01-02 03:34:30 2020-01-01 00:06:09.000 2020-01-02 03:34:30.000 369 99270 49819.5 4981950 369 99270 49819.5 4981950 -32200 32735 5255.02 525502 -126 125 1.9 190 -37 102 10027 99937 0.11111 300.11111 150.11111 15161.22222 0.11111 300.1111 150.11111 15161.22248 0.11111 300.11111 150.11110999999985 15161.22211 2020-01-01 2020-01-02 2020-01-01 00:00:37 2020-01-02 03:45:37 2020-01-01 00:00:37.000 2020-01-02 03:45:37.000 37 99937 49987 5048687 37 99937 49987 5048687 -32532 32403 4566.009900990099 461167 -127 124 -1.4356435643564356 -145 -370 101 10360 99271 1.11111 298.11111 149.61111 14961.11111 1.11111 298.1111 149.61111 14961.11137 1.11111 298.11111 149.61110999999985 14961.11100 2020-01-01 2020-01-02 2020-01-01 00:06:10 2020-01-02 03:34:31 2020-01-01 00:06:10.000 2020-01-02 03:34:31.000 370 99271 49820.5 4982050 370 99271 49820.5 4982050 -32199 32736 5256.02 525602 -125 126 2.9 290 -371 101 10361 99272 1.11411 298.11411 149.61411 14961.41141 1.11411 298.1141 149.61411 14961.4115 1.11411 298.11411 149.6141100000003 14961.41100 2020-01-01 2020-01-02 2020-01-01 00:06:11 2020-01-02 03:34:32 2020-01-01 00:06:11.000 2020-01-02 03:34:32.000 371 99272 49821.5 4982150 371 99272 49821.5 4982150 -32198 32737 5257.02 525702 -124 127 3.9 390 -372 101 10362 99273 1.11711 298.11711 149.61711 14961.71171 1.11711 298.11713 149.61711 14961.71165 1.11711 298.11711 149.61710999999985 14961.71100 2020-01-01 2020-01-02 2020-01-01 00:06:12 2020-01-02 03:34:33 2020-01-01 00:06:12.000 2020-01-02 03:34:33.000 372 99273 49822.5 4982250 372 99273 49822.5 4982250 -32197 32738 5258.02 525802 -128 127 2.34 234 -373 101 10363 99274 1.12012 298.12012 149.62012 14962.01201 1.12012 298.12012 149.62011 14962.01179 1.12012 298.12012 149.62011999999973 14962.01200 2020-01-01 2020-01-02 2020-01-01 00:06:13 2020-01-02 03:34:34 2020-01-01 00:06:13.000 2020-01-02 03:34:34.000 373 99274 49823.5 4982350 373 99274 49823.5 4982350 -32196 32739 5259.02 525902 -128 123 0.78 78 -374 101 10364 99275 1.12312 298.12312 149.62312 14962.31231 1.12312 298.1231 149.62312 14962.31208 1.12312 298.12312 149.62312000000017 14962.31200 2020-01-01 2020-01-02 2020-01-01 00:06:14 2020-01-02 03:34:35 2020-01-01 00:06:14.000 2020-01-02 03:34:35.000 374 99275 49824.5 4982450 374 99275 49824.5 4982450 -32195 32740 5260.02 526002 -127 124 1.78 178 -375 101 10365 99276 1.12612 298.12612 149.62612 14962.61261 1.12612 298.12613 149.62612 14962.61283 1.12612 298.12612 149.62612000000018 14962.61200 2020-01-01 2020-01-02 2020-01-01 00:06:15 2020-01-02 03:34:36 2020-01-01 00:06:15.000 2020-01-02 03:34:36.000 375 99276 49825.5 4982550 375 99276 49825.5 4982550 -32194 32741 5261.02 526102 -126 125 2.78 278 -376 101 10366 99277 1.12912 298.12912 149.62912 14962.91291 1.12912 298.12912 149.62912 14962.91297 1.12912 298.12912 149.62911999999974 14962.91200 2020-01-01 2020-01-02 2020-01-01 00:06:16 2020-01-02 03:34:37 2020-01-01 00:06:16.000 2020-01-02 03:34:37.000 376 99277 49826.5 4982650 376 99277 49826.5 4982650 -32193 32742 5262.02 526202 -125 126 3.78 378 +368 101 10358 99269 1.1051 298.1051 149.6051 14960.51051 1.1051 298.1051 149.6051 14960.51031 1.10510 298.10510 149.6051 14960.51000 2020-01-01 2020-01-02 2020-01-01 00:06:08 2020-01-02 03:34:29 2020-01-01 00:06:08.000 2020-01-02 03:34:29.000 368 99269 49818.5 4981850 368 99269 49818.5 4981850 -32201 32734 5254.02 525402 -127 124 0.9 90 +369 101 10359 99270 1.1081 298.1081 149.6081 14960.81081 1.1081 298.1081 149.6081 14960.81062 1.10810 298.10810 149.60809999999998 14960.81000 2020-01-01 2020-01-02 2020-01-01 00:06:09 2020-01-02 03:34:30 2020-01-01 00:06:09.000 2020-01-02 03:34:30.000 369 99270 49819.5 4981950 369 99270 49819.5 4981950 -32200 32735 5255.02 525502 -126 125 1.9 190 +37 102 10027 99937 0.11111 300.11111 150.11111 15161.22222 0.11111 300.1111 150.11111 15161.22248 0.11111 300.11111 150.1111099999999 15161.22211 2020-01-01 2020-01-02 2020-01-01 00:00:37 2020-01-02 03:45:37 2020-01-01 00:00:37.000 2020-01-02 03:45:37.000 37 99937 49987 5048687 37 99937 49987 5048687 -32532 32403 4566.009900990099 461167 -127 124 -1.4356435643564356 -145 +370 101 10360 99271 1.11111 298.11111 149.61111 14961.11111 1.11111 298.1111 149.61111 14961.11137 1.11111 298.11111 149.6111099999999 14961.11100 2020-01-01 2020-01-02 2020-01-01 00:06:10 2020-01-02 03:34:31 2020-01-01 00:06:10.000 2020-01-02 03:34:31.000 370 99271 49820.5 4982050 370 99271 49820.5 4982050 -32199 32736 5256.02 525602 -125 126 2.9 290 +371 101 10361 99272 1.11411 298.11411 149.61411 14961.41141 1.11411 298.1141 149.61411 14961.4115 1.11411 298.11411 149.61411000000015 14961.41100 2020-01-01 2020-01-02 2020-01-01 00:06:11 2020-01-02 03:34:32 2020-01-01 00:06:11.000 2020-01-02 03:34:32.000 371 99272 49821.5 4982150 371 99272 49821.5 4982150 -32198 32737 5257.02 525702 -124 127 3.9 390 +372 101 10362 99273 1.11711 298.11711 149.61711 14961.71171 1.11711 298.11713 149.61711 14961.71165 1.11711 298.11711 149.61710999999997 14961.71100 2020-01-01 2020-01-02 2020-01-01 00:06:12 2020-01-02 03:34:33 2020-01-01 00:06:12.000 2020-01-02 03:34:33.000 372 99273 49822.5 4982250 372 99273 49822.5 4982250 -32197 32738 5258.02 525802 -128 127 2.34 234 +373 101 10363 99274 1.12012 298.12012 149.62012 14962.01201 1.12012 298.12012 149.62011 14962.01179 1.12012 298.12012 149.62011999999987 14962.01200 2020-01-01 2020-01-02 2020-01-01 00:06:13 2020-01-02 03:34:34 2020-01-01 00:06:13.000 2020-01-02 03:34:34.000 373 99274 49823.5 4982350 373 99274 49823.5 4982350 -32196 32739 5259.02 525902 -128 123 0.78 78 +374 101 10364 99275 1.12312 298.12312 149.62312 14962.31231 1.12312 298.1231 149.62312 14962.31208 1.12312 298.12312 149.62312000000009 14962.31200 2020-01-01 2020-01-02 2020-01-01 00:06:14 2020-01-02 03:34:35 2020-01-01 00:06:14.000 2020-01-02 03:34:35.000 374 99275 49824.5 4982450 374 99275 49824.5 4982450 -32195 32740 5260.02 526002 -127 124 1.78 178 +375 101 10365 99276 1.12612 298.12612 149.62612 14962.61261 1.12612 298.12613 149.62612 14962.61283 1.12612 298.12612 149.62612000000004 14962.61200 2020-01-01 2020-01-02 2020-01-01 00:06:15 2020-01-02 03:34:36 2020-01-01 00:06:15.000 2020-01-02 03:34:36.000 375 99276 49825.5 4982550 375 99276 49825.5 4982550 -32194 32741 5261.02 526102 -126 125 2.78 278 +376 101 10366 99277 1.12912 298.12912 149.62912 14962.91291 1.12912 298.12912 149.62912 14962.91297 1.12912 298.12912 149.6291199999999 14962.91200 2020-01-01 2020-01-02 2020-01-01 00:06:16 2020-01-02 03:34:37 2020-01-01 00:06:16.000 2020-01-02 03:34:37.000 376 99277 49826.5 4982650 376 99277 49826.5 4982650 -32193 32742 5262.02 526202 -125 126 3.78 378 377 101 10367 99278 1.13213 298.13213 149.63213 14963.21321 1.13213 298.13214 149.63213 14963.21312 1.13213 298.13213 149.63213 14963.21300 2020-01-01 2020-01-02 2020-01-01 00:06:17 2020-01-02 03:34:38 2020-01-01 00:06:17.000 2020-01-02 03:34:38.000 377 99278 49827.5 4982750 377 99278 49827.5 4982750 -32192 32743 5263.02 526302 -124 127 4.78 478 -378 101 10368 99279 1.13513 298.13513 149.63513 14963.51351 1.13513 298.13513 149.63513 14963.51326 1.13513 298.13513 149.6351300000001 14963.51300 2020-01-01 2020-01-02 2020-01-01 00:06:18 2020-01-02 03:34:39 2020-01-01 00:06:18.000 2020-01-02 03:34:39.000 378 99279 49828.5 4982850 378 99279 49828.5 4982850 -32191 32744 5264.02 526402 -128 127 3.22 322 -379 101 10369 99280 1.13813 298.13813 149.63813 14963.81381 1.13813 298.13815 149.63814 14963.81401 1.13813 298.13813 149.63812999999985 14963.81300 2020-01-01 2020-01-02 2020-01-01 00:06:19 2020-01-02 03:34:40 2020-01-01 00:06:19.000 2020-01-02 03:34:40.000 379 99280 49829.5 4982950 379 99280 49829.5 4982950 -32190 32745 5265.02 526502 -128 127 1.66 166 -38 102 10028 99938 0.11411 300.11411 150.11411 15161.52552 0.11411 300.1141 150.11411 15161.52562 0.11411 300.11411 150.11411000000032 15161.52511 2020-01-01 2020-01-02 2020-01-01 00:00:38 2020-01-02 03:45:38 2020-01-01 00:00:38.000 2020-01-02 03:45:38.000 38 99938 49988 5048788 38 99938 49988 5048788 -32531 32404 4567.009900990099 461268 -126 125 -0.43564356435643564 -44 -380 101 10370 99281 1.14114 298.14114 149.64114 14964.11411 1.14114 298.14114 149.64114 14964.11431 1.14114 298.14114 149.64113999999984 14964.11400 2020-01-01 2020-01-02 2020-01-01 00:06:20 2020-01-02 03:34:41 2020-01-01 00:06:20.000 2020-01-02 03:34:41.000 380 99281 49830.5 4983050 380 99281 49830.5 4983050 -32189 32746 5266.02 526602 -128 124 0.1 10 -381 101 10371 99282 1.14414 298.14414 149.64414 14964.41441 1.14414 298.14413 149.64414 14964.41448 1.14414 298.14414 149.6441400000002 14964.41400 2020-01-01 2020-01-02 2020-01-01 00:06:21 2020-01-02 03:34:42 2020-01-01 00:06:21.000 2020-01-02 03:34:42.000 381 99282 49831.5 4983150 381 99282 49831.5 4983150 -32188 32747 5267.02 526702 -127 125 1.1 110 -382 101 10372 99283 1.14714 298.14714 149.64714 14964.71471 1.14714 298.14716 149.64714 14964.71459 1.14714 298.14714 149.64713999999975 14964.71400 2020-01-01 2020-01-02 2020-01-01 00:06:22 2020-01-02 03:34:43 2020-01-01 00:06:22.000 2020-01-02 03:34:43.000 382 99283 49832.5 4983250 382 99283 49832.5 4983250 -32187 32748 5268.02 526802 -126 126 2.1 210 -383 101 10373 99284 1.15015 298.15015 149.65015 14965.01501 1.15015 298.15015 149.65014 14965.01472 1.15015 298.15015 149.65014999999997 14965.01500 2020-01-01 2020-01-02 2020-01-01 00:06:23 2020-01-02 03:34:44 2020-01-01 00:06:23.000 2020-01-02 03:34:44.000 383 99284 49833.5 4983350 383 99284 49833.5 4983350 -32186 32749 5269.02 526902 -125 127 3.1 310 +378 101 10368 99279 1.13513 298.13513 149.63513 14963.51351 1.13513 298.13513 149.63513 14963.51326 1.13513 298.13513 149.63513000000003 14963.51300 2020-01-01 2020-01-02 2020-01-01 00:06:18 2020-01-02 03:34:39 2020-01-01 00:06:18.000 2020-01-02 03:34:39.000 378 99279 49828.5 4982850 378 99279 49828.5 4982850 -32191 32744 5264.02 526402 -128 127 3.22 322 +379 101 10369 99280 1.13813 298.13813 149.63813 14963.81381 1.13813 298.13815 149.63814 14963.81401 1.13813 298.13813 149.63812999999993 14963.81300 2020-01-01 2020-01-02 2020-01-01 00:06:19 2020-01-02 03:34:40 2020-01-01 00:06:19.000 2020-01-02 03:34:40.000 379 99280 49829.5 4982950 379 99280 49829.5 4982950 -32190 32745 5265.02 526502 -128 127 1.66 166 +38 102 10028 99938 0.11411 300.11411 150.11411 15161.52552 0.11411 300.1141 150.11411 15161.52562 0.11411 300.11411 150.11411000000012 15161.52511 2020-01-01 2020-01-02 2020-01-01 00:00:38 2020-01-02 03:45:38 2020-01-01 00:00:38.000 2020-01-02 03:45:38.000 38 99938 49988 5048788 38 99938 49988 5048788 -32531 32404 4567.009900990099 461268 -126 125 -0.43564356435643564 -44 +380 101 10370 99281 1.14114 298.14114 149.64114 14964.11411 1.14114 298.14114 149.64114 14964.11431 1.14114 298.14114 149.64113999999995 14964.11400 2020-01-01 2020-01-02 2020-01-01 00:06:20 2020-01-02 03:34:41 2020-01-01 00:06:20.000 2020-01-02 03:34:41.000 380 99281 49830.5 4983050 380 99281 49830.5 4983050 -32189 32746 5266.02 526602 -128 124 0.1 10 +381 101 10371 99282 1.14414 298.14414 149.64414 14964.41441 1.14414 298.14413 149.64414 14964.41448 1.14414 298.14414 149.64414000000005 14964.41400 2020-01-01 2020-01-02 2020-01-01 00:06:21 2020-01-02 03:34:42 2020-01-01 00:06:21.000 2020-01-02 03:34:42.000 381 99282 49831.5 4983150 381 99282 49831.5 4983150 -32188 32747 5267.02 526702 -127 125 1.1 110 +382 101 10372 99283 1.14714 298.14714 149.64714 14964.71471 1.14714 298.14716 149.64714 14964.71459 1.14714 298.14714 149.64713999999992 14964.71400 2020-01-01 2020-01-02 2020-01-01 00:06:22 2020-01-02 03:34:43 2020-01-01 00:06:22.000 2020-01-02 03:34:43.000 382 99283 49832.5 4983250 382 99283 49832.5 4983250 -32187 32748 5268.02 526802 -126 126 2.1 210 +383 101 10373 99284 1.15015 298.15015 149.65015 14965.01501 1.15015 298.15015 149.65014 14965.01472 1.15015 298.15015 149.65015 14965.01500 2020-01-01 2020-01-02 2020-01-01 00:06:23 2020-01-02 03:34:44 2020-01-01 00:06:23.000 2020-01-02 03:34:44.000 383 99284 49833.5 4983350 383 99284 49833.5 4983350 -32186 32749 5269.02 526902 -125 127 3.1 310 384 101 10374 99285 1.15315 298.15315 149.65315 14965.31531 1.15315 298.15317 149.65315 14965.31547 1.15315 298.15315 149.65315000000007 14965.31500 2020-01-01 2020-01-02 2020-01-01 00:06:24 2020-01-02 03:34:45 2020-01-01 00:06:24.000 2020-01-02 03:34:45.000 384 99285 49834.5 4983450 384 99285 49834.5 4983450 -32185 32750 5270.02 527002 -128 127 1.54 154 -385 101 10375 99286 1.15615 298.15615 149.65615 14965.61561 1.15615 298.15616 149.65615 14965.61578 1.15615 298.15615 149.6561500000001 14965.61500 2020-01-01 2020-01-02 2020-01-01 00:06:25 2020-01-02 03:34:46 2020-01-01 00:06:25.000 2020-01-02 03:34:46.000 385 99286 49835.5 4983550 385 99286 49835.5 4983550 -32184 32751 5271.02 527102 -128 127 -0.02 -2 -386 101 10376 99287 1.15915 298.15915 149.65915 14965.91591 1.15915 298.15915 149.65915 14965.91594 1.15915 298.15915 149.65914999999998 14965.91500 2020-01-01 2020-01-02 2020-01-01 00:06:26 2020-01-02 03:34:47 2020-01-01 00:06:26.000 2020-01-02 03:34:47.000 386 99287 49836.5 4983650 386 99287 49836.5 4983650 -32183 32752 5272.02 527202 -128 123 -1.58 -158 -387 101 10377 99288 1.16216 298.16216 149.66216 14966.21621 1.16216 298.16217 149.66216 14966.21606 1.16216 298.16216 149.6621599999999 14966.21600 2020-01-01 2020-01-02 2020-01-01 00:06:27 2020-01-02 03:34:48 2020-01-01 00:06:27.000 2020-01-02 03:34:48.000 387 99288 49837.5 4983750 387 99288 49837.5 4983750 -32182 32753 5273.02 527302 -127 124 -0.58 -58 +385 101 10375 99286 1.15615 298.15615 149.65615 14965.61561 1.15615 298.15616 149.65615 14965.61578 1.15615 298.15615 149.65615000000003 14965.61500 2020-01-01 2020-01-02 2020-01-01 00:06:25 2020-01-02 03:34:46 2020-01-01 00:06:25.000 2020-01-02 03:34:46.000 385 99286 49835.5 4983550 385 99286 49835.5 4983550 -32184 32751 5271.02 527102 -128 127 -0.02 -2 +386 101 10376 99287 1.15915 298.15915 149.65915 14965.91591 1.15915 298.15915 149.65915 14965.91594 1.15915 298.15915 149.65915 14965.91500 2020-01-01 2020-01-02 2020-01-01 00:06:26 2020-01-02 03:34:47 2020-01-01 00:06:26.000 2020-01-02 03:34:47.000 386 99287 49836.5 4983650 386 99287 49836.5 4983650 -32183 32752 5272.02 527202 -128 123 -1.58 -158 +387 101 10377 99288 1.16216 298.16216 149.66216 14966.21621 1.16216 298.16217 149.66216 14966.21606 1.16216 298.16216 149.66215999999994 14966.21600 2020-01-01 2020-01-02 2020-01-01 00:06:27 2020-01-02 03:34:48 2020-01-01 00:06:27.000 2020-01-02 03:34:48.000 387 99288 49837.5 4983750 387 99288 49837.5 4983750 -32182 32753 5273.02 527302 -127 124 -0.58 -58 388 101 10378 99289 1.16516 298.16516 149.66516 14966.51651 1.16516 298.16516 149.66516 14966.51636 1.16516 298.16516 149.66516 14966.51600 2020-01-01 2020-01-02 2020-01-01 00:06:28 2020-01-02 03:34:49 2020-01-01 00:06:28.000 2020-01-02 03:34:49.000 388 99289 49838.5 4983850 388 99289 49838.5 4983850 -32181 32754 5274.02 527402 -126 125 0.42 42 -389 101 10379 99290 1.16816 298.16816 149.66816 14966.81681 1.16816 298.16818 149.66816 14966.81695 1.16816 298.16816 149.66815999999986 14966.81600 2020-01-01 2020-01-02 2020-01-01 00:06:29 2020-01-02 03:34:50 2020-01-01 00:06:29.000 2020-01-02 03:34:50.000 389 99290 49839.5 4983950 389 99290 49839.5 4983950 -32180 32755 5275.02 527502 -125 126 1.42 142 -39 102 10029 99939 0.11711 300.11711 150.11711 15161.82882 0.11711 300.11713 150.11711 15161.82876 0.11711 300.11711 150.11710999999983 15161.82811 2020-01-01 2020-01-02 2020-01-01 00:00:39 2020-01-02 03:45:39 2020-01-01 00:00:39.000 2020-01-02 03:45:39.000 39 99939 49989 5048889 39 99939 49989 5048889 -32530 32405 4568.009900990099 461369 -125 126 0.5643564356435643 57 -390 101 10380 99291 1.17117 298.17117 149.67117 14967.11711 1.17117 298.17117 149.67117 14967.11725 1.17117 298.17117 149.67116999999976 14967.11700 2020-01-01 2020-01-02 2020-01-01 00:06:30 2020-01-02 03:34:51 2020-01-01 00:06:30.000 2020-01-02 03:34:51.000 390 99291 49840.5 4984050 390 99291 49840.5 4984050 -32179 32756 5276.02 527602 -124 127 2.42 242 +389 101 10379 99290 1.16816 298.16816 149.66816 14966.81681 1.16816 298.16818 149.66816 14966.81695 1.16816 298.16816 149.66815999999997 14966.81600 2020-01-01 2020-01-02 2020-01-01 00:06:29 2020-01-02 03:34:50 2020-01-01 00:06:29.000 2020-01-02 03:34:50.000 389 99290 49839.5 4983950 389 99290 49839.5 4983950 -32180 32755 5275.02 527502 -125 126 1.42 142 +39 102 10029 99939 0.11711 300.11711 150.11711 15161.82882 0.11711 300.11713 150.11711 15161.82876 0.11711 300.11711 150.11710999999997 15161.82811 2020-01-01 2020-01-02 2020-01-01 00:00:39 2020-01-02 03:45:39 2020-01-01 00:00:39.000 2020-01-02 03:45:39.000 39 99939 49989 5048889 39 99939 49989 5048889 -32530 32405 4568.009900990099 461369 -125 126 0.5643564356435643 57 +390 101 10380 99291 1.17117 298.17117 149.67117 14967.11711 1.17117 298.17117 149.67117 14967.11725 1.17117 298.17117 149.6711699999999 14967.11700 2020-01-01 2020-01-02 2020-01-01 00:06:30 2020-01-02 03:34:51 2020-01-01 00:06:30.000 2020-01-02 03:34:51.000 390 99291 49840.5 4984050 390 99291 49840.5 4984050 -32179 32756 5276.02 527602 -124 127 2.42 242 391 101 10381 99292 1.17417 298.17417 149.67417 14967.41741 1.17417 298.17416 149.67417 14967.41741 1.17417 298.17417 149.67417000000012 14967.41700 2020-01-01 2020-01-02 2020-01-01 00:06:31 2020-01-02 03:34:52 2020-01-01 00:06:31.000 2020-01-02 03:34:52.000 391 99292 49841.5 4984150 391 99292 49841.5 4984150 -32178 32757 5277.02 527702 -128 127 0.86 86 -392 101 10382 99293 1.17717 298.17717 149.67717 14967.71771 1.17717 298.1772 149.67717 14967.71753 1.17717 298.17717 149.67717000000022 14967.71700 2020-01-01 2020-01-02 2020-01-01 00:06:32 2020-01-02 03:34:53 2020-01-01 00:06:32.000 2020-01-02 03:34:53.000 392 99293 49842.5 4984250 392 99293 49842.5 4984250 -32177 32758 5278.02 527802 -128 123 -0.7 -70 -393 101 10383 99294 1.18018 298.18018 149.68018 14968.01801 1.18018 298.18018 149.68017 14968.01782 1.18018 298.18018 149.68017999999987 14968.01800 2020-01-01 2020-01-02 2020-01-01 00:06:33 2020-01-02 03:34:54 2020-01-01 00:06:33.000 2020-01-02 03:34:54.000 393 99294 49843.5 4984350 393 99294 49843.5 4984350 -32176 32759 5279.02 527902 -127 124 0.3 30 +392 101 10382 99293 1.17717 298.17717 149.67717 14967.71771 1.17717 298.1772 149.67717 14967.71753 1.17717 298.17717 149.67717000000007 14967.71700 2020-01-01 2020-01-02 2020-01-01 00:06:32 2020-01-02 03:34:53 2020-01-01 00:06:32.000 2020-01-02 03:34:53.000 392 99293 49842.5 4984250 392 99293 49842.5 4984250 -32177 32758 5278.02 527802 -128 123 -0.7 -70 +393 101 10383 99294 1.18018 298.18018 149.68018 14968.01801 1.18018 298.18018 149.68017 14968.01782 1.18018 298.18018 149.68017999999995 14968.01800 2020-01-01 2020-01-02 2020-01-01 00:06:33 2020-01-02 03:34:54 2020-01-01 00:06:33.000 2020-01-02 03:34:54.000 393 99294 49843.5 4984350 393 99294 49843.5 4984350 -32176 32759 5279.02 527902 -127 124 0.3 30 394 101 10384 99295 1.18318 298.18318 149.68318 14968.31831 1.18318 298.1832 149.68318 14968.31842 1.18318 298.18318 149.68318 14968.31800 2020-01-01 2020-01-02 2020-01-01 00:06:34 2020-01-02 03:34:55 2020-01-01 00:06:34.000 2020-01-02 03:34:55.000 394 99295 49844.5 4984450 394 99295 49844.5 4984450 -32175 32760 5280.02 528002 -126 125 1.3 130 -395 101 10385 99296 1.18618 298.18618 149.68618 14968.61861 1.18618 298.1862 149.68618 14968.61875 1.18618 298.18618 149.68618000000012 14968.61800 2020-01-01 2020-01-02 2020-01-01 00:06:35 2020-01-02 03:34:56 2020-01-01 00:06:35.000 2020-01-02 03:34:56.000 395 99296 49845.5 4984550 395 99296 49845.5 4984550 -32174 32761 5281.02 528102 -125 126 2.3 230 -396 101 10386 99297 1.18918 298.18918 149.68918 14968.91891 1.18918 298.18918 149.68918 14968.91889 1.18918 298.18918 149.68917999999988 14968.91800 2020-01-01 2020-01-02 2020-01-01 00:06:36 2020-01-02 03:34:57 2020-01-01 00:06:36.000 2020-01-02 03:34:57.000 396 99297 49846.5 4984650 396 99297 49846.5 4984650 -32173 32762 5282.02 528202 -124 127 3.3 330 -397 101 10387 99298 1.19219 298.19219 149.69219 14969.21921 1.19219 298.1922 149.69219 14969.21964 1.19219 298.19219 149.69218999999987 14969.21900 2020-01-01 2020-01-02 2020-01-01 00:06:37 2020-01-02 03:34:58 2020-01-01 00:06:37.000 2020-01-02 03:34:58.000 397 99298 49847.5 4984750 397 99298 49847.5 4984750 -32172 32763 5283.02 528302 -128 127 1.74 174 -398 101 10388 99299 1.19519 298.19519 149.69519 14969.51951 1.19519 298.1952 149.69519 14969.51929 1.19519 298.19519 149.69519000000022 14969.51900 2020-01-01 2020-01-02 2020-01-01 00:06:38 2020-01-02 03:34:59 2020-01-01 00:06:38.000 2020-01-02 03:34:59.000 398 99299 49848.5 4984850 398 99299 49848.5 4984850 -32171 32764 5284.02 528402 -128 123 0.18 18 -399 101 10389 99300 1.19819 298.19819 149.69819 14969.81981 1.19819 298.1982 149.69819 14969.81989 1.19819 298.19819 149.69818999999978 14969.81900 2020-01-01 2020-01-02 2020-01-01 00:06:39 2020-01-02 03:35:00 2020-01-01 00:06:39.000 2020-01-02 03:35:00.000 399 99300 49849.5 4984950 399 99300 49849.5 4984950 -32170 32765 5285.02 528502 -127 124 1.18 118 -4 102 1003 9994 0.01201 300.01201 150.01201 15151.21321 0.01201 300.01202 150.01201 15151.21318 0.01201 300.01201 150.01201000000026 15151.21301 2020-01-01 2020-01-02 2020-01-01 00:00:04 2020-01-02 03:45:04 2020-01-01 00:00:04.000 2020-01-02 03:45:04.000 4 99904 49954 5045354 4 99904 49954 5045354 -32565 32370 4533.009900990099 457834 -128 127 -1.4851485148514851 -150 -40 102 10030 99940 0.12012 300.12012 150.12012 15162.13213 0.12012 300.12012 150.12011 15162.13191 0.12012 300.12012 150.1201199999997 15162.13212 2020-01-01 2020-01-02 2020-01-01 00:00:40 2020-01-02 03:45:40 2020-01-01 00:00:40.000 2020-01-02 03:45:40.000 40 99940 49990 5048990 40 99940 49990 5048990 -32529 32406 4569.009900990099 461470 -124 127 1.5643564356435644 158 +395 101 10385 99296 1.18618 298.18618 149.68618 14968.61861 1.18618 298.1862 149.68618 14968.61875 1.18618 298.18618 149.68618000000004 14968.61800 2020-01-01 2020-01-02 2020-01-01 00:06:35 2020-01-02 03:34:56 2020-01-01 00:06:35.000 2020-01-02 03:34:56.000 395 99296 49845.5 4984550 395 99296 49845.5 4984550 -32174 32761 5281.02 528102 -125 126 2.3 230 +396 101 10386 99297 1.18918 298.18918 149.68918 14968.91891 1.18918 298.18918 149.68918 14968.91889 1.18918 298.18918 149.68917999999996 14968.91800 2020-01-01 2020-01-02 2020-01-01 00:06:36 2020-01-02 03:34:57 2020-01-01 00:06:36.000 2020-01-02 03:34:57.000 396 99297 49846.5 4984650 396 99297 49846.5 4984650 -32173 32762 5282.02 528202 -124 127 3.3 330 +397 101 10387 99298 1.19219 298.19219 149.69219 14969.21921 1.19219 298.1922 149.69219 14969.21964 1.19219 298.19219 149.69218999999993 14969.21900 2020-01-01 2020-01-02 2020-01-01 00:06:37 2020-01-02 03:34:58 2020-01-01 00:06:37.000 2020-01-02 03:34:58.000 397 99298 49847.5 4984750 397 99298 49847.5 4984750 -32172 32763 5283.02 528302 -128 127 1.74 174 +398 101 10388 99299 1.19519 298.19519 149.69519 14969.51951 1.19519 298.1952 149.69519 14969.51929 1.19519 298.19519 149.69519000000008 14969.51900 2020-01-01 2020-01-02 2020-01-01 00:06:38 2020-01-02 03:34:59 2020-01-01 00:06:38.000 2020-01-02 03:34:59.000 398 99299 49848.5 4984850 398 99299 49848.5 4984850 -32171 32764 5284.02 528402 -128 123 0.18 18 +399 101 10389 99300 1.19819 298.19819 149.69819 14969.81981 1.19819 298.1982 149.69819 14969.81989 1.19819 298.19819 149.69818999999993 14969.81900 2020-01-01 2020-01-02 2020-01-01 00:06:39 2020-01-02 03:35:00 2020-01-01 00:06:39.000 2020-01-02 03:35:00.000 399 99300 49849.5 4984950 399 99300 49849.5 4984950 -32170 32765 5285.02 528502 -127 124 1.18 118 +4 102 1003 9994 0.01201 300.01201 150.01201 15151.21321 0.01201 300.01202 150.01201 15151.21318 0.01201 300.01201 150.0120100000001 15151.21301 2020-01-01 2020-01-02 2020-01-01 00:00:04 2020-01-02 03:45:04 2020-01-01 00:00:04.000 2020-01-02 03:45:04.000 4 99904 49954 5045354 4 99904 49954 5045354 -32565 32370 4533.009900990099 457834 -128 127 -1.4851485148514851 -150 +40 102 10030 99940 0.12012 300.12012 150.12012 15162.13213 0.12012 300.12012 150.12011 15162.13191 0.12012 300.12012 150.12011999999987 15162.13212 2020-01-01 2020-01-02 2020-01-01 00:00:40 2020-01-02 03:45:40 2020-01-01 00:00:40.000 2020-01-02 03:45:40.000 40 99940 49990 5048990 40 99940 49990 5048990 -32529 32406 4569.009900990099 461470 -124 127 1.5643564356435644 158 400 101 10390 99301 1.2012 298.2012 149.7012 14970.12012 1.2012 298.2012 149.7012 14970.12022 1.20120 298.20120 149.7012 14970.12000 2020-01-01 2020-01-02 2020-01-01 00:06:40 2020-01-02 03:35:01 2020-01-01 00:06:40.000 2020-01-02 03:35:01.000 400 99301 49850.5 4985050 400 99301 49850.5 4985050 -32169 32766 5286.02 528602 -126 125 2.18 218 -401 101 10391 99302 1.2042 298.2042 149.7042 14970.42042 1.2042 298.2042 149.7042 14970.42035 1.20420 298.20420 149.7042000000001 14970.42000 2020-01-01 2020-01-02 2020-01-01 00:06:41 2020-01-02 03:35:02 2020-01-01 00:06:41.000 2020-01-02 03:35:02.000 401 99302 49851.5 4985150 401 99302 49851.5 4985150 -32168 32767 5287.02 528702 -125 126 3.18 318 -402 101 10392 99303 1.2072 298.2072 149.7072 14970.72072 1.2072 298.2072 149.70721 14970.72111 1.20720 298.20720 149.70720000000014 14970.72000 2020-01-01 2020-01-02 2020-01-01 00:06:42 2020-01-02 03:35:03 2020-01-01 00:06:42.000 2020-01-02 03:35:03.000 402 99303 49852.5 4985250 402 99303 49852.5 4985250 -32768 32370 4632.66 463266 -124 127 4.18 418 -403 101 10393 99304 1.21021 298.21021 149.71021 14971.02102 1.21021 298.2102 149.7102 14971.02077 1.21021 298.21021 149.71020999999988 14971.02100 2020-01-01 2020-01-02 2020-01-01 00:06:43 2020-01-02 03:35:04 2020-01-01 00:06:43.000 2020-01-02 03:35:04.000 403 99304 49853.5 4985350 403 99304 49853.5 4985350 -32767 32371 4633.66 463366 -128 127 2.62 262 +401 101 10391 99302 1.2042 298.2042 149.7042 14970.42042 1.2042 298.2042 149.7042 14970.42035 1.20420 298.20420 149.70420000000004 14970.42000 2020-01-01 2020-01-02 2020-01-01 00:06:41 2020-01-02 03:35:02 2020-01-01 00:06:41.000 2020-01-02 03:35:02.000 401 99302 49851.5 4985150 401 99302 49851.5 4985150 -32168 32767 5287.02 528702 -125 126 3.18 318 +402 101 10392 99303 1.2072 298.2072 149.7072 14970.72072 1.2072 298.2072 149.70721 14970.72111 1.20720 298.20720 149.70720000000003 14970.72000 2020-01-01 2020-01-02 2020-01-01 00:06:42 2020-01-02 03:35:03 2020-01-01 00:06:42.000 2020-01-02 03:35:03.000 402 99303 49852.5 4985250 402 99303 49852.5 4985250 -32768 32370 4632.66 463266 -124 127 4.18 418 +403 101 10393 99304 1.21021 298.21021 149.71021 14971.02102 1.21021 298.2102 149.7102 14971.02077 1.21021 298.21021 149.71020999999996 14971.02100 2020-01-01 2020-01-02 2020-01-01 00:06:43 2020-01-02 03:35:04 2020-01-01 00:06:43.000 2020-01-02 03:35:04.000 403 99304 49853.5 4985350 403 99304 49853.5 4985350 -32767 32371 4633.66 463366 -128 127 2.62 262 404 101 10394 99305 1.21321 298.21321 149.71321 14971.32132 1.21321 298.21323 149.71321 14971.32139 1.21321 298.21321 149.71320999999992 14971.32100 2020-01-01 2020-01-02 2020-01-01 00:06:44 2020-01-02 03:35:05 2020-01-01 00:06:44.000 2020-01-02 03:35:05.000 404 99305 49854.5 4985450 404 99305 49854.5 4985450 -32766 32372 4634.66 463466 -128 127 1.06 106 -405 101 10395 99306 1.21621 298.21621 149.71621 14971.62162 1.21621 298.21622 149.71621 14971.62169 1.21621 298.21621 149.71621000000002 14971.62100 2020-01-01 2020-01-02 2020-01-01 00:06:45 2020-01-02 03:35:06 2020-01-01 00:06:45.000 2020-01-02 03:35:06.000 405 99306 49855.5 4985550 405 99306 49855.5 4985550 -32765 32373 4635.66 463566 -128 124 -0.5 -50 -406 101 10396 99307 1.21921 298.21921 149.71921 14971.92192 1.21921 298.2192 149.71921 14971.92199 1.21921 298.21921 149.7192099999999 14971.92100 2020-01-01 2020-01-02 2020-01-01 00:06:46 2020-01-02 03:35:07 2020-01-01 00:06:46.000 2020-01-02 03:35:07.000 406 99307 49856.5 4985650 406 99307 49856.5 4985650 -32764 32374 4636.66 463666 -127 125 0.5 50 -407 101 10397 99308 1.22222 298.22222 149.72222 14972.22222 1.22222 298.22223 149.72222 14972.22257 1.22222 298.22222 149.7222199999998 14972.22200 2020-01-01 2020-01-02 2020-01-01 00:06:47 2020-01-02 03:35:08 2020-01-01 00:06:47.000 2020-01-02 03:35:08.000 407 99308 49857.5 4985750 407 99308 49857.5 4985750 -32763 32375 4637.66 463766 -126 126 1.5 150 -408 101 10398 99309 1.22522 298.22522 149.72522 14972.52252 1.22522 298.22522 149.72522 14972.52224 1.22522 298.22522 149.72522000000015 14972.52200 2020-01-01 2020-01-02 2020-01-01 00:06:48 2020-01-02 03:35:09 2020-01-01 00:06:48.000 2020-01-02 03:35:09.000 408 99309 49858.5 4985850 408 99309 49858.5 4985850 -32762 32376 4638.66 463866 -125 127 2.5 250 -409 101 10399 99310 1.22822 298.22822 149.72822 14972.82282 1.22822 298.22824 149.72822 14972.82286 1.22822 298.22822 149.72822000000025 14972.82200 2020-01-01 2020-01-02 2020-01-01 00:06:49 2020-01-02 03:35:10 2020-01-01 00:06:49.000 2020-01-02 03:35:10.000 409 99310 49859.5 4985950 409 99310 49859.5 4985950 -32761 32377 4639.66 463966 -128 127 0.94 94 -41 102 10031 99941 0.12312 300.12312 150.12312 15162.43543 0.12312 300.1231 150.12312 15162.43521 0.12312 300.12312 150.12312000000017 15162.43512 2020-01-01 2020-01-02 2020-01-01 00:00:41 2020-01-02 03:45:41 2020-01-01 00:00:41.000 2020-01-02 03:45:41.000 41 99941 49991 5049091 41 99941 49991 5049091 -32528 32407 4570.009900990099 461571 -128 127 0.0297029702970297 3 -410 101 10400 99311 1.23123 298.23123 149.73123 14973.12312 1.23123 298.23123 149.73123 14973.12316 1.23123 298.23123 149.7312299999999 14973.12300 2020-01-01 2020-01-02 2020-01-01 00:06:50 2020-01-02 03:35:11 2020-01-01 00:06:50.000 2020-01-02 03:35:11.000 410 99311 49860.5 4986050 410 99311 49860.5 4986050 -32760 32378 4640.66 464066 -128 127 -0.62 -62 -411 101 10401 99312 1.23423 298.23423 149.73423 14973.42342 1.23423 298.23422 149.73423 14973.42345 1.23423 298.23423 149.73423000000003 14973.42300 2020-01-01 2020-01-02 2020-01-01 00:06:51 2020-01-02 03:35:12 2020-01-01 00:06:51.000 2020-01-02 03:35:12.000 411 99312 49861.5 4986150 411 99312 49861.5 4986150 -32759 32379 4641.66 464166 -128 123 -2.18 -218 -412 101 10402 99313 1.23723 298.23723 149.73723 14973.72372 1.23723 298.23724 149.73724 14973.72405 1.23723 298.23723 149.73723000000015 14973.72300 2020-01-01 2020-01-02 2020-01-01 00:06:52 2020-01-02 03:35:13 2020-01-01 00:06:52.000 2020-01-02 03:35:13.000 412 99313 49862.5 4986250 412 99313 49862.5 4986250 -32758 32380 4642.66 464266 -127 124 -1.18 -118 -413 101 10403 99314 1.24024 298.24024 149.74024 14974.02402 1.24024 298.24023 149.74023 14974.02374 1.24024 298.24024 149.7402399999998 14974.02400 2020-01-01 2020-01-02 2020-01-01 00:06:53 2020-01-02 03:35:14 2020-01-01 00:06:53.000 2020-01-02 03:35:14.000 413 99314 49863.5 4986350 413 99314 49863.5 4986350 -32757 32381 4643.66 464366 -126 125 -0.18 -18 -414 101 10404 99315 1.24324 298.24324 149.74324 14974.32432 1.24324 298.24326 149.74324 14974.32433 1.24324 298.24324 149.74323999999982 14974.32400 2020-01-01 2020-01-02 2020-01-01 00:06:54 2020-01-02 03:35:15 2020-01-01 00:06:54.000 2020-01-02 03:35:15.000 414 99315 49864.5 4986450 414 99315 49864.5 4986450 -32756 32382 4644.66 464466 -125 126 0.82 82 -415 101 10405 99316 1.24624 298.24624 149.74624 14974.62462 1.24624 298.24625 149.74624 14974.62463 1.24624 298.24624 149.74624000000026 14974.62400 2020-01-01 2020-01-02 2020-01-01 00:06:55 2020-01-02 03:35:16 2020-01-01 00:06:55.000 2020-01-02 03:35:16.000 415 99316 49865.5 4986550 415 99316 49865.5 4986550 -32755 32383 4645.66 464566 -124 127 1.82 182 -416 101 10406 99317 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74923999999982 14974.92400 2020-01-01 2020-01-02 2020-01-01 00:06:56 2020-01-02 03:35:17 2020-01-01 00:06:56.000 2020-01-02 03:35:17.000 416 99317 49866.5 4986650 416 99317 49866.5 4986650 -32754 32384 4646.66 464666 -128 127 0.26 26 -417 101 10407 99318 1.25225 298.25225 149.75225 14975.22522 1.25225 298.25226 149.75225 14975.22552 1.25225 298.25225 149.7522499999997 14975.22500 2020-01-01 2020-01-02 2020-01-01 00:06:57 2020-01-02 03:35:18 2020-01-01 00:06:57.000 2020-01-02 03:35:18.000 417 99318 49867.5 4986750 417 99318 49867.5 4986750 -32753 32385 4647.66 464766 -128 123 -1.3 -130 -418 101 10408 99319 1.25525 298.25525 149.75525 14975.52552 1.25525 298.25525 149.75525 14975.52521 1.25525 298.25525 149.75525000000013 14975.52500 2020-01-01 2020-01-02 2020-01-01 00:06:58 2020-01-02 03:35:19 2020-01-01 00:06:58.000 2020-01-02 03:35:19.000 418 99319 49868.5 4986850 418 99319 49868.5 4986850 -32752 32386 4648.66 464866 -127 124 -0.3 -30 -419 101 10409 99320 1.25825 298.25825 149.75825 14975.82582 1.25825 298.25827 149.75825 14975.8258 1.25825 298.25825 149.75825000000015 14975.82500 2020-01-01 2020-01-02 2020-01-01 00:06:59 2020-01-02 03:35:20 2020-01-01 00:06:59.000 2020-01-02 03:35:20.000 419 99320 49869.5 4986950 419 99320 49869.5 4986950 -32751 32387 4649.66 464966 -126 125 0.7 70 -42 102 10032 99942 0.12612 300.12612 150.12612 15162.73873 0.12612 300.12613 150.12612 15162.73896 0.12612 300.12612 150.1261200000002 15162.73812 2020-01-01 2020-01-02 2020-01-01 00:00:42 2020-01-02 03:45:42 2020-01-01 00:00:42.000 2020-01-02 03:45:42.000 42 99942 49992 5049192 42 99942 49992 5049192 -32527 32408 4571.009900990099 461672 -128 127 -1.504950495049505 -152 -420 101 10410 99321 1.26126 298.26126 149.76126 14976.12612 1.26126 298.26126 149.76126 14976.12609 1.26126 298.26126 149.7612599999999 14976.12600 2020-01-01 2020-01-02 2020-01-01 00:07:00 2020-01-02 03:35:21 2020-01-01 00:07:00.000 2020-01-02 03:35:21.000 420 99321 49870.5 4987050 420 99321 49870.5 4987050 -32750 32388 4650.66 465066 -125 126 1.7 170 +405 101 10395 99306 1.21621 298.21621 149.71621 14971.62162 1.21621 298.21622 149.71621 14971.62169 1.21621 298.21621 149.71621 14971.62100 2020-01-01 2020-01-02 2020-01-01 00:06:45 2020-01-02 03:35:06 2020-01-01 00:06:45.000 2020-01-02 03:35:06.000 405 99306 49855.5 4985550 405 99306 49855.5 4985550 -32765 32373 4635.66 463566 -128 124 -0.5 -50 +406 101 10396 99307 1.21921 298.21921 149.71921 14971.92192 1.21921 298.2192 149.71921 14971.92199 1.21921 298.21921 149.71920999999998 14971.92100 2020-01-01 2020-01-02 2020-01-01 00:06:46 2020-01-02 03:35:07 2020-01-01 00:06:46.000 2020-01-02 03:35:07.000 406 99307 49856.5 4985650 406 99307 49856.5 4985650 -32764 32374 4636.66 463666 -127 125 0.5 50 +407 101 10397 99308 1.22222 298.22222 149.72222 14972.22222 1.22222 298.22223 149.72222 14972.22257 1.22222 298.22222 149.72221999999994 14972.22200 2020-01-01 2020-01-02 2020-01-01 00:06:47 2020-01-02 03:35:08 2020-01-01 00:06:47.000 2020-01-02 03:35:08.000 407 99308 49857.5 4985750 407 99308 49857.5 4985750 -32763 32375 4637.66 463766 -126 126 1.5 150 +408 101 10398 99309 1.22522 298.22522 149.72522 14972.52252 1.22522 298.22522 149.72522 14972.52224 1.22522 298.22522 149.72522000000004 14972.52200 2020-01-01 2020-01-02 2020-01-01 00:06:48 2020-01-02 03:35:09 2020-01-01 00:06:48.000 2020-01-02 03:35:09.000 408 99309 49858.5 4985850 408 99309 49858.5 4985850 -32762 32376 4638.66 463866 -125 127 2.5 250 +409 101 10399 99310 1.22822 298.22822 149.72822 14972.82282 1.22822 298.22824 149.72822 14972.82286 1.22822 298.22822 149.7282200000001 14972.82200 2020-01-01 2020-01-02 2020-01-01 00:06:49 2020-01-02 03:35:10 2020-01-01 00:06:49.000 2020-01-02 03:35:10.000 409 99310 49859.5 4985950 409 99310 49859.5 4985950 -32761 32377 4639.66 463966 -128 127 0.94 94 +41 102 10031 99941 0.12312 300.12312 150.12312 15162.43543 0.12312 300.1231 150.12312 15162.43521 0.12312 300.12312 150.12312000000009 15162.43512 2020-01-01 2020-01-02 2020-01-01 00:00:41 2020-01-02 03:45:41 2020-01-01 00:00:41.000 2020-01-02 03:45:41.000 41 99941 49991 5049091 41 99941 49991 5049091 -32528 32407 4570.009900990099 461571 -128 127 0.0297029702970297 3 +410 101 10400 99311 1.23123 298.23123 149.73123 14973.12312 1.23123 298.23123 149.73123 14973.12316 1.23123 298.23123 149.73122999999995 14973.12300 2020-01-01 2020-01-02 2020-01-01 00:06:50 2020-01-02 03:35:11 2020-01-01 00:06:50.000 2020-01-02 03:35:11.000 410 99311 49860.5 4986050 410 99311 49860.5 4986050 -32760 32378 4640.66 464066 -128 127 -0.62 -62 +411 101 10401 99312 1.23423 298.23423 149.73423 14973.42342 1.23423 298.23422 149.73423 14973.42345 1.23423 298.23423 149.73423 14973.42300 2020-01-01 2020-01-02 2020-01-01 00:06:51 2020-01-02 03:35:12 2020-01-01 00:06:51.000 2020-01-02 03:35:12.000 411 99312 49861.5 4986150 411 99312 49861.5 4986150 -32759 32379 4641.66 464166 -128 123 -2.18 -218 +412 101 10402 99313 1.23723 298.23723 149.73723 14973.72372 1.23723 298.23724 149.73724 14973.72405 1.23723 298.23723 149.73723000000007 14973.72300 2020-01-01 2020-01-02 2020-01-01 00:06:52 2020-01-02 03:35:13 2020-01-01 00:06:52.000 2020-01-02 03:35:13.000 412 99313 49862.5 4986250 412 99313 49862.5 4986250 -32758 32380 4642.66 464266 -127 124 -1.18 -118 +413 101 10403 99314 1.24024 298.24024 149.74024 14974.02402 1.24024 298.24023 149.74023 14974.02374 1.24024 298.24024 149.74023999999991 14974.02400 2020-01-01 2020-01-02 2020-01-01 00:06:53 2020-01-02 03:35:14 2020-01-01 00:06:53.000 2020-01-02 03:35:14.000 413 99314 49863.5 4986350 413 99314 49863.5 4986350 -32757 32381 4643.66 464366 -126 125 -0.18 -18 +414 101 10404 99315 1.24324 298.24324 149.74324 14974.32432 1.24324 298.24326 149.74324 14974.32433 1.24324 298.24324 149.7432399999999 14974.32400 2020-01-01 2020-01-02 2020-01-01 00:06:54 2020-01-02 03:35:15 2020-01-01 00:06:54.000 2020-01-02 03:35:15.000 414 99315 49864.5 4986450 414 99315 49864.5 4986450 -32756 32382 4644.66 464466 -125 126 0.82 82 +415 101 10405 99316 1.24624 298.24624 149.74624 14974.62462 1.24624 298.24625 149.74624 14974.62463 1.24624 298.24624 149.7462400000001 14974.62400 2020-01-01 2020-01-02 2020-01-01 00:06:55 2020-01-02 03:35:16 2020-01-01 00:06:55.000 2020-01-02 03:35:16.000 415 99316 49865.5 4986550 415 99316 49865.5 4986550 -32755 32383 4645.66 464566 -124 127 1.82 182 +416 101 10406 99317 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74923999999993 14974.92400 2020-01-01 2020-01-02 2020-01-01 00:06:56 2020-01-02 03:35:17 2020-01-01 00:06:56.000 2020-01-02 03:35:17.000 416 99317 49866.5 4986650 416 99317 49866.5 4986650 -32754 32384 4646.66 464666 -128 127 0.26 26 +417 101 10407 99318 1.25225 298.25225 149.75225 14975.22522 1.25225 298.25226 149.75225 14975.22552 1.25225 298.25225 149.75224999999983 14975.22500 2020-01-01 2020-01-02 2020-01-01 00:06:57 2020-01-02 03:35:18 2020-01-01 00:06:57.000 2020-01-02 03:35:18.000 417 99318 49867.5 4986750 417 99318 49867.5 4986750 -32753 32385 4647.66 464766 -128 123 -1.3 -130 +418 101 10408 99319 1.25525 298.25525 149.75525 14975.52552 1.25525 298.25525 149.75525 14975.52521 1.25525 298.25525 149.75525000000007 14975.52500 2020-01-01 2020-01-02 2020-01-01 00:06:58 2020-01-02 03:35:19 2020-01-01 00:06:58.000 2020-01-02 03:35:19.000 418 99319 49868.5 4986850 418 99319 49868.5 4986850 -32752 32386 4648.66 464866 -127 124 -0.3 -30 +419 101 10409 99320 1.25825 298.25825 149.75825 14975.82582 1.25825 298.25827 149.75825 14975.8258 1.25825 298.25825 149.75825000000006 14975.82500 2020-01-01 2020-01-02 2020-01-01 00:06:59 2020-01-02 03:35:20 2020-01-01 00:06:59.000 2020-01-02 03:35:20.000 419 99320 49869.5 4986950 419 99320 49869.5 4986950 -32751 32387 4649.66 464966 -126 125 0.7 70 +42 102 10032 99942 0.12612 300.12612 150.12612 15162.73873 0.12612 300.12613 150.12612 15162.73896 0.12612 300.12612 150.12612000000004 15162.73812 2020-01-01 2020-01-02 2020-01-01 00:00:42 2020-01-02 03:45:42 2020-01-01 00:00:42.000 2020-01-02 03:45:42.000 42 99942 49992 5049192 42 99942 49992 5049192 -32527 32408 4571.009900990099 461672 -128 127 -1.504950495049505 -152 +420 101 10410 99321 1.26126 298.26126 149.76126 14976.12612 1.26126 298.26126 149.76126 14976.12609 1.26126 298.26126 149.76126 14976.12600 2020-01-01 2020-01-02 2020-01-01 00:07:00 2020-01-02 03:35:21 2020-01-01 00:07:00.000 2020-01-02 03:35:21.000 420 99321 49870.5 4987050 420 99321 49870.5 4987050 -32750 32388 4650.66 465066 -125 126 1.7 170 421 101 10411 99322 1.26426 298.26426 149.76426 14976.42642 1.26426 298.26425 149.76426 14976.4264 1.26426 298.26426 149.76425999999995 14976.42600 2020-01-01 2020-01-02 2020-01-01 00:07:01 2020-01-02 03:35:22 2020-01-01 00:07:01.000 2020-01-02 03:35:22.000 421 99322 49871.5 4987150 421 99322 49871.5 4987150 -32749 32389 4651.66 465166 -124 127 2.7 270 -422 101 10412 99323 1.26726 298.26726 149.76726 14976.72672 1.26726 298.26727 149.76727 14976.72702 1.26726 298.26726 149.76726000000005 14976.72600 2020-01-01 2020-01-02 2020-01-01 00:07:02 2020-01-02 03:35:23 2020-01-01 00:07:02.000 2020-01-02 03:35:23.000 422 99323 49872.5 4987250 422 99323 49872.5 4987250 -32748 32390 4652.66 465266 -128 127 1.14 114 -423 101 10413 99324 1.27027 298.27027 149.77027 14977.02702 1.27027 298.27026 149.77026 14977.02667 1.27027 298.27027 149.77027000000027 14977.02700 2020-01-01 2020-01-02 2020-01-01 00:07:03 2020-01-02 03:35:24 2020-01-01 00:07:03.000 2020-01-02 03:35:24.000 423 99324 49873.5 4987350 423 99324 49873.5 4987350 -32747 32391 4653.66 465366 -128 123 -0.42 -42 -424 101 10414 99325 1.27327 298.27327 149.77327 14977.32732 1.27327 298.2733 149.77327 14977.32727 1.27327 298.27327 149.77326999999983 14977.32700 2020-01-01 2020-01-02 2020-01-01 00:07:04 2020-01-02 03:35:25 2020-01-01 00:07:04.000 2020-01-02 03:35:25.000 424 99325 49874.5 4987450 424 99325 49874.5 4987450 -32746 32392 4654.66 465466 -127 124 0.58 58 -425 101 10415 99326 1.27627 298.27627 149.77627 14977.62762 1.27627 298.27628 149.77627 14977.62756 1.27627 298.27627 149.77627000000018 14977.62700 2020-01-01 2020-01-02 2020-01-01 00:07:05 2020-01-02 03:35:26 2020-01-01 00:07:05.000 2020-01-02 03:35:26.000 425 99326 49875.5 4987550 425 99326 49875.5 4987550 -32745 32393 4655.66 465566 -126 125 1.58 158 -426 101 10416 99327 1.27927 298.27927 149.77927 14977.92792 1.27927 298.27927 149.77927 14977.92787 1.27927 298.27927 149.7792699999997 14977.92700 2020-01-01 2020-01-02 2020-01-01 00:07:06 2020-01-02 03:35:27 2020-01-01 00:07:06.000 2020-01-02 03:35:27.000 426 99327 49876.5 4987650 426 99327 49876.5 4987650 -32744 32394 4656.66 465666 -125 126 2.58 258 -427 101 10417 99328 1.28228 298.28228 149.78228 14978.22822 1.28228 298.2823 149.78228 14978.22849 1.28228 298.28228 149.78227999999993 14978.22800 2020-01-01 2020-01-02 2020-01-01 00:07:07 2020-01-02 03:35:28 2020-01-01 00:07:07.000 2020-01-02 03:35:28.000 427 99328 49877.5 4987750 427 99328 49877.5 4987750 -32743 32395 4657.66 465766 -124 127 3.58 358 +422 101 10412 99323 1.26726 298.26726 149.76726 14976.72672 1.26726 298.26727 149.76727 14976.72702 1.26726 298.26726 149.76726000000002 14976.72600 2020-01-01 2020-01-02 2020-01-01 00:07:02 2020-01-02 03:35:23 2020-01-01 00:07:02.000 2020-01-02 03:35:23.000 422 99323 49872.5 4987250 422 99323 49872.5 4987250 -32748 32390 4652.66 465266 -128 127 1.14 114 +423 101 10413 99324 1.27027 298.27027 149.77027 14977.02702 1.27027 298.27026 149.77026 14977.02667 1.27027 298.27027 149.77027000000012 14977.02700 2020-01-01 2020-01-02 2020-01-01 00:07:03 2020-01-02 03:35:24 2020-01-01 00:07:03.000 2020-01-02 03:35:24.000 423 99324 49873.5 4987350 423 99324 49873.5 4987350 -32747 32391 4653.66 465366 -128 123 -0.42 -42 +424 101 10414 99325 1.27327 298.27327 149.77327 14977.32732 1.27327 298.2733 149.77327 14977.32727 1.27327 298.27327 149.7732699999999 14977.32700 2020-01-01 2020-01-02 2020-01-01 00:07:04 2020-01-02 03:35:25 2020-01-01 00:07:04.000 2020-01-02 03:35:25.000 424 99325 49874.5 4987450 424 99325 49874.5 4987450 -32746 32392 4654.66 465466 -127 124 0.58 58 +425 101 10415 99326 1.27627 298.27627 149.77627 14977.62762 1.27627 298.27628 149.77627 14977.62756 1.27627 298.27627 149.77627000000007 14977.62700 2020-01-01 2020-01-02 2020-01-01 00:07:05 2020-01-02 03:35:26 2020-01-01 00:07:05.000 2020-01-02 03:35:26.000 425 99326 49875.5 4987550 425 99326 49875.5 4987550 -32745 32393 4655.66 465566 -126 125 1.58 158 +426 101 10416 99327 1.27927 298.27927 149.77927 14977.92792 1.27927 298.27927 149.77927 14977.92787 1.27927 298.27927 149.77926999999988 14977.92700 2020-01-01 2020-01-02 2020-01-01 00:07:06 2020-01-02 03:35:27 2020-01-01 00:07:06.000 2020-01-02 03:35:27.000 426 99327 49876.5 4987650 426 99327 49876.5 4987650 -32744 32394 4656.66 465666 -125 126 2.58 258 +427 101 10417 99328 1.28228 298.28228 149.78228 14978.22822 1.28228 298.2823 149.78228 14978.22849 1.28228 298.28228 149.78227999999996 14978.22800 2020-01-01 2020-01-02 2020-01-01 00:07:07 2020-01-02 03:35:28 2020-01-01 00:07:07.000 2020-01-02 03:35:28.000 427 99328 49877.5 4987750 427 99328 49877.5 4987750 -32743 32395 4657.66 465766 -124 127 3.58 358 428 101 10418 99329 1.28528 298.28528 149.78528 14978.52852 1.28528 298.28528 149.78528 14978.52815 1.28528 298.28528 149.78528000000003 14978.52800 2020-01-01 2020-01-02 2020-01-01 00:07:08 2020-01-02 03:35:29 2020-01-01 00:07:08.000 2020-01-02 03:35:29.000 428 99329 49878.5 4987850 428 99329 49878.5 4987850 -32742 32396 4658.66 465866 -128 127 2.02 202 -429 101 10419 99330 1.28828 298.28828 149.78828 14978.82882 1.28828 298.2883 149.78828 14978.8289 1.28828 298.28828 149.78828000000013 14978.82800 2020-01-01 2020-01-02 2020-01-01 00:07:09 2020-01-02 03:35:30 2020-01-01 00:07:09.000 2020-01-02 03:35:30.000 429 99330 49879.5 4987950 429 99330 49879.5 4987950 -32741 32397 4659.66 465966 -128 127 0.46 46 -43 102 10033 99943 0.12912 300.12912 150.12912 15163.04204 0.12912 300.12912 150.12912 15163.04211 0.12912 300.12912 150.12911999999972 15163.04112 2020-01-01 2020-01-02 2020-01-01 00:00:43 2020-01-02 03:45:43 2020-01-01 00:00:43.000 2020-01-02 03:45:43.000 43 99943 49993 5049293 43 99943 49993 5049293 -32526 32409 4572.009900990099 461773 -128 124 -3.0396039603960396 -307 -430 101 10420 99331 1.29129 298.29129 149.79129 14979.12912 1.29129 298.2913 149.79129 14979.12904 1.29129 298.29129 149.79128999999983 14979.12900 2020-01-01 2020-01-02 2020-01-01 00:07:10 2020-01-02 03:35:31 2020-01-01 00:07:10.000 2020-01-02 03:35:31.000 430 99331 49880.5 4988050 430 99331 49880.5 4988050 -32740 32398 4660.66 466066 -128 124 -1.1 -110 -431 101 10421 99332 1.29429 298.29429 149.79429 14979.42942 1.29429 298.29428 149.79429 14979.42933 1.29429 298.29429 149.79428999999985 14979.42900 2020-01-01 2020-01-02 2020-01-01 00:07:11 2020-01-02 03:35:32 2020-01-01 00:07:11.000 2020-01-02 03:35:32.000 431 99332 49881.5 4988150 431 99332 49881.5 4988150 -32739 32399 4661.66 466166 -127 125 -0.1 -10 -432 101 10422 99333 1.29729 298.29729 149.79729 14979.72972 1.29729 298.2973 149.79729 14979.72996 1.29729 298.29729 149.7972900000003 14979.72900 2020-01-01 2020-01-02 2020-01-01 00:07:12 2020-01-02 03:35:33 2020-01-01 00:07:12.000 2020-01-02 03:35:33.000 432 99333 49882.5 4988250 432 99333 49882.5 4988250 -32738 32400 4662.66 466266 -126 126 0.9 90 -433 101 10423 99334 1.3003 298.3003 149.8003 14980.03003 1.3003 298.3003 149.80029 14980.02962 1.30030 298.30030 149.80030000000016 14980.03000 2020-01-01 2020-01-02 2020-01-01 00:07:13 2020-01-02 03:35:34 2020-01-01 00:07:13.000 2020-01-02 03:35:34.000 433 99334 49883.5 4988350 433 99334 49883.5 4988350 -32737 32401 4663.66 466366 -125 127 1.9 190 -434 101 10424 99335 1.3033 298.3033 149.8033 14980.33033 1.3033 298.3033 149.8033 14980.33037 1.30330 298.30330 149.80329999999972 14980.33000 2020-01-01 2020-01-02 2020-01-01 00:07:14 2020-01-02 03:35:35 2020-01-01 00:07:14.000 2020-01-02 03:35:35.000 434 99335 49884.5 4988450 434 99335 49884.5 4988450 -32736 32402 4664.66 466466 -128 127 0.34 34 -435 101 10425 99336 1.3063 298.3063 149.8063 14980.63063 1.3063 298.3063 149.8063 14980.63051 1.30630 298.30630 149.80630000000016 14980.63000 2020-01-01 2020-01-02 2020-01-01 00:07:15 2020-01-02 03:35:36 2020-01-01 00:07:15.000 2020-01-02 03:35:36.000 435 99336 49885.5 4988550 435 99336 49885.5 4988550 -32735 32403 4665.66 466566 -128 127 -1.22 -122 -436 101 10426 99337 1.3093 298.3093 149.8093 14980.93093 1.3093 298.3093 149.8093 14980.93084 1.30930 298.30930 149.80930000000018 14980.93000 2020-01-01 2020-01-02 2020-01-01 00:07:16 2020-01-02 03:35:37 2020-01-01 00:07:16.000 2020-01-02 03:35:37.000 436 99337 49886.5 4988650 436 99337 49886.5 4988650 -32734 32404 4666.66 466666 -128 123 -2.78 -278 -437 101 10427 99338 1.31231 298.31231 149.81231 14981.23123 1.31231 298.31232 149.81231 14981.23143 1.31231 298.31231 149.81231000000008 14981.23100 2020-01-01 2020-01-02 2020-01-01 00:07:17 2020-01-02 03:35:38 2020-01-01 00:07:17.000 2020-01-02 03:35:38.000 437 99338 49887.5 4988750 437 99338 49887.5 4988750 -32733 32405 4667.66 466766 -127 124 -1.78 -178 +429 101 10419 99330 1.28828 298.28828 149.78828 14978.82882 1.28828 298.2883 149.78828 14978.8289 1.28828 298.28828 149.78828000000001 14978.82800 2020-01-01 2020-01-02 2020-01-01 00:07:09 2020-01-02 03:35:30 2020-01-01 00:07:09.000 2020-01-02 03:35:30.000 429 99330 49879.5 4987950 429 99330 49879.5 4987950 -32741 32397 4659.66 465966 -128 127 0.46 46 +43 102 10033 99943 0.12912 300.12912 150.12912 15163.04204 0.12912 300.12912 150.12912 15163.04211 0.12912 300.12912 150.1291199999999 15163.04112 2020-01-01 2020-01-02 2020-01-01 00:00:43 2020-01-02 03:45:43 2020-01-01 00:00:43.000 2020-01-02 03:45:43.000 43 99943 49993 5049293 43 99943 49993 5049293 -32526 32409 4572.009900990099 461773 -128 124 -3.0396039603960396 -307 +430 101 10420 99331 1.29129 298.29129 149.79129 14979.12912 1.29129 298.2913 149.79129 14979.12904 1.29129 298.29129 149.79128999999995 14979.12900 2020-01-01 2020-01-02 2020-01-01 00:07:10 2020-01-02 03:35:31 2020-01-01 00:07:10.000 2020-01-02 03:35:31.000 430 99331 49880.5 4988050 430 99331 49880.5 4988050 -32740 32398 4660.66 466066 -128 124 -1.1 -110 +431 101 10421 99332 1.29429 298.29429 149.79429 14979.42942 1.29429 298.29428 149.79429 14979.42933 1.29429 298.29429 149.79428999999993 14979.42900 2020-01-01 2020-01-02 2020-01-01 00:07:11 2020-01-02 03:35:32 2020-01-01 00:07:11.000 2020-01-02 03:35:32.000 431 99332 49881.5 4988150 431 99332 49881.5 4988150 -32739 32399 4661.66 466166 -127 125 -0.1 -10 +432 101 10422 99333 1.29729 298.29729 149.79729 14979.72972 1.29729 298.2973 149.79729 14979.72996 1.29729 298.29729 149.79729000000015 14979.72900 2020-01-01 2020-01-02 2020-01-01 00:07:12 2020-01-02 03:35:33 2020-01-01 00:07:12.000 2020-01-02 03:35:33.000 432 99333 49882.5 4988250 432 99333 49882.5 4988250 -32738 32400 4662.66 466266 -126 126 0.9 90 +433 101 10423 99334 1.3003 298.3003 149.8003 14980.03003 1.3003 298.3003 149.80029 14980.02962 1.30030 298.30030 149.80030000000005 14980.03000 2020-01-01 2020-01-02 2020-01-01 00:07:13 2020-01-02 03:35:34 2020-01-01 00:07:13.000 2020-01-02 03:35:34.000 433 99334 49883.5 4988350 433 99334 49883.5 4988350 -32737 32401 4663.66 466366 -125 127 1.9 190 +434 101 10424 99335 1.3033 298.3033 149.8033 14980.33033 1.3033 298.3033 149.8033 14980.33037 1.30330 298.30330 149.80329999999987 14980.33000 2020-01-01 2020-01-02 2020-01-01 00:07:14 2020-01-02 03:35:35 2020-01-01 00:07:14.000 2020-01-02 03:35:35.000 434 99335 49884.5 4988450 434 99335 49884.5 4988450 -32736 32402 4664.66 466466 -128 127 0.34 34 +435 101 10425 99336 1.3063 298.3063 149.8063 14980.63063 1.3063 298.3063 149.8063 14980.63051 1.30630 298.30630 149.80630000000008 14980.63000 2020-01-01 2020-01-02 2020-01-01 00:07:15 2020-01-02 03:35:36 2020-01-01 00:07:15.000 2020-01-02 03:35:36.000 435 99336 49885.5 4988550 435 99336 49885.5 4988550 -32735 32403 4665.66 466566 -128 127 -1.22 -122 +436 101 10426 99337 1.3093 298.3093 149.8093 14980.93093 1.3093 298.3093 149.8093 14980.93084 1.30930 298.30930 149.80930000000006 14980.93000 2020-01-01 2020-01-02 2020-01-01 00:07:16 2020-01-02 03:35:37 2020-01-01 00:07:16.000 2020-01-02 03:35:37.000 436 99337 49886.5 4988650 436 99337 49886.5 4988650 -32734 32404 4666.66 466666 -128 123 -2.78 -278 +437 101 10427 99338 1.31231 298.31231 149.81231 14981.23123 1.31231 298.31232 149.81231 14981.23143 1.31231 298.31231 149.81231 14981.23100 2020-01-01 2020-01-02 2020-01-01 00:07:17 2020-01-02 03:35:38 2020-01-01 00:07:17.000 2020-01-02 03:35:38.000 437 99338 49887.5 4988750 437 99338 49887.5 4988750 -32733 32405 4667.66 466766 -127 124 -1.78 -178 438 101 10428 99339 1.31531 298.31531 149.81531 14981.53153 1.31531 298.3153 149.81531 14981.53173 1.31531 298.31531 149.81530999999998 14981.53100 2020-01-01 2020-01-02 2020-01-01 00:07:18 2020-01-02 03:35:39 2020-01-01 00:07:18.000 2020-01-02 03:35:39.000 438 99339 49888.5 4988850 438 99339 49888.5 4988850 -32732 32406 4668.66 466866 -126 125 -0.78 -78 -439 101 10429 99340 1.31831 298.31831 149.81831 14981.83183 1.31831 298.31833 149.81831 14981.83184 1.31831 298.31831 149.81831000000005 14981.83100 2020-01-01 2020-01-02 2020-01-01 00:07:19 2020-01-02 03:35:40 2020-01-01 00:07:19.000 2020-01-02 03:35:40.000 439 99340 49889.5 4988950 439 99340 49889.5 4988950 -32731 32407 4669.66 466966 -125 126 0.22 22 +439 101 10429 99340 1.31831 298.31831 149.81831 14981.83183 1.31831 298.31833 149.81831 14981.83184 1.31831 298.31831 149.81831000000008 14981.83100 2020-01-01 2020-01-02 2020-01-01 00:07:19 2020-01-02 03:35:40 2020-01-01 00:07:19.000 2020-01-02 03:35:40.000 439 99340 49889.5 4988950 439 99340 49889.5 4988950 -32731 32407 4669.66 466966 -125 126 0.22 22 44 102 10034 99944 0.13213 300.13213 150.13213 15163.34534 0.13213 300.13214 150.13213 15163.34525 0.13213 300.13213 150.13213 15163.34513 2020-01-01 2020-01-02 2020-01-01 00:00:44 2020-01-02 03:45:44 2020-01-01 00:00:44.000 2020-01-02 03:45:44.000 44 99944 49994 5049394 44 99944 49994 5049394 -32525 32410 4573.009900990099 461874 -127 125 -2.0396039603960396 -206 -440 101 10430 99341 1.32132 298.32132 149.82132 14982.13213 1.32132 298.32132 149.82131 14982.13197 1.32132 298.32132 149.82131999999987 14982.13200 2020-01-01 2020-01-02 2020-01-01 00:07:20 2020-01-02 03:35:41 2020-01-01 00:07:20.000 2020-01-02 03:35:41.000 440 99341 49890.5 4989050 440 99341 49890.5 4989050 -32730 32408 4670.66 467066 -124 127 1.22 122 -441 101 10431 99342 1.32432 298.32432 149.82432 14982.43243 1.32432 298.3243 149.82432 14982.4323 1.32432 298.32432 149.82431999999994 14982.43200 2020-01-01 2020-01-02 2020-01-01 00:07:21 2020-01-02 03:35:42 2020-01-01 00:07:21.000 2020-01-02 03:35:42.000 441 99342 49891.5 4989150 441 99342 49891.5 4989150 -32729 32409 4671.66 467166 -128 127 -0.34 -34 -442 101 10432 99343 1.32732 298.32732 149.82732 14982.73273 1.32732 298.32733 149.82732 14982.7329 1.32732 298.32732 149.8273200000001 14982.73200 2020-01-01 2020-01-02 2020-01-01 00:07:22 2020-01-02 03:35:43 2020-01-01 00:07:22.000 2020-01-02 03:35:43.000 442 99343 49892.5 4989250 442 99343 49892.5 4989250 -32728 32410 4672.66 467266 -128 123 -1.9 -190 -443 101 10433 99344 1.33033 298.33033 149.83033 14983.03303 1.33033 298.33032 149.83033 14983.03319 1.33033 298.33033 149.83033 14983.03300 2020-01-01 2020-01-02 2020-01-01 00:07:23 2020-01-02 03:35:44 2020-01-01 00:07:23.000 2020-01-02 03:35:44.000 443 99344 49893.5 4989350 443 99344 49893.5 4989350 -32727 32411 4673.66 467366 -127 124 -0.9 -90 -444 101 10434 99345 1.33333 298.33333 149.83333 14983.33333 1.33333 298.33334 149.83333 14983.33331 1.33333 298.33333 149.83333 14983.33300 2020-01-01 2020-01-02 2020-01-01 00:07:24 2020-01-02 03:35:45 2020-01-01 00:07:24.000 2020-01-02 03:35:45.000 444 99345 49894.5 4989450 444 99345 49894.5 4989450 -32726 32412 4674.66 467466 -126 125 0.1 10 +440 101 10430 99341 1.32132 298.32132 149.82132 14982.13213 1.32132 298.32132 149.82131 14982.13197 1.32132 298.32132 149.82131999999973 14982.13200 2020-01-01 2020-01-02 2020-01-01 00:07:20 2020-01-02 03:35:41 2020-01-01 00:07:20.000 2020-01-02 03:35:41.000 440 99341 49890.5 4989050 440 99341 49890.5 4989050 -32730 32408 4670.66 467066 -124 127 1.22 122 +441 101 10431 99342 1.32432 298.32432 149.82432 14982.43243 1.32432 298.3243 149.82432 14982.4323 1.32432 298.32432 149.82431999999983 14982.43200 2020-01-01 2020-01-02 2020-01-01 00:07:21 2020-01-02 03:35:42 2020-01-01 00:07:21.000 2020-01-02 03:35:42.000 441 99342 49891.5 4989150 441 99342 49891.5 4989150 -32729 32409 4671.66 467166 -128 127 -0.34 -34 +442 101 10432 99343 1.32732 298.32732 149.82732 14982.73273 1.32732 298.32733 149.82732 14982.7329 1.32732 298.32732 149.82732000000019 14982.73200 2020-01-01 2020-01-02 2020-01-01 00:07:22 2020-01-02 03:35:43 2020-01-01 00:07:22.000 2020-01-02 03:35:43.000 442 99343 49892.5 4989250 442 99343 49892.5 4989250 -32728 32410 4672.66 467266 -128 123 -1.9 -190 +443 101 10433 99344 1.33033 298.33033 149.83033 14983.03303 1.33033 298.33032 149.83033 14983.03319 1.33033 298.33033 149.8303300000001 14983.03300 2020-01-01 2020-01-02 2020-01-01 00:07:23 2020-01-02 03:35:44 2020-01-01 00:07:23.000 2020-01-02 03:35:44.000 443 99344 49893.5 4989350 443 99344 49893.5 4989350 -32727 32411 4673.66 467366 -127 124 -0.9 -90 +444 101 10434 99345 1.33333 298.33333 149.83333 14983.33333 1.33333 298.33334 149.83333 14983.33331 1.33333 298.33333 149.83332999999996 14983.33300 2020-01-01 2020-01-02 2020-01-01 00:07:24 2020-01-02 03:35:45 2020-01-01 00:07:24.000 2020-01-02 03:35:45.000 444 99345 49894.5 4989450 444 99345 49894.5 4989450 -32726 32412 4674.66 467466 -126 125 0.1 10 445 101 10435 99346 1.33633 298.33633 149.83633 14983.63363 1.33633 298.33633 149.83633 14983.63348 1.33633 298.33633 149.83633000000006 14983.63300 2020-01-01 2020-01-02 2020-01-01 00:07:25 2020-01-02 03:35:46 2020-01-01 00:07:25.000 2020-01-02 03:35:46.000 445 99346 49895.5 4989550 445 99346 49895.5 4989550 -32725 32413 4675.66 467566 -125 126 1.1 110 -446 101 10436 99347 1.33933 298.33933 149.83933 14983.93393 1.33933 298.33932 149.83933 14983.93378 1.33933 298.33933 149.83933000000002 14983.93300 2020-01-01 2020-01-02 2020-01-01 00:07:26 2020-01-02 03:35:47 2020-01-01 00:07:26.000 2020-01-02 03:35:47.000 446 99347 49896.5 4989650 446 99347 49896.5 4989650 -32724 32414 4676.66 467666 -124 127 2.1 210 -447 101 10437 99348 1.34234 298.34234 149.84234 14984.23423 1.34234 298.34235 149.84234 14984.23437 1.34234 298.34234 149.84233999999995 14984.23400 2020-01-01 2020-01-02 2020-01-01 00:07:27 2020-01-02 03:35:48 2020-01-01 00:07:27.000 2020-01-02 03:35:48.000 447 99348 49897.5 4989750 447 99348 49897.5 4989750 -32723 32415 4677.66 467766 -128 127 0.54 54 -448 101 10438 99349 1.34534 298.34534 149.84534 14984.53453 1.34534 298.34534 149.84534 14984.53466 1.34534 298.34534 149.84533999999994 14984.53400 2020-01-01 2020-01-02 2020-01-01 00:07:28 2020-01-02 03:35:49 2020-01-01 00:07:28.000 2020-01-02 03:35:49.000 448 99349 49898.5 4989850 448 99349 49898.5 4989850 -32722 32416 4678.66 467866 -128 123 -1.02 -102 -449 101 10439 99350 1.34834 298.34834 149.84834 14984.83483 1.34834 298.34836 149.84834 14984.83478 1.34834 298.34834 149.84834000000015 14984.83400 2020-01-01 2020-01-02 2020-01-01 00:07:29 2020-01-02 03:35:50 2020-01-01 00:07:29.000 2020-01-02 03:35:50.000 449 99350 49899.5 4989950 449 99350 49899.5 4989950 -32721 32417 4679.66 467966 -127 124 -0.02 -2 -45 102 10035 99945 0.13513 300.13513 150.13513 15163.64864 0.13513 300.13513 150.13513 15163.64839 0.13513 300.13513 150.13513000000003 15163.64813 2020-01-01 2020-01-02 2020-01-01 00:00:45 2020-01-02 03:45:45 2020-01-01 00:00:45.000 2020-01-02 03:45:45.000 45 99945 49995 5049495 45 99945 49995 5049495 -32524 32411 4574.009900990099 461975 -126 126 -1.0396039603960396 -105 -450 101 10440 99351 1.35135 298.35135 149.85135 14985.13513 1.35135 298.35135 149.85134 14985.13495 1.35135 298.35135 149.85135000000008 14985.13500 2020-01-01 2020-01-02 2020-01-01 00:07:30 2020-01-02 03:35:51 2020-01-01 00:07:30.000 2020-01-02 03:35:51.000 450 99351 49900.5 4990050 450 99351 49900.5 4990050 -32720 32418 4680.66 468066 -126 125 0.98 98 -451 101 10441 99352 1.35435 298.35435 149.85435 14985.43543 1.35435 298.35434 149.85435 14985.43525 1.35435 298.35435 149.8543499999999 14985.43500 2020-01-01 2020-01-02 2020-01-01 00:07:31 2020-01-02 03:35:52 2020-01-01 00:07:31.000 2020-01-02 03:35:52.000 451 99352 49901.5 4990150 451 99352 49901.5 4990150 -32719 32419 4681.66 468166 -125 126 1.98 198 -452 101 10442 99353 1.35735 298.35735 149.85735 14985.73573 1.35735 298.35736 149.85736 14985.736 1.35735 298.35735 149.8573500000001 14985.73500 2020-01-01 2020-01-02 2020-01-01 00:07:32 2020-01-02 03:35:53 2020-01-01 00:07:32.000 2020-01-02 03:35:53.000 452 99353 49902.5 4990250 452 99353 49902.5 4990250 -32718 32420 4682.66 468266 -124 127 2.98 298 -453 101 10443 99354 1.36036 298.36036 149.86036 14986.03603 1.36036 298.36035 149.86036 14986.03614 1.36036 298.36036 149.86036000000004 14986.03600 2020-01-01 2020-01-02 2020-01-01 00:07:33 2020-01-02 03:35:54 2020-01-01 00:07:33.000 2020-01-02 03:35:54.000 453 99354 49903.5 4990350 453 99354 49903.5 4990350 -32717 32421 4683.66 468366 -128 127 1.42 142 -454 101 10444 99355 1.36336 298.36336 149.86336 14986.33633 1.36336 298.36337 149.86336 14986.33629 1.36336 298.36336 149.86335999999994 14986.33600 2020-01-01 2020-01-02 2020-01-01 00:07:34 2020-01-02 03:35:55 2020-01-01 00:07:34.000 2020-01-02 03:35:55.000 454 99355 49904.5 4990450 454 99355 49904.5 4990450 -32716 32422 4684.66 468466 -128 127 -0.14 -14 +446 101 10436 99347 1.33933 298.33933 149.83933 14983.93393 1.33933 298.33932 149.83933 14983.93378 1.33933 298.33933 149.8393300000001 14983.93300 2020-01-01 2020-01-02 2020-01-01 00:07:26 2020-01-02 03:35:47 2020-01-01 00:07:26.000 2020-01-02 03:35:47.000 446 99347 49896.5 4989650 446 99347 49896.5 4989650 -32724 32414 4676.66 467666 -124 127 2.1 210 +447 101 10437 99348 1.34234 298.34234 149.84234 14984.23423 1.34234 298.34235 149.84234 14984.23437 1.34234 298.34234 149.84233999999984 14984.23400 2020-01-01 2020-01-02 2020-01-01 00:07:27 2020-01-02 03:35:48 2020-01-01 00:07:27.000 2020-01-02 03:35:48.000 447 99348 49897.5 4989750 447 99348 49897.5 4989750 -32723 32415 4677.66 467766 -128 127 0.54 54 +448 101 10438 99349 1.34534 298.34534 149.84534 14984.53453 1.34534 298.34534 149.84534 14984.53466 1.34534 298.34534 149.84533999999988 14984.53400 2020-01-01 2020-01-02 2020-01-01 00:07:28 2020-01-02 03:35:49 2020-01-01 00:07:28.000 2020-01-02 03:35:49.000 448 99349 49898.5 4989850 448 99349 49898.5 4989850 -32722 32416 4678.66 467866 -128 123 -1.02 -102 +449 101 10439 99350 1.34834 298.34834 149.84834 14984.83483 1.34834 298.34836 149.84834 14984.83478 1.34834 298.34834 149.8483400000001 14984.83400 2020-01-01 2020-01-02 2020-01-01 00:07:29 2020-01-02 03:35:50 2020-01-01 00:07:29.000 2020-01-02 03:35:50.000 449 99350 49899.5 4989950 449 99350 49899.5 4989950 -32721 32417 4679.66 467966 -127 124 -0.02 -2 +45 102 10035 99945 0.13513 300.13513 150.13513 15163.64864 0.13513 300.13513 150.13513 15163.64839 0.13513 300.13513 150.13513000000006 15163.64813 2020-01-01 2020-01-02 2020-01-01 00:00:45 2020-01-02 03:45:45 2020-01-01 00:00:45.000 2020-01-02 03:45:45.000 45 99945 49995 5049495 45 99945 49995 5049495 -32524 32411 4574.009900990099 461975 -126 126 -1.0396039603960396 -105 +450 101 10440 99351 1.35135 298.35135 149.85135 14985.13513 1.35135 298.35135 149.85134 14985.13495 1.35135 298.35135 149.8513500000002 14985.13500 2020-01-01 2020-01-02 2020-01-01 00:07:30 2020-01-02 03:35:51 2020-01-01 00:07:30.000 2020-01-02 03:35:51.000 450 99351 49900.5 4990050 450 99351 49900.5 4990050 -32720 32418 4680.66 468066 -126 125 0.98 98 +451 101 10441 99352 1.35435 298.35435 149.85435 14985.43543 1.35435 298.35434 149.85435 14985.43525 1.35435 298.35435 149.85434999999976 14985.43500 2020-01-01 2020-01-02 2020-01-01 00:07:31 2020-01-02 03:35:52 2020-01-01 00:07:31.000 2020-01-02 03:35:52.000 451 99352 49901.5 4990150 451 99352 49901.5 4990150 -32719 32419 4681.66 468166 -125 126 1.98 198 +452 101 10442 99353 1.35735 298.35735 149.85735 14985.73573 1.35735 298.35736 149.85736 14985.736 1.35735 298.35735 149.8573500000002 14985.73500 2020-01-01 2020-01-02 2020-01-01 00:07:32 2020-01-02 03:35:53 2020-01-01 00:07:32.000 2020-01-02 03:35:53.000 452 99353 49902.5 4990250 452 99353 49902.5 4990250 -32718 32420 4682.66 468266 -124 127 2.98 298 +453 101 10443 99354 1.36036 298.36036 149.86036 14986.03603 1.36036 298.36035 149.86036 14986.03614 1.36036 298.36036 149.8603600000001 14986.03600 2020-01-01 2020-01-02 2020-01-01 00:07:33 2020-01-02 03:35:54 2020-01-01 00:07:33.000 2020-01-02 03:35:54.000 453 99354 49903.5 4990350 453 99354 49903.5 4990350 -32717 32421 4683.66 468366 -128 127 1.42 142 +454 101 10444 99355 1.36336 298.36336 149.86336 14986.33633 1.36336 298.36337 149.86336 14986.33629 1.36336 298.36336 149.86335999999986 14986.33600 2020-01-01 2020-01-02 2020-01-01 00:07:34 2020-01-02 03:35:55 2020-01-01 00:07:34.000 2020-01-02 03:35:55.000 454 99355 49904.5 4990450 454 99355 49904.5 4990450 -32716 32422 4684.66 468466 -128 127 -0.14 -14 455 101 10445 99356 1.36636 298.36636 149.86636 14986.63663 1.36636 298.36636 149.86636 14986.63641 1.36636 298.36636 149.86636 14986.63600 2020-01-01 2020-01-02 2020-01-01 00:07:35 2020-01-02 03:35:56 2020-01-01 00:07:35.000 2020-01-02 03:35:56.000 455 99356 49905.5 4990550 455 99356 49905.5 4990550 -32715 32423 4685.66 468566 -128 124 -1.7 -170 -456 101 10446 99357 1.36936 298.36936 149.86936 14986.93693 1.36936 298.36935 149.86936 14986.93672 1.36936 298.36936 149.86936000000003 14986.93600 2020-01-01 2020-01-02 2020-01-01 00:07:36 2020-01-02 03:35:57 2020-01-01 00:07:36.000 2020-01-02 03:35:57.000 456 99357 49906.5 4990650 456 99357 49906.5 4990650 -32714 32424 4686.66 468666 -127 125 -0.7 -70 -457 101 10447 99358 1.37237 298.37237 149.87237 14987.23723 1.37237 298.37238 149.87237 14987.23747 1.37237 298.37237 149.8723699999999 14987.23700 2020-01-01 2020-01-02 2020-01-01 00:07:37 2020-01-02 03:35:58 2020-01-01 00:07:37.000 2020-01-02 03:35:58.000 457 99358 49907.5 4990750 457 99358 49907.5 4990750 -32713 32425 4687.66 468766 -126 126 0.3 30 -458 101 10448 99359 1.37537 298.37537 149.87537 14987.53753 1.37537 298.37537 149.87537 14987.5376 1.37537 298.37537 149.87536999999992 14987.53700 2020-01-01 2020-01-02 2020-01-01 00:07:38 2020-01-02 03:35:59 2020-01-01 00:07:38.000 2020-01-02 03:35:59.000 458 99359 49908.5 4990850 458 99359 49908.5 4990850 -32712 32426 4688.66 468866 -125 127 1.3 130 -459 101 10449 99360 1.37837 298.37837 149.87837 14987.83783 1.37837 298.3784 149.87837 14987.83775 1.37837 298.37837 149.87837000000007 14987.83700 2020-01-01 2020-01-02 2020-01-01 00:07:39 2020-01-02 03:36:00 2020-01-01 00:07:39.000 2020-01-02 03:36:00.000 459 99360 49909.5 4990950 459 99360 49909.5 4990950 -32711 32427 4689.66 468966 -128 127 -0.26 -26 -46 102 10036 99946 0.13813 300.13813 150.13813 15163.95195 0.13813 300.13815 150.13814 15163.95214 0.13813 300.13813 150.13812999999993 15163.95113 2020-01-01 2020-01-02 2020-01-01 00:00:46 2020-01-02 03:45:46 2020-01-01 00:00:46.000 2020-01-02 03:45:46.000 46 99946 49996 5049596 46 99946 49996 5049596 -32523 32412 4575.009900990099 462076 -125 127 -0.039603960396039604 -4 -460 101 10450 99361 1.38138 298.38138 149.88138 14988.13813 1.38138 298.38138 149.88137 14988.13789 1.38138 298.38138 149.88138000000004 14988.13800 2020-01-01 2020-01-02 2020-01-01 00:07:40 2020-01-02 03:36:01 2020-01-01 00:07:40.000 2020-01-02 03:36:01.000 460 99361 49910.5 4991050 460 99361 49910.5 4991050 -32710 32428 4690.66 469066 -128 127 -1.82 -182 -461 101 10451 99362 1.38438 298.38438 149.88438 14988.43843 1.38438 298.3844 149.88438 14988.43864 1.38438 298.38438 149.88438000000002 14988.43800 2020-01-01 2020-01-02 2020-01-01 00:07:41 2020-01-02 03:36:02 2020-01-01 00:07:41.000 2020-01-02 03:36:02.000 461 99362 49911.5 4991150 461 99362 49911.5 4991150 -32709 32429 4691.66 469166 -128 123 -3.38 -338 -462 101 10452 99363 1.38738 298.38738 149.88738 14988.73873 1.38738 298.3874 149.88738 14988.73894 1.38738 298.38738 149.88738000000006 14988.73800 2020-01-01 2020-01-02 2020-01-01 00:07:42 2020-01-02 03:36:03 2020-01-01 00:07:42.000 2020-01-02 03:36:03.000 462 99363 49912.5 4991250 462 99363 49912.5 4991250 -32708 32430 4692.66 469266 -127 124 -2.38 -238 +456 101 10446 99357 1.36936 298.36936 149.86936 14986.93693 1.36936 298.36935 149.86936 14986.93672 1.36936 298.36936 149.8693600000001 14986.93600 2020-01-01 2020-01-02 2020-01-01 00:07:36 2020-01-02 03:35:57 2020-01-01 00:07:36.000 2020-01-02 03:35:57.000 456 99357 49906.5 4990650 456 99357 49906.5 4990650 -32714 32424 4686.66 468666 -127 125 -0.7 -70 +457 101 10447 99358 1.37237 298.37237 149.87237 14987.23723 1.37237 298.37238 149.87237 14987.23747 1.37237 298.37237 149.87236999999976 14987.23700 2020-01-01 2020-01-02 2020-01-01 00:07:37 2020-01-02 03:35:58 2020-01-01 00:07:37.000 2020-01-02 03:35:58.000 457 99358 49907.5 4990750 457 99358 49907.5 4990750 -32713 32425 4687.66 468766 -126 126 0.3 30 +458 101 10448 99359 1.37537 298.37537 149.87537 14987.53753 1.37537 298.37537 149.87537 14987.5376 1.37537 298.37537 149.87536999999986 14987.53700 2020-01-01 2020-01-02 2020-01-01 00:07:38 2020-01-02 03:35:59 2020-01-01 00:07:38.000 2020-01-02 03:35:59.000 458 99359 49908.5 4990850 458 99359 49908.5 4990850 -32712 32426 4688.66 468866 -125 127 1.3 130 +459 101 10449 99360 1.37837 298.37837 149.87837 14987.83783 1.37837 298.3784 149.87837 14987.83775 1.37837 298.37837 149.87837000000022 14987.83700 2020-01-01 2020-01-02 2020-01-01 00:07:39 2020-01-02 03:36:00 2020-01-01 00:07:39.000 2020-01-02 03:36:00.000 459 99360 49909.5 4990950 459 99360 49909.5 4990950 -32711 32427 4689.66 468966 -128 127 -0.26 -26 +46 102 10036 99946 0.13813 300.13813 150.13813 15163.95195 0.13813 300.13815 150.13814 15163.95214 0.13813 300.13813 150.13812999999988 15163.95113 2020-01-01 2020-01-02 2020-01-01 00:00:46 2020-01-02 03:45:46 2020-01-01 00:00:46.000 2020-01-02 03:45:46.000 46 99946 49996 5049596 46 99946 49996 5049596 -32523 32412 4575.009900990099 462076 -125 127 -0.039603960396039604 -4 +460 101 10450 99361 1.38138 298.38138 149.88138 14988.13813 1.38138 298.38138 149.88137 14988.13789 1.38138 298.38138 149.88138000000012 14988.13800 2020-01-01 2020-01-02 2020-01-01 00:07:40 2020-01-02 03:36:01 2020-01-01 00:07:40.000 2020-01-02 03:36:01.000 460 99361 49910.5 4991050 460 99361 49910.5 4991050 -32710 32428 4690.66 469066 -128 127 -1.82 -182 +461 101 10451 99362 1.38438 298.38438 149.88438 14988.43843 1.38438 298.3844 149.88438 14988.43864 1.38438 298.38438 149.88438 14988.43800 2020-01-01 2020-01-02 2020-01-01 00:07:41 2020-01-02 03:36:02 2020-01-01 00:07:41.000 2020-01-02 03:36:02.000 461 99362 49911.5 4991150 461 99362 49911.5 4991150 -32709 32429 4691.66 469166 -128 123 -3.38 -338 +462 101 10452 99363 1.38738 298.38738 149.88738 14988.73873 1.38738 298.3874 149.88738 14988.73894 1.38738 298.38738 149.8873800000001 14988.73800 2020-01-01 2020-01-02 2020-01-01 00:07:42 2020-01-02 03:36:03 2020-01-01 00:07:42.000 2020-01-02 03:36:03.000 462 99363 49912.5 4991250 462 99363 49912.5 4991250 -32708 32430 4692.66 469266 -127 124 -2.38 -238 463 101 10453 99364 1.39039 298.39039 149.89039 14989.03903 1.39039 298.39038 149.89039 14989.03907 1.39039 298.39039 149.89039 14989.03900 2020-01-01 2020-01-02 2020-01-01 00:07:43 2020-01-02 03:36:04 2020-01-01 00:07:43.000 2020-01-02 03:36:04.000 463 99364 49913.5 4991350 463 99364 49913.5 4991350 -32707 32431 4693.66 469366 -126 125 -1.38 -138 -464 101 10454 99365 1.39339 298.39339 149.89339 14989.33933 1.39339 298.3934 149.89339 14989.33922 1.39339 298.39339 149.89338999999998 14989.33900 2020-01-01 2020-01-02 2020-01-01 00:07:44 2020-01-02 03:36:05 2020-01-01 00:07:44.000 2020-01-02 03:36:05.000 464 99365 49914.5 4991450 464 99365 49914.5 4991450 -32706 32432 4694.66 469466 -125 126 -0.38 -38 +464 101 10454 99365 1.39339 298.39339 149.89339 14989.33933 1.39339 298.3934 149.89339 14989.33922 1.39339 298.39339 149.89338999999987 14989.33900 2020-01-01 2020-01-02 2020-01-01 00:07:44 2020-01-02 03:36:05 2020-01-01 00:07:44.000 2020-01-02 03:36:05.000 464 99365 49914.5 4991450 464 99365 49914.5 4991450 -32706 32432 4694.66 469466 -125 126 -0.38 -38 465 101 10455 99366 1.39639 298.39639 149.89639 14989.63963 1.39639 298.3964 149.89639 14989.63936 1.39639 298.39639 149.8963899999999 14989.63900 2020-01-01 2020-01-02 2020-01-01 00:07:45 2020-01-02 03:36:06 2020-01-01 00:07:45.000 2020-01-02 03:36:06.000 465 99366 49915.5 4991550 465 99366 49915.5 4991550 -32705 32433 4695.66 469566 -124 127 0.62 62 -466 101 10456 99367 1.39939 298.39939 149.89939 14989.93993 1.39939 298.3994 149.8994 14989.94011 1.39939 298.39939 149.89938999999998 14989.93900 2020-01-01 2020-01-02 2020-01-01 00:07:46 2020-01-02 03:36:07 2020-01-01 00:07:46.000 2020-01-02 03:36:07.000 466 99367 49916.5 4991650 466 99367 49916.5 4991650 -32704 32434 4696.66 469666 -128 127 -0.94 -94 -467 101 10457 99368 1.4024 298.4024 149.9024 14990.24024 1.4024 298.4024 149.9024 14990.24041 1.40240 298.40240 149.90240000000009 14990.24000 2020-01-01 2020-01-02 2020-01-01 00:07:47 2020-01-02 03:36:08 2020-01-01 00:07:47.000 2020-01-02 03:36:08.000 467 99368 49917.5 4991750 467 99368 49917.5 4991750 -32703 32435 4697.66 469766 -128 123 -2.5 -250 -468 101 10458 99369 1.4054 298.4054 149.9054 14990.54054 1.4054 298.4054 149.9054 14990.54058 1.40540 298.40540 149.90539999999993 14990.54000 2020-01-01 2020-01-02 2020-01-01 00:07:48 2020-01-02 03:36:09 2020-01-01 00:07:48.000 2020-01-02 03:36:09.000 468 99369 49918.5 4991850 468 99369 49918.5 4991850 -32702 32436 4698.66 469866 -127 124 -1.5 -150 -469 101 10459 99370 1.4084 298.4084 149.9084 14990.84084 1.4084 298.40842 149.9084 14990.8407 1.40840 298.40840 149.90840000000003 14990.84000 2020-01-01 2020-01-02 2020-01-01 00:07:49 2020-01-02 03:36:10 2020-01-01 00:07:49.000 2020-01-02 03:36:10.000 469 99370 49919.5 4991950 469 99370 49919.5 4991950 -32701 32437 4699.66 469966 -126 125 -0.5 -50 -47 102 10037 99947 0.14114 300.14114 150.14114 15164.25525 0.14114 300.14114 150.14114 15164.25545 0.14114 300.14114 150.14113999999995 15164.25514 2020-01-01 2020-01-02 2020-01-01 00:00:47 2020-01-02 03:45:47 2020-01-01 00:00:47.000 2020-01-02 03:45:47.000 47 99947 49997 5049697 47 99947 49997 5049697 -32522 32413 4576.009900990099 462177 -128 127 -1.5742574257425743 -159 -470 101 10460 99371 1.41141 298.41141 149.91141 14991.14114 1.41141 298.4114 149.9114 14991.14099 1.41141 298.41141 149.91141000000005 14991.14100 2020-01-01 2020-01-02 2020-01-01 00:07:50 2020-01-02 03:36:11 2020-01-01 00:07:50.000 2020-01-02 03:36:11.000 470 99371 49920.5 4992050 470 99371 49920.5 4992050 -32700 32438 4700.66 470066 -125 126 0.5 50 -471 101 10461 99372 1.41441 298.41441 149.91441 14991.44144 1.41441 298.41443 149.91441 14991.44159 1.41441 298.41441 149.91440999999995 14991.44100 2020-01-01 2020-01-02 2020-01-01 00:07:51 2020-01-02 03:36:12 2020-01-01 00:07:51.000 2020-01-02 03:36:12.000 471 99372 49921.5 4992150 471 99372 49921.5 4992150 -32699 32439 4701.66 470166 -124 127 1.5 150 -472 101 10462 99373 1.41741 298.41741 149.91741 14991.74174 1.41741 298.41742 149.91741 14991.74188 1.41741 298.41741 149.91741 14991.74100 2020-01-01 2020-01-02 2020-01-01 00:07:52 2020-01-02 03:36:13 2020-01-01 00:07:52.000 2020-01-02 03:36:13.000 472 99373 49922.5 4992250 472 99373 49922.5 4992250 -32698 32440 4702.66 470266 -128 127 -0.06 -6 -473 101 10463 99374 1.42042 298.42042 149.92042 14992.04204 1.42042 298.4204 149.92042 14992.04204 1.42042 298.42042 149.9204200000001 14992.04200 2020-01-01 2020-01-02 2020-01-01 00:07:53 2020-01-02 03:36:14 2020-01-01 00:07:53.000 2020-01-02 03:36:14.000 473 99374 49923.5 4992350 473 99374 49923.5 4992350 -32697 32441 4703.66 470366 -128 123 -1.62 -162 -474 101 10464 99375 1.42342 298.42342 149.92342 14992.34234 1.42342 298.42343 149.92342 14992.34216 1.42342 298.42342 149.9234199999999 14992.34200 2020-01-01 2020-01-02 2020-01-01 00:07:54 2020-01-02 03:36:15 2020-01-01 00:07:54.000 2020-01-02 03:36:15.000 474 99375 49924.5 4992450 474 99375 49924.5 4992450 -32696 32442 4704.66 470466 -127 124 -0.62 -62 -475 101 10465 99376 1.42642 298.42642 149.92642 14992.64264 1.42642 298.42642 149.92642 14992.64246 1.42642 298.42642 149.9264199999999 14992.64200 2020-01-01 2020-01-02 2020-01-01 00:07:55 2020-01-02 03:36:16 2020-01-01 00:07:55.000 2020-01-02 03:36:16.000 475 99376 49925.5 4992550 475 99376 49925.5 4992550 -32695 32443 4705.66 470566 -126 125 0.38 38 -476 101 10466 99377 1.42942 298.42942 149.92942 14992.94294 1.42942 298.42944 149.92943 14992.94305 1.42942 298.42942 149.9294200000001 14992.94200 2020-01-01 2020-01-02 2020-01-01 00:07:56 2020-01-02 03:36:17 2020-01-01 00:07:56.000 2020-01-02 03:36:17.000 476 99377 49926.5 4992650 476 99377 49926.5 4992650 -32694 32444 4706.66 470666 -125 126 1.38 138 -477 101 10467 99378 1.43243 298.43243 149.93243 14993.24324 1.43243 298.43243 149.93243 14993.24338 1.43243 298.43243 149.93243000000004 14993.24300 2020-01-01 2020-01-02 2020-01-01 00:07:57 2020-01-02 03:36:18 2020-01-01 00:07:57.000 2020-01-02 03:36:18.000 477 99378 49927.5 4992750 477 99378 49927.5 4992750 -32693 32445 4707.66 470766 -124 127 2.38 238 -478 101 10468 99379 1.43543 298.43543 149.93543 14993.54354 1.43543 298.43542 149.93543 14993.54352 1.43543 298.43543 149.93542999999983 14993.54300 2020-01-01 2020-01-02 2020-01-01 00:07:58 2020-01-02 03:36:19 2020-01-01 00:07:58.000 2020-01-02 03:36:19.000 478 99379 49928.5 4992850 478 99379 49928.5 4992850 -32692 32446 4708.66 470866 -128 127 0.82 82 -479 101 10469 99380 1.43843 298.43843 149.93843 14993.84384 1.43843 298.43845 149.93844 14993.84427 1.43843 298.43843 149.93843000000007 14993.84300 2020-01-01 2020-01-02 2020-01-01 00:07:59 2020-01-02 03:36:20 2020-01-01 00:07:59.000 2020-01-02 03:36:20.000 479 99380 49929.5 4992950 479 99380 49929.5 4992950 -32691 32447 4709.66 470966 -128 127 -0.74 -74 -48 102 10038 99948 0.14414 300.14414 150.14414 15164.55855 0.14414 300.14413 150.14414 15164.55863 0.14414 300.14414 150.14414000000005 15164.55814 2020-01-01 2020-01-02 2020-01-01 00:00:48 2020-01-02 03:45:48 2020-01-01 00:00:48.000 2020-01-02 03:45:48.000 48 99948 49998 5049798 48 99948 49998 5049798 -32521 32414 4577.009900990099 462278 -128 127 -3.108910891089109 -314 -480 101 10470 99381 1.44144 298.44144 149.94144 14994.14414 1.44144 298.44144 149.94143 14994.14392 1.44144 298.44144 149.94144 14994.14400 2020-01-01 2020-01-02 2020-01-01 00:08:00 2020-01-02 03:36:21 2020-01-01 00:08:00.000 2020-01-02 03:36:21.000 480 99381 49930.5 4993050 480 99381 49930.5 4993050 -32690 32448 4710.66 471066 -128 124 -2.3 -230 -481 101 10471 99382 1.44444 298.44444 149.94444 14994.44444 1.44444 298.44446 149.94444 14994.44452 1.44444 298.44444 149.94444 14994.44400 2020-01-01 2020-01-02 2020-01-01 00:08:01 2020-01-02 03:36:22 2020-01-01 00:08:01.000 2020-01-02 03:36:22.000 481 99382 49931.5 4993150 481 99382 49931.5 4993150 -32689 32449 4711.66 471166 -127 125 -1.3 -130 +466 101 10456 99367 1.39939 298.39939 149.89939 14989.93993 1.39939 298.3994 149.8994 14989.94011 1.39939 298.39939 149.89939 14989.93900 2020-01-01 2020-01-02 2020-01-01 00:07:46 2020-01-02 03:36:07 2020-01-01 00:07:46.000 2020-01-02 03:36:07.000 466 99367 49916.5 4991650 466 99367 49916.5 4991650 -32704 32434 4696.66 469666 -128 127 -0.94 -94 +467 101 10457 99368 1.4024 298.4024 149.9024 14990.24024 1.4024 298.4024 149.9024 14990.24041 1.40240 298.40240 149.90240000000023 14990.24000 2020-01-01 2020-01-02 2020-01-01 00:07:47 2020-01-02 03:36:08 2020-01-01 00:07:47.000 2020-01-02 03:36:08.000 467 99368 49917.5 4991750 467 99368 49917.5 4991750 -32703 32435 4697.66 469766 -128 123 -2.5 -250 +468 101 10458 99369 1.4054 298.4054 149.9054 14990.54054 1.4054 298.4054 149.9054 14990.54058 1.40540 298.40540 149.9053999999998 14990.54000 2020-01-01 2020-01-02 2020-01-01 00:07:48 2020-01-02 03:36:09 2020-01-01 00:07:48.000 2020-01-02 03:36:09.000 468 99369 49918.5 4991850 468 99369 49918.5 4991850 -32702 32436 4698.66 469866 -127 124 -1.5 -150 +469 101 10459 99370 1.4084 298.4084 149.9084 14990.84084 1.4084 298.40842 149.9084 14990.8407 1.40840 298.40840 149.90840000000014 14990.84000 2020-01-01 2020-01-02 2020-01-01 00:07:49 2020-01-02 03:36:10 2020-01-01 00:07:49.000 2020-01-02 03:36:10.000 469 99370 49919.5 4991950 469 99370 49919.5 4991950 -32701 32437 4699.66 469966 -126 125 -0.5 -50 +47 102 10037 99947 0.14114 300.14114 150.14114 15164.25525 0.14114 300.14114 150.14114 15164.25545 0.14114 300.14114 150.14113999999984 15164.25514 2020-01-01 2020-01-02 2020-01-01 00:00:47 2020-01-02 03:45:47 2020-01-01 00:00:47.000 2020-01-02 03:45:47.000 47 99947 49997 5049697 47 99947 49997 5049697 -32522 32413 4576.009900990099 462177 -128 127 -1.5742574257425743 -159 +470 101 10460 99371 1.41141 298.41141 149.91141 14991.14114 1.41141 298.4114 149.9114 14991.14099 1.41141 298.41141 149.91141000000013 14991.14100 2020-01-01 2020-01-02 2020-01-01 00:07:50 2020-01-02 03:36:11 2020-01-01 00:07:50.000 2020-01-02 03:36:11.000 470 99371 49920.5 4992050 470 99371 49920.5 4992050 -32700 32438 4700.66 470066 -125 126 0.5 50 +471 101 10461 99372 1.41441 298.41441 149.91441 14991.44144 1.41441 298.41443 149.91441 14991.44159 1.41441 298.41441 149.9144099999999 14991.44100 2020-01-01 2020-01-02 2020-01-01 00:07:51 2020-01-02 03:36:12 2020-01-01 00:07:51.000 2020-01-02 03:36:12.000 471 99372 49921.5 4992150 471 99372 49921.5 4992150 -32699 32439 4701.66 470166 -124 127 1.5 150 +472 101 10462 99373 1.41741 298.41741 149.91741 14991.74174 1.41741 298.41742 149.91741 14991.74188 1.41741 298.41741 149.91741000000002 14991.74100 2020-01-01 2020-01-02 2020-01-01 00:07:52 2020-01-02 03:36:13 2020-01-01 00:07:52.000 2020-01-02 03:36:13.000 472 99373 49922.5 4992250 472 99373 49922.5 4992250 -32698 32440 4702.66 470266 -128 127 -0.06 -6 +473 101 10463 99374 1.42042 298.42042 149.92042 14992.04204 1.42042 298.4204 149.92042 14992.04204 1.42042 298.42042 149.92042000000023 14992.04200 2020-01-01 2020-01-02 2020-01-01 00:07:53 2020-01-02 03:36:14 2020-01-01 00:07:53.000 2020-01-02 03:36:14.000 473 99374 49923.5 4992350 473 99374 49923.5 4992350 -32697 32441 4703.66 470366 -128 123 -1.62 -162 +474 101 10464 99375 1.42342 298.42342 149.92342 14992.34234 1.42342 298.42343 149.92342 14992.34216 1.42342 298.42342 149.9234199999998 14992.34200 2020-01-01 2020-01-02 2020-01-01 00:07:54 2020-01-02 03:36:15 2020-01-01 00:07:54.000 2020-01-02 03:36:15.000 474 99375 49924.5 4992450 474 99375 49924.5 4992450 -32696 32442 4704.66 470466 -127 124 -0.62 -62 +475 101 10465 99376 1.42642 298.42642 149.92642 14992.64264 1.42642 298.42642 149.92642 14992.64246 1.42642 298.42642 149.9264199999998 14992.64200 2020-01-01 2020-01-02 2020-01-01 00:07:55 2020-01-02 03:36:16 2020-01-01 00:07:55.000 2020-01-02 03:36:16.000 475 99376 49925.5 4992550 475 99376 49925.5 4992550 -32695 32443 4705.66 470566 -126 125 0.38 38 +476 101 10466 99377 1.42942 298.42942 149.92942 14992.94294 1.42942 298.42944 149.92943 14992.94305 1.42942 298.42942 149.92942000000025 14992.94200 2020-01-01 2020-01-02 2020-01-01 00:07:56 2020-01-02 03:36:17 2020-01-01 00:07:56.000 2020-01-02 03:36:17.000 476 99377 49926.5 4992650 476 99377 49926.5 4992650 -32694 32444 4706.66 470666 -125 126 1.38 138 +477 101 10467 99378 1.43243 298.43243 149.93243 14993.24324 1.43243 298.43243 149.93243 14993.24338 1.43243 298.43243 149.93243000000015 14993.24300 2020-01-01 2020-01-02 2020-01-01 00:07:57 2020-01-02 03:36:18 2020-01-01 00:07:57.000 2020-01-02 03:36:18.000 477 99378 49927.5 4992750 477 99378 49927.5 4992750 -32693 32445 4707.66 470766 -124 127 2.38 238 +478 101 10468 99379 1.43543 298.43543 149.93543 14993.54354 1.43543 298.43542 149.93543 14993.54352 1.43543 298.43543 149.93542999999968 14993.54300 2020-01-01 2020-01-02 2020-01-01 00:07:58 2020-01-02 03:36:19 2020-01-01 00:07:58.000 2020-01-02 03:36:19.000 478 99379 49928.5 4992850 478 99379 49928.5 4992850 -32692 32446 4708.66 470866 -128 127 0.82 82 +479 101 10469 99380 1.43843 298.43843 149.93843 14993.84384 1.43843 298.43845 149.93844 14993.84427 1.43843 298.43843 149.93843000000012 14993.84300 2020-01-01 2020-01-02 2020-01-01 00:07:59 2020-01-02 03:36:20 2020-01-01 00:07:59.000 2020-01-02 03:36:20.000 479 99380 49929.5 4992950 479 99380 49929.5 4992950 -32691 32447 4709.66 470966 -128 127 -0.74 -74 +48 102 10038 99948 0.14414 300.14414 150.14414 15164.55855 0.14414 300.14413 150.14414 15164.55863 0.14414 300.14414 150.14414000000022 15164.55814 2020-01-01 2020-01-02 2020-01-01 00:00:48 2020-01-02 03:45:48 2020-01-01 00:00:48.000 2020-01-02 03:45:48.000 48 99948 49998 5049798 48 99948 49998 5049798 -32521 32414 4577.009900990099 462278 -128 127 -3.108910891089109 -314 +480 101 10470 99381 1.44144 298.44144 149.94144 14994.14414 1.44144 298.44144 149.94143 14994.14392 1.44144 298.44144 149.94144000000003 14994.14400 2020-01-01 2020-01-02 2020-01-01 00:08:00 2020-01-02 03:36:21 2020-01-01 00:08:00.000 2020-01-02 03:36:21.000 480 99381 49930.5 4993050 480 99381 49930.5 4993050 -32690 32448 4710.66 471066 -128 124 -2.3 -230 +481 101 10471 99382 1.44444 298.44444 149.94444 14994.44444 1.44444 298.44446 149.94444 14994.44452 1.44444 298.44444 149.9444399999999 14994.44400 2020-01-01 2020-01-02 2020-01-01 00:08:01 2020-01-02 03:36:22 2020-01-01 00:08:01.000 2020-01-02 03:36:22.000 481 99382 49931.5 4993150 481 99382 49931.5 4993150 -32689 32449 4711.66 471166 -127 125 -1.3 -130 482 101 10472 99383 1.44744 298.44744 149.94744 14994.74474 1.44744 298.44745 149.94744 14994.74485 1.44744 298.44744 149.94743999999994 14994.74400 2020-01-01 2020-01-02 2020-01-01 00:08:02 2020-01-02 03:36:23 2020-01-01 00:08:02.000 2020-01-02 03:36:23.000 482 99383 49932.5 4993250 482 99383 49932.5 4993250 -32688 32450 4712.66 471266 -126 126 -0.3 -30 -483 101 10473 99384 1.45045 298.45045 149.95045 14995.04504 1.45045 298.45044 149.95044 14995.04499 1.45045 298.45045 149.95045000000005 14995.04500 2020-01-01 2020-01-02 2020-01-01 00:08:03 2020-01-02 03:36:24 2020-01-01 00:08:03.000 2020-01-02 03:36:24.000 483 99384 49933.5 4993350 483 99384 49933.5 4993350 -32687 32451 4713.66 471366 -125 127 0.7 70 -484 101 10474 99385 1.45345 298.45345 149.95345 14995.34534 1.45345 298.45346 149.95345 14995.34574 1.45345 298.45345 149.95345000000012 14995.34500 2020-01-01 2020-01-02 2020-01-01 00:08:04 2020-01-02 03:36:25 2020-01-01 00:08:04.000 2020-01-02 03:36:25.000 484 99385 49934.5 4993450 484 99385 49934.5 4993450 -32686 32452 4714.66 471466 -128 127 -0.86 -86 -485 101 10475 99386 1.45645 298.45645 149.95645 14995.64564 1.45645 298.45645 149.95645 14995.6454 1.45645 298.45645 149.9564499999999 14995.64500 2020-01-01 2020-01-02 2020-01-01 00:08:05 2020-01-02 03:36:26 2020-01-01 00:08:05.000 2020-01-02 03:36:26.000 485 99386 49935.5 4993550 485 99386 49935.5 4993550 -32685 32453 4715.66 471566 -128 127 -2.42 -242 -486 101 10476 99387 1.45945 298.45945 149.95945 14995.94594 1.45945 298.45947 149.95946 14995.94602 1.45945 298.45945 149.95945000000006 14995.94500 2020-01-01 2020-01-02 2020-01-01 00:08:06 2020-01-02 03:36:27 2020-01-01 00:08:06.000 2020-01-02 03:36:27.000 486 99387 49936.5 4993650 486 99387 49936.5 4993650 -32684 32454 4716.66 471666 -128 123 -3.98 -398 -487 101 10477 99388 1.46246 298.46246 149.96246 14996.24624 1.46246 298.46246 149.96246 14996.24633 1.46246 298.46246 149.96246000000008 14996.24600 2020-01-01 2020-01-02 2020-01-01 00:08:07 2020-01-02 03:36:28 2020-01-01 00:08:07.000 2020-01-02 03:36:28.000 487 99388 49937.5 4993750 487 99388 49937.5 4993750 -32683 32455 4717.66 471766 -127 124 -2.98 -298 -488 101 10478 99389 1.46546 298.46546 149.96546 14996.54654 1.46546 298.46545 149.96546 14996.54645 1.46546 298.46546 149.96545999999998 14996.54600 2020-01-01 2020-01-02 2020-01-01 00:08:08 2020-01-02 03:36:29 2020-01-01 00:08:08.000 2020-01-02 03:36:29.000 488 99389 49938.5 4993850 488 99389 49938.5 4993850 -32682 32456 4718.66 471866 -126 125 -1.98 -198 +483 101 10473 99384 1.45045 298.45045 149.95045 14995.04504 1.45045 298.45044 149.95044 14995.04499 1.45045 298.45045 149.95045000000016 14995.04500 2020-01-01 2020-01-02 2020-01-01 00:08:03 2020-01-02 03:36:24 2020-01-01 00:08:03.000 2020-01-02 03:36:24.000 483 99384 49933.5 4993350 483 99384 49933.5 4993350 -32687 32451 4713.66 471366 -125 127 0.7 70 +484 101 10474 99385 1.45345 298.45345 149.95345 14995.34534 1.45345 298.45346 149.95345 14995.34574 1.45345 298.45345 149.95345000000026 14995.34500 2020-01-01 2020-01-02 2020-01-01 00:08:04 2020-01-02 03:36:25 2020-01-01 00:08:04.000 2020-01-02 03:36:25.000 484 99385 49934.5 4993450 484 99385 49934.5 4993450 -32686 32452 4714.66 471466 -128 127 -0.86 -86 +485 101 10475 99386 1.45645 298.45645 149.95645 14995.64564 1.45645 298.45645 149.95645 14995.6454 1.45645 298.45645 149.95644999999982 14995.64500 2020-01-01 2020-01-02 2020-01-01 00:08:05 2020-01-02 03:36:26 2020-01-01 00:08:05.000 2020-01-02 03:36:26.000 485 99386 49935.5 4993550 485 99386 49935.5 4993550 -32685 32453 4715.66 471566 -128 127 -2.42 -242 +486 101 10476 99387 1.45945 298.45945 149.95945 14995.94594 1.45945 298.45947 149.95946 14995.94602 1.45945 298.45945 149.95945000000017 14995.94500 2020-01-01 2020-01-02 2020-01-01 00:08:06 2020-01-02 03:36:27 2020-01-01 00:08:06.000 2020-01-02 03:36:27.000 486 99387 49936.5 4993650 486 99387 49936.5 4993650 -32684 32454 4716.66 471666 -128 123 -3.98 -398 +487 101 10477 99388 1.46246 298.46246 149.96246 14996.24624 1.46246 298.46246 149.96246 14996.24633 1.46246 298.46246 149.96246000000014 14996.24600 2020-01-01 2020-01-02 2020-01-01 00:08:07 2020-01-02 03:36:28 2020-01-01 00:08:07.000 2020-01-02 03:36:28.000 487 99388 49937.5 4993750 487 99388 49937.5 4993750 -32683 32455 4717.66 471766 -127 124 -2.98 -298 +488 101 10478 99389 1.46546 298.46546 149.96546 14996.54654 1.46546 298.46545 149.96546 14996.54645 1.46546 298.46546 149.96545999999995 14996.54600 2020-01-01 2020-01-02 2020-01-01 00:08:08 2020-01-02 03:36:29 2020-01-01 00:08:08.000 2020-01-02 03:36:29.000 488 99389 49938.5 4993850 488 99389 49938.5 4993850 -32682 32456 4718.66 471866 -126 125 -1.98 -198 489 101 10479 99390 1.46846 298.46846 149.96846 14996.84684 1.46846 298.46848 149.96847 14996.84721 1.46846 298.46846 149.96846000000002 14996.84600 2020-01-01 2020-01-02 2020-01-01 00:08:09 2020-01-02 03:36:30 2020-01-01 00:08:09.000 2020-01-02 03:36:30.000 489 99390 49939.5 4993950 489 99390 49939.5 4993950 -32681 32457 4719.66 471966 -125 126 -0.98 -98 -49 102 10039 99949 0.14714 300.14714 150.14714 15164.86186 0.14714 300.14716 150.14714 15164.86173 0.14714 300.14714 150.14713999999992 15164.86114 2020-01-01 2020-01-02 2020-01-01 00:00:49 2020-01-02 03:45:49 2020-01-01 00:00:49.000 2020-01-02 03:45:49.000 49 99949 49999 5049899 49 99949 49999 5049899 -32520 32415 4578.009900990099 462379 -128 123 -4.643564356435643 -469 -490 101 10480 99391 1.47147 298.47147 149.97147 14997.14714 1.47147 298.47147 149.97146 14997.14687 1.47147 298.47147 149.97147000000012 14997.14700 2020-01-01 2020-01-02 2020-01-01 00:08:10 2020-01-02 03:36:31 2020-01-01 00:08:10.000 2020-01-02 03:36:31.000 490 99391 49940.5 4994050 490 99391 49940.5 4994050 -32680 32458 4720.66 472066 -124 127 0.02 2 -491 101 10481 99392 1.47447 298.47447 149.97447 14997.44744 1.47447 298.4745 149.97447 14997.44749 1.47447 298.47447 149.97446999999997 14997.44700 2020-01-01 2020-01-02 2020-01-01 00:08:11 2020-01-02 03:36:32 2020-01-01 00:08:11.000 2020-01-02 03:36:32.000 491 99392 49941.5 4994150 491 99392 49941.5 4994150 -32679 32459 4721.66 472166 -128 127 -1.54 -154 -492 101 10482 99393 1.47747 298.47747 149.97747 14997.74774 1.47747 298.47748 149.97747 14997.74779 1.47747 298.47747 149.9774699999999 14997.74700 2020-01-01 2020-01-02 2020-01-01 00:08:12 2020-01-02 03:36:33 2020-01-01 00:08:12.000 2020-01-02 03:36:33.000 492 99393 49942.5 4994250 492 99393 49942.5 4994250 -32678 32460 4722.66 472266 -128 123 -3.1 -310 -493 101 10483 99394 1.48048 298.48048 149.98048 14998.04804 1.48048 298.48047 149.98048 14998.04809 1.48048 298.48048 149.98048000000009 14998.04800 2020-01-01 2020-01-02 2020-01-01 00:08:13 2020-01-02 03:36:34 2020-01-01 00:08:13.000 2020-01-02 03:36:34.000 493 99394 49943.5 4994350 493 99394 49943.5 4994350 -32677 32461 4723.66 472366 -127 124 -2.1 -210 -494 101 10484 99395 1.48348 298.48348 149.98348 14998.34834 1.48348 298.4835 149.98348 14998.34868 1.48348 298.48348 149.98348000000004 14998.34800 2020-01-01 2020-01-02 2020-01-01 00:08:14 2020-01-02 03:36:35 2020-01-01 00:08:14.000 2020-01-02 03:36:35.000 494 99395 49944.5 4994450 494 99395 49944.5 4994450 -32676 32462 4724.66 472466 -126 125 -1.1 -110 -495 100 10485 99396 1.48648 298.48648 149.98648 14998.64864 1.48648 298.48648 149.98648 14998.64837 1.48648 298.48648 149.98647999999986 14998.64800 2020-01-01 2020-01-02 2020-01-01 00:08:15 2020-01-02 03:36:36 2020-01-01 00:08:15.000 2020-01-02 03:36:36.000 495 99396 49945.5 4994550 495 99396 49945.5 4994550 -32675 32463 4725.66 472566 -125 126 -0.1 -10 -496 100 10486 99397 1.48948 298.48948 149.98948 14998.94894 1.48948 298.4895 149.98948 14998.94896 1.48948 298.48948 149.9894800000001 14998.94800 2020-01-01 2020-01-02 2020-01-01 00:08:16 2020-01-02 03:36:37 2020-01-01 00:08:16.000 2020-01-02 03:36:37.000 496 99397 49946.5 4994650 496 99397 49946.5 4994650 -32674 32464 4726.66 472666 -124 127 0.9 90 -497 100 10487 99398 1.49249 298.49249 149.99249 14999.24924 1.49249 298.4925 149.99249 14999.24926 1.49249 298.49249 149.99249 14999.24900 2020-01-01 2020-01-02 2020-01-01 00:08:17 2020-01-02 03:36:38 2020-01-01 00:08:17.000 2020-01-02 03:36:38.000 497 99398 49947.5 4994750 497 99398 49947.5 4994750 -32673 32465 4727.66 472766 -128 127 -0.66 -66 -498 100 10488 99399 1.49549 298.49549 149.99549 14999.54954 1.49549 298.49548 149.99549 14999.54956 1.49549 298.49549 149.99548999999996 14999.54900 2020-01-01 2020-01-02 2020-01-01 00:08:18 2020-01-02 03:36:39 2020-01-01 00:08:18.000 2020-01-02 03:36:39.000 498 99399 49948.5 4994850 498 99399 49948.5 4994850 -32672 32466 4728.66 472866 -128 123 -2.22 -222 +49 102 10039 99949 0.14714 300.14714 150.14714 15164.86186 0.14714 300.14716 150.14714 15164.86173 0.14714 300.14714 150.14713999999972 15164.86114 2020-01-01 2020-01-02 2020-01-01 00:00:49 2020-01-02 03:45:49 2020-01-01 00:00:49.000 2020-01-02 03:45:49.000 49 99949 49999 5049899 49 99949 49999 5049899 -32520 32415 4578.009900990099 462379 -128 123 -4.643564356435643 -469 +490 101 10480 99391 1.47147 298.47147 149.97147 14997.14714 1.47147 298.47147 149.97146 14997.14687 1.47147 298.47147 149.9714700000003 14997.14700 2020-01-01 2020-01-02 2020-01-01 00:08:10 2020-01-02 03:36:31 2020-01-01 00:08:10.000 2020-01-02 03:36:31.000 490 99391 49940.5 4994050 490 99391 49940.5 4994050 -32680 32458 4720.66 472066 -124 127 0.02 2 +491 101 10481 99392 1.47447 298.47447 149.97447 14997.44744 1.47447 298.4745 149.97447 14997.44749 1.47447 298.47447 149.9744699999998 14997.44700 2020-01-01 2020-01-02 2020-01-01 00:08:11 2020-01-02 03:36:32 2020-01-01 00:08:11.000 2020-01-02 03:36:32.000 491 99392 49941.5 4994150 491 99392 49941.5 4994150 -32679 32459 4721.66 472166 -128 127 -1.54 -154 +492 101 10482 99393 1.47747 298.47747 149.97747 14997.74774 1.47747 298.47748 149.97747 14997.74779 1.47747 298.47747 149.97746999999984 14997.74700 2020-01-01 2020-01-02 2020-01-01 00:08:12 2020-01-02 03:36:33 2020-01-01 00:08:12.000 2020-01-02 03:36:33.000 492 99393 49942.5 4994250 492 99393 49942.5 4994250 -32678 32460 4722.66 472266 -128 123 -3.1 -310 +493 101 10483 99394 1.48048 298.48048 149.98048 14998.04804 1.48048 298.48047 149.98048 14998.04809 1.48048 298.48048 149.98048000000014 14998.04800 2020-01-01 2020-01-02 2020-01-01 00:08:13 2020-01-02 03:36:34 2020-01-01 00:08:13.000 2020-01-02 03:36:34.000 493 99394 49943.5 4994350 493 99394 49943.5 4994350 -32677 32461 4723.66 472366 -127 124 -2.1 -210 +494 101 10484 99395 1.48348 298.48348 149.98348 14998.34834 1.48348 298.4835 149.98348 14998.34868 1.48348 298.48348 149.98348000000018 14998.34800 2020-01-01 2020-01-02 2020-01-01 00:08:14 2020-01-02 03:36:35 2020-01-01 00:08:14.000 2020-01-02 03:36:35.000 494 99395 49944.5 4994450 494 99395 49944.5 4994450 -32676 32462 4724.66 472466 -126 125 -1.1 -110 +495 100 10485 99396 1.48648 298.48648 149.98648 14998.64864 1.48648 298.48648 149.98648 14998.64837 1.48648 298.48648 149.9864799999997 14998.64800 2020-01-01 2020-01-02 2020-01-01 00:08:15 2020-01-02 03:36:36 2020-01-01 00:08:15.000 2020-01-02 03:36:36.000 495 99396 49945.5 4994550 495 99396 49945.5 4994550 -32675 32463 4725.66 472566 -125 126 -0.1 -10 +496 100 10486 99397 1.48948 298.48948 149.98948 14998.94894 1.48948 298.4895 149.98948 14998.94896 1.48948 298.48948 149.98948000000016 14998.94800 2020-01-01 2020-01-02 2020-01-01 00:08:16 2020-01-02 03:36:37 2020-01-01 00:08:16.000 2020-01-02 03:36:37.000 496 99397 49946.5 4994650 496 99397 49946.5 4994650 -32674 32464 4726.66 472666 -124 127 0.9 90 +497 100 10487 99398 1.49249 298.49249 149.99249 14999.24924 1.49249 298.4925 149.99249 14999.24926 1.49249 298.49249 149.99249000000003 14999.24900 2020-01-01 2020-01-02 2020-01-01 00:08:17 2020-01-02 03:36:38 2020-01-01 00:08:17.000 2020-01-02 03:36:38.000 497 99398 49947.5 4994750 497 99398 49947.5 4994750 -32673 32465 4727.66 472766 -128 127 -0.66 -66 +498 100 10488 99399 1.49549 298.49549 149.99549 14999.54954 1.49549 298.49548 149.99549 14999.54956 1.49549 298.49549 149.99548999999988 14999.54900 2020-01-01 2020-01-02 2020-01-01 00:08:18 2020-01-02 03:36:39 2020-01-01 00:08:18.000 2020-01-02 03:36:39.000 498 99399 49948.5 4994850 498 99399 49948.5 4994850 -32672 32466 4728.66 472866 -128 123 -2.22 -222 499 100 10489 99400 1.49849 298.49849 149.99849 14999.84984 1.49849 298.4985 149.9985 14999.85015 1.49849 298.49849 149.99848999999998 14999.84900 2020-01-01 2020-01-02 2020-01-01 00:08:19 2020-01-02 03:36:40 2020-01-01 00:08:19.000 2020-01-02 03:36:40.000 499 99400 49949.5 4994950 499 99400 49949.5 4994950 -32671 32467 4729.66 472966 -127 124 -1.22 -122 -5 102 1004 9995 0.01501 300.01501 150.01501 15151.51651 0.01501 300.015 150.01501 15151.51648 0.01501 300.01501 150.01500999999993 15151.51601 2020-01-01 2020-01-02 2020-01-01 00:00:05 2020-01-02 03:45:05 2020-01-01 00:00:05.000 2020-01-02 03:45:05.000 5 99905 49955 5045455 5 99905 49955 5045455 -32564 32371 4534.009900990099 457935 -128 123 -3.01980198019802 -305 +5 102 1004 9995 0.01501 300.01501 150.01501 15151.51651 0.01501 300.015 150.01501 15151.51648 0.01501 300.01501 150.01500999999976 15151.51601 2020-01-01 2020-01-02 2020-01-01 00:00:05 2020-01-02 03:45:05 2020-01-01 00:00:05.000 2020-01-02 03:45:05.000 5 99905 49955 5045455 5 99905 49955 5045455 -32564 32371 4534.009900990099 457935 -128 123 -3.01980198019802 -305 50 102 10040 99950 0.15015 300.15015 150.15015 15165.16516 0.15015 300.15015 150.15014 15165.16487 0.15015 300.15015 150.15015 15165.16515 2020-01-01 2020-01-02 2020-01-01 00:00:50 2020-01-02 03:45:50 2020-01-01 00:00:50.000 2020-01-02 03:45:50.000 50 99950 50000 5050000 50 99950 50000 5050000 -32519 32416 4579.009900990099 462480 -127 124 -3.6435643564356437 -368 -500 100 10490 99401 1.5015 298.5015 150.0015 15000.15015 1.5015 298.5015 150.00149 15000.14984 1.50150 298.50150 150.00150000000008 15000.15000 2020-01-01 2020-01-02 2020-01-01 00:08:20 2020-01-02 03:36:41 2020-01-01 00:08:20.000 2020-01-02 03:36:41.000 500 99401 49950.5 4995050 500 99401 49950.5 4995050 -32670 32468 4730.66 473066 -126 125 -0.22 -22 -501 100 10491 99402 1.5045 298.5045 150.0045 15000.45045 1.5045 298.50452 150.0045 15000.45043 1.50450 298.50450 150.0044999999999 15000.45000 2020-01-01 2020-01-02 2020-01-01 00:08:21 2020-01-02 03:36:42 2020-01-01 00:08:21.000 2020-01-02 03:36:42.000 501 99402 49951.5 4995150 501 99402 49951.5 4995150 -32669 32469 4731.66 473166 -125 126 0.78 78 -502 100 10492 99403 1.5075 298.5075 150.0075 15000.75075 1.5075 298.5075 150.0075 15000.75073 1.50750 298.50750 150.00749999999994 15000.75000 2020-01-01 2020-01-02 2020-01-01 00:08:22 2020-01-02 03:36:43 2020-01-01 00:08:22.000 2020-01-02 03:36:43.000 502 99403 49952.5 4995250 502 99403 49952.5 4995250 -32668 32470 4732.66 473266 -124 127 1.78 178 +500 100 10490 99401 1.5015 298.5015 150.0015 15000.15015 1.5015 298.5015 150.00149 15000.14984 1.50150 298.50150 150.0015000000002 15000.15000 2020-01-01 2020-01-02 2020-01-01 00:08:20 2020-01-02 03:36:41 2020-01-01 00:08:20.000 2020-01-02 03:36:41.000 500 99401 49950.5 4995050 500 99401 49950.5 4995050 -32670 32468 4730.66 473066 -126 125 -0.22 -22 +501 100 10491 99402 1.5045 298.5045 150.0045 15000.45045 1.5045 298.50452 150.0045 15000.45043 1.50450 298.50450 150.00449999999972 15000.45000 2020-01-01 2020-01-02 2020-01-01 00:08:21 2020-01-02 03:36:42 2020-01-01 00:08:21.000 2020-01-02 03:36:42.000 501 99402 49951.5 4995150 501 99402 49951.5 4995150 -32669 32469 4731.66 473166 -125 126 0.78 78 +502 100 10492 99403 1.5075 298.5075 150.0075 15000.75075 1.5075 298.5075 150.0075 15000.75073 1.50750 298.50750 150.00749999999982 15000.75000 2020-01-01 2020-01-02 2020-01-01 00:08:22 2020-01-02 03:36:43 2020-01-01 00:08:22.000 2020-01-02 03:36:43.000 502 99403 49952.5 4995250 502 99403 49952.5 4995250 -32668 32470 4732.66 473266 -124 127 1.78 178 503 100 10493 99404 1.51051 298.51051 150.01051 15001.05105 1.51051 298.5105 150.01051 15001.05103 1.51051 298.51051 150.01051000000004 15001.05100 2020-01-01 2020-01-02 2020-01-01 00:08:23 2020-01-02 03:36:44 2020-01-01 00:08:23.000 2020-01-02 03:36:44.000 503 99404 49953.5 4995350 503 99404 49953.5 4995350 -32667 32471 4733.66 473366 -128 127 0.22 22 -504 100 10494 99405 1.51351 298.51351 150.01351 15001.35135 1.51351 298.51352 150.01351 15001.35162 1.51351 298.51351 150.01351 15001.35100 2020-01-01 2020-01-02 2020-01-01 00:08:24 2020-01-02 03:36:45 2020-01-01 00:08:24.000 2020-01-02 03:36:45.000 504 99405 49954.5 4995450 504 99405 49954.5 4995450 -32666 32472 4734.66 473466 -128 127 -1.34 -134 -505 100 10495 99406 1.51651 298.51651 150.01651 15001.65165 1.51651 298.5165 150.01651 15001.65131 1.51651 298.51651 150.01651 15001.65100 2020-01-01 2020-01-02 2020-01-01 00:08:25 2020-01-02 03:36:46 2020-01-01 00:08:25.000 2020-01-02 03:36:46.000 505 99406 49955.5 4995550 505 99406 49955.5 4995550 -32665 32473 4735.66 473566 -128 124 -2.9 -290 +504 100 10494 99405 1.51351 298.51351 150.01351 15001.35135 1.51351 298.51352 150.01351 15001.35162 1.51351 298.51351 150.01351000000014 15001.35100 2020-01-01 2020-01-02 2020-01-01 00:08:24 2020-01-02 03:36:45 2020-01-01 00:08:24.000 2020-01-02 03:36:45.000 504 99405 49954.5 4995450 504 99405 49954.5 4995450 -32666 32472 4734.66 473466 -128 127 -1.34 -134 +505 100 10495 99406 1.51651 298.51651 150.01651 15001.65165 1.51651 298.5165 150.01651 15001.65131 1.51651 298.51651 150.01650999999998 15001.65100 2020-01-01 2020-01-02 2020-01-01 00:08:25 2020-01-02 03:36:46 2020-01-01 00:08:25.000 2020-01-02 03:36:46.000 505 99406 49955.5 4995550 505 99406 49955.5 4995550 -32665 32473 4735.66 473566 -128 124 -2.9 -290 506 100 10496 99407 1.51951 298.51951 150.01951 15001.95195 1.51951 298.51953 150.01951 15001.9519 1.51951 298.51951 150.01951000000005 15001.95100 2020-01-01 2020-01-02 2020-01-01 00:08:26 2020-01-02 03:36:47 2020-01-01 00:08:26.000 2020-01-02 03:36:47.000 506 99407 49956.5 4995650 506 99407 49956.5 4995650 -32664 32474 4736.66 473666 -127 125 -1.9 -190 -507 100 10497 99408 1.52252 298.52252 150.02252 15002.25225 1.52252 298.52252 150.02252 15002.2522 1.52252 298.52252 150.02252000000016 15002.25200 2020-01-01 2020-01-02 2020-01-01 00:08:27 2020-01-02 03:36:48 2020-01-01 00:08:27.000 2020-01-02 03:36:48.000 507 99408 49957.5 4995750 507 99408 49957.5 4995750 -32663 32475 4737.66 473766 -126 126 -0.9 -90 -508 100 10498 99409 1.52552 298.52552 150.02552 15002.55255 1.52552 298.5255 150.02552 15002.5525 1.52552 298.52552 150.02551999999994 15002.55200 2020-01-01 2020-01-02 2020-01-01 00:08:28 2020-01-02 03:36:49 2020-01-01 00:08:28.000 2020-01-02 03:36:49.000 508 99409 49958.5 4995850 508 99409 49958.5 4995850 -32662 32476 4738.66 473866 -125 127 0.1 10 -509 100 10499 99410 1.52852 298.52852 150.02852 15002.85285 1.52852 298.52853 150.02853 15002.85312 1.52852 298.52852 150.02851999999993 15002.85200 2020-01-01 2020-01-02 2020-01-01 00:08:29 2020-01-02 03:36:50 2020-01-01 00:08:29.000 2020-01-02 03:36:50.000 509 99410 49959.5 4995950 509 99410 49959.5 4995950 -32661 32477 4739.66 473966 -128 127 -1.46 -146 +507 100 10497 99408 1.52252 298.52252 150.02252 15002.25225 1.52252 298.52252 150.02252 15002.2522 1.52252 298.52252 150.02252000000033 15002.25200 2020-01-01 2020-01-02 2020-01-01 00:08:27 2020-01-02 03:36:48 2020-01-01 00:08:27.000 2020-01-02 03:36:48.000 507 99408 49957.5 4995750 507 99408 49957.5 4995750 -32663 32475 4737.66 473766 -126 126 -0.9 -90 +508 100 10498 99409 1.52552 298.52552 150.02552 15002.55255 1.52552 298.5255 150.02552 15002.5525 1.52552 298.52552 150.02551999999983 15002.55200 2020-01-01 2020-01-02 2020-01-01 00:08:28 2020-01-02 03:36:49 2020-01-01 00:08:28.000 2020-01-02 03:36:49.000 508 99409 49958.5 4995850 508 99409 49958.5 4995850 -32662 32476 4738.66 473866 -125 127 0.1 10 +509 100 10499 99410 1.52852 298.52852 150.02852 15002.85285 1.52852 298.52853 150.02853 15002.85312 1.52852 298.52852 150.02851999999987 15002.85200 2020-01-01 2020-01-02 2020-01-01 00:08:29 2020-01-02 03:36:50 2020-01-01 00:08:29.000 2020-01-02 03:36:50.000 509 99410 49959.5 4995950 509 99410 49959.5 4995950 -32661 32477 4739.66 473966 -128 127 -1.46 -146 51 102 10041 99951 0.15315 300.15315 150.15315 15165.46846 0.15315 300.15317 150.15315 15165.46863 0.15315 300.15315 150.15315000000007 15165.46815 2020-01-01 2020-01-02 2020-01-01 00:00:51 2020-01-02 03:45:51 2020-01-01 00:00:51.000 2020-01-02 03:45:51.000 51 99951 50001 5050101 51 99951 50001 5050101 -32518 32417 4580.009900990099 462581 -126 125 -2.6435643564356437 -267 -510 100 10500 99411 1.53153 298.53153 150.03153 15003.15315 1.53153 298.53152 150.03152 15003.15278 1.53153 298.53153 150.0315300000001 15003.15300 2020-01-01 2020-01-02 2020-01-01 00:08:30 2020-01-02 03:36:51 2020-01-01 00:08:30.000 2020-01-02 03:36:51.000 510 99411 49960.5 4996050 510 99411 49960.5 4996050 -32660 32478 4740.66 474066 -128 127 -3.02 -302 -511 100 10501 99412 1.53453 298.53453 150.03453 15003.45345 1.53453 298.53455 150.03453 15003.45354 1.53453 298.53453 150.03453000000005 15003.45300 2020-01-01 2020-01-02 2020-01-01 00:08:31 2020-01-02 03:36:52 2020-01-01 00:08:31.000 2020-01-02 03:36:52.000 511 99412 49961.5 4996150 511 99412 49961.5 4996150 -32659 32479 4741.66 474166 -128 123 -4.58 -458 -512 100 10502 99413 1.53753 298.53753 150.03753 15003.75375 1.53753 298.53754 150.03753 15003.75366 1.53753 298.53753 150.0375299999999 15003.75300 2020-01-01 2020-01-02 2020-01-01 00:08:32 2020-01-02 03:36:53 2020-01-01 00:08:32.000 2020-01-02 03:36:53.000 512 99413 49962.5 4996250 512 99413 49962.5 4996250 -32658 32480 4742.66 474266 -127 124 -3.58 -358 +510 100 10500 99411 1.53153 298.53153 150.03153 15003.15315 1.53153 298.53152 150.03152 15003.15278 1.53153 298.53153 150.03153000000017 15003.15300 2020-01-01 2020-01-02 2020-01-01 00:08:30 2020-01-02 03:36:51 2020-01-01 00:08:30.000 2020-01-02 03:36:51.000 510 99411 49960.5 4996050 510 99411 49960.5 4996050 -32660 32478 4740.66 474066 -128 127 -3.02 -302 +511 100 10501 99412 1.53453 298.53453 150.03453 15003.45345 1.53453 298.53455 150.03453 15003.45354 1.53453 298.53453 150.03453000000022 15003.45300 2020-01-01 2020-01-02 2020-01-01 00:08:31 2020-01-02 03:36:52 2020-01-01 00:08:31.000 2020-01-02 03:36:52.000 511 99412 49961.5 4996150 511 99412 49961.5 4996150 -32659 32479 4741.66 474166 -128 123 -4.58 -458 +512 100 10502 99413 1.53753 298.53753 150.03753 15003.75375 1.53753 298.53754 150.03753 15003.75366 1.53753 298.53753 150.03752999999972 15003.75300 2020-01-01 2020-01-02 2020-01-01 00:08:32 2020-01-02 03:36:53 2020-01-01 00:08:32.000 2020-01-02 03:36:53.000 512 99413 49962.5 4996250 512 99413 49962.5 4996250 -32658 32480 4742.66 474266 -127 124 -3.58 -358 513 100 10503 99414 1.54054 298.54054 150.04054 15004.05405 1.54054 298.54053 150.04053 15004.05397 1.54054 298.54054 150.04054 15004.05400 2020-01-01 2020-01-02 2020-01-01 00:08:33 2020-01-02 03:36:54 2020-01-01 00:08:33.000 2020-01-02 03:36:54.000 513 99414 49963.5 4996350 513 99414 49963.5 4996350 -32657 32481 4743.66 474366 -126 125 -2.58 -258 -514 100 10504 99415 1.54354 298.54354 150.04354 15004.35435 1.54354 298.54355 150.04354 15004.35459 1.54354 298.54354 150.04354 15004.35400 2020-01-01 2020-01-02 2020-01-01 00:08:34 2020-01-02 03:36:55 2020-01-01 00:08:34.000 2020-01-02 03:36:55.000 514 99415 49964.5 4996450 514 99415 49964.5 4996450 -32656 32482 4744.66 474466 -125 126 -1.58 -158 -515 100 10505 99416 1.54654 298.54654 150.04654 15004.65465 1.54654 298.54654 150.04654 15004.65425 1.54654 298.54654 150.04653999999994 15004.65400 2020-01-01 2020-01-02 2020-01-01 00:08:35 2020-01-02 03:36:56 2020-01-01 00:08:35.000 2020-01-02 03:36:56.000 515 99416 49965.5 4996550 515 99416 49965.5 4996550 -32655 32483 4745.66 474566 -124 127 -0.58 -58 -516 100 10506 99417 1.54954 298.54954 150.04954 15004.95495 1.54954 298.54956 150.04955 15004.955 1.54954 298.54954 150.04954 15004.95400 2020-01-01 2020-01-02 2020-01-01 00:08:36 2020-01-02 03:36:57 2020-01-01 00:08:36.000 2020-01-02 03:36:57.000 516 99417 49966.5 4996650 516 99417 49966.5 4996650 -32654 32484 4746.66 474666 -128 127 -2.14 -214 -517 100 10507 99418 1.55255 298.55255 150.05255 15005.25525 1.55255 298.55255 150.05255 15005.25514 1.55255 298.55255 150.05255000000005 15005.25500 2020-01-01 2020-01-02 2020-01-01 00:08:37 2020-01-02 03:36:58 2020-01-01 00:08:37.000 2020-01-02 03:36:58.000 517 99418 49967.5 4996750 517 99418 49967.5 4996750 -32653 32485 4747.66 474766 -128 123 -3.7 -370 -518 100 10508 99419 1.55555 298.55555 150.05555 15005.55555 1.55555 298.55554 150.05555 15005.55547 1.55555 298.55555 150.05554999999993 15005.55500 2020-01-01 2020-01-02 2020-01-01 00:08:38 2020-01-02 03:36:59 2020-01-01 00:08:38.000 2020-01-02 03:36:59.000 518 99419 49968.5 4996850 518 99419 49968.5 4996850 -32652 32486 4748.66 474866 -127 124 -2.7 -270 -519 100 10509 99420 1.55855 298.55855 150.05855 15005.85585 1.55855 298.55856 150.05856 15005.85607 1.55855 298.55855 150.05854999999997 15005.85500 2020-01-01 2020-01-02 2020-01-01 00:08:39 2020-01-02 03:37:00 2020-01-01 00:08:39.000 2020-01-02 03:37:00.000 519 99420 49969.5 4996950 519 99420 49969.5 4996950 -32651 32487 4749.66 474966 -126 125 -1.7 -170 -52 102 10042 99952 0.15615 300.15615 150.15615 15165.77177 0.15615 300.15616 150.15615 15165.77193 0.15615 300.15615 150.15615 15165.77115 2020-01-01 2020-01-02 2020-01-01 00:00:52 2020-01-02 03:45:52 2020-01-01 00:00:52.000 2020-01-02 03:45:52.000 52 99952 50002 5050202 52 99952 50002 5050202 -32517 32418 4581.009900990099 462682 -125 126 -1.6435643564356435 -166 +514 100 10504 99415 1.54354 298.54354 150.04354 15004.35435 1.54354 298.54355 150.04354 15004.35459 1.54354 298.54354 150.04354000000006 15004.35400 2020-01-01 2020-01-02 2020-01-01 00:08:34 2020-01-02 03:36:55 2020-01-01 00:08:34.000 2020-01-02 03:36:55.000 514 99415 49964.5 4996450 514 99415 49964.5 4996450 -32656 32482 4744.66 474466 -125 126 -1.58 -158 +515 100 10505 99416 1.54654 298.54654 150.04654 15004.65465 1.54654 298.54654 150.04654 15004.65425 1.54654 298.54654 150.04653999999988 15004.65400 2020-01-01 2020-01-02 2020-01-01 00:08:35 2020-01-02 03:36:56 2020-01-01 00:08:35.000 2020-01-02 03:36:56.000 515 99416 49965.5 4996550 515 99416 49965.5 4996550 -32655 32483 4745.66 474566 -124 127 -0.58 -58 +516 100 10506 99417 1.54954 298.54954 150.04954 15004.95495 1.54954 298.54956 150.04955 15004.955 1.54954 298.54954 150.04953999999998 15004.95400 2020-01-01 2020-01-02 2020-01-01 00:08:36 2020-01-02 03:36:57 2020-01-01 00:08:36.000 2020-01-02 03:36:57.000 516 99417 49966.5 4996650 516 99417 49966.5 4996650 -32654 32484 4746.66 474666 -128 127 -2.14 -214 +517 100 10507 99418 1.55255 298.55255 150.05255 15005.25525 1.55255 298.55255 150.05255 15005.25514 1.55255 298.55255 150.05255000000022 15005.25500 2020-01-01 2020-01-02 2020-01-01 00:08:37 2020-01-02 03:36:58 2020-01-01 00:08:37.000 2020-01-02 03:36:58.000 517 99418 49967.5 4996750 517 99418 49967.5 4996750 -32653 32485 4747.66 474766 -128 123 -3.7 -370 +518 100 10508 99419 1.55555 298.55555 150.05555 15005.55555 1.55555 298.55554 150.05555 15005.55547 1.55555 298.55555 150.05554999999973 15005.55500 2020-01-01 2020-01-02 2020-01-01 00:08:38 2020-01-02 03:36:59 2020-01-01 00:08:38.000 2020-01-02 03:36:59.000 518 99419 49968.5 4996850 518 99419 49968.5 4996850 -32652 32486 4748.66 474866 -127 124 -2.7 -270 +519 100 10509 99420 1.55855 298.55855 150.05855 15005.85585 1.55855 298.55856 150.05856 15005.85607 1.55855 298.55855 150.05854999999985 15005.85500 2020-01-01 2020-01-02 2020-01-01 00:08:39 2020-01-02 03:37:00 2020-01-01 00:08:39.000 2020-01-02 03:37:00.000 519 99420 49969.5 4996950 519 99420 49969.5 4996950 -32651 32487 4749.66 474966 -126 125 -1.7 -170 +52 102 10042 99952 0.15615 300.15615 150.15615 15165.77177 0.15615 300.15616 150.15615 15165.77193 0.15615 300.15615 150.1561500000001 15165.77115 2020-01-01 2020-01-02 2020-01-01 00:00:52 2020-01-02 03:45:52 2020-01-01 00:00:52.000 2020-01-02 03:45:52.000 52 99952 50002 5050202 52 99952 50002 5050202 -32517 32418 4581.009900990099 462682 -125 126 -1.6435643564356435 -166 520 100 10510 99421 1.56156 298.56156 150.06156 15006.15615 1.56156 298.56155 150.06155 15006.15572 1.56156 298.56156 150.06156000000007 15006.15600 2020-01-01 2020-01-02 2020-01-01 00:08:40 2020-01-02 03:37:01 2020-01-01 00:08:40.000 2020-01-02 03:37:01.000 520 99421 49970.5 4997050 520 99421 49970.5 4997050 -32650 32488 4750.66 475066 -125 126 -0.7 -70 -521 100 10511 99422 1.56456 298.56456 150.06456 15006.45645 1.56456 298.56458 150.06456 15006.45647 1.56456 298.56456 150.06456000000003 15006.45600 2020-01-01 2020-01-02 2020-01-01 00:08:41 2020-01-02 03:37:02 2020-01-01 00:08:41.000 2020-01-02 03:37:02.000 521 99422 49971.5 4997150 521 99422 49971.5 4997150 -32649 32489 4751.66 475166 -124 127 0.3 30 -522 100 10512 99423 1.56756 298.56756 150.06756 15006.75675 1.56756 298.56757 150.06756 15006.75661 1.56756 298.56756 150.06756 15006.75600 2020-01-01 2020-01-02 2020-01-01 00:08:42 2020-01-02 03:37:03 2020-01-01 00:08:42.000 2020-01-02 03:37:03.000 522 99423 49972.5 4997250 522 99423 49972.5 4997250 -32648 32490 4752.66 475266 -128 127 -1.26 -126 -523 100 10513 99424 1.57057 298.57057 150.07057 15007.05705 1.57057 298.57056 150.07056 15007.05694 1.57057 298.57057 150.0705699999999 15007.05700 2020-01-01 2020-01-02 2020-01-01 00:08:43 2020-01-02 03:37:04 2020-01-01 00:08:43.000 2020-01-02 03:37:04.000 523 99424 49973.5 4997350 523 99424 49973.5 4997350 -32647 32491 4753.66 475366 -128 123 -2.82 -282 -524 100 10514 99425 1.57357 298.57357 150.07357 15007.35735 1.57357 298.57358 150.07357 15007.35753 1.57357 298.57357 150.07357 15007.35700 2020-01-01 2020-01-02 2020-01-01 00:08:44 2020-01-02 03:37:05 2020-01-01 00:08:44.000 2020-01-02 03:37:05.000 524 99425 49974.5 4997450 524 99425 49974.5 4997450 -32646 32492 4754.66 475466 -127 124 -1.82 -182 -525 100 10515 99426 1.57657 298.57657 150.07657 15007.65765 1.57657 298.57657 150.07657 15007.65783 1.57657 298.57657 150.07656999999986 15007.65700 2020-01-01 2020-01-02 2020-01-01 00:08:45 2020-01-02 03:37:06 2020-01-01 00:08:45.000 2020-01-02 03:37:06.000 525 99426 49975.5 4997550 525 99426 49975.5 4997550 -32645 32493 4755.66 475566 -126 125 -0.82 -82 -526 100 10516 99427 1.57957 298.57957 150.07957 15007.95795 1.57957 298.5796 150.07957 15007.95795 1.57957 298.57957 150.0795699999999 15007.95700 2020-01-01 2020-01-02 2020-01-01 00:08:46 2020-01-02 03:37:07 2020-01-01 00:08:46.000 2020-01-02 03:37:07.000 526 99427 49976.5 4997650 526 99427 49976.5 4997650 -32644 32494 4756.66 475666 -125 126 0.18 18 -527 100 10517 99428 1.58258 298.58258 150.08258 15008.25825 1.58258 298.58258 150.08258 15008.25811 1.58258 298.58258 150.08258000000015 15008.25800 2020-01-01 2020-01-02 2020-01-01 00:08:47 2020-01-02 03:37:08 2020-01-01 00:08:47.000 2020-01-02 03:37:08.000 527 99428 49977.5 4997750 527 99428 49977.5 4997750 -32643 32495 4757.66 475766 -124 127 1.18 118 -528 100 10518 99429 1.58558 298.58558 150.08558 15008.55855 1.58558 298.58557 150.08558 15008.5584 1.58558 298.58558 150.08558000000025 15008.55800 2020-01-01 2020-01-02 2020-01-01 00:08:48 2020-01-02 03:37:09 2020-01-01 00:08:48.000 2020-01-02 03:37:09.000 528 99429 49978.5 4997850 528 99429 49978.5 4997850 -32642 32496 4758.66 475866 -128 127 -0.38 -38 -529 100 10519 99430 1.58858 298.58858 150.08858 15008.85885 1.58858 298.5886 150.08859 15008.859 1.58858 298.58858 150.08857999999975 15008.85800 2020-01-01 2020-01-02 2020-01-01 00:08:49 2020-01-02 03:37:10 2020-01-01 00:08:49.000 2020-01-02 03:37:10.000 529 99430 49979.5 4997950 529 99430 49979.5 4997950 -32641 32497 4759.66 475966 -128 127 -1.94 -194 +521 100 10511 99422 1.56456 298.56456 150.06456 15006.45645 1.56456 298.56458 150.06456 15006.45647 1.56456 298.56456 150.0645600000001 15006.45600 2020-01-01 2020-01-02 2020-01-01 00:08:41 2020-01-02 03:37:02 2020-01-01 00:08:41.000 2020-01-02 03:37:02.000 521 99422 49971.5 4997150 521 99422 49971.5 4997150 -32649 32489 4751.66 475166 -124 127 0.3 30 +522 100 10512 99423 1.56756 298.56756 150.06756 15006.75675 1.56756 298.56757 150.06756 15006.75661 1.56756 298.56756 150.06756000000004 15006.75600 2020-01-01 2020-01-02 2020-01-01 00:08:42 2020-01-02 03:37:03 2020-01-01 00:08:42.000 2020-01-02 03:37:03.000 522 99423 49972.5 4997250 522 99423 49972.5 4997250 -32648 32490 4752.66 475266 -128 127 -1.26 -126 +523 100 10513 99424 1.57057 298.57057 150.07057 15007.05705 1.57057 298.57056 150.07056 15007.05694 1.57057 298.57057 150.07056999999992 15007.05700 2020-01-01 2020-01-02 2020-01-01 00:08:43 2020-01-02 03:37:04 2020-01-01 00:08:43.000 2020-01-02 03:37:04.000 523 99424 49973.5 4997350 523 99424 49973.5 4997350 -32647 32491 4753.66 475366 -128 123 -2.82 -282 +524 100 10514 99425 1.57357 298.57357 150.07357 15007.35735 1.57357 298.57358 150.07357 15007.35753 1.57357 298.57357 150.07356999999996 15007.35700 2020-01-01 2020-01-02 2020-01-01 00:08:44 2020-01-02 03:37:05 2020-01-01 00:08:44.000 2020-01-02 03:37:05.000 524 99425 49974.5 4997450 524 99425 49974.5 4997450 -32646 32492 4754.66 475466 -127 124 -1.82 -182 +525 100 10515 99426 1.57657 298.57657 150.07657 15007.65765 1.57657 298.57657 150.07657 15007.65783 1.57657 298.57657 150.07656999999998 15007.65700 2020-01-01 2020-01-02 2020-01-01 00:08:45 2020-01-02 03:37:06 2020-01-01 00:08:45.000 2020-01-02 03:37:06.000 525 99426 49975.5 4997550 525 99426 49975.5 4997550 -32645 32493 4755.66 475566 -126 125 -0.82 -82 +526 100 10516 99427 1.57957 298.57957 150.07957 15007.95795 1.57957 298.5796 150.07957 15007.95795 1.57957 298.57957 150.07956999999993 15007.95700 2020-01-01 2020-01-02 2020-01-01 00:08:46 2020-01-02 03:37:07 2020-01-01 00:08:46.000 2020-01-02 03:37:07.000 526 99427 49976.5 4997650 526 99427 49976.5 4997650 -32644 32494 4756.66 475666 -125 126 0.18 18 +527 100 10517 99428 1.58258 298.58258 150.08258 15008.25825 1.58258 298.58258 150.08258 15008.25811 1.58258 298.58258 150.0825800000001 15008.25800 2020-01-01 2020-01-02 2020-01-01 00:08:47 2020-01-02 03:37:08 2020-01-01 00:08:47.000 2020-01-02 03:37:08.000 527 99428 49977.5 4997750 527 99428 49977.5 4997750 -32643 32495 4757.66 475766 -124 127 1.18 118 +528 100 10518 99429 1.58558 298.58558 150.08558 15008.55855 1.58558 298.58557 150.08558 15008.5584 1.58558 298.58558 150.08558000000005 15008.55800 2020-01-01 2020-01-02 2020-01-01 00:08:48 2020-01-02 03:37:09 2020-01-01 00:08:48.000 2020-01-02 03:37:09.000 528 99429 49978.5 4997850 528 99429 49978.5 4997850 -32642 32496 4758.66 475866 -128 127 -0.38 -38 +529 100 10519 99430 1.58858 298.58858 150.08858 15008.85885 1.58858 298.5886 150.08859 15008.859 1.58858 298.58858 150.08857999999992 15008.85800 2020-01-01 2020-01-02 2020-01-01 00:08:49 2020-01-02 03:37:10 2020-01-01 00:08:49.000 2020-01-02 03:37:10.000 529 99430 49979.5 4997950 529 99430 49979.5 4997950 -32641 32497 4759.66 475966 -128 127 -1.94 -194 53 102 10043 99953 0.15915 300.15915 150.15915 15166.07507 0.15915 300.15915 150.15915 15166.07511 0.15915 300.15915 150.15915 15166.07415 2020-01-01 2020-01-02 2020-01-01 00:00:53 2020-01-02 03:45:53 2020-01-01 00:00:53.000 2020-01-02 03:45:53.000 53 99953 50003 5050303 53 99953 50003 5050303 -32516 32419 4582.009900990099 462783 -124 127 -0.6435643564356436 -65 530 100 10520 99431 1.59159 298.59159 150.09159 15009.15915 1.59159 298.59158 150.09159 15009.15929 1.59159 298.59159 150.09159 15009.15900 2020-01-01 2020-01-02 2020-01-01 00:08:50 2020-01-02 03:37:11 2020-01-01 00:08:50.000 2020-01-02 03:37:11.000 530 99431 49980.5 4998050 530 99431 49980.5 4998050 -32640 32498 4760.66 476066 -128 124 -3.5 -350 -531 100 10521 99432 1.59459 298.59459 150.09459 15009.45945 1.59459 298.5946 150.09459 15009.45941 1.59459 298.59459 150.0945900000001 15009.45900 2020-01-01 2020-01-02 2020-01-01 00:08:51 2020-01-02 03:37:12 2020-01-01 00:08:51.000 2020-01-02 03:37:12.000 531 99432 49981.5 4998150 531 99432 49981.5 4998150 -32639 32499 4761.66 476166 -127 125 -2.5 -250 -532 100 10522 99433 1.59759 298.59759 150.09759 15009.75975 1.59759 298.5976 150.09759 15009.75958 1.59759 298.59759 150.0975899999999 15009.75900 2020-01-01 2020-01-02 2020-01-01 00:08:52 2020-01-02 03:37:13 2020-01-01 00:08:52.000 2020-01-02 03:37:13.000 532 99433 49982.5 4998250 532 99433 49982.5 4998250 -32638 32500 4762.66 476266 -126 126 -1.5 -150 -533 100 10523 99434 1.6006 298.6006 150.1006 15010.06006 1.6006 298.6006 150.10059 15010.05988 1.60060 298.60060 150.10059999999984 15010.06000 2020-01-01 2020-01-02 2020-01-01 00:08:53 2020-01-02 03:37:14 2020-01-01 00:08:53.000 2020-01-02 03:37:14.000 533 99434 49983.5 4998350 533 99434 49983.5 4998350 -32637 32501 4763.66 476366 -125 127 -0.5 -50 -534 100 10524 99435 1.6036 298.6036 150.1036 15010.36036 1.6036 298.6036 150.1036 15010.36063 1.60360 298.60360 150.10360000000026 15010.36000 2020-01-01 2020-01-02 2020-01-01 00:08:54 2020-01-02 03:37:15 2020-01-01 00:08:54.000 2020-01-02 03:37:15.000 534 99435 49984.5 4998450 534 99435 49984.5 4998450 -32636 32502 4764.66 476466 -128 127 -2.06 -206 -535 100 10525 99436 1.6066 298.6066 150.1066 15010.66066 1.6066 298.6066 150.1066 15010.66077 1.60660 298.60660 150.10659999999976 15010.66000 2020-01-01 2020-01-02 2020-01-01 00:08:55 2020-01-02 03:37:16 2020-01-01 00:08:55.000 2020-01-02 03:37:16.000 535 99436 49985.5 4998550 535 99436 49985.5 4998550 -32635 32503 4765.66 476566 -128 127 -3.62 -362 -536 100 10526 99437 1.6096 298.6096 150.1096 15010.96096 1.6096 298.60962 150.1096 15010.96092 1.60960 298.60960 150.1095999999998 15010.96000 2020-01-01 2020-01-02 2020-01-01 00:08:56 2020-01-02 03:37:17 2020-01-01 00:08:56.000 2020-01-02 03:37:17.000 536 99437 49986.5 4998650 536 99437 49986.5 4998650 -32634 32504 4766.66 476666 -128 123 -5.18 -518 -537 100 10527 99438 1.61261 298.61261 150.11261 15011.26126 1.61261 298.6126 150.11261 15011.26105 1.61261 298.61261 150.1126100000001 15011.26100 2020-01-01 2020-01-02 2020-01-01 00:08:57 2020-01-02 03:37:18 2020-01-01 00:08:57.000 2020-01-02 03:37:18.000 537 99438 49987.5 4998750 537 99438 49987.5 4998750 -32633 32505 4767.66 476766 -127 124 -4.18 -418 -538 100 10528 99439 1.61561 298.61561 150.11561 15011.56156 1.61561 298.6156 150.11561 15011.56135 1.61561 298.61561 150.11561000000015 15011.56100 2020-01-01 2020-01-02 2020-01-01 00:08:58 2020-01-02 03:37:19 2020-01-01 00:08:58.000 2020-01-02 03:37:19.000 538 99439 49988.5 4998850 538 99439 49988.5 4998850 -32632 32506 4768.66 476866 -126 125 -3.18 -318 -539 100 10529 99440 1.61861 298.61861 150.11861 15011.86186 1.61861 298.61862 150.11862 15011.8621 1.61861 298.61861 150.11860999999993 15011.86100 2020-01-01 2020-01-02 2020-01-01 00:08:59 2020-01-02 03:37:20 2020-01-01 00:08:59.000 2020-01-02 03:37:20.000 539 99440 49989.5 4998950 539 99440 49989.5 4998950 -32631 32507 4769.66 476966 -125 126 -2.18 -218 -54 102 10044 99954 0.16216 300.16216 150.16216 15166.37837 0.16216 300.16217 150.16216 15166.37822 0.16216 300.16216 150.1621599999999 15166.37816 2020-01-01 2020-01-02 2020-01-01 00:00:54 2020-01-02 03:45:54 2020-01-01 00:00:54.000 2020-01-02 03:45:54.000 54 99954 50004 5050404 54 99954 50004 5050404 -32515 32420 4583.009900990099 462884 -128 127 -2.1782178217821784 -220 +531 100 10521 99432 1.59459 298.59459 150.09459 15009.45945 1.59459 298.5946 150.09459 15009.45941 1.59459 298.59459 150.09459000000004 15009.45900 2020-01-01 2020-01-02 2020-01-01 00:08:51 2020-01-02 03:37:12 2020-01-01 00:08:51.000 2020-01-02 03:37:12.000 531 99432 49981.5 4998150 531 99432 49981.5 4998150 -32639 32499 4761.66 476166 -127 125 -2.5 -250 +532 100 10522 99433 1.59759 298.59759 150.09759 15009.75975 1.59759 298.5976 150.09759 15009.75958 1.59759 298.59759 150.09758999999997 15009.75900 2020-01-01 2020-01-02 2020-01-01 00:08:52 2020-01-02 03:37:13 2020-01-01 00:08:52.000 2020-01-02 03:37:13.000 532 99433 49982.5 4998250 532 99433 49982.5 4998250 -32638 32500 4762.66 476266 -126 126 -1.5 -150 +533 100 10523 99434 1.6006 298.6006 150.1006 15010.06006 1.6006 298.6006 150.10059 15010.05988 1.60060 298.60060 150.1005999999999 15010.06000 2020-01-01 2020-01-02 2020-01-01 00:08:53 2020-01-02 03:37:14 2020-01-01 00:08:53.000 2020-01-02 03:37:14.000 533 99434 49983.5 4998350 533 99434 49983.5 4998350 -32637 32501 4763.66 476366 -125 127 -0.5 -50 +534 100 10524 99435 1.6036 298.6036 150.1036 15010.36036 1.6036 298.6036 150.1036 15010.36063 1.60360 298.60360 150.10360000000009 15010.36000 2020-01-01 2020-01-02 2020-01-01 00:08:54 2020-01-02 03:37:15 2020-01-01 00:08:54.000 2020-01-02 03:37:15.000 534 99435 49984.5 4998450 534 99435 49984.5 4998450 -32636 32502 4764.66 476466 -128 127 -2.06 -206 +535 100 10525 99436 1.6066 298.6066 150.1066 15010.66066 1.6066 298.6066 150.1066 15010.66077 1.60660 298.60660 150.10659999999993 15010.66000 2020-01-01 2020-01-02 2020-01-01 00:08:55 2020-01-02 03:37:16 2020-01-01 00:08:55.000 2020-01-02 03:37:16.000 535 99436 49985.5 4998550 535 99436 49985.5 4998550 -32635 32503 4765.66 476566 -128 127 -3.62 -362 +536 100 10526 99437 1.6096 298.6096 150.1096 15010.96096 1.6096 298.60962 150.1096 15010.96092 1.60960 298.60960 150.1095999999999 15010.96000 2020-01-01 2020-01-02 2020-01-01 00:08:56 2020-01-02 03:37:17 2020-01-01 00:08:56.000 2020-01-02 03:37:17.000 536 99437 49986.5 4998650 536 99437 49986.5 4998650 -32634 32504 4766.66 476666 -128 123 -5.18 -518 +537 100 10527 99438 1.61261 298.61261 150.11261 15011.26126 1.61261 298.6126 150.11261 15011.26105 1.61261 298.61261 150.11261000000007 15011.26100 2020-01-01 2020-01-02 2020-01-01 00:08:57 2020-01-02 03:37:18 2020-01-01 00:08:57.000 2020-01-02 03:37:18.000 537 99438 49987.5 4998750 537 99438 49987.5 4998750 -32633 32505 4767.66 476766 -127 124 -4.18 -418 +538 100 10528 99439 1.61561 298.61561 150.11561 15011.56156 1.61561 298.6156 150.11561 15011.56135 1.61561 298.61561 150.11561000000003 15011.56100 2020-01-01 2020-01-02 2020-01-01 00:08:58 2020-01-02 03:37:19 2020-01-01 00:08:58.000 2020-01-02 03:37:19.000 538 99439 49988.5 4998850 538 99439 49988.5 4998850 -32632 32506 4768.66 476866 -126 125 -3.18 -318 +539 100 10529 99440 1.61861 298.61861 150.11861 15011.86186 1.61861 298.61862 150.11862 15011.8621 1.61861 298.61861 150.11860999999988 15011.86100 2020-01-01 2020-01-02 2020-01-01 00:08:59 2020-01-02 03:37:20 2020-01-01 00:08:59.000 2020-01-02 03:37:20.000 539 99440 49989.5 4998950 539 99440 49989.5 4998950 -32631 32507 4769.66 476966 -125 126 -2.18 -218 +54 102 10044 99954 0.16216 300.16216 150.16216 15166.37837 0.16216 300.16217 150.16216 15166.37822 0.16216 300.16216 150.16215999999994 15166.37816 2020-01-01 2020-01-02 2020-01-01 00:00:54 2020-01-02 03:45:54 2020-01-01 00:00:54.000 2020-01-02 03:45:54.000 54 99954 50004 5050404 54 99954 50004 5050404 -32515 32420 4583.009900990099 462884 -128 127 -2.1782178217821784 -220 540 100 10530 99441 1.62162 298.62162 150.12162 15012.16216 1.62162 298.6216 150.12162 15012.16224 1.62162 298.62162 150.12161999999992 15012.16200 2020-01-01 2020-01-02 2020-01-01 00:09:00 2020-01-02 03:37:21 2020-01-01 00:09:00.000 2020-01-02 03:37:21.000 540 99441 49990.5 4999050 540 99441 49990.5 4999050 -32630 32508 4770.66 477066 -124 127 -1.18 -118 541 100 10531 99442 1.62462 298.62462 150.12462 15012.46246 1.62462 298.62463 150.12462 15012.46239 1.62462 298.62462 150.12462 15012.46200 2020-01-01 2020-01-02 2020-01-01 00:09:01 2020-01-02 03:37:22 2020-01-01 00:09:01.000 2020-01-02 03:37:22.000 541 99442 49991.5 4999150 541 99442 49991.5 4999150 -32629 32509 4771.66 477166 -128 127 -2.74 -274 -542 100 10532 99443 1.62762 298.62762 150.12762 15012.76276 1.62762 298.62762 150.12762 15012.76252 1.62762 298.62762 150.1276199999999 15012.76200 2020-01-01 2020-01-02 2020-01-01 00:09:02 2020-01-02 03:37:23 2020-01-01 00:09:02.000 2020-01-02 03:37:23.000 542 99443 49992.5 4999250 542 99443 49992.5 4999250 -32628 32510 4772.66 477266 -128 123 -4.3 -430 -543 100 10533 99444 1.63063 298.63063 150.13063 15013.06306 1.63063 298.63065 150.13063 15013.06327 1.63063 298.63063 150.13062999999977 15013.06300 2020-01-01 2020-01-02 2020-01-01 00:09:03 2020-01-02 03:37:24 2020-01-01 00:09:03.000 2020-01-02 03:37:24.000 543 99444 49993.5 4999350 543 99444 49993.5 4999350 -32627 32511 4773.66 477366 -127 124 -3.3 -330 -544 100 10534 99445 1.63363 298.63363 150.13363 15013.36336 1.63363 298.63364 150.13363 15013.36358 1.63363 298.63363 150.13363000000015 15013.36300 2020-01-01 2020-01-02 2020-01-01 00:09:04 2020-01-02 03:37:25 2020-01-01 00:09:04.000 2020-01-02 03:37:25.000 544 99445 49994.5 4999450 544 99445 49994.5 4999450 -32626 32512 4774.66 477466 -126 125 -2.3 -230 -545 100 10535 99446 1.63663 298.63663 150.13663 15013.66366 1.63663 298.63663 150.13663 15013.6637 1.63663 298.63663 150.13663000000028 15013.66300 2020-01-01 2020-01-02 2020-01-01 00:09:05 2020-01-02 03:37:26 2020-01-01 00:09:05.000 2020-01-02 03:37:26.000 545 99446 49995.5 4999550 545 99446 49995.5 4999550 -32625 32513 4775.66 477566 -125 126 -1.3 -130 -546 100 10536 99447 1.63963 298.63963 150.13963 15013.96396 1.63963 298.63965 150.13963 15013.96385 1.63963 298.63963 150.13962999999978 15013.96300 2020-01-01 2020-01-02 2020-01-01 00:09:06 2020-01-02 03:37:27 2020-01-01 00:09:06.000 2020-01-02 03:37:27.000 546 99447 49996.5 4999650 546 99447 49996.5 4999650 -32624 32514 4776.66 477666 -124 127 -0.3 -30 +542 100 10532 99443 1.62762 298.62762 150.12762 15012.76276 1.62762 298.62762 150.12762 15012.76252 1.62762 298.62762 150.12761999999998 15012.76200 2020-01-01 2020-01-02 2020-01-01 00:09:02 2020-01-02 03:37:23 2020-01-01 00:09:02.000 2020-01-02 03:37:23.000 542 99443 49992.5 4999250 542 99443 49992.5 4999250 -32628 32510 4772.66 477266 -128 123 -4.3 -430 +543 100 10533 99444 1.63063 298.63063 150.13063 15013.06306 1.63063 298.63065 150.13063 15013.06327 1.63063 298.63063 150.13062999999994 15013.06300 2020-01-01 2020-01-02 2020-01-01 00:09:03 2020-01-02 03:37:24 2020-01-01 00:09:03.000 2020-01-02 03:37:24.000 543 99444 49993.5 4999350 543 99444 49993.5 4999350 -32627 32511 4773.66 477366 -127 124 -3.3 -330 +544 100 10534 99445 1.63363 298.63363 150.13363 15013.36336 1.63363 298.63364 150.13363 15013.36358 1.63363 298.63363 150.13363000000004 15013.36300 2020-01-01 2020-01-02 2020-01-01 00:09:04 2020-01-02 03:37:25 2020-01-01 00:09:04.000 2020-01-02 03:37:25.000 544 99445 49994.5 4999450 544 99445 49994.5 4999450 -32626 32512 4774.66 477466 -126 125 -2.3 -230 +545 100 10535 99446 1.63663 298.63663 150.13663 15013.66366 1.63663 298.63663 150.13663 15013.6637 1.63663 298.63663 150.13663000000008 15013.66300 2020-01-01 2020-01-02 2020-01-01 00:09:05 2020-01-02 03:37:26 2020-01-01 00:09:05.000 2020-01-02 03:37:26.000 545 99446 49995.5 4999550 545 99446 49995.5 4999550 -32625 32513 4775.66 477566 -125 126 -1.3 -130 +546 100 10536 99447 1.63963 298.63963 150.13963 15013.96396 1.63963 298.63965 150.13963 15013.96385 1.63963 298.63963 150.1396299999999 15013.96300 2020-01-01 2020-01-02 2020-01-01 00:09:06 2020-01-02 03:37:27 2020-01-01 00:09:06.000 2020-01-02 03:37:27.000 546 99447 49996.5 4999650 546 99447 49996.5 4999650 -32624 32514 4776.66 477666 -124 127 -0.3 -30 547 100 10537 99448 1.64264 298.64264 150.14264 15014.26426 1.64264 298.64264 150.14263 15014.26399 1.64264 298.64264 150.14264 15014.26400 2020-01-01 2020-01-02 2020-01-01 00:09:07 2020-01-02 03:37:28 2020-01-01 00:09:07.000 2020-01-02 03:37:28.000 547 99448 49997.5 4999750 547 99448 49997.5 4999750 -32623 32515 4777.66 477766 -128 127 -1.86 -186 -548 100 10538 99449 1.64564 298.64564 150.14564 15014.56456 1.64564 298.64566 150.14564 15014.56474 1.64564 298.64564 150.14564000000013 15014.56400 2020-01-01 2020-01-02 2020-01-01 00:09:08 2020-01-02 03:37:29 2020-01-01 00:09:08.000 2020-01-02 03:37:29.000 548 99449 49998.5 4999850 548 99449 49998.5 4999850 -32622 32516 4778.66 477866 -128 123 -3.42 -342 -549 100 10539 99450 1.64864 298.64864 150.14864 15014.86486 1.64864 298.64865 150.14865 15014.86504 1.64864 298.64864 150.14863999999994 15014.86400 2020-01-01 2020-01-02 2020-01-01 00:09:09 2020-01-02 03:37:30 2020-01-01 00:09:09.000 2020-01-02 03:37:30.000 549 99450 49999.5 4999950 549 99450 49999.5 4999950 -32621 32517 4779.66 477966 -127 124 -2.42 -242 +548 100 10538 99449 1.64564 298.64564 150.14564 15014.56456 1.64564 298.64566 150.14564 15014.56474 1.64564 298.64564 150.14564000000007 15014.56400 2020-01-01 2020-01-02 2020-01-01 00:09:08 2020-01-02 03:37:29 2020-01-01 00:09:08.000 2020-01-02 03:37:29.000 548 99449 49998.5 4999850 548 99449 49998.5 4999850 -32622 32516 4778.66 477866 -128 123 -3.42 -342 +549 100 10539 99450 1.64864 298.64864 150.14864 15014.86486 1.64864 298.64865 150.14865 15014.86504 1.64864 298.64864 150.14863999999997 15014.86400 2020-01-01 2020-01-02 2020-01-01 00:09:09 2020-01-02 03:37:30 2020-01-01 00:09:09.000 2020-01-02 03:37:30.000 549 99450 49999.5 4999950 549 99450 49999.5 4999950 -32621 32517 4779.66 477966 -127 124 -2.42 -242 55 102 10045 99955 0.16516 300.16516 150.16516 15166.68168 0.16516 300.16516 150.16516 15166.68151 0.16516 300.16516 150.16515999999996 15166.68116 2020-01-01 2020-01-02 2020-01-01 00:00:55 2020-01-02 03:45:55 2020-01-01 00:00:55.000 2020-01-02 03:45:55.000 55 99955 50005 5050505 55 99955 50005 5050505 -32514 32421 4584.009900990099 462985 -128 123 -3.712871287128713 -375 -550 100 10540 99451 1.65165 298.65165 150.15165 15015.16516 1.65165 298.65164 150.15165 15015.16521 1.65165 298.65165 150.15164999999982 15015.16500 2020-01-01 2020-01-02 2020-01-01 00:09:10 2020-01-02 03:37:31 2020-01-01 00:09:10.000 2020-01-02 03:37:31.000 550 99451 50000.5 5000050 550 99451 50000.5 5000050 -32620 32518 4780.66 478066 -126 125 -1.42 -142 -551 100 10541 99452 1.65465 298.65465 150.15465 15015.46546 1.65465 298.65466 150.15465 15015.46533 1.65465 298.65465 150.1546500000003 15015.46500 2020-01-01 2020-01-02 2020-01-01 00:09:11 2020-01-02 03:37:32 2020-01-01 00:09:11.000 2020-01-02 03:37:32.000 551 99452 50001.5 5000150 551 99452 50001.5 5000150 -32619 32519 4781.66 478166 -125 126 -0.42 -42 -552 100 10542 99453 1.65765 298.65765 150.15765 15015.76576 1.65765 298.65765 150.15765 15015.76562 1.65765 298.65765 150.1576499999998 15015.76500 2020-01-01 2020-01-02 2020-01-01 00:09:12 2020-01-02 03:37:33 2020-01-01 00:09:12.000 2020-01-02 03:37:33.000 552 99453 50002.5 5000250 552 99453 50002.5 5000250 -32618 32520 4782.66 478266 -124 127 0.58 58 -553 100 10543 99454 1.66066 298.66066 150.16066 15016.06606 1.66066 298.66068 150.16066 15016.06621 1.66066 298.66066 150.16065999999967 15016.06600 2020-01-01 2020-01-02 2020-01-01 00:09:13 2020-01-02 03:37:34 2020-01-01 00:09:13.000 2020-01-02 03:37:34.000 553 99454 50003.5 5000350 553 99454 50003.5 5000350 -32617 32521 4783.66 478366 -128 127 -0.98 -98 -554 100 10544 99455 1.66366 298.66366 150.16366 15016.36636 1.66366 298.66367 150.16366 15016.36651 1.66366 298.66366 150.16366000000014 15016.36600 2020-01-01 2020-01-02 2020-01-01 00:09:14 2020-01-02 03:37:35 2020-01-01 00:09:14.000 2020-01-02 03:37:35.000 554 99455 50004.5 5000450 554 99455 50004.5 5000450 -32616 32522 4784.66 478466 -128 127 -2.54 -254 -555 100 10545 99456 1.66666 298.66666 150.16666 15016.66666 1.66666 298.66666 150.16666 15016.66668 1.66666 298.66666 150.16666000000018 15016.66600 2020-01-01 2020-01-02 2020-01-01 00:09:15 2020-01-02 03:37:36 2020-01-01 00:09:15.000 2020-01-02 03:37:36.000 555 99456 50005.5 5000550 555 99456 50005.5 5000550 -32615 32523 4785.66 478566 -128 124 -4.1 -410 -556 100 10546 99457 1.66966 298.66966 150.16966 15016.96696 1.66966 298.66968 150.16966 15016.9668 1.66966 298.66966 150.16965999999968 15016.96600 2020-01-01 2020-01-02 2020-01-01 00:09:16 2020-01-02 03:37:37 2020-01-01 00:09:16.000 2020-01-02 03:37:37.000 556 99457 50006.5 5000650 556 99457 50006.5 5000650 -32614 32524 4786.66 478666 -127 125 -3.1 -310 +550 100 10540 99451 1.65165 298.65165 150.15165 15015.16516 1.65165 298.65164 150.15165 15015.16521 1.65165 298.65165 150.1516499999999 15015.16500 2020-01-01 2020-01-02 2020-01-01 00:09:10 2020-01-02 03:37:31 2020-01-01 00:09:10.000 2020-01-02 03:37:31.000 550 99451 50000.5 5000050 550 99451 50000.5 5000050 -32620 32518 4780.66 478066 -126 125 -1.42 -142 +551 100 10541 99452 1.65465 298.65465 150.15465 15015.46546 1.65465 298.65466 150.15465 15015.46533 1.65465 298.65465 150.15465000000012 15015.46500 2020-01-01 2020-01-02 2020-01-01 00:09:11 2020-01-02 03:37:32 2020-01-01 00:09:11.000 2020-01-02 03:37:32.000 551 99452 50001.5 5000150 551 99452 50001.5 5000150 -32619 32519 4781.66 478166 -125 126 -0.42 -42 +552 100 10542 99453 1.65765 298.65765 150.15765 15015.76576 1.65765 298.65765 150.15765 15015.76562 1.65765 298.65765 150.15764999999996 15015.76500 2020-01-01 2020-01-02 2020-01-01 00:09:12 2020-01-02 03:37:33 2020-01-01 00:09:12.000 2020-01-02 03:37:33.000 552 99453 50002.5 5000250 552 99453 50002.5 5000250 -32618 32520 4782.66 478266 -124 127 0.58 58 +553 100 10543 99454 1.66066 298.66066 150.16066 15016.06606 1.66066 298.66068 150.16066 15016.06621 1.66066 298.66066 150.16065999999984 15016.06600 2020-01-01 2020-01-02 2020-01-01 00:09:13 2020-01-02 03:37:34 2020-01-01 00:09:13.000 2020-01-02 03:37:34.000 553 99454 50003.5 5000350 553 99454 50003.5 5000350 -32617 32521 4783.66 478366 -128 127 -0.98 -98 +554 100 10544 99455 1.66366 298.66366 150.16366 15016.36636 1.66366 298.66367 150.16366 15016.36651 1.66366 298.66366 150.16366000000008 15016.36600 2020-01-01 2020-01-02 2020-01-01 00:09:14 2020-01-02 03:37:35 2020-01-01 00:09:14.000 2020-01-02 03:37:35.000 554 99455 50004.5 5000450 554 99455 50004.5 5000450 -32616 32522 4784.66 478466 -128 127 -2.54 -254 +555 100 10545 99456 1.66666 298.66666 150.16666 15016.66666 1.66666 298.66666 150.16666 15016.66668 1.66666 298.66666 150.16666000000004 15016.66600 2020-01-01 2020-01-02 2020-01-01 00:09:15 2020-01-02 03:37:36 2020-01-01 00:09:15.000 2020-01-02 03:37:36.000 555 99456 50005.5 5000550 555 99456 50005.5 5000550 -32615 32523 4785.66 478566 -128 124 -4.1 -410 +556 100 10546 99457 1.66966 298.66966 150.16966 15016.96696 1.66966 298.66968 150.16966 15016.9668 1.66966 298.66966 150.16965999999985 15016.96600 2020-01-01 2020-01-02 2020-01-01 00:09:16 2020-01-02 03:37:37 2020-01-01 00:09:16.000 2020-01-02 03:37:37.000 556 99457 50006.5 5000650 556 99457 50006.5 5000650 -32614 32524 4786.66 478666 -127 125 -3.1 -310 557 100 10547 99458 1.67267 298.67267 150.17267 15017.26726 1.67267 298.67267 150.17267 15017.26709 1.67267 298.67267 150.17266999999995 15017.26700 2020-01-01 2020-01-02 2020-01-01 00:09:17 2020-01-02 03:37:38 2020-01-01 00:09:17.000 2020-01-02 03:37:38.000 557 99458 50007.5 5000750 557 99458 50007.5 5000750 -32613 32525 4787.66 478766 -126 126 -2.1 -210 -558 100 10548 99459 1.67567 298.67567 150.17567 15017.56756 1.67567 298.6757 150.17567 15017.56769 1.67567 298.67567 150.17567000000003 15017.56700 2020-01-01 2020-01-02 2020-01-01 00:09:18 2020-01-02 03:37:39 2020-01-01 00:09:18.000 2020-01-02 03:37:39.000 558 99459 50008.5 5000850 558 99459 50008.5 5000850 -32612 32526 4788.66 478866 -125 127 -1.1 -110 -559 100 10549 99460 1.67867 298.67867 150.17867 15017.86786 1.67867 298.67868 150.17868 15017.86802 1.67867 298.67867 150.17866999999987 15017.86700 2020-01-01 2020-01-02 2020-01-01 00:09:19 2020-01-02 03:37:40 2020-01-01 00:09:19.000 2020-01-02 03:37:40.000 559 99460 50009.5 5000950 559 99460 50009.5 5000950 -32611 32527 4789.66 478966 -128 127 -2.66 -266 -56 102 10046 99956 0.16816 300.16816 150.16816 15166.98498 0.16816 300.16818 150.16816 15166.98512 0.16816 300.16816 150.16815999999986 15166.98416 2020-01-01 2020-01-02 2020-01-01 00:00:56 2020-01-02 03:45:56 2020-01-01 00:00:56.000 2020-01-02 03:45:56.000 56 99956 50006 5050606 56 99956 50006 5050606 -32513 32422 4585.009900990099 463086 -127 124 -2.712871287128713 -274 -560 100 10550 99461 1.68168 298.68168 150.18168 15018.16816 1.68168 298.68167 150.18168 15018.16815 1.68168 298.68168 150.1816799999998 15018.16800 2020-01-01 2020-01-02 2020-01-01 00:09:20 2020-01-02 03:37:41 2020-01-01 00:09:20.000 2020-01-02 03:37:41.000 560 99461 50010.5 5001050 560 99461 50010.5 5001050 -32610 32528 4790.66 479066 -128 127 -4.22 -422 -561 100 10551 99462 1.68468 298.68468 150.18468 15018.46846 1.68468 298.6847 150.18468 15018.46826 1.68468 298.68468 150.18468000000018 15018.46800 2020-01-01 2020-01-02 2020-01-01 00:09:21 2020-01-02 03:37:42 2020-01-01 00:09:21.000 2020-01-02 03:37:42.000 561 99462 50011.5 5001150 561 99462 50011.5 5001150 -32609 32529 4791.66 479166 -128 123 -5.78 -578 -562 100 10552 99463 1.68768 298.68768 150.18768 15018.76876 1.68768 298.68768 150.18768 15018.76856 1.68768 298.68768 150.18767999999972 15018.76800 2020-01-01 2020-01-02 2020-01-01 00:09:22 2020-01-02 03:37:43 2020-01-01 00:09:22.000 2020-01-02 03:37:43.000 562 99463 50012.5 5001250 562 99463 50012.5 5001250 -32608 32530 4792.66 479266 -127 124 -4.78 -478 -563 100 10553 99464 1.69069 298.69069 150.19069 15019.06906 1.69069 298.6907 150.19069 15019.06915 1.69069 298.69069 150.19068999999996 15019.06900 2020-01-01 2020-01-02 2020-01-01 00:09:23 2020-01-02 03:37:44 2020-01-01 00:09:23.000 2020-01-02 03:37:44.000 563 99464 50013.5 5001350 563 99464 50013.5 5001350 -32607 32531 4793.66 479366 -126 125 -3.78 -378 +558 100 10548 99459 1.67567 298.67567 150.17567 15017.56756 1.67567 298.6757 150.17567 15017.56769 1.67567 298.67567 150.17567 15017.56700 2020-01-01 2020-01-02 2020-01-01 00:09:18 2020-01-02 03:37:39 2020-01-01 00:09:18.000 2020-01-02 03:37:39.000 558 99459 50008.5 5000850 558 99459 50008.5 5000850 -32612 32526 4788.66 478866 -125 127 -1.1 -110 +559 100 10549 99460 1.67867 298.67867 150.17867 15017.86786 1.67867 298.67868 150.17868 15017.86802 1.67867 298.67867 150.17866999999998 15017.86700 2020-01-01 2020-01-02 2020-01-01 00:09:19 2020-01-02 03:37:40 2020-01-01 00:09:19.000 2020-01-02 03:37:40.000 559 99460 50009.5 5000950 559 99460 50009.5 5000950 -32611 32527 4789.66 478966 -128 127 -2.66 -266 +56 102 10046 99956 0.16816 300.16816 150.16816 15166.98498 0.16816 300.16818 150.16816 15166.98512 0.16816 300.16816 150.16815999999997 15166.98416 2020-01-01 2020-01-02 2020-01-01 00:00:56 2020-01-02 03:45:56 2020-01-01 00:00:56.000 2020-01-02 03:45:56.000 56 99956 50006 5050606 56 99956 50006 5050606 -32513 32422 4585.009900990099 463086 -127 124 -2.712871287128713 -274 +560 100 10550 99461 1.68168 298.68168 150.18168 15018.16816 1.68168 298.68167 150.18168 15018.16815 1.68168 298.68168 150.18167999999991 15018.16800 2020-01-01 2020-01-02 2020-01-01 00:09:20 2020-01-02 03:37:41 2020-01-01 00:09:20.000 2020-01-02 03:37:41.000 560 99461 50010.5 5001050 560 99461 50010.5 5001050 -32610 32528 4790.66 479066 -128 127 -4.22 -422 +561 100 10551 99462 1.68468 298.68468 150.18468 15018.46846 1.68468 298.6847 150.18468 15018.46826 1.68468 298.68468 150.18468000000007 15018.46800 2020-01-01 2020-01-02 2020-01-01 00:09:21 2020-01-02 03:37:42 2020-01-01 00:09:21.000 2020-01-02 03:37:42.000 561 99462 50011.5 5001150 561 99462 50011.5 5001150 -32609 32529 4791.66 479166 -128 123 -5.78 -578 +562 100 10552 99463 1.68768 298.68768 150.18768 15018.76876 1.68768 298.68768 150.18768 15018.76856 1.68768 298.68768 150.18767999999992 15018.76800 2020-01-01 2020-01-02 2020-01-01 00:09:22 2020-01-02 03:37:43 2020-01-01 00:09:22.000 2020-01-02 03:37:43.000 562 99463 50012.5 5001250 562 99463 50012.5 5001250 -32608 32530 4792.66 479266 -127 124 -4.78 -478 +563 100 10553 99464 1.69069 298.69069 150.19069 15019.06906 1.69069 298.6907 150.19069 15019.06915 1.69069 298.69069 150.19069 15019.06900 2020-01-01 2020-01-02 2020-01-01 00:09:23 2020-01-02 03:37:44 2020-01-01 00:09:23.000 2020-01-02 03:37:44.000 563 99464 50013.5 5001350 563 99464 50013.5 5001350 -32607 32531 4793.66 479366 -126 125 -3.78 -378 564 100 10554 99465 1.69369 298.69369 150.19369 15019.36936 1.69369 298.6937 150.19369 15019.36948 1.69369 298.69369 150.19369000000003 15019.36900 2020-01-01 2020-01-02 2020-01-01 00:09:24 2020-01-02 03:37:45 2020-01-01 00:09:24.000 2020-01-02 03:37:45.000 564 99465 50014.5 5001450 564 99465 50014.5 5001450 -32606 32532 4794.66 479466 -125 126 -2.78 -278 -565 100 10555 99466 1.69669 298.69669 150.19669 15019.66966 1.69669 298.6967 150.19669 15019.66962 1.69669 298.69669 150.19669000000013 15019.66900 2020-01-01 2020-01-02 2020-01-01 00:09:25 2020-01-02 03:37:46 2020-01-01 00:09:25.000 2020-01-02 03:37:46.000 565 99466 50015.5 5001550 565 99466 50015.5 5001550 -32605 32533 4795.66 479566 -124 127 -1.78 -178 -566 100 10556 99467 1.69969 298.69969 150.19969 15019.96996 1.69969 298.6997 150.1997 15019.97037 1.69969 298.69969 150.19968999999998 15019.96900 2020-01-01 2020-01-02 2020-01-01 00:09:26 2020-01-02 03:37:47 2020-01-01 00:09:26.000 2020-01-02 03:37:47.000 566 99467 50016.5 5001650 566 99467 50016.5 5001650 -32604 32534 4796.66 479666 -128 127 -3.34 -334 -567 100 10557 99468 1.7027 298.7027 150.2027 15020.27027 1.7027 298.7027 150.2027 15020.27003 1.70270 298.70270 150.20269999999985 15020.27000 2020-01-01 2020-01-02 2020-01-01 00:09:27 2020-01-02 03:37:48 2020-01-01 00:09:27.000 2020-01-02 03:37:48.000 567 99468 50017.5 5001750 567 99468 50017.5 5001750 -32603 32535 4797.66 479766 -128 123 -4.9 -490 -568 100 10558 99469 1.7057 298.7057 150.2057 15020.57057 1.7057 298.70572 150.2057 15020.57066 1.70570 298.70570 150.20570000000032 15020.57000 2020-01-01 2020-01-02 2020-01-01 00:09:28 2020-01-02 03:37:49 2020-01-01 00:09:28.000 2020-01-02 03:37:49.000 568 99469 50018.5 5001850 568 99469 50018.5 5001850 -32602 32536 4798.66 479866 -127 124 -3.9 -390 -569 100 10559 99470 1.7087 298.7087 150.2087 15020.87087 1.7087 298.7087 150.2087 15020.87095 1.70870 298.70870 150.20869999999982 15020.87000 2020-01-01 2020-01-02 2020-01-01 00:09:29 2020-01-02 03:37:50 2020-01-01 00:09:29.000 2020-01-02 03:37:50.000 569 99470 50019.5 5001950 569 99470 50019.5 5001950 -32601 32537 4799.66 479966 -126 125 -2.9 -290 -57 102 10047 99957 0.17117 300.17117 150.17117 15167.28828 0.17117 300.17117 150.17117 15167.28841 0.17117 300.17117 150.17116999999973 15167.28817 2020-01-01 2020-01-02 2020-01-01 00:00:57 2020-01-02 03:45:57 2020-01-01 00:00:57.000 2020-01-02 03:45:57.000 57 99957 50007 5050707 57 99957 50007 5050707 -32512 32423 4586.009900990099 463187 -126 125 -1.7128712871287128 -173 -570 100 10560 99471 1.71171 298.71171 150.21171 15021.17117 1.71171 298.7117 150.21171 15021.17109 1.71171 298.71171 150.2117099999997 15021.17100 2020-01-01 2020-01-02 2020-01-01 00:09:30 2020-01-02 03:37:51 2020-01-01 00:09:30.000 2020-01-02 03:37:51.000 570 99471 50020.5 5002050 570 99471 50020.5 5002050 -32600 32538 4800.66 480066 -125 126 -1.9 -190 -571 100 10561 99472 1.71471 298.71471 150.21471 15021.47147 1.71471 298.71472 150.21471 15021.47184 1.71471 298.71471 150.21471000000017 15021.47100 2020-01-01 2020-01-02 2020-01-01 00:09:31 2020-01-02 03:37:52 2020-01-01 00:09:31.000 2020-01-02 03:37:52.000 571 99472 50021.5 5002150 571 99472 50021.5 5002150 -32599 32539 4801.66 480166 -124 127 -0.9 -90 -572 100 10562 99473 1.71771 298.71771 150.21771 15021.77177 1.71771 298.7177 150.21771 15021.7715 1.71771 298.71771 150.2177100000002 15021.77100 2020-01-01 2020-01-02 2020-01-01 00:09:32 2020-01-02 03:37:53 2020-01-01 00:09:32.000 2020-01-02 03:37:53.000 572 99473 50022.5 5002250 572 99473 50022.5 5002250 -32598 32540 4802.66 480266 -128 127 -2.46 -246 -573 100 10563 99474 1.72072 298.72072 150.22072 15022.07207 1.72072 298.72073 150.22072 15022.07212 1.72072 298.72072 150.22071999999986 15022.07200 2020-01-01 2020-01-02 2020-01-01 00:09:33 2020-01-02 03:37:54 2020-01-01 00:09:33.000 2020-01-02 03:37:54.000 573 99474 50023.5 5002350 573 99474 50023.5 5002350 -32597 32541 4803.66 480366 -128 123 -4.02 -402 +565 100 10555 99466 1.69669 298.69669 150.19669 15019.66966 1.69669 298.6967 150.19669 15019.66962 1.69669 298.69669 150.19669000000002 15019.66900 2020-01-01 2020-01-02 2020-01-01 00:09:25 2020-01-02 03:37:46 2020-01-01 00:09:25.000 2020-01-02 03:37:46.000 565 99466 50015.5 5001550 565 99466 50015.5 5001550 -32605 32533 4795.66 479566 -124 127 -1.78 -178 +566 100 10556 99467 1.69969 298.69969 150.19969 15019.96996 1.69969 298.6997 150.1997 15019.97037 1.69969 298.69969 150.19969 15019.96900 2020-01-01 2020-01-02 2020-01-01 00:09:26 2020-01-02 03:37:47 2020-01-01 00:09:26.000 2020-01-02 03:37:47.000 566 99467 50016.5 5001650 566 99467 50016.5 5001650 -32604 32534 4796.66 479666 -128 127 -3.34 -334 +567 100 10557 99468 1.7027 298.7027 150.2027 15020.27027 1.7027 298.7027 150.2027 15020.27003 1.70270 298.70270 150.2026999999999 15020.27000 2020-01-01 2020-01-02 2020-01-01 00:09:27 2020-01-02 03:37:48 2020-01-01 00:09:27.000 2020-01-02 03:37:48.000 567 99468 50017.5 5001750 567 99468 50017.5 5001750 -32603 32535 4797.66 479766 -128 123 -4.9 -490 +568 100 10558 99469 1.7057 298.7057 150.2057 15020.57057 1.7057 298.70572 150.2057 15020.57066 1.70570 298.70570 150.20570000000015 15020.57000 2020-01-01 2020-01-02 2020-01-01 00:09:28 2020-01-02 03:37:49 2020-01-01 00:09:28.000 2020-01-02 03:37:49.000 568 99469 50018.5 5001850 568 99469 50018.5 5001850 -32602 32536 4798.66 479866 -127 124 -3.9 -390 +569 100 10559 99470 1.7087 298.7087 150.2087 15020.87087 1.7087 298.7087 150.2087 15020.87095 1.70870 298.70870 150.20869999999996 15020.87000 2020-01-01 2020-01-02 2020-01-01 00:09:29 2020-01-02 03:37:50 2020-01-01 00:09:29.000 2020-01-02 03:37:50.000 569 99470 50019.5 5001950 569 99470 50019.5 5001950 -32601 32537 4799.66 479966 -126 125 -2.9 -290 +57 102 10047 99957 0.17117 300.17117 150.17117 15167.28828 0.17117 300.17117 150.17117 15167.28841 0.17117 300.17117 150.1711699999999 15167.28817 2020-01-01 2020-01-02 2020-01-01 00:00:57 2020-01-02 03:45:57 2020-01-01 00:00:57.000 2020-01-02 03:45:57.000 57 99957 50007 5050707 57 99957 50007 5050707 -32512 32423 4586.009900990099 463187 -126 125 -1.7128712871287128 -173 +570 100 10560 99471 1.71171 298.71171 150.21171 15021.17117 1.71171 298.7117 150.21171 15021.17109 1.71171 298.71171 150.21170999999987 15021.17100 2020-01-01 2020-01-02 2020-01-01 00:09:30 2020-01-02 03:37:51 2020-01-01 00:09:30.000 2020-01-02 03:37:51.000 570 99471 50020.5 5002050 570 99471 50020.5 5002050 -32600 32538 4800.66 480066 -125 126 -1.9 -190 +571 100 10561 99472 1.71471 298.71471 150.21471 15021.47147 1.71471 298.71472 150.21471 15021.47184 1.71471 298.71471 150.21471000000008 15021.47100 2020-01-01 2020-01-02 2020-01-01 00:09:31 2020-01-02 03:37:52 2020-01-01 00:09:31.000 2020-01-02 03:37:52.000 571 99472 50021.5 5002150 571 99472 50021.5 5002150 -32599 32539 4801.66 480166 -124 127 -0.9 -90 +572 100 10562 99473 1.71771 298.71771 150.21771 15021.77177 1.71771 298.7177 150.21771 15021.7715 1.71771 298.71771 150.21771000000004 15021.77100 2020-01-01 2020-01-02 2020-01-01 00:09:32 2020-01-02 03:37:53 2020-01-01 00:09:32.000 2020-01-02 03:37:53.000 572 99473 50022.5 5002250 572 99473 50022.5 5002250 -32598 32540 4802.66 480266 -128 127 -2.46 -246 +573 100 10563 99474 1.72072 298.72072 150.22072 15022.07207 1.72072 298.72073 150.22072 15022.07212 1.72072 298.72072 150.22071999999991 15022.07200 2020-01-01 2020-01-02 2020-01-01 00:09:33 2020-01-02 03:37:54 2020-01-01 00:09:33.000 2020-01-02 03:37:54.000 573 99474 50023.5 5002350 573 99474 50023.5 5002350 -32597 32541 4803.66 480366 -128 123 -4.02 -402 574 100 10564 99475 1.72372 298.72372 150.22372 15022.37237 1.72372 298.72372 150.22372 15022.37243 1.72372 298.72372 150.22372 15022.37200 2020-01-01 2020-01-02 2020-01-01 00:09:34 2020-01-02 03:37:55 2020-01-01 00:09:34.000 2020-01-02 03:37:55.000 574 99475 50024.5 5002450 574 99475 50024.5 5002450 -32596 32542 4804.66 480466 -127 124 -3.02 -302 -575 100 10565 99476 1.72672 298.72672 150.22672 15022.67267 1.72672 298.7267 150.22672 15022.67272 1.72672 298.72672 150.22672000000006 15022.67200 2020-01-01 2020-01-02 2020-01-01 00:09:35 2020-01-02 03:37:56 2020-01-01 00:09:35.000 2020-01-02 03:37:56.000 575 99476 50025.5 5002550 575 99476 50025.5 5002550 -32595 32543 4805.66 480566 -126 125 -2.02 -202 -576 100 10566 99477 1.72972 298.72972 150.22972 15022.97297 1.72972 298.72974 150.22973 15022.97332 1.72972 298.72972 150.22971999999987 15022.97200 2020-01-01 2020-01-02 2020-01-01 00:09:36 2020-01-02 03:37:57 2020-01-01 00:09:36.000 2020-01-02 03:37:57.000 576 99477 50026.5 5002650 576 99477 50026.5 5002650 -32594 32544 4806.66 480666 -125 126 -1.02 -102 -577 100 10567 99478 1.73273 298.73273 150.23273 15023.27327 1.73273 298.73273 150.23272 15023.27297 1.73273 298.73273 150.23272999999983 15023.27300 2020-01-01 2020-01-02 2020-01-01 00:09:37 2020-01-02 03:37:58 2020-01-01 00:09:37.000 2020-01-02 03:37:58.000 577 99478 50027.5 5002750 577 99478 50027.5 5002750 -32593 32545 4807.66 480766 -124 127 -0.02 -2 -578 100 10568 99479 1.73573 298.73573 150.23573 15023.57357 1.73573 298.73575 150.23573 15023.57359 1.73573 298.73573 150.23573000000022 15023.57300 2020-01-01 2020-01-02 2020-01-01 00:09:38 2020-01-02 03:37:59 2020-01-01 00:09:38.000 2020-01-02 03:37:59.000 578 99479 50028.5 5002850 578 99479 50028.5 5002850 -32592 32546 4808.66 480866 -128 127 -1.58 -158 -579 100 10569 99480 1.73873 298.73873 150.23873 15023.87387 1.73873 298.73874 150.23873 15023.8739 1.73873 298.73873 150.23872999999972 15023.87300 2020-01-01 2020-01-02 2020-01-01 00:09:39 2020-01-02 03:38:00 2020-01-01 00:09:39.000 2020-01-02 03:38:00.000 579 99480 50029.5 5002950 579 99480 50029.5 5002950 -32591 32547 4809.66 480966 -128 123 -3.14 -314 -58 102 10048 99958 0.17417 300.17417 150.17417 15167.59159 0.17417 300.17416 150.17417 15167.59159 0.17417 300.17417 150.17417000000015 15167.59117 2020-01-01 2020-01-02 2020-01-01 00:00:58 2020-01-02 03:45:58 2020-01-01 00:00:58.000 2020-01-02 03:45:58.000 58 99958 50008 5050808 58 99958 50008 5050808 -32511 32424 4587.009900990099 463288 -125 126 -0.7128712871287128 -72 +575 100 10565 99476 1.72672 298.72672 150.22672 15022.67267 1.72672 298.7267 150.22672 15022.67272 1.72672 298.72672 150.22672000000003 15022.67200 2020-01-01 2020-01-02 2020-01-01 00:09:35 2020-01-02 03:37:56 2020-01-01 00:09:35.000 2020-01-02 03:37:56.000 575 99476 50025.5 5002550 575 99476 50025.5 5002550 -32595 32543 4805.66 480566 -126 125 -2.02 -202 +576 100 10566 99477 1.72972 298.72972 150.22972 15022.97297 1.72972 298.72974 150.22973 15022.97332 1.72972 298.72972 150.22971999999993 15022.97200 2020-01-01 2020-01-02 2020-01-01 00:09:36 2020-01-02 03:37:57 2020-01-01 00:09:36.000 2020-01-02 03:37:57.000 576 99477 50026.5 5002650 576 99477 50026.5 5002650 -32594 32544 4806.66 480666 -125 126 -1.02 -102 +577 100 10567 99478 1.73273 298.73273 150.23273 15023.27327 1.73273 298.73273 150.23272 15023.27297 1.73273 298.73273 150.23272999999995 15023.27300 2020-01-01 2020-01-02 2020-01-01 00:09:37 2020-01-02 03:37:58 2020-01-01 00:09:37.000 2020-01-02 03:37:58.000 577 99478 50027.5 5002750 577 99478 50027.5 5002750 -32593 32545 4807.66 480766 -124 127 -0.02 -2 +578 100 10568 99479 1.73573 298.73573 150.23573 15023.57357 1.73573 298.73575 150.23573 15023.57359 1.73573 298.73573 150.2357300000001 15023.57300 2020-01-01 2020-01-02 2020-01-01 00:09:38 2020-01-02 03:37:59 2020-01-01 00:09:38.000 2020-01-02 03:37:59.000 578 99479 50028.5 5002850 578 99479 50028.5 5002850 -32592 32546 4808.66 480866 -128 127 -1.58 -158 +579 100 10569 99480 1.73873 298.73873 150.23873 15023.87387 1.73873 298.73874 150.23873 15023.8739 1.73873 298.73873 150.23872999999992 15023.87300 2020-01-01 2020-01-02 2020-01-01 00:09:39 2020-01-02 03:38:00 2020-01-01 00:09:39.000 2020-01-02 03:38:00.000 579 99480 50029.5 5002950 579 99480 50029.5 5002950 -32591 32547 4809.66 480966 -128 123 -3.14 -314 +58 102 10048 99958 0.17417 300.17417 150.17417 15167.59159 0.17417 300.17416 150.17417 15167.59159 0.17417 300.17417 150.17417000000012 15167.59117 2020-01-01 2020-01-02 2020-01-01 00:00:58 2020-01-02 03:45:58 2020-01-01 00:00:58.000 2020-01-02 03:45:58.000 58 99958 50008 5050808 58 99958 50008 5050808 -32511 32424 4587.009900990099 463288 -125 126 -0.7128712871287128 -72 580 100 10570 99481 1.74174 298.74174 150.24174 15024.17417 1.74174 298.74173 150.24174 15024.17419 1.74174 298.74174 150.24174 15024.17400 2020-01-01 2020-01-02 2020-01-01 00:09:40 2020-01-02 03:38:01 2020-01-01 00:09:40.000 2020-01-02 03:38:01.000 580 99481 50030.5 5003050 580 99481 50030.5 5003050 -32590 32548 4810.66 481066 -127 124 -2.14 -214 581 100 10571 99482 1.74474 298.74474 150.24474 15024.47447 1.74474 298.74475 150.24474 15024.47478 1.74474 298.74474 150.24474000000006 15024.47400 2020-01-01 2020-01-02 2020-01-01 00:09:41 2020-01-02 03:38:02 2020-01-01 00:09:41.000 2020-01-02 03:38:02.000 581 99482 50031.5 5003150 581 99482 50031.5 5003150 -32589 32549 4811.66 481166 -126 125 -1.14 -114 -582 100 10572 99483 1.74774 298.74774 150.24774 15024.77477 1.74774 298.74774 150.24774 15024.77447 1.74774 298.74774 150.2477400000001 15024.77400 2020-01-01 2020-01-02 2020-01-01 00:09:42 2020-01-02 03:38:03 2020-01-01 00:09:42.000 2020-01-02 03:38:03.000 582 99483 50032.5 5003250 582 99483 50032.5 5003250 -32588 32550 4812.66 481266 -125 126 -0.14 -14 -583 100 10573 99484 1.75075 298.75075 150.25075 15025.07507 1.75075 298.75076 150.25075 15025.07507 1.75075 298.75075 150.25074999999984 15025.07500 2020-01-01 2020-01-02 2020-01-01 00:09:43 2020-01-02 03:38:04 2020-01-01 00:09:43.000 2020-01-02 03:38:04.000 583 99484 50033.5 5003350 583 99484 50033.5 5003350 -32587 32551 4813.66 481366 -124 127 0.86 86 -584 100 10574 99485 1.75375 298.75375 150.25375 15025.37537 1.75375 298.75375 150.25375 15025.37536 1.75375 298.75375 150.25374999999988 15025.37500 2020-01-01 2020-01-02 2020-01-01 00:09:44 2020-01-02 03:38:05 2020-01-01 00:09:44.000 2020-01-02 03:38:05.000 584 99485 50034.5 5003450 584 99485 50034.5 5003450 -32586 32552 4814.66 481466 -128 127 -0.7 -70 -585 100 10575 99486 1.75675 298.75675 150.25675 15025.67567 1.75675 298.75674 150.25675 15025.67566 1.75675 298.75675 150.25675000000007 15025.67500 2020-01-01 2020-01-02 2020-01-01 00:09:45 2020-01-02 03:38:06 2020-01-01 00:09:45.000 2020-01-02 03:38:06.000 585 99486 50035.5 5003550 585 99486 50035.5 5003550 -32585 32553 4815.66 481566 -128 127 -2.26 -226 -586 100 10576 99487 1.75975 298.75975 150.25975 15025.97597 1.75975 298.75977 150.25976 15025.97625 1.75975 298.75975 150.25974999999985 15025.97500 2020-01-01 2020-01-02 2020-01-01 00:09:46 2020-01-02 03:38:07 2020-01-01 00:09:46.000 2020-01-02 03:38:07.000 586 99487 50036.5 5003650 586 99487 50036.5 5003650 -32584 32554 4816.66 481666 -128 123 -3.82 -382 -587 100 10577 99488 1.76276 298.76276 150.26276 15026.27627 1.76276 298.76276 150.26275 15026.27594 1.76276 298.76276 150.26275999999973 15026.27600 2020-01-01 2020-01-02 2020-01-01 00:09:47 2020-01-02 03:38:08 2020-01-01 00:09:47.000 2020-01-02 03:38:08.000 587 99488 50037.5 5003750 587 99488 50037.5 5003750 -32583 32555 4817.66 481766 -127 124 -2.82 -282 -588 100 10578 99489 1.76576 298.76576 150.26576 15026.57657 1.76576 298.76578 150.26576 15026.57654 1.76576 298.76576 150.2657600000002 15026.57600 2020-01-01 2020-01-02 2020-01-01 00:09:48 2020-01-02 03:38:09 2020-01-01 00:09:48.000 2020-01-02 03:38:09.000 588 99489 50038.5 5003850 588 99489 50038.5 5003850 -32582 32556 4818.66 481866 -126 125 -1.82 -182 -589 100 10579 99490 1.76876 298.76876 150.26876 15026.87687 1.76876 298.76877 150.26876 15026.87683 1.76876 298.76876 150.26876000000024 15026.87600 2020-01-01 2020-01-02 2020-01-01 00:09:49 2020-01-02 03:38:10 2020-01-01 00:09:49.000 2020-01-02 03:38:10.000 589 99490 50039.5 5003950 589 99490 50039.5 5003950 -32581 32557 4819.66 481966 -125 126 -0.82 -82 -59 102 10049 99959 0.17717 300.17717 150.17717 15167.89489 0.17717 300.1772 150.17717 15167.8947 0.17717 300.17717 150.17717000000025 15167.89417 2020-01-01 2020-01-02 2020-01-01 00:00:59 2020-01-02 03:45:59 2020-01-01 00:00:59.000 2020-01-02 03:45:59.000 59 99959 50009 5050909 59 99959 50009 5050909 -32510 32425 4588.009900990099 463389 -124 127 0.2871287128712871 29 -590 100 10580 99491 1.77177 298.77177 150.27177 15027.17717 1.77177 298.77176 150.27177 15027.17713 1.77177 298.77177 150.2717699999999 15027.17700 2020-01-01 2020-01-02 2020-01-01 00:09:50 2020-01-02 03:38:11 2020-01-01 00:09:50.000 2020-01-02 03:38:11.000 590 99491 50040.5 5004050 590 99491 50040.5 5004050 -32580 32558 4820.66 482066 -124 127 0.18 18 +582 100 10572 99483 1.74774 298.74774 150.24774 15024.77477 1.74774 298.74774 150.24774 15024.77447 1.74774 298.74774 150.24774000000002 15024.77400 2020-01-01 2020-01-02 2020-01-01 00:09:42 2020-01-02 03:38:03 2020-01-01 00:09:42.000 2020-01-02 03:38:03.000 582 99483 50032.5 5003250 582 99483 50032.5 5003250 -32588 32550 4812.66 481266 -125 126 -0.14 -14 +583 100 10573 99484 1.75075 298.75075 150.25075 15025.07507 1.75075 298.75076 150.25075 15025.07507 1.75075 298.75075 150.25074999999995 15025.07500 2020-01-01 2020-01-02 2020-01-01 00:09:43 2020-01-02 03:38:04 2020-01-01 00:09:43.000 2020-01-02 03:38:04.000 583 99484 50033.5 5003350 583 99484 50033.5 5003350 -32587 32551 4813.66 481366 -124 127 0.86 86 +584 100 10574 99485 1.75375 298.75375 150.25375 15025.37537 1.75375 298.75375 150.25375 15025.37536 1.75375 298.75375 150.25374999999994 15025.37500 2020-01-01 2020-01-02 2020-01-01 00:09:44 2020-01-02 03:38:05 2020-01-01 00:09:44.000 2020-01-02 03:38:05.000 584 99485 50034.5 5003450 584 99485 50034.5 5003450 -32586 32552 4814.66 481466 -128 127 -0.7 -70 +585 100 10575 99486 1.75675 298.75675 150.25675 15025.67567 1.75675 298.75674 150.25675 15025.67566 1.75675 298.75675 150.25675000000012 15025.67500 2020-01-01 2020-01-02 2020-01-01 00:09:45 2020-01-02 03:38:06 2020-01-01 00:09:45.000 2020-01-02 03:38:06.000 585 99486 50035.5 5003550 585 99486 50035.5 5003550 -32585 32553 4815.66 481566 -128 127 -2.26 -226 +586 100 10576 99487 1.75975 298.75975 150.25975 15025.97597 1.75975 298.75977 150.25976 15025.97625 1.75975 298.75975 150.25974999999997 15025.97500 2020-01-01 2020-01-02 2020-01-01 00:09:46 2020-01-02 03:38:07 2020-01-01 00:09:46.000 2020-01-02 03:38:07.000 586 99487 50036.5 5003650 586 99487 50036.5 5003650 -32584 32554 4816.66 481666 -128 123 -3.82 -382 +587 100 10577 99488 1.76276 298.76276 150.26276 15026.27627 1.76276 298.76276 150.26275 15026.27594 1.76276 298.76276 150.2627599999999 15026.27600 2020-01-01 2020-01-02 2020-01-01 00:09:47 2020-01-02 03:38:08 2020-01-01 00:09:47.000 2020-01-02 03:38:08.000 587 99488 50037.5 5003750 587 99488 50037.5 5003750 -32583 32555 4817.66 481766 -127 124 -2.82 -282 +588 100 10578 99489 1.76576 298.76576 150.26576 15026.57657 1.76576 298.76578 150.26576 15026.57654 1.76576 298.76576 150.2657600000001 15026.57600 2020-01-01 2020-01-02 2020-01-01 00:09:48 2020-01-02 03:38:09 2020-01-01 00:09:48.000 2020-01-02 03:38:09.000 588 99489 50038.5 5003850 588 99489 50038.5 5003850 -32582 32556 4818.66 481866 -126 125 -1.82 -182 +589 100 10579 99490 1.76876 298.76876 150.26876 15026.87687 1.76876 298.76877 150.26876 15026.87683 1.76876 298.76876 150.26876000000007 15026.87600 2020-01-01 2020-01-02 2020-01-01 00:09:49 2020-01-02 03:38:10 2020-01-01 00:09:49.000 2020-01-02 03:38:10.000 589 99490 50039.5 5003950 589 99490 50039.5 5003950 -32581 32557 4819.66 481966 -125 126 -0.82 -82 +59 102 10049 99959 0.17717 300.17717 150.17717 15167.89489 0.17717 300.1772 150.17717 15167.8947 0.17717 300.17717 150.17717000000007 15167.89417 2020-01-01 2020-01-02 2020-01-01 00:00:59 2020-01-02 03:45:59 2020-01-01 00:00:59.000 2020-01-02 03:45:59.000 59 99959 50009 5050909 59 99959 50009 5050909 -32510 32425 4588.009900990099 463389 -124 127 0.2871287128712871 29 +590 100 10580 99491 1.77177 298.77177 150.27177 15027.17717 1.77177 298.77176 150.27177 15027.17713 1.77177 298.77177 150.27176999999995 15027.17700 2020-01-01 2020-01-02 2020-01-01 00:09:50 2020-01-02 03:38:11 2020-01-01 00:09:50.000 2020-01-02 03:38:11.000 590 99491 50040.5 5004050 590 99491 50040.5 5004050 -32580 32558 4820.66 482066 -124 127 0.18 18 591 100 10581 99492 1.77477 298.77477 150.27477 15027.47747 1.77477 298.77478 150.27477 15027.47775 1.77477 298.77477 150.27477 15027.47700 2020-01-01 2020-01-02 2020-01-01 00:09:51 2020-01-02 03:38:12 2020-01-01 00:09:51.000 2020-01-02 03:38:12.000 591 99492 50041.5 5004150 591 99492 50041.5 5004150 -32579 32559 4821.66 482166 -128 127 -1.38 -138 -592 100 10582 99493 1.77777 298.77777 150.27777 15027.77777 1.77777 298.77777 150.27777 15027.77742 1.77777 298.77777 150.2777700000001 15027.77700 2020-01-01 2020-01-02 2020-01-01 00:09:52 2020-01-02 03:38:13 2020-01-01 00:09:52.000 2020-01-02 03:38:13.000 592 99493 50042.5 5004250 592 99493 50042.5 5004250 -32578 32560 4822.66 482266 -128 123 -2.94 -294 -593 100 10583 99494 1.78078 298.78078 150.28078 15028.07807 1.78078 298.7808 150.28078 15028.078 1.78078 298.78078 150.28077999999974 15028.07800 2020-01-01 2020-01-02 2020-01-01 00:09:53 2020-01-02 03:38:14 2020-01-01 00:09:53.000 2020-01-02 03:38:14.000 593 99494 50043.5 5004350 593 99494 50043.5 5004350 -32577 32561 4823.66 482366 -127 124 -1.94 -194 -594 100 10584 99495 1.78378 298.78378 150.28378 15028.37837 1.78378 298.78378 150.28378 15028.3783 1.78378 298.78378 150.28377999999984 15028.37800 2020-01-01 2020-01-02 2020-01-01 00:09:54 2020-01-02 03:38:15 2020-01-01 00:09:54.000 2020-01-02 03:38:15.000 594 99495 50044.5 5004450 594 99495 50044.5 5004450 -32576 32562 4824.66 482466 -126 125 -0.94 -94 -595 100 10585 99496 1.78678 298.78678 150.28678 15028.67867 1.78678 298.78677 150.28678 15028.6786 1.78678 298.78678 150.28678000000025 15028.67800 2020-01-01 2020-01-02 2020-01-01 00:09:55 2020-01-02 03:38:16 2020-01-01 00:09:55.000 2020-01-02 03:38:16.000 595 99496 50045.5 5004550 595 99496 50045.5 5004550 -32575 32563 4825.66 482566 -125 126 0.06 6 -596 100 10586 99497 1.78978 298.78978 150.28978 15028.97897 1.78978 298.7898 150.28979 15028.97922 1.78978 298.78978 150.28977999999975 15028.97800 2020-01-01 2020-01-02 2020-01-01 00:09:56 2020-01-02 03:38:17 2020-01-01 00:09:56.000 2020-01-02 03:38:17.000 596 99497 50046.5 5004650 596 99497 50046.5 5004650 -32574 32564 4826.66 482666 -124 127 1.06 106 +592 100 10582 99493 1.77777 298.77777 150.27777 15027.77777 1.77777 298.77777 150.27777 15027.77742 1.77777 298.77777 150.27777000000003 15027.77700 2020-01-01 2020-01-02 2020-01-01 00:09:52 2020-01-02 03:38:13 2020-01-01 00:09:52.000 2020-01-02 03:38:13.000 592 99493 50042.5 5004250 592 99493 50042.5 5004250 -32578 32560 4822.66 482266 -128 123 -2.94 -294 +593 100 10583 99494 1.78078 298.78078 150.28078 15028.07807 1.78078 298.7808 150.28078 15028.078 1.78078 298.78078 150.28077999999994 15028.07800 2020-01-01 2020-01-02 2020-01-01 00:09:53 2020-01-02 03:38:14 2020-01-01 00:09:53.000 2020-01-02 03:38:14.000 593 99494 50043.5 5004350 593 99494 50043.5 5004350 -32577 32561 4823.66 482366 -127 124 -1.94 -194 +594 100 10584 99495 1.78378 298.78378 150.28378 15028.37837 1.78378 298.78378 150.28378 15028.3783 1.78378 298.78378 150.28377999999992 15028.37800 2020-01-01 2020-01-02 2020-01-01 00:09:54 2020-01-02 03:38:15 2020-01-01 00:09:54.000 2020-01-02 03:38:15.000 594 99495 50044.5 5004450 594 99495 50044.5 5004450 -32576 32562 4824.66 482466 -126 125 -0.94 -94 +595 100 10585 99496 1.78678 298.78678 150.28678 15028.67867 1.78678 298.78677 150.28678 15028.6786 1.78678 298.78678 150.28678000000008 15028.67800 2020-01-01 2020-01-02 2020-01-01 00:09:55 2020-01-02 03:38:16 2020-01-01 00:09:55.000 2020-01-02 03:38:16.000 595 99496 50045.5 5004550 595 99496 50045.5 5004550 -32575 32563 4825.66 482566 -125 126 0.06 6 +596 100 10586 99497 1.78978 298.78978 150.28978 15028.97897 1.78978 298.7898 150.28979 15028.97922 1.78978 298.78978 150.28977999999992 15028.97800 2020-01-01 2020-01-02 2020-01-01 00:09:56 2020-01-02 03:38:17 2020-01-01 00:09:56.000 2020-01-02 03:38:17.000 596 99497 50046.5 5004650 596 99497 50046.5 5004650 -32574 32564 4826.66 482666 -124 127 1.06 106 597 100 10587 99498 1.79279 298.79279 150.29279 15029.27927 1.79279 298.7928 150.29278 15029.27888 1.79279 298.79279 150.29279000000002 15029.27900 2020-01-01 2020-01-02 2020-01-01 00:09:57 2020-01-02 03:38:18 2020-01-01 00:09:57.000 2020-01-02 03:38:18.000 597 99498 50047.5 5004750 597 99498 50047.5 5004750 -32573 32565 4827.66 482766 -128 127 -0.5 -50 -598 100 10588 99499 1.79579 298.79579 150.29579 15029.57957 1.79579 298.7958 150.29579 15029.57964 1.79579 298.79579 150.2957900000001 15029.57900 2020-01-01 2020-01-02 2020-01-01 00:09:58 2020-01-02 03:38:19 2020-01-01 00:09:58.000 2020-01-02 03:38:19.000 598 99499 50048.5 5004850 598 99499 50048.5 5004850 -32572 32566 4828.66 482866 -128 123 -2.06 -206 -599 100 10589 99500 1.79879 298.79879 150.29879 15029.87987 1.79879 298.7988 150.29879 15029.87977 1.79879 298.79879 150.29879000000014 15029.87900 2020-01-01 2020-01-02 2020-01-01 00:09:59 2020-01-02 03:38:20 2020-01-01 00:09:59.000 2020-01-02 03:38:20.000 599 99500 50049.5 5004950 599 99500 50049.5 5004950 -32571 32567 4829.66 482966 -127 124 -1.06 -106 -6 102 1005 9996 0.01801 300.01801 150.01801 15151.81981 0.01801 300.018 150.01801 15151.81978 0.01801 300.01801 150.0180099999998 15151.81901 2020-01-01 2020-01-02 2020-01-01 00:00:06 2020-01-02 03:45:06 2020-01-01 00:00:06.000 2020-01-02 03:45:06.000 6 99906 49956 5045556 6 99906 49956 5045556 -32563 32372 4535.009900990099 458036 -127 124 -2.01980198019802 -204 -60 102 10050 99960 0.18018 300.18018 150.18018 15168.19819 0.18018 300.18018 150.18017 15168.198 0.18018 300.18018 150.1801799999999 15168.19818 2020-01-01 2020-01-02 2020-01-01 00:01:00 2020-01-02 03:46:00 2020-01-01 00:01:00.000 2020-01-02 03:46:00.000 60 99960 50010 5051010 60 99960 50010 5051010 -32509 32426 4589.009900990099 463490 -128 127 -1.2475247524752475 -126 -600 100 10590 99501 1.8018 298.8018 150.3018 15030.18018 1.8018 298.8018 150.3018 15030.1801 1.80180 298.80180 150.30179999999987 15030.18000 2020-01-01 2020-01-02 2020-01-01 00:10:00 2020-01-02 03:38:21 2020-01-01 00:10:00.000 2020-01-02 03:38:21.000 600 99501 50050.5 5005050 600 99501 50050.5 5005050 -32570 32568 4830.66 483066 -126 125 -0.06 -6 +598 100 10588 99499 1.79579 298.79579 150.29579 15029.57957 1.79579 298.7958 150.29579 15029.57964 1.79579 298.79579 150.29579000000007 15029.57900 2020-01-01 2020-01-02 2020-01-01 00:09:58 2020-01-02 03:38:19 2020-01-01 00:09:58.000 2020-01-02 03:38:19.000 598 99499 50048.5 5004850 598 99499 50048.5 5004850 -32572 32566 4828.66 482866 -128 123 -2.06 -206 +599 100 10589 99500 1.79879 298.79879 150.29879 15029.87987 1.79879 298.7988 150.29879 15029.87977 1.79879 298.79879 150.29879000000003 15029.87900 2020-01-01 2020-01-02 2020-01-01 00:09:59 2020-01-02 03:38:20 2020-01-01 00:09:59.000 2020-01-02 03:38:20.000 599 99500 50049.5 5004950 599 99500 50049.5 5004950 -32571 32567 4829.66 482966 -127 124 -1.06 -106 +6 102 1005 9996 0.01801 300.01801 150.01801 15151.81981 0.01801 300.018 150.01801 15151.81978 0.01801 300.01801 150.0180099999999 15151.81901 2020-01-01 2020-01-02 2020-01-01 00:00:06 2020-01-02 03:45:06 2020-01-01 00:00:06.000 2020-01-02 03:45:06.000 6 99906 49956 5045556 6 99906 49956 5045556 -32563 32372 4535.009900990099 458036 -127 124 -2.01980198019802 -204 +60 102 10050 99960 0.18018 300.18018 150.18018 15168.19819 0.18018 300.18018 150.18017 15168.198 0.18018 300.18018 150.18017999999995 15168.19818 2020-01-01 2020-01-02 2020-01-01 00:01:00 2020-01-02 03:46:00 2020-01-01 00:01:00.000 2020-01-02 03:46:00.000 60 99960 50010 5051010 60 99960 50010 5051010 -32509 32426 4589.009900990099 463490 -128 127 -1.2475247524752475 -126 +600 100 10590 99501 1.8018 298.8018 150.3018 15030.18018 1.8018 298.8018 150.3018 15030.1801 1.80180 298.80180 150.3018 15030.18000 2020-01-01 2020-01-02 2020-01-01 00:10:00 2020-01-02 03:38:21 2020-01-01 00:10:00.000 2020-01-02 03:38:21.000 600 99501 50050.5 5005050 600 99501 50050.5 5005050 -32570 32568 4830.66 483066 -126 125 -0.06 -6 601 100 10591 99502 1.8048 298.8048 150.3048 15030.48048 1.8048 298.8048 150.3048 15030.4807 1.80480 298.80480 150.30479999999991 15030.48000 2020-01-01 2020-01-02 2020-01-01 00:10:01 2020-01-02 03:38:22 2020-01-01 00:10:01.000 2020-01-02 03:38:22.000 601 99502 50051.5 5005150 601 99502 50051.5 5005150 -32569 32569 4831.66 483166 -125 126 0.94 94 602 100 10592 99503 1.8078 298.8078 150.3078 15030.78078 1.8078 298.8078 150.3078 15030.78035 1.80780 298.80780 150.3078 15030.78000 2020-01-01 2020-01-02 2020-01-01 00:10:02 2020-01-02 03:38:23 2020-01-01 00:10:02.000 2020-01-02 03:38:23.000 602 99503 50052.5 5005250 602 99503 50052.5 5005250 -32568 32570 4832.66 483266 -124 127 1.94 194 -603 100 10593 99504 1.81081 298.81081 150.31081 15031.08108 1.81081 298.81082 150.31081 15031.0811 1.81081 298.81081 150.31081000000026 15031.08100 2020-01-01 2020-01-02 2020-01-01 00:10:03 2020-01-02 03:38:24 2020-01-01 00:10:03.000 2020-01-02 03:38:24.000 603 99504 50053.5 5005350 603 99504 50053.5 5005350 -32567 32571 4833.66 483366 -128 127 0.38 38 -604 100 10594 99505 1.81381 298.81381 150.31381 15031.38138 1.81381 298.8138 150.31381 15031.38124 1.81381 298.81381 150.31380999999976 15031.38100 2020-01-01 2020-01-02 2020-01-01 00:10:04 2020-01-02 03:38:25 2020-01-01 00:10:04.000 2020-01-02 03:38:25.000 604 99505 50054.5 5005450 604 99505 50054.5 5005450 -32566 32572 4834.66 483466 -128 123 -1.18 -118 -605 100 10595 99506 1.81681 298.81681 150.31681 15031.68168 1.81681 298.8168 150.31681 15031.68157 1.81681 298.81681 150.31681000000015 15031.68100 2020-01-01 2020-01-02 2020-01-01 00:10:05 2020-01-02 03:38:26 2020-01-01 00:10:05.000 2020-01-02 03:38:26.000 605 99506 50055.5 5005550 605 99506 50055.5 5005550 -32565 32573 4835.66 483566 -127 124 -0.18 -18 -606 100 10596 99507 1.81981 298.81981 150.31981 15031.98198 1.81981 298.81982 150.31982 15031.98217 1.81981 298.81981 150.31981000000027 15031.98100 2020-01-01 2020-01-02 2020-01-01 00:10:06 2020-01-02 03:38:27 2020-01-01 00:10:06.000 2020-01-02 03:38:27.000 606 99507 50056.5 5005650 606 99507 50056.5 5005650 -32564 32574 4836.66 483666 -126 125 0.82 82 -607 100 10597 99508 1.82282 298.82282 150.32282 15032.28228 1.82282 298.8228 150.32282 15032.28246 1.82282 298.82282 150.32281999999992 15032.28200 2020-01-01 2020-01-02 2020-01-01 00:10:07 2020-01-02 03:38:28 2020-01-01 00:10:07.000 2020-01-02 03:38:28.000 607 99508 50057.5 5005750 607 99508 50057.5 5005750 -32563 32575 4837.66 483766 -125 126 1.82 182 -608 100 10598 99509 1.82582 298.82582 150.32582 15032.58258 1.82582 298.82584 150.32582 15032.58258 1.82582 298.82582 150.32582 15032.58200 2020-01-01 2020-01-02 2020-01-01 00:10:08 2020-01-02 03:38:29 2020-01-01 00:10:08.000 2020-01-02 03:38:29.000 608 99509 50058.5 5005850 608 99509 50058.5 5005850 -32562 32576 4838.66 483866 -124 127 2.82 282 -609 100 10599 99510 1.82882 298.82882 150.32882 15032.88288 1.82882 298.82883 150.32882 15032.88274 1.82882 298.82882 150.32882000000006 15032.88200 2020-01-01 2020-01-02 2020-01-01 00:10:09 2020-01-02 03:38:30 2020-01-01 00:10:09.000 2020-01-02 03:38:30.000 609 99510 50059.5 5005950 609 99510 50059.5 5005950 -32561 32577 4839.66 483966 -128 127 1.26 126 +603 100 10593 99504 1.81081 298.81081 150.31081 15031.08108 1.81081 298.81082 150.31081 15031.0811 1.81081 298.81081 150.31081000000006 15031.08100 2020-01-01 2020-01-02 2020-01-01 00:10:03 2020-01-02 03:38:24 2020-01-01 00:10:03.000 2020-01-02 03:38:24.000 603 99504 50053.5 5005350 603 99504 50053.5 5005350 -32567 32571 4833.66 483366 -128 127 0.38 38 +604 100 10594 99505 1.81381 298.81381 150.31381 15031.38138 1.81381 298.8138 150.31381 15031.38124 1.81381 298.81381 150.31380999999993 15031.38100 2020-01-01 2020-01-02 2020-01-01 00:10:04 2020-01-02 03:38:25 2020-01-01 00:10:04.000 2020-01-02 03:38:25.000 604 99505 50054.5 5005450 604 99505 50054.5 5005450 -32566 32572 4834.66 483466 -128 123 -1.18 -118 +605 100 10595 99506 1.81681 298.81681 150.31681 15031.68168 1.81681 298.8168 150.31681 15031.68157 1.81681 298.81681 150.31681000000003 15031.68100 2020-01-01 2020-01-02 2020-01-01 00:10:05 2020-01-02 03:38:26 2020-01-01 00:10:05.000 2020-01-02 03:38:26.000 605 99506 50055.5 5005550 605 99506 50055.5 5005550 -32565 32573 4835.66 483566 -127 124 -0.18 -18 +606 100 10596 99507 1.81981 298.81981 150.31981 15031.98198 1.81981 298.81982 150.31982 15031.98217 1.81981 298.81981 150.31981000000007 15031.98100 2020-01-01 2020-01-02 2020-01-01 00:10:06 2020-01-02 03:38:27 2020-01-01 00:10:06.000 2020-01-02 03:38:27.000 606 99507 50056.5 5005650 606 99507 50056.5 5005650 -32564 32574 4836.66 483666 -126 125 0.82 82 +607 100 10597 99508 1.82282 298.82282 150.32282 15032.28228 1.82282 298.8228 150.32282 15032.28246 1.82282 298.82282 150.32281999999998 15032.28200 2020-01-01 2020-01-02 2020-01-01 00:10:07 2020-01-02 03:38:28 2020-01-01 00:10:07.000 2020-01-02 03:38:28.000 607 99508 50057.5 5005750 607 99508 50057.5 5005750 -32563 32575 4837.66 483766 -125 126 1.82 182 +608 100 10598 99509 1.82582 298.82582 150.32582 15032.58258 1.82582 298.82584 150.32582 15032.58258 1.82582 298.82582 150.32582000000002 15032.58200 2020-01-01 2020-01-02 2020-01-01 00:10:08 2020-01-02 03:38:29 2020-01-01 00:10:08.000 2020-01-02 03:38:29.000 608 99509 50058.5 5005850 608 99509 50058.5 5005850 -32562 32576 4838.66 483866 -124 127 2.82 282 +609 100 10599 99510 1.82882 298.82882 150.32882 15032.88288 1.82882 298.82883 150.32882 15032.88274 1.82882 298.82882 150.32882000000012 15032.88200 2020-01-01 2020-01-02 2020-01-01 00:10:09 2020-01-02 03:38:30 2020-01-01 00:10:09.000 2020-01-02 03:38:30.000 609 99510 50059.5 5005950 609 99510 50059.5 5005950 -32561 32577 4839.66 483966 -128 127 1.26 126 61 102 10051 99961 0.18318 300.18318 150.18318 15168.5015 0.18318 300.1832 150.18318 15168.5016 0.18318 300.18318 150.18318 15168.50118 2020-01-01 2020-01-02 2020-01-01 00:01:01 2020-01-02 03:46:01 2020-01-01 00:01:01.000 2020-01-02 03:46:01.000 61 99961 50011 5051111 61 99961 50011 5051111 -32508 32427 4590.009900990099 463591 -128 123 -2.782178217821782 -281 -610 100 10600 99511 1.83183 298.83183 150.33183 15033.18318 1.83183 298.83182 150.33183 15033.18304 1.83183 298.83183 150.33182999999994 15033.18300 2020-01-01 2020-01-02 2020-01-01 00:10:10 2020-01-02 03:38:31 2020-01-01 00:10:10.000 2020-01-02 03:38:31.000 610 99511 50060.5 5006050 610 99511 50060.5 5006050 -32560 32578 4840.66 484066 -128 127 -0.3 -30 -611 100 10601 99512 1.83483 298.83483 150.33483 15033.48348 1.83483 298.83484 150.33483 15033.48363 1.83483 298.83483 150.3348299999999 15033.48300 2020-01-01 2020-01-02 2020-01-01 00:10:11 2020-01-02 03:38:32 2020-01-01 00:10:11.000 2020-01-02 03:38:32.000 611 99512 50061.5 5006150 611 99512 50061.5 5006150 -32559 32579 4841.66 484166 -128 123 -1.86 -186 -612 100 10602 99513 1.83783 298.83783 150.33783 15033.78378 1.83783 298.83783 150.33783 15033.78393 1.83783 298.83783 150.3378300000001 15033.78300 2020-01-01 2020-01-02 2020-01-01 00:10:12 2020-01-02 03:38:33 2020-01-01 00:10:12.000 2020-01-02 03:38:33.000 612 99513 50062.5 5006250 612 99513 50062.5 5006250 -32558 32580 4842.66 484266 -127 124 -0.86 -86 -613 100 10603 99514 1.84084 298.84084 150.34084 15034.08408 1.84084 298.84085 150.34084 15034.08405 1.84084 298.84084 150.34084000000004 15034.08400 2020-01-01 2020-01-02 2020-01-01 00:10:13 2020-01-02 03:38:34 2020-01-01 00:10:13.000 2020-01-02 03:38:34.000 613 99514 50063.5 5006350 613 99514 50063.5 5006350 -32557 32581 4843.66 484366 -126 125 0.14 14 -614 100 10604 99515 1.84384 298.84384 150.34384 15034.38438 1.84384 298.84384 150.34384 15034.38421 1.84384 298.84384 150.34383999999983 15034.38400 2020-01-01 2020-01-02 2020-01-01 00:10:14 2020-01-02 03:38:35 2020-01-01 00:10:14.000 2020-01-02 03:38:35.000 614 99515 50064.5 5006450 614 99515 50064.5 5006450 -32556 32582 4844.66 484466 -125 126 1.14 114 -615 100 10605 99516 1.84684 298.84684 150.34684 15034.68468 1.84684 298.84683 150.34684 15034.68452 1.84684 298.84684 150.34684000000007 15034.68400 2020-01-01 2020-01-02 2020-01-01 00:10:15 2020-01-02 03:38:36 2020-01-01 00:10:15.000 2020-01-02 03:38:36.000 615 99516 50065.5 5006550 615 99516 50065.5 5006550 -32555 32583 4845.66 484566 -124 127 2.14 214 -616 100 10606 99517 1.84984 298.84984 150.34984 15034.98498 1.84984 298.84985 150.34985 15034.98527 1.84984 298.84984 150.34984000000003 15034.98400 2020-01-01 2020-01-02 2020-01-01 00:10:16 2020-01-02 03:38:37 2020-01-01 00:10:16.000 2020-01-02 03:38:37.000 616 99517 50066.5 5006650 616 99517 50066.5 5006650 -32554 32584 4846.66 484666 -128 127 0.58 58 -617 100 10607 99518 1.85285 298.85285 150.35285 15035.28528 1.85285 298.85284 150.35285 15035.2854 1.85285 298.85285 150.35285 15035.28500 2020-01-01 2020-01-02 2020-01-01 00:10:17 2020-01-02 03:38:38 2020-01-01 00:10:17.000 2020-01-02 03:38:38.000 617 99518 50067.5 5006750 617 99518 50067.5 5006750 -32553 32585 4847.66 484766 -128 123 -0.98 -98 +610 100 10600 99511 1.83183 298.83183 150.33183 15033.18318 1.83183 298.83182 150.33183 15033.18304 1.83183 298.83183 150.33182999999977 15033.18300 2020-01-01 2020-01-02 2020-01-01 00:10:10 2020-01-02 03:38:31 2020-01-01 00:10:10.000 2020-01-02 03:38:31.000 610 99511 50060.5 5006050 610 99511 50060.5 5006050 -32560 32578 4840.66 484066 -128 127 -0.3 -30 +611 100 10601 99512 1.83483 298.83483 150.33483 15033.48348 1.83483 298.83484 150.33483 15033.48363 1.83483 298.83483 150.3348299999998 15033.48300 2020-01-01 2020-01-02 2020-01-01 00:10:11 2020-01-02 03:38:32 2020-01-01 00:10:11.000 2020-01-02 03:38:32.000 611 99512 50061.5 5006150 611 99512 50061.5 5006150 -32559 32579 4841.66 484166 -128 123 -1.86 -186 +612 100 10602 99513 1.83783 298.83783 150.33783 15033.78378 1.83783 298.83783 150.33783 15033.78393 1.83783 298.83783 150.33783000000028 15033.78300 2020-01-01 2020-01-02 2020-01-01 00:10:12 2020-01-02 03:38:33 2020-01-01 00:10:12.000 2020-01-02 03:38:33.000 612 99513 50062.5 5006250 612 99513 50062.5 5006250 -32558 32580 4842.66 484266 -127 124 -0.86 -86 +613 100 10603 99514 1.84084 298.84084 150.34084 15034.08408 1.84084 298.84085 150.34084 15034.08405 1.84084 298.84084 150.34084000000016 15034.08400 2020-01-01 2020-01-02 2020-01-01 00:10:13 2020-01-02 03:38:34 2020-01-01 00:10:13.000 2020-01-02 03:38:34.000 613 99514 50063.5 5006350 613 99514 50063.5 5006350 -32557 32581 4843.66 484366 -126 125 0.14 14 +614 100 10604 99515 1.84384 298.84384 150.34384 15034.38438 1.84384 298.84384 150.34384 15034.38421 1.84384 298.84384 150.34383999999966 15034.38400 2020-01-01 2020-01-02 2020-01-01 00:10:14 2020-01-02 03:38:35 2020-01-01 00:10:14.000 2020-01-02 03:38:35.000 614 99515 50064.5 5006450 614 99515 50064.5 5006450 -32556 32582 4844.66 484466 -125 126 1.14 114 +615 100 10605 99516 1.84684 298.84684 150.34684 15034.68468 1.84684 298.84683 150.34684 15034.68452 1.84684 298.84684 150.34684000000013 15034.68400 2020-01-01 2020-01-02 2020-01-01 00:10:15 2020-01-02 03:38:36 2020-01-01 00:10:15.000 2020-01-02 03:38:36.000 615 99516 50065.5 5006550 615 99516 50065.5 5006550 -32555 32583 4845.66 484566 -124 127 2.14 214 +616 100 10606 99517 1.84984 298.84984 150.34984 15034.98498 1.84984 298.84985 150.34985 15034.98527 1.84984 298.84984 150.34984000000017 15034.98400 2020-01-01 2020-01-02 2020-01-01 00:10:16 2020-01-02 03:38:37 2020-01-01 00:10:16.000 2020-01-02 03:38:37.000 616 99517 50066.5 5006650 616 99517 50066.5 5006650 -32554 32584 4846.66 484666 -128 127 0.58 58 +617 100 10607 99518 1.85285 298.85285 150.35285 15035.28528 1.85285 298.85284 150.35285 15035.2854 1.85285 298.85285 150.3528499999999 15035.28500 2020-01-01 2020-01-02 2020-01-01 00:10:17 2020-01-02 03:38:38 2020-01-01 00:10:17.000 2020-01-02 03:38:38.000 617 99518 50067.5 5006750 617 99518 50067.5 5006750 -32553 32585 4847.66 484766 -128 123 -0.98 -98 618 100 10608 99519 1.85585 298.85585 150.35585 15035.58558 1.85585 298.85587 150.35585 15035.58551 1.85585 298.85585 150.35584999999995 15035.58500 2020-01-01 2020-01-02 2020-01-01 00:10:18 2020-01-02 03:38:39 2020-01-01 00:10:18.000 2020-01-02 03:38:39.000 618 99519 50068.5 5006850 618 99519 50068.5 5006850 -32552 32586 4848.66 484866 -127 124 0.02 2 -619 100 10609 99520 1.85885 298.85885 150.35885 15035.88588 1.85885 298.85886 150.35885 15035.88568 1.85885 298.85885 150.35885 15035.88500 2020-01-01 2020-01-02 2020-01-01 00:10:19 2020-01-02 03:38:40 2020-01-01 00:10:19.000 2020-01-02 03:38:40.000 619 99520 50069.5 5006950 619 99520 50069.5 5006950 -32551 32587 4849.66 484966 -126 125 1.02 102 -62 102 10052 99962 0.18618 300.18618 150.18618 15168.8048 0.18618 300.1862 150.18618 15168.80494 0.18618 300.18618 150.18618000000004 15168.80418 2020-01-01 2020-01-02 2020-01-01 00:01:02 2020-01-02 03:46:02 2020-01-01 00:01:02.000 2020-01-02 03:46:02.000 62 99962 50012 5051212 62 99962 50012 5051212 -32507 32428 4591.009900990099 463692 -127 124 -1.7821782178217822 -180 -620 100 10610 99521 1.86186 298.86186 150.36186 15036.18618 1.86186 298.86185 150.36185 15036.18598 1.86186 298.86186 150.3618600000001 15036.18600 2020-01-01 2020-01-02 2020-01-01 00:10:20 2020-01-02 03:38:41 2020-01-01 00:10:20.000 2020-01-02 03:38:41.000 620 99521 50070.5 5007050 620 99521 50070.5 5007050 -32550 32588 4850.66 485066 -125 126 2.02 202 -621 100 10611 99522 1.86486 298.86486 150.36486 15036.48648 1.86486 298.86487 150.36486 15036.48673 1.86486 298.86486 150.3648599999999 15036.48600 2020-01-01 2020-01-02 2020-01-01 00:10:21 2020-01-02 03:38:42 2020-01-01 00:10:21.000 2020-01-02 03:38:42.000 621 99522 50071.5 5007150 621 99522 50071.5 5007150 -32549 32589 4851.66 485166 -124 127 3.02 302 -622 100 10612 99523 1.86786 298.86786 150.36786 15036.78678 1.86786 298.86786 150.36786 15036.78687 1.86786 298.86786 150.36786000000006 15036.78600 2020-01-01 2020-01-02 2020-01-01 00:10:22 2020-01-02 03:38:43 2020-01-01 00:10:22.000 2020-01-02 03:38:43.000 622 99523 50072.5 5007250 622 99523 50072.5 5007250 -32548 32590 4852.66 485266 -128 127 1.46 146 -623 100 10613 99524 1.87087 298.87087 150.37087 15037.08708 1.87087 298.87088 150.37087 15037.08702 1.87087 298.87087 150.37087000000008 15037.08700 2020-01-01 2020-01-02 2020-01-01 00:10:23 2020-01-02 03:38:44 2020-01-01 00:10:23.000 2020-01-02 03:38:44.000 623 99524 50073.5 5007350 623 99524 50073.5 5007350 -32547 32591 4853.66 485366 -128 123 -0.1 -10 -624 100 10614 99525 1.87387 298.87387 150.37387 15037.38738 1.87387 298.87387 150.37387 15037.38716 1.87387 298.87387 150.37386999999998 15037.38700 2020-01-01 2020-01-02 2020-01-01 00:10:24 2020-01-02 03:38:45 2020-01-01 00:10:24.000 2020-01-02 03:38:45.000 624 99525 50074.5 5007450 624 99525 50074.5 5007450 -32546 32592 4854.66 485466 -127 124 0.9 90 +619 100 10609 99520 1.85885 298.85885 150.35885 15035.88588 1.85885 298.85886 150.35885 15035.88568 1.85885 298.85885 150.35885000000002 15035.88500 2020-01-01 2020-01-02 2020-01-01 00:10:19 2020-01-02 03:38:40 2020-01-01 00:10:19.000 2020-01-02 03:38:40.000 619 99520 50069.5 5006950 619 99520 50069.5 5006950 -32551 32587 4849.66 484966 -126 125 1.02 102 +62 102 10052 99962 0.18618 300.18618 150.18618 15168.8048 0.18618 300.1862 150.18618 15168.80494 0.18618 300.18618 150.1861800000001 15168.80418 2020-01-01 2020-01-02 2020-01-01 00:01:02 2020-01-02 03:46:02 2020-01-01 00:01:02.000 2020-01-02 03:46:02.000 62 99962 50012 5051212 62 99962 50012 5051212 -32507 32428 4591.009900990099 463692 -127 124 -1.7821782178217822 -180 +620 100 10610 99521 1.86186 298.86186 150.36186 15036.18618 1.86186 298.86185 150.36185 15036.18598 1.86186 298.86186 150.36186000000026 15036.18600 2020-01-01 2020-01-02 2020-01-01 00:10:20 2020-01-02 03:38:41 2020-01-01 00:10:20.000 2020-01-02 03:38:41.000 620 99521 50070.5 5007050 620 99521 50070.5 5007050 -32550 32588 4850.66 485066 -125 126 2.02 202 +621 100 10611 99522 1.86486 298.86486 150.36486 15036.48648 1.86486 298.86487 150.36486 15036.48673 1.86486 298.86486 150.3648599999998 15036.48600 2020-01-01 2020-01-02 2020-01-01 00:10:21 2020-01-02 03:38:42 2020-01-01 00:10:21.000 2020-01-02 03:38:42.000 621 99522 50071.5 5007150 621 99522 50071.5 5007150 -32549 32589 4851.66 485166 -124 127 3.02 302 +622 100 10612 99523 1.86786 298.86786 150.36786 15036.78678 1.86786 298.86786 150.36786 15036.78687 1.86786 298.86786 150.36786000000018 15036.78600 2020-01-01 2020-01-02 2020-01-01 00:10:22 2020-01-02 03:38:43 2020-01-01 00:10:22.000 2020-01-02 03:38:43.000 622 99523 50072.5 5007250 622 99523 50072.5 5007250 -32548 32590 4852.66 485266 -128 127 1.46 146 +623 100 10613 99524 1.87087 298.87087 150.37087 15037.08708 1.87087 298.87088 150.37087 15037.08702 1.87087 298.87087 150.37087000000014 15037.08700 2020-01-01 2020-01-02 2020-01-01 00:10:23 2020-01-02 03:38:44 2020-01-01 00:10:23.000 2020-01-02 03:38:44.000 623 99524 50073.5 5007350 623 99524 50073.5 5007350 -32547 32591 4853.66 485366 -128 123 -0.1 -10 +624 100 10614 99525 1.87387 298.87387 150.37387 15037.38738 1.87387 298.87387 150.37387 15037.38716 1.87387 298.87387 150.37386999999995 15037.38700 2020-01-01 2020-01-02 2020-01-01 00:10:24 2020-01-02 03:38:45 2020-01-01 00:10:24.000 2020-01-02 03:38:45.000 624 99525 50074.5 5007450 624 99525 50074.5 5007450 -32546 32592 4854.66 485466 -127 124 0.9 90 625 100 10615 99526 1.87687 298.87687 150.37687 15037.68768 1.87687 298.8769 150.37687 15037.68791 1.87687 298.87687 150.37687000000003 15037.68700 2020-01-01 2020-01-02 2020-01-01 00:10:25 2020-01-02 03:38:46 2020-01-01 00:10:25.000 2020-01-02 03:38:46.000 625 99526 50075.5 5007550 625 99526 50075.5 5007550 -32545 32593 4855.66 485566 -126 125 1.9 190 -626 100 10616 99527 1.87987 298.87987 150.37987 15037.98798 1.87987 298.87988 150.37988 15037.9882 1.87987 298.87987 150.37987000000004 15037.98700 2020-01-01 2020-01-02 2020-01-01 00:10:26 2020-01-02 03:38:47 2020-01-01 00:10:26.000 2020-01-02 03:38:47.000 626 99527 50076.5 5007650 626 99527 50076.5 5007650 -32544 32594 4856.66 485666 -125 126 2.9 290 -627 100 10617 99528 1.88288 298.88288 150.38288 15038.28828 1.88288 298.88287 150.38288 15038.28834 1.88288 298.88288 150.38287999999997 15038.28800 2020-01-01 2020-01-02 2020-01-01 00:10:27 2020-01-02 03:38:48 2020-01-01 00:10:27.000 2020-01-02 03:38:48.000 627 99528 50077.5 5007750 627 99528 50077.5 5007750 -32543 32595 4857.66 485766 -124 127 3.9 390 -628 100 10618 99529 1.88588 298.88588 150.38588 15038.58858 1.88588 298.8859 150.38588 15038.58849 1.88588 298.88588 150.3858799999999 15038.58800 2020-01-01 2020-01-02 2020-01-01 00:10:28 2020-01-02 03:38:49 2020-01-01 00:10:28.000 2020-01-02 03:38:49.000 628 99529 50078.5 5007850 628 99529 50078.5 5007850 -32542 32596 4858.66 485866 -128 127 2.34 234 -629 100 10619 99530 1.88888 298.88888 150.38888 15038.88888 1.88888 298.8889 150.38888 15038.88862 1.88888 298.88888 150.38888000000014 15038.88800 2020-01-01 2020-01-02 2020-01-01 00:10:29 2020-01-02 03:38:50 2020-01-01 00:10:29.000 2020-01-02 03:38:50.000 629 99530 50079.5 5007950 629 99530 50079.5 5007950 -32541 32597 4859.66 485966 -128 123 0.78 78 -63 102 10053 99963 0.18918 300.18918 150.18918 15169.1081 0.18918 300.18918 150.18918 15169.10808 0.18918 300.18918 150.18917999999996 15169.10718 2020-01-01 2020-01-02 2020-01-01 00:01:03 2020-01-02 03:46:03 2020-01-01 00:01:03.000 2020-01-02 03:46:03.000 63 99963 50013 5051313 63 99963 50013 5051313 -32506 32429 4592.009900990099 463793 -126 125 -0.7821782178217822 -79 -630 100 10620 99531 1.89189 298.89189 150.39189 15039.18918 1.89189 298.8919 150.39189 15039.18937 1.89189 298.89189 150.39189000000005 15039.18900 2020-01-01 2020-01-02 2020-01-01 00:10:30 2020-01-02 03:38:51 2020-01-01 00:10:30.000 2020-01-02 03:38:51.000 630 99531 50080.5 5008050 630 99531 50080.5 5008050 -32540 32598 4860.66 486066 -127 124 1.78 178 -631 100 10621 99532 1.89489 298.89489 150.39489 15039.48948 1.89489 298.8949 150.39489 15039.48968 1.89489 298.89489 150.39488999999986 15039.48900 2020-01-01 2020-01-02 2020-01-01 00:10:31 2020-01-02 03:38:52 2020-01-01 00:10:31.000 2020-01-02 03:38:52.000 631 99532 50081.5 5008150 631 99532 50081.5 5008150 -32539 32599 4861.66 486166 -126 125 2.78 278 -632 100 10622 99533 1.89789 298.89789 150.39789 15039.78978 1.89789 298.8979 150.39789 15039.78984 1.89789 298.89789 150.3978900000001 15039.78900 2020-01-01 2020-01-02 2020-01-01 00:10:32 2020-01-02 03:38:53 2020-01-01 00:10:32.000 2020-01-02 03:38:53.000 632 99533 50082.5 5008250 632 99533 50082.5 5008250 -32538 32600 4862.66 486266 -125 126 3.78 378 -633 100 10623 99534 1.9009 298.9009 150.4009 15040.09009 1.9009 298.9009 150.40089 15040.08996 1.90090 298.90090 150.4009 15040.09000 2020-01-01 2020-01-02 2020-01-01 00:10:33 2020-01-02 03:38:54 2020-01-01 00:10:33.000 2020-01-02 03:38:54.000 633 99534 50083.5 5008350 633 99534 50083.5 5008350 -32537 32601 4863.66 486366 -124 127 4.78 478 -634 100 10624 99535 1.9039 298.9039 150.4039 15040.39039 1.9039 298.9039 150.4039 15040.39009 1.90390 298.90390 150.40389999999996 15040.39000 2020-01-01 2020-01-02 2020-01-01 00:10:34 2020-01-02 03:38:55 2020-01-01 00:10:34.000 2020-01-02 03:38:55.000 634 99535 50084.5 5008450 634 99535 50084.5 5008450 -32536 32602 4864.66 486466 -128 127 3.22 322 +626 100 10616 99527 1.87987 298.87987 150.37987 15037.98798 1.87987 298.87988 150.37988 15037.9882 1.87987 298.87987 150.37987000000012 15037.98700 2020-01-01 2020-01-02 2020-01-01 00:10:26 2020-01-02 03:38:47 2020-01-01 00:10:26.000 2020-01-02 03:38:47.000 626 99527 50076.5 5007650 626 99527 50076.5 5007650 -32544 32594 4856.66 485666 -125 126 2.9 290 +627 100 10617 99528 1.88288 298.88288 150.38288 15038.28828 1.88288 298.88287 150.38288 15038.28834 1.88288 298.88288 150.3828799999998 15038.28800 2020-01-01 2020-01-02 2020-01-01 00:10:27 2020-01-02 03:38:48 2020-01-01 00:10:27.000 2020-01-02 03:38:48.000 627 99528 50077.5 5007750 627 99528 50077.5 5007750 -32543 32595 4857.66 485766 -124 127 3.9 390 +628 100 10618 99529 1.88588 298.88588 150.38588 15038.58858 1.88588 298.8859 150.38588 15038.58849 1.88588 298.88588 150.38587999999984 15038.58800 2020-01-01 2020-01-02 2020-01-01 00:10:28 2020-01-02 03:38:49 2020-01-01 00:10:28.000 2020-01-02 03:38:49.000 628 99529 50078.5 5007850 628 99529 50078.5 5007850 -32542 32596 4858.66 485866 -128 127 2.34 234 +629 100 10619 99530 1.88888 298.88888 150.38888 15038.88888 1.88888 298.8889 150.38888 15038.88862 1.88888 298.88888 150.3888800000003 15038.88800 2020-01-01 2020-01-02 2020-01-01 00:10:29 2020-01-02 03:38:50 2020-01-01 00:10:29.000 2020-01-02 03:38:50.000 629 99530 50079.5 5007950 629 99530 50079.5 5007950 -32541 32597 4859.66 485966 -128 123 0.78 78 +63 102 10053 99963 0.18918 300.18918 150.18918 15169.1081 0.18918 300.18918 150.18918 15169.10808 0.18918 300.18918 150.1891799999999 15169.10718 2020-01-01 2020-01-02 2020-01-01 00:01:03 2020-01-02 03:46:03 2020-01-01 00:01:03.000 2020-01-02 03:46:03.000 63 99963 50013 5051313 63 99963 50013 5051313 -32506 32429 4592.009900990099 463793 -126 125 -0.7821782178217822 -79 +630 100 10620 99531 1.89189 298.89189 150.39189 15039.18918 1.89189 298.8919 150.39189 15039.18937 1.89189 298.89189 150.3918900000002 15039.18900 2020-01-01 2020-01-02 2020-01-01 00:10:30 2020-01-02 03:38:51 2020-01-01 00:10:30.000 2020-01-02 03:38:51.000 630 99531 50080.5 5008050 630 99531 50080.5 5008050 -32540 32598 4860.66 486066 -127 124 1.78 178 +631 100 10621 99532 1.89489 298.89489 150.39489 15039.48948 1.89489 298.8949 150.39489 15039.48968 1.89489 298.89489 150.3948899999997 15039.48900 2020-01-01 2020-01-02 2020-01-01 00:10:31 2020-01-02 03:38:52 2020-01-01 00:10:31.000 2020-01-02 03:38:52.000 631 99532 50081.5 5008150 631 99532 50081.5 5008150 -32539 32599 4861.66 486166 -126 125 2.78 278 +632 100 10622 99533 1.89789 298.89789 150.39789 15039.78978 1.89789 298.8979 150.39789 15039.78984 1.89789 298.89789 150.39789000000016 15039.78900 2020-01-01 2020-01-02 2020-01-01 00:10:32 2020-01-02 03:38:53 2020-01-01 00:10:32.000 2020-01-02 03:38:53.000 632 99533 50082.5 5008250 632 99533 50082.5 5008250 -32538 32600 4862.66 486266 -125 126 3.78 378 +633 100 10623 99534 1.9009 298.9009 150.4009 15040.09009 1.9009 298.9009 150.40089 15040.08996 1.90090 298.90090 150.40090000000004 15040.09000 2020-01-01 2020-01-02 2020-01-01 00:10:33 2020-01-02 03:38:54 2020-01-01 00:10:33.000 2020-01-02 03:38:54.000 633 99534 50083.5 5008350 633 99534 50083.5 5008350 -32537 32601 4863.66 486366 -124 127 4.78 478 +634 100 10624 99535 1.9039 298.9039 150.4039 15040.39039 1.9039 298.9039 150.4039 15040.39009 1.90390 298.90390 150.40389999999988 15040.39000 2020-01-01 2020-01-02 2020-01-01 00:10:34 2020-01-02 03:38:55 2020-01-01 00:10:34.000 2020-01-02 03:38:55.000 634 99535 50084.5 5008450 634 99535 50084.5 5008450 -32536 32602 4864.66 486466 -128 127 3.22 322 635 100 10625 99536 1.9069 298.9069 150.4069 15040.69069 1.9069 298.90692 150.4069 15040.69084 1.90690 298.90690 150.40689999999998 15040.69000 2020-01-01 2020-01-02 2020-01-01 00:10:35 2020-01-02 03:38:56 2020-01-01 00:10:35.000 2020-01-02 03:38:56.000 635 99536 50085.5 5008550 635 99536 50085.5 5008550 -32535 32603 4865.66 486566 -128 127 1.66 166 -636 100 10626 99537 1.9099 298.9099 150.4099 15040.99099 1.90991 298.9099 150.40991 15040.99115 1.90990 298.90990 150.40990000000002 15040.99000 2020-01-01 2020-01-02 2020-01-01 00:10:36 2020-01-02 03:38:57 2020-01-01 00:10:36.000 2020-01-02 03:38:57.000 636 99537 50086.5 5008650 636 99537 50086.5 5008650 -32534 32604 4866.66 486666 -128 124 0.1 10 -637 100 10627 99538 1.91291 298.91291 150.41291 15041.29129 1.91291 298.9129 150.41291 15041.29131 1.91291 298.91291 150.4129099999999 15041.29100 2020-01-01 2020-01-02 2020-01-01 00:10:37 2020-01-02 03:38:58 2020-01-01 00:10:37.000 2020-01-02 03:38:58.000 637 99538 50087.5 5008750 637 99538 50087.5 5008750 -32533 32605 4867.66 486766 -127 125 1.1 110 -638 100 10628 99539 1.91591 298.91591 150.41591 15041.59159 1.91591 298.91592 150.41591 15041.59143 1.91591 298.91591 150.41590999999994 15041.59100 2020-01-01 2020-01-02 2020-01-01 00:10:38 2020-01-02 03:38:59 2020-01-01 00:10:38.000 2020-01-02 03:38:59.000 638 99539 50088.5 5008850 638 99539 50088.5 5008850 -32532 32606 4868.66 486866 -126 126 2.1 210 -639 100 10629 99540 1.91891 298.91891 150.41891 15041.89189 1.91891 298.9189 150.41891 15041.89172 1.91891 298.91891 150.4189100000001 15041.89100 2020-01-01 2020-01-02 2020-01-01 00:10:39 2020-01-02 03:39:00 2020-01-01 00:10:39.000 2020-01-02 03:39:00.000 639 99540 50089.5 5008950 639 99540 50089.5 5008950 -32531 32607 4869.66 486966 -125 127 3.1 310 -64 102 10054 99964 0.19219 300.19219 150.19219 15169.41141 0.19219 300.1922 150.19219 15169.41184 0.19219 300.19219 150.19218999999993 15169.41119 2020-01-01 2020-01-02 2020-01-01 00:01:04 2020-01-02 03:46:04 2020-01-01 00:01:04.000 2020-01-02 03:46:04.000 64 99964 50014 5051414 64 99964 50014 5051414 -32505 32430 4593.009900990099 463894 -125 126 0.21782178217821782 22 -640 100 10630 99541 1.92192 298.92192 150.42192 15042.19219 1.92192 298.92194 150.42192 15042.19232 1.92192 298.92192 150.42192 15042.19200 2020-01-01 2020-01-02 2020-01-01 00:10:40 2020-01-02 03:39:01 2020-01-01 00:10:40.000 2020-01-02 03:39:01.000 640 99541 50090.5 5009050 640 99541 50090.5 5009050 -32530 32608 4870.66 487066 -128 127 1.54 154 -641 100 10631 99542 1.92492 298.92492 150.42492 15042.49249 1.92492 298.92493 150.42492 15042.49265 1.92492 298.92492 150.42492000000001 15042.49200 2020-01-01 2020-01-02 2020-01-01 00:10:41 2020-01-02 03:39:02 2020-01-01 00:10:41.000 2020-01-02 03:39:02.000 641 99542 50091.5 5009150 641 99542 50091.5 5009150 -32529 32609 4871.66 487166 -128 127 -0.02 -2 +636 100 10626 99537 1.9099 298.9099 150.4099 15040.99099 1.90991 298.9099 150.40991 15040.99115 1.90990 298.90990 150.40990000000005 15040.99000 2020-01-01 2020-01-02 2020-01-01 00:10:36 2020-01-02 03:38:57 2020-01-01 00:10:36.000 2020-01-02 03:38:57.000 636 99537 50086.5 5008650 636 99537 50086.5 5008650 -32534 32604 4866.66 486666 -128 124 0.1 10 +637 100 10627 99538 1.91291 298.91291 150.41291 15041.29129 1.91291 298.9129 150.41291 15041.29131 1.91291 298.91291 150.41290999999973 15041.29100 2020-01-01 2020-01-02 2020-01-01 00:10:37 2020-01-02 03:38:58 2020-01-01 00:10:37.000 2020-01-02 03:38:58.000 637 99538 50087.5 5008750 637 99538 50087.5 5008750 -32533 32605 4867.66 486766 -127 125 1.1 110 +638 100 10628 99539 1.91591 298.91591 150.41591 15041.59159 1.91591 298.91592 150.41591 15041.59143 1.91591 298.91591 150.41590999999983 15041.59100 2020-01-01 2020-01-02 2020-01-01 00:10:38 2020-01-02 03:38:59 2020-01-01 00:10:38.000 2020-01-02 03:38:59.000 638 99539 50088.5 5008850 638 99539 50088.5 5008850 -32532 32606 4868.66 486866 -126 126 2.1 210 +639 100 10629 99540 1.91891 298.91891 150.41891 15041.89189 1.91891 298.9189 150.41891 15041.89172 1.91891 298.91891 150.4189100000002 15041.89100 2020-01-01 2020-01-02 2020-01-01 00:10:39 2020-01-02 03:39:00 2020-01-01 00:10:39.000 2020-01-02 03:39:00.000 639 99540 50089.5 5008950 639 99540 50089.5 5008950 -32531 32607 4869.66 486966 -125 127 3.1 310 +64 102 10054 99964 0.19219 300.19219 150.19219 15169.41141 0.19219 300.1922 150.19219 15169.41184 0.19219 300.19219 150.19218999999984 15169.41119 2020-01-01 2020-01-02 2020-01-01 00:01:04 2020-01-02 03:46:04 2020-01-01 00:01:04.000 2020-01-02 03:46:04.000 64 99964 50014 5051414 64 99964 50014 5051414 -32505 32430 4593.009900990099 463894 -125 126 0.21782178217821782 22 +640 100 10630 99541 1.92192 298.92192 150.42192 15042.19219 1.92192 298.92194 150.42192 15042.19232 1.92192 298.92192 150.42192000000009 15042.19200 2020-01-01 2020-01-02 2020-01-01 00:10:40 2020-01-02 03:39:01 2020-01-01 00:10:40.000 2020-01-02 03:39:01.000 640 99541 50090.5 5009050 640 99541 50090.5 5009050 -32530 32608 4870.66 487066 -128 127 1.54 154 +641 100 10631 99542 1.92492 298.92492 150.42492 15042.49249 1.92492 298.92493 150.42492 15042.49265 1.92492 298.92492 150.42492 15042.49200 2020-01-01 2020-01-02 2020-01-01 00:10:41 2020-01-02 03:39:02 2020-01-01 00:10:41.000 2020-01-02 03:39:02.000 641 99542 50091.5 5009150 641 99542 50091.5 5009150 -32529 32609 4871.66 487166 -128 127 -0.02 -2 642 100 10632 99543 1.92792 298.92792 150.42792 15042.79279 1.92792 298.92792 150.42792 15042.79278 1.92792 298.92792 150.42792000000006 15042.79200 2020-01-01 2020-01-02 2020-01-01 00:10:42 2020-01-02 03:39:03 2020-01-01 00:10:42.000 2020-01-02 03:39:03.000 642 99543 50092.5 5009250 642 99543 50092.5 5009250 -32528 32610 4872.66 487266 -128 123 -1.58 -158 -643 100 10633 99544 1.93093 298.93093 150.43093 15043.09309 1.93093 298.93094 150.43092 15043.0929 1.93093 298.93093 150.43093000000016 15043.09300 2020-01-01 2020-01-02 2020-01-01 00:10:43 2020-01-02 03:39:04 2020-01-01 00:10:43.000 2020-01-02 03:39:04.000 643 99544 50093.5 5009350 643 99544 50093.5 5009350 -32527 32611 4873.66 487366 -127 124 -0.58 -58 -644 100 10634 99545 1.93393 298.93393 150.43393 15043.39339 1.93393 298.93393 150.43393 15043.39319 1.93393 298.93393 150.43392999999995 15043.39300 2020-01-01 2020-01-02 2020-01-01 00:10:44 2020-01-02 03:39:05 2020-01-01 00:10:44.000 2020-01-02 03:39:05.000 644 99545 50094.5 5009450 644 99545 50094.5 5009450 -32526 32612 4874.66 487466 -126 125 0.42 42 -645 100 10635 99546 1.93693 298.93693 150.43693 15043.69369 1.93693 298.93695 150.43693 15043.69379 1.93693 298.93693 150.43692999999993 15043.69300 2020-01-01 2020-01-02 2020-01-01 00:10:45 2020-01-02 03:39:06 2020-01-01 00:10:45.000 2020-01-02 03:39:06.000 645 99546 50095.5 5009550 645 99546 50095.5 5009550 -32525 32613 4875.66 487566 -125 126 1.42 142 -646 100 10636 99547 1.93993 298.93993 150.43993 15043.99399 1.93994 298.93994 150.43994 15043.99412 1.93993 298.93993 150.43993000000017 15043.99300 2020-01-01 2020-01-02 2020-01-01 00:10:46 2020-01-02 03:39:07 2020-01-01 00:10:46.000 2020-01-02 03:39:07.000 646 99547 50096.5 5009650 646 99547 50096.5 5009650 -32524 32614 4876.66 487666 -124 127 2.42 242 -647 100 10637 99548 1.94294 298.94294 150.44294 15044.29429 1.94294 298.94293 150.44294 15044.29425 1.94294 298.94294 150.44294000000005 15044.29400 2020-01-01 2020-01-02 2020-01-01 00:10:47 2020-01-02 03:39:08 2020-01-01 00:10:47.000 2020-01-02 03:39:08.000 647 99548 50097.5 5009750 647 99548 50097.5 5009750 -32523 32615 4877.66 487766 -128 127 0.86 86 -648 100 10638 99549 1.94594 298.94594 150.44594 15044.59459 1.94594 298.94595 150.44595 15044.595 1.94594 298.94594 150.4459399999999 15044.59400 2020-01-01 2020-01-02 2020-01-01 00:10:48 2020-01-02 03:39:09 2020-01-01 00:10:48.000 2020-01-02 03:39:09.000 648 99549 50098.5 5009850 648 99549 50098.5 5009850 -32522 32616 4878.66 487866 -128 123 -0.7 -70 -649 100 10639 99550 1.94894 298.94894 150.44894 15044.89489 1.94894 298.94894 150.44894 15044.89467 1.94894 298.94894 150.4489400000001 15044.89400 2020-01-01 2020-01-02 2020-01-01 00:10:49 2020-01-02 03:39:10 2020-01-01 00:10:49.000 2020-01-02 03:39:10.000 649 99550 50099.5 5009950 649 99550 50099.5 5009950 -32521 32617 4879.66 487966 -127 124 0.3 30 -65 102 10055 99965 0.19519 300.19519 150.19519 15169.71471 0.19519 300.1952 150.19519 15169.71448 0.19519 300.19519 150.19519000000008 15169.71419 2020-01-01 2020-01-02 2020-01-01 00:01:05 2020-01-02 03:46:05 2020-01-01 00:01:05.000 2020-01-02 03:46:05.000 65 99965 50015 5051515 65 99965 50015 5051515 -32504 32431 4594.009900990099 463995 -124 127 1.2178217821782178 123 -650 100 10640 99551 1.95195 298.95195 150.45195 15045.19519 1.95195 298.95197 150.45195 15045.19525 1.95195 298.95195 150.45195 15045.19500 2020-01-01 2020-01-02 2020-01-01 00:10:50 2020-01-02 03:39:11 2020-01-01 00:10:50.000 2020-01-02 03:39:11.000 650 99551 50100.5 5010050 650 99551 50100.5 5010050 -32520 32618 4880.66 488066 -126 125 1.3 130 -651 100 10641 99552 1.95495 298.95495 150.45495 15045.49549 1.95495 298.95496 150.45495 15045.49558 1.95495 298.95495 150.45494999999994 15045.49500 2020-01-01 2020-01-02 2020-01-01 00:10:51 2020-01-02 03:39:12 2020-01-01 00:10:51.000 2020-01-02 03:39:12.000 651 99552 50101.5 5010150 651 99552 50101.5 5010150 -32519 32619 4881.66 488166 -125 126 2.3 230 -652 100 10642 99553 1.95795 298.95795 150.45795 15045.79579 1.95795 298.95795 150.45795 15045.79572 1.95795 298.95795 150.45795 15045.79500 2020-01-01 2020-01-02 2020-01-01 00:10:52 2020-01-02 03:39:13 2020-01-01 00:10:52.000 2020-01-02 03:39:13.000 652 99553 50102.5 5010250 652 99553 50102.5 5010250 -32518 32620 4882.66 488266 -124 127 3.3 330 -653 100 10643 99554 1.96096 298.96096 150.46096 15046.09609 1.96096 298.96097 150.46096 15046.09647 1.96096 298.96096 150.46096000000006 15046.09600 2020-01-01 2020-01-02 2020-01-01 00:10:53 2020-01-02 03:39:14 2020-01-01 00:10:53.000 2020-01-02 03:39:14.000 653 99554 50103.5 5010350 653 99554 50103.5 5010350 -32517 32621 4883.66 488366 -128 127 1.74 174 -654 100 10644 99555 1.96396 298.96396 150.46396 15046.39639 1.96396 298.96396 150.46396 15046.39613 1.96396 298.96396 150.46395999999993 15046.39600 2020-01-01 2020-01-02 2020-01-01 00:10:54 2020-01-02 03:39:15 2020-01-01 00:10:54.000 2020-01-02 03:39:15.000 654 99555 50104.5 5010450 654 99555 50104.5 5010450 -32516 32622 4884.66 488466 -128 123 0.18 18 -655 100 10645 99556 1.96696 298.96696 150.46696 15046.69669 1.96696 298.96698 150.46696 15046.69676 1.96696 298.96696 150.46695999999991 15046.69600 2020-01-01 2020-01-02 2020-01-01 00:10:55 2020-01-02 03:39:16 2020-01-01 00:10:55.000 2020-01-02 03:39:16.000 655 99556 50105.5 5010550 655 99556 50105.5 5010550 -32515 32623 4885.66 488566 -127 124 1.18 118 -656 100 10646 99557 1.96996 298.96996 150.46996 15046.99699 1.96997 298.96997 150.46997 15046.99706 1.96996 298.96996 150.46996000000007 15046.99600 2020-01-01 2020-01-02 2020-01-01 00:10:56 2020-01-02 03:39:17 2020-01-01 00:10:56.000 2020-01-02 03:39:17.000 656 99557 50106.5 5010650 656 99557 50106.5 5010650 -32514 32624 4886.66 488666 -126 125 2.18 218 -657 100 10647 99558 1.97297 298.97297 150.47297 15047.29729 1.97297 298.97296 150.47297 15047.29735 1.97297 298.97297 150.47297000000003 15047.29700 2020-01-01 2020-01-02 2020-01-01 00:10:57 2020-01-02 03:39:18 2020-01-01 00:10:57.000 2020-01-02 03:39:18.000 657 99558 50107.5 5010750 657 99558 50107.5 5010750 -32513 32625 4887.66 488766 -125 126 3.18 318 +643 100 10633 99544 1.93093 298.93093 150.43093 15043.09309 1.93093 298.93094 150.43092 15043.0929 1.93093 298.93093 150.43093000000033 15043.09300 2020-01-01 2020-01-02 2020-01-01 00:10:43 2020-01-02 03:39:04 2020-01-01 00:10:43.000 2020-01-02 03:39:04.000 643 99544 50093.5 5009350 643 99544 50093.5 5009350 -32527 32611 4873.66 487366 -127 124 -0.58 -58 +644 100 10634 99545 1.93393 298.93393 150.43393 15043.39339 1.93393 298.93393 150.43393 15043.39319 1.93393 298.93393 150.43392999999983 15043.39300 2020-01-01 2020-01-02 2020-01-01 00:10:44 2020-01-02 03:39:05 2020-01-01 00:10:44.000 2020-01-02 03:39:05.000 644 99545 50094.5 5009450 644 99545 50094.5 5009450 -32526 32612 4874.66 487466 -126 125 0.42 42 +645 100 10635 99546 1.93693 298.93693 150.43693 15043.69369 1.93693 298.93695 150.43693 15043.69379 1.93693 298.93693 150.43692999999988 15043.69300 2020-01-01 2020-01-02 2020-01-01 00:10:45 2020-01-02 03:39:06 2020-01-01 00:10:45.000 2020-01-02 03:39:06.000 645 99546 50095.5 5009550 645 99546 50095.5 5009550 -32525 32613 4875.66 487566 -125 126 1.42 142 +646 100 10636 99547 1.93993 298.93993 150.43993 15043.99399 1.93994 298.93994 150.43994 15043.99412 1.93993 298.93993 150.43993000000023 15043.99300 2020-01-01 2020-01-02 2020-01-01 00:10:46 2020-01-02 03:39:07 2020-01-01 00:10:46.000 2020-01-02 03:39:07.000 646 99547 50096.5 5009650 646 99547 50096.5 5009650 -32524 32614 4876.66 487666 -124 127 2.42 242 +647 100 10637 99548 1.94294 298.94294 150.44294 15044.29429 1.94294 298.94293 150.44294 15044.29425 1.94294 298.94294 150.44294000000022 15044.29400 2020-01-01 2020-01-02 2020-01-01 00:10:47 2020-01-02 03:39:08 2020-01-01 00:10:47.000 2020-01-02 03:39:08.000 647 99548 50097.5 5009750 647 99548 50097.5 5009750 -32523 32615 4877.66 487766 -128 127 0.86 86 +648 100 10638 99549 1.94594 298.94594 150.44594 15044.59459 1.94594 298.94595 150.44595 15044.595 1.94594 298.94594 150.44593999999972 15044.59400 2020-01-01 2020-01-02 2020-01-01 00:10:48 2020-01-02 03:39:09 2020-01-01 00:10:48.000 2020-01-02 03:39:09.000 648 99549 50098.5 5009850 648 99549 50098.5 5009850 -32522 32616 4878.66 487866 -128 123 -0.7 -70 +649 100 10639 99550 1.94894 298.94894 150.44894 15044.89489 1.94894 298.94894 150.44894 15044.89467 1.94894 298.94894 150.4489400000002 15044.89400 2020-01-01 2020-01-02 2020-01-01 00:10:49 2020-01-02 03:39:10 2020-01-01 00:10:49.000 2020-01-02 03:39:10.000 649 99550 50099.5 5009950 649 99550 50099.5 5009950 -32521 32617 4879.66 487966 -127 124 0.3 30 +65 102 10055 99965 0.19519 300.19519 150.19519 15169.71471 0.19519 300.1952 150.19519 15169.71448 0.19519 300.19519 150.19519000000025 15169.71419 2020-01-01 2020-01-02 2020-01-01 00:01:05 2020-01-02 03:46:05 2020-01-01 00:01:05.000 2020-01-02 03:46:05.000 65 99965 50015 5051515 65 99965 50015 5051515 -32504 32431 4594.009900990099 463995 -124 127 1.2178217821782178 123 +650 100 10640 99551 1.95195 298.95195 150.45195 15045.19519 1.95195 298.95197 150.45195 15045.19525 1.95195 298.95195 150.45195000000007 15045.19500 2020-01-01 2020-01-02 2020-01-01 00:10:50 2020-01-02 03:39:11 2020-01-01 00:10:50.000 2020-01-02 03:39:11.000 650 99551 50100.5 5010050 650 99551 50100.5 5010050 -32520 32618 4880.66 488066 -126 125 1.3 130 +651 100 10641 99552 1.95495 298.95495 150.45495 15045.49549 1.95495 298.95496 150.45495 15045.49558 1.95495 298.95495 150.45494999999988 15045.49500 2020-01-01 2020-01-02 2020-01-01 00:10:51 2020-01-02 03:39:12 2020-01-01 00:10:51.000 2020-01-02 03:39:12.000 651 99552 50101.5 5010150 651 99552 50101.5 5010150 -32519 32619 4881.66 488166 -125 126 2.3 230 +652 100 10642 99553 1.95795 298.95795 150.45795 15045.79579 1.95795 298.95795 150.45795 15045.79572 1.95795 298.95795 150.45794999999998 15045.79500 2020-01-01 2020-01-02 2020-01-01 00:10:52 2020-01-02 03:39:13 2020-01-01 00:10:52.000 2020-01-02 03:39:13.000 652 99553 50102.5 5010250 652 99553 50102.5 5010250 -32518 32620 4882.66 488266 -124 127 3.3 330 +653 100 10643 99554 1.96096 298.96096 150.46096 15046.09609 1.96096 298.96097 150.46096 15046.09647 1.96096 298.96096 150.46096000000023 15046.09600 2020-01-01 2020-01-02 2020-01-01 00:10:53 2020-01-02 03:39:14 2020-01-01 00:10:53.000 2020-01-02 03:39:14.000 653 99554 50103.5 5010350 653 99554 50103.5 5010350 -32517 32621 4883.66 488366 -128 127 1.74 174 +654 100 10644 99555 1.96396 298.96396 150.46396 15046.39639 1.96396 298.96396 150.46396 15046.39613 1.96396 298.96396 150.46395999999973 15046.39600 2020-01-01 2020-01-02 2020-01-01 00:10:54 2020-01-02 03:39:15 2020-01-01 00:10:54.000 2020-01-02 03:39:15.000 654 99555 50104.5 5010450 654 99555 50104.5 5010450 -32516 32622 4884.66 488466 -128 123 0.18 18 +655 100 10645 99556 1.96696 298.96696 150.46696 15046.69669 1.96696 298.96698 150.46696 15046.69676 1.96696 298.96696 150.46695999999983 15046.69600 2020-01-01 2020-01-02 2020-01-01 00:10:55 2020-01-02 03:39:16 2020-01-01 00:10:55.000 2020-01-02 03:39:16.000 655 99556 50105.5 5010550 655 99556 50105.5 5010550 -32515 32623 4885.66 488566 -127 124 1.18 118 +656 100 10646 99557 1.96996 298.96996 150.46996 15046.99699 1.96997 298.96997 150.46997 15046.99706 1.96996 298.96996 150.46996000000024 15046.99600 2020-01-01 2020-01-02 2020-01-01 00:10:56 2020-01-02 03:39:17 2020-01-01 00:10:56.000 2020-01-02 03:39:17.000 656 99557 50106.5 5010650 656 99557 50106.5 5010650 -32514 32624 4886.66 488666 -126 125 2.18 218 +657 100 10647 99558 1.97297 298.97297 150.47297 15047.29729 1.97297 298.97296 150.47297 15047.29735 1.97297 298.97297 150.47297000000012 15047.29700 2020-01-01 2020-01-02 2020-01-01 00:10:57 2020-01-02 03:39:18 2020-01-01 00:10:57.000 2020-01-02 03:39:18.000 657 99558 50107.5 5010750 657 99558 50107.5 5010750 -32513 32625 4887.66 488766 -125 126 3.18 318 658 100 10648 99559 1.97597 298.97597 150.47597 15047.59759 1.97597 298.97598 150.47597 15047.59794 1.97597 298.97597 150.47597000000002 15047.59700 2020-01-01 2020-01-02 2020-01-01 00:10:58 2020-01-02 03:39:19 2020-01-01 00:10:58.000 2020-01-02 03:39:19.000 658 99559 50108.5 5010850 658 99559 50108.5 5010850 -32512 32626 4888.66 488866 -124 127 4.18 418 659 100 10649 99560 1.97897 298.97897 150.47897 15047.89789 1.97897 298.97897 150.47897 15047.8976 1.97897 298.97897 150.4789700000001 15047.89700 2020-01-01 2020-01-02 2020-01-01 00:10:59 2020-01-02 03:39:20 2020-01-01 00:10:59.000 2020-01-02 03:39:20.000 659 99560 50109.5 5010950 659 99560 50109.5 5010950 -32511 32627 4889.66 488966 -128 127 2.62 262 -66 102 10056 99966 0.19819 300.19819 150.19819 15170.01801 0.19819 300.1982 150.19819 15170.01808 0.19819 300.19819 150.19818999999993 15170.01719 2020-01-01 2020-01-02 2020-01-01 00:01:06 2020-01-02 03:46:06 2020-01-01 00:01:06.000 2020-01-02 03:46:06.000 66 99966 50016 5051616 66 99966 50016 5051616 -32503 32432 4595.009900990099 464096 -128 127 -0.31683168316831684 -32 +66 102 10056 99966 0.19819 300.19819 150.19819 15170.01801 0.19819 300.1982 150.19819 15170.01808 0.19819 300.19819 150.19818999999976 15170.01719 2020-01-01 2020-01-02 2020-01-01 00:01:06 2020-01-02 03:46:06 2020-01-01 00:01:06.000 2020-01-02 03:46:06.000 66 99966 50016 5051616 66 99966 50016 5051616 -32503 32432 4595.009900990099 464096 -128 127 -0.31683168316831684 -32 660 100 10650 99561 1.98198 298.98198 150.48198 15048.19819 1.98198 298.982 150.48198 15048.19822 1.98198 298.98198 150.48197999999996 15048.19800 2020-01-01 2020-01-02 2020-01-01 00:11:00 2020-01-02 03:39:21 2020-01-01 00:11:00.000 2020-01-02 03:39:21.000 660 99561 50110.5 5011050 660 99561 50110.5 5011050 -32510 32628 4890.66 489066 -128 127 1.06 106 -661 100 10651 99562 1.98498 298.98498 150.48498 15048.49849 1.98498 298.985 150.48498 15048.49853 1.98498 298.98498 150.48497999999998 15048.49800 2020-01-01 2020-01-02 2020-01-01 00:11:01 2020-01-02 03:39:22 2020-01-01 00:11:01.000 2020-01-02 03:39:22.000 661 99562 50111.5 5011150 661 99562 50111.5 5011150 -32509 32629 4891.66 489166 -128 124 -0.5 -50 +661 100 10651 99562 1.98498 298.98498 150.48498 15048.49849 1.98498 298.985 150.48498 15048.49853 1.98498 298.98498 150.48497999999987 15048.49800 2020-01-01 2020-01-02 2020-01-01 00:11:01 2020-01-02 03:39:22 2020-01-01 00:11:01.000 2020-01-02 03:39:22.000 661 99562 50111.5 5011150 661 99562 50111.5 5011150 -32509 32629 4891.66 489166 -128 124 -0.5 -50 662 100 10652 99563 1.98798 298.98798 150.48798 15048.79879 1.98798 298.98798 150.48798 15048.79882 1.98798 298.98798 150.4879799999999 15048.79800 2020-01-01 2020-01-02 2020-01-01 00:11:02 2020-01-02 03:39:23 2020-01-01 00:11:02.000 2020-01-02 03:39:23.000 662 99563 50112.5 5011250 662 99563 50112.5 5011250 -32508 32630 4892.66 489266 -127 125 0.5 50 -663 100 10653 99564 1.99099 298.99099 150.49099 15049.09909 1.99099 298.991 150.49099 15049.09942 1.99099 298.99099 150.4909900000001 15049.09900 2020-01-01 2020-01-02 2020-01-01 00:11:03 2020-01-02 03:39:24 2020-01-01 00:11:03.000 2020-01-02 03:39:24.000 663 99564 50113.5 5011350 663 99564 50113.5 5011350 -32507 32631 4893.66 489366 -126 126 1.5 150 -664 100 10654 99565 1.99399 298.99399 150.49399 15049.39939 1.99399 298.994 150.49399 15049.39911 1.99399 298.99399 150.49399000000005 15049.39900 2020-01-01 2020-01-02 2020-01-01 00:11:04 2020-01-02 03:39:25 2020-01-01 00:11:04.000 2020-01-02 03:39:25.000 664 99565 50114.5 5011450 664 99565 50114.5 5011450 -32506 32632 4894.66 489466 -125 127 2.5 250 -665 100 10655 99566 1.99699 298.99699 150.49699 15049.69969 1.99699 298.997 150.49699 15049.6997 1.99699 298.99699 150.49698999999993 15049.69900 2020-01-01 2020-01-02 2020-01-01 00:11:05 2020-01-02 03:39:26 2020-01-01 00:11:05.000 2020-01-02 03:39:26.000 665 99566 50115.5 5011550 665 99566 50115.5 5011550 -32505 32633 4895.66 489566 -128 127 0.94 94 +663 100 10653 99564 1.99099 298.99099 150.49099 15049.09909 1.99099 298.991 150.49099 15049.09942 1.99099 298.99099 150.49099000000015 15049.09900 2020-01-01 2020-01-02 2020-01-01 00:11:03 2020-01-02 03:39:24 2020-01-01 00:11:03.000 2020-01-02 03:39:24.000 663 99564 50113.5 5011350 663 99564 50113.5 5011350 -32507 32631 4893.66 489366 -126 126 1.5 150 +664 100 10654 99565 1.99399 298.99399 150.49399 15049.39939 1.99399 298.994 150.49399 15049.39911 1.99399 298.99399 150.49399000000025 15049.39900 2020-01-01 2020-01-02 2020-01-01 00:11:04 2020-01-02 03:39:25 2020-01-01 00:11:04.000 2020-01-02 03:39:25.000 664 99565 50114.5 5011450 664 99565 50114.5 5011450 -32506 32632 4894.66 489466 -125 127 2.5 250 +665 100 10655 99566 1.99699 298.99699 150.49699 15049.69969 1.99699 298.997 150.49699 15049.6997 1.99699 298.99699 150.49698999999976 15049.69900 2020-01-01 2020-01-02 2020-01-01 00:11:05 2020-01-02 03:39:26 2020-01-01 00:11:05.000 2020-01-02 03:39:26.000 665 99566 50115.5 5011550 665 99566 50115.5 5011550 -32505 32633 4895.66 489566 -128 127 0.94 94 666 100 10656 99567 2 299 150.5 15050 2 299 150.5 15050 2.00000 299.00000 150.5 15050.00000 2020-01-01 2020-01-02 2020-01-01 00:11:06 2020-01-02 03:39:27 2020-01-01 00:11:06.000 2020-01-02 03:39:27.000 666 99567 50116.5 5011650 666 99567 50116.5 5011650 -32504 32634 4896.66 489666 -128 127 -0.62 -62 -667 100 10657 99568 2.003 299.003 150.503 15050.3003 2.003 299.003 150.503 15050.30029 2.00300 299.00300 150.50300000000004 15050.30000 2020-01-01 2020-01-02 2020-01-01 00:11:07 2020-01-02 03:39:28 2020-01-01 00:11:07.000 2020-01-02 03:39:28.000 667 99568 50117.5 5011750 667 99568 50117.5 5011750 -32503 32635 4897.66 489766 -128 123 -2.18 -218 -668 100 10658 99569 2.006 299.006 150.506 15050.6006 2.006 299.006 150.506 15050.60089 2.00600 299.00600 150.50599999999997 15050.60000 2020-01-01 2020-01-02 2020-01-01 00:11:08 2020-01-02 03:39:29 2020-01-01 00:11:08.000 2020-01-02 03:39:29.000 668 99569 50118.5 5011850 668 99569 50118.5 5011850 -32502 32636 4898.66 489866 -127 124 -1.18 -118 -669 100 10659 99570 2.009 299.009 150.509 15050.9009 2.009 299.009 150.509 15050.90057 2.00900 299.00900 150.509 15050.90000 2020-01-01 2020-01-02 2020-01-01 00:11:09 2020-01-02 03:39:30 2020-01-01 00:11:09.000 2020-01-02 03:39:30.000 669 99570 50119.5 5011950 669 99570 50119.5 5011950 -32501 32637 4899.66 489966 -126 125 -0.18 -18 +667 100 10657 99568 2.003 299.003 150.503 15050.3003 2.003 299.003 150.503 15050.30029 2.00300 299.00300 150.5030000000001 15050.30000 2020-01-01 2020-01-02 2020-01-01 00:11:07 2020-01-02 03:39:28 2020-01-01 00:11:07.000 2020-01-02 03:39:28.000 667 99568 50117.5 5011750 667 99568 50117.5 5011750 -32503 32635 4897.66 489766 -128 123 -2.18 -218 +668 100 10658 99569 2.006 299.006 150.506 15050.6006 2.006 299.006 150.506 15050.60089 2.00600 299.00600 150.50599999999991 15050.60000 2020-01-01 2020-01-02 2020-01-01 00:11:08 2020-01-02 03:39:29 2020-01-01 00:11:08.000 2020-01-02 03:39:29.000 668 99569 50118.5 5011850 668 99569 50118.5 5011850 -32502 32636 4898.66 489866 -127 124 -1.18 -118 +669 100 10659 99570 2.009 299.009 150.509 15050.9009 2.009 299.009 150.509 15050.90057 2.00900 299.00900 150.50900000000001 15050.90000 2020-01-01 2020-01-02 2020-01-01 00:11:09 2020-01-02 03:39:30 2020-01-01 00:11:09.000 2020-01-02 03:39:30.000 669 99570 50119.5 5011950 669 99570 50119.5 5011950 -32501 32637 4899.66 489966 -126 125 -0.18 -18 67 102 10057 99967 0.2012 300.2012 150.2012 15170.32132 0.2012 300.2012 150.2012 15170.32142 0.20120 300.20120 150.20120000000003 15170.32120 2020-01-01 2020-01-02 2020-01-01 00:01:07 2020-01-02 03:46:07 2020-01-01 00:01:07.000 2020-01-02 03:46:07.000 67 99967 50017 5051717 67 99967 50017 5051717 -32502 32433 4596.009900990099 464197 -128 127 -1.8514851485148516 -187 -670 100 10660 99571 2.01201 299.01201 150.51201 15051.2012 2.01201 299.01202 150.51201 15051.20117 2.01201 299.01201 150.5120100000001 15051.20100 2020-01-01 2020-01-02 2020-01-01 00:11:10 2020-01-02 03:39:31 2020-01-01 00:11:10.000 2020-01-02 03:39:31.000 670 99571 50120.5 5012050 670 99571 50120.5 5012050 -32500 32638 4900.66 490066 -125 126 0.82 82 -671 100 10661 99572 2.01501 299.01501 150.51501 15051.5015 2.01501 299.015 150.51501 15051.50146 2.01501 299.01501 150.51500999999993 15051.50100 2020-01-01 2020-01-02 2020-01-01 00:11:11 2020-01-02 03:39:32 2020-01-01 00:11:11.000 2020-01-02 03:39:32.000 671 99572 50121.5 5012150 671 99572 50121.5 5012150 -32499 32639 4901.66 490166 -124 127 1.82 182 -672 100 10662 99573 2.01801 299.01801 150.51801 15051.8018 2.01801 299.018 150.51801 15051.80176 2.01801 299.01801 150.5180099999999 15051.80100 2020-01-01 2020-01-02 2020-01-01 00:11:12 2020-01-02 03:39:33 2020-01-01 00:11:12.000 2020-01-02 03:39:33.000 672 99573 50122.5 5012250 672 99573 50122.5 5012250 -32498 32640 4902.66 490266 -128 127 0.26 26 -673 100 10663 99574 2.02102 299.02102 150.52102 15052.1021 2.02102 299.02103 150.52102 15052.1024 2.02102 299.02102 150.52102000000005 15052.10200 2020-01-01 2020-01-02 2020-01-01 00:11:13 2020-01-02 03:39:34 2020-01-01 00:11:13.000 2020-01-02 03:39:34.000 673 99574 50123.5 5012350 673 99574 50123.5 5012350 -32497 32641 4903.66 490366 -128 123 -1.3 -130 -674 100 10664 99575 2.02402 299.02402 150.52402 15052.4024 2.02402 299.02402 150.52402 15052.40204 2.02402 299.02402 150.52402000000004 15052.40200 2020-01-01 2020-01-02 2020-01-01 00:11:14 2020-01-02 03:39:35 2020-01-01 00:11:14.000 2020-01-02 03:39:35.000 674 99575 50124.5 5012450 674 99575 50124.5 5012450 -32496 32642 4904.66 490466 -127 124 -0.3 -30 -675 100 10665 99576 2.02702 299.02702 150.52702 15052.7027 2.02702 299.02704 150.52702 15052.70264 2.02702 299.02702 150.52701999999982 15052.70200 2020-01-01 2020-01-02 2020-01-01 00:11:15 2020-01-02 03:39:36 2020-01-01 00:11:15.000 2020-01-02 03:39:36.000 675 99576 50125.5 5012550 675 99576 50125.5 5012550 -32495 32643 4905.66 490566 -126 125 0.7 70 +670 100 10660 99571 2.01201 299.01201 150.51201 15051.2012 2.01201 299.01202 150.51201 15051.20117 2.01201 299.01201 150.51201000000026 15051.20100 2020-01-01 2020-01-02 2020-01-01 00:11:10 2020-01-02 03:39:31 2020-01-01 00:11:10.000 2020-01-02 03:39:31.000 670 99571 50120.5 5012050 670 99571 50120.5 5012050 -32500 32638 4900.66 490066 -125 126 0.82 82 +671 100 10661 99572 2.01501 299.01501 150.51501 15051.5015 2.01501 299.015 150.51501 15051.50146 2.01501 299.01501 150.51500999999976 15051.50100 2020-01-01 2020-01-02 2020-01-01 00:11:11 2020-01-02 03:39:32 2020-01-01 00:11:11.000 2020-01-02 03:39:32.000 671 99572 50121.5 5012150 671 99572 50121.5 5012150 -32499 32639 4901.66 490166 -124 127 1.82 182 +672 100 10662 99573 2.01801 299.01801 150.51801 15051.8018 2.01801 299.018 150.51801 15051.80176 2.01801 299.01801 150.5180099999998 15051.80100 2020-01-01 2020-01-02 2020-01-01 00:11:12 2020-01-02 03:39:33 2020-01-01 00:11:12.000 2020-01-02 03:39:33.000 672 99573 50122.5 5012250 672 99573 50122.5 5012250 -32498 32640 4902.66 490266 -128 127 0.26 26 +673 100 10663 99574 2.02102 299.02102 150.52102 15052.1021 2.02102 299.02103 150.52102 15052.1024 2.02102 299.02102 150.5210200000001 15052.10200 2020-01-01 2020-01-02 2020-01-01 00:11:13 2020-01-02 03:39:34 2020-01-01 00:11:13.000 2020-01-02 03:39:34.000 673 99574 50123.5 5012350 673 99574 50123.5 5012350 -32497 32641 4903.66 490366 -128 123 -1.3 -130 +674 100 10664 99575 2.02402 299.02402 150.52402 15052.4024 2.02402 299.02402 150.52402 15052.40204 2.02402 299.02402 150.52402000000015 15052.40200 2020-01-01 2020-01-02 2020-01-01 00:11:14 2020-01-02 03:39:35 2020-01-01 00:11:14.000 2020-01-02 03:39:35.000 674 99575 50124.5 5012450 674 99575 50124.5 5012450 -32496 32642 4904.66 490466 -127 124 -0.3 -30 +675 100 10665 99576 2.02702 299.02702 150.52702 15052.7027 2.02702 299.02704 150.52702 15052.70264 2.02702 299.02702 150.52701999999977 15052.70200 2020-01-01 2020-01-02 2020-01-01 00:11:15 2020-01-02 03:39:36 2020-01-01 00:11:15.000 2020-01-02 03:39:36.000 675 99576 50125.5 5012550 675 99576 50125.5 5012550 -32495 32643 4905.66 490566 -126 125 0.7 70 676 100 10666 99577 2.03003 299.03003 150.53003 15053.003 2.03003 299.03003 150.53002 15053.00293 2.03003 299.03003 150.53002999999993 15053.00300 2020-01-01 2020-01-02 2020-01-01 00:11:16 2020-01-02 03:39:37 2020-01-01 00:11:16.000 2020-01-02 03:39:37.000 676 99577 50126.5 5012650 676 99577 50126.5 5012650 -32494 32644 4906.66 490666 -125 126 1.7 170 677 100 10667 99578 2.03303 299.03303 150.53303 15053.3033 2.03303 299.03302 150.53303 15053.30323 2.03303 299.03303 150.53303 15053.30300 2020-01-01 2020-01-02 2020-01-01 00:11:17 2020-01-02 03:39:38 2020-01-01 00:11:17.000 2020-01-02 03:39:38.000 677 99578 50127.5 5012750 677 99578 50127.5 5012750 -32493 32645 4907.66 490766 -124 127 2.7 270 -678 100 10668 99579 2.03603 299.03603 150.53603 15053.6036 2.03603 299.03604 150.53603 15053.60387 2.03603 299.03603 150.53602999999998 15053.60300 2020-01-01 2020-01-02 2020-01-01 00:11:18 2020-01-02 03:39:39 2020-01-01 00:11:18.000 2020-01-02 03:39:39.000 678 99579 50128.5 5012850 678 99579 50128.5 5012850 -32492 32646 4908.66 490866 -128 127 1.14 114 +678 100 10668 99579 2.03603 299.03603 150.53603 15053.6036 2.03603 299.03604 150.53603 15053.60387 2.03603 299.03603 150.5360299999999 15053.60300 2020-01-01 2020-01-02 2020-01-01 00:11:18 2020-01-02 03:39:39 2020-01-01 00:11:18.000 2020-01-02 03:39:39.000 678 99579 50128.5 5012850 678 99579 50128.5 5012850 -32492 32646 4908.66 490866 -128 127 1.14 114 679 100 10669 99580 2.03903 299.03903 150.53903 15053.9039 2.03903 299.03903 150.53903 15053.90351 2.03903 299.03903 150.53902999999994 15053.90300 2020-01-01 2020-01-02 2020-01-01 00:11:19 2020-01-02 03:39:40 2020-01-01 00:11:19.000 2020-01-02 03:39:40.000 679 99580 50129.5 5012950 679 99580 50129.5 5012950 -32491 32647 4909.66 490966 -128 123 -0.42 -42 -68 102 10058 99968 0.2042 300.2042 150.2042 15170.62462 0.2042 300.2042 150.2042 15170.62457 0.20420 300.20420 150.20420000000004 15170.62420 2020-01-01 2020-01-02 2020-01-01 00:01:08 2020-01-02 03:46:08 2020-01-01 00:01:08.000 2020-01-02 03:46:08.000 68 99968 50018 5051818 68 99968 50018 5051818 -32501 32434 4597.009900990099 464298 -128 124 -3.386138613861386 -342 -680 100 10670 99581 2.04204 299.04204 150.54204 15054.2042 2.04204 299.04205 150.54204 15054.20426 2.04204 299.04204 150.54204000000004 15054.20400 2020-01-01 2020-01-02 2020-01-01 00:11:20 2020-01-02 03:39:41 2020-01-01 00:11:20.000 2020-01-02 03:39:41.000 680 99581 50130.5 5013050 680 99581 50130.5 5013050 -32490 32648 4910.66 491066 -127 124 0.58 58 -681 100 10671 99582 2.04504 299.04504 150.54504 15054.5045 2.04504 299.04504 150.54504 15054.5044 2.04504 299.04504 150.54504000000009 15054.50400 2020-01-01 2020-01-02 2020-01-01 00:11:21 2020-01-02 03:39:42 2020-01-01 00:11:21.000 2020-01-02 03:39:42.000 681 99582 50131.5 5013150 681 99582 50131.5 5013150 -32489 32649 4911.66 491166 -126 125 1.58 158 -682 100 10672 99583 2.04804 299.04804 150.54804 15054.8048 2.04804 299.04803 150.54804 15054.80474 2.04804 299.04804 150.5480399999999 15054.80400 2020-01-01 2020-01-02 2020-01-01 00:11:22 2020-01-02 03:39:43 2020-01-01 00:11:22.000 2020-01-02 03:39:43.000 682 99583 50132.5 5013250 682 99583 50132.5 5013250 -32488 32650 4912.66 491266 -125 126 2.58 258 +68 102 10058 99968 0.2042 300.2042 150.2042 15170.62462 0.2042 300.2042 150.2042 15170.62457 0.20420 300.20420 150.2042000000001 15170.62420 2020-01-01 2020-01-02 2020-01-01 00:01:08 2020-01-02 03:46:08 2020-01-01 00:01:08.000 2020-01-02 03:46:08.000 68 99968 50018 5051818 68 99968 50018 5051818 -32501 32434 4597.009900990099 464298 -128 124 -3.386138613861386 -342 +680 100 10670 99581 2.04204 299.04204 150.54204 15054.2042 2.04204 299.04205 150.54204 15054.20426 2.04204 299.04204 150.54204000000016 15054.20400 2020-01-01 2020-01-02 2020-01-01 00:11:20 2020-01-02 03:39:41 2020-01-01 00:11:20.000 2020-01-02 03:39:41.000 680 99581 50130.5 5013050 680 99581 50130.5 5013050 -32490 32648 4910.66 491066 -127 124 0.58 58 +681 100 10671 99582 2.04504 299.04504 150.54504 15054.5045 2.04504 299.04504 150.54504 15054.5044 2.04504 299.04504 150.54504000000026 15054.50400 2020-01-01 2020-01-02 2020-01-01 00:11:21 2020-01-02 03:39:42 2020-01-01 00:11:21.000 2020-01-02 03:39:42.000 681 99582 50131.5 5013150 681 99582 50131.5 5013150 -32489 32649 4911.66 491166 -126 125 1.58 158 +682 100 10672 99583 2.04804 299.04804 150.54804 15054.8048 2.04804 299.04803 150.54804 15054.80474 2.04804 299.04804 150.5480399999998 15054.80400 2020-01-01 2020-01-02 2020-01-01 00:11:22 2020-01-02 03:39:43 2020-01-01 00:11:22.000 2020-01-02 03:39:43.000 682 99583 50132.5 5013250 682 99583 50132.5 5013250 -32488 32650 4912.66 491266 -125 126 2.58 258 683 100 10673 99584 2.05105 299.05105 150.55105 15055.1051 2.05105 299.05106 150.55105 15055.10533 2.05105 299.05105 150.55105 15055.10500 2020-01-01 2020-01-02 2020-01-01 00:11:23 2020-01-02 03:39:44 2020-01-01 00:11:23.000 2020-01-02 03:39:44.000 683 99584 50133.5 5013350 683 99584 50133.5 5013350 -32487 32651 4913.66 491366 -124 127 3.58 358 -684 100 10674 99585 2.05405 299.05405 150.55405 15055.4054 2.05405 299.05405 150.55404 15055.40498 2.05405 299.05405 150.55405000000007 15055.40500 2020-01-01 2020-01-02 2020-01-01 00:11:24 2020-01-02 03:39:45 2020-01-01 00:11:24.000 2020-01-02 03:39:45.000 684 99585 50134.5 5013450 684 99585 50134.5 5013450 -32486 32652 4914.66 491466 -128 127 2.02 202 -685 100 10675 99586 2.05705 299.05705 150.55705 15055.7057 2.05705 299.05707 150.55705 15055.70573 2.05705 299.05705 150.55704999999998 15055.70500 2020-01-01 2020-01-02 2020-01-01 00:11:25 2020-01-02 03:39:46 2020-01-01 00:11:25.000 2020-01-02 03:39:46.000 685 99586 50135.5 5013550 685 99586 50135.5 5013550 -32485 32653 4915.66 491566 -128 127 0.46 46 -686 100 10676 99587 2.06006 299.06006 150.56006 15056.006 2.06006 299.06006 150.56005 15056.00587 2.06006 299.06006 150.5600599999999 15056.00600 2020-01-01 2020-01-02 2020-01-01 00:11:26 2020-01-02 03:39:47 2020-01-01 00:11:26.000 2020-01-02 03:39:47.000 686 99587 50136.5 5013650 686 99587 50136.5 5013650 -32484 32654 4916.66 491666 -128 124 -1.1 -110 -687 100 10677 99588 2.06306 299.06306 150.56306 15056.3063 2.06306 299.06305 150.56306 15056.30621 2.06306 299.06306 150.56306000000012 15056.30600 2020-01-01 2020-01-02 2020-01-01 00:11:27 2020-01-02 03:39:48 2020-01-01 00:11:27.000 2020-01-02 03:39:48.000 687 99588 50137.5 5013750 687 99588 50137.5 5013750 -32483 32655 4917.66 491766 -127 125 -0.1 -10 -688 100 10678 99589 2.06606 299.06606 150.56606 15056.6066 2.06606 299.06607 150.56606 15056.60681 2.06606 299.06606 150.56605999999996 15056.60600 2020-01-01 2020-01-02 2020-01-01 00:11:28 2020-01-02 03:39:49 2020-01-01 00:11:28.000 2020-01-02 03:39:49.000 688 99589 50138.5 5013850 688 99589 50138.5 5013850 -32482 32656 4918.66 491866 -126 126 0.9 90 -689 100 10679 99590 2.06906 299.06906 150.56906 15056.9069 2.06906 299.06906 150.56907 15056.9071 2.06906 299.06906 150.5690599999999 15056.90600 2020-01-01 2020-01-02 2020-01-01 00:11:29 2020-01-02 03:39:50 2020-01-01 00:11:29.000 2020-01-02 03:39:50.000 689 99590 50139.5 5013950 689 99590 50139.5 5013950 -32481 32657 4919.66 491966 -125 127 1.9 190 -69 102 10059 99969 0.2072 300.2072 150.2072 15170.92792 0.2072 300.2072 150.20721 15170.92832 0.20720 300.20720 150.20720000000003 15170.92720 2020-01-01 2020-01-02 2020-01-01 00:01:09 2020-01-02 03:46:09 2020-01-01 00:01:09.000 2020-01-02 03:46:09.000 69 99969 50019 5051919 69 99969 50019 5051919 -32500 32435 4598.009900990099 464399 -127 125 -2.386138613861386 -241 -690 100 10680 99591 2.07207 299.07207 150.57207 15057.2072 2.07207 299.07208 150.57207 15057.2072 2.07207 299.07207 150.57207000000008 15057.20700 2020-01-01 2020-01-02 2020-01-01 00:11:30 2020-01-02 03:39:51 2020-01-01 00:11:30.000 2020-01-02 03:39:51.000 690 99591 50140.5 5014050 690 99591 50140.5 5014050 -32480 32658 4920.66 492066 -128 127 0.34 34 -691 100 10681 99592 2.07507 299.07507 150.57507 15057.5075 2.07507 299.07507 150.57507 15057.50734 2.07507 299.07507 150.57507000000004 15057.50700 2020-01-01 2020-01-02 2020-01-01 00:11:31 2020-01-02 03:39:52 2020-01-01 00:11:31.000 2020-01-02 03:39:52.000 691 99592 50141.5 5014150 691 99592 50141.5 5014150 -32479 32659 4921.66 492166 -128 127 -1.22 -122 -692 100 10682 99593 2.07807 299.07807 150.57807 15057.8078 2.07807 299.07806 150.57807 15057.80767 2.07807 299.07807 150.57806999999985 15057.80700 2020-01-01 2020-01-02 2020-01-01 00:11:32 2020-01-02 03:39:53 2020-01-01 00:11:32.000 2020-01-02 03:39:53.000 692 99593 50142.5 5014250 692 99593 50142.5 5014250 -32478 32660 4922.66 492266 -128 123 -2.78 -278 +684 100 10674 99585 2.05405 299.05405 150.55405 15055.4054 2.05405 299.05405 150.55404 15055.40498 2.05405 299.05405 150.55405000000013 15055.40500 2020-01-01 2020-01-02 2020-01-01 00:11:24 2020-01-02 03:39:45 2020-01-01 00:11:24.000 2020-01-02 03:39:45.000 684 99585 50134.5 5013450 684 99585 50134.5 5013450 -32486 32652 4914.66 491466 -128 127 2.02 202 +685 100 10675 99586 2.05705 299.05705 150.55705 15055.7057 2.05705 299.05707 150.55705 15055.70573 2.05705 299.05705 150.55704999999995 15055.70500 2020-01-01 2020-01-02 2020-01-01 00:11:25 2020-01-02 03:39:46 2020-01-01 00:11:25.000 2020-01-02 03:39:46.000 685 99586 50135.5 5013550 685 99586 50135.5 5013550 -32485 32653 4915.66 491566 -128 127 0.46 46 +686 100 10676 99587 2.06006 299.06006 150.56006 15056.006 2.06006 299.06006 150.56005 15056.00587 2.06006 299.06006 150.56005999999982 15056.00600 2020-01-01 2020-01-02 2020-01-01 00:11:26 2020-01-02 03:39:47 2020-01-01 00:11:26.000 2020-01-02 03:39:47.000 686 99587 50136.5 5013650 686 99587 50136.5 5013650 -32484 32654 4916.66 491666 -128 124 -1.1 -110 +687 100 10677 99588 2.06306 299.06306 150.56306 15056.3063 2.06306 299.06305 150.56306 15056.30621 2.06306 299.06306 150.5630600000003 15056.30600 2020-01-01 2020-01-02 2020-01-01 00:11:27 2020-01-02 03:39:48 2020-01-01 00:11:27.000 2020-01-02 03:39:48.000 687 99588 50137.5 5013750 687 99588 50137.5 5013750 -32483 32655 4917.66 491766 -127 125 -0.1 -10 +688 100 10678 99589 2.06606 299.06606 150.56606 15056.6066 2.06606 299.06607 150.56606 15056.60681 2.06606 299.06606 150.5660599999998 15056.60600 2020-01-01 2020-01-02 2020-01-01 00:11:28 2020-01-02 03:39:49 2020-01-01 00:11:28.000 2020-01-02 03:39:49.000 688 99589 50138.5 5013850 688 99589 50138.5 5013850 -32482 32656 4918.66 491866 -126 126 0.9 90 +689 100 10679 99590 2.06906 299.06906 150.56906 15056.9069 2.06906 299.06906 150.56907 15056.9071 2.06906 299.06906 150.56905999999984 15056.90600 2020-01-01 2020-01-02 2020-01-01 00:11:29 2020-01-02 03:39:50 2020-01-01 00:11:29.000 2020-01-02 03:39:50.000 689 99590 50139.5 5013950 689 99590 50139.5 5013950 -32481 32657 4919.66 491966 -125 127 1.9 190 +69 102 10059 99969 0.2072 300.2072 150.2072 15170.92792 0.2072 300.2072 150.20721 15170.92832 0.20720 300.20720 150.20720000000014 15170.92720 2020-01-01 2020-01-02 2020-01-01 00:01:09 2020-01-02 03:46:09 2020-01-01 00:01:09.000 2020-01-02 03:46:09.000 69 99969 50019 5051919 69 99969 50019 5051919 -32500 32435 4598.009900990099 464399 -127 125 -2.386138613861386 -241 +690 100 10680 99591 2.07207 299.07207 150.57207 15057.2072 2.07207 299.07208 150.57207 15057.2072 2.07207 299.07207 150.57207000000014 15057.20700 2020-01-01 2020-01-02 2020-01-01 00:11:30 2020-01-02 03:39:51 2020-01-01 00:11:30.000 2020-01-02 03:39:51.000 690 99591 50140.5 5014050 690 99591 50140.5 5014050 -32480 32658 4920.66 492066 -128 127 0.34 34 +691 100 10681 99592 2.07507 299.07507 150.57507 15057.5075 2.07507 299.07507 150.57507 15057.50734 2.07507 299.07507 150.57507000000018 15057.50700 2020-01-01 2020-01-02 2020-01-01 00:11:31 2020-01-02 03:39:52 2020-01-01 00:11:31.000 2020-01-02 03:39:52.000 691 99592 50141.5 5014150 691 99592 50141.5 5014150 -32479 32659 4921.66 492166 -128 127 -1.22 -122 +692 100 10682 99593 2.07807 299.07807 150.57807 15057.8078 2.07807 299.07806 150.57807 15057.80767 2.07807 299.07807 150.57806999999968 15057.80700 2020-01-01 2020-01-02 2020-01-01 00:11:32 2020-01-02 03:39:53 2020-01-01 00:11:32.000 2020-01-02 03:39:53.000 692 99593 50142.5 5014250 692 99593 50142.5 5014250 -32478 32660 4922.66 492266 -128 123 -2.78 -278 693 100 10683 99594 2.08108 299.08108 150.58108 15058.1081 2.08108 299.0811 150.58108 15058.10827 2.08108 299.08108 150.58107999999996 15058.10800 2020-01-01 2020-01-02 2020-01-01 00:11:33 2020-01-02 03:39:54 2020-01-01 00:11:33.000 2020-01-02 03:39:54.000 693 99594 50143.5 5014350 693 99594 50143.5 5014350 -32477 32661 4923.66 492366 -127 124 -1.78 -178 -694 100 10684 99595 2.08408 299.08408 150.58408 15058.4084 2.08408 299.08408 150.58408 15058.40857 2.08408 299.08408 150.58408 15058.40800 2020-01-01 2020-01-02 2020-01-01 00:11:34 2020-01-02 03:39:55 2020-01-01 00:11:34.000 2020-01-02 03:39:55.000 694 99595 50144.5 5014450 694 99595 50144.5 5014450 -32476 32662 4924.66 492466 -126 125 -0.78 -78 -695 100 10685 99596 2.08708 299.08708 150.58708 15058.7087 2.08708 299.0871 150.58708 15058.70867 2.08708 299.08708 150.58707999999987 15058.70800 2020-01-01 2020-01-02 2020-01-01 00:11:35 2020-01-02 03:39:56 2020-01-01 00:11:35.000 2020-01-02 03:39:56.000 695 99596 50145.5 5014550 695 99596 50145.5 5014550 -32475 32663 4925.66 492566 -125 126 0.22 22 -696 100 10686 99597 2.09009 299.09009 150.59009 15059.009 2.09009 299.0901 150.59008 15059.00885 2.09009 299.09009 150.5900899999998 15059.00900 2020-01-01 2020-01-02 2020-01-01 00:11:36 2020-01-02 03:39:57 2020-01-01 00:11:36.000 2020-01-02 03:39:57.000 696 99597 50146.5 5014650 696 99597 50146.5 5014650 -32474 32664 4926.66 492666 -124 127 1.22 122 -697 100 10687 99598 2.09309 299.09309 150.59309 15059.3093 2.09309 299.09308 150.59309 15059.30915 2.09309 299.09309 150.5930900000002 15059.30900 2020-01-01 2020-01-02 2020-01-01 00:11:37 2020-01-02 03:39:58 2020-01-01 00:11:37.000 2020-01-02 03:39:58.000 697 99598 50147.5 5014750 697 99598 50147.5 5014750 -32473 32665 4927.66 492766 -128 127 -0.34 -34 -698 100 10688 99599 2.09609 299.09609 150.59609 15059.6096 2.09609 299.0961 150.59609 15059.6099 2.09609 299.09609 150.59608999999972 15059.60900 2020-01-01 2020-01-02 2020-01-01 00:11:38 2020-01-02 03:39:59 2020-01-01 00:11:38.000 2020-01-02 03:39:59.000 698 99599 50148.5 5014850 698 99599 50148.5 5014850 -32472 32666 4928.66 492866 -128 123 -1.9 -190 -699 100 10689 99600 2.09909 299.09909 150.59909 15059.9099 2.09909 299.0991 150.5991 15059.91003 2.09909 299.09909 150.59908999999982 15059.90900 2020-01-01 2020-01-02 2020-01-01 00:11:39 2020-01-02 03:40:00 2020-01-01 00:11:39.000 2020-01-02 03:40:00.000 699 99600 50149.5 5014950 699 99600 50149.5 5014950 -32471 32667 4929.66 492966 -127 124 -0.9 -90 -7 102 1006 9997 0.02102 300.02102 150.02102 15152.12312 0.02102 300.02103 150.02102 15152.12342 0.02102 300.02102 150.0210200000001 15152.12302 2020-01-01 2020-01-02 2020-01-01 00:00:07 2020-01-02 03:45:07 2020-01-01 00:00:07.000 2020-01-02 03:45:07.000 7 99907 49957 5045657 7 99907 49957 5045657 -32562 32373 4536.009900990099 458137 -126 125 -1.0198019801980198 -103 -70 102 10060 99970 0.21021 300.21021 150.21021 15171.23123 0.21021 300.2102 150.2102 15171.23097 0.21021 300.21021 150.21020999999988 15171.23121 2020-01-01 2020-01-02 2020-01-01 00:01:10 2020-01-02 03:46:10 2020-01-01 00:01:10.000 2020-01-02 03:46:10.000 70 99970 50020 5052020 70 99970 50020 5052020 -32499 32436 4599.009900990099 464500 -126 126 -1.386138613861386 -140 +694 100 10684 99595 2.08408 299.08408 150.58408 15058.4084 2.08408 299.08408 150.58408 15058.40857 2.08408 299.08408 150.58407999999986 15058.40800 2020-01-01 2020-01-02 2020-01-01 00:11:34 2020-01-02 03:39:55 2020-01-01 00:11:34.000 2020-01-02 03:39:55.000 694 99595 50144.5 5014450 694 99595 50144.5 5014450 -32476 32662 4924.66 492466 -126 125 -0.78 -78 +695 100 10685 99596 2.08708 299.08708 150.58708 15058.7087 2.08708 299.0871 150.58708 15058.70867 2.08708 299.08708 150.58707999999996 15058.70800 2020-01-01 2020-01-02 2020-01-01 00:11:35 2020-01-02 03:39:56 2020-01-01 00:11:35.000 2020-01-02 03:39:56.000 695 99596 50145.5 5014550 695 99596 50145.5 5014550 -32475 32663 4925.66 492566 -125 126 0.22 22 +696 100 10686 99597 2.09009 299.09009 150.59009 15059.009 2.09009 299.0901 150.59008 15059.00885 2.09009 299.09009 150.59008999999992 15059.00900 2020-01-01 2020-01-02 2020-01-01 00:11:36 2020-01-02 03:39:57 2020-01-01 00:11:36.000 2020-01-02 03:39:57.000 696 99597 50146.5 5014650 696 99597 50146.5 5014650 -32474 32664 4926.66 492666 -124 127 1.22 122 +697 100 10687 99598 2.09309 299.09309 150.59309 15059.3093 2.09309 299.09308 150.59309 15059.30915 2.09309 299.09309 150.59309000000007 15059.30900 2020-01-01 2020-01-02 2020-01-01 00:11:37 2020-01-02 03:39:58 2020-01-01 00:11:37.000 2020-01-02 03:39:58.000 697 99598 50147.5 5014750 697 99598 50147.5 5014750 -32473 32665 4927.66 492766 -128 127 -0.34 -34 +698 100 10688 99599 2.09609 299.09609 150.59609 15059.6096 2.09609 299.0961 150.59609 15059.6099 2.09609 299.09609 150.59608999999992 15059.60900 2020-01-01 2020-01-02 2020-01-01 00:11:38 2020-01-02 03:39:59 2020-01-01 00:11:38.000 2020-01-02 03:39:59.000 698 99599 50148.5 5014850 698 99599 50148.5 5014850 -32472 32666 4928.66 492866 -128 123 -1.9 -190 +699 100 10689 99600 2.09909 299.09909 150.59909 15059.9099 2.09909 299.0991 150.5991 15059.91003 2.09909 299.09909 150.59908999999993 15059.90900 2020-01-01 2020-01-02 2020-01-01 00:11:39 2020-01-02 03:40:00 2020-01-01 00:11:39.000 2020-01-02 03:40:00.000 699 99600 50149.5 5014950 699 99600 50149.5 5014950 -32471 32667 4929.66 492966 -127 124 -0.9 -90 +7 102 1006 9997 0.02102 300.02102 150.02102 15152.12312 0.02102 300.02103 150.02102 15152.12342 0.02102 300.02102 150.02102000000005 15152.12302 2020-01-01 2020-01-02 2020-01-01 00:00:07 2020-01-02 03:45:07 2020-01-01 00:00:07.000 2020-01-02 03:45:07.000 7 99907 49957 5045657 7 99907 49957 5045657 -32562 32373 4536.009900990099 458137 -126 125 -1.0198019801980198 -103 +70 102 10060 99970 0.21021 300.21021 150.21021 15171.23123 0.21021 300.2102 150.2102 15171.23097 0.21021 300.21021 150.21021 15171.23121 2020-01-01 2020-01-02 2020-01-01 00:01:10 2020-01-02 03:46:10 2020-01-01 00:01:10.000 2020-01-02 03:46:10.000 70 99970 50020 5052020 70 99970 50020 5052020 -32499 32436 4599.009900990099 464500 -126 126 -1.386138613861386 -140 700 100 10690 99601 2.1021 299.1021 150.6021 15060.21021 2.1021 299.1021 150.6021 15060.21014 2.10210 299.10210 150.60210000000004 15060.21000 2020-01-01 2020-01-02 2020-01-01 00:11:40 2020-01-02 03:40:01 2020-01-01 00:11:40.000 2020-01-02 03:40:01.000 700 99601 50150.5 5015050 700 99601 50150.5 5015050 -32470 32668 4930.66 493066 -126 125 0.1 10 -701 100 10691 99602 2.1051 299.1051 150.6051 15060.51051 2.1051 299.1051 150.6051 15060.51031 2.10510 299.10510 150.60510000000014 15060.51000 2020-01-01 2020-01-02 2020-01-01 00:11:41 2020-01-02 03:40:02 2020-01-01 00:11:41.000 2020-01-02 03:40:02.000 701 99602 50151.5 5015150 701 99602 50151.5 5015150 -32469 32669 4931.66 493166 -125 126 1.1 110 -702 100 10692 99603 2.1081 299.1081 150.6081 15060.81081 2.1081 299.1081 150.6081 15060.81062 2.10810 299.10810 150.60809999999998 15060.81000 2020-01-01 2020-01-02 2020-01-01 00:11:42 2020-01-02 03:40:03 2020-01-01 00:11:42.000 2020-01-02 03:40:03.000 702 99603 50152.5 5015250 702 99603 50152.5 5015250 -32468 32670 4932.66 493266 -124 127 2.1 210 -703 100 10693 99604 2.11111 299.11111 150.61111 15061.11111 2.11111 299.1111 150.61111 15061.11137 2.11111 299.11111 150.61110999999985 15061.11100 2020-01-01 2020-01-02 2020-01-01 00:11:43 2020-01-02 03:40:04 2020-01-01 00:11:43.000 2020-01-02 03:40:04.000 703 99604 50153.5 5015350 703 99604 50153.5 5015350 -32467 32671 4933.66 493366 -128 127 0.54 54 -704 100 10694 99605 2.11411 299.11411 150.61411 15061.41141 2.11411 299.1141 150.61411 15061.41151 2.11411 299.11411 150.61411000000032 15061.41100 2020-01-01 2020-01-02 2020-01-01 00:11:44 2020-01-02 03:40:05 2020-01-01 00:11:44.000 2020-01-02 03:40:05.000 704 99605 50154.5 5015450 704 99605 50154.5 5015450 -32466 32672 4934.66 493466 -128 123 -1.02 -102 -705 100 10695 99606 2.11711 299.11711 150.61711 15061.71171 2.11711 299.11713 150.61711 15061.71165 2.11711 299.11711 150.61710999999983 15061.71100 2020-01-01 2020-01-02 2020-01-01 00:11:45 2020-01-02 03:40:06 2020-01-01 00:11:45.000 2020-01-02 03:40:06.000 705 99606 50155.5 5015550 705 99606 50155.5 5015550 -32465 32673 4935.66 493566 -127 124 -0.02 -2 -706 100 10696 99607 2.12012 299.12012 150.62012 15062.01201 2.12012 299.12012 150.62011 15062.01179 2.12012 299.12012 150.6201199999997 15062.01200 2020-01-01 2020-01-02 2020-01-01 00:11:46 2020-01-02 03:40:07 2020-01-01 00:11:46.000 2020-01-02 03:40:07.000 706 99607 50156.5 5015650 706 99607 50156.5 5015650 -32464 32674 4936.66 493666 -126 125 0.98 98 -707 100 10697 99608 2.12312 299.12312 150.62312 15062.31231 2.12312 299.1231 150.62312 15062.31208 2.12312 299.12312 150.62312000000017 15062.31200 2020-01-01 2020-01-02 2020-01-01 00:11:47 2020-01-02 03:40:08 2020-01-01 00:11:47.000 2020-01-02 03:40:08.000 707 99608 50157.5 5015750 707 99608 50157.5 5015750 -32463 32675 4937.66 493766 -125 126 1.98 198 -708 100 10698 99609 2.12612 299.12612 150.62612 15062.61261 2.12612 299.12613 150.62612 15062.61283 2.12612 299.12612 150.6261200000002 15062.61200 2020-01-01 2020-01-02 2020-01-01 00:11:48 2020-01-02 03:40:09 2020-01-01 00:11:48.000 2020-01-02 03:40:09.000 708 99609 50158.5 5015850 708 99609 50158.5 5015850 -32462 32676 4938.66 493866 -124 127 2.98 298 -709 100 10699 99610 2.12912 299.12912 150.62912 15062.91291 2.12912 299.12912 150.62912 15062.91298 2.12912 299.12912 150.62911999999972 15062.91200 2020-01-01 2020-01-02 2020-01-01 00:11:49 2020-01-02 03:40:10 2020-01-01 00:11:49.000 2020-01-02 03:40:10.000 709 99610 50159.5 5015950 709 99610 50159.5 5015950 -32461 32677 4939.66 493966 -128 127 1.42 142 +701 100 10691 99602 2.1051 299.1051 150.6051 15060.51051 2.1051 299.1051 150.6051 15060.51031 2.10510 299.10510 150.6051 15060.51000 2020-01-01 2020-01-02 2020-01-01 00:11:41 2020-01-02 03:40:02 2020-01-01 00:11:41.000 2020-01-02 03:40:02.000 701 99602 50151.5 5015150 701 99602 50151.5 5015150 -32469 32669 4931.66 493166 -125 126 1.1 110 +702 100 10692 99603 2.1081 299.1081 150.6081 15060.81081 2.1081 299.1081 150.6081 15060.81062 2.10810 299.10810 150.6081 15060.81000 2020-01-01 2020-01-02 2020-01-01 00:11:42 2020-01-02 03:40:03 2020-01-01 00:11:42.000 2020-01-02 03:40:03.000 702 99603 50152.5 5015250 702 99603 50152.5 5015250 -32468 32670 4932.66 493266 -124 127 2.1 210 +703 100 10693 99604 2.11111 299.11111 150.61111 15061.11111 2.11111 299.1111 150.61111 15061.11137 2.11111 299.11111 150.6111099999999 15061.11100 2020-01-01 2020-01-02 2020-01-01 00:11:43 2020-01-02 03:40:04 2020-01-01 00:11:43.000 2020-01-02 03:40:04.000 703 99604 50153.5 5015350 703 99604 50153.5 5015350 -32467 32671 4933.66 493366 -128 127 0.54 54 +704 100 10694 99605 2.11411 299.11411 150.61411 15061.41141 2.11411 299.1141 150.61411 15061.41151 2.11411 299.11411 150.61411000000015 15061.41100 2020-01-01 2020-01-02 2020-01-01 00:11:44 2020-01-02 03:40:05 2020-01-01 00:11:44.000 2020-01-02 03:40:05.000 704 99605 50154.5 5015450 704 99605 50154.5 5015450 -32466 32672 4934.66 493466 -128 123 -1.02 -102 +705 100 10695 99606 2.11711 299.11711 150.61711 15061.71171 2.11711 299.11713 150.61711 15061.71165 2.11711 299.11711 150.61710999999997 15061.71100 2020-01-01 2020-01-02 2020-01-01 00:11:45 2020-01-02 03:40:06 2020-01-01 00:11:45.000 2020-01-02 03:40:06.000 705 99606 50155.5 5015550 705 99606 50155.5 5015550 -32465 32673 4935.66 493566 -127 124 -0.02 -2 +706 100 10696 99607 2.12012 299.12012 150.62012 15062.01201 2.12012 299.12012 150.62011 15062.01179 2.12012 299.12012 150.62011999999987 15062.01200 2020-01-01 2020-01-02 2020-01-01 00:11:46 2020-01-02 03:40:07 2020-01-01 00:11:46.000 2020-01-02 03:40:07.000 706 99607 50156.5 5015650 706 99607 50156.5 5015650 -32464 32674 4936.66 493666 -126 125 0.98 98 +707 100 10697 99608 2.12312 299.12312 150.62312 15062.31231 2.12312 299.1231 150.62312 15062.31208 2.12312 299.12312 150.62312000000009 15062.31200 2020-01-01 2020-01-02 2020-01-01 00:11:47 2020-01-02 03:40:08 2020-01-01 00:11:47.000 2020-01-02 03:40:08.000 707 99608 50157.5 5015750 707 99608 50157.5 5015750 -32463 32675 4937.66 493766 -125 126 1.98 198 +708 100 10698 99609 2.12612 299.12612 150.62612 15062.61261 2.12612 299.12613 150.62612 15062.61283 2.12612 299.12612 150.62612000000004 15062.61200 2020-01-01 2020-01-02 2020-01-01 00:11:48 2020-01-02 03:40:09 2020-01-01 00:11:48.000 2020-01-02 03:40:09.000 708 99609 50158.5 5015850 708 99609 50158.5 5015850 -32462 32676 4938.66 493866 -124 127 2.98 298 +709 100 10699 99610 2.12912 299.12912 150.62912 15062.91291 2.12912 299.12912 150.62912 15062.91298 2.12912 299.12912 150.6291199999999 15062.91200 2020-01-01 2020-01-02 2020-01-01 00:11:49 2020-01-02 03:40:10 2020-01-01 00:11:49.000 2020-01-02 03:40:10.000 709 99610 50159.5 5015950 709 99610 50159.5 5015950 -32461 32677 4939.66 493966 -128 127 1.42 142 71 102 10061 99971 0.21321 300.21321 150.21321 15171.53453 0.21321 300.21323 150.21321 15171.5346 0.21321 300.21321 150.21320999999992 15171.53421 2020-01-01 2020-01-02 2020-01-01 00:01:11 2020-01-02 03:46:11 2020-01-01 00:01:11.000 2020-01-02 03:46:11.000 71 99971 50021 5052121 71 99971 50021 5052121 -32498 32437 4600.009900990099 464601 -125 127 -0.38613861386138615 -39 710 100 10700 99611 2.13213 299.13213 150.63213 15063.21321 2.13213 299.13214 150.63213 15063.21311 2.13213 299.13213 150.63213 15063.21300 2020-01-01 2020-01-02 2020-01-01 00:11:50 2020-01-02 03:40:11 2020-01-01 00:11:50.000 2020-01-02 03:40:11.000 710 99611 50160.5 5016050 710 99611 50160.5 5016050 -32460 32678 4940.66 494066 -128 127 -0.14 -14 -711 100 10701 99612 2.13513 299.13513 150.63513 15063.51351 2.13513 299.13513 150.63513 15063.51325 2.13513 299.13513 150.63513000000006 15063.51300 2020-01-01 2020-01-02 2020-01-01 00:11:51 2020-01-02 03:40:12 2020-01-01 00:11:51.000 2020-01-02 03:40:12.000 711 99612 50161.5 5016150 711 99612 50161.5 5016150 -32459 32679 4941.66 494166 -128 124 -1.7 -170 -712 100 10702 99613 2.13813 299.13813 150.63813 15063.81381 2.13813 299.13815 150.63814 15063.81401 2.13813 299.13813 150.63812999999988 15063.81300 2020-01-01 2020-01-02 2020-01-01 00:11:52 2020-01-02 03:40:13 2020-01-01 00:11:52.000 2020-01-02 03:40:13.000 712 99613 50162.5 5016250 712 99613 50162.5 5016250 -32458 32680 4942.66 494266 -127 125 -0.7 -70 -713 100 10703 99614 2.14114 299.14114 150.64114 15064.11411 2.14114 299.14114 150.64114 15064.11431 2.14114 299.14114 150.64113999999984 15064.11400 2020-01-01 2020-01-02 2020-01-01 00:11:53 2020-01-02 03:40:14 2020-01-01 00:11:53.000 2020-01-02 03:40:14.000 713 99614 50163.5 5016350 713 99614 50163.5 5016350 -32457 32681 4943.66 494366 -126 126 0.3 30 -714 100 10704 99615 2.14414 299.14414 150.64414 15064.41441 2.14414 299.14413 150.64414 15064.41448 2.14414 299.14414 150.64414000000022 15064.41400 2020-01-01 2020-01-02 2020-01-01 00:11:54 2020-01-02 03:40:15 2020-01-01 00:11:54.000 2020-01-02 03:40:15.000 714 99615 50164.5 5016450 714 99615 50164.5 5016450 -32456 32682 4944.66 494466 -125 127 1.3 130 -715 100 10705 99616 2.14714 299.14714 150.64714 15064.71471 2.14714 299.14716 150.64714 15064.71458 2.14714 299.14714 150.64713999999972 15064.71400 2020-01-01 2020-01-02 2020-01-01 00:11:55 2020-01-02 03:40:16 2020-01-01 00:11:55.000 2020-01-02 03:40:16.000 715 99616 50165.5 5016550 715 99616 50165.5 5016550 -32455 32683 4945.66 494566 -128 127 -0.26 -26 +711 100 10701 99612 2.13513 299.13513 150.63513 15063.51351 2.13513 299.13513 150.63513 15063.51325 2.13513 299.13513 150.63513000000003 15063.51300 2020-01-01 2020-01-02 2020-01-01 00:11:51 2020-01-02 03:40:12 2020-01-01 00:11:51.000 2020-01-02 03:40:12.000 711 99612 50161.5 5016150 711 99612 50161.5 5016150 -32459 32679 4941.66 494166 -128 124 -1.7 -170 +712 100 10702 99613 2.13813 299.13813 150.63813 15063.81381 2.13813 299.13815 150.63814 15063.81401 2.13813 299.13813 150.63812999999993 15063.81300 2020-01-01 2020-01-02 2020-01-01 00:11:52 2020-01-02 03:40:13 2020-01-01 00:11:52.000 2020-01-02 03:40:13.000 712 99613 50162.5 5016250 712 99613 50162.5 5016250 -32458 32680 4942.66 494266 -127 125 -0.7 -70 +713 100 10703 99614 2.14114 299.14114 150.64114 15064.11411 2.14114 299.14114 150.64114 15064.11431 2.14114 299.14114 150.64113999999995 15064.11400 2020-01-01 2020-01-02 2020-01-01 00:11:53 2020-01-02 03:40:14 2020-01-01 00:11:53.000 2020-01-02 03:40:14.000 713 99614 50163.5 5016350 713 99614 50163.5 5016350 -32457 32681 4943.66 494366 -126 126 0.3 30 +714 100 10704 99615 2.14414 299.14414 150.64414 15064.41441 2.14414 299.14413 150.64414 15064.41448 2.14414 299.14414 150.64414000000005 15064.41400 2020-01-01 2020-01-02 2020-01-01 00:11:54 2020-01-02 03:40:15 2020-01-01 00:11:54.000 2020-01-02 03:40:15.000 714 99615 50164.5 5016450 714 99615 50164.5 5016450 -32456 32682 4944.66 494466 -125 127 1.3 130 +715 100 10705 99616 2.14714 299.14714 150.64714 15064.71471 2.14714 299.14716 150.64714 15064.71458 2.14714 299.14714 150.64713999999992 15064.71400 2020-01-01 2020-01-02 2020-01-01 00:11:55 2020-01-02 03:40:16 2020-01-01 00:11:55.000 2020-01-02 03:40:16.000 715 99616 50165.5 5016550 715 99616 50165.5 5016550 -32455 32683 4945.66 494566 -128 127 -0.26 -26 716 100 10706 99617 2.15015 299.15015 150.65015 15065.01501 2.15015 299.15015 150.65014 15065.01472 2.15015 299.15015 150.65015 15065.01500 2020-01-01 2020-01-02 2020-01-01 00:11:56 2020-01-02 03:40:17 2020-01-01 00:11:56.000 2020-01-02 03:40:17.000 716 99617 50166.5 5016650 716 99617 50166.5 5016650 -32454 32684 4946.66 494666 -128 127 -1.82 -182 717 100 10707 99618 2.15315 299.15315 150.65315 15065.31531 2.15315 299.15317 150.65315 15065.31547 2.15315 299.15315 150.65315000000007 15065.31500 2020-01-01 2020-01-02 2020-01-01 00:11:57 2020-01-02 03:40:18 2020-01-01 00:11:57.000 2020-01-02 03:40:18.000 717 99618 50167.5 5016750 717 99618 50167.5 5016750 -32453 32685 4947.66 494766 -128 123 -3.38 -338 -718 100 10708 99619 2.15615 299.15615 150.65615 15065.61561 2.15615 299.15616 150.65615 15065.61578 2.15615 299.15615 150.6561500000001 15065.61500 2020-01-01 2020-01-02 2020-01-01 00:11:58 2020-01-02 03:40:19 2020-01-01 00:11:58.000 2020-01-02 03:40:19.000 718 99619 50168.5 5016850 718 99619 50168.5 5016850 -32452 32686 4948.66 494866 -127 124 -2.38 -238 +718 100 10708 99619 2.15615 299.15615 150.65615 15065.61561 2.15615 299.15616 150.65615 15065.61578 2.15615 299.15615 150.65615000000003 15065.61500 2020-01-01 2020-01-02 2020-01-01 00:11:58 2020-01-02 03:40:19 2020-01-01 00:11:58.000 2020-01-02 03:40:19.000 718 99619 50168.5 5016850 718 99619 50168.5 5016850 -32452 32686 4948.66 494866 -127 124 -2.38 -238 719 100 10709 99620 2.15915 299.15915 150.65915 15065.91591 2.15915 299.15915 150.65915 15065.91595 2.15915 299.15915 150.65915 15065.91500 2020-01-01 2020-01-02 2020-01-01 00:11:59 2020-01-02 03:40:20 2020-01-01 00:11:59.000 2020-01-02 03:40:20.000 719 99620 50169.5 5016950 719 99620 50169.5 5016950 -32451 32687 4949.66 494966 -126 125 -1.38 -138 72 102 10062 99972 0.21621 300.21621 150.21621 15171.83783 0.21621 300.21622 150.21621 15171.83791 0.21621 300.21621 150.21621 15171.83721 2020-01-01 2020-01-02 2020-01-01 00:01:12 2020-01-02 03:46:12 2020-01-01 00:01:12.000 2020-01-02 03:46:12.000 72 99972 50022 5052222 72 99972 50022 5052222 -32497 32438 4601.009900990099 464702 -128 127 -1.9207920792079207 -194 -720 100 10710 99621 2.16216 299.16216 150.66216 15066.21621 2.16216 299.16217 150.66216 15066.21606 2.16216 299.16216 150.6621599999999 15066.21600 2020-01-01 2020-01-02 2020-01-01 00:12:00 2020-01-02 03:40:21 2020-01-01 00:12:00.000 2020-01-02 03:40:21.000 720 99621 50170.5 5017050 720 99621 50170.5 5017050 -32450 32688 4950.66 495066 -125 126 -0.38 -38 +720 100 10710 99621 2.16216 299.16216 150.66216 15066.21621 2.16216 299.16217 150.66216 15066.21606 2.16216 299.16216 150.66215999999994 15066.21600 2020-01-01 2020-01-02 2020-01-01 00:12:00 2020-01-02 03:40:21 2020-01-01 00:12:00.000 2020-01-02 03:40:21.000 720 99621 50170.5 5017050 720 99621 50170.5 5017050 -32450 32688 4950.66 495066 -125 126 -0.38 -38 721 100 10711 99622 2.16516 299.16516 150.66516 15066.51651 2.16516 299.16516 150.66516 15066.51635 2.16516 299.16516 150.66515999999996 15066.51600 2020-01-01 2020-01-02 2020-01-01 00:12:01 2020-01-02 03:40:22 2020-01-01 00:12:01.000 2020-01-02 03:40:22.000 721 99622 50171.5 5017150 721 99622 50171.5 5017150 -32449 32689 4951.66 495166 -124 127 0.62 62 -722 100 10712 99623 2.16816 299.16816 150.66816 15066.81681 2.16816 299.16818 150.66816 15066.81695 2.16816 299.16816 150.66815999999986 15066.81600 2020-01-01 2020-01-02 2020-01-01 00:12:02 2020-01-02 03:40:23 2020-01-01 00:12:02.000 2020-01-02 03:40:23.000 722 99623 50172.5 5017250 722 99623 50172.5 5017250 -32448 32690 4952.66 495266 -128 127 -0.94 -94 -723 100 10713 99624 2.17117 299.17117 150.67117 15067.11711 2.17117 299.17117 150.67117 15067.11724 2.17117 299.17117 150.67116999999973 15067.11700 2020-01-01 2020-01-02 2020-01-01 00:12:03 2020-01-02 03:40:24 2020-01-01 00:12:03.000 2020-01-02 03:40:24.000 723 99624 50173.5 5017350 723 99624 50173.5 5017350 -32447 32691 4953.66 495366 -128 123 -2.5 -250 -724 100 10714 99625 2.17417 299.17417 150.67417 15067.41741 2.17417 299.17416 150.67417 15067.41742 2.17417 299.17417 150.67417000000015 15067.41700 2020-01-01 2020-01-02 2020-01-01 00:12:04 2020-01-02 03:40:25 2020-01-01 00:12:04.000 2020-01-02 03:40:25.000 724 99625 50174.5 5017450 724 99625 50174.5 5017450 -32446 32692 4954.66 495466 -127 124 -1.5 -150 -725 100 10715 99626 2.17717 299.17717 150.67717 15067.71771 2.17717 299.1772 150.67717 15067.71752 2.17717 299.17717 150.67717000000025 15067.71700 2020-01-01 2020-01-02 2020-01-01 00:12:05 2020-01-02 03:40:26 2020-01-01 00:12:05.000 2020-01-02 03:40:26.000 725 99626 50175.5 5017550 725 99626 50175.5 5017550 -32445 32693 4955.66 495566 -126 125 -0.5 -50 -726 100 10716 99627 2.18018 299.18018 150.68018 15068.01801 2.18018 299.18018 150.68017 15068.01782 2.18018 299.18018 150.6801799999999 15068.01800 2020-01-01 2020-01-02 2020-01-01 00:12:06 2020-01-02 03:40:27 2020-01-01 00:12:06.000 2020-01-02 03:40:27.000 726 99627 50176.5 5017650 726 99627 50176.5 5017650 -32444 32694 4956.66 495666 -125 126 0.5 50 +722 100 10712 99623 2.16816 299.16816 150.66816 15066.81681 2.16816 299.16818 150.66816 15066.81695 2.16816 299.16816 150.66815999999997 15066.81600 2020-01-01 2020-01-02 2020-01-01 00:12:02 2020-01-02 03:40:23 2020-01-01 00:12:02.000 2020-01-02 03:40:23.000 722 99623 50172.5 5017250 722 99623 50172.5 5017250 -32448 32690 4952.66 495266 -128 127 -0.94 -94 +723 100 10713 99624 2.17117 299.17117 150.67117 15067.11711 2.17117 299.17117 150.67117 15067.11724 2.17117 299.17117 150.6711699999999 15067.11700 2020-01-01 2020-01-02 2020-01-01 00:12:03 2020-01-02 03:40:24 2020-01-01 00:12:03.000 2020-01-02 03:40:24.000 723 99624 50173.5 5017350 723 99624 50173.5 5017350 -32447 32691 4953.66 495366 -128 123 -2.5 -250 +724 100 10714 99625 2.17417 299.17417 150.67417 15067.41741 2.17417 299.17416 150.67417 15067.41742 2.17417 299.17417 150.67417000000012 15067.41700 2020-01-01 2020-01-02 2020-01-01 00:12:04 2020-01-02 03:40:25 2020-01-01 00:12:04.000 2020-01-02 03:40:25.000 724 99625 50174.5 5017450 724 99625 50174.5 5017450 -32446 32692 4954.66 495466 -127 124 -1.5 -150 +725 100 10715 99626 2.17717 299.17717 150.67717 15067.71771 2.17717 299.1772 150.67717 15067.71752 2.17717 299.17717 150.67717000000007 15067.71700 2020-01-01 2020-01-02 2020-01-01 00:12:05 2020-01-02 03:40:26 2020-01-01 00:12:05.000 2020-01-02 03:40:26.000 725 99626 50175.5 5017550 725 99626 50175.5 5017550 -32445 32693 4955.66 495566 -126 125 -0.5 -50 +726 100 10716 99627 2.18018 299.18018 150.68018 15068.01801 2.18018 299.18018 150.68017 15068.01782 2.18018 299.18018 150.68017999999995 15068.01800 2020-01-01 2020-01-02 2020-01-01 00:12:06 2020-01-02 03:40:27 2020-01-01 00:12:06.000 2020-01-02 03:40:27.000 726 99627 50176.5 5017650 726 99627 50176.5 5017650 -32444 32694 4956.66 495666 -125 126 0.5 50 727 100 10717 99628 2.18318 299.18318 150.68318 15068.31831 2.18318 299.1832 150.68318 15068.31842 2.18318 299.18318 150.68318 15068.31800 2020-01-01 2020-01-02 2020-01-01 00:12:07 2020-01-02 03:40:28 2020-01-01 00:12:07.000 2020-01-02 03:40:28.000 727 99628 50177.5 5017750 727 99628 50177.5 5017750 -32443 32695 4957.66 495766 -124 127 1.5 150 -728 100 10718 99629 2.18618 299.18618 150.68618 15068.61861 2.18618 299.1862 150.68618 15068.61875 2.18618 299.18618 150.6861800000001 15068.61800 2020-01-01 2020-01-02 2020-01-01 00:12:08 2020-01-02 03:40:29 2020-01-01 00:12:08.000 2020-01-02 03:40:29.000 728 99629 50178.5 5017850 728 99629 50178.5 5017850 -32442 32696 4958.66 495866 -128 127 -0.06 -6 -729 100 10719 99630 2.18918 299.18918 150.68918 15068.91891 2.18918 299.18918 150.68918 15068.91889 2.18918 299.18918 150.6891799999999 15068.91800 2020-01-01 2020-01-02 2020-01-01 00:12:09 2020-01-02 03:40:30 2020-01-01 00:12:09.000 2020-01-02 03:40:30.000 729 99630 50179.5 5017950 729 99630 50179.5 5017950 -32441 32697 4959.66 495966 -128 123 -1.62 -162 -73 102 10063 99973 0.21921 300.21921 150.21921 15172.14114 0.21921 300.2192 150.21921 15172.14121 0.21921 300.21921 150.2192099999999 15172.14021 2020-01-01 2020-01-02 2020-01-01 00:01:13 2020-01-02 03:46:13 2020-01-01 00:01:13.000 2020-01-02 03:46:13.000 73 99973 50023 5052323 73 99973 50023 5052323 -32496 32439 4602.009900990099 464803 -128 127 -3.4554455445544554 -349 -730 100 10720 99631 2.19219 299.19219 150.69219 15069.21921 2.19219 299.1922 150.69219 15069.21965 2.19219 299.19219 150.69218999999984 15069.21900 2020-01-01 2020-01-02 2020-01-01 00:12:10 2020-01-02 03:40:31 2020-01-01 00:12:10.000 2020-01-02 03:40:31.000 730 99631 50180.5 5018050 730 99631 50180.5 5018050 -32440 32698 4960.66 496066 -127 124 -0.62 -62 -731 100 10721 99632 2.19519 299.19519 150.69519 15069.51951 2.19519 299.1952 150.69519 15069.51928 2.19519 299.19519 150.69519000000025 15069.51900 2020-01-01 2020-01-02 2020-01-01 00:12:11 2020-01-02 03:40:32 2020-01-01 00:12:11.000 2020-01-02 03:40:32.000 731 99632 50181.5 5018150 731 99632 50181.5 5018150 -32439 32699 4961.66 496166 -126 125 0.38 38 -732 100 10722 99633 2.19819 299.19819 150.69819 15069.81981 2.19819 299.1982 150.69819 15069.81988 2.19819 299.19819 150.69818999999976 15069.81900 2020-01-01 2020-01-02 2020-01-01 00:12:12 2020-01-02 03:40:33 2020-01-01 00:12:12.000 2020-01-02 03:40:33.000 732 99633 50182.5 5018250 732 99633 50182.5 5018250 -32438 32700 4962.66 496266 -125 126 1.38 138 +728 100 10718 99629 2.18618 299.18618 150.68618 15068.61861 2.18618 299.1862 150.68618 15068.61875 2.18618 299.18618 150.68618000000004 15068.61800 2020-01-01 2020-01-02 2020-01-01 00:12:08 2020-01-02 03:40:29 2020-01-01 00:12:08.000 2020-01-02 03:40:29.000 728 99629 50178.5 5017850 728 99629 50178.5 5017850 -32442 32696 4958.66 495866 -128 127 -0.06 -6 +729 100 10719 99630 2.18918 299.18918 150.68918 15068.91891 2.18918 299.18918 150.68918 15068.91889 2.18918 299.18918 150.68917999999996 15068.91800 2020-01-01 2020-01-02 2020-01-01 00:12:09 2020-01-02 03:40:30 2020-01-01 00:12:09.000 2020-01-02 03:40:30.000 729 99630 50179.5 5017950 729 99630 50179.5 5017950 -32441 32697 4959.66 495966 -128 123 -1.62 -162 +73 102 10063 99973 0.21921 300.21921 150.21921 15172.14114 0.21921 300.2192 150.21921 15172.14121 0.21921 300.21921 150.21921 15172.14021 2020-01-01 2020-01-02 2020-01-01 00:01:13 2020-01-02 03:46:13 2020-01-01 00:01:13.000 2020-01-02 03:46:13.000 73 99973 50023 5052323 73 99973 50023 5052323 -32496 32439 4602.009900990099 464803 -128 127 -3.4554455445544554 -349 +730 100 10720 99631 2.19219 299.19219 150.69219 15069.21921 2.19219 299.1922 150.69219 15069.21965 2.19219 299.19219 150.69218999999993 15069.21900 2020-01-01 2020-01-02 2020-01-01 00:12:10 2020-01-02 03:40:31 2020-01-01 00:12:10.000 2020-01-02 03:40:31.000 730 99631 50180.5 5018050 730 99631 50180.5 5018050 -32440 32698 4960.66 496066 -127 124 -0.62 -62 +731 100 10721 99632 2.19519 299.19519 150.69519 15069.51951 2.19519 299.1952 150.69519 15069.51928 2.19519 299.19519 150.69519000000008 15069.51900 2020-01-01 2020-01-02 2020-01-01 00:12:11 2020-01-02 03:40:32 2020-01-01 00:12:11.000 2020-01-02 03:40:32.000 731 99632 50181.5 5018150 731 99632 50181.5 5018150 -32439 32699 4961.66 496166 -126 125 0.38 38 +732 100 10722 99633 2.19819 299.19819 150.69819 15069.81981 2.19819 299.1982 150.69819 15069.81988 2.19819 299.19819 150.69818999999993 15069.81900 2020-01-01 2020-01-02 2020-01-01 00:12:12 2020-01-02 03:40:33 2020-01-01 00:12:12.000 2020-01-02 03:40:33.000 732 99633 50182.5 5018250 732 99633 50182.5 5018250 -32438 32700 4962.66 496266 -125 126 1.38 138 733 100 10723 99634 2.2012 299.2012 150.7012 15070.12012 2.2012 299.2012 150.7012 15070.12022 2.20120 299.20120 150.70120000000003 15070.12000 2020-01-01 2020-01-02 2020-01-01 00:12:13 2020-01-02 03:40:34 2020-01-01 00:12:13.000 2020-01-02 03:40:34.000 733 99634 50183.5 5018350 733 99634 50183.5 5018350 -32437 32701 4963.66 496366 -124 127 2.38 238 -734 100 10724 99635 2.2042 299.2042 150.7042 15070.42042 2.2042 299.2042 150.7042 15070.42036 2.20420 299.20420 150.7042000000001 15070.42000 2020-01-01 2020-01-02 2020-01-01 00:12:14 2020-01-02 03:40:35 2020-01-01 00:12:14.000 2020-01-02 03:40:35.000 734 99635 50184.5 5018450 734 99635 50184.5 5018450 -32436 32702 4964.66 496466 -128 127 0.82 82 -735 100 10725 99636 2.2072 299.2072 150.7072 15070.72072 2.2072 299.2072 150.70721 15070.72111 2.20720 299.20720 150.70720000000014 15070.72000 2020-01-01 2020-01-02 2020-01-01 00:12:15 2020-01-02 03:40:36 2020-01-01 00:12:15.000 2020-01-02 03:40:36.000 735 99636 50185.5 5018550 735 99636 50185.5 5018550 -32435 32703 4965.66 496566 -128 127 -0.74 -74 -736 100 10726 99637 2.21021 299.21021 150.71021 15071.02102 2.21021 299.2102 150.7102 15071.02076 2.21021 299.21021 150.71020999999988 15071.02100 2020-01-01 2020-01-02 2020-01-01 00:12:16 2020-01-02 03:40:37 2020-01-01 00:12:16.000 2020-01-02 03:40:37.000 736 99637 50186.5 5018650 736 99637 50186.5 5018650 -32434 32704 4966.66 496666 -128 124 -2.3 -230 +734 100 10724 99635 2.2042 299.2042 150.7042 15070.42042 2.2042 299.2042 150.7042 15070.42036 2.20420 299.20420 150.70420000000007 15070.42000 2020-01-01 2020-01-02 2020-01-01 00:12:14 2020-01-02 03:40:35 2020-01-01 00:12:14.000 2020-01-02 03:40:35.000 734 99635 50184.5 5018450 734 99635 50184.5 5018450 -32436 32702 4964.66 496466 -128 127 0.82 82 +735 100 10725 99636 2.2072 299.2072 150.7072 15070.72072 2.2072 299.2072 150.70721 15070.72111 2.20720 299.20720 150.70720000000003 15070.72000 2020-01-01 2020-01-02 2020-01-01 00:12:15 2020-01-02 03:40:36 2020-01-01 00:12:15.000 2020-01-02 03:40:36.000 735 99636 50185.5 5018550 735 99636 50185.5 5018550 -32435 32703 4965.66 496566 -128 127 -0.74 -74 +736 100 10726 99637 2.21021 299.21021 150.71021 15071.02102 2.21021 299.2102 150.7102 15071.02076 2.21021 299.21021 150.71021 15071.02100 2020-01-01 2020-01-02 2020-01-01 00:12:16 2020-01-02 03:40:37 2020-01-01 00:12:16.000 2020-01-02 03:40:37.000 736 99637 50186.5 5018650 736 99637 50186.5 5018650 -32434 32704 4966.66 496666 -128 124 -2.3 -230 737 100 10727 99638 2.21321 299.21321 150.71321 15071.32132 2.21321 299.21323 150.71321 15071.32139 2.21321 299.21321 150.71320999999992 15071.32100 2020-01-01 2020-01-02 2020-01-01 00:12:17 2020-01-02 03:40:38 2020-01-01 00:12:17.000 2020-01-02 03:40:38.000 737 99638 50187.5 5018750 737 99638 50187.5 5018750 -32433 32705 4967.66 496766 -127 125 -1.3 -130 738 100 10728 99639 2.21621 299.21621 150.71621 15071.62162 2.21621 299.21622 150.71621 15071.62169 2.21621 299.21621 150.71621 15071.62100 2020-01-01 2020-01-02 2020-01-01 00:12:18 2020-01-02 03:40:39 2020-01-01 00:12:18.000 2020-01-02 03:40:39.000 738 99639 50188.5 5018850 738 99639 50188.5 5018850 -32432 32706 4968.66 496866 -126 126 -0.3 -30 -739 100 10729 99640 2.21921 299.21921 150.71921 15071.92192 2.21921 299.2192 150.71921 15071.92199 2.21921 299.21921 150.7192099999999 15071.92100 2020-01-01 2020-01-02 2020-01-01 00:12:19 2020-01-02 03:40:40 2020-01-01 00:12:19.000 2020-01-02 03:40:40.000 739 99640 50189.5 5018950 739 99640 50189.5 5018950 -32431 32707 4969.66 496966 -125 127 0.7 70 -74 102 10064 99974 0.22222 300.22222 150.22222 15172.44444 0.22222 300.22223 150.22222 15172.4448 0.22222 300.22222 150.22221999999977 15172.44422 2020-01-01 2020-01-02 2020-01-01 00:01:14 2020-01-02 03:46:14 2020-01-01 00:01:14.000 2020-01-02 03:46:14.000 74 99974 50024 5052424 74 99974 50024 5052424 -32495 32440 4603.009900990099 464904 -128 123 -4.99009900990099 -504 -740 100 10730 99641 2.22222 299.22222 150.72222 15072.22222 2.22222 299.22223 150.72222 15072.22258 2.22222 299.22222 150.72221999999977 15072.22200 2020-01-01 2020-01-02 2020-01-01 00:12:20 2020-01-02 03:40:41 2020-01-01 00:12:20.000 2020-01-02 03:40:41.000 740 99641 50190.5 5019050 740 99641 50190.5 5019050 -32430 32708 4970.66 497066 -128 127 -0.86 -86 -741 100 10731 99642 2.22522 299.22522 150.72522 15072.52252 2.22522 299.22522 150.72522 15072.52223 2.22522 299.22522 150.72522000000015 15072.52200 2020-01-01 2020-01-02 2020-01-01 00:12:21 2020-01-02 03:40:42 2020-01-01 00:12:21.000 2020-01-02 03:40:42.000 741 99642 50191.5 5019150 741 99642 50191.5 5019150 -32429 32709 4971.66 497166 -128 127 -2.42 -242 -742 100 10732 99643 2.22822 299.22822 150.72822 15072.82282 2.22822 299.22824 150.72822 15072.82286 2.22822 299.22822 150.72822000000028 15072.82200 2020-01-01 2020-01-02 2020-01-01 00:12:22 2020-01-02 03:40:43 2020-01-01 00:12:22.000 2020-01-02 03:40:43.000 742 99643 50192.5 5019250 742 99643 50192.5 5019250 -32428 32710 4972.66 497266 -128 123 -3.98 -398 -743 100 10733 99644 2.23123 299.23123 150.73123 15073.12312 2.23123 299.23123 150.73123 15073.12316 2.23123 299.23123 150.73122999999993 15073.12300 2020-01-01 2020-01-02 2020-01-01 00:12:23 2020-01-02 03:40:44 2020-01-01 00:12:23.000 2020-01-02 03:40:44.000 743 99644 50193.5 5019350 743 99644 50193.5 5019350 -32427 32711 4973.66 497366 -127 124 -2.98 -298 -744 100 10734 99645 2.23423 299.23423 150.73423 15073.42342 2.23423 299.23422 150.73423 15073.42345 2.23423 299.23423 150.73423000000003 15073.42300 2020-01-01 2020-01-02 2020-01-01 00:12:24 2020-01-02 03:40:45 2020-01-01 00:12:24.000 2020-01-02 03:40:45.000 744 99645 50194.5 5019450 744 99645 50194.5 5019450 -32426 32712 4974.66 497466 -126 125 -1.98 -198 -745 100 10735 99646 2.23723 299.23723 150.73723 15073.72372 2.23723 299.23724 150.73724 15073.72405 2.23723 299.23723 150.73723000000012 15073.72300 2020-01-01 2020-01-02 2020-01-01 00:12:25 2020-01-02 03:40:46 2020-01-01 00:12:25.000 2020-01-02 03:40:46.000 745 99646 50195.5 5019550 745 99646 50195.5 5019550 -32425 32713 4975.66 497566 -125 126 -0.98 -98 -746 100 10736 99647 2.24024 299.24024 150.74024 15074.02402 2.24024 299.24023 150.74023 15074.02373 2.24024 299.24024 150.74023999999977 15074.02400 2020-01-01 2020-01-02 2020-01-01 00:12:26 2020-01-02 03:40:47 2020-01-01 00:12:26.000 2020-01-02 03:40:47.000 746 99647 50196.5 5019650 746 99647 50196.5 5019650 -32424 32714 4976.66 497666 -124 127 0.02 2 -747 100 10737 99648 2.24324 299.24324 150.74324 15074.32432 2.24324 299.24326 150.74324 15074.32433 2.24324 299.24324 150.74323999999982 15074.32400 2020-01-01 2020-01-02 2020-01-01 00:12:27 2020-01-02 03:40:48 2020-01-01 00:12:27.000 2020-01-02 03:40:48.000 747 99648 50197.5 5019750 747 99648 50197.5 5019750 -32423 32715 4977.66 497766 -128 127 -1.54 -154 -748 100 10738 99649 2.24624 299.24624 150.74624 15074.62462 2.24624 299.24625 150.74624 15074.62463 2.24624 299.24624 150.74624000000028 15074.62400 2020-01-01 2020-01-02 2020-01-01 00:12:28 2020-01-02 03:40:49 2020-01-01 00:12:28.000 2020-01-02 03:40:49.000 748 99649 50198.5 5019850 748 99649 50198.5 5019850 -32422 32716 4978.66 497866 -128 123 -3.1 -310 -749 100 10739 99650 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.7492399999998 15074.92400 2020-01-01 2020-01-02 2020-01-01 00:12:29 2020-01-02 03:40:50 2020-01-01 00:12:29.000 2020-01-02 03:40:50.000 749 99650 50199.5 5019950 749 99650 50199.5 5019950 -32421 32717 4979.66 497966 -127 124 -2.1 -210 -75 102 10065 99975 0.22522 300.22522 150.22522 15172.74774 0.22522 300.22522 150.22522 15172.74745 0.22522 300.22522 150.22522000000015 15172.74722 2020-01-01 2020-01-02 2020-01-01 00:01:15 2020-01-02 03:46:15 2020-01-01 00:01:15.000 2020-01-02 03:46:15.000 75 99975 50025 5052525 75 99975 50025 5052525 -32494 32441 4604.009900990099 465005 -127 124 -3.99009900990099 -403 -750 100 10740 99651 2.25225 299.25225 150.75225 15075.22522 2.25225 299.25226 150.75225 15075.22552 2.25225 299.25225 150.75224999999966 15075.22500 2020-01-01 2020-01-02 2020-01-01 00:12:30 2020-01-02 03:40:51 2020-01-01 00:12:30.000 2020-01-02 03:40:51.000 750 99651 50200.5 5020050 750 99651 50200.5 5020050 -32420 32718 4980.66 498066 -126 125 -1.1 -110 -751 100 10741 99652 2.25525 299.25525 150.75525 15075.52552 2.25525 299.25525 150.75525 15075.5252 2.25525 299.25525 150.75525000000013 15075.52500 2020-01-01 2020-01-02 2020-01-01 00:12:31 2020-01-02 03:40:52 2020-01-01 00:12:31.000 2020-01-02 03:40:52.000 751 99652 50201.5 5020150 751 99652 50201.5 5020150 -32419 32719 4981.66 498166 -125 126 -0.1 -10 -752 100 10742 99653 2.25825 299.25825 150.75825 15075.82582 2.25825 299.25827 150.75825 15075.8258 2.25825 299.25825 150.75825000000017 15075.82500 2020-01-01 2020-01-02 2020-01-01 00:12:32 2020-01-02 03:40:53 2020-01-01 00:12:32.000 2020-01-02 03:40:53.000 752 99653 50202.5 5020250 752 99653 50202.5 5020250 -32418 32720 4982.66 498266 -124 127 0.9 90 -753 100 10743 99654 2.26126 299.26126 150.76126 15076.12612 2.26126 299.26126 150.76126 15076.12609 2.26126 299.26126 150.7612599999999 15076.12600 2020-01-01 2020-01-02 2020-01-01 00:12:33 2020-01-02 03:40:54 2020-01-01 00:12:33.000 2020-01-02 03:40:54.000 753 99654 50203.5 5020350 753 99654 50203.5 5020350 -32417 32721 4983.66 498366 -128 127 -0.66 -66 +739 100 10729 99640 2.21921 299.21921 150.71921 15071.92192 2.21921 299.2192 150.71921 15071.92199 2.21921 299.21921 150.71920999999998 15071.92100 2020-01-01 2020-01-02 2020-01-01 00:12:19 2020-01-02 03:40:40 2020-01-01 00:12:19.000 2020-01-02 03:40:40.000 739 99640 50189.5 5018950 739 99640 50189.5 5018950 -32431 32707 4969.66 496966 -125 127 0.7 70 +74 102 10064 99974 0.22222 300.22222 150.22222 15172.44444 0.22222 300.22223 150.22222 15172.4448 0.22222 300.22222 150.22221999999994 15172.44422 2020-01-01 2020-01-02 2020-01-01 00:01:14 2020-01-02 03:46:14 2020-01-01 00:01:14.000 2020-01-02 03:46:14.000 74 99974 50024 5052424 74 99974 50024 5052424 -32495 32440 4603.009900990099 464904 -128 123 -4.99009900990099 -504 +740 100 10730 99641 2.22222 299.22222 150.72222 15072.22222 2.22222 299.22223 150.72222 15072.22258 2.22222 299.22222 150.72221999999994 15072.22200 2020-01-01 2020-01-02 2020-01-01 00:12:20 2020-01-02 03:40:41 2020-01-01 00:12:20.000 2020-01-02 03:40:41.000 740 99641 50190.5 5019050 740 99641 50190.5 5019050 -32430 32708 4970.66 497066 -128 127 -0.86 -86 +741 100 10731 99642 2.22522 299.22522 150.72522 15072.52252 2.22522 299.22522 150.72522 15072.52223 2.22522 299.22522 150.72522000000004 15072.52200 2020-01-01 2020-01-02 2020-01-01 00:12:21 2020-01-02 03:40:42 2020-01-01 00:12:21.000 2020-01-02 03:40:42.000 741 99642 50191.5 5019150 741 99642 50191.5 5019150 -32429 32709 4971.66 497166 -128 127 -2.42 -242 +742 100 10732 99643 2.22822 299.22822 150.72822 15072.82282 2.22822 299.22824 150.72822 15072.82286 2.22822 299.22822 150.72822000000008 15072.82200 2020-01-01 2020-01-02 2020-01-01 00:12:22 2020-01-02 03:40:43 2020-01-01 00:12:22.000 2020-01-02 03:40:43.000 742 99643 50192.5 5019250 742 99643 50192.5 5019250 -32428 32710 4972.66 497266 -128 123 -3.98 -398 +743 100 10733 99644 2.23123 299.23123 150.73123 15073.12312 2.23123 299.23123 150.73123 15073.12316 2.23123 299.23123 150.73122999999998 15073.12300 2020-01-01 2020-01-02 2020-01-01 00:12:23 2020-01-02 03:40:44 2020-01-01 00:12:23.000 2020-01-02 03:40:44.000 743 99644 50193.5 5019350 743 99644 50193.5 5019350 -32427 32711 4973.66 497366 -127 124 -2.98 -298 +744 100 10734 99645 2.23423 299.23423 150.73423 15073.42342 2.23423 299.23422 150.73423 15073.42345 2.23423 299.23423 150.73423 15073.42300 2020-01-01 2020-01-02 2020-01-01 00:12:24 2020-01-02 03:40:45 2020-01-01 00:12:24.000 2020-01-02 03:40:45.000 744 99645 50194.5 5019450 744 99645 50194.5 5019450 -32426 32712 4974.66 497466 -126 125 -1.98 -198 +745 100 10735 99646 2.23723 299.23723 150.73723 15073.72372 2.23723 299.23724 150.73724 15073.72405 2.23723 299.23723 150.73723000000007 15073.72300 2020-01-01 2020-01-02 2020-01-01 00:12:25 2020-01-02 03:40:46 2020-01-01 00:12:25.000 2020-01-02 03:40:46.000 745 99646 50195.5 5019550 745 99646 50195.5 5019550 -32425 32713 4975.66 497566 -125 126 -0.98 -98 +746 100 10736 99647 2.24024 299.24024 150.74024 15074.02402 2.24024 299.24023 150.74023 15074.02373 2.24024 299.24024 150.74023999999994 15074.02400 2020-01-01 2020-01-02 2020-01-01 00:12:26 2020-01-02 03:40:47 2020-01-01 00:12:26.000 2020-01-02 03:40:47.000 746 99647 50196.5 5019650 746 99647 50196.5 5019650 -32424 32714 4976.66 497666 -124 127 0.02 2 +747 100 10737 99648 2.24324 299.24324 150.74324 15074.32432 2.24324 299.24326 150.74324 15074.32433 2.24324 299.24324 150.7432399999999 15074.32400 2020-01-01 2020-01-02 2020-01-01 00:12:27 2020-01-02 03:40:48 2020-01-01 00:12:27.000 2020-01-02 03:40:48.000 747 99648 50197.5 5019750 747 99648 50197.5 5019750 -32423 32715 4977.66 497766 -128 127 -1.54 -154 +748 100 10738 99649 2.24624 299.24624 150.74624 15074.62462 2.24624 299.24625 150.74624 15074.62463 2.24624 299.24624 150.7462400000001 15074.62400 2020-01-01 2020-01-02 2020-01-01 00:12:28 2020-01-02 03:40:49 2020-01-01 00:12:28.000 2020-01-02 03:40:49.000 748 99649 50198.5 5019850 748 99649 50198.5 5019850 -32422 32716 4978.66 497866 -128 123 -3.1 -310 +749 100 10739 99650 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74923999999996 15074.92400 2020-01-01 2020-01-02 2020-01-01 00:12:29 2020-01-02 03:40:50 2020-01-01 00:12:29.000 2020-01-02 03:40:50.000 749 99650 50199.5 5019950 749 99650 50199.5 5019950 -32421 32717 4979.66 497966 -127 124 -2.1 -210 +75 102 10065 99975 0.22522 300.22522 150.22522 15172.74774 0.22522 300.22522 150.22522 15172.74745 0.22522 300.22522 150.22522000000004 15172.74722 2020-01-01 2020-01-02 2020-01-01 00:01:15 2020-01-02 03:46:15 2020-01-01 00:01:15.000 2020-01-02 03:46:15.000 75 99975 50025 5052525 75 99975 50025 5052525 -32494 32441 4604.009900990099 465005 -127 124 -3.99009900990099 -403 +750 100 10740 99651 2.25225 299.25225 150.75225 15075.22522 2.25225 299.25226 150.75225 15075.22552 2.25225 299.25225 150.75224999999983 15075.22500 2020-01-01 2020-01-02 2020-01-01 00:12:30 2020-01-02 03:40:51 2020-01-01 00:12:30.000 2020-01-02 03:40:51.000 750 99651 50200.5 5020050 750 99651 50200.5 5020050 -32420 32718 4980.66 498066 -126 125 -1.1 -110 +751 100 10741 99652 2.25525 299.25525 150.75525 15075.52552 2.25525 299.25525 150.75525 15075.5252 2.25525 299.25525 150.75525000000007 15075.52500 2020-01-01 2020-01-02 2020-01-01 00:12:31 2020-01-02 03:40:52 2020-01-01 00:12:31.000 2020-01-02 03:40:52.000 751 99652 50201.5 5020150 751 99652 50201.5 5020150 -32419 32719 4981.66 498166 -125 126 -0.1 -10 +752 100 10742 99653 2.25825 299.25825 150.75825 15075.82582 2.25825 299.25827 150.75825 15075.8258 2.25825 299.25825 150.75825000000003 15075.82500 2020-01-01 2020-01-02 2020-01-01 00:12:32 2020-01-02 03:40:53 2020-01-01 00:12:32.000 2020-01-02 03:40:53.000 752 99653 50202.5 5020250 752 99653 50202.5 5020250 -32418 32720 4982.66 498266 -124 127 0.9 90 +753 100 10743 99654 2.26126 299.26126 150.76126 15076.12612 2.26126 299.26126 150.76126 15076.12609 2.26126 299.26126 150.76126 15076.12600 2020-01-01 2020-01-02 2020-01-01 00:12:33 2020-01-02 03:40:54 2020-01-01 00:12:33.000 2020-01-02 03:40:54.000 753 99654 50203.5 5020350 753 99654 50203.5 5020350 -32417 32721 4983.66 498366 -128 127 -0.66 -66 754 100 10744 99655 2.26426 299.26426 150.76426 15076.42642 2.26426 299.26425 150.76426 15076.4264 2.26426 299.26426 150.76425999999995 15076.42600 2020-01-01 2020-01-02 2020-01-01 00:12:34 2020-01-02 03:40:55 2020-01-01 00:12:34.000 2020-01-02 03:40:55.000 754 99655 50204.5 5020450 754 99655 50204.5 5020450 -32416 32722 4984.66 498466 -128 123 -2.22 -222 -755 100 10745 99656 2.26726 299.26726 150.76726 15076.72672 2.26726 299.26727 150.76727 15076.72703 2.26726 299.26726 150.76726000000002 15076.72600 2020-01-01 2020-01-02 2020-01-01 00:12:35 2020-01-02 03:40:56 2020-01-01 00:12:35.000 2020-01-02 03:40:56.000 755 99656 50205.5 5020550 755 99656 50205.5 5020550 -32415 32723 4985.66 498566 -127 124 -1.22 -122 -756 100 10746 99657 2.27027 299.27027 150.77027 15077.02702 2.27027 299.27026 150.77026 15077.02667 2.27027 299.27027 150.77027000000027 15077.02700 2020-01-01 2020-01-02 2020-01-01 00:12:36 2020-01-02 03:40:57 2020-01-01 00:12:36.000 2020-01-02 03:40:57.000 756 99657 50206.5 5020650 756 99657 50206.5 5020650 -32414 32724 4986.66 498666 -126 125 -0.22 -22 -757 100 10747 99658 2.27327 299.27327 150.77327 15077.32732 2.27327 299.2733 150.77327 15077.32727 2.27327 299.27327 150.7732699999998 15077.32700 2020-01-01 2020-01-02 2020-01-01 00:12:37 2020-01-02 03:40:58 2020-01-01 00:12:37.000 2020-01-02 03:40:58.000 757 99658 50207.5 5020750 757 99658 50207.5 5020750 -32413 32725 4987.66 498766 -125 126 0.78 78 -758 100 10748 99659 2.27627 299.27627 150.77627 15077.62762 2.27627 299.27628 150.77627 15077.62756 2.27627 299.27627 150.77627000000018 15077.62700 2020-01-01 2020-01-02 2020-01-01 00:12:38 2020-01-02 03:40:59 2020-01-01 00:12:38.000 2020-01-02 03:40:59.000 758 99659 50208.5 5020850 758 99659 50208.5 5020850 -32412 32726 4988.66 498866 -124 127 1.78 178 -759 100 10749 99660 2.27927 299.27927 150.77927 15077.92792 2.27927 299.27927 150.77927 15077.92787 2.27927 299.27927 150.7792699999997 15077.92700 2020-01-01 2020-01-02 2020-01-01 00:12:39 2020-01-02 03:41:00 2020-01-01 00:12:39.000 2020-01-02 03:41:00.000 759 99660 50209.5 5020950 759 99660 50209.5 5020950 -32411 32727 4989.66 498966 -128 127 0.22 22 -76 102 10066 99976 0.22822 300.22822 150.22822 15173.05105 0.22822 300.22824 150.22822 15173.05109 0.22822 300.22822 150.22822000000028 15173.05022 2020-01-01 2020-01-02 2020-01-01 00:01:16 2020-01-02 03:46:16 2020-01-01 00:01:16.000 2020-01-02 03:46:16.000 76 99976 50026 5052626 76 99976 50026 5052626 -32493 32442 4605.009900990099 465106 -126 125 -2.99009900990099 -302 -760 100 10750 99661 2.28228 299.28228 150.78228 15078.22822 2.28228 299.2823 150.78228 15078.2285 2.28228 299.28228 150.78227999999996 15078.22800 2020-01-01 2020-01-02 2020-01-01 00:12:40 2020-01-02 03:41:01 2020-01-01 00:12:40.000 2020-01-02 03:41:01.000 760 99661 50210.5 5021050 760 99661 50210.5 5021050 -32410 32728 4990.66 499066 -128 127 -1.34 -134 +755 100 10745 99656 2.26726 299.26726 150.76726 15076.72672 2.26726 299.26727 150.76727 15076.72703 2.26726 299.26726 150.76726 15076.72600 2020-01-01 2020-01-02 2020-01-01 00:12:35 2020-01-02 03:40:56 2020-01-01 00:12:35.000 2020-01-02 03:40:56.000 755 99656 50205.5 5020550 755 99656 50205.5 5020550 -32415 32723 4985.66 498566 -127 124 -1.22 -122 +756 100 10746 99657 2.27027 299.27027 150.77027 15077.02702 2.27027 299.27026 150.77026 15077.02667 2.27027 299.27027 150.7702700000001 15077.02700 2020-01-01 2020-01-02 2020-01-01 00:12:36 2020-01-02 03:40:57 2020-01-01 00:12:36.000 2020-01-02 03:40:57.000 756 99657 50206.5 5020650 756 99657 50206.5 5020650 -32414 32724 4986.66 498666 -126 125 -0.22 -22 +757 100 10747 99658 2.27327 299.27327 150.77327 15077.32732 2.27327 299.2733 150.77327 15077.32727 2.27327 299.27327 150.7732699999999 15077.32700 2020-01-01 2020-01-02 2020-01-01 00:12:37 2020-01-02 03:40:58 2020-01-01 00:12:37.000 2020-01-02 03:40:58.000 757 99658 50207.5 5020750 757 99658 50207.5 5020750 -32413 32725 4987.66 498766 -125 126 0.78 78 +758 100 10748 99659 2.27627 299.27627 150.77627 15077.62762 2.27627 299.27628 150.77627 15077.62756 2.27627 299.27627 150.77627000000007 15077.62700 2020-01-01 2020-01-02 2020-01-01 00:12:38 2020-01-02 03:40:59 2020-01-01 00:12:38.000 2020-01-02 03:40:59.000 758 99659 50208.5 5020850 758 99659 50208.5 5020850 -32412 32726 4988.66 498866 -124 127 1.78 178 +759 100 10749 99660 2.27927 299.27927 150.77927 15077.92792 2.27927 299.27927 150.77927 15077.92787 2.27927 299.27927 150.7792699999999 15077.92700 2020-01-01 2020-01-02 2020-01-01 00:12:39 2020-01-02 03:41:00 2020-01-01 00:12:39.000 2020-01-02 03:41:00.000 759 99660 50209.5 5020950 759 99660 50209.5 5020950 -32411 32727 4989.66 498966 -128 127 0.22 22 +76 102 10066 99976 0.22822 300.22822 150.22822 15173.05105 0.22822 300.22824 150.22822 15173.05109 0.22822 300.22822 150.22822000000008 15173.05022 2020-01-01 2020-01-02 2020-01-01 00:01:16 2020-01-02 03:46:16 2020-01-01 00:01:16.000 2020-01-02 03:46:16.000 76 99976 50026 5052626 76 99976 50026 5052626 -32493 32442 4605.009900990099 465106 -126 125 -2.99009900990099 -302 +760 100 10750 99661 2.28228 299.28228 150.78228 15078.22822 2.28228 299.2823 150.78228 15078.2285 2.28228 299.28228 150.78228 15078.22800 2020-01-01 2020-01-02 2020-01-01 00:12:40 2020-01-02 03:41:01 2020-01-01 00:12:40.000 2020-01-02 03:41:01.000 760 99661 50210.5 5021050 760 99661 50210.5 5021050 -32410 32728 4990.66 499066 -128 127 -1.34 -134 761 100 10751 99662 2.28528 299.28528 150.78528 15078.52852 2.28528 299.28528 150.78528 15078.52814 2.28528 299.28528 150.78528000000003 15078.52800 2020-01-01 2020-01-02 2020-01-01 00:12:41 2020-01-02 03:41:02 2020-01-01 00:12:41.000 2020-01-02 03:41:02.000 761 99662 50211.5 5021150 761 99662 50211.5 5021150 -32409 32729 4991.66 499166 -128 124 -2.9 -290 -762 100 10752 99663 2.28828 299.28828 150.78828 15078.82882 2.28828 299.2883 150.78828 15078.82889 2.28828 299.28828 150.78828000000013 15078.82800 2020-01-01 2020-01-02 2020-01-01 00:12:42 2020-01-02 03:41:03 2020-01-01 00:12:42.000 2020-01-02 03:41:03.000 762 99663 50212.5 5021250 762 99663 50212.5 5021250 -32408 32730 4992.66 499266 -127 125 -1.9 -190 -763 100 10753 99664 2.29129 299.29129 150.79129 15079.12912 2.29129 299.2913 150.79129 15079.12904 2.29129 299.29129 150.7912899999998 15079.12900 2020-01-01 2020-01-02 2020-01-01 00:12:43 2020-01-02 03:41:04 2020-01-01 00:12:43.000 2020-01-02 03:41:04.000 763 99664 50213.5 5021350 763 99664 50213.5 5021350 -32407 32731 4993.66 499366 -126 126 -0.9 -90 -764 100 10754 99665 2.29429 299.29429 150.79429 15079.42942 2.29429 299.29428 150.79429 15079.42933 2.29429 299.29429 150.79428999999985 15079.42900 2020-01-01 2020-01-02 2020-01-01 00:12:44 2020-01-02 03:41:05 2020-01-01 00:12:44.000 2020-01-02 03:41:05.000 764 99665 50214.5 5021450 764 99665 50214.5 5021450 -32406 32732 4994.66 499466 -125 127 0.1 10 -765 100 10755 99666 2.29729 299.29729 150.79729 15079.72972 2.29729 299.2973 150.79729 15079.72996 2.29729 299.29729 150.79729000000032 15079.72900 2020-01-01 2020-01-02 2020-01-01 00:12:45 2020-01-02 03:41:06 2020-01-01 00:12:45.000 2020-01-02 03:41:06.000 765 99666 50215.5 5021550 765 99666 50215.5 5021550 -32405 32733 4995.66 499566 -128 127 -1.46 -146 -766 100 10756 99667 2.3003 299.3003 150.8003 15080.03003 2.3003 299.3003 150.80029 15080.02961 2.30030 299.30030 150.8003000000002 15080.03000 2020-01-01 2020-01-02 2020-01-01 00:12:46 2020-01-02 03:41:07 2020-01-01 00:12:46.000 2020-01-02 03:41:07.000 766 99667 50216.5 5021650 766 99667 50216.5 5021650 -32404 32734 4996.66 499666 -128 127 -3.02 -302 -767 100 10757 99668 2.3033 299.3033 150.8033 15080.33033 2.3033 299.3033 150.8033 15080.33036 2.30330 299.30330 150.8032999999997 15080.33000 2020-01-01 2020-01-02 2020-01-01 00:12:47 2020-01-02 03:41:08 2020-01-01 00:12:47.000 2020-01-02 03:41:08.000 767 99668 50217.5 5021750 767 99668 50217.5 5021750 -32403 32735 4997.66 499766 -128 123 -4.58 -458 -768 100 10758 99669 2.3063 299.3063 150.8063 15080.63063 2.3063 299.3063 150.8063 15080.6305 2.30630 299.30630 150.80630000000016 15080.63000 2020-01-01 2020-01-02 2020-01-01 00:12:48 2020-01-02 03:41:09 2020-01-01 00:12:48.000 2020-01-02 03:41:09.000 768 99669 50218.5 5021850 768 99669 50218.5 5021850 -32402 32736 4998.66 499866 -127 124 -3.58 -358 -769 100 10759 99670 2.3093 299.3093 150.8093 15080.93093 2.3093 299.3093 150.8093 15080.93084 2.30930 299.30930 150.8093000000002 15080.93000 2020-01-01 2020-01-02 2020-01-01 00:12:49 2020-01-02 03:41:10 2020-01-01 00:12:49.000 2020-01-02 03:41:10.000 769 99670 50219.5 5021950 769 99670 50219.5 5021950 -32401 32737 4999.66 499966 -126 125 -2.58 -258 -77 102 10067 99977 0.23123 300.23123 150.23123 15173.35435 0.23123 300.23123 150.23123 15173.35439 0.23123 300.23123 150.23122999999993 15173.35423 2020-01-01 2020-01-02 2020-01-01 00:01:17 2020-01-02 03:46:17 2020-01-01 00:01:17.000 2020-01-02 03:46:17.000 77 99977 50027 5052727 77 99977 50027 5052727 -32492 32443 4606.009900990099 465207 -125 126 -1.99009900990099 -201 -770 100 10760 99671 2.31231 299.31231 150.81231 15081.23123 2.31231 299.31232 150.81231 15081.23144 2.31231 299.31231 150.81230999999988 15081.23100 2020-01-01 2020-01-02 2020-01-01 00:12:50 2020-01-02 03:41:11 2020-01-01 00:12:50.000 2020-01-02 03:41:11.000 770 99671 50220.5 5022050 770 99671 50220.5 5022050 -32400 32738 5000.66 500066 -125 126 -1.58 -158 +762 100 10752 99663 2.28828 299.28828 150.78828 15078.82882 2.28828 299.2883 150.78828 15078.82889 2.28828 299.28828 150.78828 15078.82800 2020-01-01 2020-01-02 2020-01-01 00:12:42 2020-01-02 03:41:03 2020-01-01 00:12:42.000 2020-01-02 03:41:03.000 762 99663 50212.5 5021250 762 99663 50212.5 5021250 -32408 32730 4992.66 499266 -127 125 -1.9 -190 +763 100 10753 99664 2.29129 299.29129 150.79129 15079.12912 2.29129 299.2913 150.79129 15079.12904 2.29129 299.29129 150.79128999999998 15079.12900 2020-01-01 2020-01-02 2020-01-01 00:12:43 2020-01-02 03:41:04 2020-01-01 00:12:43.000 2020-01-02 03:41:04.000 763 99664 50213.5 5021350 763 99664 50213.5 5021350 -32407 32731 4993.66 499366 -126 126 -0.9 -90 +764 100 10754 99665 2.29429 299.29429 150.79429 15079.42942 2.29429 299.29428 150.79429 15079.42933 2.29429 299.29429 150.7942899999999 15079.42900 2020-01-01 2020-01-02 2020-01-01 00:12:44 2020-01-02 03:41:05 2020-01-01 00:12:44.000 2020-01-02 03:41:05.000 764 99665 50214.5 5021450 764 99665 50214.5 5021450 -32406 32732 4994.66 499466 -125 127 0.1 10 +765 100 10755 99666 2.29729 299.29729 150.79729 15079.72972 2.29729 299.2973 150.79729 15079.72996 2.29729 299.29729 150.79729000000015 15079.72900 2020-01-01 2020-01-02 2020-01-01 00:12:45 2020-01-02 03:41:06 2020-01-01 00:12:45.000 2020-01-02 03:41:06.000 765 99666 50215.5 5021550 765 99666 50215.5 5021550 -32405 32733 4995.66 499566 -128 127 -1.46 -146 +766 100 10756 99667 2.3003 299.3003 150.8003 15080.03003 2.3003 299.3003 150.80029 15080.02961 2.30030 299.30030 150.80030000000002 15080.03000 2020-01-01 2020-01-02 2020-01-01 00:12:46 2020-01-02 03:41:07 2020-01-01 00:12:46.000 2020-01-02 03:41:07.000 766 99667 50216.5 5021650 766 99667 50216.5 5021650 -32404 32734 4996.66 499666 -128 127 -3.02 -302 +767 100 10757 99668 2.3033 299.3033 150.8033 15080.33033 2.3033 299.3033 150.8033 15080.33036 2.30330 299.30330 150.80329999999987 15080.33000 2020-01-01 2020-01-02 2020-01-01 00:12:47 2020-01-02 03:41:08 2020-01-01 00:12:47.000 2020-01-02 03:41:08.000 767 99668 50217.5 5021750 767 99668 50217.5 5021750 -32403 32735 4997.66 499766 -128 123 -4.58 -458 +768 100 10758 99669 2.3063 299.3063 150.8063 15080.63063 2.3063 299.3063 150.8063 15080.6305 2.30630 299.30630 150.8063000000001 15080.63000 2020-01-01 2020-01-02 2020-01-01 00:12:48 2020-01-02 03:41:09 2020-01-01 00:12:48.000 2020-01-02 03:41:09.000 768 99669 50218.5 5021850 768 99669 50218.5 5021850 -32402 32736 4998.66 499866 -127 124 -3.58 -358 +769 100 10759 99670 2.3093 299.3093 150.8093 15080.93093 2.3093 299.3093 150.8093 15080.93084 2.30930 299.30930 150.80930000000004 15080.93000 2020-01-01 2020-01-02 2020-01-01 00:12:49 2020-01-02 03:41:10 2020-01-01 00:12:49.000 2020-01-02 03:41:10.000 769 99670 50219.5 5021950 769 99670 50219.5 5021950 -32401 32737 4999.66 499966 -126 125 -2.58 -258 +77 102 10067 99977 0.23123 300.23123 150.23123 15173.35435 0.23123 300.23123 150.23123 15173.35439 0.23123 300.23123 150.23122999999998 15173.35423 2020-01-01 2020-01-02 2020-01-01 00:01:17 2020-01-02 03:46:17 2020-01-01 00:01:17.000 2020-01-02 03:46:17.000 77 99977 50027 5052727 77 99977 50027 5052727 -32492 32443 4606.009900990099 465207 -125 126 -1.99009900990099 -201 +770 100 10760 99671 2.31231 299.31231 150.81231 15081.23123 2.31231 299.31232 150.81231 15081.23144 2.31231 299.31231 150.81230999999997 15081.23100 2020-01-01 2020-01-02 2020-01-01 00:12:50 2020-01-02 03:41:11 2020-01-01 00:12:50.000 2020-01-02 03:41:11.000 770 99671 50220.5 5022050 770 99671 50220.5 5022050 -32400 32738 5000.66 500066 -125 126 -1.58 -158 771 100 10761 99672 2.31531 299.31531 150.81531 15081.53153 2.31531 299.3153 150.81531 15081.53173 2.31531 299.31531 150.81530999999998 15081.53100 2020-01-01 2020-01-02 2020-01-01 00:12:51 2020-01-02 03:41:12 2020-01-01 00:12:51.000 2020-01-02 03:41:12.000 771 99672 50221.5 5022150 771 99672 50221.5 5022150 -32399 32739 5001.66 500166 -124 127 -0.58 -58 -772 100 10762 99673 2.31831 299.31831 150.81831 15081.83183 2.31831 299.31833 150.81831 15081.83183 2.31831 299.31831 150.81831000000005 15081.83100 2020-01-01 2020-01-02 2020-01-01 00:12:52 2020-01-02 03:41:13 2020-01-01 00:12:52.000 2020-01-02 03:41:13.000 772 99673 50222.5 5022250 772 99673 50222.5 5022250 -32398 32740 5002.66 500266 -128 127 -2.14 -214 -773 100 10763 99674 2.32132 299.32132 150.82132 15082.13213 2.32132 299.32132 150.82131 15082.13197 2.32132 299.32132 150.82131999999973 15082.13200 2020-01-01 2020-01-02 2020-01-01 00:12:53 2020-01-02 03:41:14 2020-01-01 00:12:53.000 2020-01-02 03:41:14.000 773 99674 50223.5 5022350 773 99674 50223.5 5022350 -32397 32741 5003.66 500366 -128 123 -3.7 -370 -774 100 10764 99675 2.32432 299.32432 150.82432 15082.43243 2.32432 299.3243 150.82432 15082.43231 2.32432 299.32432 150.82431999999983 15082.43200 2020-01-01 2020-01-02 2020-01-01 00:12:54 2020-01-02 03:41:15 2020-01-01 00:12:54.000 2020-01-02 03:41:15.000 774 99675 50224.5 5022450 774 99675 50224.5 5022450 -32396 32742 5004.66 500466 -127 124 -2.7 -270 -775 100 10765 99676 2.32732 299.32732 150.82732 15082.73273 2.32732 299.32733 150.82732 15082.73291 2.32732 299.32732 150.8273200000002 15082.73200 2020-01-01 2020-01-02 2020-01-01 00:12:55 2020-01-02 03:41:16 2020-01-01 00:12:55.000 2020-01-02 03:41:16.000 775 99676 50225.5 5022550 775 99676 50225.5 5022550 -32395 32743 5005.66 500566 -126 125 -1.7 -170 -776 100 10766 99677 2.33033 299.33033 150.83033 15083.03303 2.33033 299.33032 150.83033 15083.0332 2.33033 299.33033 150.8303300000001 15083.03300 2020-01-01 2020-01-02 2020-01-01 00:12:56 2020-01-02 03:41:17 2020-01-01 00:12:56.000 2020-01-02 03:41:17.000 776 99677 50226.5 5022650 776 99677 50226.5 5022650 -32394 32744 5006.66 500666 -125 126 -0.7 -70 -777 100 10767 99678 2.33333 299.33333 150.83333 15083.33333 2.33333 299.33334 150.83333 15083.3333 2.33333 299.33333 150.83333 15083.33300 2020-01-01 2020-01-02 2020-01-01 00:12:57 2020-01-02 03:41:18 2020-01-01 00:12:57.000 2020-01-02 03:41:18.000 777 99678 50227.5 5022750 777 99678 50227.5 5022750 -32393 32745 5007.66 500766 -124 127 0.3 30 +772 100 10762 99673 2.31831 299.31831 150.81831 15081.83183 2.31831 299.31833 150.81831 15081.83183 2.31831 299.31831 150.81831000000003 15081.83100 2020-01-01 2020-01-02 2020-01-01 00:12:52 2020-01-02 03:41:13 2020-01-01 00:12:52.000 2020-01-02 03:41:13.000 772 99673 50222.5 5022250 772 99673 50222.5 5022250 -32398 32740 5002.66 500266 -128 127 -2.14 -214 +773 100 10763 99674 2.32132 299.32132 150.82132 15082.13213 2.32132 299.32132 150.82131 15082.13197 2.32132 299.32132 150.8213199999999 15082.13200 2020-01-01 2020-01-02 2020-01-01 00:12:53 2020-01-02 03:41:14 2020-01-01 00:12:53.000 2020-01-02 03:41:14.000 773 99674 50223.5 5022350 773 99674 50223.5 5022350 -32397 32741 5003.66 500366 -128 123 -3.7 -370 +774 100 10764 99675 2.32432 299.32432 150.82432 15082.43243 2.32432 299.3243 150.82432 15082.43231 2.32432 299.32432 150.82431999999994 15082.43200 2020-01-01 2020-01-02 2020-01-01 00:12:54 2020-01-02 03:41:15 2020-01-01 00:12:54.000 2020-01-02 03:41:15.000 774 99675 50224.5 5022450 774 99675 50224.5 5022450 -32396 32742 5004.66 500466 -127 124 -2.7 -270 +775 100 10765 99676 2.32732 299.32732 150.82732 15082.73273 2.32732 299.32733 150.82732 15082.73291 2.32732 299.32732 150.8273200000001 15082.73200 2020-01-01 2020-01-02 2020-01-01 00:12:55 2020-01-02 03:41:16 2020-01-01 00:12:55.000 2020-01-02 03:41:16.000 775 99676 50225.5 5022550 775 99676 50225.5 5022550 -32395 32743 5005.66 500566 -126 125 -1.7 -170 +776 100 10766 99677 2.33033 299.33033 150.83033 15083.03303 2.33033 299.33032 150.83033 15083.0332 2.33033 299.33033 150.83033 15083.03300 2020-01-01 2020-01-02 2020-01-01 00:12:56 2020-01-02 03:41:17 2020-01-01 00:12:56.000 2020-01-02 03:41:17.000 776 99677 50226.5 5022650 776 99677 50226.5 5022650 -32394 32744 5006.66 500666 -125 126 -0.7 -70 +777 100 10767 99678 2.33333 299.33333 150.83333 15083.33333 2.33333 299.33334 150.83333 15083.3333 2.33333 299.33333 150.83333000000002 15083.33300 2020-01-01 2020-01-02 2020-01-01 00:12:57 2020-01-02 03:41:18 2020-01-01 00:12:57.000 2020-01-02 03:41:18.000 777 99678 50227.5 5022750 777 99678 50227.5 5022750 -32393 32745 5007.66 500766 -124 127 0.3 30 778 100 10768 99679 2.33633 299.33633 150.83633 15083.63363 2.33633 299.33633 150.83633 15083.63348 2.33633 299.33633 150.83633000000006 15083.63300 2020-01-01 2020-01-02 2020-01-01 00:12:58 2020-01-02 03:41:19 2020-01-01 00:12:58.000 2020-01-02 03:41:19.000 778 99679 50228.5 5022850 778 99679 50228.5 5022850 -32392 32746 5008.66 500866 -128 127 -1.26 -126 -779 100 10769 99680 2.33933 299.33933 150.83933 15083.93393 2.33933 299.33932 150.83933 15083.93378 2.33933 299.33933 150.8393300000001 15083.93300 2020-01-01 2020-01-02 2020-01-01 00:12:59 2020-01-02 03:41:20 2020-01-01 00:12:59.000 2020-01-02 03:41:20.000 779 99680 50229.5 5022950 779 99680 50229.5 5022950 -32391 32747 5009.66 500966 -128 123 -2.82 -282 +779 100 10769 99680 2.33933 299.33933 150.83933 15083.93393 2.33933 299.33932 150.83933 15083.93378 2.33933 299.33933 150.83933000000002 15083.93300 2020-01-01 2020-01-02 2020-01-01 00:12:59 2020-01-02 03:41:20 2020-01-01 00:12:59.000 2020-01-02 03:41:20.000 779 99680 50229.5 5022950 779 99680 50229.5 5022950 -32391 32747 5009.66 500966 -128 123 -2.82 -282 78 102 10068 99978 0.23423 300.23423 150.23423 15173.65765 0.23423 300.23422 150.23423 15173.65769 0.23423 300.23423 150.23423 15173.65723 2020-01-01 2020-01-02 2020-01-01 00:01:18 2020-01-02 03:46:18 2020-01-01 00:01:18.000 2020-01-02 03:46:18.000 78 99978 50028 5052828 78 99978 50028 5052828 -32491 32444 4607.009900990099 465308 -124 127 -0.9900990099009901 -100 -780 100 10770 99681 2.34234 299.34234 150.84234 15084.23423 2.34234 299.34235 150.84234 15084.23437 2.34234 299.34234 150.84233999999995 15084.23400 2020-01-01 2020-01-02 2020-01-01 00:13:00 2020-01-02 03:41:21 2020-01-01 00:13:00.000 2020-01-02 03:41:21.000 780 99681 50230.5 5023050 780 99681 50230.5 5023050 -32390 32748 5010.66 501066 -127 124 -1.82 -182 -781 100 10771 99682 2.34534 299.34534 150.84534 15084.53453 2.34534 299.34534 150.84534 15084.53467 2.34534 299.34534 150.84533999999994 15084.53400 2020-01-01 2020-01-02 2020-01-01 00:13:01 2020-01-02 03:41:22 2020-01-01 00:13:01.000 2020-01-02 03:41:22.000 781 99682 50231.5 5023150 781 99682 50231.5 5023150 -32389 32749 5011.66 501166 -126 125 -0.82 -82 -782 100 10772 99683 2.34834 299.34834 150.84834 15084.83483 2.34834 299.34836 150.84834 15084.83477 2.34834 299.34834 150.84834000000018 15084.83400 2020-01-01 2020-01-02 2020-01-01 00:13:02 2020-01-02 03:41:23 2020-01-01 00:13:02.000 2020-01-02 03:41:23.000 782 99683 50232.5 5023250 782 99683 50232.5 5023250 -32388 32750 5012.66 501266 -125 126 0.18 18 -783 100 10773 99684 2.35135 299.35135 150.85135 15085.13513 2.35135 299.35135 150.85134 15085.13495 2.35135 299.35135 150.85135000000005 15085.13500 2020-01-01 2020-01-02 2020-01-01 00:13:03 2020-01-02 03:41:24 2020-01-01 00:13:03.000 2020-01-02 03:41:24.000 783 99684 50233.5 5023350 783 99684 50233.5 5023350 -32387 32751 5013.66 501366 -124 127 1.18 118 -784 100 10774 99685 2.35435 299.35435 150.85435 15085.43543 2.35435 299.35434 150.85435 15085.43525 2.35435 299.35435 150.8543499999999 15085.43500 2020-01-01 2020-01-02 2020-01-01 00:13:04 2020-01-02 03:41:25 2020-01-01 00:13:04.000 2020-01-02 03:41:25.000 784 99685 50234.5 5023450 784 99685 50234.5 5023450 -32386 32752 5014.66 501466 -128 127 -0.38 -38 -785 100 10775 99686 2.35735 299.35735 150.85735 15085.73573 2.35735 299.35736 150.85736 15085.736 2.35735 299.35735 150.8573500000001 15085.73500 2020-01-01 2020-01-02 2020-01-01 00:13:05 2020-01-02 03:41:26 2020-01-01 00:13:05.000 2020-01-02 03:41:26.000 785 99686 50235.5 5023550 785 99686 50235.5 5023550 -32385 32753 5015.66 501566 -128 127 -1.94 -194 -786 100 10776 99687 2.36036 299.36036 150.86036 15086.03603 2.36036 299.36035 150.86036 15086.03614 2.36036 299.36036 150.86036000000001 15086.03600 2020-01-01 2020-01-02 2020-01-01 00:13:06 2020-01-02 03:41:27 2020-01-01 00:13:06.000 2020-01-02 03:41:27.000 786 99687 50236.5 5023650 786 99687 50236.5 5023650 -32384 32754 5016.66 501666 -128 124 -3.5 -350 -787 100 10777 99688 2.36336 299.36336 150.86336 15086.33633 2.36336 299.36337 150.86336 15086.33628 2.36336 299.36336 150.86335999999994 15086.33600 2020-01-01 2020-01-02 2020-01-01 00:13:07 2020-01-02 03:41:28 2020-01-01 00:13:07.000 2020-01-02 03:41:28.000 787 99688 50237.5 5023750 787 99688 50237.5 5023750 -32383 32755 5017.66 501766 -127 125 -2.5 -250 +780 100 10770 99681 2.34234 299.34234 150.84234 15084.23423 2.34234 299.34235 150.84234 15084.23437 2.34234 299.34234 150.84233999999984 15084.23400 2020-01-01 2020-01-02 2020-01-01 00:13:00 2020-01-02 03:41:21 2020-01-01 00:13:00.000 2020-01-02 03:41:21.000 780 99681 50230.5 5023050 780 99681 50230.5 5023050 -32390 32748 5010.66 501066 -127 124 -1.82 -182 +781 100 10771 99682 2.34534 299.34534 150.84534 15084.53453 2.34534 299.34534 150.84534 15084.53467 2.34534 299.34534 150.84533999999988 15084.53400 2020-01-01 2020-01-02 2020-01-01 00:13:01 2020-01-02 03:41:22 2020-01-01 00:13:01.000 2020-01-02 03:41:22.000 781 99682 50231.5 5023150 781 99682 50231.5 5023150 -32389 32749 5011.66 501166 -126 125 -0.82 -82 +782 100 10772 99683 2.34834 299.34834 150.84834 15084.83483 2.34834 299.34836 150.84834 15084.83477 2.34834 299.34834 150.84834000000006 15084.83400 2020-01-01 2020-01-02 2020-01-01 00:13:02 2020-01-02 03:41:23 2020-01-01 00:13:02.000 2020-01-02 03:41:23.000 782 99683 50232.5 5023250 782 99683 50232.5 5023250 -32388 32750 5012.66 501266 -125 126 0.18 18 +783 100 10773 99684 2.35135 299.35135 150.85135 15085.13513 2.35135 299.35135 150.85134 15085.13495 2.35135 299.35135 150.85135000000022 15085.13500 2020-01-01 2020-01-02 2020-01-01 00:13:03 2020-01-02 03:41:24 2020-01-01 00:13:03.000 2020-01-02 03:41:24.000 783 99684 50233.5 5023350 783 99684 50233.5 5023350 -32387 32751 5013.66 501366 -124 127 1.18 118 +784 100 10774 99685 2.35435 299.35435 150.85435 15085.43543 2.35435 299.35434 150.85435 15085.43525 2.35435 299.35435 150.85434999999973 15085.43500 2020-01-01 2020-01-02 2020-01-01 00:13:04 2020-01-02 03:41:25 2020-01-01 00:13:04.000 2020-01-02 03:41:25.000 784 99685 50234.5 5023450 784 99685 50234.5 5023450 -32386 32752 5014.66 501466 -128 127 -0.38 -38 +785 100 10775 99686 2.35735 299.35735 150.85735 15085.73573 2.35735 299.35736 150.85736 15085.736 2.35735 299.35735 150.8573500000002 15085.73500 2020-01-01 2020-01-02 2020-01-01 00:13:05 2020-01-02 03:41:26 2020-01-01 00:13:05.000 2020-01-02 03:41:26.000 785 99686 50235.5 5023550 785 99686 50235.5 5023550 -32385 32753 5015.66 501566 -128 127 -1.94 -194 +786 100 10776 99687 2.36036 299.36036 150.86036 15086.03603 2.36036 299.36035 150.86036 15086.03614 2.36036 299.36036 150.86036000000007 15086.03600 2020-01-01 2020-01-02 2020-01-01 00:13:06 2020-01-02 03:41:27 2020-01-01 00:13:06.000 2020-01-02 03:41:27.000 786 99687 50236.5 5023650 786 99687 50236.5 5023650 -32384 32754 5016.66 501666 -128 124 -3.5 -350 +787 100 10777 99688 2.36336 299.36336 150.86336 15086.33633 2.36336 299.36337 150.86336 15086.33628 2.36336 299.36336 150.8633599999999 15086.33600 2020-01-01 2020-01-02 2020-01-01 00:13:07 2020-01-02 03:41:28 2020-01-01 00:13:07.000 2020-01-02 03:41:28.000 787 99688 50237.5 5023750 787 99688 50237.5 5023750 -32383 32755 5017.66 501766 -127 125 -2.5 -250 788 100 10778 99689 2.36636 299.36636 150.86636 15086.63663 2.36636 299.36636 150.86636 15086.63641 2.36636 299.36636 150.86636 15086.63600 2020-01-01 2020-01-02 2020-01-01 00:13:08 2020-01-02 03:41:29 2020-01-01 00:13:08.000 2020-01-02 03:41:29.000 788 99689 50238.5 5023850 788 99689 50238.5 5023850 -32382 32756 5018.66 501866 -126 126 -1.5 -150 -789 100 10779 99690 2.36936 299.36936 150.86936 15086.93693 2.36936 299.36935 150.86936 15086.93672 2.36936 299.36936 150.86936000000003 15086.93600 2020-01-01 2020-01-02 2020-01-01 00:13:09 2020-01-02 03:41:30 2020-01-01 00:13:09.000 2020-01-02 03:41:30.000 789 99690 50239.5 5023950 789 99690 50239.5 5023950 -32381 32757 5019.66 501966 -125 127 -0.5 -50 -79 102 10069 99979 0.23723 300.23723 150.23723 15173.96096 0.23723 300.23724 150.23724 15173.96129 0.23723 300.23723 150.23723000000007 15173.96023 2020-01-01 2020-01-02 2020-01-01 00:01:19 2020-01-02 03:46:19 2020-01-01 00:01:19.000 2020-01-02 03:46:19.000 79 99979 50029 5052929 79 99979 50029 5052929 -32490 32445 4608.009900990099 465409 -128 127 -2.5247524752475248 -255 -790 100 10780 99691 2.37237 299.37237 150.87237 15087.23723 2.37237 299.37238 150.87237 15087.23747 2.37237 299.37237 150.87236999999993 15087.23700 2020-01-01 2020-01-02 2020-01-01 00:13:10 2020-01-02 03:41:31 2020-01-01 00:13:10.000 2020-01-02 03:41:31.000 790 99691 50240.5 5024050 790 99691 50240.5 5024050 -32380 32758 5020.66 502066 -128 127 -2.06 -206 -791 100 10781 99692 2.37537 299.37537 150.87537 15087.53753 2.37537 299.37537 150.87537 15087.53761 2.37537 299.37537 150.87536999999992 15087.53700 2020-01-01 2020-01-02 2020-01-01 00:13:11 2020-01-02 03:41:32 2020-01-01 00:13:11.000 2020-01-02 03:41:32.000 791 99692 50241.5 5024150 791 99692 50241.5 5024150 -32379 32759 5021.66 502166 -128 127 -3.62 -362 -792 100 10782 99693 2.37837 299.37837 150.87837 15087.83783 2.37837 299.3784 150.87837 15087.83775 2.37837 299.37837 150.87837000000007 15087.83700 2020-01-01 2020-01-02 2020-01-01 00:13:12 2020-01-02 03:41:33 2020-01-01 00:13:12.000 2020-01-02 03:41:33.000 792 99693 50242.5 5024250 792 99693 50242.5 5024250 -32378 32760 5022.66 502266 -128 123 -5.18 -518 -793 100 10783 99694 2.38138 299.38138 150.88138 15088.13813 2.38138 299.38138 150.88137 15088.13789 2.38138 299.38138 150.88138 15088.13800 2020-01-01 2020-01-02 2020-01-01 00:13:13 2020-01-02 03:41:34 2020-01-01 00:13:13.000 2020-01-02 03:41:34.000 793 99694 50243.5 5024350 793 99694 50243.5 5024350 -32377 32761 5023.66 502366 -127 124 -4.18 -418 +789 100 10779 99690 2.36936 299.36936 150.86936 15086.93693 2.36936 299.36935 150.86936 15086.93672 2.36936 299.36936 150.86936000000009 15086.93600 2020-01-01 2020-01-02 2020-01-01 00:13:09 2020-01-02 03:41:30 2020-01-01 00:13:09.000 2020-01-02 03:41:30.000 789 99690 50239.5 5023950 789 99690 50239.5 5023950 -32381 32757 5019.66 501966 -125 127 -0.5 -50 +79 102 10069 99979 0.23723 300.23723 150.23723 15173.96096 0.23723 300.23724 150.23724 15173.96129 0.23723 300.23723 150.23723000000012 15173.96023 2020-01-01 2020-01-02 2020-01-01 00:01:19 2020-01-02 03:46:19 2020-01-01 00:01:19.000 2020-01-02 03:46:19.000 79 99979 50029 5052929 79 99979 50029 5052929 -32490 32445 4608.009900990099 465409 -128 127 -2.5247524752475248 -255 +790 100 10780 99691 2.37237 299.37237 150.87237 15087.23723 2.37237 299.37238 150.87237 15087.23747 2.37237 299.37237 150.87236999999973 15087.23700 2020-01-01 2020-01-02 2020-01-01 00:13:10 2020-01-02 03:41:31 2020-01-01 00:13:10.000 2020-01-02 03:41:31.000 790 99691 50240.5 5024050 790 99691 50240.5 5024050 -32380 32758 5020.66 502066 -128 127 -2.06 -206 +791 100 10781 99692 2.37537 299.37537 150.87537 15087.53753 2.37537 299.37537 150.87537 15087.53761 2.37537 299.37537 150.87536999999983 15087.53700 2020-01-01 2020-01-02 2020-01-01 00:13:11 2020-01-02 03:41:32 2020-01-01 00:13:11.000 2020-01-02 03:41:32.000 791 99692 50241.5 5024150 791 99692 50241.5 5024150 -32379 32759 5021.66 502166 -128 127 -3.62 -362 +792 100 10782 99693 2.37837 299.37837 150.87837 15087.83783 2.37837 299.3784 150.87837 15087.83775 2.37837 299.37837 150.87837000000025 15087.83700 2020-01-01 2020-01-02 2020-01-01 00:13:12 2020-01-02 03:41:33 2020-01-01 00:13:12.000 2020-01-02 03:41:33.000 792 99693 50242.5 5024250 792 99693 50242.5 5024250 -32378 32760 5022.66 502266 -128 123 -5.18 -518 +793 100 10783 99694 2.38138 299.38138 150.88138 15088.13813 2.38138 299.38138 150.88137 15088.13789 2.38138 299.38138 150.88138000000012 15088.13800 2020-01-01 2020-01-02 2020-01-01 00:13:13 2020-01-02 03:41:34 2020-01-01 00:13:13.000 2020-01-02 03:41:34.000 793 99694 50243.5 5024350 793 99694 50243.5 5024350 -32377 32761 5023.66 502366 -127 124 -4.18 -418 794 100 10784 99695 2.38438 299.38438 150.88438 15088.43843 2.38438 299.3844 150.88438 15088.43864 2.38438 299.38438 150.88438000000002 15088.43800 2020-01-01 2020-01-02 2020-01-01 00:13:14 2020-01-02 03:41:35 2020-01-01 00:13:14.000 2020-01-02 03:41:35.000 794 99695 50244.5 5024450 794 99695 50244.5 5024450 -32376 32762 5024.66 502466 -126 125 -3.18 -318 -795 100 10785 99696 2.38738 299.38738 150.88738 15088.73873 2.38738 299.3874 150.88738 15088.73894 2.38738 299.38738 150.88738000000006 15088.73800 2020-01-01 2020-01-02 2020-01-01 00:13:15 2020-01-02 03:41:36 2020-01-01 00:13:15.000 2020-01-02 03:41:36.000 795 99696 50245.5 5024550 795 99696 50245.5 5024550 -32375 32763 5025.66 502566 -125 126 -2.18 -218 +795 100 10785 99696 2.38738 299.38738 150.88738 15088.73873 2.38738 299.3874 150.88738 15088.73894 2.38738 299.38738 150.8873800000001 15088.73800 2020-01-01 2020-01-02 2020-01-01 00:13:15 2020-01-02 03:41:36 2020-01-01 00:13:15.000 2020-01-02 03:41:36.000 795 99696 50245.5 5024550 795 99696 50245.5 5024550 -32375 32763 5025.66 502566 -125 126 -2.18 -218 796 100 10786 99697 2.39039 299.39039 150.89039 15089.03903 2.39039 299.39038 150.89039 15089.03908 2.39039 299.39039 150.89038999999997 15089.03900 2020-01-01 2020-01-02 2020-01-01 00:13:16 2020-01-02 03:41:37 2020-01-01 00:13:16.000 2020-01-02 03:41:37.000 796 99697 50246.5 5024650 796 99697 50246.5 5024650 -32374 32764 5026.66 502666 -124 127 -1.18 -118 -797 100 10787 99698 2.39339 299.39339 150.89339 15089.33933 2.39339 299.3934 150.89339 15089.33921 2.39339 299.39339 150.89338999999998 15089.33900 2020-01-01 2020-01-02 2020-01-01 00:13:17 2020-01-02 03:41:38 2020-01-01 00:13:17.000 2020-01-02 03:41:38.000 797 99698 50247.5 5024750 797 99698 50247.5 5024750 -32373 32765 5027.66 502766 -128 127 -2.74 -274 +797 100 10787 99698 2.39339 299.39339 150.89339 15089.33933 2.39339 299.3934 150.89339 15089.33921 2.39339 299.39339 150.89338999999987 15089.33900 2020-01-01 2020-01-02 2020-01-01 00:13:17 2020-01-02 03:41:38 2020-01-01 00:13:17.000 2020-01-02 03:41:38.000 797 99698 50247.5 5024750 797 99698 50247.5 5024750 -32373 32765 5027.66 502766 -128 127 -2.74 -274 798 100 10788 99699 2.39639 299.39639 150.89639 15089.63963 2.39639 299.3964 150.89639 15089.63936 2.39639 299.39639 150.8963899999999 15089.63900 2020-01-01 2020-01-02 2020-01-01 00:13:18 2020-01-02 03:41:39 2020-01-01 00:13:18.000 2020-01-02 03:41:39.000 798 99699 50248.5 5024850 798 99699 50248.5 5024850 -32372 32766 5028.66 502866 -128 123 -4.3 -430 799 100 10789 99700 2.39939 299.39939 150.89939 15089.93993 2.39939 299.3994 150.8994 15089.94011 2.39939 299.39939 150.89938999999998 15089.93900 2020-01-01 2020-01-02 2020-01-01 00:13:19 2020-01-02 03:41:40 2020-01-01 00:13:19.000 2020-01-02 03:41:40.000 799 99700 50249.5 5024950 799 99700 50249.5 5024950 -32371 32767 5029.66 502966 -127 124 -3.3 -330 -8 102 1007 9998 0.02402 300.02402 150.02402 15152.42642 0.02402 300.02402 150.02402 15152.42607 0.02402 300.02402 150.02402000000004 15152.42602 2020-01-01 2020-01-02 2020-01-01 00:00:08 2020-01-02 03:45:08 2020-01-01 00:00:08.000 2020-01-02 03:45:08.000 8 99908 49958 5045758 8 99908 49958 5045758 -32561 32374 4537.009900990099 458238 -125 126 -0.019801980198019802 -2 -80 102 10070 99980 0.24024 300.24024 150.24024 15174.26426 0.24024 300.24023 150.24023 15174.26397 0.24024 300.24024 150.24023999999994 15174.26424 2020-01-01 2020-01-02 2020-01-01 00:01:20 2020-01-02 03:46:20 2020-01-01 00:01:20.000 2020-01-02 03:46:20.000 80 99980 50030 5053030 80 99980 50030 5053030 -32489 32446 4609.009900990099 465510 -128 123 -4.0594059405940595 -410 -800 100 10790 99701 2.4024 299.4024 150.9024 15090.24024 2.4024 299.4024 150.9024 15090.24041 2.40240 299.40240 150.90240000000006 15090.24000 2020-01-01 2020-01-02 2020-01-01 00:13:20 2020-01-02 03:41:41 2020-01-01 00:13:20.000 2020-01-02 03:41:41.000 800 99701 50250.5 5025050 800 99701 50250.5 5025050 -32768 32167 4375.3 437530 -126 125 -2.3 -230 -801 100 10791 99702 2.4054 299.4054 150.9054 15090.54054 2.4054 299.4054 150.9054 15090.54058 2.40540 299.40540 150.90539999999993 15090.54000 2020-01-01 2020-01-02 2020-01-01 00:13:21 2020-01-02 03:41:42 2020-01-01 00:13:21.000 2020-01-02 03:41:42.000 801 99702 50251.5 5025150 801 99702 50251.5 5025150 -32767 32168 4376.3 437630 -125 126 -1.3 -130 -802 100 10792 99703 2.4084 299.4084 150.9084 15090.84084 2.4084 299.40842 150.9084 15090.84069 2.40840 299.40840 150.90840000000003 15090.84000 2020-01-01 2020-01-02 2020-01-01 00:13:22 2020-01-02 03:41:43 2020-01-01 00:13:22.000 2020-01-02 03:41:43.000 802 99703 50252.5 5025250 802 99703 50252.5 5025250 -32766 32169 4377.3 437730 -124 127 -0.3 -30 -803 100 10793 99704 2.41141 299.41141 150.91141 15091.14114 2.41141 299.4114 150.9114 15091.14098 2.41141 299.41141 150.91141000000005 15091.14100 2020-01-01 2020-01-02 2020-01-01 00:13:23 2020-01-02 03:41:44 2020-01-01 00:13:23.000 2020-01-02 03:41:44.000 803 99704 50253.5 5025350 803 99704 50253.5 5025350 -32765 32170 4378.3 437830 -128 127 -1.86 -186 -804 100 10794 99705 2.41441 299.41441 150.91441 15091.44144 2.41441 299.41443 150.91441 15091.44158 2.41441 299.41441 150.91440999999998 15091.44100 2020-01-01 2020-01-02 2020-01-01 00:13:24 2020-01-02 03:41:45 2020-01-01 00:13:24.000 2020-01-02 03:41:45.000 804 99705 50254.5 5025450 804 99705 50254.5 5025450 -32764 32171 4379.3 437930 -128 123 -3.42 -342 -805 100 10795 99706 2.41741 299.41741 150.91741 15091.74174 2.41741 299.41742 150.91741 15091.74188 2.41741 299.41741 150.91741 15091.74100 2020-01-01 2020-01-02 2020-01-01 00:13:25 2020-01-02 03:41:46 2020-01-01 00:13:25.000 2020-01-02 03:41:46.000 805 99706 50255.5 5025550 805 99706 50255.5 5025550 -32763 32172 4380.3 438030 -127 124 -2.42 -242 -806 100 10796 99707 2.42042 299.42042 150.92042 15092.04204 2.42042 299.4204 150.92042 15092.04205 2.42042 299.42042 150.9204200000001 15092.04200 2020-01-01 2020-01-02 2020-01-01 00:13:26 2020-01-02 03:41:47 2020-01-01 00:13:26.000 2020-01-02 03:41:47.000 806 99707 50256.5 5025650 806 99707 50256.5 5025650 -32762 32173 4381.3 438130 -126 125 -1.42 -142 -807 100 10797 99708 2.42342 299.42342 150.92342 15092.34234 2.42342 299.42343 150.92342 15092.34216 2.42342 299.42342 150.92341999999994 15092.34200 2020-01-01 2020-01-02 2020-01-01 00:13:27 2020-01-02 03:41:48 2020-01-01 00:13:27.000 2020-01-02 03:41:48.000 807 99708 50257.5 5025750 807 99708 50257.5 5025750 -32761 32174 4382.3 438230 -125 126 -0.42 -42 -808 100 10798 99709 2.42642 299.42642 150.92642 15092.64264 2.42642 299.42642 150.92642 15092.64245 2.42642 299.42642 150.9264199999999 15092.64200 2020-01-01 2020-01-02 2020-01-01 00:13:28 2020-01-02 03:41:49 2020-01-01 00:13:28.000 2020-01-02 03:41:49.000 808 99709 50258.5 5025850 808 99709 50258.5 5025850 -32760 32175 4383.3 438330 -124 127 0.58 58 -809 100 10799 99710 2.42942 299.42942 150.92942 15092.94294 2.42942 299.42944 150.92943 15092.94305 2.42942 299.42942 150.9294200000001 15092.94200 2020-01-01 2020-01-02 2020-01-01 00:13:29 2020-01-02 03:41:50 2020-01-01 00:13:29.000 2020-01-02 03:41:50.000 809 99710 50259.5 5025950 809 99710 50259.5 5025950 -32759 32176 4384.3 438430 -128 127 -0.98 -98 -81 102 10071 99981 0.24324 300.24324 150.24324 15174.56756 0.24324 300.24326 150.24324 15174.56758 0.24324 300.24324 150.2432399999999 15174.56724 2020-01-01 2020-01-02 2020-01-01 00:01:21 2020-01-02 03:46:21 2020-01-01 00:01:21.000 2020-01-02 03:46:21.000 81 99981 50031 5053131 81 99981 50031 5053131 -32488 32447 4610.009900990099 465611 -127 124 -3.0594059405940595 -309 -810 100 10800 99711 2.43243 299.43243 150.93243 15093.24324 2.43243 299.43243 150.93243 15093.24338 2.43243 299.43243 150.93243000000004 15093.24300 2020-01-01 2020-01-02 2020-01-01 00:13:30 2020-01-02 03:41:51 2020-01-01 00:13:30.000 2020-01-02 03:41:51.000 810 99711 50260.5 5026050 810 99711 50260.5 5026050 -32758 32177 4385.3 438530 -128 127 -2.54 -254 -811 100 10801 99712 2.43543 299.43543 150.93543 15093.54354 2.43543 299.43542 150.93543 15093.54353 2.43543 299.43543 150.93542999999983 15093.54300 2020-01-01 2020-01-02 2020-01-01 00:13:31 2020-01-02 03:41:52 2020-01-01 00:13:31.000 2020-01-02 03:41:52.000 811 99712 50261.5 5026150 811 99712 50261.5 5026150 -32757 32178 4386.3 438630 -128 124 -4.1 -410 -812 100 10802 99713 2.43843 299.43843 150.93843 15093.84384 2.43843 299.43845 150.93844 15093.84428 2.43843 299.43843 150.93843000000007 15093.84300 2020-01-01 2020-01-02 2020-01-01 00:13:32 2020-01-02 03:41:53 2020-01-01 00:13:32.000 2020-01-02 03:41:53.000 812 99713 50262.5 5026250 812 99713 50262.5 5026250 -32756 32179 4387.3 438730 -127 125 -3.1 -310 -813 100 10803 99714 2.44144 299.44144 150.94144 15094.14414 2.44144 299.44144 150.94143 15094.14392 2.44144 299.44144 150.94143999999997 15094.14400 2020-01-01 2020-01-02 2020-01-01 00:13:33 2020-01-02 03:41:54 2020-01-01 00:13:33.000 2020-01-02 03:41:54.000 813 99714 50263.5 5026350 813 99714 50263.5 5026350 -32755 32180 4388.3 438830 -126 126 -2.1 -210 -814 100 10804 99715 2.44444 299.44444 150.94444 15094.44444 2.44444 299.44446 150.94444 15094.44452 2.44444 299.44444 150.94444 15094.44400 2020-01-01 2020-01-02 2020-01-01 00:13:34 2020-01-02 03:41:55 2020-01-01 00:13:34.000 2020-01-02 03:41:55.000 814 99715 50264.5 5026450 814 99715 50264.5 5026450 -32754 32181 4389.3 438930 -125 127 -1.1 -110 +8 102 1007 9998 0.02402 300.02402 150.02402 15152.42642 0.02402 300.02402 150.02402 15152.42607 0.02402 300.02402 150.02402000000015 15152.42602 2020-01-01 2020-01-02 2020-01-01 00:00:08 2020-01-02 03:45:08 2020-01-01 00:00:08.000 2020-01-02 03:45:08.000 8 99908 49958 5045758 8 99908 49958 5045758 -32561 32374 4537.009900990099 458238 -125 126 -0.019801980198019802 -2 +80 102 10070 99980 0.24024 300.24024 150.24024 15174.26426 0.24024 300.24023 150.24023 15174.26397 0.24024 300.24024 150.24023999999977 15174.26424 2020-01-01 2020-01-02 2020-01-01 00:01:20 2020-01-02 03:46:20 2020-01-01 00:01:20.000 2020-01-02 03:46:20.000 80 99980 50030 5053030 80 99980 50030 5053030 -32489 32446 4609.009900990099 465510 -128 123 -4.0594059405940595 -410 +800 100 10790 99701 2.4024 299.4024 150.9024 15090.24024 2.4024 299.4024 150.9024 15090.24041 2.40240 299.40240 150.90240000000026 15090.24000 2020-01-01 2020-01-02 2020-01-01 00:13:20 2020-01-02 03:41:41 2020-01-01 00:13:20.000 2020-01-02 03:41:41.000 800 99701 50250.5 5025050 800 99701 50250.5 5025050 -32768 32167 4375.3 437530 -126 125 -2.3 -230 +801 100 10791 99702 2.4054 299.4054 150.9054 15090.54054 2.4054 299.4054 150.9054 15090.54058 2.40540 299.40540 150.90539999999976 15090.54000 2020-01-01 2020-01-02 2020-01-01 00:13:21 2020-01-02 03:41:42 2020-01-01 00:13:21.000 2020-01-02 03:41:42.000 801 99702 50251.5 5025150 801 99702 50251.5 5025150 -32767 32168 4376.3 437630 -125 126 -1.3 -130 +802 100 10792 99703 2.4084 299.4084 150.9084 15090.84084 2.4084 299.40842 150.9084 15090.84069 2.40840 299.40840 150.90840000000014 15090.84000 2020-01-01 2020-01-02 2020-01-01 00:13:22 2020-01-02 03:41:43 2020-01-01 00:13:22.000 2020-01-02 03:41:43.000 802 99703 50252.5 5025250 802 99703 50252.5 5025250 -32766 32169 4377.3 437730 -124 127 -0.3 -30 +803 100 10793 99704 2.41141 299.41141 150.91141 15091.14114 2.41141 299.4114 150.9114 15091.14098 2.41141 299.41141 150.9114100000001 15091.14100 2020-01-01 2020-01-02 2020-01-01 00:13:23 2020-01-02 03:41:44 2020-01-01 00:13:23.000 2020-01-02 03:41:44.000 803 99704 50253.5 5025350 803 99704 50253.5 5025350 -32765 32170 4378.3 437830 -128 127 -1.86 -186 +804 100 10794 99705 2.41441 299.41441 150.91441 15091.44144 2.41441 299.41443 150.91441 15091.44158 2.41441 299.41441 150.91440999999992 15091.44100 2020-01-01 2020-01-02 2020-01-01 00:13:24 2020-01-02 03:41:45 2020-01-01 00:13:24.000 2020-01-02 03:41:45.000 804 99705 50254.5 5025450 804 99705 50254.5 5025450 -32764 32171 4379.3 437930 -128 123 -3.42 -342 +805 100 10795 99706 2.41741 299.41741 150.91741 15091.74174 2.41741 299.41742 150.91741 15091.74188 2.41741 299.41741 150.91741000000002 15091.74100 2020-01-01 2020-01-02 2020-01-01 00:13:25 2020-01-02 03:41:46 2020-01-01 00:13:25.000 2020-01-02 03:41:46.000 805 99706 50255.5 5025550 805 99706 50255.5 5025550 -32763 32172 4380.3 438030 -127 124 -2.42 -242 +806 100 10796 99707 2.42042 299.42042 150.92042 15092.04204 2.42042 299.4204 150.92042 15092.04205 2.42042 299.42042 150.92042000000026 15092.04200 2020-01-01 2020-01-02 2020-01-01 00:13:26 2020-01-02 03:41:47 2020-01-01 00:13:26.000 2020-01-02 03:41:47.000 806 99707 50256.5 5025650 806 99707 50256.5 5025650 -32762 32173 4381.3 438130 -126 125 -1.42 -142 +807 100 10797 99708 2.42342 299.42342 150.92342 15092.34234 2.42342 299.42343 150.92342 15092.34216 2.42342 299.42342 150.92341999999977 15092.34200 2020-01-01 2020-01-02 2020-01-01 00:13:27 2020-01-02 03:41:48 2020-01-01 00:13:27.000 2020-01-02 03:41:48.000 807 99708 50257.5 5025750 807 99708 50257.5 5025750 -32761 32174 4382.3 438230 -125 126 -0.42 -42 +808 100 10798 99709 2.42642 299.42642 150.92642 15092.64264 2.42642 299.42642 150.92642 15092.64245 2.42642 299.42642 150.9264199999998 15092.64200 2020-01-01 2020-01-02 2020-01-01 00:13:28 2020-01-02 03:41:49 2020-01-01 00:13:28.000 2020-01-02 03:41:49.000 808 99709 50258.5 5025850 808 99709 50258.5 5025850 -32760 32175 4383.3 438330 -124 127 0.58 58 +809 100 10799 99710 2.42942 299.42942 150.92942 15092.94294 2.42942 299.42944 150.92943 15092.94305 2.42942 299.42942 150.92942000000028 15092.94200 2020-01-01 2020-01-02 2020-01-01 00:13:29 2020-01-02 03:41:50 2020-01-01 00:13:29.000 2020-01-02 03:41:50.000 809 99710 50259.5 5025950 809 99710 50259.5 5025950 -32759 32176 4384.3 438430 -128 127 -0.98 -98 +81 102 10071 99981 0.24324 300.24324 150.24324 15174.56756 0.24324 300.24326 150.24324 15174.56758 0.24324 300.24324 150.24323999999982 15174.56724 2020-01-01 2020-01-02 2020-01-01 00:01:21 2020-01-02 03:46:21 2020-01-01 00:01:21.000 2020-01-02 03:46:21.000 81 99981 50031 5053131 81 99981 50031 5053131 -32488 32447 4610.009900990099 465611 -127 124 -3.0594059405940595 -309 +810 100 10800 99711 2.43243 299.43243 150.93243 15093.24324 2.43243 299.43243 150.93243 15093.24338 2.43243 299.43243 150.93243000000015 15093.24300 2020-01-01 2020-01-02 2020-01-01 00:13:30 2020-01-02 03:41:51 2020-01-01 00:13:30.000 2020-01-02 03:41:51.000 810 99711 50260.5 5026050 810 99711 50260.5 5026050 -32758 32177 4385.3 438530 -128 127 -2.54 -254 +811 100 10801 99712 2.43543 299.43543 150.93543 15093.54354 2.43543 299.43542 150.93543 15093.54353 2.43543 299.43543 150.93542999999966 15093.54300 2020-01-01 2020-01-02 2020-01-01 00:13:31 2020-01-02 03:41:52 2020-01-01 00:13:31.000 2020-01-02 03:41:52.000 811 99712 50261.5 5026150 811 99712 50261.5 5026150 -32757 32178 4386.3 438630 -128 124 -4.1 -410 +812 100 10802 99713 2.43843 299.43843 150.93843 15093.84384 2.43843 299.43845 150.93844 15093.84428 2.43843 299.43843 150.93843000000012 15093.84300 2020-01-01 2020-01-02 2020-01-01 00:13:32 2020-01-02 03:41:53 2020-01-01 00:13:32.000 2020-01-02 03:41:53.000 812 99713 50262.5 5026250 812 99713 50262.5 5026250 -32756 32179 4387.3 438730 -127 125 -3.1 -310 +813 100 10803 99714 2.44144 299.44144 150.94144 15094.14414 2.44144 299.44144 150.94143 15094.14392 2.44144 299.44144 150.94144 15094.14400 2020-01-01 2020-01-02 2020-01-01 00:13:33 2020-01-02 03:41:54 2020-01-01 00:13:33.000 2020-01-02 03:41:54.000 813 99714 50263.5 5026350 813 99714 50263.5 5026350 -32755 32180 4388.3 438830 -126 126 -2.1 -210 +814 100 10804 99715 2.44444 299.44444 150.94444 15094.44444 2.44444 299.44446 150.94444 15094.44452 2.44444 299.44444 150.9444399999999 15094.44400 2020-01-01 2020-01-02 2020-01-01 00:13:34 2020-01-02 03:41:55 2020-01-01 00:13:34.000 2020-01-02 03:41:55.000 814 99715 50264.5 5026450 814 99715 50264.5 5026450 -32754 32181 4389.3 438930 -125 127 -1.1 -110 815 100 10805 99716 2.44744 299.44744 150.94744 15094.74474 2.44744 299.44745 150.94744 15094.74485 2.44744 299.44744 150.94743999999994 15094.74400 2020-01-01 2020-01-02 2020-01-01 00:13:35 2020-01-02 03:41:56 2020-01-01 00:13:35.000 2020-01-02 03:41:56.000 815 99716 50265.5 5026550 815 99716 50265.5 5026550 -32753 32182 4390.3 439030 -128 127 -2.66 -266 -816 100 10806 99717 2.45045 299.45045 150.95045 15095.04504 2.45045 299.45044 150.95045 15095.045 2.45045 299.45045 150.95045000000005 15095.04500 2020-01-01 2020-01-02 2020-01-01 00:13:36 2020-01-02 03:41:57 2020-01-01 00:13:36.000 2020-01-02 03:41:57.000 816 99717 50266.5 5026650 816 99717 50266.5 5026650 -32752 32183 4391.3 439130 -128 127 -4.22 -422 -817 100 10807 99718 2.45345 299.45345 150.95345 15095.34534 2.45345 299.45346 150.95345 15095.34574 2.45345 299.45345 150.9534500000001 15095.34500 2020-01-01 2020-01-02 2020-01-01 00:13:37 2020-01-02 03:41:58 2020-01-01 00:13:37.000 2020-01-02 03:41:58.000 817 99718 50267.5 5026750 817 99718 50267.5 5026750 -32751 32184 4392.3 439230 -128 123 -5.78 -578 -818 100 10808 99719 2.45645 299.45645 150.95645 15095.64564 2.45645 299.45645 150.95645 15095.64539 2.45645 299.45645 150.9564499999999 15095.64500 2020-01-01 2020-01-02 2020-01-01 00:13:38 2020-01-02 03:41:59 2020-01-01 00:13:38.000 2020-01-02 03:41:59.000 818 99719 50268.5 5026850 818 99719 50268.5 5026850 -32750 32185 4393.3 439330 -127 124 -4.78 -478 -819 100 10809 99720 2.45945 299.45945 150.95945 15095.94594 2.45945 299.45947 150.95946 15095.94602 2.45945 299.45945 150.95945000000006 15095.94500 2020-01-01 2020-01-02 2020-01-01 00:13:39 2020-01-02 03:42:00 2020-01-01 00:13:39.000 2020-01-02 03:42:00.000 819 99720 50269.5 5026950 819 99720 50269.5 5026950 -32749 32186 4394.3 439430 -126 125 -3.78 -378 -82 102 10072 99982 0.24624 300.24624 150.24624 15174.87087 0.24624 300.24625 150.24624 15174.87088 0.24624 300.24624 150.2462400000001 15174.87024 2020-01-01 2020-01-02 2020-01-01 00:01:22 2020-01-02 03:46:22 2020-01-01 00:01:22.000 2020-01-02 03:46:22.000 82 99982 50032 5053232 82 99982 50032 5053232 -32487 32448 4611.009900990099 465712 -126 125 -2.0594059405940595 -208 -820 100 10810 99721 2.46246 299.46246 150.96246 15096.24624 2.46246 299.46246 150.96246 15096.24633 2.46246 299.46246 150.96246000000008 15096.24600 2020-01-01 2020-01-02 2020-01-01 00:13:40 2020-01-02 03:42:01 2020-01-01 00:13:40.000 2020-01-02 03:42:01.000 820 99721 50270.5 5027050 820 99721 50270.5 5027050 -32748 32187 4395.3 439530 -125 126 -2.78 -278 -821 100 10811 99722 2.46546 299.46546 150.96546 15096.54654 2.46546 299.46545 150.96546 15096.54646 2.46546 299.46546 150.96545999999998 15096.54600 2020-01-01 2020-01-02 2020-01-01 00:13:41 2020-01-02 03:42:02 2020-01-01 00:13:41.000 2020-01-02 03:42:02.000 821 99722 50271.5 5027150 821 99722 50271.5 5027150 -32747 32188 4396.3 439630 -124 127 -1.78 -178 +816 100 10806 99717 2.45045 299.45045 150.95045 15095.04504 2.45045 299.45044 150.95045 15095.045 2.45045 299.45045 150.95045000000016 15095.04500 2020-01-01 2020-01-02 2020-01-01 00:13:36 2020-01-02 03:41:57 2020-01-01 00:13:36.000 2020-01-02 03:41:57.000 816 99717 50266.5 5026650 816 99717 50266.5 5026650 -32752 32183 4391.3 439130 -128 127 -4.22 -422 +817 100 10807 99718 2.45345 299.45345 150.95345 15095.34534 2.45345 299.45346 150.95345 15095.34574 2.45345 299.45345 150.95345000000026 15095.34500 2020-01-01 2020-01-02 2020-01-01 00:13:37 2020-01-02 03:41:58 2020-01-01 00:13:37.000 2020-01-02 03:41:58.000 817 99718 50267.5 5026750 817 99718 50267.5 5026750 -32751 32184 4392.3 439230 -128 123 -5.78 -578 +818 100 10808 99719 2.45645 299.45645 150.95645 15095.64564 2.45645 299.45645 150.95645 15095.64539 2.45645 299.45645 150.9564499999998 15095.64500 2020-01-01 2020-01-02 2020-01-01 00:13:38 2020-01-02 03:41:59 2020-01-01 00:13:38.000 2020-01-02 03:41:59.000 818 99719 50268.5 5026850 818 99719 50268.5 5026850 -32750 32185 4393.3 439330 -127 124 -4.78 -478 +819 100 10809 99720 2.45945 299.45945 150.95945 15095.94594 2.45945 299.45947 150.95946 15095.94602 2.45945 299.45945 150.95945000000017 15095.94500 2020-01-01 2020-01-02 2020-01-01 00:13:39 2020-01-02 03:42:00 2020-01-01 00:13:39.000 2020-01-02 03:42:00.000 819 99720 50269.5 5026950 819 99720 50269.5 5026950 -32749 32186 4394.3 439430 -126 125 -3.78 -378 +82 102 10072 99982 0.24624 300.24624 150.24624 15174.87087 0.24624 300.24625 150.24624 15174.87088 0.24624 300.24624 150.24624000000028 15174.87024 2020-01-01 2020-01-02 2020-01-01 00:01:22 2020-01-02 03:46:22 2020-01-01 00:01:22.000 2020-01-02 03:46:22.000 82 99982 50032 5053232 82 99982 50032 5053232 -32487 32448 4611.009900990099 465712 -126 125 -2.0594059405940595 -208 +820 100 10810 99721 2.46246 299.46246 150.96246 15096.24624 2.46246 299.46246 150.96246 15096.24633 2.46246 299.46246 150.96246000000014 15096.24600 2020-01-01 2020-01-02 2020-01-01 00:13:40 2020-01-02 03:42:01 2020-01-01 00:13:40.000 2020-01-02 03:42:01.000 820 99721 50270.5 5027050 820 99721 50270.5 5027050 -32748 32187 4395.3 439530 -125 126 -2.78 -278 +821 100 10811 99722 2.46546 299.46546 150.96546 15096.54654 2.46546 299.46545 150.96546 15096.54646 2.46546 299.46546 150.96545999999995 15096.54600 2020-01-01 2020-01-02 2020-01-01 00:13:41 2020-01-02 03:42:02 2020-01-01 00:13:41.000 2020-01-02 03:42:02.000 821 99722 50271.5 5027150 821 99722 50271.5 5027150 -32747 32188 4396.3 439630 -124 127 -1.78 -178 822 100 10812 99723 2.46846 299.46846 150.96846 15096.84684 2.46846 299.46848 150.96847 15096.84721 2.46846 299.46846 150.96846000000002 15096.84600 2020-01-01 2020-01-02 2020-01-01 00:13:42 2020-01-02 03:42:03 2020-01-01 00:13:42.000 2020-01-02 03:42:03.000 822 99723 50272.5 5027250 822 99723 50272.5 5027250 -32746 32189 4397.3 439730 -128 127 -3.34 -334 -823 100 10813 99724 2.47147 299.47147 150.97147 15097.14714 2.47147 299.47147 150.97146 15097.14686 2.47147 299.47147 150.97147000000012 15097.14700 2020-01-01 2020-01-02 2020-01-01 00:13:43 2020-01-02 03:42:04 2020-01-01 00:13:43.000 2020-01-02 03:42:04.000 823 99724 50273.5 5027350 823 99724 50273.5 5027350 -32745 32190 4398.3 439830 -128 123 -4.9 -490 -824 100 10814 99725 2.47447 299.47447 150.97447 15097.44744 2.47447 299.4745 150.97447 15097.44749 2.47447 299.47447 150.97446999999997 15097.44700 2020-01-01 2020-01-02 2020-01-01 00:13:44 2020-01-02 03:42:05 2020-01-01 00:13:44.000 2020-01-02 03:42:05.000 824 99725 50274.5 5027450 824 99725 50274.5 5027450 -32744 32191 4399.3 439930 -127 124 -3.9 -390 -825 100 10815 99726 2.47747 299.47747 150.97747 15097.74774 2.47747 299.47748 150.97747 15097.74779 2.47747 299.47747 150.9774699999999 15097.74700 2020-01-01 2020-01-02 2020-01-01 00:13:45 2020-01-02 03:42:06 2020-01-01 00:13:45.000 2020-01-02 03:42:06.000 825 99726 50275.5 5027550 825 99726 50275.5 5027550 -32743 32192 4400.3 440030 -126 125 -2.9 -290 -826 100 10816 99727 2.48048 299.48048 150.98048 15098.04804 2.48048 299.48047 150.98048 15098.04809 2.48048 299.48048 150.98048000000009 15098.04800 2020-01-01 2020-01-02 2020-01-01 00:13:46 2020-01-02 03:42:07 2020-01-01 00:13:46.000 2020-01-02 03:42:07.000 826 99727 50276.5 5027650 826 99727 50276.5 5027650 -32742 32193 4401.3 440130 -125 126 -1.9 -190 -827 100 10817 99728 2.48348 299.48348 150.98348 15098.34834 2.48348 299.4835 150.98348 15098.34869 2.48348 299.48348 150.98348000000004 15098.34800 2020-01-01 2020-01-02 2020-01-01 00:13:47 2020-01-02 03:42:08 2020-01-01 00:13:47.000 2020-01-02 03:42:08.000 827 99728 50277.5 5027750 827 99728 50277.5 5027750 -32741 32194 4402.3 440230 -124 127 -0.9 -90 -828 100 10818 99729 2.48648 299.48648 150.98648 15098.64864 2.48648 299.48648 150.98648 15098.64837 2.48648 299.48648 150.98647999999986 15098.64800 2020-01-01 2020-01-02 2020-01-01 00:13:48 2020-01-02 03:42:09 2020-01-01 00:13:48.000 2020-01-02 03:42:09.000 828 99729 50278.5 5027850 828 99729 50278.5 5027850 -32740 32195 4403.3 440330 -128 127 -2.46 -246 -829 100 10819 99730 2.48948 299.48948 150.98948 15098.94894 2.48948 299.4895 150.98948 15098.94896 2.48948 299.48948 150.9894800000001 15098.94800 2020-01-01 2020-01-02 2020-01-01 00:13:49 2020-01-02 03:42:10 2020-01-01 00:13:49.000 2020-01-02 03:42:10.000 829 99730 50279.5 5027950 829 99730 50279.5 5027950 -32739 32196 4404.3 440430 -128 123 -4.02 -402 -83 102 10073 99983 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24923999999996 15175.17324 2020-01-01 2020-01-02 2020-01-01 00:01:23 2020-01-02 03:46:23 2020-01-01 00:01:23.000 2020-01-02 03:46:23.000 83 99983 50033 5053333 83 99983 50033 5053333 -32486 32449 4612.009900990099 465813 -125 126 -1.0594059405940595 -107 -830 100 10820 99731 2.49249 299.49249 150.99249 15099.24924 2.49249 299.4925 150.99249 15099.24926 2.49249 299.49249 150.99249 15099.24900 2020-01-01 2020-01-02 2020-01-01 00:13:50 2020-01-02 03:42:11 2020-01-01 00:13:50.000 2020-01-02 03:42:11.000 830 99731 50280.5 5028050 830 99731 50280.5 5028050 -32738 32197 4405.3 440530 -127 124 -3.02 -302 -831 100 10821 99732 2.49549 299.49549 150.99549 15099.54954 2.49549 299.49548 150.99549 15099.54956 2.49549 299.49549 150.99548999999996 15099.54900 2020-01-01 2020-01-02 2020-01-01 00:13:51 2020-01-02 03:42:12 2020-01-01 00:13:51.000 2020-01-02 03:42:12.000 831 99732 50281.5 5028150 831 99732 50281.5 5028150 -32737 32198 4406.3 440630 -126 125 -2.02 -202 +823 100 10813 99724 2.47147 299.47147 150.97147 15097.14714 2.47147 299.47147 150.97146 15097.14686 2.47147 299.47147 150.9714700000003 15097.14700 2020-01-01 2020-01-02 2020-01-01 00:13:43 2020-01-02 03:42:04 2020-01-01 00:13:43.000 2020-01-02 03:42:04.000 823 99724 50273.5 5027350 823 99724 50273.5 5027350 -32745 32190 4398.3 439830 -128 123 -4.9 -490 +824 100 10814 99725 2.47447 299.47447 150.97447 15097.44744 2.47447 299.4745 150.97447 15097.44749 2.47447 299.47447 150.9744699999998 15097.44700 2020-01-01 2020-01-02 2020-01-01 00:13:44 2020-01-02 03:42:05 2020-01-01 00:13:44.000 2020-01-02 03:42:05.000 824 99725 50274.5 5027450 824 99725 50274.5 5027450 -32744 32191 4399.3 439930 -127 124 -3.9 -390 +825 100 10815 99726 2.47747 299.47747 150.97747 15097.74774 2.47747 299.47748 150.97747 15097.74779 2.47747 299.47747 150.97746999999984 15097.74700 2020-01-01 2020-01-02 2020-01-01 00:13:45 2020-01-02 03:42:06 2020-01-01 00:13:45.000 2020-01-02 03:42:06.000 825 99726 50275.5 5027550 825 99726 50275.5 5027550 -32743 32192 4400.3 440030 -126 125 -2.9 -290 +826 100 10816 99727 2.48048 299.48048 150.98048 15098.04804 2.48048 299.48047 150.98048 15098.04809 2.48048 299.48048 150.98048000000014 15098.04800 2020-01-01 2020-01-02 2020-01-01 00:13:46 2020-01-02 03:42:07 2020-01-01 00:13:46.000 2020-01-02 03:42:07.000 826 99727 50276.5 5027650 826 99727 50276.5 5027650 -32742 32193 4401.3 440130 -125 126 -1.9 -190 +827 100 10817 99728 2.48348 299.48348 150.98348 15098.34834 2.48348 299.4835 150.98348 15098.34869 2.48348 299.48348 150.98348000000018 15098.34800 2020-01-01 2020-01-02 2020-01-01 00:13:47 2020-01-02 03:42:08 2020-01-01 00:13:47.000 2020-01-02 03:42:08.000 827 99728 50277.5 5027750 827 99728 50277.5 5027750 -32741 32194 4402.3 440230 -124 127 -0.9 -90 +828 100 10818 99729 2.48648 299.48648 150.98648 15098.64864 2.48648 299.48648 150.98648 15098.64837 2.48648 299.48648 150.9864799999997 15098.64800 2020-01-01 2020-01-02 2020-01-01 00:13:48 2020-01-02 03:42:09 2020-01-01 00:13:48.000 2020-01-02 03:42:09.000 828 99729 50278.5 5027850 828 99729 50278.5 5027850 -32740 32195 4403.3 440330 -128 127 -2.46 -246 +829 100 10819 99730 2.48948 299.48948 150.98948 15098.94894 2.48948 299.4895 150.98948 15098.94896 2.48948 299.48948 150.98948000000016 15098.94800 2020-01-01 2020-01-02 2020-01-01 00:13:49 2020-01-02 03:42:10 2020-01-01 00:13:49.000 2020-01-02 03:42:10.000 829 99730 50279.5 5027950 829 99730 50279.5 5027950 -32739 32196 4404.3 440430 -128 123 -4.02 -402 +83 102 10073 99983 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.2492399999998 15175.17324 2020-01-01 2020-01-02 2020-01-01 00:01:23 2020-01-02 03:46:23 2020-01-01 00:01:23.000 2020-01-02 03:46:23.000 83 99983 50033 5053333 83 99983 50033 5053333 -32486 32449 4612.009900990099 465813 -125 126 -1.0594059405940595 -107 +830 100 10820 99731 2.49249 299.49249 150.99249 15099.24924 2.49249 299.4925 150.99249 15099.24926 2.49249 299.49249 150.99249000000003 15099.24900 2020-01-01 2020-01-02 2020-01-01 00:13:50 2020-01-02 03:42:11 2020-01-01 00:13:50.000 2020-01-02 03:42:11.000 830 99731 50280.5 5028050 830 99731 50280.5 5028050 -32738 32197 4405.3 440530 -127 124 -3.02 -302 +831 100 10821 99732 2.49549 299.49549 150.99549 15099.54954 2.49549 299.49548 150.99549 15099.54956 2.49549 299.49549 150.99548999999988 15099.54900 2020-01-01 2020-01-02 2020-01-01 00:13:51 2020-01-02 03:42:12 2020-01-01 00:13:51.000 2020-01-02 03:42:12.000 831 99732 50281.5 5028150 831 99732 50281.5 5028150 -32737 32198 4406.3 440630 -126 125 -2.02 -202 832 100 10822 99733 2.49849 299.49849 150.99849 15099.84984 2.49849 299.4985 150.9985 15099.85016 2.49849 299.49849 150.99848999999998 15099.84900 2020-01-01 2020-01-02 2020-01-01 00:13:52 2020-01-02 03:42:13 2020-01-01 00:13:52.000 2020-01-02 03:42:13.000 832 99733 50282.5 5028250 832 99733 50282.5 5028250 -32736 32199 4407.3 440730 -125 126 -1.02 -102 -833 100 10823 99734 2.5015 299.5015 151.0015 15100.15015 2.5015 299.5015 151.00149 15100.14983 2.50150 299.50150 151.00150000000008 15100.15000 2020-01-01 2020-01-02 2020-01-01 00:13:53 2020-01-02 03:42:14 2020-01-01 00:13:53.000 2020-01-02 03:42:14.000 833 99734 50283.5 5028350 833 99734 50283.5 5028350 -32735 32200 4408.3 440830 -124 127 -0.02 -2 -834 100 10824 99735 2.5045 299.5045 151.0045 15100.45045 2.5045 299.50452 151.0045 15100.45043 2.50450 299.50450 151.00449999999992 15100.45000 2020-01-01 2020-01-02 2020-01-01 00:13:54 2020-01-02 03:42:15 2020-01-01 00:13:54.000 2020-01-02 03:42:15.000 834 99735 50284.5 5028450 834 99735 50284.5 5028450 -32734 32201 4409.3 440930 -128 127 -1.58 -158 -835 100 10825 99736 2.5075 299.5075 151.0075 15100.75075 2.5075 299.5075 151.0075 15100.75073 2.50750 299.50750 151.00749999999994 15100.75000 2020-01-01 2020-01-02 2020-01-01 00:13:55 2020-01-02 03:42:16 2020-01-01 00:13:55.000 2020-01-02 03:42:16.000 835 99736 50285.5 5028550 835 99736 50285.5 5028550 -32733 32202 4410.3 441030 -128 123 -3.14 -314 +833 100 10823 99734 2.5015 299.5015 151.0015 15100.15015 2.5015 299.5015 151.00149 15100.14983 2.50150 299.50150 151.0015000000002 15100.15000 2020-01-01 2020-01-02 2020-01-01 00:13:53 2020-01-02 03:42:14 2020-01-01 00:13:53.000 2020-01-02 03:42:14.000 833 99734 50283.5 5028350 833 99734 50283.5 5028350 -32735 32200 4408.3 440830 -124 127 -0.02 -2 +834 100 10824 99735 2.5045 299.5045 151.0045 15100.45045 2.5045 299.50452 151.0045 15100.45043 2.50450 299.50450 151.00449999999972 15100.45000 2020-01-01 2020-01-02 2020-01-01 00:13:54 2020-01-02 03:42:15 2020-01-01 00:13:54.000 2020-01-02 03:42:15.000 834 99735 50284.5 5028450 834 99735 50284.5 5028450 -32734 32201 4409.3 440930 -128 127 -1.58 -158 +835 100 10825 99736 2.5075 299.5075 151.0075 15100.75075 2.5075 299.5075 151.0075 15100.75073 2.50750 299.50750 151.00749999999982 15100.75000 2020-01-01 2020-01-02 2020-01-01 00:13:55 2020-01-02 03:42:16 2020-01-01 00:13:55.000 2020-01-02 03:42:16.000 835 99736 50285.5 5028550 835 99736 50285.5 5028550 -32733 32202 4410.3 441030 -128 123 -3.14 -314 836 100 10826 99737 2.51051 299.51051 151.01051 15101.05105 2.51051 299.5105 151.01051 15101.05103 2.51051 299.51051 151.01051000000004 15101.05100 2020-01-01 2020-01-02 2020-01-01 00:13:56 2020-01-02 03:42:17 2020-01-01 00:13:56.000 2020-01-02 03:42:17.000 836 99737 50286.5 5028650 836 99737 50286.5 5028650 -32732 32203 4411.3 441130 -127 124 -2.14 -214 -837 100 10827 99738 2.51351 299.51351 151.01351 15101.35135 2.51351 299.51352 151.01351 15101.35162 2.51351 299.51351 151.01351 15101.35100 2020-01-01 2020-01-02 2020-01-01 00:13:57 2020-01-02 03:42:18 2020-01-01 00:13:57.000 2020-01-02 03:42:18.000 837 99738 50287.5 5028750 837 99738 50287.5 5028750 -32731 32204 4412.3 441230 -126 125 -1.14 -114 -838 100 10828 99739 2.51651 299.51651 151.01651 15101.65165 2.51651 299.5165 151.01651 15101.6513 2.51651 299.51651 151.01651 15101.65100 2020-01-01 2020-01-02 2020-01-01 00:13:58 2020-01-02 03:42:19 2020-01-01 00:13:58.000 2020-01-02 03:42:19.000 838 99739 50288.5 5028850 838 99739 50288.5 5028850 -32730 32205 4413.3 441330 -125 126 -0.14 -14 +837 100 10827 99738 2.51351 299.51351 151.01351 15101.35135 2.51351 299.51352 151.01351 15101.35162 2.51351 299.51351 151.01351000000014 15101.35100 2020-01-01 2020-01-02 2020-01-01 00:13:57 2020-01-02 03:42:18 2020-01-01 00:13:57.000 2020-01-02 03:42:18.000 837 99738 50287.5 5028750 837 99738 50287.5 5028750 -32731 32204 4412.3 441230 -126 125 -1.14 -114 +838 100 10828 99739 2.51651 299.51651 151.01651 15101.65165 2.51651 299.5165 151.01651 15101.6513 2.51651 299.51651 151.01650999999998 15101.65100 2020-01-01 2020-01-02 2020-01-01 00:13:58 2020-01-02 03:42:19 2020-01-01 00:13:58.000 2020-01-02 03:42:19.000 838 99739 50288.5 5028850 838 99739 50288.5 5028850 -32730 32205 4413.3 441330 -125 126 -0.14 -14 839 100 10829 99740 2.51951 299.51951 151.01951 15101.95195 2.51951 299.51953 151.01951 15101.9519 2.51951 299.51951 151.01951000000005 15101.95100 2020-01-01 2020-01-02 2020-01-01 00:13:59 2020-01-02 03:42:20 2020-01-01 00:13:59.000 2020-01-02 03:42:20.000 839 99740 50289.5 5028950 839 99740 50289.5 5028950 -32729 32206 4414.3 441430 -124 127 0.86 86 -84 102 10074 99984 0.25225 300.25225 150.25225 15175.47747 0.25225 300.25226 150.25225 15175.47778 0.25225 300.25225 150.25224999999986 15175.47725 2020-01-01 2020-01-02 2020-01-01 00:01:24 2020-01-02 03:46:24 2020-01-01 00:01:24.000 2020-01-02 03:46:24.000 84 99984 50034 5053434 84 99984 50034 5053434 -32485 32450 4613.009900990099 465914 -124 127 -0.0594059405940594 -6 -840 100 10830 99741 2.52252 299.52252 151.02252 15102.25225 2.52252 299.52252 151.02252 15102.2522 2.52252 299.52252 151.02252000000016 15102.25200 2020-01-01 2020-01-02 2020-01-01 00:14:00 2020-01-02 03:42:21 2020-01-01 00:14:00.000 2020-01-02 03:42:21.000 840 99741 50290.5 5029050 840 99741 50290.5 5029050 -32728 32207 4415.3 441530 -128 127 -0.7 -70 -841 100 10831 99742 2.52552 299.52552 151.02552 15102.55255 2.52552 299.5255 151.02552 15102.5525 2.52552 299.52552 151.02551999999994 15102.55200 2020-01-01 2020-01-02 2020-01-01 00:14:01 2020-01-02 03:42:22 2020-01-01 00:14:01.000 2020-01-02 03:42:22.000 841 99742 50291.5 5029150 841 99742 50291.5 5029150 -32727 32208 4416.3 441630 -128 127 -2.26 -226 -842 100 10832 99743 2.52852 299.52852 151.02852 15102.85285 2.52852 299.52853 151.02853 15102.85313 2.52852 299.52852 151.02851999999993 15102.85200 2020-01-01 2020-01-02 2020-01-01 00:14:02 2020-01-02 03:42:23 2020-01-01 00:14:02.000 2020-01-02 03:42:23.000 842 99743 50292.5 5029250 842 99743 50292.5 5029250 -32726 32209 4417.3 441730 -128 123 -3.82 -382 -843 100 10833 99744 2.53153 299.53153 151.03153 15103.15315 2.53153 299.53152 151.03152 15103.15278 2.53153 299.53153 151.0315300000001 15103.15300 2020-01-01 2020-01-02 2020-01-01 00:14:03 2020-01-02 03:42:24 2020-01-01 00:14:03.000 2020-01-02 03:42:24.000 843 99744 50293.5 5029350 843 99744 50293.5 5029350 -32725 32210 4418.3 441830 -127 124 -2.82 -282 -844 100 10834 99745 2.53453 299.53453 151.03453 15103.45345 2.53453 299.53455 151.03453 15103.45353 2.53453 299.53453 151.03453000000005 15103.45300 2020-01-01 2020-01-02 2020-01-01 00:14:04 2020-01-02 03:42:25 2020-01-01 00:14:04.000 2020-01-02 03:42:25.000 844 99745 50294.5 5029450 844 99745 50294.5 5029450 -32724 32211 4419.3 441930 -126 125 -1.82 -182 -845 100 10835 99746 2.53753 299.53753 151.03753 15103.75375 2.53753 299.53754 151.03753 15103.75366 2.53753 299.53753 151.0375299999999 15103.75300 2020-01-01 2020-01-02 2020-01-01 00:14:05 2020-01-02 03:42:26 2020-01-01 00:14:05.000 2020-01-02 03:42:26.000 845 99746 50295.5 5029550 845 99746 50295.5 5029550 -32723 32212 4420.3 442030 -125 126 -0.82 -82 +84 102 10074 99984 0.25225 300.25225 150.25225 15175.47747 0.25225 300.25226 150.25225 15175.47778 0.25225 300.25225 150.25224999999966 15175.47725 2020-01-01 2020-01-02 2020-01-01 00:01:24 2020-01-02 03:46:24 2020-01-01 00:01:24.000 2020-01-02 03:46:24.000 84 99984 50034 5053434 84 99984 50034 5053434 -32485 32450 4613.009900990099 465914 -124 127 -0.0594059405940594 -6 +840 100 10830 99741 2.52252 299.52252 151.02252 15102.25225 2.52252 299.52252 151.02252 15102.2522 2.52252 299.52252 151.02252000000033 15102.25200 2020-01-01 2020-01-02 2020-01-01 00:14:00 2020-01-02 03:42:21 2020-01-01 00:14:00.000 2020-01-02 03:42:21.000 840 99741 50290.5 5029050 840 99741 50290.5 5029050 -32728 32207 4415.3 441530 -128 127 -0.7 -70 +841 100 10831 99742 2.52552 299.52552 151.02552 15102.55255 2.52552 299.5255 151.02552 15102.5525 2.52552 299.52552 151.02551999999983 15102.55200 2020-01-01 2020-01-02 2020-01-01 00:14:01 2020-01-02 03:42:22 2020-01-01 00:14:01.000 2020-01-02 03:42:22.000 841 99742 50291.5 5029150 841 99742 50291.5 5029150 -32727 32208 4416.3 441630 -128 127 -2.26 -226 +842 100 10832 99743 2.52852 299.52852 151.02852 15102.85285 2.52852 299.52853 151.02853 15102.85313 2.52852 299.52852 151.02851999999987 15102.85200 2020-01-01 2020-01-02 2020-01-01 00:14:02 2020-01-02 03:42:23 2020-01-01 00:14:02.000 2020-01-02 03:42:23.000 842 99743 50292.5 5029250 842 99743 50292.5 5029250 -32726 32209 4417.3 441730 -128 123 -3.82 -382 +843 100 10833 99744 2.53153 299.53153 151.03153 15103.15315 2.53153 299.53152 151.03152 15103.15278 2.53153 299.53153 151.03153000000017 15103.15300 2020-01-01 2020-01-02 2020-01-01 00:14:03 2020-01-02 03:42:24 2020-01-01 00:14:03.000 2020-01-02 03:42:24.000 843 99744 50293.5 5029350 843 99744 50293.5 5029350 -32725 32210 4418.3 441830 -127 124 -2.82 -282 +844 100 10834 99745 2.53453 299.53453 151.03453 15103.45345 2.53453 299.53455 151.03453 15103.45353 2.53453 299.53453 151.03453000000022 15103.45300 2020-01-01 2020-01-02 2020-01-01 00:14:04 2020-01-02 03:42:25 2020-01-01 00:14:04.000 2020-01-02 03:42:25.000 844 99745 50294.5 5029450 844 99745 50294.5 5029450 -32724 32211 4419.3 441930 -126 125 -1.82 -182 +845 100 10835 99746 2.53753 299.53753 151.03753 15103.75375 2.53753 299.53754 151.03753 15103.75366 2.53753 299.53753 151.03752999999972 15103.75300 2020-01-01 2020-01-02 2020-01-01 00:14:05 2020-01-02 03:42:26 2020-01-01 00:14:05.000 2020-01-02 03:42:26.000 845 99746 50295.5 5029550 845 99746 50295.5 5029550 -32723 32212 4420.3 442030 -125 126 -0.82 -82 846 100 10836 99747 2.54054 299.54054 151.04054 15104.05405 2.54054 299.54053 151.04053 15104.05397 2.54054 299.54054 151.04054 15104.05400 2020-01-01 2020-01-02 2020-01-01 00:14:06 2020-01-02 03:42:27 2020-01-01 00:14:06.000 2020-01-02 03:42:27.000 846 99747 50296.5 5029650 846 99747 50296.5 5029650 -32722 32213 4421.3 442130 -124 127 0.18 18 -847 100 10837 99748 2.54354 299.54354 151.04354 15104.35435 2.54354 299.54355 151.04354 15104.3546 2.54354 299.54354 151.04354 15104.35400 2020-01-01 2020-01-02 2020-01-01 00:14:07 2020-01-02 03:42:28 2020-01-01 00:14:07.000 2020-01-02 03:42:28.000 847 99748 50297.5 5029750 847 99748 50297.5 5029750 -32721 32214 4422.3 442230 -128 127 -1.38 -138 -848 100 10838 99749 2.54654 299.54654 151.04654 15104.65465 2.54654 299.54654 151.04654 15104.65425 2.54654 299.54654 151.04653999999994 15104.65400 2020-01-01 2020-01-02 2020-01-01 00:14:08 2020-01-02 03:42:29 2020-01-01 00:14:08.000 2020-01-02 03:42:29.000 848 99749 50298.5 5029850 848 99749 50298.5 5029850 -32720 32215 4423.3 442330 -128 123 -2.94 -294 -849 100 10839 99750 2.54954 299.54954 151.04954 15104.95495 2.54954 299.54956 151.04954 15104.95499 2.54954 299.54954 151.04954 15104.95400 2020-01-01 2020-01-02 2020-01-01 00:14:09 2020-01-02 03:42:30 2020-01-01 00:14:09.000 2020-01-02 03:42:30.000 849 99750 50299.5 5029950 849 99750 50299.5 5029950 -32719 32216 4424.3 442430 -127 124 -1.94 -194 -85 102 10075 99985 0.25525 300.25525 150.25525 15175.78078 0.25525 300.25525 150.25525 15175.78046 0.25525 300.25525 150.25525000000007 15175.78025 2020-01-01 2020-01-02 2020-01-01 00:01:25 2020-01-02 03:46:25 2020-01-01 00:01:25.000 2020-01-02 03:46:25.000 85 99985 50035 5053535 85 99985 50035 5053535 -32484 32451 4614.009900990099 466015 -128 127 -1.5940594059405941 -161 -850 100 10840 99751 2.55255 299.55255 151.05255 15105.25525 2.55255 299.55255 151.05255 15105.25514 2.55255 299.55255 151.05255000000008 15105.25500 2020-01-01 2020-01-02 2020-01-01 00:14:10 2020-01-02 03:42:31 2020-01-01 00:14:10.000 2020-01-02 03:42:31.000 850 99751 50300.5 5030050 850 99751 50300.5 5030050 -32718 32217 4425.3 442530 -126 125 -0.94 -94 -851 100 10841 99752 2.55555 299.55555 151.05555 15105.55555 2.55555 299.55554 151.05555 15105.55547 2.55555 299.55555 151.05554999999993 15105.55500 2020-01-01 2020-01-02 2020-01-01 00:14:11 2020-01-02 03:42:32 2020-01-01 00:14:11.000 2020-01-02 03:42:32.000 851 99752 50301.5 5030150 851 99752 50301.5 5030150 -32717 32218 4426.3 442630 -125 126 0.06 6 -852 100 10842 99753 2.55855 299.55855 151.05855 15105.85585 2.55855 299.55856 151.05856 15105.85607 2.55855 299.55855 151.05854999999997 15105.85500 2020-01-01 2020-01-02 2020-01-01 00:14:12 2020-01-02 03:42:33 2020-01-01 00:14:12.000 2020-01-02 03:42:33.000 852 99753 50302.5 5030250 852 99753 50302.5 5030250 -32716 32219 4427.3 442730 -124 127 1.06 106 +847 100 10837 99748 2.54354 299.54354 151.04354 15104.35435 2.54354 299.54355 151.04354 15104.3546 2.54354 299.54354 151.04354000000006 15104.35400 2020-01-01 2020-01-02 2020-01-01 00:14:07 2020-01-02 03:42:28 2020-01-01 00:14:07.000 2020-01-02 03:42:28.000 847 99748 50297.5 5029750 847 99748 50297.5 5029750 -32721 32214 4422.3 442230 -128 127 -1.38 -138 +848 100 10838 99749 2.54654 299.54654 151.04654 15104.65465 2.54654 299.54654 151.04654 15104.65425 2.54654 299.54654 151.04653999999988 15104.65400 2020-01-01 2020-01-02 2020-01-01 00:14:08 2020-01-02 03:42:29 2020-01-01 00:14:08.000 2020-01-02 03:42:29.000 848 99749 50298.5 5029850 848 99749 50298.5 5029850 -32720 32215 4423.3 442330 -128 123 -2.94 -294 +849 100 10839 99750 2.54954 299.54954 151.04954 15104.95495 2.54954 299.54956 151.04954 15104.95499 2.54954 299.54954 151.04953999999998 15104.95400 2020-01-01 2020-01-02 2020-01-01 00:14:09 2020-01-02 03:42:30 2020-01-01 00:14:09.000 2020-01-02 03:42:30.000 849 99750 50299.5 5029950 849 99750 50299.5 5029950 -32719 32216 4424.3 442430 -127 124 -1.94 -194 +85 102 10075 99985 0.25525 300.25525 150.25525 15175.78078 0.25525 300.25525 150.25525 15175.78046 0.25525 300.25525 150.25525000000013 15175.78025 2020-01-01 2020-01-02 2020-01-01 00:01:25 2020-01-02 03:46:25 2020-01-01 00:01:25.000 2020-01-02 03:46:25.000 85 99985 50035 5053535 85 99985 50035 5053535 -32484 32451 4614.009900990099 466015 -128 127 -1.5940594059405941 -161 +850 100 10840 99751 2.55255 299.55255 151.05255 15105.25525 2.55255 299.55255 151.05255 15105.25514 2.55255 299.55255 151.05255000000022 15105.25500 2020-01-01 2020-01-02 2020-01-01 00:14:10 2020-01-02 03:42:31 2020-01-01 00:14:10.000 2020-01-02 03:42:31.000 850 99751 50300.5 5030050 850 99751 50300.5 5030050 -32718 32217 4425.3 442530 -126 125 -0.94 -94 +851 100 10841 99752 2.55555 299.55555 151.05555 15105.55555 2.55555 299.55554 151.05555 15105.55547 2.55555 299.55555 151.05554999999973 15105.55500 2020-01-01 2020-01-02 2020-01-01 00:14:11 2020-01-02 03:42:32 2020-01-01 00:14:11.000 2020-01-02 03:42:32.000 851 99752 50301.5 5030150 851 99752 50301.5 5030150 -32717 32218 4426.3 442630 -125 126 0.06 6 +852 100 10842 99753 2.55855 299.55855 151.05855 15105.85585 2.55855 299.55856 151.05856 15105.85607 2.55855 299.55855 151.05854999999985 15105.85500 2020-01-01 2020-01-02 2020-01-01 00:14:12 2020-01-02 03:42:33 2020-01-01 00:14:12.000 2020-01-02 03:42:33.000 852 99753 50302.5 5030250 852 99753 50302.5 5030250 -32716 32219 4427.3 442730 -124 127 1.06 106 853 100 10843 99754 2.56156 299.56156 151.06156 15106.15615 2.56156 299.56155 151.06155 15106.15571 2.56156 299.56156 151.06156000000007 15106.15600 2020-01-01 2020-01-02 2020-01-01 00:14:13 2020-01-02 03:42:34 2020-01-01 00:14:13.000 2020-01-02 03:42:34.000 853 99754 50303.5 5030350 853 99754 50303.5 5030350 -32715 32220 4428.3 442830 -128 127 -0.5 -50 -854 100 10844 99755 2.56456 299.56456 151.06456 15106.45645 2.56456 299.56458 151.06456 15106.45646 2.56456 299.56456 151.06456000000003 15106.45600 2020-01-01 2020-01-02 2020-01-01 00:14:14 2020-01-02 03:42:35 2020-01-01 00:14:14.000 2020-01-02 03:42:35.000 854 99755 50304.5 5030450 854 99755 50304.5 5030450 -32714 32221 4429.3 442930 -128 123 -2.06 -206 +854 100 10844 99755 2.56456 299.56456 151.06456 15106.45645 2.56456 299.56458 151.06456 15106.45646 2.56456 299.56456 151.0645600000001 15106.45600 2020-01-01 2020-01-02 2020-01-01 00:14:14 2020-01-02 03:42:35 2020-01-01 00:14:14.000 2020-01-02 03:42:35.000 854 99755 50304.5 5030450 854 99755 50304.5 5030450 -32714 32221 4429.3 442930 -128 123 -2.06 -206 855 100 10845 99756 2.56756 299.56756 151.06756 15106.75675 2.56756 299.56757 151.06756 15106.75661 2.56756 299.56756 151.06756000000001 15106.75600 2020-01-01 2020-01-02 2020-01-01 00:14:15 2020-01-02 03:42:36 2020-01-01 00:14:15.000 2020-01-02 03:42:36.000 855 99756 50305.5 5030550 855 99756 50305.5 5030550 -32713 32222 4430.3 443030 -127 124 -1.06 -106 -856 100 10846 99757 2.57057 299.57057 151.07057 15107.05705 2.57057 299.57056 151.07056 15107.05694 2.57057 299.57057 151.07056999999995 15107.05700 2020-01-01 2020-01-02 2020-01-01 00:14:16 2020-01-02 03:42:37 2020-01-01 00:14:16.000 2020-01-02 03:42:37.000 856 99757 50306.5 5030650 856 99757 50306.5 5030650 -32712 32223 4431.3 443130 -126 125 -0.06 -6 +856 100 10846 99757 2.57057 299.57057 151.07057 15107.05705 2.57057 299.57056 151.07056 15107.05694 2.57057 299.57057 151.0705699999999 15107.05700 2020-01-01 2020-01-02 2020-01-01 00:14:16 2020-01-02 03:42:37 2020-01-01 00:14:16.000 2020-01-02 03:42:37.000 856 99757 50306.5 5030650 856 99757 50306.5 5030650 -32712 32223 4431.3 443130 -126 125 -0.06 -6 857 100 10847 99758 2.57357 299.57357 151.07357 15107.35735 2.57357 299.57358 151.07357 15107.35754 2.57357 299.57357 151.07356999999996 15107.35700 2020-01-01 2020-01-02 2020-01-01 00:14:17 2020-01-02 03:42:38 2020-01-01 00:14:17.000 2020-01-02 03:42:38.000 857 99758 50307.5 5030750 857 99758 50307.5 5030750 -32711 32224 4432.3 443230 -125 126 0.94 94 -858 100 10848 99759 2.57657 299.57657 151.07657 15107.65765 2.57657 299.57657 151.07657 15107.65783 2.57657 299.57657 151.07656999999998 15107.65700 2020-01-01 2020-01-02 2020-01-01 00:14:18 2020-01-02 03:42:39 2020-01-01 00:14:18.000 2020-01-02 03:42:39.000 858 99759 50308.5 5030850 858 99759 50308.5 5030850 -32710 32225 4433.3 443330 -124 127 1.94 194 -859 100 10849 99760 2.57957 299.57957 151.07957 15107.95795 2.57957 299.5796 151.07957 15107.95794 2.57957 299.57957 151.07956999999993 15107.95700 2020-01-01 2020-01-02 2020-01-01 00:14:19 2020-01-02 03:42:40 2020-01-01 00:14:19.000 2020-01-02 03:42:40.000 859 99760 50309.5 5030950 859 99760 50309.5 5030950 -32709 32226 4434.3 443430 -128 127 0.38 38 -86 102 10076 99986 0.25825 300.25825 150.25825 15176.08408 0.25825 300.25827 150.25825 15176.08406 0.25825 300.25825 150.25825000000003 15176.08325 2020-01-01 2020-01-02 2020-01-01 00:01:26 2020-01-02 03:46:26 2020-01-01 00:01:26.000 2020-01-02 03:46:26.000 86 99986 50036 5053636 86 99986 50036 5053636 -32483 32452 4615.009900990099 466116 -128 123 -3.128712871287129 -316 -860 100 10850 99761 2.58258 299.58258 151.08258 15108.25825 2.58258 299.58258 151.08258 15108.25811 2.58258 299.58258 151.08258000000012 15108.25800 2020-01-01 2020-01-02 2020-01-01 00:14:20 2020-01-02 03:42:41 2020-01-01 00:14:20.000 2020-01-02 03:42:41.000 860 99761 50310.5 5031050 860 99761 50310.5 5031050 -32708 32227 4435.3 443530 -128 123 -1.18 -118 -861 100 10851 99762 2.58558 299.58558 151.08558 15108.55855 2.58558 299.58557 151.08558 15108.55841 2.58558 299.58558 151.08558000000005 15108.55800 2020-01-01 2020-01-02 2020-01-01 00:14:21 2020-01-02 03:42:42 2020-01-01 00:14:21.000 2020-01-02 03:42:42.000 861 99762 50311.5 5031150 861 99762 50311.5 5031150 -32707 32228 4436.3 443630 -127 124 -0.18 -18 -862 100 10852 99763 2.58858 299.58858 151.08858 15108.85885 2.58858 299.5886 151.08859 15108.85901 2.58858 299.58858 151.08857999999992 15108.85800 2020-01-01 2020-01-02 2020-01-01 00:14:22 2020-01-02 03:42:43 2020-01-01 00:14:22.000 2020-01-02 03:42:43.000 862 99763 50312.5 5031250 862 99763 50312.5 5031250 -32706 32229 4437.3 443730 -126 125 0.82 82 +858 100 10848 99759 2.57657 299.57657 151.07657 15107.65765 2.57657 299.57657 151.07657 15107.65783 2.57657 299.57657 151.07656999999986 15107.65700 2020-01-01 2020-01-02 2020-01-01 00:14:18 2020-01-02 03:42:39 2020-01-01 00:14:18.000 2020-01-02 03:42:39.000 858 99759 50308.5 5030850 858 99759 50308.5 5030850 -32710 32225 4433.3 443330 -124 127 1.94 194 +859 100 10849 99760 2.57957 299.57957 151.07957 15107.95795 2.57957 299.5796 151.07957 15107.95794 2.57957 299.57957 151.0795699999999 15107.95700 2020-01-01 2020-01-02 2020-01-01 00:14:19 2020-01-02 03:42:40 2020-01-01 00:14:19.000 2020-01-02 03:42:40.000 859 99760 50309.5 5030950 859 99760 50309.5 5030950 -32709 32226 4434.3 443430 -128 127 0.38 38 +86 102 10076 99986 0.25825 300.25825 150.25825 15176.08408 0.25825 300.25827 150.25825 15176.08406 0.25825 300.25825 150.25825000000017 15176.08325 2020-01-01 2020-01-02 2020-01-01 00:01:26 2020-01-02 03:46:26 2020-01-01 00:01:26.000 2020-01-02 03:46:26.000 86 99986 50036 5053636 86 99986 50036 5053636 -32483 32452 4615.009900990099 466116 -128 123 -3.128712871287129 -316 +860 100 10850 99761 2.58258 299.58258 151.08258 15108.25825 2.58258 299.58258 151.08258 15108.25811 2.58258 299.58258 151.08258000000015 15108.25800 2020-01-01 2020-01-02 2020-01-01 00:14:20 2020-01-02 03:42:41 2020-01-01 00:14:20.000 2020-01-02 03:42:41.000 860 99761 50310.5 5031050 860 99761 50310.5 5031050 -32708 32227 4435.3 443530 -128 123 -1.18 -118 +861 100 10851 99762 2.58558 299.58558 151.08558 15108.55855 2.58558 299.58557 151.08558 15108.55841 2.58558 299.58558 151.08558000000025 15108.55800 2020-01-01 2020-01-02 2020-01-01 00:14:21 2020-01-02 03:42:42 2020-01-01 00:14:21.000 2020-01-02 03:42:42.000 861 99762 50311.5 5031150 861 99762 50311.5 5031150 -32707 32228 4436.3 443630 -127 124 -0.18 -18 +862 100 10852 99763 2.58858 299.58858 151.08858 15108.85885 2.58858 299.5886 151.08859 15108.85901 2.58858 299.58858 151.08857999999975 15108.85800 2020-01-01 2020-01-02 2020-01-01 00:14:22 2020-01-02 03:42:43 2020-01-01 00:14:22.000 2020-01-02 03:42:43.000 862 99763 50312.5 5031250 862 99763 50312.5 5031250 -32706 32229 4437.3 443730 -126 125 0.82 82 863 100 10853 99764 2.59159 299.59159 151.09159 15109.15915 2.59159 299.59158 151.09159 15109.1593 2.59159 299.59159 151.09159 15109.15900 2020-01-01 2020-01-02 2020-01-01 00:14:23 2020-01-02 03:42:44 2020-01-01 00:14:23.000 2020-01-02 03:42:44.000 863 99764 50313.5 5031350 863 99764 50313.5 5031350 -32705 32230 4438.3 443830 -125 126 1.82 182 -864 100 10854 99765 2.59459 299.59459 151.09459 15109.45945 2.59459 299.5946 151.09459 15109.45941 2.59459 299.59459 151.09459000000004 15109.45900 2020-01-01 2020-01-02 2020-01-01 00:14:24 2020-01-02 03:42:45 2020-01-01 00:14:24.000 2020-01-02 03:42:45.000 864 99765 50314.5 5031450 864 99765 50314.5 5031450 -32704 32231 4439.3 443930 -124 127 2.82 282 -865 100 10855 99766 2.59759 299.59759 151.09759 15109.75975 2.59759 299.5976 151.09759 15109.75958 2.59759 299.59759 151.09759000000008 15109.75900 2020-01-01 2020-01-02 2020-01-01 00:14:25 2020-01-02 03:42:46 2020-01-01 00:14:25.000 2020-01-02 03:42:46.000 865 99766 50315.5 5031550 865 99766 50315.5 5031550 -32703 32232 4440.3 444030 -128 127 1.26 126 -866 100 10856 99767 2.6006 299.6006 151.1006 15110.06006 2.6006 299.6006 151.10059 15110.05988 2.60060 299.60060 151.10059999999984 15110.06000 2020-01-01 2020-01-02 2020-01-01 00:14:26 2020-01-02 03:42:47 2020-01-01 00:14:26.000 2020-01-02 03:42:47.000 866 99767 50316.5 5031650 866 99767 50316.5 5031650 -32702 32233 4441.3 444130 -128 127 -0.3 -30 -867 100 10857 99768 2.6036 299.6036 151.1036 15110.36036 2.6036 299.6036 151.1036 15110.36063 2.60360 299.60360 151.10360000000026 15110.36000 2020-01-01 2020-01-02 2020-01-01 00:14:27 2020-01-02 03:42:48 2020-01-01 00:14:27.000 2020-01-02 03:42:48.000 867 99768 50317.5 5031750 867 99768 50317.5 5031750 -32701 32234 4442.3 444230 -128 123 -1.86 -186 -868 100 10858 99769 2.6066 299.6066 151.1066 15110.66066 2.6066 299.6066 151.1066 15110.66078 2.60660 299.60660 151.10659999999976 15110.66000 2020-01-01 2020-01-02 2020-01-01 00:14:28 2020-01-02 03:42:49 2020-01-01 00:14:28.000 2020-01-02 03:42:49.000 868 99769 50318.5 5031850 868 99769 50318.5 5031850 -32700 32235 4443.3 444330 -127 124 -0.86 -86 -869 100 10859 99770 2.6096 299.6096 151.1096 15110.96096 2.6096 299.60962 151.1096 15110.96091 2.60960 299.60960 151.1095999999998 15110.96000 2020-01-01 2020-01-02 2020-01-01 00:14:29 2020-01-02 03:42:50 2020-01-01 00:14:29.000 2020-01-02 03:42:50.000 869 99770 50319.5 5031950 869 99770 50319.5 5031950 -32699 32236 4444.3 444430 -126 125 0.14 14 -87 102 10077 99987 0.26126 300.26126 150.26126 15176.38738 0.26126 300.26126 150.26126 15176.38736 0.26126 300.26126 150.2612599999999 15176.38726 2020-01-01 2020-01-02 2020-01-01 00:01:27 2020-01-02 03:46:27 2020-01-01 00:01:27.000 2020-01-02 03:46:27.000 87 99987 50037 5053737 87 99987 50037 5053737 -32482 32453 4616.009900990099 466217 -127 124 -2.128712871287129 -215 -870 100 10860 99771 2.61261 299.61261 151.11261 15111.26126 2.61261 299.6126 151.11261 15111.26105 2.61261 299.61261 151.1126100000001 15111.26100 2020-01-01 2020-01-02 2020-01-01 00:14:30 2020-01-02 03:42:51 2020-01-01 00:14:30.000 2020-01-02 03:42:51.000 870 99771 50320.5 5032050 870 99771 50320.5 5032050 -32698 32237 4445.3 444530 -125 126 1.14 114 -871 100 10861 99772 2.61561 299.61561 151.11561 15111.56156 2.61561 299.6156 151.11561 15111.56135 2.61561 299.61561 151.11561000000017 15111.56100 2020-01-01 2020-01-02 2020-01-01 00:14:31 2020-01-02 03:42:52 2020-01-01 00:14:31.000 2020-01-02 03:42:52.000 871 99772 50321.5 5032150 871 99772 50321.5 5032150 -32697 32238 4446.3 444630 -124 127 2.14 214 -872 100 10862 99773 2.61861 299.61861 151.11861 15111.86186 2.61861 299.61862 151.11862 15111.8621 2.61861 299.61861 151.11860999999993 15111.86100 2020-01-01 2020-01-02 2020-01-01 00:14:32 2020-01-02 03:42:53 2020-01-01 00:14:32.000 2020-01-02 03:42:53.000 872 99773 50322.5 5032250 872 99773 50322.5 5032250 -32696 32239 4447.3 444730 -128 127 0.58 58 +864 100 10854 99765 2.59459 299.59459 151.09459 15109.45945 2.59459 299.5946 151.09459 15109.45941 2.59459 299.59459 151.0945900000001 15109.45900 2020-01-01 2020-01-02 2020-01-01 00:14:24 2020-01-02 03:42:45 2020-01-01 00:14:24.000 2020-01-02 03:42:45.000 864 99765 50314.5 5031450 864 99765 50314.5 5031450 -32704 32231 4439.3 443930 -124 127 2.82 282 +865 100 10855 99766 2.59759 299.59759 151.09759 15109.75975 2.59759 299.5976 151.09759 15109.75958 2.59759 299.59759 151.09759 15109.75900 2020-01-01 2020-01-02 2020-01-01 00:14:25 2020-01-02 03:42:46 2020-01-01 00:14:25.000 2020-01-02 03:42:46.000 865 99766 50315.5 5031550 865 99766 50315.5 5031550 -32703 32232 4440.3 444030 -128 127 1.26 126 +866 100 10856 99767 2.6006 299.6006 151.1006 15110.06006 2.6006 299.6006 151.10059 15110.05988 2.60060 299.60060 151.10059999999987 15110.06000 2020-01-01 2020-01-02 2020-01-01 00:14:26 2020-01-02 03:42:47 2020-01-01 00:14:26.000 2020-01-02 03:42:47.000 866 99767 50316.5 5031650 866 99767 50316.5 5031650 -32702 32233 4441.3 444130 -128 127 -0.3 -30 +867 100 10857 99768 2.6036 299.6036 151.1036 15110.36036 2.6036 299.6036 151.1036 15110.36063 2.60360 299.60360 151.10360000000009 15110.36000 2020-01-01 2020-01-02 2020-01-01 00:14:27 2020-01-02 03:42:48 2020-01-01 00:14:27.000 2020-01-02 03:42:48.000 867 99768 50317.5 5031750 867 99768 50317.5 5031750 -32701 32234 4442.3 444230 -128 123 -1.86 -186 +868 100 10858 99769 2.6066 299.6066 151.1066 15110.66066 2.6066 299.6066 151.1066 15110.66078 2.60660 299.60660 151.10659999999993 15110.66000 2020-01-01 2020-01-02 2020-01-01 00:14:28 2020-01-02 03:42:49 2020-01-01 00:14:28.000 2020-01-02 03:42:49.000 868 99769 50318.5 5031850 868 99769 50318.5 5031850 -32700 32235 4443.3 444330 -127 124 -0.86 -86 +869 100 10859 99770 2.6096 299.6096 151.1096 15110.96096 2.6096 299.60962 151.1096 15110.96091 2.60960 299.60960 151.1095999999999 15110.96000 2020-01-01 2020-01-02 2020-01-01 00:14:29 2020-01-02 03:42:50 2020-01-01 00:14:29.000 2020-01-02 03:42:50.000 869 99770 50319.5 5031950 869 99770 50319.5 5031950 -32699 32236 4444.3 444430 -126 125 0.14 14 +87 102 10077 99987 0.26126 300.26126 150.26126 15176.38738 0.26126 300.26126 150.26126 15176.38736 0.26126 300.26126 150.26126000000002 15176.38726 2020-01-01 2020-01-02 2020-01-01 00:01:27 2020-01-02 03:46:27 2020-01-01 00:01:27.000 2020-01-02 03:46:27.000 87 99987 50037 5053737 87 99987 50037 5053737 -32482 32453 4616.009900990099 466217 -127 124 -2.128712871287129 -215 +870 100 10860 99771 2.61261 299.61261 151.11261 15111.26126 2.61261 299.6126 151.11261 15111.26105 2.61261 299.61261 151.11261000000007 15111.26100 2020-01-01 2020-01-02 2020-01-01 00:14:30 2020-01-02 03:42:51 2020-01-01 00:14:30.000 2020-01-02 03:42:51.000 870 99771 50320.5 5032050 870 99771 50320.5 5032050 -32698 32237 4445.3 444530 -125 126 1.14 114 +871 100 10861 99772 2.61561 299.61561 151.11561 15111.56156 2.61561 299.6156 151.11561 15111.56135 2.61561 299.61561 151.11561000000003 15111.56100 2020-01-01 2020-01-02 2020-01-01 00:14:31 2020-01-02 03:42:52 2020-01-01 00:14:31.000 2020-01-02 03:42:52.000 871 99772 50321.5 5032150 871 99772 50321.5 5032150 -32697 32238 4446.3 444630 -124 127 2.14 214 +872 100 10862 99773 2.61861 299.61861 151.11861 15111.86186 2.61861 299.61862 151.11862 15111.8621 2.61861 299.61861 151.11860999999982 15111.86100 2020-01-01 2020-01-02 2020-01-01 00:14:32 2020-01-02 03:42:53 2020-01-01 00:14:32.000 2020-01-02 03:42:53.000 872 99773 50322.5 5032250 872 99773 50322.5 5032250 -32696 32239 4447.3 444730 -128 127 0.58 58 873 100 10863 99774 2.62162 299.62162 151.12162 15112.16216 2.62162 299.6216 151.12162 15112.16224 2.62162 299.62162 151.12161999999992 15112.16200 2020-01-01 2020-01-02 2020-01-01 00:14:33 2020-01-02 03:42:54 2020-01-01 00:14:33.000 2020-01-02 03:42:54.000 873 99774 50323.5 5032350 873 99774 50323.5 5032350 -32695 32240 4448.3 444830 -128 123 -0.98 -98 -874 100 10864 99775 2.62462 299.62462 151.12462 15112.46246 2.62462 299.62463 151.12462 15112.46238 2.62462 299.62462 151.12462000000002 15112.46200 2020-01-01 2020-01-02 2020-01-01 00:14:34 2020-01-02 03:42:55 2020-01-01 00:14:34.000 2020-01-02 03:42:55.000 874 99775 50324.5 5032450 874 99775 50324.5 5032450 -32694 32241 4449.3 444930 -127 124 0.02 2 -875 100 10865 99776 2.62762 299.62762 151.12762 15112.76276 2.62762 299.62762 151.12762 15112.76252 2.62762 299.62762 151.12761999999987 15112.76200 2020-01-01 2020-01-02 2020-01-01 00:14:35 2020-01-02 03:42:56 2020-01-01 00:14:35.000 2020-01-02 03:42:56.000 875 99776 50325.5 5032550 875 99776 50325.5 5032550 -32693 32242 4450.3 445030 -126 125 1.02 102 -876 100 10866 99777 2.63063 299.63063 151.13063 15113.06306 2.63063 299.63065 151.13063 15113.06327 2.63063 299.63063 151.13062999999977 15113.06300 2020-01-01 2020-01-02 2020-01-01 00:14:36 2020-01-02 03:42:57 2020-01-01 00:14:36.000 2020-01-02 03:42:57.000 876 99777 50326.5 5032650 876 99777 50326.5 5032650 -32692 32243 4451.3 445130 -125 126 2.02 202 -877 100 10867 99778 2.63363 299.63363 151.13363 15113.36336 2.63363 299.63364 151.13363 15113.36358 2.63363 299.63363 151.13363000000015 15113.36300 2020-01-01 2020-01-02 2020-01-01 00:14:37 2020-01-02 03:42:58 2020-01-01 00:14:37.000 2020-01-02 03:42:58.000 877 99778 50327.5 5032750 877 99778 50327.5 5032750 -32691 32244 4452.3 445230 -124 127 3.02 302 -878 100 10868 99779 2.63663 299.63663 151.13663 15113.66366 2.63663 299.63663 151.13663 15113.66371 2.63663 299.63663 151.13663000000028 15113.66300 2020-01-01 2020-01-02 2020-01-01 00:14:38 2020-01-02 03:42:59 2020-01-01 00:14:38.000 2020-01-02 03:42:59.000 878 99779 50328.5 5032850 878 99779 50328.5 5032850 -32690 32245 4453.3 445330 -128 127 1.46 146 -879 100 10869 99780 2.63963 299.63963 151.13963 15113.96396 2.63963 299.63965 151.13963 15113.96385 2.63963 299.63963 151.13962999999978 15113.96300 2020-01-01 2020-01-02 2020-01-01 00:14:39 2020-01-02 03:43:00 2020-01-01 00:14:39.000 2020-01-02 03:43:00.000 879 99780 50329.5 5032950 879 99780 50329.5 5032950 -32689 32246 4454.3 445430 -128 123 -0.1 -10 +874 100 10864 99775 2.62462 299.62462 151.12462 15112.46246 2.62462 299.62463 151.12462 15112.46238 2.62462 299.62462 151.12461999999996 15112.46200 2020-01-01 2020-01-02 2020-01-01 00:14:34 2020-01-02 03:42:55 2020-01-01 00:14:34.000 2020-01-02 03:42:55.000 874 99775 50324.5 5032450 874 99775 50324.5 5032450 -32694 32241 4449.3 444930 -127 124 0.02 2 +875 100 10865 99776 2.62762 299.62762 151.12762 15112.76276 2.62762 299.62762 151.12762 15112.76252 2.62762 299.62762 151.12761999999998 15112.76200 2020-01-01 2020-01-02 2020-01-01 00:14:35 2020-01-02 03:42:56 2020-01-01 00:14:35.000 2020-01-02 03:42:56.000 875 99776 50325.5 5032550 875 99776 50325.5 5032550 -32693 32242 4450.3 445030 -126 125 1.02 102 +876 100 10866 99777 2.63063 299.63063 151.13063 15113.06306 2.63063 299.63065 151.13063 15113.06327 2.63063 299.63063 151.1306299999999 15113.06300 2020-01-01 2020-01-02 2020-01-01 00:14:36 2020-01-02 03:42:57 2020-01-01 00:14:36.000 2020-01-02 03:42:57.000 876 99777 50326.5 5032650 876 99777 50326.5 5032650 -32692 32243 4451.3 445130 -125 126 2.02 202 +877 100 10867 99778 2.63363 299.63363 151.13363 15113.36336 2.63363 299.63364 151.13363 15113.36358 2.63363 299.63363 151.13363000000004 15113.36300 2020-01-01 2020-01-02 2020-01-01 00:14:37 2020-01-02 03:42:58 2020-01-01 00:14:37.000 2020-01-02 03:42:58.000 877 99778 50327.5 5032750 877 99778 50327.5 5032750 -32691 32244 4452.3 445230 -124 127 3.02 302 +878 100 10868 99779 2.63663 299.63663 151.13663 15113.66366 2.63663 299.63663 151.13663 15113.66371 2.63663 299.63663 151.13663000000008 15113.66300 2020-01-01 2020-01-02 2020-01-01 00:14:38 2020-01-02 03:42:59 2020-01-01 00:14:38.000 2020-01-02 03:42:59.000 878 99779 50328.5 5032850 878 99779 50328.5 5032850 -32690 32245 4453.3 445330 -128 127 1.46 146 +879 100 10869 99780 2.63963 299.63963 151.13963 15113.96396 2.63963 299.63965 151.13963 15113.96385 2.63963 299.63963 151.1396299999999 15113.96300 2020-01-01 2020-01-02 2020-01-01 00:14:39 2020-01-02 03:43:00 2020-01-01 00:14:39.000 2020-01-02 03:43:00.000 879 99780 50329.5 5032950 879 99780 50329.5 5032950 -32689 32246 4454.3 445430 -128 123 -0.1 -10 88 102 10078 99988 0.26426 300.26426 150.26426 15176.69069 0.26426 300.26425 150.26426 15176.69066 0.26426 300.26426 150.26425999999995 15176.69026 2020-01-01 2020-01-02 2020-01-01 00:01:28 2020-01-02 03:46:28 2020-01-01 00:01:28.000 2020-01-02 03:46:28.000 88 99988 50038 5053838 88 99988 50038 5053838 -32481 32454 4617.009900990099 466318 -126 125 -1.1287128712871286 -114 880 100 10870 99781 2.64264 299.64264 151.14264 15114.26426 2.64264 299.64264 151.14263 15114.26399 2.64264 299.64264 151.14264 15114.26400 2020-01-01 2020-01-02 2020-01-01 00:14:40 2020-01-02 03:43:01 2020-01-01 00:14:40.000 2020-01-02 03:43:01.000 880 99781 50330.5 5033050 880 99781 50330.5 5033050 -32688 32247 4455.3 445530 -127 124 0.9 90 -881 100 10871 99782 2.64564 299.64564 151.14564 15114.56456 2.64564 299.64566 151.14564 15114.56474 2.64564 299.64564 151.14564000000013 15114.56400 2020-01-01 2020-01-02 2020-01-01 00:14:41 2020-01-02 03:43:02 2020-01-01 00:14:41.000 2020-01-02 03:43:02.000 881 99782 50331.5 5033150 881 99782 50331.5 5033150 -32687 32248 4456.3 445630 -126 125 1.9 190 -882 100 10872 99783 2.64864 299.64864 151.14864 15114.86486 2.64864 299.64865 151.14865 15114.86504 2.64864 299.64864 151.14863999999992 15114.86400 2020-01-01 2020-01-02 2020-01-01 00:14:42 2020-01-02 03:43:03 2020-01-01 00:14:42.000 2020-01-02 03:43:03.000 882 99783 50332.5 5033250 882 99783 50332.5 5033250 -32686 32249 4457.3 445730 -125 126 2.9 290 -883 100 10873 99784 2.65165 299.65165 151.15165 15115.16516 2.65165 299.65164 151.15165 15115.16522 2.65165 299.65165 151.15164999999982 15115.16500 2020-01-01 2020-01-02 2020-01-01 00:14:43 2020-01-02 03:43:04 2020-01-01 00:14:43.000 2020-01-02 03:43:04.000 883 99784 50333.5 5033350 883 99784 50333.5 5033350 -32685 32250 4458.3 445830 -124 127 3.9 390 -884 100 10874 99785 2.65465 299.65465 151.15465 15115.46546 2.65465 299.65466 151.15465 15115.46532 2.65465 299.65465 151.1546500000003 15115.46500 2020-01-01 2020-01-02 2020-01-01 00:14:44 2020-01-02 03:43:05 2020-01-01 00:14:44.000 2020-01-02 03:43:05.000 884 99785 50334.5 5033450 884 99785 50334.5 5033450 -32684 32251 4459.3 445930 -128 127 2.34 234 -885 100 10875 99786 2.65765 299.65765 151.15765 15115.76576 2.65765 299.65765 151.15765 15115.76562 2.65765 299.65765 151.1576499999998 15115.76500 2020-01-01 2020-01-02 2020-01-01 00:14:45 2020-01-02 03:43:06 2020-01-01 00:14:45.000 2020-01-02 03:43:06.000 885 99786 50335.5 5033550 885 99786 50335.5 5033550 -32683 32252 4460.3 446030 -128 123 0.78 78 -886 100 10876 99787 2.66066 299.66066 151.16066 15116.06606 2.66066 299.66068 151.16066 15116.06621 2.66066 299.66066 151.16065999999967 15116.06600 2020-01-01 2020-01-02 2020-01-01 00:14:46 2020-01-02 03:43:07 2020-01-01 00:14:46.000 2020-01-02 03:43:07.000 886 99787 50336.5 5033650 886 99787 50336.5 5033650 -32682 32253 4461.3 446130 -127 124 1.78 178 -887 100 10877 99788 2.66366 299.66366 151.16366 15116.36636 2.66366 299.66367 151.16366 15116.36651 2.66366 299.66366 151.16366000000014 15116.36600 2020-01-01 2020-01-02 2020-01-01 00:14:47 2020-01-02 03:43:08 2020-01-01 00:14:47.000 2020-01-02 03:43:08.000 887 99788 50337.5 5033750 887 99788 50337.5 5033750 -32681 32254 4462.3 446230 -126 125 2.78 278 -888 100 10878 99789 2.66666 299.66666 151.16666 15116.66666 2.66666 299.66666 151.16666 15116.66669 2.66666 299.66666 151.16666000000018 15116.66600 2020-01-01 2020-01-02 2020-01-01 00:14:48 2020-01-02 03:43:09 2020-01-01 00:14:48.000 2020-01-02 03:43:09.000 888 99789 50338.5 5033850 888 99789 50338.5 5033850 -32680 32255 4463.3 446330 -125 126 3.78 378 -889 100 10879 99790 2.66966 299.66966 151.16966 15116.96696 2.66966 299.66968 151.16966 15116.96679 2.66966 299.66966 151.16965999999968 15116.96600 2020-01-01 2020-01-02 2020-01-01 00:14:49 2020-01-02 03:43:10 2020-01-01 00:14:49.000 2020-01-02 03:43:10.000 889 99790 50339.5 5033950 889 99790 50339.5 5033950 -32679 32256 4464.3 446430 -124 127 4.78 478 +881 100 10871 99782 2.64564 299.64564 151.14564 15114.56456 2.64564 299.64566 151.14564 15114.56474 2.64564 299.64564 151.14564000000007 15114.56400 2020-01-01 2020-01-02 2020-01-01 00:14:41 2020-01-02 03:43:02 2020-01-01 00:14:41.000 2020-01-02 03:43:02.000 881 99782 50331.5 5033150 881 99782 50331.5 5033150 -32687 32248 4456.3 445630 -126 125 1.9 190 +882 100 10872 99783 2.64864 299.64864 151.14864 15114.86486 2.64864 299.64865 151.14865 15114.86504 2.64864 299.64864 151.14863999999997 15114.86400 2020-01-01 2020-01-02 2020-01-01 00:14:42 2020-01-02 03:43:03 2020-01-01 00:14:42.000 2020-01-02 03:43:03.000 882 99783 50332.5 5033250 882 99783 50332.5 5033250 -32686 32249 4457.3 445730 -125 126 2.9 290 +883 100 10873 99784 2.65165 299.65165 151.15165 15115.16516 2.65165 299.65164 151.15165 15115.16522 2.65165 299.65165 151.1516499999999 15115.16500 2020-01-01 2020-01-02 2020-01-01 00:14:43 2020-01-02 03:43:04 2020-01-01 00:14:43.000 2020-01-02 03:43:04.000 883 99784 50333.5 5033350 883 99784 50333.5 5033350 -32685 32250 4458.3 445830 -124 127 3.9 390 +884 100 10874 99785 2.65465 299.65465 151.15465 15115.46546 2.65465 299.65466 151.15465 15115.46532 2.65465 299.65465 151.15465000000012 15115.46500 2020-01-01 2020-01-02 2020-01-01 00:14:44 2020-01-02 03:43:05 2020-01-01 00:14:44.000 2020-01-02 03:43:05.000 884 99785 50334.5 5033450 884 99785 50334.5 5033450 -32684 32251 4459.3 445930 -128 127 2.34 234 +885 100 10875 99786 2.65765 299.65765 151.15765 15115.76576 2.65765 299.65765 151.15765 15115.76562 2.65765 299.65765 151.15764999999996 15115.76500 2020-01-01 2020-01-02 2020-01-01 00:14:45 2020-01-02 03:43:06 2020-01-01 00:14:45.000 2020-01-02 03:43:06.000 885 99786 50335.5 5033550 885 99786 50335.5 5033550 -32683 32252 4460.3 446030 -128 123 0.78 78 +886 100 10876 99787 2.66066 299.66066 151.16066 15116.06606 2.66066 299.66068 151.16066 15116.06621 2.66066 299.66066 151.16065999999984 15116.06600 2020-01-01 2020-01-02 2020-01-01 00:14:46 2020-01-02 03:43:07 2020-01-01 00:14:46.000 2020-01-02 03:43:07.000 886 99787 50336.5 5033650 886 99787 50336.5 5033650 -32682 32253 4461.3 446130 -127 124 1.78 178 +887 100 10877 99788 2.66366 299.66366 151.16366 15116.36636 2.66366 299.66367 151.16366 15116.36651 2.66366 299.66366 151.16366000000008 15116.36600 2020-01-01 2020-01-02 2020-01-01 00:14:47 2020-01-02 03:43:08 2020-01-01 00:14:47.000 2020-01-02 03:43:08.000 887 99788 50337.5 5033750 887 99788 50337.5 5033750 -32681 32254 4462.3 446230 -126 125 2.78 278 +888 100 10878 99789 2.66666 299.66666 151.16666 15116.66666 2.66666 299.66666 151.16666 15116.66669 2.66666 299.66666 151.16666000000004 15116.66600 2020-01-01 2020-01-02 2020-01-01 00:14:48 2020-01-02 03:43:09 2020-01-01 00:14:48.000 2020-01-02 03:43:09.000 888 99789 50338.5 5033850 888 99789 50338.5 5033850 -32680 32255 4463.3 446330 -125 126 3.78 378 +889 100 10879 99790 2.66966 299.66966 151.16966 15116.96696 2.66966 299.66968 151.16966 15116.96679 2.66966 299.66966 151.16965999999985 15116.96600 2020-01-01 2020-01-02 2020-01-01 00:14:49 2020-01-02 03:43:10 2020-01-01 00:14:49.000 2020-01-02 03:43:10.000 889 99790 50339.5 5033950 889 99790 50339.5 5033950 -32679 32256 4464.3 446430 -124 127 4.78 478 89 102 10079 99989 0.26726 300.26726 150.26726 15176.99399 0.26726 300.26727 150.26727 15176.9943 0.26726 300.26726 150.26726000000002 15176.99326 2020-01-01 2020-01-02 2020-01-01 00:01:29 2020-01-02 03:46:29 2020-01-01 00:01:29.000 2020-01-02 03:46:29.000 89 99989 50039 5053939 89 99989 50039 5053939 -32480 32455 4618.009900990099 466419 -125 126 -0.12871287128712872 -13 890 100 10880 99791 2.67267 299.67267 151.17267 15117.26726 2.67267 299.67267 151.17267 15117.26708 2.67267 299.67267 151.17266999999995 15117.26700 2020-01-01 2020-01-02 2020-01-01 00:14:50 2020-01-02 03:43:11 2020-01-01 00:14:50.000 2020-01-02 03:43:11.000 890 99791 50340.5 5034050 890 99791 50340.5 5034050 -32678 32257 4465.3 446530 -128 127 3.22 322 -891 100 10881 99792 2.67567 299.67567 151.17567 15117.56756 2.67567 299.6757 151.17567 15117.56768 2.67567 299.67567 151.17567000000005 15117.56700 2020-01-01 2020-01-02 2020-01-01 00:14:51 2020-01-02 03:43:12 2020-01-01 00:14:51.000 2020-01-02 03:43:12.000 891 99792 50341.5 5034150 891 99792 50341.5 5034150 -32677 32258 4466.3 446630 -128 127 1.66 166 -892 100 10882 99793 2.67867 299.67867 151.17867 15117.86786 2.67867 299.67868 151.17868 15117.86802 2.67867 299.67867 151.17866999999987 15117.86700 2020-01-01 2020-01-02 2020-01-01 00:14:52 2020-01-02 03:43:13 2020-01-01 00:14:52.000 2020-01-02 03:43:13.000 892 99793 50342.5 5034250 892 99793 50342.5 5034250 -32676 32259 4467.3 446730 -128 124 0.1 10 -893 100 10883 99794 2.68168 299.68168 151.18168 15118.16816 2.68168 299.68167 151.18168 15118.16816 2.68168 299.68168 151.1816799999998 15118.16800 2020-01-01 2020-01-02 2020-01-01 00:14:53 2020-01-02 03:43:14 2020-01-01 00:14:53.000 2020-01-02 03:43:14.000 893 99794 50343.5 5034350 893 99794 50343.5 5034350 -32675 32260 4468.3 446830 -127 125 1.1 110 -894 100 10884 99795 2.68468 299.68468 151.18468 15118.46846 2.68468 299.6847 151.18468 15118.46826 2.68468 299.68468 151.18468000000018 15118.46800 2020-01-01 2020-01-02 2020-01-01 00:14:54 2020-01-02 03:43:15 2020-01-01 00:14:54.000 2020-01-02 03:43:15.000 894 99795 50344.5 5034450 894 99795 50344.5 5034450 -32674 32261 4469.3 446930 -126 126 2.1 210 -895 100 10885 99796 2.68768 299.68768 151.18768 15118.76876 2.68768 299.68768 151.18768 15118.76855 2.68768 299.68768 151.18767999999972 15118.76800 2020-01-01 2020-01-02 2020-01-01 00:14:55 2020-01-02 03:43:16 2020-01-01 00:14:55.000 2020-01-02 03:43:16.000 895 99796 50345.5 5034550 895 99796 50345.5 5034550 -32673 32262 4470.3 447030 -125 127 3.1 310 -896 100 10886 99797 2.69069 299.69069 151.19069 15119.06906 2.69069 299.6907 151.19069 15119.06915 2.69069 299.69069 151.19068999999993 15119.06900 2020-01-01 2020-01-02 2020-01-01 00:14:56 2020-01-02 03:43:17 2020-01-01 00:14:56.000 2020-01-02 03:43:17.000 896 99797 50346.5 5034650 896 99797 50346.5 5034650 -32672 32263 4471.3 447130 -128 127 1.54 154 +891 100 10881 99792 2.67567 299.67567 151.17567 15117.56756 2.67567 299.6757 151.17567 15117.56768 2.67567 299.67567 151.17567 15117.56700 2020-01-01 2020-01-02 2020-01-01 00:14:51 2020-01-02 03:43:12 2020-01-01 00:14:51.000 2020-01-02 03:43:12.000 891 99792 50341.5 5034150 891 99792 50341.5 5034150 -32677 32258 4466.3 446630 -128 127 1.66 166 +892 100 10882 99793 2.67867 299.67867 151.17867 15117.86786 2.67867 299.67868 151.17868 15117.86802 2.67867 299.67867 151.17867 15117.86700 2020-01-01 2020-01-02 2020-01-01 00:14:52 2020-01-02 03:43:13 2020-01-01 00:14:52.000 2020-01-02 03:43:13.000 892 99793 50342.5 5034250 892 99793 50342.5 5034250 -32676 32259 4467.3 446730 -128 124 0.1 10 +893 100 10883 99794 2.68168 299.68168 151.18168 15118.16816 2.68168 299.68167 151.18168 15118.16816 2.68168 299.68168 151.18167999999991 15118.16800 2020-01-01 2020-01-02 2020-01-01 00:14:53 2020-01-02 03:43:14 2020-01-01 00:14:53.000 2020-01-02 03:43:14.000 893 99794 50343.5 5034350 893 99794 50343.5 5034350 -32675 32260 4468.3 446830 -127 125 1.1 110 +894 100 10884 99795 2.68468 299.68468 151.18468 15118.46846 2.68468 299.6847 151.18468 15118.46826 2.68468 299.68468 151.18468000000007 15118.46800 2020-01-01 2020-01-02 2020-01-01 00:14:54 2020-01-02 03:43:15 2020-01-01 00:14:54.000 2020-01-02 03:43:15.000 894 99795 50344.5 5034450 894 99795 50344.5 5034450 -32674 32261 4469.3 446930 -126 126 2.1 210 +895 100 10885 99796 2.68768 299.68768 151.18768 15118.76876 2.68768 299.68768 151.18768 15118.76855 2.68768 299.68768 151.18767999999992 15118.76800 2020-01-01 2020-01-02 2020-01-01 00:14:55 2020-01-02 03:43:16 2020-01-01 00:14:55.000 2020-01-02 03:43:16.000 895 99796 50345.5 5034550 895 99796 50345.5 5034550 -32673 32262 4470.3 447030 -125 127 3.1 310 +896 100 10886 99797 2.69069 299.69069 151.19069 15119.06906 2.69069 299.6907 151.19069 15119.06915 2.69069 299.69069 151.19069 15119.06900 2020-01-01 2020-01-02 2020-01-01 00:14:56 2020-01-02 03:43:17 2020-01-01 00:14:56.000 2020-01-02 03:43:17.000 896 99797 50346.5 5034650 896 99797 50346.5 5034650 -32672 32263 4471.3 447130 -128 127 1.54 154 897 100 10887 99798 2.69369 299.69369 151.19369 15119.36936 2.69369 299.6937 151.19369 15119.36949 2.69369 299.69369 151.19369000000003 15119.36900 2020-01-01 2020-01-02 2020-01-01 00:14:57 2020-01-02 03:43:18 2020-01-01 00:14:57.000 2020-01-02 03:43:18.000 897 99798 50347.5 5034750 897 99798 50347.5 5034750 -32671 32264 4472.3 447230 -128 127 -0.02 -2 -898 100 10888 99799 2.69669 299.69669 151.19669 15119.66966 2.69669 299.6967 151.19669 15119.66963 2.69669 299.69669 151.19669000000013 15119.66900 2020-01-01 2020-01-02 2020-01-01 00:14:58 2020-01-02 03:43:19 2020-01-01 00:14:58.000 2020-01-02 03:43:19.000 898 99799 50348.5 5034850 898 99799 50348.5 5034850 -32670 32265 4473.3 447330 -128 123 -1.58 -158 -899 100 10889 99800 2.69969 299.69969 151.19969 15119.96996 2.69969 299.6997 151.1997 15119.97038 2.69969 299.69969 151.19968999999995 15119.96900 2020-01-01 2020-01-02 2020-01-01 00:14:59 2020-01-02 03:43:20 2020-01-01 00:14:59.000 2020-01-02 03:43:20.000 899 99800 50349.5 5034950 899 99800 50349.5 5034950 -32669 32266 4474.3 447430 -127 124 -0.58 -58 -9 102 1008 9999 0.02702 300.02702 150.02702 15152.72972 0.02702 300.02704 150.02702 15152.72966 0.02702 300.02702 150.02701999999977 15152.72902 2020-01-01 2020-01-02 2020-01-01 00:00:09 2020-01-02 03:45:09 2020-01-01 00:00:09.000 2020-01-02 03:45:09.000 9 99909 49959 5045859 9 99909 49959 5045859 -32560 32375 4538.009900990099 458339 -124 127 0.9801980198019802 99 -90 102 10080 99990 0.27027 300.27027 150.27027 15177.29729 0.27027 300.27026 150.27026 15177.29694 0.27027 300.27027 150.27027000000027 15177.29727 2020-01-01 2020-01-02 2020-01-01 00:01:30 2020-01-02 03:46:30 2020-01-01 00:01:30.000 2020-01-02 03:46:30.000 90 99990 50040 5054040 90 99990 50040 5054040 -32479 32456 4619.009900990099 466520 -124 127 0.8712871287128713 88 -900 100 10890 99801 2.7027 299.7027 151.2027 15120.27027 2.7027 299.7027 151.2027 15120.27003 2.70270 299.70270 151.20269999999985 15120.27000 2020-01-01 2020-01-02 2020-01-01 00:15:00 2020-01-02 03:43:21 2020-01-01 00:15:00.000 2020-01-02 03:43:21.000 900 99801 50350.5 5035050 900 99801 50350.5 5035050 -32668 32267 4475.3 447530 -126 125 0.42 42 -901 100 10891 99802 2.7057 299.7057 151.2057 15120.57057 2.7057 299.70572 151.2057 15120.57066 2.70570 299.70570 151.20570000000032 15120.57000 2020-01-01 2020-01-02 2020-01-01 00:15:01 2020-01-02 03:43:22 2020-01-01 00:15:01.000 2020-01-02 03:43:22.000 901 99802 50351.5 5035150 901 99802 50351.5 5035150 -32667 32268 4476.3 447630 -125 126 1.42 142 -902 100 10892 99803 2.7087 299.7087 151.2087 15120.87087 2.7087 299.7087 151.2087 15120.87095 2.70870 299.70870 151.20869999999982 15120.87000 2020-01-01 2020-01-02 2020-01-01 00:15:02 2020-01-02 03:43:23 2020-01-01 00:15:02.000 2020-01-02 03:43:23.000 902 99803 50352.5 5035250 902 99803 50352.5 5035250 -32666 32269 4477.3 447730 -124 127 2.42 242 -903 100 10893 99804 2.71171 299.71171 151.21171 15121.17117 2.71171 299.7117 151.21171 15121.1711 2.71171 299.71171 151.2117099999997 15121.17100 2020-01-01 2020-01-02 2020-01-01 00:15:03 2020-01-02 03:43:24 2020-01-01 00:15:03.000 2020-01-02 03:43:24.000 903 99804 50353.5 5035350 903 99804 50353.5 5035350 -32665 32270 4478.3 447830 -128 127 0.86 86 -904 100 10894 99805 2.71471 299.71471 151.21471 15121.47147 2.71471 299.71472 151.21471 15121.47185 2.71471 299.71471 151.21471000000017 15121.47100 2020-01-01 2020-01-02 2020-01-01 00:15:04 2020-01-02 03:43:25 2020-01-01 00:15:04.000 2020-01-02 03:43:25.000 904 99805 50354.5 5035450 904 99805 50354.5 5035450 -32664 32271 4479.3 447930 -128 123 -0.7 -70 -905 100 10895 99806 2.71771 299.71771 151.21771 15121.77177 2.71771 299.7177 151.21771 15121.77149 2.71771 299.71771 151.2177100000002 15121.77100 2020-01-01 2020-01-02 2020-01-01 00:15:05 2020-01-02 03:43:26 2020-01-01 00:15:05.000 2020-01-02 03:43:26.000 905 99806 50355.5 5035550 905 99806 50355.5 5035550 -32663 32272 4480.3 448030 -127 124 0.3 30 -906 100 10896 99807 2.72072 299.72072 151.22072 15122.07207 2.72072 299.72073 151.22072 15122.07212 2.72072 299.72072 151.22071999999986 15122.07200 2020-01-01 2020-01-02 2020-01-01 00:15:06 2020-01-02 03:43:27 2020-01-01 00:15:06.000 2020-01-02 03:43:27.000 906 99807 50356.5 5035650 906 99807 50356.5 5035650 -32662 32273 4481.3 448130 -126 125 1.3 130 +898 100 10888 99799 2.69669 299.69669 151.19669 15119.66966 2.69669 299.6967 151.19669 15119.66963 2.69669 299.69669 151.19669 15119.66900 2020-01-01 2020-01-02 2020-01-01 00:14:58 2020-01-02 03:43:19 2020-01-01 00:14:58.000 2020-01-02 03:43:19.000 898 99799 50348.5 5034850 898 99799 50348.5 5034850 -32670 32265 4473.3 447330 -128 123 -1.58 -158 +899 100 10889 99800 2.69969 299.69969 151.19969 15119.96996 2.69969 299.6997 151.1997 15119.97038 2.69969 299.69969 151.19969 15119.96900 2020-01-01 2020-01-02 2020-01-01 00:14:59 2020-01-02 03:43:20 2020-01-01 00:14:59.000 2020-01-02 03:43:20.000 899 99800 50349.5 5034950 899 99800 50349.5 5034950 -32669 32266 4474.3 447430 -127 124 -0.58 -58 +9 102 1008 9999 0.02702 300.02702 150.02702 15152.72972 0.02702 300.02704 150.02702 15152.72966 0.02702 300.02702 150.02701999999985 15152.72902 2020-01-01 2020-01-02 2020-01-01 00:00:09 2020-01-02 03:45:09 2020-01-01 00:00:09.000 2020-01-02 03:45:09.000 9 99909 49959 5045859 9 99909 49959 5045859 -32560 32375 4538.009900990099 458339 -124 127 0.9801980198019802 99 +90 102 10080 99990 0.27027 300.27027 150.27027 15177.29729 0.27027 300.27026 150.27026 15177.29694 0.27027 300.27027 150.27027000000007 15177.29727 2020-01-01 2020-01-02 2020-01-01 00:01:30 2020-01-02 03:46:30 2020-01-01 00:01:30.000 2020-01-02 03:46:30.000 90 99990 50040 5054040 90 99990 50040 5054040 -32479 32456 4619.009900990099 466520 -124 127 0.8712871287128713 88 +900 100 10890 99801 2.7027 299.7027 151.2027 15120.27027 2.7027 299.7027 151.2027 15120.27003 2.70270 299.70270 151.2026999999999 15120.27000 2020-01-01 2020-01-02 2020-01-01 00:15:00 2020-01-02 03:43:21 2020-01-01 00:15:00.000 2020-01-02 03:43:21.000 900 99801 50350.5 5035050 900 99801 50350.5 5035050 -32668 32267 4475.3 447530 -126 125 0.42 42 +901 100 10891 99802 2.7057 299.7057 151.2057 15120.57057 2.7057 299.70572 151.2057 15120.57066 2.70570 299.70570 151.20570000000015 15120.57000 2020-01-01 2020-01-02 2020-01-01 00:15:01 2020-01-02 03:43:22 2020-01-01 00:15:01.000 2020-01-02 03:43:22.000 901 99802 50351.5 5035150 901 99802 50351.5 5035150 -32667 32268 4476.3 447630 -125 126 1.42 142 +902 100 10892 99803 2.7087 299.7087 151.2087 15120.87087 2.7087 299.7087 151.2087 15120.87095 2.70870 299.70870 151.20869999999996 15120.87000 2020-01-01 2020-01-02 2020-01-01 00:15:02 2020-01-02 03:43:23 2020-01-01 00:15:02.000 2020-01-02 03:43:23.000 902 99803 50352.5 5035250 902 99803 50352.5 5035250 -32666 32269 4477.3 447730 -124 127 2.42 242 +903 100 10893 99804 2.71171 299.71171 151.21171 15121.17117 2.71171 299.7117 151.21171 15121.1711 2.71171 299.71171 151.21170999999987 15121.17100 2020-01-01 2020-01-02 2020-01-01 00:15:03 2020-01-02 03:43:24 2020-01-01 00:15:03.000 2020-01-02 03:43:24.000 903 99804 50353.5 5035350 903 99804 50353.5 5035350 -32665 32270 4478.3 447830 -128 127 0.86 86 +904 100 10894 99805 2.71471 299.71471 151.21471 15121.47147 2.71471 299.71472 151.21471 15121.47185 2.71471 299.71471 151.21471000000008 15121.47100 2020-01-01 2020-01-02 2020-01-01 00:15:04 2020-01-02 03:43:25 2020-01-01 00:15:04.000 2020-01-02 03:43:25.000 904 99805 50354.5 5035450 904 99805 50354.5 5035450 -32664 32271 4479.3 447930 -128 123 -0.7 -70 +905 100 10895 99806 2.71771 299.71771 151.21771 15121.77177 2.71771 299.7177 151.21771 15121.77149 2.71771 299.71771 151.21771000000004 15121.77100 2020-01-01 2020-01-02 2020-01-01 00:15:05 2020-01-02 03:43:26 2020-01-01 00:15:05.000 2020-01-02 03:43:26.000 905 99806 50355.5 5035550 905 99806 50355.5 5035550 -32663 32272 4480.3 448030 -127 124 0.3 30 +906 100 10896 99807 2.72072 299.72072 151.22072 15122.07207 2.72072 299.72073 151.22072 15122.07212 2.72072 299.72072 151.22071999999991 15122.07200 2020-01-01 2020-01-02 2020-01-01 00:15:06 2020-01-02 03:43:27 2020-01-01 00:15:06.000 2020-01-02 03:43:27.000 906 99807 50356.5 5035650 906 99807 50356.5 5035650 -32662 32273 4481.3 448130 -126 125 1.3 130 907 100 10897 99808 2.72372 299.72372 151.22372 15122.37237 2.72372 299.72372 151.22372 15122.37243 2.72372 299.72372 151.22372 15122.37200 2020-01-01 2020-01-02 2020-01-01 00:15:07 2020-01-02 03:43:28 2020-01-01 00:15:07.000 2020-01-02 03:43:28.000 907 99808 50357.5 5035750 907 99808 50357.5 5035750 -32661 32274 4482.3 448230 -125 126 2.3 230 -908 100 10898 99809 2.72672 299.72672 151.22672 15122.67267 2.72672 299.7267 151.22672 15122.67272 2.72672 299.72672 151.22672000000009 15122.67200 2020-01-01 2020-01-02 2020-01-01 00:15:08 2020-01-02 03:43:29 2020-01-01 00:15:08.000 2020-01-02 03:43:29.000 908 99809 50358.5 5035850 908 99809 50358.5 5035850 -32660 32275 4483.3 448330 -124 127 3.3 330 -909 100 10899 99810 2.72972 299.72972 151.22972 15122.97297 2.72972 299.72974 151.22973 15122.97332 2.72972 299.72972 151.22971999999987 15122.97200 2020-01-01 2020-01-02 2020-01-01 00:15:09 2020-01-02 03:43:30 2020-01-01 00:15:09.000 2020-01-02 03:43:30.000 909 99810 50359.5 5035950 909 99810 50359.5 5035950 -32659 32276 4484.3 448430 -128 127 1.74 174 -91 102 10081 99991 0.27327 300.27327 150.27327 15177.6006 0.27327 300.2733 150.27327 15177.60054 0.27327 300.27327 150.2732699999998 15177.60027 2020-01-01 2020-01-02 2020-01-01 00:01:31 2020-01-02 03:46:31 2020-01-01 00:01:31.000 2020-01-02 03:46:31.000 91 99991 50041 5054141 91 99991 50041 5054141 -32478 32457 4620.009900990099 466621 -128 127 -0.6633663366336634 -67 -910 100 10900 99811 2.73273 299.73273 151.23273 15123.27327 2.73273 299.73273 151.23272 15123.27296 2.73273 299.73273 151.23272999999983 15123.27300 2020-01-01 2020-01-02 2020-01-01 00:15:10 2020-01-02 03:43:31 2020-01-01 00:15:10.000 2020-01-02 03:43:31.000 910 99811 50360.5 5036050 910 99811 50360.5 5036050 -32658 32277 4485.3 448530 -128 123 0.18 18 -911 100 10901 99812 2.73573 299.73573 151.23573 15123.57357 2.73573 299.73575 151.23573 15123.57359 2.73573 299.73573 151.23573000000022 15123.57300 2020-01-01 2020-01-02 2020-01-01 00:15:11 2020-01-02 03:43:32 2020-01-01 00:15:11.000 2020-01-02 03:43:32.000 911 99812 50361.5 5036150 911 99812 50361.5 5036150 -32657 32278 4486.3 448630 -127 124 1.18 118 -912 100 10902 99813 2.73873 299.73873 151.23873 15123.87387 2.73873 299.73874 151.23873 15123.8739 2.73873 299.73873 151.23872999999972 15123.87300 2020-01-01 2020-01-02 2020-01-01 00:15:12 2020-01-02 03:43:33 2020-01-01 00:15:12.000 2020-01-02 03:43:33.000 912 99813 50362.5 5036250 912 99813 50362.5 5036250 -32656 32279 4487.3 448730 -126 125 2.18 218 -913 100 10903 99814 2.74174 299.74174 151.24174 15124.17417 2.74174 299.74173 151.24174 15124.17419 2.74174 299.74174 151.24173999999996 15124.17400 2020-01-01 2020-01-02 2020-01-01 00:15:13 2020-01-02 03:43:34 2020-01-01 00:15:13.000 2020-01-02 03:43:34.000 913 99814 50363.5 5036350 913 99814 50363.5 5036350 -32655 32280 4488.3 448830 -125 126 3.18 318 +908 100 10898 99809 2.72672 299.72672 151.22672 15122.67267 2.72672 299.7267 151.22672 15122.67272 2.72672 299.72672 151.22672000000003 15122.67200 2020-01-01 2020-01-02 2020-01-01 00:15:08 2020-01-02 03:43:29 2020-01-01 00:15:08.000 2020-01-02 03:43:29.000 908 99809 50358.5 5035850 908 99809 50358.5 5035850 -32660 32275 4483.3 448330 -124 127 3.3 330 +909 100 10899 99810 2.72972 299.72972 151.22972 15122.97297 2.72972 299.72974 151.22973 15122.97332 2.72972 299.72972 151.22971999999993 15122.97200 2020-01-01 2020-01-02 2020-01-01 00:15:09 2020-01-02 03:43:30 2020-01-01 00:15:09.000 2020-01-02 03:43:30.000 909 99810 50359.5 5035950 909 99810 50359.5 5035950 -32659 32276 4484.3 448430 -128 127 1.74 174 +91 102 10081 99991 0.27327 300.27327 150.27327 15177.6006 0.27327 300.2733 150.27327 15177.60054 0.27327 300.27327 150.2732699999999 15177.60027 2020-01-01 2020-01-02 2020-01-01 00:01:31 2020-01-02 03:46:31 2020-01-01 00:01:31.000 2020-01-02 03:46:31.000 91 99991 50041 5054141 91 99991 50041 5054141 -32478 32457 4620.009900990099 466621 -128 127 -0.6633663366336634 -67 +910 100 10900 99811 2.73273 299.73273 151.23273 15123.27327 2.73273 299.73273 151.23272 15123.27296 2.73273 299.73273 151.23272999999995 15123.27300 2020-01-01 2020-01-02 2020-01-01 00:15:10 2020-01-02 03:43:31 2020-01-01 00:15:10.000 2020-01-02 03:43:31.000 910 99811 50360.5 5036050 910 99811 50360.5 5036050 -32658 32277 4485.3 448530 -128 123 0.18 18 +911 100 10901 99812 2.73573 299.73573 151.23573 15123.57357 2.73573 299.73575 151.23573 15123.57359 2.73573 299.73573 151.2357300000001 15123.57300 2020-01-01 2020-01-02 2020-01-01 00:15:11 2020-01-02 03:43:32 2020-01-01 00:15:11.000 2020-01-02 03:43:32.000 911 99812 50361.5 5036150 911 99812 50361.5 5036150 -32657 32278 4486.3 448630 -127 124 1.18 118 +912 100 10902 99813 2.73873 299.73873 151.23873 15123.87387 2.73873 299.73874 151.23873 15123.8739 2.73873 299.73873 151.23872999999992 15123.87300 2020-01-01 2020-01-02 2020-01-01 00:15:12 2020-01-02 03:43:33 2020-01-01 00:15:12.000 2020-01-02 03:43:33.000 912 99813 50362.5 5036250 912 99813 50362.5 5036250 -32656 32279 4487.3 448730 -126 125 2.18 218 +913 100 10903 99814 2.74174 299.74174 151.24174 15124.17417 2.74174 299.74173 151.24174 15124.17419 2.74174 299.74174 151.24174000000002 15124.17400 2020-01-01 2020-01-02 2020-01-01 00:15:13 2020-01-02 03:43:34 2020-01-01 00:15:13.000 2020-01-02 03:43:34.000 913 99814 50363.5 5036350 913 99814 50363.5 5036350 -32655 32280 4488.3 448830 -125 126 3.18 318 914 100 10904 99815 2.74474 299.74474 151.24474 15124.47447 2.74474 299.74475 151.24474 15124.47479 2.74474 299.74474 151.24474000000006 15124.47400 2020-01-01 2020-01-02 2020-01-01 00:15:14 2020-01-02 03:43:35 2020-01-01 00:15:14.000 2020-01-02 03:43:35.000 914 99815 50364.5 5036450 914 99815 50364.5 5036450 -32654 32281 4489.3 448930 -124 127 4.18 418 -915 100 10905 99816 2.74774 299.74774 151.24774 15124.77477 2.74774 299.74774 151.24774 15124.77447 2.74774 299.74774 151.24774000000014 15124.77400 2020-01-01 2020-01-02 2020-01-01 00:15:15 2020-01-02 03:43:36 2020-01-01 00:15:15.000 2020-01-02 03:43:36.000 915 99816 50365.5 5036550 915 99816 50365.5 5036550 -32653 32282 4490.3 449030 -128 127 2.62 262 -916 100 10906 99817 2.75075 299.75075 151.25075 15125.07507 2.75075 299.75076 151.25075 15125.07507 2.75075 299.75075 151.2507499999998 15125.07500 2020-01-01 2020-01-02 2020-01-01 00:15:16 2020-01-02 03:43:37 2020-01-01 00:15:16.000 2020-01-02 03:43:37.000 916 99817 50366.5 5036650 916 99817 50366.5 5036650 -32652 32283 4491.3 449130 -128 127 1.06 106 -917 100 10907 99818 2.75375 299.75375 151.25375 15125.37537 2.75375 299.75375 151.25375 15125.37536 2.75375 299.75375 151.25374999999988 15125.37500 2020-01-01 2020-01-02 2020-01-01 00:15:17 2020-01-02 03:43:38 2020-01-01 00:15:17.000 2020-01-02 03:43:38.000 917 99818 50367.5 5036750 917 99818 50367.5 5036750 -32651 32284 4492.3 449230 -128 124 -0.5 -50 -918 100 10908 99819 2.75675 299.75675 151.25675 15125.67567 2.75675 299.75674 151.25675 15125.67566 2.75675 299.75675 151.25675000000007 15125.67500 2020-01-01 2020-01-02 2020-01-01 00:15:18 2020-01-02 03:43:39 2020-01-01 00:15:18.000 2020-01-02 03:43:39.000 918 99819 50368.5 5036850 918 99819 50368.5 5036850 -32650 32285 4493.3 449330 -127 125 0.5 50 -919 100 10909 99820 2.75975 299.75975 151.25975 15125.97597 2.75975 299.75977 151.25976 15125.97626 2.75975 299.75975 151.25974999999983 15125.97500 2020-01-01 2020-01-02 2020-01-01 00:15:19 2020-01-02 03:43:40 2020-01-01 00:15:19.000 2020-01-02 03:43:40.000 919 99820 50369.5 5036950 919 99820 50369.5 5036950 -32649 32286 4494.3 449430 -126 126 1.5 150 -92 102 10082 99992 0.27627 300.27627 150.27627 15177.9039 0.27627 300.27628 150.27627 15177.90384 0.27627 300.27627 150.27627000000018 15177.90327 2020-01-01 2020-01-02 2020-01-01 00:01:32 2020-01-02 03:46:32 2020-01-01 00:01:32.000 2020-01-02 03:46:32.000 92 99992 50042 5054242 92 99992 50042 5054242 -32477 32458 4621.009900990099 466722 -128 123 -2.198019801980198 -222 -920 100 10910 99821 2.76276 299.76276 151.26276 15126.27627 2.76276 299.76276 151.26275 15126.27594 2.76276 299.76276 151.26275999999973 15126.27600 2020-01-01 2020-01-02 2020-01-01 00:15:20 2020-01-02 03:43:41 2020-01-01 00:15:20.000 2020-01-02 03:43:41.000 920 99821 50370.5 5037050 920 99821 50370.5 5037050 -32648 32287 4495.3 449530 -125 127 2.5 250 -921 100 10911 99822 2.76576 299.76576 151.26576 15126.57657 2.76576 299.76578 151.26576 15126.57654 2.76576 299.76576 151.2657600000002 15126.57600 2020-01-01 2020-01-02 2020-01-01 00:15:21 2020-01-02 03:43:42 2020-01-01 00:15:21.000 2020-01-02 03:43:42.000 921 99822 50371.5 5037150 921 99822 50371.5 5037150 -32647 32288 4496.3 449630 -128 127 0.94 94 -922 100 10912 99823 2.76876 299.76876 151.26876 15126.87687 2.76876 299.76877 151.26876 15126.87683 2.76876 299.76876 151.26876000000024 15126.87600 2020-01-01 2020-01-02 2020-01-01 00:15:22 2020-01-02 03:43:43 2020-01-01 00:15:22.000 2020-01-02 03:43:43.000 922 99823 50372.5 5037250 922 99823 50372.5 5037250 -32646 32289 4497.3 449730 -128 127 -0.62 -62 -923 100 10913 99824 2.77177 299.77177 151.27177 15127.17717 2.77177 299.77176 151.27177 15127.17713 2.77177 299.77177 151.2717699999999 15127.17700 2020-01-01 2020-01-02 2020-01-01 00:15:23 2020-01-02 03:43:44 2020-01-01 00:15:23.000 2020-01-02 03:43:44.000 923 99824 50373.5 5037350 923 99824 50373.5 5037350 -32645 32290 4498.3 449830 -128 123 -2.18 -218 +915 100 10905 99816 2.74774 299.74774 151.24774 15124.77477 2.74774 299.74774 151.24774 15124.77447 2.74774 299.74774 151.24774000000002 15124.77400 2020-01-01 2020-01-02 2020-01-01 00:15:15 2020-01-02 03:43:36 2020-01-01 00:15:15.000 2020-01-02 03:43:36.000 915 99816 50365.5 5036550 915 99816 50365.5 5036550 -32653 32282 4490.3 449030 -128 127 2.62 262 +916 100 10906 99817 2.75075 299.75075 151.25075 15125.07507 2.75075 299.75076 151.25075 15125.07507 2.75075 299.75075 151.25074999999995 15125.07500 2020-01-01 2020-01-02 2020-01-01 00:15:16 2020-01-02 03:43:37 2020-01-01 00:15:16.000 2020-01-02 03:43:37.000 916 99817 50366.5 5036650 916 99817 50366.5 5036650 -32652 32283 4491.3 449130 -128 127 1.06 106 +917 100 10907 99818 2.75375 299.75375 151.25375 15125.37537 2.75375 299.75375 151.25375 15125.37536 2.75375 299.75375 151.25374999999994 15125.37500 2020-01-01 2020-01-02 2020-01-01 00:15:17 2020-01-02 03:43:38 2020-01-01 00:15:17.000 2020-01-02 03:43:38.000 917 99818 50367.5 5036750 917 99818 50367.5 5036750 -32651 32284 4492.3 449230 -128 124 -0.5 -50 +918 100 10908 99819 2.75675 299.75675 151.25675 15125.67567 2.75675 299.75674 151.25675 15125.67566 2.75675 299.75675 151.25675000000018 15125.67500 2020-01-01 2020-01-02 2020-01-01 00:15:18 2020-01-02 03:43:39 2020-01-01 00:15:18.000 2020-01-02 03:43:39.000 918 99819 50368.5 5036850 918 99819 50368.5 5036850 -32650 32285 4493.3 449330 -127 125 0.5 50 +919 100 10909 99820 2.75975 299.75975 151.25975 15125.97597 2.75975 299.75977 151.25976 15125.97626 2.75975 299.75975 151.25974999999997 15125.97500 2020-01-01 2020-01-02 2020-01-01 00:15:19 2020-01-02 03:43:40 2020-01-01 00:15:19.000 2020-01-02 03:43:40.000 919 99820 50369.5 5036950 919 99820 50369.5 5036950 -32649 32286 4494.3 449430 -126 126 1.5 150 +92 102 10082 99992 0.27627 300.27627 150.27627 15177.9039 0.27627 300.27628 150.27627 15177.90384 0.27627 300.27627 150.27627000000007 15177.90327 2020-01-01 2020-01-02 2020-01-01 00:01:32 2020-01-02 03:46:32 2020-01-01 00:01:32.000 2020-01-02 03:46:32.000 92 99992 50042 5054242 92 99992 50042 5054242 -32477 32458 4621.009900990099 466722 -128 123 -2.198019801980198 -222 +920 100 10910 99821 2.76276 299.76276 151.26276 15126.27627 2.76276 299.76276 151.26275 15126.27594 2.76276 299.76276 151.2627599999999 15126.27600 2020-01-01 2020-01-02 2020-01-01 00:15:20 2020-01-02 03:43:41 2020-01-01 00:15:20.000 2020-01-02 03:43:41.000 920 99821 50370.5 5037050 920 99821 50370.5 5037050 -32648 32287 4495.3 449530 -125 127 2.5 250 +921 100 10911 99822 2.76576 299.76576 151.26576 15126.57657 2.76576 299.76578 151.26576 15126.57654 2.76576 299.76576 151.2657600000001 15126.57600 2020-01-01 2020-01-02 2020-01-01 00:15:21 2020-01-02 03:43:42 2020-01-01 00:15:21.000 2020-01-02 03:43:42.000 921 99822 50371.5 5037150 921 99822 50371.5 5037150 -32647 32288 4496.3 449630 -128 127 0.94 94 +922 100 10912 99823 2.76876 299.76876 151.26876 15126.87687 2.76876 299.76877 151.26876 15126.87683 2.76876 299.76876 151.26876000000007 15126.87600 2020-01-01 2020-01-02 2020-01-01 00:15:22 2020-01-02 03:43:43 2020-01-01 00:15:22.000 2020-01-02 03:43:43.000 922 99823 50372.5 5037250 922 99823 50372.5 5037250 -32646 32289 4497.3 449730 -128 127 -0.62 -62 +923 100 10913 99824 2.77177 299.77177 151.27177 15127.17717 2.77177 299.77176 151.27177 15127.17713 2.77177 299.77177 151.27176999999995 15127.17700 2020-01-01 2020-01-02 2020-01-01 00:15:23 2020-01-02 03:43:44 2020-01-01 00:15:23.000 2020-01-02 03:43:44.000 923 99824 50373.5 5037350 923 99824 50373.5 5037350 -32645 32290 4498.3 449830 -128 123 -2.18 -218 924 100 10914 99825 2.77477 299.77477 151.27477 15127.47747 2.77477 299.77478 151.27477 15127.47776 2.77477 299.77477 151.27477 15127.47700 2020-01-01 2020-01-02 2020-01-01 00:15:24 2020-01-02 03:43:45 2020-01-01 00:15:24.000 2020-01-02 03:43:45.000 924 99825 50374.5 5037450 924 99825 50374.5 5037450 -32644 32291 4499.3 449930 -127 124 -1.18 -118 -925 100 10915 99826 2.77777 299.77777 151.27777 15127.77777 2.77777 299.77777 151.27777 15127.77741 2.77777 299.77777 151.2777700000001 15127.77700 2020-01-01 2020-01-02 2020-01-01 00:15:25 2020-01-02 03:43:46 2020-01-01 00:15:25.000 2020-01-02 03:43:46.000 925 99826 50375.5 5037550 925 99826 50375.5 5037550 -32643 32292 4500.3 450030 -126 125 -0.18 -18 -926 100 10916 99827 2.78078 299.78078 151.28078 15128.07807 2.78078 299.7808 151.28078 15128.078 2.78078 299.78078 151.28077999999974 15128.07800 2020-01-01 2020-01-02 2020-01-01 00:15:26 2020-01-02 03:43:47 2020-01-01 00:15:26.000 2020-01-02 03:43:47.000 926 99827 50376.5 5037650 926 99827 50376.5 5037650 -32642 32293 4501.3 450130 -125 126 0.82 82 -927 100 10917 99828 2.78378 299.78378 151.28378 15128.37837 2.78378 299.78378 151.28378 15128.3783 2.78378 299.78378 151.28377999999984 15128.37800 2020-01-01 2020-01-02 2020-01-01 00:15:27 2020-01-02 03:43:48 2020-01-01 00:15:27.000 2020-01-02 03:43:48.000 927 99828 50377.5 5037750 927 99828 50377.5 5037750 -32641 32294 4502.3 450230 -124 127 1.82 182 -928 100 10918 99829 2.78678 299.78678 151.28678 15128.67867 2.78678 299.78677 151.28678 15128.6786 2.78678 299.78678 151.28678000000025 15128.67800 2020-01-01 2020-01-02 2020-01-01 00:15:28 2020-01-02 03:43:49 2020-01-01 00:15:28.000 2020-01-02 03:43:49.000 928 99829 50378.5 5037850 928 99829 50378.5 5037850 -32640 32295 4503.3 450330 -128 127 0.26 26 -929 100 10919 99830 2.78978 299.78978 151.28978 15128.97897 2.78978 299.7898 151.28979 15128.97923 2.78978 299.78978 151.28977999999975 15128.97800 2020-01-01 2020-01-02 2020-01-01 00:15:29 2020-01-02 03:43:50 2020-01-01 00:15:29.000 2020-01-02 03:43:50.000 929 99830 50379.5 5037950 929 99830 50379.5 5037950 -32639 32296 4504.3 450430 -128 123 -1.3 -130 -93 102 10083 99993 0.27927 300.27927 150.27927 15178.2072 0.27927 300.27927 150.27927 15178.20715 0.27927 300.27927 150.2792699999997 15178.20627 2020-01-01 2020-01-02 2020-01-01 00:01:33 2020-01-02 03:46:33 2020-01-01 00:01:33.000 2020-01-02 03:46:33.000 93 99993 50043 5054343 93 99993 50043 5054343 -32476 32459 4622.009900990099 466823 -127 124 -1.198019801980198 -121 -930 100 10920 99831 2.79279 299.79279 151.29279 15129.27927 2.79279 299.7928 151.29278 15129.27888 2.79279 299.79279 151.29279 15129.27900 2020-01-01 2020-01-02 2020-01-01 00:15:30 2020-01-02 03:43:51 2020-01-01 00:15:30.000 2020-01-02 03:43:51.000 930 99831 50380.5 5038050 930 99831 50380.5 5038050 -32638 32297 4505.3 450530 -127 124 -0.3 -30 -931 100 10921 99832 2.79579 299.79579 151.29579 15129.57957 2.79579 299.7958 151.29579 15129.57963 2.79579 299.79579 151.2957900000001 15129.57900 2020-01-01 2020-01-02 2020-01-01 00:15:31 2020-01-02 03:43:52 2020-01-01 00:15:31.000 2020-01-02 03:43:52.000 931 99832 50381.5 5038150 931 99832 50381.5 5038150 -32637 32298 4506.3 450630 -126 125 0.7 70 -932 100 10922 99833 2.79879 299.79879 151.29879 15129.87987 2.79879 299.7988 151.29879 15129.87977 2.79879 299.79879 151.29879000000017 15129.87900 2020-01-01 2020-01-02 2020-01-01 00:15:32 2020-01-02 03:43:53 2020-01-01 00:15:32.000 2020-01-02 03:43:53.000 932 99833 50382.5 5038250 932 99833 50382.5 5038250 -32636 32299 4507.3 450730 -125 126 1.7 170 -933 100 10923 99834 2.8018 299.8018 151.3018 15130.18018 2.8018 299.8018 151.3018 15130.18011 2.80180 299.80180 151.30179999999984 15130.18000 2020-01-01 2020-01-02 2020-01-01 00:15:33 2020-01-02 03:43:54 2020-01-01 00:15:33.000 2020-01-02 03:43:54.000 933 99834 50383.5 5038350 933 99834 50383.5 5038350 -32635 32300 4508.3 450830 -124 127 2.7 270 +925 100 10915 99826 2.77777 299.77777 151.27777 15127.77777 2.77777 299.77777 151.27777 15127.77741 2.77777 299.77777 151.27777000000003 15127.77700 2020-01-01 2020-01-02 2020-01-01 00:15:25 2020-01-02 03:43:46 2020-01-01 00:15:25.000 2020-01-02 03:43:46.000 925 99826 50375.5 5037550 925 99826 50375.5 5037550 -32643 32292 4500.3 450030 -126 125 -0.18 -18 +926 100 10916 99827 2.78078 299.78078 151.28078 15128.07807 2.78078 299.7808 151.28078 15128.078 2.78078 299.78078 151.28077999999994 15128.07800 2020-01-01 2020-01-02 2020-01-01 00:15:26 2020-01-02 03:43:47 2020-01-01 00:15:26.000 2020-01-02 03:43:47.000 926 99827 50376.5 5037650 926 99827 50376.5 5037650 -32642 32293 4501.3 450130 -125 126 0.82 82 +927 100 10917 99828 2.78378 299.78378 151.28378 15128.37837 2.78378 299.78378 151.28378 15128.3783 2.78378 299.78378 151.28377999999992 15128.37800 2020-01-01 2020-01-02 2020-01-01 00:15:27 2020-01-02 03:43:48 2020-01-01 00:15:27.000 2020-01-02 03:43:48.000 927 99828 50377.5 5037750 927 99828 50377.5 5037750 -32641 32294 4502.3 450230 -124 127 1.82 182 +928 100 10918 99829 2.78678 299.78678 151.28678 15128.67867 2.78678 299.78677 151.28678 15128.6786 2.78678 299.78678 151.28678000000008 15128.67800 2020-01-01 2020-01-02 2020-01-01 00:15:28 2020-01-02 03:43:49 2020-01-01 00:15:28.000 2020-01-02 03:43:49.000 928 99829 50378.5 5037850 928 99829 50378.5 5037850 -32640 32295 4503.3 450330 -128 127 0.26 26 +929 100 10919 99830 2.78978 299.78978 151.28978 15128.97897 2.78978 299.7898 151.28979 15128.97923 2.78978 299.78978 151.28977999999995 15128.97800 2020-01-01 2020-01-02 2020-01-01 00:15:29 2020-01-02 03:43:50 2020-01-01 00:15:29.000 2020-01-02 03:43:50.000 929 99830 50379.5 5037950 929 99830 50379.5 5037950 -32639 32296 4504.3 450430 -128 123 -1.3 -130 +93 102 10083 99993 0.27927 300.27927 150.27927 15178.2072 0.27927 300.27927 150.27927 15178.20715 0.27927 300.27927 150.2792699999999 15178.20627 2020-01-01 2020-01-02 2020-01-01 00:01:33 2020-01-02 03:46:33 2020-01-01 00:01:33.000 2020-01-02 03:46:33.000 93 99993 50043 5054343 93 99993 50043 5054343 -32476 32459 4622.009900990099 466823 -127 124 -1.198019801980198 -121 +930 100 10920 99831 2.79279 299.79279 151.29279 15129.27927 2.79279 299.7928 151.29278 15129.27888 2.79279 299.79279 151.29279000000002 15129.27900 2020-01-01 2020-01-02 2020-01-01 00:15:30 2020-01-02 03:43:51 2020-01-01 00:15:30.000 2020-01-02 03:43:51.000 930 99831 50380.5 5038050 930 99831 50380.5 5038050 -32638 32297 4505.3 450530 -127 124 -0.3 -30 +931 100 10921 99832 2.79579 299.79579 151.29579 15129.57957 2.79579 299.7958 151.29579 15129.57963 2.79579 299.79579 151.29579000000007 15129.57900 2020-01-01 2020-01-02 2020-01-01 00:15:31 2020-01-02 03:43:52 2020-01-01 00:15:31.000 2020-01-02 03:43:52.000 931 99832 50381.5 5038150 931 99832 50381.5 5038150 -32637 32298 4506.3 450630 -126 125 0.7 70 +932 100 10922 99833 2.79879 299.79879 151.29879 15129.87987 2.79879 299.7988 151.29879 15129.87977 2.79879 299.79879 151.29879000000003 15129.87900 2020-01-01 2020-01-02 2020-01-01 00:15:32 2020-01-02 03:43:53 2020-01-01 00:15:32.000 2020-01-02 03:43:53.000 932 99833 50382.5 5038250 932 99833 50382.5 5038250 -32636 32299 4507.3 450730 -125 126 1.7 170 +933 100 10923 99834 2.8018 299.8018 151.3018 15130.18018 2.8018 299.8018 151.3018 15130.18011 2.80180 299.80180 151.3018 15130.18000 2020-01-01 2020-01-02 2020-01-01 00:15:33 2020-01-02 03:43:54 2020-01-01 00:15:33.000 2020-01-02 03:43:54.000 933 99834 50383.5 5038350 933 99834 50383.5 5038350 -32635 32300 4508.3 450830 -124 127 2.7 270 934 100 10924 99835 2.8048 299.8048 151.3048 15130.48048 2.8048 299.8048 151.3048 15130.48071 2.80480 299.80480 151.30479999999991 15130.48000 2020-01-01 2020-01-02 2020-01-01 00:15:34 2020-01-02 03:43:55 2020-01-01 00:15:34.000 2020-01-02 03:43:55.000 934 99835 50384.5 5038450 934 99835 50384.5 5038450 -32634 32301 4509.3 450930 -128 127 1.14 114 -935 100 10925 99836 2.8078 299.8078 151.3078 15130.78078 2.8078 299.8078 151.3078 15130.78034 2.80780 299.80780 151.30780000000001 15130.78000 2020-01-01 2020-01-02 2020-01-01 00:15:35 2020-01-02 03:43:56 2020-01-01 00:15:35.000 2020-01-02 03:43:56.000 935 99836 50385.5 5038550 935 99836 50385.5 5038550 -32633 32302 4510.3 451030 -128 123 -0.42 -42 -936 100 10926 99837 2.81081 299.81081 151.31081 15131.08108 2.81081 299.81082 151.31081 15131.0811 2.81081 299.81081 151.31081000000026 15131.08100 2020-01-01 2020-01-02 2020-01-01 00:15:36 2020-01-02 03:43:57 2020-01-01 00:15:36.000 2020-01-02 03:43:57.000 936 99837 50386.5 5038650 936 99837 50386.5 5038650 -32632 32303 4511.3 451130 -127 124 0.58 58 -937 100 10927 99838 2.81381 299.81381 151.31381 15131.38138 2.81381 299.8138 151.31381 15131.38124 2.81381 299.81381 151.31380999999976 15131.38100 2020-01-01 2020-01-02 2020-01-01 00:15:37 2020-01-02 03:43:58 2020-01-01 00:15:37.000 2020-01-02 03:43:58.000 937 99838 50387.5 5038750 937 99838 50387.5 5038750 -32631 32304 4512.3 451230 -126 125 1.58 158 -938 100 10928 99839 2.81681 299.81681 151.31681 15131.68168 2.81681 299.8168 151.31681 15131.68157 2.81681 299.81681 151.31681000000015 15131.68100 2020-01-01 2020-01-02 2020-01-01 00:15:38 2020-01-02 03:43:59 2020-01-01 00:15:38.000 2020-01-02 03:43:59.000 938 99839 50388.5 5038850 938 99839 50388.5 5038850 -32630 32305 4513.3 451330 -125 126 2.58 258 -939 100 10929 99840 2.81981 299.81981 151.31981 15131.98198 2.81982 299.81982 151.31982 15131.98217 2.81981 299.81981 151.31981000000027 15131.98100 2020-01-01 2020-01-02 2020-01-01 00:15:39 2020-01-02 03:44:00 2020-01-01 00:15:39.000 2020-01-02 03:44:00.000 939 99840 50389.5 5038950 939 99840 50389.5 5038950 -32629 32306 4514.3 451430 -124 127 3.58 358 -94 102 10084 99994 0.28228 300.28228 150.28228 15178.51051 0.28228 300.2823 150.28228 15178.51078 0.28228 300.28228 150.28227999999996 15178.51028 2020-01-01 2020-01-02 2020-01-01 00:01:34 2020-01-02 03:46:34 2020-01-01 00:01:34.000 2020-01-02 03:46:34.000 94 99994 50044 5054444 94 99994 50044 5054444 -32475 32460 4623.009900990099 466924 -126 125 -0.19801980198019803 -20 -940 100 10930 99841 2.82282 299.82282 151.32282 15132.28228 2.82282 299.8228 151.32282 15132.28247 2.82282 299.82282 151.3228199999999 15132.28200 2020-01-01 2020-01-02 2020-01-01 00:15:40 2020-01-02 03:44:01 2020-01-01 00:15:40.000 2020-01-02 03:44:01.000 940 99841 50390.5 5039050 940 99841 50390.5 5039050 -32628 32307 4515.3 451530 -128 127 2.02 202 -941 100 10931 99842 2.82582 299.82582 151.32582 15132.58258 2.82582 299.82584 151.32582 15132.58257 2.82582 299.82582 151.32582000000002 15132.58200 2020-01-01 2020-01-02 2020-01-01 00:15:41 2020-01-02 03:44:02 2020-01-01 00:15:41.000 2020-01-02 03:44:02.000 941 99842 50391.5 5039150 941 99842 50391.5 5039150 -32627 32308 4516.3 451630 -128 127 0.46 46 -942 100 10932 99843 2.82882 299.82882 151.32882 15132.88288 2.82882 299.82883 151.32882 15132.88275 2.82882 299.82882 151.32882000000012 15132.88200 2020-01-01 2020-01-02 2020-01-01 00:15:42 2020-01-02 03:44:03 2020-01-01 00:15:42.000 2020-01-02 03:44:03.000 942 99843 50392.5 5039250 942 99843 50392.5 5039250 -32626 32309 4517.3 451730 -128 124 -1.1 -110 -943 100 10933 99844 2.83183 299.83183 151.33183 15133.18318 2.83183 299.83182 151.33183 15133.18304 2.83183 299.83183 151.33182999999977 15133.18300 2020-01-01 2020-01-02 2020-01-01 00:15:43 2020-01-02 03:44:04 2020-01-01 00:15:43.000 2020-01-02 03:44:04.000 943 99844 50393.5 5039350 943 99844 50393.5 5039350 -32625 32310 4518.3 451830 -127 125 -0.1 -10 -944 100 10934 99845 2.83483 299.83483 151.33483 15133.48348 2.83483 299.83484 151.33483 15133.48364 2.83483 299.83483 151.3348299999998 15133.48300 2020-01-01 2020-01-02 2020-01-01 00:15:44 2020-01-02 03:44:05 2020-01-01 00:15:44.000 2020-01-02 03:44:05.000 944 99845 50394.5 5039450 944 99845 50394.5 5039450 -32624 32311 4519.3 451930 -126 126 0.9 90 -945 100 10935 99846 2.83783 299.83783 151.33783 15133.78378 2.83783 299.83783 151.33783 15133.78393 2.83783 299.83783 151.33783000000028 15133.78300 2020-01-01 2020-01-02 2020-01-01 00:15:45 2020-01-02 03:44:06 2020-01-01 00:15:45.000 2020-01-02 03:44:06.000 945 99846 50395.5 5039550 945 99846 50395.5 5039550 -32623 32312 4520.3 452030 -125 127 1.9 190 -946 100 10936 99847 2.84084 299.84084 151.34084 15134.08408 2.84084 299.84085 151.34084 15134.08404 2.84084 299.84084 151.34084000000018 15134.08400 2020-01-01 2020-01-02 2020-01-01 00:15:46 2020-01-02 03:44:07 2020-01-01 00:15:46.000 2020-01-02 03:44:07.000 946 99847 50396.5 5039650 946 99847 50396.5 5039650 -32622 32313 4521.3 452130 -128 127 0.34 34 -947 100 10937 99848 2.84384 299.84384 151.34384 15134.38438 2.84384 299.84384 151.34384 15134.38421 2.84384 299.84384 151.34383999999966 15134.38400 2020-01-01 2020-01-02 2020-01-01 00:15:47 2020-01-02 03:44:08 2020-01-01 00:15:47.000 2020-01-02 03:44:08.000 947 99848 50397.5 5039750 947 99848 50397.5 5039750 -32621 32314 4522.3 452230 -128 127 -1.22 -122 -948 100 10938 99849 2.84684 299.84684 151.34684 15134.68468 2.84684 299.84683 151.34684 15134.68452 2.84684 299.84684 151.34684000000013 15134.68400 2020-01-01 2020-01-02 2020-01-01 00:15:48 2020-01-02 03:44:09 2020-01-01 00:15:48.000 2020-01-02 03:44:09.000 948 99849 50398.5 5039850 948 99849 50398.5 5039850 -32620 32315 4523.3 452330 -128 123 -2.78 -278 -949 100 10939 99850 2.84984 299.84984 151.34984 15134.98498 2.84985 299.84985 151.34985 15134.98527 2.84984 299.84984 151.34984000000017 15134.98400 2020-01-01 2020-01-02 2020-01-01 00:15:49 2020-01-02 03:44:10 2020-01-01 00:15:49.000 2020-01-02 03:44:10.000 949 99850 50399.5 5039950 949 99850 50399.5 5039950 -32619 32316 4524.3 452430 -127 124 -1.78 -178 +935 100 10925 99836 2.8078 299.8078 151.3078 15130.78078 2.8078 299.8078 151.3078 15130.78034 2.80780 299.80780 151.3078 15130.78000 2020-01-01 2020-01-02 2020-01-01 00:15:35 2020-01-02 03:43:56 2020-01-01 00:15:35.000 2020-01-02 03:43:56.000 935 99836 50385.5 5038550 935 99836 50385.5 5038550 -32633 32302 4510.3 451030 -128 123 -0.42 -42 +936 100 10926 99837 2.81081 299.81081 151.31081 15131.08108 2.81081 299.81082 151.31081 15131.0811 2.81081 299.81081 151.31081000000006 15131.08100 2020-01-01 2020-01-02 2020-01-01 00:15:36 2020-01-02 03:43:57 2020-01-01 00:15:36.000 2020-01-02 03:43:57.000 936 99837 50386.5 5038650 936 99837 50386.5 5038650 -32632 32303 4511.3 451130 -127 124 0.58 58 +937 100 10927 99838 2.81381 299.81381 151.31381 15131.38138 2.81381 299.8138 151.31381 15131.38124 2.81381 299.81381 151.31380999999993 15131.38100 2020-01-01 2020-01-02 2020-01-01 00:15:37 2020-01-02 03:43:58 2020-01-01 00:15:37.000 2020-01-02 03:43:58.000 937 99838 50387.5 5038750 937 99838 50387.5 5038750 -32631 32304 4512.3 451230 -126 125 1.58 158 +938 100 10928 99839 2.81681 299.81681 151.31681 15131.68168 2.81681 299.8168 151.31681 15131.68157 2.81681 299.81681 151.31681000000003 15131.68100 2020-01-01 2020-01-02 2020-01-01 00:15:38 2020-01-02 03:43:59 2020-01-01 00:15:38.000 2020-01-02 03:43:59.000 938 99839 50388.5 5038850 938 99839 50388.5 5038850 -32630 32305 4513.3 451330 -125 126 2.58 258 +939 100 10929 99840 2.81981 299.81981 151.31981 15131.98198 2.81982 299.81982 151.31982 15131.98217 2.81981 299.81981 151.31981000000007 15131.98100 2020-01-01 2020-01-02 2020-01-01 00:15:39 2020-01-02 03:44:00 2020-01-01 00:15:39.000 2020-01-02 03:44:00.000 939 99840 50389.5 5038950 939 99840 50389.5 5038950 -32629 32306 4514.3 451430 -124 127 3.58 358 +94 102 10084 99994 0.28228 300.28228 150.28228 15178.51051 0.28228 300.2823 150.28228 15178.51078 0.28228 300.28228 150.28228 15178.51028 2020-01-01 2020-01-02 2020-01-01 00:01:34 2020-01-02 03:46:34 2020-01-01 00:01:34.000 2020-01-02 03:46:34.000 94 99994 50044 5054444 94 99994 50044 5054444 -32475 32460 4623.009900990099 466924 -126 125 -0.19801980198019803 -20 +940 100 10930 99841 2.82282 299.82282 151.32282 15132.28228 2.82282 299.8228 151.32282 15132.28247 2.82282 299.82282 151.32281999999998 15132.28200 2020-01-01 2020-01-02 2020-01-01 00:15:40 2020-01-02 03:44:01 2020-01-01 00:15:40.000 2020-01-02 03:44:01.000 940 99841 50390.5 5039050 940 99841 50390.5 5039050 -32628 32307 4515.3 451530 -128 127 2.02 202 +941 100 10931 99842 2.82582 299.82582 151.32582 15132.58258 2.82582 299.82584 151.32582 15132.58257 2.82582 299.82582 151.32582 15132.58200 2020-01-01 2020-01-02 2020-01-01 00:15:41 2020-01-02 03:44:02 2020-01-01 00:15:41.000 2020-01-02 03:44:02.000 941 99842 50391.5 5039150 941 99842 50391.5 5039150 -32627 32308 4516.3 451630 -128 127 0.46 46 +942 100 10932 99843 2.82882 299.82882 151.32882 15132.88288 2.82882 299.82883 151.32882 15132.88275 2.82882 299.82882 151.32882000000006 15132.88200 2020-01-01 2020-01-02 2020-01-01 00:15:42 2020-01-02 03:44:03 2020-01-01 00:15:42.000 2020-01-02 03:44:03.000 942 99843 50392.5 5039250 942 99843 50392.5 5039250 -32626 32309 4517.3 451730 -128 124 -1.1 -110 +943 100 10933 99844 2.83183 299.83183 151.33183 15133.18318 2.83183 299.83182 151.33183 15133.18304 2.83183 299.83183 151.33182999999994 15133.18300 2020-01-01 2020-01-02 2020-01-01 00:15:43 2020-01-02 03:44:04 2020-01-01 00:15:43.000 2020-01-02 03:44:04.000 943 99844 50393.5 5039350 943 99844 50393.5 5039350 -32625 32310 4518.3 451830 -127 125 -0.1 -10 +944 100 10934 99845 2.83483 299.83483 151.33483 15133.48348 2.83483 299.83484 151.33483 15133.48364 2.83483 299.83483 151.3348299999999 15133.48300 2020-01-01 2020-01-02 2020-01-01 00:15:44 2020-01-02 03:44:05 2020-01-01 00:15:44.000 2020-01-02 03:44:05.000 944 99845 50394.5 5039450 944 99845 50394.5 5039450 -32624 32311 4519.3 451930 -126 126 0.9 90 +945 100 10935 99846 2.83783 299.83783 151.33783 15133.78378 2.83783 299.83783 151.33783 15133.78393 2.83783 299.83783 151.3378300000001 15133.78300 2020-01-01 2020-01-02 2020-01-01 00:15:45 2020-01-02 03:44:06 2020-01-01 00:15:45.000 2020-01-02 03:44:06.000 945 99846 50395.5 5039550 945 99846 50395.5 5039550 -32623 32312 4520.3 452030 -125 127 1.9 190 +946 100 10936 99847 2.84084 299.84084 151.34084 15134.08408 2.84084 299.84085 151.34084 15134.08404 2.84084 299.84084 151.34084000000004 15134.08400 2020-01-01 2020-01-02 2020-01-01 00:15:46 2020-01-02 03:44:07 2020-01-01 00:15:46.000 2020-01-02 03:44:07.000 946 99847 50396.5 5039650 946 99847 50396.5 5039650 -32622 32313 4521.3 452130 -128 127 0.34 34 +947 100 10937 99848 2.84384 299.84384 151.34384 15134.38438 2.84384 299.84384 151.34384 15134.38421 2.84384 299.84384 151.34383999999983 15134.38400 2020-01-01 2020-01-02 2020-01-01 00:15:47 2020-01-02 03:44:08 2020-01-01 00:15:47.000 2020-01-02 03:44:08.000 947 99848 50397.5 5039750 947 99848 50397.5 5039750 -32621 32314 4522.3 452230 -128 127 -1.22 -122 +948 100 10938 99849 2.84684 299.84684 151.34684 15134.68468 2.84684 299.84683 151.34684 15134.68452 2.84684 299.84684 151.34684000000007 15134.68400 2020-01-01 2020-01-02 2020-01-01 00:15:48 2020-01-02 03:44:09 2020-01-01 00:15:48.000 2020-01-02 03:44:09.000 948 99849 50398.5 5039850 948 99849 50398.5 5039850 -32620 32315 4523.3 452330 -128 123 -2.78 -278 +949 100 10939 99850 2.84984 299.84984 151.34984 15134.98498 2.84985 299.84985 151.34985 15134.98527 2.84984 299.84984 151.34984000000003 15134.98400 2020-01-01 2020-01-02 2020-01-01 00:15:49 2020-01-02 03:44:10 2020-01-01 00:15:49.000 2020-01-02 03:44:10.000 949 99850 50399.5 5039950 949 99850 50399.5 5039950 -32619 32316 4524.3 452430 -127 124 -1.78 -178 95 102 10085 99995 0.28528 300.28528 150.28528 15178.81381 0.28528 300.28528 150.28528 15178.81343 0.28528 300.28528 150.28528000000003 15178.81328 2020-01-01 2020-01-02 2020-01-01 00:01:35 2020-01-02 03:46:35 2020-01-01 00:01:35.000 2020-01-02 03:46:35.000 95 99995 50045 5054545 95 99995 50045 5054545 -32474 32461 4624.009900990099 467025 -125 126 0.801980198019802 81 -950 100 10940 99851 2.85285 299.85285 151.35285 15135.28528 2.85285 299.85284 151.35285 15135.28541 2.85285 299.85285 151.35284999999993 15135.28500 2020-01-01 2020-01-02 2020-01-01 00:15:50 2020-01-02 03:44:11 2020-01-01 00:15:50.000 2020-01-02 03:44:11.000 950 99851 50400.5 5040050 950 99851 50400.5 5040050 -32618 32317 4525.3 452530 -126 125 -0.78 -78 +950 100 10940 99851 2.85285 299.85285 151.35285 15135.28528 2.85285 299.85284 151.35285 15135.28541 2.85285 299.85285 151.35285000000005 15135.28500 2020-01-01 2020-01-02 2020-01-01 00:15:50 2020-01-02 03:44:11 2020-01-01 00:15:50.000 2020-01-02 03:44:11.000 950 99851 50400.5 5040050 950 99851 50400.5 5040050 -32618 32317 4525.3 452530 -126 125 -0.78 -78 951 100 10941 99852 2.85585 299.85585 151.35585 15135.58558 2.85585 299.85587 151.35585 15135.58551 2.85585 299.85585 151.35584999999995 15135.58500 2020-01-01 2020-01-02 2020-01-01 00:15:51 2020-01-02 03:44:12 2020-01-01 00:15:51.000 2020-01-02 03:44:12.000 951 99852 50401.5 5040150 951 99852 50401.5 5040150 -32617 32318 4526.3 452630 -125 126 0.22 22 -952 100 10942 99853 2.85885 299.85885 151.35885 15135.88588 2.85885 299.85886 151.35885 15135.88568 2.85885 299.85885 151.35885 15135.88500 2020-01-01 2020-01-02 2020-01-01 00:15:52 2020-01-02 03:44:13 2020-01-01 00:15:52.000 2020-01-02 03:44:13.000 952 99853 50402.5 5040250 952 99853 50402.5 5040250 -32616 32319 4527.3 452730 -124 127 1.22 122 -953 100 10943 99854 2.86186 299.86186 151.36186 15136.18618 2.86186 299.86185 151.36185 15136.18598 2.86186 299.86186 151.3618600000001 15136.18600 2020-01-01 2020-01-02 2020-01-01 00:15:53 2020-01-02 03:44:14 2020-01-01 00:15:53.000 2020-01-02 03:44:14.000 953 99854 50403.5 5040350 953 99854 50403.5 5040350 -32615 32320 4528.3 452830 -128 127 -0.34 -34 -954 100 10944 99855 2.86486 299.86486 151.36486 15136.48648 2.86486 299.86487 151.36486 15136.48674 2.86486 299.86486 151.3648599999999 15136.48600 2020-01-01 2020-01-02 2020-01-01 00:15:54 2020-01-02 03:44:15 2020-01-01 00:15:54.000 2020-01-02 03:44:15.000 954 99855 50404.5 5040450 954 99855 50404.5 5040450 -32614 32321 4529.3 452930 -128 123 -1.9 -190 -955 100 10945 99856 2.86786 299.86786 151.36786 15136.78678 2.86786 299.86786 151.36786 15136.78688 2.86786 299.86786 151.36786000000006 15136.78600 2020-01-01 2020-01-02 2020-01-01 00:15:55 2020-01-02 03:44:16 2020-01-01 00:15:55.000 2020-01-02 03:44:16.000 955 99856 50405.5 5040550 955 99856 50405.5 5040550 -32613 32322 4530.3 453030 -127 124 -0.9 -90 -956 100 10946 99857 2.87087 299.87087 151.37087 15137.08708 2.87087 299.87088 151.37087 15137.08701 2.87087 299.87087 151.37087000000008 15137.08700 2020-01-01 2020-01-02 2020-01-01 00:15:56 2020-01-02 03:44:17 2020-01-01 00:15:56.000 2020-01-02 03:44:17.000 956 99857 50406.5 5040650 956 99857 50406.5 5040650 -32612 32323 4531.3 453130 -126 125 0.1 10 -957 100 10947 99858 2.87387 299.87387 151.37387 15137.38738 2.87387 299.87387 151.37387 15137.38716 2.87387 299.87387 151.37386999999998 15137.38700 2020-01-01 2020-01-02 2020-01-01 00:15:57 2020-01-02 03:44:18 2020-01-01 00:15:57.000 2020-01-02 03:44:18.000 957 99858 50407.5 5040750 957 99858 50407.5 5040750 -32611 32324 4532.3 453230 -125 126 1.1 110 +952 100 10942 99853 2.85885 299.85885 151.35885 15135.88588 2.85885 299.85886 151.35885 15135.88568 2.85885 299.85885 151.35885000000005 15135.88500 2020-01-01 2020-01-02 2020-01-01 00:15:52 2020-01-02 03:44:13 2020-01-01 00:15:52.000 2020-01-02 03:44:13.000 952 99853 50402.5 5040250 952 99853 50402.5 5040250 -32616 32319 4527.3 452730 -124 127 1.22 122 +953 100 10943 99854 2.86186 299.86186 151.36186 15136.18618 2.86186 299.86185 151.36185 15136.18598 2.86186 299.86186 151.36186000000026 15136.18600 2020-01-01 2020-01-02 2020-01-01 00:15:53 2020-01-02 03:44:14 2020-01-01 00:15:53.000 2020-01-02 03:44:14.000 953 99854 50403.5 5040350 953 99854 50403.5 5040350 -32615 32320 4528.3 452830 -128 127 -0.34 -34 +954 100 10944 99855 2.86486 299.86486 151.36486 15136.48648 2.86486 299.86487 151.36486 15136.48674 2.86486 299.86486 151.3648599999998 15136.48600 2020-01-01 2020-01-02 2020-01-01 00:15:54 2020-01-02 03:44:15 2020-01-01 00:15:54.000 2020-01-02 03:44:15.000 954 99855 50404.5 5040450 954 99855 50404.5 5040450 -32614 32321 4529.3 452930 -128 123 -1.9 -190 +955 100 10945 99856 2.86786 299.86786 151.36786 15136.78678 2.86786 299.86786 151.36786 15136.78688 2.86786 299.86786 151.36786000000018 15136.78600 2020-01-01 2020-01-02 2020-01-01 00:15:55 2020-01-02 03:44:16 2020-01-01 00:15:55.000 2020-01-02 03:44:16.000 955 99856 50405.5 5040550 955 99856 50405.5 5040550 -32613 32322 4530.3 453030 -127 124 -0.9 -90 +956 100 10946 99857 2.87087 299.87087 151.37087 15137.08708 2.87087 299.87088 151.37087 15137.08701 2.87087 299.87087 151.37087000000014 15137.08700 2020-01-01 2020-01-02 2020-01-01 00:15:56 2020-01-02 03:44:17 2020-01-01 00:15:56.000 2020-01-02 03:44:17.000 956 99857 50406.5 5040650 956 99857 50406.5 5040650 -32612 32323 4531.3 453130 -126 125 0.1 10 +957 100 10947 99858 2.87387 299.87387 151.37387 15137.38738 2.87387 299.87387 151.37387 15137.38716 2.87387 299.87387 151.37386999999993 15137.38700 2020-01-01 2020-01-02 2020-01-01 00:15:57 2020-01-02 03:44:18 2020-01-01 00:15:57.000 2020-01-02 03:44:18.000 957 99858 50407.5 5040750 957 99858 50407.5 5040750 -32611 32324 4532.3 453230 -125 126 1.1 110 958 100 10948 99859 2.87687 299.87687 151.37687 15137.68768 2.87687 299.8769 151.37687 15137.68791 2.87687 299.87687 151.37687000000003 15137.68700 2020-01-01 2020-01-02 2020-01-01 00:15:58 2020-01-02 03:44:19 2020-01-01 00:15:58.000 2020-01-02 03:44:19.000 958 99859 50408.5 5040850 958 99859 50408.5 5040850 -32610 32325 4533.3 453330 -124 127 2.1 210 -959 100 10949 99860 2.87987 299.87987 151.37987 15137.98798 2.87988 299.87988 151.37988 15137.9882 2.87987 299.87987 151.37987000000004 15137.98700 2020-01-01 2020-01-02 2020-01-01 00:15:59 2020-01-02 03:44:20 2020-01-01 00:15:59.000 2020-01-02 03:44:20.000 959 99860 50409.5 5040950 959 99860 50409.5 5040950 -32609 32326 4534.3 453430 -128 127 0.54 54 -96 102 10086 99996 0.28828 300.28828 150.28828 15179.11711 0.28828 300.2883 150.28828 15179.11718 0.28828 300.28828 150.28828000000001 15179.11628 2020-01-01 2020-01-02 2020-01-01 00:01:36 2020-01-02 03:46:36 2020-01-01 00:01:36.000 2020-01-02 03:46:36.000 96 99996 50046 5054646 96 99996 50046 5054646 -32473 32462 4625.009900990099 467126 -124 127 1.801980198019802 182 -960 100 10950 99861 2.88288 299.88288 151.38288 15138.28828 2.88288 299.88287 151.38288 15138.28834 2.88288 299.88288 151.38287999999997 15138.28800 2020-01-01 2020-01-02 2020-01-01 00:16:00 2020-01-02 03:44:21 2020-01-01 00:16:00.000 2020-01-02 03:44:21.000 960 99861 50410.5 5041050 960 99861 50410.5 5041050 -32608 32327 4535.3 453530 -128 123 -1.02 -102 -961 100 10951 99862 2.88588 299.88588 151.38588 15138.58858 2.88588 299.8859 151.38588 15138.58848 2.88588 299.88588 151.3858799999999 15138.58800 2020-01-01 2020-01-02 2020-01-01 00:16:01 2020-01-02 03:44:22 2020-01-01 00:16:01.000 2020-01-02 03:44:22.000 961 99862 50411.5 5041150 961 99862 50411.5 5041150 -32607 32328 4536.3 453630 -127 124 -0.02 -2 -962 100 10952 99863 2.88888 299.88888 151.38888 15138.88888 2.88888 299.8889 151.38888 15138.88862 2.88888 299.88888 151.38888000000014 15138.88800 2020-01-01 2020-01-02 2020-01-01 00:16:02 2020-01-02 03:44:23 2020-01-01 00:16:02.000 2020-01-02 03:44:23.000 962 99863 50412.5 5041250 962 99863 50412.5 5041250 -32606 32329 4537.3 453730 -126 125 0.98 98 -963 100 10953 99864 2.89189 299.89189 151.39189 15139.18918 2.89189 299.8919 151.39189 15139.18937 2.89189 299.89189 151.39189000000005 15139.18900 2020-01-01 2020-01-02 2020-01-01 00:16:03 2020-01-02 03:44:24 2020-01-01 00:16:03.000 2020-01-02 03:44:24.000 963 99864 50413.5 5041350 963 99864 50413.5 5041350 -32605 32330 4538.3 453830 -125 126 1.98 198 -964 100 10954 99865 2.89489 299.89489 151.39489 15139.48948 2.89489 299.8949 151.39489 15139.48968 2.89489 299.89489 151.39488999999986 15139.48900 2020-01-01 2020-01-02 2020-01-01 00:16:04 2020-01-02 03:44:25 2020-01-01 00:16:04.000 2020-01-02 03:44:25.000 964 99865 50414.5 5041450 964 99865 50414.5 5041450 -32604 32331 4539.3 453930 -124 127 2.98 298 -965 100 10955 99866 2.89789 299.89789 151.39789 15139.78978 2.89789 299.8979 151.39789 15139.78985 2.89789 299.89789 151.3978900000001 15139.78900 2020-01-01 2020-01-02 2020-01-01 00:16:05 2020-01-02 03:44:26 2020-01-01 00:16:05.000 2020-01-02 03:44:26.000 965 99866 50415.5 5041550 965 99866 50415.5 5041550 -32603 32332 4540.3 454030 -128 127 1.42 142 -966 100 10956 99867 2.9009 299.9009 151.4009 15140.09009 2.9009 299.9009 151.40089 15140.08996 2.90090 299.90090 151.4009 15140.09000 2020-01-01 2020-01-02 2020-01-01 00:16:06 2020-01-02 03:44:27 2020-01-01 00:16:06.000 2020-01-02 03:44:27.000 966 99867 50416.5 5041650 966 99867 50416.5 5041650 -32602 32333 4541.3 454130 -128 127 -0.14 -14 -967 100 10957 99868 2.9039 299.9039 151.4039 15140.39039 2.9039 299.9039 151.4039 15140.39009 2.90390 299.90390 151.40389999999996 15140.39000 2020-01-01 2020-01-02 2020-01-01 00:16:07 2020-01-02 03:44:28 2020-01-01 00:16:07.000 2020-01-02 03:44:28.000 967 99868 50417.5 5041750 967 99868 50417.5 5041750 -32601 32334 4542.3 454230 -128 124 -1.7 -170 +959 100 10949 99860 2.87987 299.87987 151.37987 15137.98798 2.87988 299.87988 151.37988 15137.9882 2.87987 299.87987 151.37987000000012 15137.98700 2020-01-01 2020-01-02 2020-01-01 00:15:59 2020-01-02 03:44:20 2020-01-01 00:15:59.000 2020-01-02 03:44:20.000 959 99860 50409.5 5040950 959 99860 50409.5 5040950 -32609 32326 4534.3 453430 -128 127 0.54 54 +96 102 10086 99996 0.28828 300.28828 150.28828 15179.11711 0.28828 300.2883 150.28828 15179.11718 0.28828 300.28828 150.28828000000013 15179.11628 2020-01-01 2020-01-02 2020-01-01 00:01:36 2020-01-02 03:46:36 2020-01-01 00:01:36.000 2020-01-02 03:46:36.000 96 99996 50046 5054646 96 99996 50046 5054646 -32473 32462 4625.009900990099 467126 -124 127 1.801980198019802 182 +960 100 10950 99861 2.88288 299.88288 151.38288 15138.28828 2.88288 299.88287 151.38288 15138.28834 2.88288 299.88288 151.3828799999998 15138.28800 2020-01-01 2020-01-02 2020-01-01 00:16:00 2020-01-02 03:44:21 2020-01-01 00:16:00.000 2020-01-02 03:44:21.000 960 99861 50410.5 5041050 960 99861 50410.5 5041050 -32608 32327 4535.3 453530 -128 123 -1.02 -102 +961 100 10951 99862 2.88588 299.88588 151.38588 15138.58858 2.88588 299.8859 151.38588 15138.58848 2.88588 299.88588 151.38587999999984 15138.58800 2020-01-01 2020-01-02 2020-01-01 00:16:01 2020-01-02 03:44:22 2020-01-01 00:16:01.000 2020-01-02 03:44:22.000 961 99862 50411.5 5041150 961 99862 50411.5 5041150 -32607 32328 4536.3 453630 -127 124 -0.02 -2 +962 100 10952 99863 2.88888 299.88888 151.38888 15138.88888 2.88888 299.8889 151.38888 15138.88862 2.88888 299.88888 151.3888800000003 15138.88800 2020-01-01 2020-01-02 2020-01-01 00:16:02 2020-01-02 03:44:23 2020-01-01 00:16:02.000 2020-01-02 03:44:23.000 962 99863 50412.5 5041250 962 99863 50412.5 5041250 -32606 32329 4537.3 453730 -126 125 0.98 98 +963 100 10953 99864 2.89189 299.89189 151.39189 15139.18918 2.89189 299.8919 151.39189 15139.18937 2.89189 299.89189 151.3918900000002 15139.18900 2020-01-01 2020-01-02 2020-01-01 00:16:03 2020-01-02 03:44:24 2020-01-01 00:16:03.000 2020-01-02 03:44:24.000 963 99864 50413.5 5041350 963 99864 50413.5 5041350 -32605 32330 4538.3 453830 -125 126 1.98 198 +964 100 10954 99865 2.89489 299.89489 151.39489 15139.48948 2.89489 299.8949 151.39489 15139.48968 2.89489 299.89489 151.3948899999997 15139.48900 2020-01-01 2020-01-02 2020-01-01 00:16:04 2020-01-02 03:44:25 2020-01-01 00:16:04.000 2020-01-02 03:44:25.000 964 99865 50414.5 5041450 964 99865 50414.5 5041450 -32604 32331 4539.3 453930 -124 127 2.98 298 +965 100 10955 99866 2.89789 299.89789 151.39789 15139.78978 2.89789 299.8979 151.39789 15139.78985 2.89789 299.89789 151.39789000000016 15139.78900 2020-01-01 2020-01-02 2020-01-01 00:16:05 2020-01-02 03:44:26 2020-01-01 00:16:05.000 2020-01-02 03:44:26.000 965 99866 50415.5 5041550 965 99866 50415.5 5041550 -32603 32332 4540.3 454030 -128 127 1.42 142 +966 100 10956 99867 2.9009 299.9009 151.4009 15140.09009 2.9009 299.9009 151.40089 15140.08996 2.90090 299.90090 151.40090000000006 15140.09000 2020-01-01 2020-01-02 2020-01-01 00:16:06 2020-01-02 03:44:27 2020-01-01 00:16:06.000 2020-01-02 03:44:27.000 966 99867 50416.5 5041650 966 99867 50416.5 5041650 -32602 32333 4541.3 454130 -128 127 -0.14 -14 +967 100 10957 99868 2.9039 299.9039 151.4039 15140.39039 2.9039 299.9039 151.4039 15140.39009 2.90390 299.90390 151.40389999999988 15140.39000 2020-01-01 2020-01-02 2020-01-01 00:16:07 2020-01-02 03:44:28 2020-01-01 00:16:07.000 2020-01-02 03:44:28.000 967 99868 50417.5 5041750 967 99868 50417.5 5041750 -32601 32334 4542.3 454230 -128 124 -1.7 -170 968 100 10958 99869 2.9069 299.9069 151.4069 15140.69069 2.9069 299.90692 151.4069 15140.69084 2.90690 299.90690 151.40689999999998 15140.69000 2020-01-01 2020-01-02 2020-01-01 00:16:08 2020-01-02 03:44:29 2020-01-01 00:16:08.000 2020-01-02 03:44:29.000 968 99869 50418.5 5041850 968 99869 50418.5 5041850 -32600 32335 4543.3 454330 -127 125 -0.7 -70 -969 100 10959 99870 2.9099 299.9099 151.4099 15140.99099 2.90991 299.9099 151.40991 15140.99114 2.90990 299.90990 151.40990000000002 15140.99000 2020-01-01 2020-01-02 2020-01-01 00:16:09 2020-01-02 03:44:30 2020-01-01 00:16:09.000 2020-01-02 03:44:30.000 969 99870 50419.5 5041950 969 99870 50419.5 5041950 -32599 32336 4544.3 454430 -126 126 0.3 30 -97 102 10087 99997 0.29129 300.29129 150.29129 15179.42042 0.29129 300.2913 150.29129 15179.42033 0.29129 300.29129 150.29128999999998 15179.42029 2020-01-01 2020-01-02 2020-01-01 00:01:37 2020-01-02 03:46:37 2020-01-01 00:01:37.000 2020-01-02 03:46:37.000 97 99997 50047 5054747 97 99997 50047 5054747 -32472 32463 4626.009900990099 467227 -128 127 0.26732673267326734 27 -970 100 10960 99871 2.91291 299.91291 151.41291 15141.29129 2.91291 299.9129 151.41291 15141.29132 2.91291 299.91291 151.41290999999993 15141.29100 2020-01-01 2020-01-02 2020-01-01 00:16:10 2020-01-02 03:44:31 2020-01-01 00:16:10.000 2020-01-02 03:44:31.000 970 99871 50420.5 5042050 970 99871 50420.5 5042050 -32598 32337 4545.3 454530 -125 127 1.3 130 -971 100 10961 99872 2.91591 299.91591 151.41591 15141.59159 2.91591 299.91592 151.41591 15141.59142 2.91591 299.91591 151.41590999999994 15141.59100 2020-01-01 2020-01-02 2020-01-01 00:16:11 2020-01-02 03:44:32 2020-01-01 00:16:11.000 2020-01-02 03:44:32.000 971 99872 50421.5 5042150 971 99872 50421.5 5042150 -32597 32338 4546.3 454630 -128 127 -0.26 -26 -972 100 10962 99873 2.91891 299.91891 151.41891 15141.89189 2.91891 299.9189 151.41891 15141.89172 2.91891 299.91891 151.4189100000001 15141.89100 2020-01-01 2020-01-02 2020-01-01 00:16:12 2020-01-02 03:44:33 2020-01-01 00:16:12.000 2020-01-02 03:44:33.000 972 99873 50422.5 5042250 972 99873 50422.5 5042250 -32596 32339 4547.3 454730 -128 127 -1.82 -182 -973 100 10963 99874 2.92192 299.92192 151.42192 15142.19219 2.92192 299.92194 151.42192 15142.19232 2.92192 299.92192 151.42192 15142.19200 2020-01-01 2020-01-02 2020-01-01 00:16:13 2020-01-02 03:44:34 2020-01-01 00:16:13.000 2020-01-02 03:44:34.000 973 99874 50423.5 5042350 973 99874 50423.5 5042350 -32595 32340 4548.3 454830 -128 123 -3.38 -338 -974 100 10964 99875 2.92492 299.92492 151.42492 15142.49249 2.92492 299.92493 151.42492 15142.49265 2.92492 299.92492 151.42492000000001 15142.49200 2020-01-01 2020-01-02 2020-01-01 00:16:14 2020-01-02 03:44:35 2020-01-01 00:16:14.000 2020-01-02 03:44:35.000 974 99875 50424.5 5042450 974 99875 50424.5 5042450 -32594 32341 4549.3 454930 -127 124 -2.38 -238 +969 100 10959 99870 2.9099 299.9099 151.4099 15140.99099 2.90991 299.9099 151.40991 15140.99114 2.90990 299.90990 151.40990000000008 15140.99000 2020-01-01 2020-01-02 2020-01-01 00:16:09 2020-01-02 03:44:30 2020-01-01 00:16:09.000 2020-01-02 03:44:30.000 969 99870 50419.5 5041950 969 99870 50419.5 5041950 -32599 32336 4544.3 454430 -126 126 0.3 30 +97 102 10087 99997 0.29129 300.29129 150.29129 15179.42042 0.29129 300.2913 150.29129 15179.42033 0.29129 300.29129 150.2912899999998 15179.42029 2020-01-01 2020-01-02 2020-01-01 00:01:37 2020-01-02 03:46:37 2020-01-01 00:01:37.000 2020-01-02 03:46:37.000 97 99997 50047 5054747 97 99997 50047 5054747 -32472 32463 4626.009900990099 467227 -128 127 0.26732673267326734 27 +970 100 10960 99871 2.91291 299.91291 151.41291 15141.29129 2.91291 299.9129 151.41291 15141.29132 2.91291 299.91291 151.41290999999973 15141.29100 2020-01-01 2020-01-02 2020-01-01 00:16:10 2020-01-02 03:44:31 2020-01-01 00:16:10.000 2020-01-02 03:44:31.000 970 99871 50420.5 5042050 970 99871 50420.5 5042050 -32598 32337 4545.3 454530 -125 127 1.3 130 +971 100 10961 99872 2.91591 299.91591 151.41591 15141.59159 2.91591 299.91592 151.41591 15141.59142 2.91591 299.91591 151.41590999999983 15141.59100 2020-01-01 2020-01-02 2020-01-01 00:16:11 2020-01-02 03:44:32 2020-01-01 00:16:11.000 2020-01-02 03:44:32.000 971 99872 50421.5 5042150 971 99872 50421.5 5042150 -32597 32338 4546.3 454630 -128 127 -0.26 -26 +972 100 10962 99873 2.91891 299.91891 151.41891 15141.89189 2.91891 299.9189 151.41891 15141.89172 2.91891 299.91891 151.4189100000002 15141.89100 2020-01-01 2020-01-02 2020-01-01 00:16:12 2020-01-02 03:44:33 2020-01-01 00:16:12.000 2020-01-02 03:44:33.000 972 99873 50422.5 5042250 972 99873 50422.5 5042250 -32596 32339 4547.3 454730 -128 127 -1.82 -182 +973 100 10963 99874 2.92192 299.92192 151.42192 15142.19219 2.92192 299.92194 151.42192 15142.19232 2.92192 299.92192 151.4219200000001 15142.19200 2020-01-01 2020-01-02 2020-01-01 00:16:13 2020-01-02 03:44:34 2020-01-01 00:16:13.000 2020-01-02 03:44:34.000 973 99874 50423.5 5042350 973 99874 50423.5 5042350 -32595 32340 4548.3 454830 -128 123 -3.38 -338 +974 100 10964 99875 2.92492 299.92492 151.42492 15142.49249 2.92492 299.92493 151.42492 15142.49265 2.92492 299.92492 151.42491999999996 15142.49200 2020-01-01 2020-01-02 2020-01-01 00:16:14 2020-01-02 03:44:35 2020-01-01 00:16:14.000 2020-01-02 03:44:35.000 974 99875 50424.5 5042450 974 99875 50424.5 5042450 -32594 32341 4549.3 454930 -127 124 -2.38 -238 975 100 10965 99876 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792000000006 15142.79200 2020-01-01 2020-01-02 2020-01-01 00:16:15 2020-01-02 03:44:36 2020-01-01 00:16:15.000 2020-01-02 03:44:36.000 975 99876 50425.5 5042550 975 99876 50425.5 5042550 -32593 32342 4550.3 455030 -126 125 -1.38 -138 -976 100 10966 99877 2.93093 299.93093 151.43093 15143.09309 2.93093 299.93094 151.43092 15143.09289 2.93093 299.93093 151.43093000000016 15143.09300 2020-01-01 2020-01-02 2020-01-01 00:16:16 2020-01-02 03:44:37 2020-01-01 00:16:16.000 2020-01-02 03:44:37.000 976 99877 50426.5 5042650 976 99877 50426.5 5042650 -32592 32343 4551.3 455130 -125 126 -0.38 -38 -977 100 10967 99878 2.93393 299.93393 151.43393 15143.39339 2.93393 299.93393 151.43393 15143.39318 2.93393 299.93393 151.43392999999995 15143.39300 2020-01-01 2020-01-02 2020-01-01 00:16:17 2020-01-02 03:44:38 2020-01-01 00:16:17.000 2020-01-02 03:44:38.000 977 99878 50427.5 5042750 977 99878 50427.5 5042750 -32591 32344 4552.3 455230 -124 127 0.62 62 -978 100 10968 99879 2.93693 299.93693 151.43693 15143.69369 2.93693 299.93695 151.43693 15143.69378 2.93693 299.93693 151.43692999999993 15143.69300 2020-01-01 2020-01-02 2020-01-01 00:16:18 2020-01-02 03:44:39 2020-01-01 00:16:18.000 2020-01-02 03:44:39.000 978 99879 50428.5 5042850 978 99879 50428.5 5042850 -32590 32345 4553.3 455330 -128 127 -0.94 -94 -979 100 10969 99880 2.93993 299.93993 151.43993 15143.99399 2.93994 299.93994 151.43994 15143.99412 2.93993 299.93993 151.43993000000017 15143.99300 2020-01-01 2020-01-02 2020-01-01 00:16:19 2020-01-02 03:44:40 2020-01-01 00:16:19.000 2020-01-02 03:44:40.000 979 99880 50429.5 5042950 979 99880 50429.5 5042950 -32589 32346 4554.3 455430 -128 123 -2.5 -250 -98 102 10088 99998 0.29429 300.29429 150.29429 15179.72372 0.29429 300.29428 150.29429 15179.72363 0.29429 300.29429 150.29428999999993 15179.72329 2020-01-01 2020-01-02 2020-01-01 00:01:38 2020-01-02 03:46:38 2020-01-01 00:01:38.000 2020-01-02 03:46:38.000 98 99998 50048 5054848 98 99998 50048 5054848 -32471 32464 4627.009900990099 467328 -128 127 -1.2673267326732673 -128 -980 100 10970 99881 2.94294 299.94294 151.44294 15144.29429 2.94294 299.94293 151.44294 15144.29426 2.94294 299.94294 151.44294000000005 15144.29400 2020-01-01 2020-01-02 2020-01-01 00:16:20 2020-01-02 03:44:41 2020-01-01 00:16:20.000 2020-01-02 03:44:41.000 980 99881 50430.5 5043050 980 99881 50430.5 5043050 -32588 32347 4555.3 455530 -127 124 -1.5 -150 -981 100 10971 99882 2.94594 299.94594 151.44594 15144.59459 2.94594 299.94595 151.44595 15144.59501 2.94594 299.94594 151.4459399999999 15144.59400 2020-01-01 2020-01-02 2020-01-01 00:16:21 2020-01-02 03:44:42 2020-01-01 00:16:21.000 2020-01-02 03:44:42.000 981 99882 50431.5 5043150 981 99882 50431.5 5043150 -32587 32348 4556.3 455630 -126 125 -0.5 -50 -982 100 10972 99883 2.94894 299.94894 151.44894 15144.89489 2.94894 299.94894 151.44894 15144.89466 2.94894 299.94894 151.4489400000001 15144.89400 2020-01-01 2020-01-02 2020-01-01 00:16:22 2020-01-02 03:44:43 2020-01-01 00:16:22.000 2020-01-02 03:44:43.000 982 99883 50432.5 5043250 982 99883 50432.5 5043250 -32586 32349 4557.3 455730 -125 126 0.5 50 -983 100 10973 99884 2.95195 299.95195 151.45195 15145.19519 2.95195 299.95197 151.45195 15145.19525 2.95195 299.95195 151.45195 15145.19500 2020-01-01 2020-01-02 2020-01-01 00:16:23 2020-01-02 03:44:44 2020-01-01 00:16:23.000 2020-01-02 03:44:44.000 983 99884 50433.5 5043350 983 99884 50433.5 5043350 -32585 32350 4558.3 455830 -124 127 1.5 150 -984 100 10974 99885 2.95495 299.95495 151.45495 15145.49549 2.95495 299.95496 151.45495 15145.49559 2.95495 299.95495 151.45494999999994 15145.49500 2020-01-01 2020-01-02 2020-01-01 00:16:24 2020-01-02 03:44:45 2020-01-01 00:16:24.000 2020-01-02 03:44:45.000 984 99885 50434.5 5043450 984 99885 50434.5 5043450 -32584 32351 4559.3 455930 -128 127 -0.06 -6 -985 100 10975 99886 2.95795 299.95795 151.45795 15145.79579 2.95795 299.95795 151.45795 15145.79573 2.95795 299.95795 151.45795 15145.79500 2020-01-01 2020-01-02 2020-01-01 00:16:25 2020-01-02 03:44:46 2020-01-01 00:16:25.000 2020-01-02 03:44:46.000 985 99886 50435.5 5043550 985 99886 50435.5 5043550 -32583 32352 4560.3 456030 -128 123 -1.62 -162 -986 100 10976 99887 2.96096 299.96096 151.46096 15146.09609 2.96096 299.96097 151.46096 15146.09648 2.96096 299.96096 151.46096000000009 15146.09600 2020-01-01 2020-01-02 2020-01-01 00:16:26 2020-01-02 03:44:47 2020-01-01 00:16:26.000 2020-01-02 03:44:47.000 986 99887 50436.5 5043650 986 99887 50436.5 5043650 -32582 32353 4561.3 456130 -127 124 -0.62 -62 -987 100 10977 99888 2.96396 299.96396 151.46396 15146.39639 2.96396 299.96396 151.46396 15146.39612 2.96396 299.96396 151.46395999999993 15146.39600 2020-01-01 2020-01-02 2020-01-01 00:16:27 2020-01-02 03:44:48 2020-01-01 00:16:27.000 2020-01-02 03:44:48.000 987 99888 50437.5 5043750 987 99888 50437.5 5043750 -32581 32354 4562.3 456230 -126 125 0.38 38 -988 100 10978 99889 2.96696 299.96696 151.46696 15146.69669 2.96696 299.96698 151.46696 15146.69676 2.96696 299.96696 151.46695999999991 15146.69600 2020-01-01 2020-01-02 2020-01-01 00:16:28 2020-01-02 03:44:49 2020-01-01 00:16:28.000 2020-01-02 03:44:49.000 988 99889 50438.5 5043850 988 99889 50438.5 5043850 -32580 32355 4563.3 456330 -125 126 1.38 138 -989 100 10979 99890 2.96996 299.96996 151.46996 15146.99699 2.96997 299.96997 151.46997 15146.99706 2.96996 299.96996 151.46996000000007 15146.99600 2020-01-01 2020-01-02 2020-01-01 00:16:29 2020-01-02 03:44:50 2020-01-01 00:16:29.000 2020-01-02 03:44:50.000 989 99890 50439.5 5043950 989 99890 50439.5 5043950 -32579 32356 4564.3 456430 -124 127 2.38 238 -99 102 10089 99999 0.29729 300.29729 150.29729 15180.02702 0.29729 300.2973 150.29729 15180.02726 0.29729 300.29729 150.29729000000012 15180.02629 2020-01-01 2020-01-02 2020-01-01 00:01:39 2020-01-02 03:46:39 2020-01-01 00:01:39.000 2020-01-02 03:46:39.000 99 99999 50049 5054949 99 99999 50049 5054949 -32470 32465 4628.009900990099 467429 -128 123 -2.801980198019802 -283 -990 100 10980 99891 2.97297 299.97297 151.47297 15147.29729 2.97297 299.97296 151.47297 15147.29735 2.97297 299.97297 151.47297000000003 15147.29700 2020-01-01 2020-01-02 2020-01-01 00:16:30 2020-01-02 03:44:51 2020-01-01 00:16:30.000 2020-01-02 03:44:51.000 990 99891 50440.5 5044050 990 99891 50440.5 5044050 -32578 32357 4565.3 456530 -128 127 0.82 82 -991 100 10981 99892 2.97597 299.97597 151.47597 15147.59759 2.97597 299.97598 151.47597 15147.59795 2.97597 299.97597 151.47597000000002 15147.59700 2020-01-01 2020-01-02 2020-01-01 00:16:31 2020-01-02 03:44:52 2020-01-01 00:16:31.000 2020-01-02 03:44:52.000 991 99892 50441.5 5044150 991 99892 50441.5 5044150 -32577 32358 4566.3 456630 -128 127 -0.74 -74 +976 100 10966 99877 2.93093 299.93093 151.43093 15143.09309 2.93093 299.93094 151.43092 15143.09289 2.93093 299.93093 151.43093000000033 15143.09300 2020-01-01 2020-01-02 2020-01-01 00:16:16 2020-01-02 03:44:37 2020-01-01 00:16:16.000 2020-01-02 03:44:37.000 976 99877 50426.5 5042650 976 99877 50426.5 5042650 -32592 32343 4551.3 455130 -125 126 -0.38 -38 +977 100 10967 99878 2.93393 299.93393 151.43393 15143.39339 2.93393 299.93393 151.43393 15143.39318 2.93393 299.93393 151.4339299999998 15143.39300 2020-01-01 2020-01-02 2020-01-01 00:16:17 2020-01-02 03:44:38 2020-01-01 00:16:17.000 2020-01-02 03:44:38.000 977 99878 50427.5 5042750 977 99878 50427.5 5042750 -32591 32344 4552.3 455230 -124 127 0.62 62 +978 100 10968 99879 2.93693 299.93693 151.43693 15143.69369 2.93693 299.93695 151.43693 15143.69378 2.93693 299.93693 151.43692999999988 15143.69300 2020-01-01 2020-01-02 2020-01-01 00:16:18 2020-01-02 03:44:39 2020-01-01 00:16:18.000 2020-01-02 03:44:39.000 978 99879 50428.5 5042850 978 99879 50428.5 5042850 -32590 32345 4553.3 455330 -128 127 -0.94 -94 +979 100 10969 99880 2.93993 299.93993 151.43993 15143.99399 2.93994 299.93994 151.43994 15143.99412 2.93993 299.93993 151.43993000000023 15143.99300 2020-01-01 2020-01-02 2020-01-01 00:16:19 2020-01-02 03:44:40 2020-01-01 00:16:19.000 2020-01-02 03:44:40.000 979 99880 50429.5 5042950 979 99880 50429.5 5042950 -32589 32346 4554.3 455430 -128 123 -2.5 -250 +98 102 10088 99998 0.29429 300.29429 150.29429 15179.72372 0.29429 300.29428 150.29429 15179.72363 0.29429 300.29429 150.29428999999985 15179.72329 2020-01-01 2020-01-02 2020-01-01 00:01:38 2020-01-02 03:46:38 2020-01-01 00:01:38.000 2020-01-02 03:46:38.000 98 99998 50048 5054848 98 99998 50048 5054848 -32471 32464 4627.009900990099 467328 -128 127 -1.2673267326732673 -128 +980 100 10970 99881 2.94294 299.94294 151.44294 15144.29429 2.94294 299.94293 151.44294 15144.29426 2.94294 299.94294 151.44294000000022 15144.29400 2020-01-01 2020-01-02 2020-01-01 00:16:20 2020-01-02 03:44:41 2020-01-01 00:16:20.000 2020-01-02 03:44:41.000 980 99881 50430.5 5043050 980 99881 50430.5 5043050 -32588 32347 4555.3 455530 -127 124 -1.5 -150 +981 100 10971 99882 2.94594 299.94594 151.44594 15144.59459 2.94594 299.94595 151.44595 15144.59501 2.94594 299.94594 151.44593999999972 15144.59400 2020-01-01 2020-01-02 2020-01-01 00:16:21 2020-01-02 03:44:42 2020-01-01 00:16:21.000 2020-01-02 03:44:42.000 981 99882 50431.5 5043150 981 99882 50431.5 5043150 -32587 32348 4556.3 455630 -126 125 -0.5 -50 +982 100 10972 99883 2.94894 299.94894 151.44894 15144.89489 2.94894 299.94894 151.44894 15144.89466 2.94894 299.94894 151.4489400000002 15144.89400 2020-01-01 2020-01-02 2020-01-01 00:16:22 2020-01-02 03:44:43 2020-01-01 00:16:22.000 2020-01-02 03:44:43.000 982 99883 50432.5 5043250 982 99883 50432.5 5043250 -32586 32349 4557.3 455730 -125 126 0.5 50 +983 100 10973 99884 2.95195 299.95195 151.45195 15145.19519 2.95195 299.95197 151.45195 15145.19525 2.95195 299.95195 151.4519500000001 15145.19500 2020-01-01 2020-01-02 2020-01-01 00:16:23 2020-01-02 03:44:44 2020-01-01 00:16:23.000 2020-01-02 03:44:44.000 983 99884 50433.5 5043350 983 99884 50433.5 5043350 -32585 32350 4558.3 455830 -124 127 1.5 150 +984 100 10974 99885 2.95495 299.95495 151.45495 15145.49549 2.95495 299.95496 151.45495 15145.49559 2.95495 299.95495 151.45494999999988 15145.49500 2020-01-01 2020-01-02 2020-01-01 00:16:24 2020-01-02 03:44:45 2020-01-01 00:16:24.000 2020-01-02 03:44:45.000 984 99885 50434.5 5043450 984 99885 50434.5 5043450 -32584 32351 4559.3 455930 -128 127 -0.06 -6 +985 100 10975 99886 2.95795 299.95795 151.45795 15145.79579 2.95795 299.95795 151.45795 15145.79573 2.95795 299.95795 151.45794999999998 15145.79500 2020-01-01 2020-01-02 2020-01-01 00:16:25 2020-01-02 03:44:46 2020-01-01 00:16:25.000 2020-01-02 03:44:46.000 985 99886 50435.5 5043550 985 99886 50435.5 5043550 -32583 32352 4560.3 456030 -128 123 -1.62 -162 +986 100 10976 99887 2.96096 299.96096 151.46096 15146.09609 2.96096 299.96097 151.46096 15146.09648 2.96096 299.96096 151.46096000000023 15146.09600 2020-01-01 2020-01-02 2020-01-01 00:16:26 2020-01-02 03:44:47 2020-01-01 00:16:26.000 2020-01-02 03:44:47.000 986 99887 50436.5 5043650 986 99887 50436.5 5043650 -32582 32353 4561.3 456130 -127 124 -0.62 -62 +987 100 10977 99888 2.96396 299.96396 151.46396 15146.39639 2.96396 299.96396 151.46396 15146.39612 2.96396 299.96396 151.46395999999973 15146.39600 2020-01-01 2020-01-02 2020-01-01 00:16:27 2020-01-02 03:44:48 2020-01-01 00:16:27.000 2020-01-02 03:44:48.000 987 99888 50437.5 5043750 987 99888 50437.5 5043750 -32581 32354 4562.3 456230 -126 125 0.38 38 +988 100 10978 99889 2.96696 299.96696 151.46696 15146.69669 2.96696 299.96698 151.46696 15146.69676 2.96696 299.96696 151.46695999999983 15146.69600 2020-01-01 2020-01-02 2020-01-01 00:16:28 2020-01-02 03:44:49 2020-01-01 00:16:28.000 2020-01-02 03:44:49.000 988 99889 50438.5 5043850 988 99889 50438.5 5043850 -32580 32355 4563.3 456330 -125 126 1.38 138 +989 100 10979 99890 2.96996 299.96996 151.46996 15146.99699 2.96997 299.96997 151.46997 15146.99706 2.96996 299.96996 151.46996000000024 15146.99600 2020-01-01 2020-01-02 2020-01-01 00:16:29 2020-01-02 03:44:50 2020-01-01 00:16:29.000 2020-01-02 03:44:50.000 989 99890 50439.5 5043950 989 99890 50439.5 5043950 -32579 32356 4564.3 456430 -124 127 2.38 238 +99 102 10089 99999 0.29729 300.29729 150.29729 15180.02702 0.29729 300.2973 150.29729 15180.02726 0.29729 300.29729 150.29729000000032 15180.02629 2020-01-01 2020-01-02 2020-01-01 00:01:39 2020-01-02 03:46:39 2020-01-01 00:01:39.000 2020-01-02 03:46:39.000 99 99999 50049 5054949 99 99999 50049 5054949 -32470 32465 4628.009900990099 467429 -128 123 -2.801980198019802 -283 +990 100 10980 99891 2.97297 299.97297 151.47297 15147.29729 2.97297 299.97296 151.47297 15147.29735 2.97297 299.97297 151.47297000000015 15147.29700 2020-01-01 2020-01-02 2020-01-01 00:16:30 2020-01-02 03:44:51 2020-01-01 00:16:30.000 2020-01-02 03:44:51.000 990 99891 50440.5 5044050 990 99891 50440.5 5044050 -32578 32357 4565.3 456530 -128 127 0.82 82 +991 100 10981 99892 2.97597 299.97597 151.47597 15147.59759 2.97597 299.97598 151.47597 15147.59795 2.97597 299.97597 151.47597 15147.59700 2020-01-01 2020-01-02 2020-01-01 00:16:31 2020-01-02 03:44:52 2020-01-01 00:16:31.000 2020-01-02 03:44:52.000 991 99892 50441.5 5044150 991 99892 50441.5 5044150 -32577 32358 4566.3 456630 -128 127 -0.74 -74 992 100 10982 99893 2.97897 299.97897 151.47897 15147.89789 2.97897 299.97897 151.47897 15147.89759 2.97897 299.97897 151.4789700000001 15147.89700 2020-01-01 2020-01-02 2020-01-01 00:16:32 2020-01-02 03:44:53 2020-01-01 00:16:32.000 2020-01-02 03:44:53.000 992 99893 50442.5 5044250 992 99893 50442.5 5044250 -32576 32359 4567.3 456730 -128 124 -2.3 -230 -993 100 10983 99894 2.98198 299.98198 151.48198 15148.19819 2.98198 299.982 151.48198 15148.19823 2.98198 299.98198 151.48197999999996 15148.19800 2020-01-01 2020-01-02 2020-01-01 00:16:33 2020-01-02 03:44:54 2020-01-01 00:16:33.000 2020-01-02 03:44:54.000 993 99894 50443.5 5044350 993 99894 50443.5 5044350 -32575 32360 4568.3 456830 -127 125 -1.3 -130 -994 100 10984 99895 2.98498 299.98498 151.48498 15148.49849 2.98498 299.985 151.48498 15148.49853 2.98498 299.98498 151.48497999999998 15148.49800 2020-01-01 2020-01-02 2020-01-01 00:16:34 2020-01-02 03:44:55 2020-01-01 00:16:34.000 2020-01-02 03:44:55.000 994 99895 50444.5 5044450 994 99895 50444.5 5044450 -32574 32361 4569.3 456930 -126 126 -0.3 -30 +993 100 10983 99894 2.98198 299.98198 151.48198 15148.19819 2.98198 299.982 151.48198 15148.19823 2.98198 299.98198 151.48198 15148.19800 2020-01-01 2020-01-02 2020-01-01 00:16:33 2020-01-02 03:44:54 2020-01-01 00:16:33.000 2020-01-02 03:44:54.000 993 99894 50443.5 5044350 993 99894 50443.5 5044350 -32575 32360 4568.3 456830 -127 125 -1.3 -130 +994 100 10984 99895 2.98498 299.98498 151.48498 15148.49849 2.98498 299.985 151.48498 15148.49853 2.98498 299.98498 151.48497999999984 15148.49800 2020-01-01 2020-01-02 2020-01-01 00:16:34 2020-01-02 03:44:55 2020-01-01 00:16:34.000 2020-01-02 03:44:55.000 994 99895 50444.5 5044450 994 99895 50444.5 5044450 -32574 32361 4569.3 456930 -126 126 -0.3 -30 995 100 10985 99896 2.98798 299.98798 151.48798 15148.79879 2.98798 299.98798 151.48798 15148.79882 2.98798 299.98798 151.4879799999999 15148.79800 2020-01-01 2020-01-02 2020-01-01 00:16:35 2020-01-02 03:44:56 2020-01-01 00:16:35.000 2020-01-02 03:44:56.000 995 99896 50445.5 5044550 995 99896 50445.5 5044550 -32573 32362 4570.3 457030 -125 127 0.7 70 -996 100 10986 99897 2.99099 299.99099 151.49099 15149.09909 2.99099 299.991 151.49099 15149.09942 2.99099 299.99099 151.49099000000012 15149.09900 2020-01-01 2020-01-02 2020-01-01 00:16:36 2020-01-02 03:44:57 2020-01-01 00:16:36.000 2020-01-02 03:44:57.000 996 99897 50446.5 5044650 996 99897 50446.5 5044650 -32572 32363 4571.3 457130 -128 127 -0.86 -86 -997 100 10987 99898 2.99399 299.99399 151.49399 15149.39939 2.99399 299.994 151.49399 15149.3991 2.99399 299.99399 151.49399000000005 15149.39900 2020-01-01 2020-01-02 2020-01-01 00:16:37 2020-01-02 03:44:58 2020-01-01 00:16:37.000 2020-01-02 03:44:58.000 997 99898 50447.5 5044750 997 99898 50447.5 5044750 -32571 32364 4572.3 457230 -128 127 -2.42 -242 -998 100 10988 99899 2.99699 299.99699 151.49699 15149.69969 2.99699 299.997 151.49699 15149.6997 2.99699 299.99699 151.49698999999993 15149.69900 2020-01-01 2020-01-02 2020-01-01 00:16:38 2020-01-02 03:44:59 2020-01-01 00:16:38.000 2020-01-02 03:44:59.000 998 99899 50448.5 5044850 998 99899 50448.5 5044850 -32570 32365 4573.3 457330 -128 123 -3.98 -398 +996 100 10986 99897 2.99099 299.99099 151.49099 15149.09909 2.99099 299.991 151.49099 15149.09942 2.99099 299.99099 151.49099000000015 15149.09900 2020-01-01 2020-01-02 2020-01-01 00:16:36 2020-01-02 03:44:57 2020-01-01 00:16:36.000 2020-01-02 03:44:57.000 996 99897 50446.5 5044650 996 99897 50446.5 5044650 -32572 32363 4571.3 457130 -128 127 -0.86 -86 +997 100 10987 99898 2.99399 299.99399 151.49399 15149.39939 2.99399 299.994 151.49399 15149.3991 2.99399 299.99399 151.49399000000025 15149.39900 2020-01-01 2020-01-02 2020-01-01 00:16:37 2020-01-02 03:44:58 2020-01-01 00:16:37.000 2020-01-02 03:44:58.000 997 99898 50447.5 5044750 997 99898 50447.5 5044750 -32571 32364 4572.3 457230 -128 127 -2.42 -242 +998 100 10988 99899 2.99699 299.99699 151.49699 15149.69969 2.99699 299.997 151.49699 15149.6997 2.99699 299.99699 151.49698999999976 15149.69900 2020-01-01 2020-01-02 2020-01-01 00:16:38 2020-01-02 03:44:59 2020-01-01 00:16:38.000 2020-01-02 03:44:59.000 998 99899 50448.5 5044850 998 99899 50448.5 5044850 -32570 32365 4573.3 457330 -128 123 -3.98 -398 ---- select with states ---- -1 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N -2 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N From 66b852cf6990a4e336de271f6ff529a1ac708b7d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 2 Nov 2020 23:55:32 +0800 Subject: [PATCH 0095/1088] Revert "Revert "Try fix IfAggCombinator with NullAggCombinator"" This reverts commit 962a89843cc30df136458e3fe8d95efe3bd28537. --- .../AggregateFunctionCount.cpp | 2 +- .../AggregateFunctionCount.h | 2 +- .../AggregateFunctionIf.cpp | 160 ++++++++++++++++++ src/AggregateFunctions/AggregateFunctionIf.h | 4 + .../AggregateFunctionNull.cpp | 2 +- .../AggregateFunctionWindowFunnel.h | 3 +- src/AggregateFunctions/IAggregateFunction.h | 4 +- ...able_type_with_if_agg_combinator.reference | 3 + ...5_nullable_type_with_if_agg_combinator.sql | 6 + 9 files changed, 181 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.reference create mode 100644 tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql diff --git a/src/AggregateFunctions/AggregateFunctionCount.cpp b/src/AggregateFunctions/AggregateFunctionCount.cpp index 6ea63bedaf0..05824947b87 100644 --- a/src/AggregateFunctions/AggregateFunctionCount.cpp +++ b/src/AggregateFunctions/AggregateFunctionCount.cpp @@ -8,7 +8,7 @@ namespace DB { AggregateFunctionPtr AggregateFunctionCount::getOwnNullAdapter( - const AggregateFunctionPtr &, const DataTypes & types, const Array & params) const + const AggregateFunctionPtr &, const DataTypes & types, const Array & params, const AggregateFunctionProperties & /*properties*/) const { return std::make_shared(types[0], params); } diff --git a/src/AggregateFunctions/AggregateFunctionCount.h b/src/AggregateFunctions/AggregateFunctionCount.h index 29c5de0021c..eb1583df92a 100644 --- a/src/AggregateFunctions/AggregateFunctionCount.h +++ b/src/AggregateFunctions/AggregateFunctionCount.h @@ -69,7 +69,7 @@ public: } AggregateFunctionPtr getOwnNullAdapter( - const AggregateFunctionPtr &, const DataTypes & types, const Array & params) const override; + const AggregateFunctionPtr &, const DataTypes & types, const Array & params, const AggregateFunctionProperties & /*properties*/) const override; }; diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index 19a175de911..47afddaf7ff 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -1,6 +1,7 @@ #include #include #include "registerAggregateFunctions.h" +#include "AggregateFunctionNull.h" namespace DB @@ -8,6 +9,7 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -40,6 +42,164 @@ public: } }; +/** There are two cases: for single argument and variadic. + * Code for single argument is much more efficient. + */ +template +class AggregateFunctionIfNullUnary final + : public AggregateFunctionNullBase> +{ +private: + size_t num_arguments; + + using Base = AggregateFunctionNullBase>; +public: + + String getName() const override + { + return Base::getName() + "If"; + } + + AggregateFunctionIfNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : Base(std::move(nested_function_), arguments, params), num_arguments(arguments.size()) + { + if (num_arguments == 0) + throw Exception("Aggregate function " + getName() + " require at least one argument", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + static inline bool singleFilter(const IColumn ** columns, size_t row_num, size_t num_arguments) + { + const IColumn * filter_column = columns[num_arguments - 1]; + if (const ColumnNullable * nullable_column = typeid_cast(filter_column)) + filter_column = nullable_column->getNestedColumnPtr().get(); + + return assert_cast(*filter_column).getData()[row_num]; + } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + const ColumnNullable * column = assert_cast(columns[0]); + const IColumn * nested_column = &column->getNestedColumn(); + if (!column->isNullAt(row_num) && singleFilter(columns, row_num, num_arguments)) + { + this->setFlag(place); + this->nested_function->add(this->nestedPlace(place), &nested_column, row_num, arena); + } + } +}; + +template +class AggregateFunctionIfNullVariadic final + : public AggregateFunctionNullBase> +{ +public: + + String getName() const override + { + return Base::getName() + "If"; + } + + AggregateFunctionIfNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : Base(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size()) + { + if (number_of_arguments == 1) + throw Exception("Logical error: single argument is passed to AggregateFunctionIfNullVariadic", ErrorCodes::LOGICAL_ERROR); + + if (number_of_arguments > MAX_ARGS) + throw Exception("Maximum number of arguments for aggregate function with Nullable types is " + toString(size_t(MAX_ARGS)), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (size_t i = 0; i < number_of_arguments; ++i) + is_nullable[i] = arguments[i]->isNullable(); + } + + static inline bool singleFilter(const IColumn ** columns, size_t row_num, size_t num_arguments) + { + return assert_cast(*columns[num_arguments - 1]).getData()[row_num]; + } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + /// This container stores the columns we really pass to the nested function. + const IColumn * nested_columns[number_of_arguments]; + + for (size_t i = 0; i < number_of_arguments; ++i) + { + if (is_nullable[i]) + { + const ColumnNullable & nullable_col = assert_cast(*columns[i]); + if (null_is_skipped && nullable_col.isNullAt(row_num)) + { + /// If at least one column has a null value in the current row, + /// we don't process this row. + return; + } + nested_columns[i] = &nullable_col.getNestedColumn(); + } + else + nested_columns[i] = columns[i]; + } + + if (singleFilter(nested_columns, row_num, number_of_arguments)) + { + this->setFlag(place); + this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); + } + } + +private: + using Base = AggregateFunctionNullBase>; + + enum { MAX_ARGS = 8 }; + size_t number_of_arguments = 0; + std::array is_nullable; /// Plain array is better than std::vector due to one indirection less. +}; + + +AggregateFunctionPtr AggregateFunctionIf::getOwnNullAdapter( + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, + const Array & params, const AggregateFunctionProperties & properties) const +{ + bool return_type_is_nullable = !properties.returns_default_when_only_null && getReturnType()->canBeInsideNullable(); + size_t nullable_size = std::count_if(arguments.begin(), arguments.end(), [](const auto & element) { return element->isNullable(); }); + return_type_is_nullable &= nullable_size != 1 || !arguments.back()->isNullable(); /// If only condition is nullable. we should non-nullable type. + bool serialize_flag = return_type_is_nullable || properties.returns_default_when_only_null; + + if (arguments.size() <= 2 && arguments.front()->isNullable()) + { + if (return_type_is_nullable) + { + return std::make_shared>(nested_func, arguments, params); + } + else + { + if (serialize_flag) + return std::make_shared>(nested_func, arguments, params); + else + return std::make_shared>(nested_func, arguments, params); + } + } + else + { + if (return_type_is_nullable) + { + return std::make_shared>(nested_function, arguments, params); + } + else + { + if (serialize_flag) + return std::make_shared>(nested_function, arguments, params); + else + return std::make_shared>(nested_function, arguments, params); + } + } +} + void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory & factory) { factory.registerCombinator(std::make_shared()); diff --git a/src/AggregateFunctions/AggregateFunctionIf.h b/src/AggregateFunctions/AggregateFunctionIf.h index f04450c9142..d5d2b9be0dd 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.h +++ b/src/AggregateFunctions/AggregateFunctionIf.h @@ -109,6 +109,10 @@ public: { return nested_func->isState(); } + + AggregateFunctionPtr getOwnNullAdapter( + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, + const Array & params, const AggregateFunctionProperties & properties) const override; }; } diff --git a/src/AggregateFunctions/AggregateFunctionNull.cpp b/src/AggregateFunctions/AggregateFunctionNull.cpp index 5e0d6ee6e21..f584ae1f34c 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.cpp +++ b/src/AggregateFunctions/AggregateFunctionNull.cpp @@ -72,7 +72,7 @@ public: assert(nested_function); - if (auto adapter = nested_function->getOwnNullAdapter(nested_function, arguments, params)) + if (auto adapter = nested_function->getOwnNullAdapter(nested_function, arguments, params, properties)) return adapter; /// If applied to aggregate function with -State combinator, we apply -Null combinator to it's nested_function instead of itself. diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index 3297819a9ff..fe45fec4b76 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -241,7 +241,8 @@ public: } AggregateFunctionPtr getOwnNullAdapter( - const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params) const override + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, + const AggregateFunctionProperties & /*properties*/) const override { return std::make_shared>(nested_function, arguments, params); } diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 4f9552d2345..b5a15eb8cbe 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -33,6 +33,7 @@ using ConstAggregateDataPtr = const char *; class IAggregateFunction; using AggregateFunctionPtr = std::shared_ptr; +struct AggregateFunctionProperties; /** Aggregate functions interface. * Instances of classes with this interface do not contain the data itself for aggregation, @@ -185,7 +186,8 @@ public: * arguments and params are for nested_function. */ virtual AggregateFunctionPtr getOwnNullAdapter( - const AggregateFunctionPtr & /*nested_function*/, const DataTypes & /*arguments*/, const Array & /*params*/) const + const AggregateFunctionPtr & /*nested_function*/, const DataTypes & /*arguments*/, + const Array & /*params*/, const AggregateFunctionProperties & /*properties*/) const { return nullptr; } diff --git a/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.reference b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.reference new file mode 100644 index 00000000000..77f38b722ce --- /dev/null +++ b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.reference @@ -0,0 +1,3 @@ +\N Nullable(UInt8) +\N Nullable(UInt8) +0 UInt8 diff --git a/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql new file mode 100644 index 00000000000..852660117f5 --- /dev/null +++ b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql @@ -0,0 +1,6 @@ +-- Value nullable +SELECT anyIf(CAST(number, 'Nullable(UInt8)'), number = 3) AS a, toTypeName(a) FROM numbers(2); +-- Value and condition nullable +SELECT anyIf(number, number = 3) AS a, toTypeName(a) FROM (SELECT CAST(number, 'Nullable(UInt8)') AS number FROM numbers(2)); +-- Condition nullable +SELECT anyIf(CAST(number, 'UInt8'), number = 3) AS a, toTypeName(a) FROM (SELECT CAST(number, 'Nullable(UInt8)') AS number FROM numbers(2)); From 869578f072157321cd05db1a9842b45be3d4dcec Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 2 Nov 2020 23:57:14 +0800 Subject: [PATCH 0096/1088] ISSUES-16574 try fix if suffix agg function with remote query --- src/AggregateFunctions/AggregateFunctionIf.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index 47afddaf7ff..d654387e5db 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -59,7 +59,7 @@ public: String getName() const override { - return Base::getName() + "If"; + return Base::getName(); } AggregateFunctionIfNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) @@ -100,7 +100,7 @@ public: String getName() const override { - return Base::getName() + "If"; + return Base::getName(); } AggregateFunctionIfNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) From fcbc0fb91e459f70611ae83c82959aaf5bb5ceb6 Mon Sep 17 00:00:00 2001 From: myrrc Date: Tue, 3 Nov 2020 16:12:27 +0300 Subject: [PATCH 0097/1088] added specialized performance test --- tests/performance/avg_weighted.xml | 31 ++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) create mode 100644 tests/performance/avg_weighted.xml diff --git a/tests/performance/avg_weighted.xml b/tests/performance/avg_weighted.xml new file mode 100644 index 00000000000..5682e5bbfb9 --- /dev/null +++ b/tests/performance/avg_weighted.xml @@ -0,0 +1,31 @@ + + DROP TABLE IF EXISTS perf_avg + SET allow_experimental_bigint_types=1 + + CREATE TABLE perf_avg( + num UInt64, + num_u Decimal256(75) DEFAULT toDecimal256(num / 100000, 75), + num_f Float64 DEFAULT num + ) ENGINE = MergeTree() ORDER BY tuple() + + + + INSERT INTO perf_avg(num) + SELECT number / r + FROM system.numbers + ARRAY JOIN range(1, 1000000) AS r + LIMIT 500000000 + + + SELECT avg(num) FROM perf_avg + SELECT avg(num_u) FROM perf_avg + SELECT avg(num_f) FROM perf_avg + SELECT avgWeighted(num_f, num) FROM perf_avg + SELECT avgWeighted(num_f, num_f) FROM perf_avg + SELECT avgWeighted(num_f, num_u) FROM perf_avg + SELECT avgWeighted(num_u, num_f) FROM perf_avg + SELECT avgWeighted(num_u, num) FROM perf_avg + SELECT avgWeighted(num_u, num_u) FROM perf_avg + + DROP TABLE IF EXISTS perf_avg + From 93c2ad603426e7de3deea8c5226bbf4884c4e2a1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 3 Nov 2020 17:49:30 +0300 Subject: [PATCH 0098/1088] Something work --- src/Common/ZooKeeper/TestKeeperStorage.cpp | 180 ++++++++++++++++++++- src/Common/ZooKeeper/TestKeeperStorage.h | 22 +-- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 18 +++ src/Common/ZooKeeper/ZooKeeperCommon.h | 4 + src/Common/ZooKeeper/ZooKeeperImpl.cpp | 6 +- src/Server/TestKeeperTCPHandler.cpp | 33 +++- src/Server/TestKeeperTCPHandler.h | 6 +- 7 files changed, 252 insertions(+), 17 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Common/ZooKeeper/TestKeeperStorage.cpp index 96593b291d6..2024daab7ee 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.cpp +++ b/src/Common/ZooKeeper/TestKeeperStorage.cpp @@ -1,5 +1,9 @@ #include #include +#include +#include +#include +#include namespace zkutil { @@ -13,6 +17,14 @@ static String parentPath(const String & path) return "/"; } + +TestKeeperStorage::TestKeeperStorage() +{ + container.emplace("/", Node()); + + processing_thread = ThreadFromGlobalPool([this] { processingThread(); }); +} + using Undo = std::function; struct TestKeeperStorageRequest @@ -27,11 +39,22 @@ struct TestKeeperStorageRequest virtual ~TestKeeperStorageRequest() {} }; +struct TestKeeperStorageHeartbeatRequest final : public TestKeeperStorageRequest +{ + using TestKeeperStorageRequest::TestKeeperStorageRequest; + std::pair process(TestKeeperStorage::Container & /* container */, int64_t /* zxid */) const override + { + return {zk_request->makeResponse(), {}}; + } +}; + struct TestKeeperStorageCreateRequest final : public TestKeeperStorageRequest { + using TestKeeperStorageRequest::TestKeeperStorageRequest; std::pair process(TestKeeperStorage::Container & container, int64_t zxid) const override { + LOG_DEBUG(&Poco::Logger::get("STORAGE"), "EXECUTING CREATE REQUEST"); Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Undo undo; Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); @@ -104,6 +127,161 @@ struct TestKeeperStorageCreateRequest final : public TestKeeperStorageRequest } }; +struct TestKeeperStorageGetRequest final : public TestKeeperStorageRequest +{ + using TestKeeperStorageRequest::TestKeeperStorageRequest; + std::pair process(TestKeeperStorage::Container & container, int64_t /* zxid */) const override + { + LOG_DEBUG(&Poco::Logger::get("STORAGE"), "EXECUTING GET REQUEST"); + Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); + Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); + + auto it = container.find(request.path); + if (it == container.end()) + { + response.error = Coordination::Error::ZNONODE; + } + else + { + response.stat = it->second.stat; + response.data = it->second.data; + response.error = Coordination::Error::ZOK; + } + + return { response_ptr, {} }; + } +}; + +void TestKeeperStorage::processingThread() +{ + setThreadName("TestKeeperSProc"); + + LOG_DEBUG(&Poco::Logger::get("STORAGE"), "LOOPING IN THREAD"); + try + { + while (!shutdown) + { + RequestInfo info; + + UInt64 max_wait = UInt64(operation_timeout.totalMilliseconds()); + + if (requests_queue.tryPop(info, max_wait)) + { + if (shutdown) + break; + + ++zxid; + + auto zk_request = info.request->zk_request; + LOG_DEBUG(&Poco::Logger::get("STORAGE"), "GOT REQUEST {}", zk_request->getOpNum()); + Coordination::ZooKeeperResponsePtr response; + if (zk_request->xid == -2) + { + response = std::make_shared(); + response->xid = zk_request->xid; + response->zxid = zxid; + } + else + { + zk_request->addRootPath(root_path); + LOG_DEBUG(&Poco::Logger::get("STORAGE"), "PROCESSING REQUEST"); + std::tie(response, std::ignore) = info.request->process(container, zxid); + response->xid = zk_request->xid; + LOG_DEBUG(&Poco::Logger::get("STORAGE"), "SENDING XID {}", response->xid); + response->zxid = zxid; + + response->removeRootPath(root_path); + } + + LOG_DEBUG(&Poco::Logger::get("STORAGE"), "SENDING RESPONSE"); + info.response_callback(response); + LOG_DEBUG(&Poco::Logger::get("STORAGE"), "DONE"); + } + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + finalize(); + } +} + + +void TestKeeperStorage::finalize() +{ + { + std::lock_guard lock(push_request_mutex); + + if (shutdown) + return; + shutdown = true; + } + try + { + RequestInfo info; + while (requests_queue.tryPop(info)) + { + auto response = info.request->zk_request->makeResponse(); + response->error = Coordination::Error::ZSESSIONEXPIRED; + try + { + info.response_callback(response); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + +} + +TestKeeperStorage::AsyncResponse TestKeeperStorage::putRequest(const Coordination::ZooKeeperRequestPtr & request) +{ + auto promise = std::make_shared>(); + auto future = promise->get_future(); + TestKeeperStorageRequestPtr storage_request; + if (request->xid == -2) + storage_request = std::make_shared(request); + else if (request->getOpNum() == 1) + storage_request = std::make_shared(request); + else if (request->getOpNum() == 4) + storage_request = std::make_shared(request); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "UNKNOWN EVENT WITH OPNUM {}", request->getOpNum()) +; + RequestInfo request_info; + request_info.time = clock::now(); + request_info.request = storage_request; + request_info.response_callback = [promise] (const Coordination::ZooKeeperResponsePtr & response) { promise->set_value(response); }; + + std::lock_guard lock(push_request_mutex); + if (!requests_queue.tryPush(std::move(request_info), operation_timeout.totalMilliseconds())) + throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::LOGICAL_ERROR); + LOG_DEBUG(&Poco::Logger::get("STORAGE"), "PUSHED"); + return future; +} + + +TestKeeperStorage::~TestKeeperStorage() +{ + try + { + finalize(); + if (processing_thread.joinable()) + processing_thread.join(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + + - } diff --git a/src/Common/ZooKeeper/TestKeeperStorage.h b/src/Common/ZooKeeper/TestKeeperStorage.h index 893f9f2842c..86be2e0eeaf 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.h +++ b/src/Common/ZooKeeper/TestKeeperStorage.h @@ -4,19 +4,21 @@ #include #include #include +#include namespace zkutil { using namespace DB; +struct TestKeeperStorageRequest; +using TestKeeperStorageRequestPtr = std::shared_ptr; class TestKeeperStorage { public: - struct TestKeeperRequest; - using TestKeeperRequestPtr = std::shared_ptr; + Poco::Timespan operation_timeout{10000}; std::atomic session_id_counter{0}; struct Node @@ -39,6 +41,7 @@ public: String root_path; std::atomic zxid{0}; + std::atomic shutdown{false}; Watches watches; Watches list_watches; /// Watches for 'list' request (watches on children). @@ -47,26 +50,25 @@ public: struct RequestInfo { - TestKeeperRequestPtr request; - Coordination::ResponseCallback callback; - Coordination::WatchCallback watch; + TestKeeperStorageRequestPtr request; + std::function response_callback; clock::time_point time; }; + std::mutex push_request_mutex; using RequestsQueue = ConcurrentBoundedQueue; RequestsQueue requests_queue{1}; - void pushRequest(RequestInfo && request); - void finalize(); ThreadFromGlobalPool processing_thread; void processingThread(); - void writeResponse(const Coordination::ZooKeeperResponsePtr & response); - public: - void putRequest(const Coordination::ZooKeeperRequestPtr & request, std::shared_ptr response_out); + using AsyncResponse = std::future; + TestKeeperStorage(); + ~TestKeeperStorage(); + AsyncResponse putRequest(const Coordination::ZooKeeperRequestPtr & request); int64_t getSessionID() { diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index eb04536ae00..7d7bbbd8dc3 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -36,6 +37,8 @@ static void write(bool x, WriteBuffer & out) static void write(const String & s, WriteBuffer & out) { + + LOG_DEBUG(&Poco::Logger::get("LOG"), "S SIZE {}", s.size()); write(int32_t(s.size()), out); out.write(s.data(), s.size()); } @@ -170,6 +173,20 @@ static void read(ACL & acl, ReadBuffer & in) read(acl.id, in); } +void ZooKeeperResponse::write(WriteBuffer & out) const +{ + /// Excessive copy to calculate length. + WriteBufferFromOwnString buf; + LOG_DEBUG(&Poco::Logger::get("LOG"), "WRITING {}", xid); + Coordination::write(xid, buf); + Coordination::write(zxid, buf); + Coordination::write(error, buf); + writeImpl(buf); + LOG_DEBUG(&Poco::Logger::get("LOG"), "BUFFER LENGTH {}", buf.str().length()); + Coordination::write(buf.str(), out); + out.next(); +} + void ZooKeeperRequest::write(WriteBuffer & out) const { /// Excessive copy to calculate length. @@ -247,6 +264,7 @@ void ZooKeeperCreateResponse::readImpl(ReadBuffer & in) void ZooKeeperCreateResponse::writeImpl(WriteBuffer & out) const { + LOG_DEBUG(&Poco::Logger::get("LOG"), "WRITE IMPL ON: {}", path_created); Coordination::write(path_created, out); } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 05886fc0468..f40de116da2 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -27,9 +27,13 @@ using OpNum = int32_t; struct ZooKeeperResponse : virtual Response { + XID xid = 0; + int64_t zxid; + virtual ~ZooKeeperResponse() override = default; virtual void readImpl(ReadBuffer &) = 0; virtual void writeImpl(WriteBuffer &) const = 0; + void write(WriteBuffer & out) const; virtual OpNum getOpNum() const = 0; }; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index f3c8b537cf2..30b30695eea 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include @@ -750,9 +751,12 @@ void ZooKeeper::receiveEvent() Error err; read(length); + std::cerr << "RECEIVED LENGTH " << length << std::endl; size_t count_before_event = in->count(); read(xid); + std::cerr << "RECEIVED XID " << xid << std::endl; read(zxid); + std::cerr << "RECEIVED ZXID " << zxid << std::endl; read(err); RequestInfo request_info; @@ -806,7 +810,7 @@ void ZooKeeper::receiveEvent() auto it = operations.find(xid); if (it == operations.end()) - throw Exception("Received response for unknown xid", Error::ZRUNTIMEINCONSISTENCY); + throw Exception("Received response for unknown xid " + toString(xid), Error::ZRUNTIMEINCONSISTENCY); /// After this point, we must invoke callback, that we've grabbed from 'operations'. /// Invariant: all callbacks are invoked either in case of success or in case of error. diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index a0679554f64..baf78e1fc59 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB { @@ -279,16 +280,23 @@ void TestKeeperTCPHandler::runImpl() while (true) { UInt64 max_wait = operation_timeout.totalMicroseconds(); + using namespace std::chrono_literals; + if (!responses.empty() && responses.front().wait_for(100ms) == std::future_status::ready) + { + auto response = responses.front().get(); + response->write(*out); + responses.pop(); + } + if (in->poll(max_wait)) { receiveHeartbeatRequest(); - sendHeartbeatResponse(); } } } -void TestKeeperTCPHandler::receiveHeartbeatRequest() +bool TestKeeperTCPHandler::receiveHeartbeatRequest() { LOG_DEBUG(log, "Receiving heartbeat event"); int32_t length; @@ -301,12 +309,14 @@ void TestKeeperTCPHandler::receiveHeartbeatRequest() LOG_DEBUG(log, "Received xid {}", xid); + Coordination::ZooKeeperRequestPtr request; if (xid == -2) { int32_t opnum; read(opnum, *in); LOG_DEBUG(log, "RRECEIVED OP NUM {}", opnum); - auto request = std::make_shared(); + request = std::make_shared(); + request->xid = xid; request->readImpl(*in); int32_t readed = in->count() - total_count; if (readed != length) @@ -314,10 +324,25 @@ void TestKeeperTCPHandler::receiveHeartbeatRequest() } else { - LOG_INFO(log, "UNKNOWN EVENT xid:{}", xid); + int32_t opnum; + read(opnum, *in); + LOG_DEBUG(log, "RRECEIVED OP NUM {}", opnum); + if (opnum == 1) + request = std::make_shared(); + else if (opnum == 4) + request = std::make_shared(); + request->readImpl(*in); + request->xid = xid; + int32_t readed = in->count() - total_count; + if (readed != length) + LOG_DEBUG(log, "EXPECTED TO READ {}, BUT GOT {}", length, readed); + LOG_DEBUG(log, "REQUEST PUTTED TO STORAGE"); } + responses.push(test_keeper_storage->putRequest(request)); + LOG_DEBUG(log, "Event received"); + return false; } diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h index 695f5f04382..fd2c6227e73 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/TestKeeperTCPHandler.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -32,6 +33,9 @@ private: std::shared_ptr test_keeper_storage; Poco::Timespan operation_timeout; + std::queue responses; + + /// Streams for reading/writing from/to client connection socket. std::shared_ptr in; std::shared_ptr out; @@ -41,7 +45,7 @@ private: void sendHandshake(); void receiveHandshake(); - void receiveHeartbeatRequest(); + bool receiveHeartbeatRequest(); void sendHeartbeatResponse(); void receiveCreateRequest(); From ab1b7267b682a6ebb854c4f8fefaf6136427a2f5 Mon Sep 17 00:00:00 2001 From: myrrc Date: Tue, 3 Nov 2020 17:56:07 +0300 Subject: [PATCH 0099/1088] fixed the build, added some comments --- cmake/find/avro.cmake | 1 + cmake/find/ssl.cmake | 2 + .../AggregateFunctionAvg.cpp | 15 ++++++-- src/AggregateFunctions/AggregateFunctionAvg.h | 21 +++++++++-- .../AggregateFunctionAvgWeighted.cpp | 37 ++++++++++++++++++- .../AggregateFunctionAvgWeighted.h | 27 +++++++++++--- 6 files changed, 91 insertions(+), 12 deletions(-) diff --git a/cmake/find/avro.cmake b/cmake/find/avro.cmake index e0f73d99111..74ccda3489f 100644 --- a/cmake/find/avro.cmake +++ b/cmake/find/avro.cmake @@ -1,3 +1,4 @@ +# Needed when using Apache Avro serialization format option (ENABLE_AVRO "Enable Avro" ${ENABLE_LIBRARIES}) if (NOT ENABLE_AVRO) diff --git a/cmake/find/ssl.cmake b/cmake/find/ssl.cmake index 9058857c173..f7ac9174202 100644 --- a/cmake/find/ssl.cmake +++ b/cmake/find/ssl.cmake @@ -1,3 +1,5 @@ +# Needed when securely connecting to an external server, e.g. +# clickhouse-client --host ... --secure option(ENABLE_SSL "Enable ssl" ${ENABLE_LIBRARIES}) if(NOT ENABLE_SSL) diff --git a/src/AggregateFunctions/AggregateFunctionAvg.cpp b/src/AggregateFunctions/AggregateFunctionAvg.cpp index bec0a778c2d..0a2f9e57fac 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvg.cpp @@ -25,11 +25,20 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const assertNoParameters(name, parameters); assertUnary(name, argument_types); - if (!allowType(argument_types[0])) - throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, + const DataTypePtr& data_type = argument_types[0]; + + if (!allowType(data_type)) + throw Exception("Illegal type " + data_type->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return std::make_shared(argument_types); + AggregateFunctionPtr res; + + if (isDecimal(data_type)) + res.reset(createWithDecimalType(*data_type, argument_types)); + else + res.reset(createWithNumericType(*data_type, argument_types)); + + return res; } } diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 7c2ad3d0984..29d9d90cc5d 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -21,6 +21,13 @@ struct RationalFraction Float64 NO_SANITIZE_UNDEFINED result() const { return numerator / denominator; } }; +template constexpr bool DecimalOrExtendedInt = + IsDecimalNumber + || std::is_same_v + || std::is_same_v + || std::is_same_v + || std::is_same_v; + /** * The discussion showed that the easiest (and simplest) way is to cast both the columns of numerator and denominator * to Float64. Another way would be to write some template magic that figures out the appropriate numerator @@ -78,14 +85,22 @@ public: } }; -class AggregateFunctionAvg final : public AggregateFunctionAvgBase +template +class AggregateFunctionAvg final : public AggregateFunctionAvgBase> { public: - using AggregateFunctionAvgBase::AggregateFunctionAvgBase; + using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const final { - this->data(place).numerator += columns[0]->getFloat64(row_num); + if constexpr(IsDecimalNumber) + this->data(place).numerator += columns[0]->getFloat64(row_num); + else if constexpr(DecimalOrExtendedInt) + this->data(place).numerator += static_cast( + static_cast &>(*columns[0]).getData()[row_num]); + else + this->data(place).numerator += static_cast &>(*columns[0]).getData()[row_num]; + ++this->data(place).denominator; } diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp index 9045bff002a..b9499dee460 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp @@ -27,6 +27,39 @@ bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) noexcept return allow(l_dt) && allow(r_dt); } +#define AT_SWITCH(LINE) \ + switch (which.idx) \ + { \ + LINE(Int8); LINE(Int16); LINE(Int32); LINE(Int64); LINE(Int128); LINE(Int256); \ + LINE(UInt8); LINE(UInt16); LINE(UInt32); LINE(UInt64); LINE(UInt128); LINE(UInt256); \ + LINE(Decimal32); LINE(Decimal64); LINE(Decimal128); LINE(Decimal256); \ + LINE(Float32); LINE(Float64); \ + default: return nullptr; \ + } + +template +static IAggregateFunction * create(const IDataType & second_type, TArgs && ... args) +{ + const WhichDataType which(second_type); + +#define LINE(Type) \ + case TypeIndex::Type: return new AggregateFunctionAvgWeighted(std::forward(args)...) + AT_SWITCH(LINE) +#undef LINE +} + +// Not using helper functions because there are no templates for binary decimal/numeric function. +template +static IAggregateFunction * create(const IDataType & first_type, const IDataType & second_type, TArgs && ... args) +{ + const WhichDataType which(first_type); + +#define LINE(Type) \ + case TypeIndex::Type: return create(second_type, std::forward(args)...) + AT_SWITCH(LINE) +#undef LINE +} + AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name, const DataTypes & argument_types, const Array & parameters) { assertNoParameters(name, parameters); @@ -42,7 +75,9 @@ AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name " are non-conforming as arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return std::make_shared(argument_types); + AggregateFunctionPtr ptr; + ptr.reset(create(*data_type, *data_type_weight, argument_types)); + return ptr; } } diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index ca9f0757cba..db72693fa6f 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -5,17 +5,34 @@ namespace DB { -class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase +template +class AggregateFunctionAvgWeighted final : + public AggregateFunctionAvgBase> { public: - using AggregateFunctionAvgBase::AggregateFunctionAvgBase; + using Base = AggregateFunctionAvgBase>; + using Base::Base; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - const auto value = columns[0]->getFloat64(row_num); - const auto weight = columns[1]->getFloat64(row_num); + const Float64 value = [&columns, row_num] { + if constexpr(IsDecimalNumber) + return columns[0]->getFloat64(row_num); + else + return static_cast(static_cast&>(*columns[0]).getData()[row_num]); + }(); - this->data(place).numerator += value * weight; + using WeightRet = std::conditional_t, Float64, Weight>; + const WeightRet weight = [&columns, row_num]() -> WeightRet { + if constexpr(IsDecimalNumber) + return columns[1]->getFloat64(row_num); + else if constexpr(DecimalOrExtendedInt) + return static_cast(static_cast&>(*columns[1]).getData()[row_num]); + else + return static_cast&>(*columns[1]).getData()[row_num]; + }(); + + this->data(place).numerator += weight * value; this->data(place).denominator += weight; } From 5700e5e46d3c495342149a6484798e4157766fa7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 3 Nov 2020 17:57:40 +0300 Subject: [PATCH 0100/1088] Fix response serialization --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 7d7bbbd8dc3..5ab22474d72 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -181,7 +181,8 @@ void ZooKeeperResponse::write(WriteBuffer & out) const Coordination::write(xid, buf); Coordination::write(zxid, buf); Coordination::write(error, buf); - writeImpl(buf); + if (error == Error::ZOK) + writeImpl(buf); LOG_DEBUG(&Poco::Logger::get("LOG"), "BUFFER LENGTH {}", buf.str().length()); Coordination::write(buf.str(), out); out.next(); From 5c6bd218adb19902657601fa60f3186de8a71ab3 Mon Sep 17 00:00:00 2001 From: myrrc Date: Tue, 3 Nov 2020 18:01:29 +0300 Subject: [PATCH 0101/1088] adjusted the perftest --- src/AggregateFunctions/AggregateFunctionAvgWeighted.h | 4 ++-- tests/performance/avg_weighted.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index db72693fa6f..0c486232e52 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -24,9 +24,9 @@ public: using WeightRet = std::conditional_t, Float64, Weight>; const WeightRet weight = [&columns, row_num]() -> WeightRet { - if constexpr(IsDecimalNumber) + if constexpr(IsDecimalNumber) /// Unable to cast to double -> use the virtual method return columns[1]->getFloat64(row_num); - else if constexpr(DecimalOrExtendedInt) + else if constexpr(DecimalOrExtendedInt) /// Casting to double, otherwise += would be ambitious. return static_cast(static_cast&>(*columns[1]).getData()[row_num]); else return static_cast&>(*columns[1]).getData()[row_num]; diff --git a/tests/performance/avg_weighted.xml b/tests/performance/avg_weighted.xml index 5682e5bbfb9..bd0ab2395ff 100644 --- a/tests/performance/avg_weighted.xml +++ b/tests/performance/avg_weighted.xml @@ -13,8 +13,8 @@ INSERT INTO perf_avg(num) SELECT number / r FROM system.numbers - ARRAY JOIN range(1, 1000000) AS r - LIMIT 500000000 + ARRAY JOIN range(1, 10000) AS r + LIMIT 5000000 SELECT avg(num) FROM perf_avg From 598532b58216d1ab1cf9f3ee766778a55476801f Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 3 Nov 2020 18:01:56 +0300 Subject: [PATCH 0102/1088] Remove strange method --- src/Common/ZooKeeper/TestKeeper.cpp | 3 --- src/Common/ZooKeeper/TestKeeperStorage.cpp | 1 - 2 files changed, 4 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 4f7beadef5f..b25e20afeda 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -31,7 +31,6 @@ using Undo = std::function; struct TestKeeperRequest : virtual Request { - virtual bool isMutable() const { return false; } virtual ResponsePtr createResponse() const = 0; virtual std::pair process(TestKeeper::Container & container, int64_t zxid) const = 0; virtual void processWatches(TestKeeper::Watches & /*watches*/, TestKeeper::Watches & /*list_watches*/) const {} @@ -85,7 +84,6 @@ struct TestKeeperRemoveRequest final : RemoveRequest, TestKeeperRequest { TestKeeperRemoveRequest() = default; explicit TestKeeperRemoveRequest(const RemoveRequest & base) : RemoveRequest(base) {} - bool isMutable() const override { return true; } ResponsePtr createResponse() const override; std::pair process(TestKeeper::Container & container, int64_t zxid) const override; @@ -112,7 +110,6 @@ struct TestKeeperSetRequest final : SetRequest, TestKeeperRequest { TestKeeperSetRequest() = default; explicit TestKeeperSetRequest(const SetRequest & base) : SetRequest(base) {} - bool isMutable() const override { return true; } ResponsePtr createResponse() const override; std::pair process(TestKeeper::Container & container, int64_t zxid) const override; diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Common/ZooKeeper/TestKeeperStorage.cpp index 2024daab7ee..50564e6ae54 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.cpp +++ b/src/Common/ZooKeeper/TestKeeperStorage.cpp @@ -34,7 +34,6 @@ struct TestKeeperStorageRequest TestKeeperStorageRequest(const Coordination::ZooKeeperRequestPtr & zk_request_) : zk_request(zk_request_) {} - virtual bool isMutable() const { return false; } virtual std::pair process(TestKeeperStorage::Container & container, int64_t zxid) const = 0; virtual ~TestKeeperStorageRequest() {} }; From d12f2f64fb168f05345309872a3dd2bb95d4ecab Mon Sep 17 00:00:00 2001 From: Konstantin Malanchev Date: Tue, 3 Nov 2020 13:59:27 +0100 Subject: [PATCH 0103/1088] Add some of missed cmath functions acosh asinh atan2 atanh cosh hypot log1p sinh --- src/Functions/acosh.cpp | 19 +++++++++++++++++++ src/Functions/asinh.cpp | 19 +++++++++++++++++++ src/Functions/atan2.cpp | 19 +++++++++++++++++++ src/Functions/atanh.cpp | 19 +++++++++++++++++++ src/Functions/cosh.cpp | 19 +++++++++++++++++++ src/Functions/hypot.cpp | 19 +++++++++++++++++++ src/Functions/log1p.cpp | 19 +++++++++++++++++++ src/Functions/registerFunctionsMath.cpp | 17 ++++++++++++++++- src/Functions/sinh.cpp | 19 +++++++++++++++++++ 9 files changed, 168 insertions(+), 1 deletion(-) create mode 100644 src/Functions/acosh.cpp create mode 100644 src/Functions/asinh.cpp create mode 100644 src/Functions/atan2.cpp create mode 100644 src/Functions/atanh.cpp create mode 100644 src/Functions/cosh.cpp create mode 100644 src/Functions/hypot.cpp create mode 100644 src/Functions/log1p.cpp create mode 100644 src/Functions/sinh.cpp diff --git a/src/Functions/acosh.cpp b/src/Functions/acosh.cpp new file mode 100644 index 00000000000..b30bce19230 --- /dev/null +++ b/src/Functions/acosh.cpp @@ -0,0 +1,19 @@ +#include +#include + +namespace DB +{ +namespace +{ + +struct AcoshName { static constexpr auto name = "acosh"; }; +using FunctionAcosh = FunctionMathUnary>; + +} + +void registerFunctionAcosh(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/asinh.cpp b/src/Functions/asinh.cpp new file mode 100644 index 00000000000..a383c6ad6e5 --- /dev/null +++ b/src/Functions/asinh.cpp @@ -0,0 +1,19 @@ +#include +#include + +namespace DB +{ +namespace +{ + +struct AsinhName { static constexpr auto name = "asinh"; }; +using FunctionAsinh = FunctionMathUnary>; + +} + +void registerFunctionAsinh(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/atan2.cpp b/src/Functions/atan2.cpp new file mode 100644 index 00000000000..e04891a1977 --- /dev/null +++ b/src/Functions/atan2.cpp @@ -0,0 +1,19 @@ +#include +#include + +namespace DB +{ +namespace +{ + +struct Atan2Name { static constexpr auto name = "atan2"; }; +using FunctionAtan2 = FunctionMathBinaryFloat64>; + +} + +void registerFunctionAtan2(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/atanh.cpp b/src/Functions/atanh.cpp new file mode 100644 index 00000000000..d0a5409fc57 --- /dev/null +++ b/src/Functions/atanh.cpp @@ -0,0 +1,19 @@ +#include +#include + +namespace DB +{ +namespace +{ + +struct AtanhName { static constexpr auto name = "atanh"; }; +using FunctionAtanh = FunctionMathUnary>; + +} + +void registerFunctionAtanh(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/cosh.cpp b/src/Functions/cosh.cpp new file mode 100644 index 00000000000..e09b7a1b1cb --- /dev/null +++ b/src/Functions/cosh.cpp @@ -0,0 +1,19 @@ +#include +#include + +namespace DB +{ +namespace +{ + +struct CoshName { static constexpr auto name = "cosh"; }; +using FunctionCosh = FunctionMathUnary>; + +} + +void registerFunctionCosh(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/hypot.cpp b/src/Functions/hypot.cpp new file mode 100644 index 00000000000..aa5a98f0e2a --- /dev/null +++ b/src/Functions/hypot.cpp @@ -0,0 +1,19 @@ +#include +#include + +namespace DB +{ +namespace +{ + +struct HypotName { static constexpr auto name = "hypot"; }; +using FunctionHypot = FunctionMathBinaryFloat64>; + +} + +void registerFunctionHypot(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/log1p.cpp b/src/Functions/log1p.cpp new file mode 100644 index 00000000000..f888f51abfe --- /dev/null +++ b/src/Functions/log1p.cpp @@ -0,0 +1,19 @@ +#include +#include + +namespace DB +{ +namespace +{ + +struct Log1pName { static constexpr auto name = "log1p"; }; +using FunctionLog1p = FunctionMathUnary>; + +} + +void registerFunctionLog1p(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/registerFunctionsMath.cpp b/src/Functions/registerFunctionsMath.cpp index e102c725050..004fccfbec1 100644 --- a/src/Functions/registerFunctionsMath.cpp +++ b/src/Functions/registerFunctionsMath.cpp @@ -1,6 +1,5 @@ namespace DB { - class FunctionFactory; void registerFunctionE(FunctionFactory & factory); @@ -9,6 +8,7 @@ void registerFunctionExp(FunctionFactory & factory); void registerFunctionLog(FunctionFactory & factory); void registerFunctionExp2(FunctionFactory & factory); void registerFunctionLog2(FunctionFactory & factory); +void registerFunctionLog1p(FunctionFactory & factory); void registerFunctionExp10(FunctionFactory & factory); void registerFunctionLog10(FunctionFactory & factory); void registerFunctionSqrt(FunctionFactory & factory); @@ -23,8 +23,15 @@ void registerFunctionTan(FunctionFactory & factory); void registerFunctionAsin(FunctionFactory & factory); void registerFunctionAcos(FunctionFactory & factory); void registerFunctionAtan(FunctionFactory & factory); +void registerFunctionAtan2(FunctionFactory & factory); void registerFunctionSigmoid(FunctionFactory & factory); +void registerFunctionHypot(FunctionFactory & factory); +void registerFunctionSinh(FunctionFactory & factory); +void registerFunctionCosh(FunctionFactory & factory); void registerFunctionTanh(FunctionFactory & factory); +void registerFunctionAsinh(FunctionFactory & factory); +void registerFunctionAcosh(FunctionFactory & factory); +void registerFunctionAtanh(FunctionFactory & factory); void registerFunctionPow(FunctionFactory & factory); @@ -36,6 +43,7 @@ void registerFunctionsMath(FunctionFactory & factory) registerFunctionLog(factory); registerFunctionExp2(factory); registerFunctionLog2(factory); + registerFunctionLog1p(factory); registerFunctionExp10(factory); registerFunctionLog10(factory); registerFunctionSqrt(factory); @@ -50,8 +58,15 @@ void registerFunctionsMath(FunctionFactory & factory) registerFunctionAsin(factory); registerFunctionAcos(factory); registerFunctionAtan(factory); + registerFunctionAtan2(factory); registerFunctionSigmoid(factory); + registerFunctionHypot(factory); + registerFunctionSinh(factory); + registerFunctionCosh(factory); registerFunctionTanh(factory); + registerFunctionAsinh(factory); + registerFunctionAcosh(factory); + registerFunctionAtanh(factory); registerFunctionPow(factory); } diff --git a/src/Functions/sinh.cpp b/src/Functions/sinh.cpp new file mode 100644 index 00000000000..6543a8e7438 --- /dev/null +++ b/src/Functions/sinh.cpp @@ -0,0 +1,19 @@ +#include +#include + +namespace DB +{ +namespace +{ + +struct SinhName { static constexpr auto name = "sinh"; }; +using FunctionSinh = FunctionMathUnary>; + +} + +void registerFunctionSinh(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} From 7d138ae717a6fb46f6caea4eb0e9e54ff25e3487 Mon Sep 17 00:00:00 2001 From: Konstantin Malanchev Date: Tue, 3 Nov 2020 16:32:04 +0100 Subject: [PATCH 0104/1088] clang-format --- src/Functions/acosh.cpp | 10 ++++++---- src/Functions/asinh.cpp | 10 ++++++---- src/Functions/atan2.cpp | 10 ++++++---- src/Functions/atanh.cpp | 10 ++++++---- src/Functions/cosh.cpp | 10 ++++++---- src/Functions/hypot.cpp | 10 ++++++---- src/Functions/log1p.cpp | 10 ++++++---- src/Functions/sinh.cpp | 10 ++++++---- 8 files changed, 48 insertions(+), 32 deletions(-) diff --git a/src/Functions/acosh.cpp b/src/Functions/acosh.cpp index b30bce19230..20473faafc7 100644 --- a/src/Functions/acosh.cpp +++ b/src/Functions/acosh.cpp @@ -1,13 +1,15 @@ -#include #include +#include namespace DB { namespace { - -struct AcoshName { static constexpr auto name = "acosh"; }; -using FunctionAcosh = FunctionMathUnary>; + struct AcoshName + { + static constexpr auto name = "acosh"; + }; + using FunctionAcosh = FunctionMathUnary>; } diff --git a/src/Functions/asinh.cpp b/src/Functions/asinh.cpp index a383c6ad6e5..fd2ac6c7a9f 100644 --- a/src/Functions/asinh.cpp +++ b/src/Functions/asinh.cpp @@ -1,13 +1,15 @@ -#include #include +#include namespace DB { namespace { - -struct AsinhName { static constexpr auto name = "asinh"; }; -using FunctionAsinh = FunctionMathUnary>; + struct AsinhName + { + static constexpr auto name = "asinh"; + }; + using FunctionAsinh = FunctionMathUnary>; } diff --git a/src/Functions/atan2.cpp b/src/Functions/atan2.cpp index e04891a1977..0e363440d09 100644 --- a/src/Functions/atan2.cpp +++ b/src/Functions/atan2.cpp @@ -1,13 +1,15 @@ -#include #include +#include namespace DB { namespace { - -struct Atan2Name { static constexpr auto name = "atan2"; }; -using FunctionAtan2 = FunctionMathBinaryFloat64>; + struct Atan2Name + { + static constexpr auto name = "atan2"; + }; + using FunctionAtan2 = FunctionMathBinaryFloat64>; } diff --git a/src/Functions/atanh.cpp b/src/Functions/atanh.cpp index d0a5409fc57..a8154c719fc 100644 --- a/src/Functions/atanh.cpp +++ b/src/Functions/atanh.cpp @@ -1,13 +1,15 @@ -#include #include +#include namespace DB { namespace { - -struct AtanhName { static constexpr auto name = "atanh"; }; -using FunctionAtanh = FunctionMathUnary>; + struct AtanhName + { + static constexpr auto name = "atanh"; + }; + using FunctionAtanh = FunctionMathUnary>; } diff --git a/src/Functions/cosh.cpp b/src/Functions/cosh.cpp index e09b7a1b1cb..88753fcb95b 100644 --- a/src/Functions/cosh.cpp +++ b/src/Functions/cosh.cpp @@ -1,13 +1,15 @@ -#include #include +#include namespace DB { namespace { - -struct CoshName { static constexpr auto name = "cosh"; }; -using FunctionCosh = FunctionMathUnary>; + struct CoshName + { + static constexpr auto name = "cosh"; + }; + using FunctionCosh = FunctionMathUnary>; } diff --git a/src/Functions/hypot.cpp b/src/Functions/hypot.cpp index aa5a98f0e2a..00297713d11 100644 --- a/src/Functions/hypot.cpp +++ b/src/Functions/hypot.cpp @@ -1,13 +1,15 @@ -#include #include +#include namespace DB { namespace { - -struct HypotName { static constexpr auto name = "hypot"; }; -using FunctionHypot = FunctionMathBinaryFloat64>; + struct HypotName + { + static constexpr auto name = "hypot"; + }; + using FunctionHypot = FunctionMathBinaryFloat64>; } diff --git a/src/Functions/log1p.cpp b/src/Functions/log1p.cpp index f888f51abfe..dc3a16224be 100644 --- a/src/Functions/log1p.cpp +++ b/src/Functions/log1p.cpp @@ -1,13 +1,15 @@ -#include #include +#include namespace DB { namespace { - -struct Log1pName { static constexpr auto name = "log1p"; }; -using FunctionLog1p = FunctionMathUnary>; + struct Log1pName + { + static constexpr auto name = "log1p"; + }; + using FunctionLog1p = FunctionMathUnary>; } diff --git a/src/Functions/sinh.cpp b/src/Functions/sinh.cpp index 6543a8e7438..84fe0e805b7 100644 --- a/src/Functions/sinh.cpp +++ b/src/Functions/sinh.cpp @@ -1,13 +1,15 @@ -#include #include +#include namespace DB { namespace { - -struct SinhName { static constexpr auto name = "sinh"; }; -using FunctionSinh = FunctionMathUnary>; + struct SinhName + { + static constexpr auto name = "sinh"; + }; + using FunctionSinh = FunctionMathUnary>; } From 382d67d10a65844989927cd04e205af6e9b98f49 Mon Sep 17 00:00:00 2001 From: Konstantin Malanchev Date: Tue, 3 Nov 2020 18:01:07 +0100 Subject: [PATCH 0105/1088] Add tests --- .../00087_math_functions.reference | 40 ++++++++++++++++ .../0_stateless/00087_math_functions.sql | 48 +++++++++++++++++++ 2 files changed, 88 insertions(+) diff --git a/tests/queries/0_stateless/00087_math_functions.reference b/tests/queries/0_stateless/00087_math_functions.reference index 3e709ddfe52..e02aac11faf 100644 --- a/tests/queries/0_stateless/00087_math_functions.reference +++ b/tests/queries/0_stateless/00087_math_functions.reference @@ -72,3 +72,43 @@ 1 1 1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/00087_math_functions.sql b/tests/queries/0_stateless/00087_math_functions.sql index 8de8a774ff8..8b4b84ead9b 100644 --- a/tests/queries/0_stateless/00087_math_functions.sql +++ b/tests/queries/0_stateless/00087_math_functions.sql @@ -56,6 +56,12 @@ select log2(2) = 1; select log2(4) = 2; select sum(abs(log2(exp2(x)) - x) < 1.0e-9) / count() from system.one array join range(1000) as x; +select log1p(-1) = -inf; +select log1p(0) = 0; +select abs(log1p(exp(2) - 1) - 2) < 1e8; +select abs(log1p(exp(3) - 1) - 3) < 1e8; +select sum(abs(log1p(exp(x) - 1) - x) < 1e-8) / count() from system.one array join range(100) as x; + select sin(0) = 0; select sin(pi() / 4) = 1 / sqrt(2); select sin(pi() / 2) = 1; @@ -82,6 +88,48 @@ select acos(-1) = pi(); select atan(0) = 0; select atan(1) = pi() / 4; +select atan2(0, 1) = 0; +select atan2(0, 2) = 0; +select atan2(1, 0) = pi() / 2; +select atan2(1, 1) = pi() / 4; +select atan2(-1, -1) = -3 * pi() / 4); + +select hypot(0, 1) = 1; +select hypot(1, 0) = 1; +select hypot(1, 1) = sqrt(2); +select hypot(-1, 1) = sqrt(2); +select hypot(3, 4) = 5; + +select sinh(0) = 0; +select sinh(1) = -sinh(-1); +select abs(sinh(1) - 0.5 * (e() - exp(-1))) < 1e-9; +select abs(sinh(2) - 0.5 * (exp(2) - exp(-2))) < 1e-9; +select (sum(abs(sinh(x) - 0.5 * (exp(x) - exp(-x)))) < 1e-9) / count() from system.one array join range(1000) as x; + +select cosh(0) = 1; +select cosh(1) = cosh(-1); +select abs(cosh(1) - 0.5 * (e() + exp(-1))) < 1e-9; +select abs(pow(cosh(1), 2) - pow(sinh(1), 2) - 1) < 1e-9; +select (sum(abs(cosh(x) * cosh(x) - sinh(x) * sinh(x) - 1)) < 1e-9) / count() from system.one array join range(1000) as x; + +select asinh(0) = 0; +select asinh(1) = -asinh(-1); +select abs(asinh(1) - ln(1 + sqrt(2))) < 1e-9; +select (asinh(sinh(1)) - 1) < 1e-9 +select sum(abs(asinh(sinh(x)) - x) < 1e-9) / count() from system.one array join range(1000) as x; + +select acosh(1) = 0; +select acosh(2) = acosh(-2); +select abs(acosh(2) - ln(2 + sqrt(3))) < 1e-9; +select (acosh(cosh(2)) - 2) < 1e-9 +select sum(abs(acosh(cosh(x)) - x) < 1e-9) / count() from system.one array join range(1, 1001) as x; + +select atanh(0) = 0; +select atanh(0.5) = -asinh(-0.5); +select abs(atanh(0.9) - 0.5 * ln(19)) < 1e-9; +select (atanh(tanh(1)) - 1) < 1e-9 +select sum(abs(asinh(sinh(x)) - x) < 1e-9) / count() from system.one array join range(1000) as x; + select erf(0) = 0; select erf(-10) = -1; select erf(10) = 1; From 6e2e1da709f6e921e900cd86b5a05b465cc10d9b Mon Sep 17 00:00:00 2001 From: Konstantin Malanchev Date: Wed, 4 Nov 2020 11:02:25 +0100 Subject: [PATCH 0106/1088] ya.make --- src/Functions/ya.make | 192 ++++++++++++++++++++++-------------------- 1 file changed, 100 insertions(+), 92 deletions(-) diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 3aea31aa538..491c46414fa 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -34,27 +34,92 @@ PEERDIR( # "Arcadia" build is slightly deficient. It lacks many libraries that we need. SRCS( + CRC.cpp + FunctionFQDN.cpp + FunctionFactory.cpp + FunctionHelpers.cpp + FunctionJoinGet.cpp + FunctionsAES.cpp + FunctionsCoding.cpp + FunctionsConversion.cpp + FunctionsEmbeddedDictionaries.cpp + FunctionsExternalDictionaries.cpp + FunctionsExternalModels.cpp + FunctionsHashing.cpp + FunctionsJSON.cpp + FunctionsLogical.cpp + FunctionsRandom.cpp + FunctionsRound.cpp + FunctionsStringArray.cpp + FunctionsStringSimilarity.cpp + GatherUtils/concat.cpp + GatherUtils/createArraySink.cpp + GatherUtils/createArraySource.cpp + GatherUtils/createValueSource.cpp + GatherUtils/has_all.cpp + GatherUtils/has_any.cpp + GatherUtils/has_substr.cpp + GatherUtils/push.cpp + GatherUtils/resizeConstantSize.cpp + GatherUtils/resizeDynamicSize.cpp + GatherUtils/sliceDynamicOffsetBounded.cpp + GatherUtils/sliceDynamicOffsetUnbounded.cpp + GatherUtils/sliceFromLeftConstantOffsetBounded.cpp + GatherUtils/sliceFromLeftConstantOffsetUnbounded.cpp + GatherUtils/sliceFromRightConstantOffsetBounded.cpp + GatherUtils/sliceFromRightConstantOffsetUnbounded.cpp + GeoHash.cpp + IFunction.cpp + TargetSpecific.cpp + URL/URLHierarchy.cpp + URL/URLPathHierarchy.cpp + URL/basename.cpp + URL/cutFragment.cpp + URL/cutQueryString.cpp + URL/cutQueryStringAndFragment.cpp + URL/cutToFirstSignificantSubdomain.cpp + URL/cutURLParameter.cpp + URL/cutWWW.cpp + URL/decodeURLComponent.cpp + URL/domain.cpp + URL/domainWithoutWWW.cpp + URL/extractURLParameter.cpp + URL/extractURLParameterNames.cpp + URL/extractURLParameters.cpp + URL/firstSignificantSubdomain.cpp + URL/fragment.cpp + URL/netloc.cpp + URL/path.cpp + URL/pathFull.cpp + URL/port.cpp + URL/protocol.cpp + URL/queryString.cpp + URL/queryStringAndFragment.cpp + URL/registerFunctionsURL.cpp + URL/tldLookup.generated.cpp + URL/topLevelDomain.cpp abs.cpp acos.cpp + acosh.cpp addDays.cpp addHours.cpp addMinutes.cpp addMonths.cpp addQuarters.cpp - addressToLine.cpp - addressToSymbol.cpp addSeconds.cpp addWeeks.cpp addYears.cpp + addressToLine.cpp + addressToSymbol.cpp aes_decrypt_mysql.cpp aes_encrypt_mysql.cpp appendTrailingCharIfAbsent.cpp - array/arrayAll.cpp + array/array.cpp array/arrayAUC.cpp + array/arrayAll.cpp array/arrayCompact.cpp array/arrayConcat.cpp array/arrayCount.cpp - array/array.cpp array/arrayCumSum.cpp array/arrayCumSumNonNegative.cpp array/arrayDifference.cpp @@ -93,9 +158,9 @@ SRCS( array/countEqual.cpp array/emptyArray.cpp array/emptyArrayToSingle.cpp + array/has.cpp array/hasAll.cpp array/hasAny.cpp - array/has.cpp array/hasSubstr.cpp array/indexOf.cpp array/length.cpp @@ -104,8 +169,11 @@ SRCS( array/range.cpp array/registerFunctionsArray.cpp asin.cpp + asinh.cpp assumeNotNull.cpp atan.cpp + atan2.cpp + atanh.cpp bar.cpp base64Decode.cpp base64Encode.cpp @@ -113,7 +181,6 @@ SRCS( bitBoolMaskAnd.cpp bitBoolMaskOr.cpp bitCount.cpp - bitmaskToList.cpp bitNot.cpp bitOr.cpp bitRotateLeft.cpp @@ -121,11 +188,12 @@ SRCS( bitShiftLeft.cpp bitShiftRight.cpp bitSwapLastTwo.cpp + bitTest.cpp bitTestAll.cpp bitTestAny.cpp - bitTest.cpp bitWrapperFunc.cpp bitXor.cpp + bitmaskToList.cpp blockNumber.cpp blockSerializedSize.cpp blockSize.cpp @@ -136,8 +204,8 @@ SRCS( concat.cpp convertCharset.cpp cos.cpp + cosh.cpp countDigits.cpp - CRC.cpp currentDatabase.cpp currentUser.cpp dateDiff.cpp @@ -153,16 +221,16 @@ SRCS( encrypt.cpp endsWith.cpp equals.cpp - erfc.cpp erf.cpp + erfc.cpp errorCodeToName.cpp evalMLMethod.cpp + exp.cpp exp10.cpp exp2.cpp - exp.cpp + extract.cpp extractAllGroupsHorizontal.cpp extractAllGroupsVertical.cpp - extract.cpp extractGroups.cpp extractTimeZoneFromFunctionArguments.cpp filesystem.cpp @@ -176,47 +244,13 @@ SRCS( fromUnixTimestamp64Micro.cpp fromUnixTimestamp64Milli.cpp fromUnixTimestamp64Nano.cpp - FunctionFactory.cpp - FunctionFQDN.cpp - FunctionHelpers.cpp - FunctionJoinGet.cpp - FunctionsAES.cpp - FunctionsCoding.cpp - FunctionsConversion.cpp - FunctionsEmbeddedDictionaries.cpp - FunctionsExternalDictionaries.cpp - FunctionsExternalModels.cpp - FunctionsHashing.cpp - FunctionsJSON.cpp - FunctionsLogical.cpp - FunctionsRandom.cpp - FunctionsRound.cpp - FunctionsStringArray.cpp - FunctionsStringSimilarity.cpp fuzzBits.cpp - GatherUtils/concat.cpp - GatherUtils/createArraySink.cpp - GatherUtils/createArraySource.cpp - GatherUtils/createValueSource.cpp - GatherUtils/has_all.cpp - GatherUtils/has_any.cpp - GatherUtils/has_substr.cpp - GatherUtils/push.cpp - GatherUtils/resizeConstantSize.cpp - GatherUtils/resizeDynamicSize.cpp - GatherUtils/sliceDynamicOffsetBounded.cpp - GatherUtils/sliceDynamicOffsetUnbounded.cpp - GatherUtils/sliceFromLeftConstantOffsetBounded.cpp - GatherUtils/sliceFromLeftConstantOffsetUnbounded.cpp - GatherUtils/sliceFromRightConstantOffsetBounded.cpp - GatherUtils/sliceFromRightConstantOffsetUnbounded.cpp gcd.cpp generateUUIDv4.cpp - GeoHash.cpp + geoToH3.cpp geohashDecode.cpp geohashEncode.cpp geohashesInBox.cpp - geoToH3.cpp getMacro.cpp getScalar.cpp getSetting.cpp @@ -233,20 +267,20 @@ SRCS( h3HexAreaM2.cpp h3IndexesAreNeighbors.cpp h3IsValid.cpp - h3kRing.cpp h3ToChildren.cpp h3ToParent.cpp h3ToString.cpp + h3kRing.cpp hasColumnInTable.cpp hasThreadFuzzer.cpp - hasTokenCaseInsensitive.cpp hasToken.cpp + hasTokenCaseInsensitive.cpp hostName.cpp + hypot.cpp identity.cpp if.cpp ifNotFinite.cpp ifNull.cpp - IFunction.cpp ignore.cpp ilike.cpp in.cpp @@ -272,9 +306,10 @@ SRCS( lessOrEquals.cpp lgamma.cpp like.cpp - log10.cpp - log2.cpp log.cpp + log10.cpp + log1p.cpp + log2.cpp logTrace.cpp lowCardinalityIndices.cpp lowCardinalityKeys.cpp @@ -292,56 +327,56 @@ SRCS( multiMatchAllIndices.cpp multiMatchAny.cpp multiMatchAnyIndex.cpp - multiply.cpp + multiSearchAllPositions.cpp multiSearchAllPositionsCaseInsensitive.cpp multiSearchAllPositionsCaseInsensitiveUTF8.cpp - multiSearchAllPositions.cpp multiSearchAllPositionsUTF8.cpp + multiSearchAny.cpp multiSearchAnyCaseInsensitive.cpp multiSearchAnyCaseInsensitiveUTF8.cpp - multiSearchAny.cpp multiSearchAnyUTF8.cpp + multiSearchFirstIndex.cpp multiSearchFirstIndexCaseInsensitive.cpp multiSearchFirstIndexCaseInsensitiveUTF8.cpp - multiSearchFirstIndex.cpp multiSearchFirstIndexUTF8.cpp + multiSearchFirstPosition.cpp multiSearchFirstPositionCaseInsensitive.cpp multiSearchFirstPositionCaseInsensitiveUTF8.cpp - multiSearchFirstPosition.cpp multiSearchFirstPositionUTF8.cpp + multiply.cpp negate.cpp neighbor.cpp - normalizedQueryHash.cpp normalizeQuery.cpp + normalizedQueryHash.cpp notEmpty.cpp notEquals.cpp notILike.cpp notLike.cpp - now64.cpp now.cpp + now64.cpp nullIf.cpp pi.cpp plus.cpp pointInEllipses.cpp pointInPolygon.cpp + position.cpp positionCaseInsensitive.cpp positionCaseInsensitiveUTF8.cpp - position.cpp positionUTF8.cpp pow.cpp + rand.cpp rand64.cpp randConstant.cpp - rand.cpp randomFixedString.cpp randomPrintableASCII.cpp randomString.cpp randomStringUTF8.cpp regexpQuoteMeta.cpp + registerFunctions.cpp registerFunctionsArithmetic.cpp registerFunctionsComparison.cpp registerFunctionsConditional.cpp registerFunctionsConsistentHashing.cpp - registerFunctions.cpp registerFunctionsDateTime.cpp registerFunctionsFormatting.cpp registerFunctionsGeo.cpp @@ -379,6 +414,7 @@ SRCS( runningDifferenceStartingWithFirstValue.cpp sigmoid.cpp sin.cpp + sinh.cpp sleep.cpp sleepEachRow.cpp sqrt.cpp @@ -395,7 +431,6 @@ SRCS( subtractYears.cpp tan.cpp tanh.cpp - TargetSpecific.cpp tgamma.cpp throwIf.cpp tid.cpp @@ -404,7 +439,6 @@ SRCS( timezone.cpp toColumnTypeName.cpp toCustomWeek.cpp - today.cpp toDayOfMonth.cpp toDayOfWeek.cpp toDayOfYear.cpp @@ -431,8 +465,8 @@ SRCS( toStartOfFifteenMinutes.cpp toStartOfFiveMinute.cpp toStartOfHour.cpp - toStartOfInterval.cpp toStartOfISOYear.cpp + toStartOfInterval.cpp toStartOfMinute.cpp toStartOfMonth.cpp toStartOfQuarter.cpp @@ -446,10 +480,11 @@ SRCS( toUnixTimestamp64Milli.cpp toUnixTimestamp64Nano.cpp toValidUTF8.cpp - toYear.cpp toYYYYMM.cpp toYYYYMMDD.cpp toYYYYMMDDhhmmss.cpp + toYear.cpp + today.cpp transform.cpp trap.cpp trim.cpp @@ -459,33 +494,6 @@ SRCS( upper.cpp upperUTF8.cpp uptime.cpp - URL/basename.cpp - URL/cutFragment.cpp - URL/cutQueryStringAndFragment.cpp - URL/cutQueryString.cpp - URL/cutToFirstSignificantSubdomain.cpp - URL/cutURLParameter.cpp - URL/cutWWW.cpp - URL/decodeURLComponent.cpp - URL/domain.cpp - URL/domainWithoutWWW.cpp - URL/extractURLParameter.cpp - URL/extractURLParameterNames.cpp - URL/extractURLParameters.cpp - URL/firstSignificantSubdomain.cpp - URL/fragment.cpp - URL/netloc.cpp - URL/path.cpp - URL/pathFull.cpp - URL/port.cpp - URL/protocol.cpp - URL/queryStringAndFragment.cpp - URL/queryString.cpp - URL/registerFunctionsURL.cpp - URL/tldLookup.generated.cpp - URL/topLevelDomain.cpp - URL/URLHierarchy.cpp - URL/URLPathHierarchy.cpp version.cpp visibleWidth.cpp visitParamExtractBool.cpp From 43b2d20314b2b356780b48dc1c4d0e3382443072 Mon Sep 17 00:00:00 2001 From: myrrc Date: Wed, 4 Nov 2020 16:14:07 +0300 Subject: [PATCH 0107/1088] updated the decimal template magic --- .../AggregateFunctionAvg.cpp | 3 +- src/AggregateFunctions/AggregateFunctionAvg.h | 111 ++++++++++++------ .../AggregateFunctionAvgWeighted.cpp | 16 ++- .../AggregateFunctionAvgWeighted.h | 33 ++---- tests/performance/avg_weighted.xml | 7 +- 5 files changed, 110 insertions(+), 60 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAvg.cpp b/src/AggregateFunctions/AggregateFunctionAvg.cpp index 0a2f9e57fac..1511a9f54e5 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvg.cpp @@ -34,7 +34,8 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const AggregateFunctionPtr res; if (isDecimal(data_type)) - res.reset(createWithDecimalType(*data_type, argument_types)); + res.reset(createWithDecimalType( + *data_type, getDecimalScale(*data_type), argument_types)); else res.reset(createWithNumericType(*data_type, argument_types)); diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 29d9d90cc5d..6e1ba180236 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -1,25 +1,19 @@ #pragma once +#include #include #include #include #include #include #include +#include "Core/DecimalFunctions.h" namespace DB { - -template -struct RationalFraction -{ - Float64 numerator{0}; - Denominator denominator{0}; - - /// Allow division by zero as sometimes we need to return NaN. - Float64 NO_SANITIZE_UNDEFINED result() const { return numerator / denominator; } -}; +template +using DecimalOrVectorCol = std::conditional_t, ColumnDecimal, ColumnVector>; template constexpr bool DecimalOrExtendedInt = IsDecimalNumber @@ -29,29 +23,78 @@ template constexpr bool DecimalOrExtendedInt = || std::is_same_v; /** - * The discussion showed that the easiest (and simplest) way is to cast both the columns of numerator and denominator - * to Float64. Another way would be to write some template magic that figures out the appropriate numerator - * and denominator (and the resulting type) in favour of extended integral types (UInt128 e.g.) and Decimals ( - * which are a mess themselves). The second way is also a bit useless because now Decimals are not used in functions - * like avg. - * - * The ability to explicitly specify the denominator is made for avg (it uses the integral value as the denominator is - * simply the length of the supplied list). - * + * Helper class to encapsulate values conversion for avg and avgWeighted. + */ +template +struct AvgFraction +{ + Numerator numerator{0}; + Denominator denominator{0}; + + static constexpr bool any_is_decimal = IsDecimalNumber || IsDecimalNumber; + + /// Allow division by zero as sometimes we need to return NaN. + /// Invoked only is either Numerator or Denominator are Decimal. + std::enable_if_t NO_SANITIZE_UNDEFINED divide(UInt32 scale) const + { + if constexpr (IsDecimalNumber && IsDecimalNumber) + return DecimalUtils::convertTo(numerator / denominator, scale); + + /// Numerator is always casted to Float64 to divide correctly if the denominator is not Float64. + const Float64 num_converted = [scale](Numerator n) + { + if constexpr (IsDecimalNumber) + return DecimalUtils::convertTo(n, scale); + else + return static_cast(n); /// all other types, including extended integral. + } (numerator); + + const auto denom_converted = [scale](Denominator d) -> + std::conditional_t, Float64, Denominator> + { + if constexpr (IsDecimalNumber) + return DecimalUtils::convertTo(d, scale); + else if constexpr (DecimalOrExtendedInt) + /// no way to divide Float64 and extended integral type without an explicit cast. + return static_cast(d); + else + return d; /// can divide on float, no cast required. + } (denominator); + + return num_converted / denom_converted; + } + + std::enable_if_t NO_SANITIZE_UNDEFINED divide() const + { + if constexpr (DecimalOrExtendedInt) /// if extended int + return static_cast(numerator) / static_cast(denominator); + else + return static_cast(numerator) / denominator; + } +}; + + +/** * @tparam Derived When deriving from this class, use the child class name as in CRTP, e.g. * class Self : Agg. */ -template +template class AggregateFunctionAvgBase : public - IAggregateFunctionDataHelper, Derived> + IAggregateFunctionDataHelper, Derived> { public: - using Fraction = RationalFraction; + using Fraction = AvgFraction; using Base = IAggregateFunctionDataHelper; - explicit AggregateFunctionAvgBase(const DataTypes & argument_types_): Base(argument_types_, {}) {} + /// ctor for native types + explicit AggregateFunctionAvgBase(const DataTypes & argument_types_) + : Base(argument_types_, {}), scale(0) {} - DataTypePtr getReturnType() const override { return std::make_shared>(); } + /// ctor for Decimals + AggregateFunctionAvgBase(UInt32 scale_, const DataTypes & argument_types_) + : Base(argument_types_, {}), scale(scale_) {} + + DataTypePtr getReturnType() const final { return std::make_shared>(); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override { @@ -81,26 +124,24 @@ public: void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override { - static_cast &>(to).getData().push_back(this->data(place).result()); + if constexpr (IsDecimalNumber || IsDecimalNumber) + static_cast &>(to).getData().push_back(this->data(place).divide(scale)); + else + static_cast &>(to).getData().push_back(this->data(place).divide()); } +private: + UInt32 scale; }; template -class AggregateFunctionAvg final : public AggregateFunctionAvgBase> +class AggregateFunctionAvg final : public AggregateFunctionAvgBase> { public: - using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; + using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const final { - if constexpr(IsDecimalNumber) - this->data(place).numerator += columns[0]->getFloat64(row_num); - else if constexpr(DecimalOrExtendedInt) - this->data(place).numerator += static_cast( - static_cast &>(*columns[0]).getData()[row_num]); - else - this->data(place).numerator += static_cast &>(*columns[0]).getData()[row_num]; - + this->data(place).numerator += static_cast &>(*columns[0]).getData()[row_num]; ++this->data(place).denominator; } diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp index b9499dee460..1a505bb24f1 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp @@ -76,7 +76,21 @@ AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); AggregateFunctionPtr ptr; - ptr.reset(create(*data_type, *data_type_weight, argument_types)); + + const bool left_decimal = isDecimal(data_type); + const bool right_decimal = isDecimal(data_type_weight); + + if (left_decimal && right_decimal) + ptr.reset(create(*data_type, *data_type_weight, + getDecimalScale((sizeof(*data_type) > sizeof(*data_type_weight)) ? *data_type : *data_type_weight), + argument_types)); + else if (left_decimal) + ptr.reset(create(*data_type, *data_type_weight, getDecimalScale(*data_type), argument_types)); + else if (right_decimal) + ptr.reset(create(*data_type, *data_type_weight, getDecimalScale(*data_type_weight), argument_types)); + else + ptr.reset(create(*data_type, *data_type_weight, argument_types)); + return ptr; } } diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index 0c486232e52..b345d64bbfa 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -5,35 +5,28 @@ namespace DB { +template +using FieldType = std::conditional_t, + std::conditional_t, + Decimal256, Decimal128>, + NearestFieldType>; + template class AggregateFunctionAvgWeighted final : - public AggregateFunctionAvgBase> + public AggregateFunctionAvgBase, FieldType, AggregateFunctionAvgWeighted> { public: - using Base = AggregateFunctionAvgBase>; + using Base = AggregateFunctionAvgBase< + FieldType, FieldType, AggregateFunctionAvgWeighted>; using Base::Base; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - const Float64 value = [&columns, row_num] { - if constexpr(IsDecimalNumber) - return columns[0]->getFloat64(row_num); - else - return static_cast(static_cast&>(*columns[0]).getData()[row_num]); - }(); + const Value value = static_cast &>(*columns[0]).getData()[row_num]; + const Weight weight = static_cast &>(*columns[1]).getData()[row_num]; - using WeightRet = std::conditional_t, Float64, Weight>; - const WeightRet weight = [&columns, row_num]() -> WeightRet { - if constexpr(IsDecimalNumber) /// Unable to cast to double -> use the virtual method - return columns[1]->getFloat64(row_num); - else if constexpr(DecimalOrExtendedInt) /// Casting to double, otherwise += would be ambitious. - return static_cast(static_cast&>(*columns[1]).getData()[row_num]); - else - return static_cast&>(*columns[1]).getData()[row_num]; - }(); - - this->data(place).numerator += weight * value; - this->data(place).denominator += weight; + this->data(place).numerator += static_cast>(value) * weight; + this->data(place).denominator += static_cast>(weight); } String getName() const override { return "avgWeighted"; } diff --git a/tests/performance/avg_weighted.xml b/tests/performance/avg_weighted.xml index bd0ab2395ff..13bc7e58c44 100644 --- a/tests/performance/avg_weighted.xml +++ b/tests/performance/avg_weighted.xml @@ -4,7 +4,7 @@ CREATE TABLE perf_avg( num UInt64, - num_u Decimal256(75) DEFAULT toDecimal256(num / 100000, 75), + num_u Decimal256(75) DEFAULT toDecimal256(num / 400000, 75), num_f Float64 DEFAULT num ) ENGINE = MergeTree() ORDER BY tuple() @@ -13,11 +13,12 @@ INSERT INTO perf_avg(num) SELECT number / r FROM system.numbers - ARRAY JOIN range(1, 10000) AS r - LIMIT 5000000 + ARRAY JOIN range(1, 400000) AS r + LIMIT 200000000 SELECT avg(num) FROM perf_avg + SELECT avg(2 * num) FROM perf_avg SELECT avg(num_u) FROM perf_avg SELECT avg(num_f) FROM perf_avg SELECT avgWeighted(num_f, num) FROM perf_avg From dc51482e78e2f56784e4ddc343b9d8027269ce34 Mon Sep 17 00:00:00 2001 From: Grigory Buteyko Date: Wed, 4 Nov 2020 17:14:00 +0300 Subject: [PATCH 0108/1088] Improved numeric stability and stricter invariants in TDigest. Fixes bug when TDigest centroids array will grow beyond reasonable means and trigger exception inTDigest::deserialize during database Merge operations --- src/AggregateFunctions/QuantileTDigest.h | 59 ++++++++++++++++-------- 1 file changed, 40 insertions(+), 19 deletions(-) diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index 02d43ede66d..c09797573c4 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -40,6 +40,7 @@ class TDigest { using Value = Float32; using Count = Float32; + using BetterFloat = Float64; // For intermediate results and sum(Count). Must have better precision, than Count /** The centroid stores the weight of points around their mean value */ @@ -55,13 +56,6 @@ class TDigest , count(count_) {} - Centroid & operator+=(const Centroid & other) - { - count += other.count; - mean += other.count * (other.mean - mean) / count; - return *this; - } - bool operator<(const Centroid & other) const { return mean < other.mean; @@ -89,8 +83,8 @@ class TDigest using Centroids = PODArrayWithStackMemory; Centroids centroids; - Count count = 0; - UInt32 unmerged = 0; + BetterFloat count = 0; + size_t unmerged = 0; struct RadixSortTraits { @@ -111,6 +105,7 @@ class TDigest }; /** Adds a centroid `c` to the digest + * centroid must be valid, validity is checked in add(), deserialize() and is maintained by compress() */ void addCentroid(const Centroid & c) { @@ -138,47 +133,63 @@ public: auto l = centroids.begin(); auto r = std::next(l); - Count sum = 0; + const BetterFloat count_epsilon_4 = count * params.epsilon * 4; // Compiler is unable to do this optimization + BetterFloat sum = 0; + BetterFloat l_mean = l->mean; // We have high-precision temporaries for numeric stability + BetterFloat l_count = l->count; while (r != centroids.end()) { + if (l->mean == r->mean) // Perfect aggregation (fast). We compare l->mean, not l_mean, to avoid identical elements after compress + { + l_count += r->count; + l->count = l_count; + ++r; + continue; + } // we use quantile which gives us the smallest error /// The ratio of the part of the histogram to l, including the half l to the entire histogram. That is, what level quantile in position l. - Value ql = (sum + l->count * 0.5) / count; - Value err = ql * (1 - ql); + BetterFloat ql = (sum + l_count * 0.5) / count; + BetterFloat err = ql * (1 - ql); /// The ratio of the portion of the histogram to l, including l and half r to the entire histogram. That is, what level is the quantile in position r. - Value qr = (sum + l->count + r->count * 0.5) / count; - Value err2 = qr * (1 - qr); + BetterFloat qr = (sum + l_count + r->count * 0.5) / count; + BetterFloat err2 = qr * (1 - qr); if (err > err2) err = err2; - Value k = 4 * count * err * params.epsilon; + BetterFloat k = count_epsilon_4 * err; /** The ratio of the weight of the glued column pair to all values is not greater, * than epsilon multiply by a certain quadratic coefficient, which in the median is 1 (4 * 1/2 * 1/2), * and at the edges decreases and is approximately equal to the distance to the edge * 4. */ - if (l->count + r->count <= k) + if (l_count + r->count <= k) { // it is possible to merge left and right /// The left column "eats" the right. - *l += *r; + l_count += r->count; + l_mean += r->count * (r->mean - l_mean) / l_count; // Symmetric algo (M1*C1 + M2*C2)/(C1+C2) is numerically better, but slower + l->mean = l_mean; + l->count = l_count; } else { // not enough capacity, check the next pair - sum += l->count; + sum += l->count; // Not l_count, otherwise actual sum of elements will be different ++l; /// We skip all the values "eaten" earlier. if (l != r) *l = *r; + l_mean = l->mean; + l_count = l->count; } ++r; } + count = sum + l_count; // Update count, changed due inaccurancy /// At the end of the loop, all values to the right of l were "eaten". centroids.resize(l - centroids.begin() + 1); @@ -192,6 +203,8 @@ public: */ void add(T x, UInt64 cnt = 1) { + if (cnt == 0) + return; // Count 0 breaks compress() assumptions addCentroid(Centroid(Value(x), Count(cnt))); } @@ -220,8 +233,16 @@ public: buf.read(reinterpret_cast(centroids.data()), size * sizeof(centroids[0])); count = 0; - for (const auto & c : centroids) + for (size_t i = 0; i != centroids.size(); ++i) + { + Centroid & c = centroids[i]; + if (c.count <= 0 || std::isnan(c.count) || std::isnan(c.mean)) // invalid count breaks compress(), invalid mean breaks sort() + { + centroids.resize(i); // Exception safety, without this line we will end up with TDigest with invalid centroids + throw std::runtime_error("Invalid centroid " + std::to_string(c.count) + ":" + std::to_string(c.mean)); + } count += c.count; + } } Count getCount() From 832d37c424ab059f9ff87ba25c8d05d1a40fb4ee Mon Sep 17 00:00:00 2001 From: myrrc Date: Wed, 4 Nov 2020 17:23:04 +0300 Subject: [PATCH 0109/1088] added some corner cases, made Decimal convertTo const --- .../AggregateFunctionAvg.cpp | 2 +- src/AggregateFunctions/AggregateFunctionAvg.h | 29 +++++++++++-------- .../AggregateFunctionAvgWeighted.cpp | 8 ++--- .../AggregateFunctionAvgWeighted.h | 21 ++++++++++---- src/Core/Types.h | 2 +- 5 files changed, 38 insertions(+), 24 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAvg.cpp b/src/AggregateFunctions/AggregateFunctionAvg.cpp index 1511a9f54e5..9b1c3d6cef6 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvg.cpp @@ -35,7 +35,7 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const if (isDecimal(data_type)) res.reset(createWithDecimalType( - *data_type, getDecimalScale(*data_type), argument_types)); + *data_type, argument_types, getDecimalScale(*data_type))); else res.reset(createWithNumericType(*data_type, argument_types)); diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 6e1ba180236..2733327d236 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -31,18 +31,26 @@ struct AvgFraction Numerator numerator{0}; Denominator denominator{0}; - static constexpr bool any_is_decimal = IsDecimalNumber || IsDecimalNumber; - /// Allow division by zero as sometimes we need to return NaN. /// Invoked only is either Numerator or Denominator are Decimal. - std::enable_if_t NO_SANITIZE_UNDEFINED divide(UInt32 scale) const + Float64 NO_SANITIZE_UNDEFINED divideIfAnyDecimal(UInt32 scale) const { if constexpr (IsDecimalNumber && IsDecimalNumber) - return DecimalUtils::convertTo(numerator / denominator, scale); + { + if constexpr(std::is_same_v && std::is_same_v) + ///Special case as Decimal256 / Decimal128 = compile error (as Decimal128 is not parametrized by a wide + ///int), but an __int128 instead + return DecimalUtils::convertTo( + numerator / (denominator.template convertTo()), scale); + else + return DecimalUtils::convertTo(numerator / denominator, scale); + } /// Numerator is always casted to Float64 to divide correctly if the denominator is not Float64. const Float64 num_converted = [scale](Numerator n) { + (void) scale; + if constexpr (IsDecimalNumber) return DecimalUtils::convertTo(n, scale); else @@ -52,6 +60,8 @@ struct AvgFraction const auto denom_converted = [scale](Denominator d) -> std::conditional_t, Float64, Denominator> { + (void) scale; + if constexpr (IsDecimalNumber) return DecimalUtils::convertTo(d, scale); else if constexpr (DecimalOrExtendedInt) @@ -64,7 +74,7 @@ struct AvgFraction return num_converted / denom_converted; } - std::enable_if_t NO_SANITIZE_UNDEFINED divide() const + Float64 NO_SANITIZE_UNDEFINED divide() const { if constexpr (DecimalOrExtendedInt) /// if extended int return static_cast(numerator) / static_cast(denominator); @@ -86,12 +96,7 @@ public: using Fraction = AvgFraction; using Base = IAggregateFunctionDataHelper; - /// ctor for native types - explicit AggregateFunctionAvgBase(const DataTypes & argument_types_) - : Base(argument_types_, {}), scale(0) {} - - /// ctor for Decimals - AggregateFunctionAvgBase(UInt32 scale_, const DataTypes & argument_types_) + explicit AggregateFunctionAvgBase(const DataTypes & argument_types_, UInt32 scale_ = 0) : Base(argument_types_, {}), scale(scale_) {} DataTypePtr getReturnType() const final { return std::make_shared>(); } @@ -125,7 +130,7 @@ public: void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override { if constexpr (IsDecimalNumber || IsDecimalNumber) - static_cast &>(to).getData().push_back(this->data(place).divide(scale)); + static_cast &>(to).getData().push_back(this->data(place).divideIfAnyDecimal(scale)); else static_cast &>(to).getData().push_back(this->data(place).divide()); } diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp index 1a505bb24f1..3fc9ebb8865 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp @@ -82,12 +82,12 @@ AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name if (left_decimal && right_decimal) ptr.reset(create(*data_type, *data_type_weight, - getDecimalScale((sizeof(*data_type) > sizeof(*data_type_weight)) ? *data_type : *data_type_weight), - argument_types)); + argument_types, + getDecimalScale((sizeof(*data_type) > sizeof(*data_type_weight)) ? *data_type : *data_type_weight))); else if (left_decimal) - ptr.reset(create(*data_type, *data_type_weight, getDecimalScale(*data_type), argument_types)); + ptr.reset(create(*data_type, *data_type_weight, argument_types, getDecimalScale(*data_type))); else if (right_decimal) - ptr.reset(create(*data_type, *data_type_weight, getDecimalScale(*data_type_weight), argument_types)); + ptr.reset(create(*data_type, *data_type_weight, argument_types, getDecimalScale(*data_type_weight))); else ptr.reset(create(*data_type, *data_type_weight, argument_types)); diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index b345d64bbfa..b31fd1d0323 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -7,25 +7,34 @@ namespace DB { template using FieldType = std::conditional_t, - std::conditional_t, - Decimal256, Decimal128>, - NearestFieldType>; + std::conditional_t, Decimal256, Decimal128>, + std::conditional_t, + Float64, // no way to do UInt128 * UInt128, better cast to Float64 + NearestFieldType>>; + +template +using MaxFieldType = std::conditional_t<(sizeof(FieldType) > sizeof(FieldType)), + FieldType, FieldType>; template class AggregateFunctionAvgWeighted final : - public AggregateFunctionAvgBase, FieldType, AggregateFunctionAvgWeighted> + public AggregateFunctionAvgBase< + MaxFieldType, FieldType, AggregateFunctionAvgWeighted> { public: using Base = AggregateFunctionAvgBase< - FieldType, FieldType, AggregateFunctionAvgWeighted>; + MaxFieldType, FieldType, AggregateFunctionAvgWeighted>; using Base::Base; + using ValueT = MaxFieldType; + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { const Value value = static_cast &>(*columns[0]).getData()[row_num]; const Weight weight = static_cast &>(*columns[1]).getData()[row_num]; - this->data(place).numerator += static_cast>(value) * weight; + this->data(place).numerator += static_cast(value) * static_cast(weight); + this->data(place).denominator += static_cast>(weight); } diff --git a/src/Core/Types.h b/src/Core/Types.h index 3157598adc0..28ca7146aaf 100644 --- a/src/Core/Types.h +++ b/src/Core/Types.h @@ -145,7 +145,7 @@ struct Decimal operator T () const { return value; } template - U convertTo() + U convertTo() const { /// no IsDecimalNumber defined yet if constexpr (std::is_same_v> || From 591ebcef49bd7e2b87f61448eb9a9b67ef2e3c00 Mon Sep 17 00:00:00 2001 From: myrrc Date: Wed, 4 Nov 2020 18:23:29 +0300 Subject: [PATCH 0110/1088] fixing test results --- src/AggregateFunctions/AggregateFunctionAvg.h | 11 +- .../AggregateFunctionAvgWeighted.h | 12 +- ...01518_nullable_aggregate_states2.reference | 3652 ++++++++--------- 3 files changed, 1840 insertions(+), 1835 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 2733327d236..3a40b4f16ab 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -37,7 +37,7 @@ struct AvgFraction { if constexpr (IsDecimalNumber && IsDecimalNumber) { - if constexpr(std::is_same_v && std::is_same_v) + if constexpr (std::is_same_v && std::is_same_v) ///Special case as Decimal256 / Decimal128 = compile error (as Decimal128 is not parametrized by a wide ///int), but an __int128 instead return DecimalUtils::convertTo( @@ -139,10 +139,15 @@ private: }; template -class AggregateFunctionAvg final : public AggregateFunctionAvgBase> +using AvgFieldType = std::conditional_t, + std::conditional_t, Decimal256, Decimal128>, + NearestFieldType>; + +template +class AggregateFunctionAvg final : public AggregateFunctionAvgBase, UInt64, AggregateFunctionAvg> { public: - using AggregateFunctionAvgBase>::AggregateFunctionAvgBase; + using AggregateFunctionAvgBase, UInt64, AggregateFunctionAvg>::AggregateFunctionAvgBase; void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const final { diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h index b31fd1d0323..43312d391fe 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.h +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.h @@ -6,24 +6,24 @@ namespace DB { template -using FieldType = std::conditional_t, +using AvgWeightedFieldType = std::conditional_t, std::conditional_t, Decimal256, Decimal128>, std::conditional_t, Float64, // no way to do UInt128 * UInt128, better cast to Float64 NearestFieldType>>; template -using MaxFieldType = std::conditional_t<(sizeof(FieldType) > sizeof(FieldType)), - FieldType, FieldType>; +using MaxFieldType = std::conditional_t<(sizeof(AvgWeightedFieldType) > sizeof(AvgWeightedFieldType)), + AvgWeightedFieldType, AvgWeightedFieldType>; template class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase< - MaxFieldType, FieldType, AggregateFunctionAvgWeighted> + MaxFieldType, AvgWeightedFieldType, AggregateFunctionAvgWeighted> { public: using Base = AggregateFunctionAvgBase< - MaxFieldType, FieldType, AggregateFunctionAvgWeighted>; + MaxFieldType, AvgWeightedFieldType, AggregateFunctionAvgWeighted>; using Base::Base; using ValueT = MaxFieldType; @@ -35,7 +35,7 @@ public: this->data(place).numerator += static_cast(value) * static_cast(weight); - this->data(place).denominator += static_cast>(weight); + this->data(place).denominator += static_cast>(weight); } String getName() const override { return "avgWeighted"; } diff --git a/tests/queries/0_stateless/01518_nullable_aggregate_states2.reference b/tests/queries/0_stateless/01518_nullable_aggregate_states2.reference index e393fd798fb..7e353902742 100644 --- a/tests/queries/0_stateless/01518_nullable_aggregate_states2.reference +++ b/tests/queries/0_stateless/01518_nullable_aggregate_states2.reference @@ -7,1004 +7,1004 @@ -4 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N -5 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 0 102 0 99900 0 300 150 15150 0 300 150 15150 0.00000 300.00000 150 15150.00000 2020-01-01 2020-01-02 2020-01-01 00:00:00 2020-01-02 03:45:00 2020-01-01 00:00:00.000 2020-01-02 03:45:00.000 0 99900 49950 5044950 0 99900 49950 5044950 -32569 32366 4529.009900990099 457430 -127 124 -2.9504950495049505 -298 -1 102 1 9991 0.003 300.003 150.003 15150.3033 0.003 300.003 150.003 15150.30329 0.00300 300.00300 150.00300000000004 15150.30300 2020-01-01 2020-01-02 2020-01-01 00:00:01 2020-01-02 03:45:01 2020-01-01 00:00:01.000 2020-01-02 03:45:01.000 1 99901 49951 5045051 1 99901 49951 5045051 -32568 32367 4530.009900990099 457531 -126 125 -1.9504950495049505 -197 -10 102 10 99910 0.03003 300.03003 150.03003 15153.03303 0.03003 300.03003 150.03002 15153.03296 0.03003 300.03003 150.03002999999993 15153.03303 2020-01-01 2020-01-02 2020-01-01 00:00:10 2020-01-02 03:45:10 2020-01-01 00:00:10.000 2020-01-02 03:45:10.000 10 99910 49960 5045960 10 99910 49960 5045960 -32559 32376 4539.009900990099 458440 -128 127 -0.5544554455445545 -56 -100 101 100 99001 0.3003 297.3003 148.8003 14880.03003 0.3003 297.3003 148.80029 14880.02962 0.30030 297.30030 148.80030000000005 14880.03000 2020-01-01 2020-01-02 2020-01-01 00:01:40 2020-01-02 03:30:01 2020-01-01 00:01:40.000 2020-01-02 03:30:01.000 100 99001 49550.5 4955050 100 99001 49550.5 4955050 -32469 32466 4986.02 498602 -127 124 -0.86 -86 -101 101 10091 99002 0.3033 297.3033 148.8033 14880.33033 0.3033 297.3033 148.8033 14880.33035 0.30330 297.30330 148.80329999999987 14880.33000 2020-01-01 2020-01-02 2020-01-01 00:01:41 2020-01-02 03:30:02 2020-01-01 00:01:41.000 2020-01-02 03:30:02.000 101 99002 49551.5 4955150 101 99002 49551.5 4955150 -32468 32467 4987.02 498702 -126 125 0.14 14 -102 101 10092 99003 0.3063 297.3063 148.8063 14880.63063 0.3063 297.3063 148.8063 14880.6305 0.30630 297.30630 148.80630000000008 14880.63000 2020-01-01 2020-01-02 2020-01-01 00:01:42 2020-01-02 03:30:03 2020-01-01 00:01:42.000 2020-01-02 03:30:03.000 102 99003 49552.5 4955250 102 99003 49552.5 4955250 -32467 32468 4988.02 498802 -125 126 1.14 114 -103 101 10093 99004 0.3093 297.3093 148.8093 14880.93093 0.3093 297.3093 148.8093 14880.93085 0.30930 297.30930 148.80930000000006 14880.93000 2020-01-01 2020-01-02 2020-01-01 00:01:43 2020-01-02 03:30:04 2020-01-01 00:01:43.000 2020-01-02 03:30:04.000 103 99004 49553.5 4955350 103 99004 49553.5 4955350 -32466 32469 4989.02 498902 -124 127 2.14 214 -104 101 10094 99005 0.31231 297.31231 148.81231 14881.23123 0.31231 297.31232 148.81231 14881.23144 0.31231 297.31231 148.81230999999997 14881.23100 2020-01-01 2020-01-02 2020-01-01 00:01:44 2020-01-02 03:30:05 2020-01-01 00:01:44.000 2020-01-02 03:30:05.000 104 99005 49554.5 4955450 104 99005 49554.5 4955450 -32465 32470 4990.02 499002 -128 127 0.58 58 -105 101 10095 99006 0.31531 297.31531 148.81531 14881.53153 0.31531 297.3153 148.81531 14881.53174 0.31531 297.31531 148.81530999999998 14881.53100 2020-01-01 2020-01-02 2020-01-01 00:01:45 2020-01-02 03:30:06 2020-01-01 00:01:45.000 2020-01-02 03:30:06.000 105 99006 49555.5 4955550 105 99006 49555.5 4955550 -32464 32471 4991.02 499102 -128 123 -0.98 -98 -106 101 10096 99007 0.31831 297.31831 148.81831 14881.83183 0.31831 297.31833 148.81831 14881.83182 0.31831 297.31831 148.81831000000003 14881.83100 2020-01-01 2020-01-02 2020-01-01 00:01:46 2020-01-02 03:30:07 2020-01-01 00:01:46.000 2020-01-02 03:30:07.000 106 99007 49556.5 4955650 106 99007 49556.5 4955650 -32463 32472 4992.02 499202 -127 124 0.02 2 -107 101 10097 99008 0.32132 297.32132 148.82132 14882.13213 0.32132 297.32132 148.82131 14882.13197 0.32132 297.32132 148.8213199999999 14882.13200 2020-01-01 2020-01-02 2020-01-01 00:01:47 2020-01-02 03:30:08 2020-01-01 00:01:47.000 2020-01-02 03:30:08.000 107 99008 49557.5 4955750 107 99008 49557.5 4955750 -32462 32473 4993.02 499302 -126 125 1.02 102 -108 101 10098 99009 0.32432 297.32432 148.82432 14882.43243 0.32432 297.3243 148.82432 14882.43232 0.32432 297.32432 148.82431999999994 14882.43200 2020-01-01 2020-01-02 2020-01-01 00:01:48 2020-01-02 03:30:09 2020-01-01 00:01:48.000 2020-01-02 03:30:09.000 108 99009 49558.5 4955850 108 99009 49558.5 4955850 -32461 32474 4994.02 499402 -125 126 2.02 202 -109 101 10099 99010 0.32732 297.32732 148.82732 14882.73273 0.32732 297.32733 148.82732 14882.7329 0.32732 297.32732 148.8273200000001 14882.73200 2020-01-01 2020-01-02 2020-01-01 00:01:49 2020-01-02 03:30:10 2020-01-01 00:01:49.000 2020-01-02 03:30:10.000 109 99010 49559.5 4955950 109 99010 49559.5 4955950 -32460 32475 4995.02 499502 -124 127 3.02 302 +1 102 1 9991 0.003 300.003 150.003 15150.3033 0.003 300.003 150.003 15150.30329 0.00300 300.00300 150.003 15150.30300 2020-01-01 2020-01-02 2020-01-01 00:00:01 2020-01-02 03:45:01 2020-01-01 00:00:01.000 2020-01-02 03:45:01.000 1 99901 49951 5045051 1 99901 49951 5045051 -32568 32367 4530.009900990099 457531 -126 125 -1.9504950495049505 -197 +10 102 10 99910 0.03003 300.03003 150.03003 15153.03303 0.03003 300.03003 150.03002 15153.03296 0.03003 300.03003 150.03003 15153.03303 2020-01-01 2020-01-02 2020-01-01 00:00:10 2020-01-02 03:45:10 2020-01-01 00:00:10.000 2020-01-02 03:45:10.000 10 99910 49960 5045960 10 99910 49960 5045960 -32559 32376 4539.009900990099 458440 -128 127 -0.5544554455445545 -56 +100 101 100 99001 0.3003 297.3003 148.8003 14880.03003 0.3003 297.3003 148.80029 14880.02962 0.30030 297.30030 148.8003 14880.03000 2020-01-01 2020-01-02 2020-01-01 00:01:40 2020-01-02 03:30:01 2020-01-01 00:01:40.000 2020-01-02 03:30:01.000 100 99001 49550.5 4955050 100 99001 49550.5 4955050 -32469 32466 4986.02 498602 -127 124 -0.86 -86 +101 101 10091 99002 0.3033 297.3033 148.8033 14880.33033 0.3033 297.3033 148.8033 14880.33035 0.30330 297.30330 148.8033 14880.33000 2020-01-01 2020-01-02 2020-01-01 00:01:41 2020-01-02 03:30:02 2020-01-01 00:01:41.000 2020-01-02 03:30:02.000 101 99002 49551.5 4955150 101 99002 49551.5 4955150 -32468 32467 4987.02 498702 -126 125 0.14 14 +102 101 10092 99003 0.3063 297.3063 148.8063 14880.63063 0.3063 297.3063 148.8063 14880.6305 0.30630 297.30630 148.8063 14880.63000 2020-01-01 2020-01-02 2020-01-01 00:01:42 2020-01-02 03:30:03 2020-01-01 00:01:42.000 2020-01-02 03:30:03.000 102 99003 49552.5 4955250 102 99003 49552.5 4955250 -32467 32468 4988.02 498802 -125 126 1.14 114 +103 101 10093 99004 0.3093 297.3093 148.8093 14880.93093 0.3093 297.3093 148.8093 14880.93085 0.30930 297.30930 148.8093 14880.93000 2020-01-01 2020-01-02 2020-01-01 00:01:43 2020-01-02 03:30:04 2020-01-01 00:01:43.000 2020-01-02 03:30:04.000 103 99004 49553.5 4955350 103 99004 49553.5 4955350 -32466 32469 4989.02 498902 -124 127 2.14 214 +104 101 10094 99005 0.31231 297.31231 148.81231 14881.23123 0.31231 297.31232 148.81231 14881.23144 0.31231 297.31231 148.81231 14881.23100 2020-01-01 2020-01-02 2020-01-01 00:01:44 2020-01-02 03:30:05 2020-01-01 00:01:44.000 2020-01-02 03:30:05.000 104 99005 49554.5 4955450 104 99005 49554.5 4955450 -32465 32470 4990.02 499002 -128 127 0.58 58 +105 101 10095 99006 0.31531 297.31531 148.81531 14881.53153 0.31531 297.3153 148.81531 14881.53174 0.31531 297.31531 148.81531 14881.53100 2020-01-01 2020-01-02 2020-01-01 00:01:45 2020-01-02 03:30:06 2020-01-01 00:01:45.000 2020-01-02 03:30:06.000 105 99006 49555.5 4955550 105 99006 49555.5 4955550 -32464 32471 4991.02 499102 -128 123 -0.98 -98 +106 101 10096 99007 0.31831 297.31831 148.81831 14881.83183 0.31831 297.31833 148.81831 14881.83182 0.31831 297.31831 148.81831 14881.83100 2020-01-01 2020-01-02 2020-01-01 00:01:46 2020-01-02 03:30:07 2020-01-01 00:01:46.000 2020-01-02 03:30:07.000 106 99007 49556.5 4955650 106 99007 49556.5 4955650 -32463 32472 4992.02 499202 -127 124 0.02 2 +107 101 10097 99008 0.32132 297.32132 148.82132 14882.13213 0.32132 297.32132 148.82131 14882.13197 0.32132 297.32132 148.82132 14882.13200 2020-01-01 2020-01-02 2020-01-01 00:01:47 2020-01-02 03:30:08 2020-01-01 00:01:47.000 2020-01-02 03:30:08.000 107 99008 49557.5 4955750 107 99008 49557.5 4955750 -32462 32473 4993.02 499302 -126 125 1.02 102 +108 101 10098 99009 0.32432 297.32432 148.82432 14882.43243 0.32432 297.3243 148.82432 14882.43232 0.32432 297.32432 148.82432 14882.43200 2020-01-01 2020-01-02 2020-01-01 00:01:48 2020-01-02 03:30:09 2020-01-01 00:01:48.000 2020-01-02 03:30:09.000 108 99009 49558.5 4955850 108 99009 49558.5 4955850 -32461 32474 4994.02 499402 -125 126 2.02 202 +109 101 10099 99010 0.32732 297.32732 148.82732 14882.73273 0.32732 297.32733 148.82732 14882.7329 0.32732 297.32732 148.82732 14882.73200 2020-01-01 2020-01-02 2020-01-01 00:01:49 2020-01-02 03:30:10 2020-01-01 00:01:49.000 2020-01-02 03:30:10.000 109 99010 49559.5 4955950 109 99010 49559.5 4955950 -32460 32475 4995.02 499502 -124 127 3.02 302 11 102 10001 99911 0.03303 300.03303 150.03303 15153.33633 0.03303 300.03302 150.03303 15153.33627 0.03303 300.03303 150.03303 15153.33603 2020-01-01 2020-01-02 2020-01-01 00:00:11 2020-01-02 03:45:11 2020-01-01 00:00:11.000 2020-01-02 03:45:11.000 11 99911 49961 5046061 11 99911 49961 5046061 -32558 32377 4540.009900990099 458541 -128 123 -2.089108910891089 -211 110 101 10100 99011 0.33033 297.33033 148.83033 14883.03303 0.33033 297.33032 148.83033 14883.03321 0.33033 297.33033 148.83033 14883.03300 2020-01-01 2020-01-02 2020-01-01 00:01:50 2020-01-02 03:30:11 2020-01-01 00:01:50.000 2020-01-02 03:30:11.000 110 99011 49560.5 4956050 110 99011 49560.5 4956050 -32459 32476 4996.02 499602 -128 127 1.46 146 -111 101 10101 99012 0.33333 297.33333 148.83333 14883.33333 0.33333 297.33334 148.83333 14883.33329 0.33333 297.33333 148.83333 14883.33300 2020-01-01 2020-01-02 2020-01-01 00:01:51 2020-01-02 03:30:12 2020-01-01 00:01:51.000 2020-01-02 03:30:12.000 111 99012 49561.5 4956150 111 99012 49561.5 4956150 -32458 32477 4997.02 499702 -128 123 -0.1 -10 -112 101 10102 99013 0.33633 297.33633 148.83633 14883.63363 0.33633 297.33633 148.83633 14883.63348 0.33633 297.33633 148.83633000000006 14883.63300 2020-01-01 2020-01-02 2020-01-01 00:01:52 2020-01-02 03:30:13 2020-01-01 00:01:52.000 2020-01-02 03:30:13.000 112 99013 49562.5 4956250 112 99013 49562.5 4956250 -32457 32478 4998.02 499802 -127 124 0.9 90 +111 101 10101 99012 0.33333 297.33333 148.83333 14883.33333 0.33333 297.33334 148.83333 14883.33329 0.33333 297.33333 148.83333000000002 14883.33300 2020-01-01 2020-01-02 2020-01-01 00:01:51 2020-01-02 03:30:12 2020-01-01 00:01:51.000 2020-01-02 03:30:12.000 111 99012 49561.5 4956150 111 99012 49561.5 4956150 -32458 32477 4997.02 499702 -128 123 -0.1 -10 +112 101 10102 99013 0.33633 297.33633 148.83633 14883.63363 0.33633 297.33633 148.83633 14883.63348 0.33633 297.33633 148.83633 14883.63300 2020-01-01 2020-01-02 2020-01-01 00:01:52 2020-01-02 03:30:13 2020-01-01 00:01:52.000 2020-01-02 03:30:13.000 112 99013 49562.5 4956250 112 99013 49562.5 4956250 -32457 32478 4998.02 499802 -127 124 0.9 90 113 101 10103 99014 0.33933 297.33933 148.83933 14883.93393 0.33933 297.33932 148.83933 14883.9338 0.33933 297.33933 148.83933000000002 14883.93300 2020-01-01 2020-01-02 2020-01-01 00:01:53 2020-01-02 03:30:14 2020-01-01 00:01:53.000 2020-01-02 03:30:14.000 113 99014 49563.5 4956350 113 99014 49563.5 4956350 -32456 32479 4999.02 499902 -126 125 1.9 190 -114 101 10104 99015 0.34234 297.34234 148.84234 14884.23423 0.34234 297.34235 148.84234 14884.23437 0.34234 297.34234 148.84233999999998 14884.23400 2020-01-01 2020-01-02 2020-01-01 00:01:54 2020-01-02 03:30:15 2020-01-01 00:01:54.000 2020-01-02 03:30:15.000 114 99015 49564.5 4956450 114 99015 49564.5 4956450 -32455 32480 5000.02 500002 -125 126 2.9 290 -115 101 10105 99016 0.34534 297.34534 148.84534 14884.53453 0.34534 297.34534 148.84534 14884.53468 0.34534 297.34534 148.84533999999994 14884.53400 2020-01-01 2020-01-02 2020-01-01 00:01:55 2020-01-02 03:30:16 2020-01-01 00:01:55.000 2020-01-02 03:30:16.000 115 99016 49565.5 4956550 115 99016 49565.5 4956550 -32454 32481 5001.02 500102 -124 127 3.9 390 -116 101 10106 99017 0.34834 297.34834 148.84834 14884.83483 0.34834 297.34836 148.84834 14884.83476 0.34834 297.34834 148.84834000000015 14884.83400 2020-01-01 2020-01-02 2020-01-01 00:01:56 2020-01-02 03:30:17 2020-01-01 00:01:56.000 2020-01-02 03:30:17.000 116 99017 49566.5 4956650 116 99017 49566.5 4956650 -32453 32482 5002.02 500202 -128 127 2.34 234 -117 101 10107 99018 0.35135 297.35135 148.85135 14885.13513 0.35135 297.35135 148.85134 14885.13495 0.35135 297.35135 148.85135000000005 14885.13500 2020-01-01 2020-01-02 2020-01-01 00:01:57 2020-01-02 03:30:18 2020-01-01 00:01:57.000 2020-01-02 03:30:18.000 117 99018 49567.5 4956750 117 99018 49567.5 4956750 -32452 32483 5003.02 500302 -128 123 0.78 78 -118 101 10108 99019 0.35435 297.35435 148.85435 14885.43543 0.35435 297.35434 148.85435 14885.43526 0.35435 297.35435 148.8543499999999 14885.43500 2020-01-01 2020-01-02 2020-01-01 00:01:58 2020-01-02 03:30:19 2020-01-01 00:01:58.000 2020-01-02 03:30:19.000 118 99019 49568.5 4956850 118 99019 49568.5 4956850 -32451 32484 5004.02 500402 -127 124 1.78 178 -119 101 10109 99020 0.35735 297.35735 148.85735 14885.73573 0.35735 297.35736 148.85736 14885.736 0.35735 297.35735 148.8573500000001 14885.73500 2020-01-01 2020-01-02 2020-01-01 00:01:59 2020-01-02 03:30:20 2020-01-01 00:01:59.000 2020-01-02 03:30:20.000 119 99020 49569.5 4956950 119 99020 49569.5 4956950 -32450 32485 5005.02 500502 -126 125 2.78 278 +114 101 10104 99015 0.34234 297.34234 148.84234 14884.23423 0.34234 297.34235 148.84234 14884.23437 0.34234 297.34234 148.84234 14884.23400 2020-01-01 2020-01-02 2020-01-01 00:01:54 2020-01-02 03:30:15 2020-01-01 00:01:54.000 2020-01-02 03:30:15.000 114 99015 49564.5 4956450 114 99015 49564.5 4956450 -32455 32480 5000.02 500002 -125 126 2.9 290 +115 101 10105 99016 0.34534 297.34534 148.84534 14884.53453 0.34534 297.34534 148.84534 14884.53468 0.34534 297.34534 148.84534 14884.53400 2020-01-01 2020-01-02 2020-01-01 00:01:55 2020-01-02 03:30:16 2020-01-01 00:01:55.000 2020-01-02 03:30:16.000 115 99016 49565.5 4956550 115 99016 49565.5 4956550 -32454 32481 5001.02 500102 -124 127 3.9 390 +116 101 10106 99017 0.34834 297.34834 148.84834 14884.83483 0.34834 297.34836 148.84834 14884.83476 0.34834 297.34834 148.84834 14884.83400 2020-01-01 2020-01-02 2020-01-01 00:01:56 2020-01-02 03:30:17 2020-01-01 00:01:56.000 2020-01-02 03:30:17.000 116 99017 49566.5 4956650 116 99017 49566.5 4956650 -32453 32482 5002.02 500202 -128 127 2.34 234 +117 101 10107 99018 0.35135 297.35135 148.85135 14885.13513 0.35135 297.35135 148.85134 14885.13495 0.35135 297.35135 148.85135 14885.13500 2020-01-01 2020-01-02 2020-01-01 00:01:57 2020-01-02 03:30:18 2020-01-01 00:01:57.000 2020-01-02 03:30:18.000 117 99018 49567.5 4956750 117 99018 49567.5 4956750 -32452 32483 5003.02 500302 -128 123 0.78 78 +118 101 10108 99019 0.35435 297.35435 148.85435 14885.43543 0.35435 297.35434 148.85435 14885.43526 0.35435 297.35435 148.85434999999998 14885.43500 2020-01-01 2020-01-02 2020-01-01 00:01:58 2020-01-02 03:30:19 2020-01-01 00:01:58.000 2020-01-02 03:30:19.000 118 99019 49568.5 4956850 118 99019 49568.5 4956850 -32451 32484 5004.02 500402 -127 124 1.78 178 +119 101 10109 99020 0.35735 297.35735 148.85735 14885.73573 0.35735 297.35736 148.85736 14885.736 0.35735 297.35735 148.85735 14885.73500 2020-01-01 2020-01-02 2020-01-01 00:01:59 2020-01-02 03:30:20 2020-01-01 00:01:59.000 2020-01-02 03:30:20.000 119 99020 49569.5 4956950 119 99020 49569.5 4956950 -32450 32485 5005.02 500502 -126 125 2.78 278 12 102 10002 99912 0.03603 300.03603 150.03603 15153.63963 0.03603 300.03604 150.03603 15153.6399 0.03603 300.03603 150.03603 15153.63903 2020-01-01 2020-01-02 2020-01-01 00:00:12 2020-01-02 03:45:12 2020-01-01 00:00:12.000 2020-01-02 03:45:12.000 12 99912 49962 5046162 12 99912 49962 5046162 -32557 32378 4541.009900990099 458642 -127 124 -1.0891089108910892 -110 -120 101 10110 99021 0.36036 297.36036 148.86036 14886.03603 0.36036 297.36035 148.86036 14886.03615 0.36036 297.36036 148.86036000000004 14886.03600 2020-01-01 2020-01-02 2020-01-01 00:02:00 2020-01-02 03:30:21 2020-01-01 00:02:00.000 2020-01-02 03:30:21.000 120 99021 49570.5 4957050 120 99021 49570.5 4957050 -32449 32486 5006.02 500602 -125 126 3.78 378 -121 101 10111 99022 0.36336 297.36336 148.86336 14886.33633 0.36336 297.36337 148.86336 14886.33627 0.36336 297.36336 148.86335999999994 14886.33600 2020-01-01 2020-01-02 2020-01-01 00:02:01 2020-01-02 03:30:22 2020-01-01 00:02:01.000 2020-01-02 03:30:22.000 121 99022 49571.5 4957150 121 99022 49571.5 4957150 -32448 32487 5007.02 500702 -124 127 4.78 478 -122 101 10112 99023 0.36636 297.36636 148.86636 14886.63663 0.36636 297.36636 148.86636 14886.63642 0.36636 297.36636 148.86636 14886.63600 2020-01-01 2020-01-02 2020-01-01 00:02:02 2020-01-02 03:30:23 2020-01-01 00:02:02.000 2020-01-02 03:30:23.000 122 99023 49572.5 4957250 122 99023 49572.5 4957250 -32447 32488 5008.02 500802 -128 127 3.22 322 -123 101 10113 99024 0.36936 297.36936 148.86936 14886.93693 0.36936 297.36935 148.86936 14886.93673 0.36936 297.36936 148.86936000000003 14886.93600 2020-01-01 2020-01-02 2020-01-01 00:02:03 2020-01-02 03:30:24 2020-01-01 00:02:03.000 2020-01-02 03:30:24.000 123 99024 49573.5 4957350 123 99024 49573.5 4957350 -32446 32489 5009.02 500902 -128 127 1.66 166 -124 101 10114 99025 0.37237 297.37237 148.87237 14887.23723 0.37237 297.37238 148.87237 14887.23746 0.37237 297.37237 148.87236999999993 14887.23700 2020-01-01 2020-01-02 2020-01-01 00:02:04 2020-01-02 03:30:25 2020-01-01 00:02:04.000 2020-01-02 03:30:25.000 124 99025 49574.5 4957450 124 99025 49574.5 4957450 -32445 32490 5010.02 501002 -128 124 0.1 10 -125 101 10115 99026 0.37537 297.37537 148.87537 14887.53753 0.37537 297.37537 148.87537 14887.53762 0.37537 297.37537 148.87536999999992 14887.53700 2020-01-01 2020-01-02 2020-01-01 00:02:05 2020-01-02 03:30:26 2020-01-01 00:02:05.000 2020-01-02 03:30:26.000 125 99026 49575.5 4957550 125 99026 49575.5 4957550 -32444 32491 5011.02 501102 -127 125 1.1 110 -126 101 10116 99027 0.37837 297.37837 148.87837 14887.83783 0.37837 297.3784 148.87837 14887.83774 0.37837 297.37837 148.87837000000007 14887.83700 2020-01-01 2020-01-02 2020-01-01 00:02:06 2020-01-02 03:30:27 2020-01-01 00:02:06.000 2020-01-02 03:30:27.000 126 99027 49576.5 4957650 126 99027 49576.5 4957650 -32443 32492 5012.02 501202 -126 126 2.1 210 +120 101 10110 99021 0.36036 297.36036 148.86036 14886.03603 0.36036 297.36035 148.86036 14886.03615 0.36036 297.36036 148.86036000000001 14886.03600 2020-01-01 2020-01-02 2020-01-01 00:02:00 2020-01-02 03:30:21 2020-01-01 00:02:00.000 2020-01-02 03:30:21.000 120 99021 49570.5 4957050 120 99021 49570.5 4957050 -32449 32486 5006.02 500602 -125 126 3.78 378 +121 101 10111 99022 0.36336 297.36336 148.86336 14886.33633 0.36336 297.36337 148.86336 14886.33627 0.36336 297.36336 148.86336 14886.33600 2020-01-01 2020-01-02 2020-01-01 00:02:01 2020-01-02 03:30:22 2020-01-01 00:02:01.000 2020-01-02 03:30:22.000 121 99022 49571.5 4957150 121 99022 49571.5 4957150 -32448 32487 5007.02 500702 -124 127 4.78 478 +122 101 10112 99023 0.36636 297.36636 148.86636 14886.63663 0.36636 297.36636 148.86636 14886.63642 0.36636 297.36636 148.86636000000001 14886.63600 2020-01-01 2020-01-02 2020-01-01 00:02:02 2020-01-02 03:30:23 2020-01-01 00:02:02.000 2020-01-02 03:30:23.000 122 99023 49572.5 4957250 122 99023 49572.5 4957250 -32447 32488 5008.02 500802 -128 127 3.22 322 +123 101 10113 99024 0.36936 297.36936 148.86936 14886.93693 0.36936 297.36935 148.86936 14886.93673 0.36936 297.36936 148.86936 14886.93600 2020-01-01 2020-01-02 2020-01-01 00:02:03 2020-01-02 03:30:24 2020-01-01 00:02:03.000 2020-01-02 03:30:24.000 123 99024 49573.5 4957350 123 99024 49573.5 4957350 -32446 32489 5009.02 500902 -128 127 1.66 166 +124 101 10114 99025 0.37237 297.37237 148.87237 14887.23723 0.37237 297.37238 148.87237 14887.23746 0.37237 297.37237 148.87237 14887.23700 2020-01-01 2020-01-02 2020-01-01 00:02:04 2020-01-02 03:30:25 2020-01-01 00:02:04.000 2020-01-02 03:30:25.000 124 99025 49574.5 4957450 124 99025 49574.5 4957450 -32445 32490 5010.02 501002 -128 124 0.1 10 +125 101 10115 99026 0.37537 297.37537 148.87537 14887.53753 0.37537 297.37537 148.87537 14887.53762 0.37537 297.37537 148.87537 14887.53700 2020-01-01 2020-01-02 2020-01-01 00:02:05 2020-01-02 03:30:26 2020-01-01 00:02:05.000 2020-01-02 03:30:26.000 125 99026 49575.5 4957550 125 99026 49575.5 4957550 -32444 32491 5011.02 501102 -127 125 1.1 110 +126 101 10116 99027 0.37837 297.37837 148.87837 14887.83783 0.37837 297.3784 148.87837 14887.83774 0.37837 297.37837 148.87837 14887.83700 2020-01-01 2020-01-02 2020-01-01 00:02:06 2020-01-02 03:30:27 2020-01-01 00:02:06.000 2020-01-02 03:30:27.000 126 99027 49576.5 4957650 126 99027 49576.5 4957650 -32443 32492 5012.02 501202 -126 126 2.1 210 127 101 10117 99028 0.38138 297.38138 148.88138 14888.13813 0.38138 297.38138 148.88137 14888.13789 0.38138 297.38138 148.88138 14888.13800 2020-01-01 2020-01-02 2020-01-01 00:02:07 2020-01-02 03:30:28 2020-01-01 00:02:07.000 2020-01-02 03:30:28.000 127 99028 49577.5 4957750 127 99028 49577.5 4957750 -32442 32493 5013.02 501302 -125 127 3.1 310 -128 101 10118 99029 0.38438 297.38438 148.88438 14888.43843 0.38438 297.3844 148.88438 14888.43862 0.38438 297.38438 148.88438000000002 14888.43800 2020-01-01 2020-01-02 2020-01-01 00:02:08 2020-01-02 03:30:29 2020-01-01 00:02:08.000 2020-01-02 03:30:29.000 128 99029 49578.5 4957850 128 99029 49578.5 4957850 -32441 32494 5014.02 501402 -128 127 1.54 154 -129 101 10119 99030 0.38738 297.38738 148.88738 14888.73873 0.38738 297.3874 148.88738 14888.73894 0.38738 297.38738 148.88738000000006 14888.73800 2020-01-01 2020-01-02 2020-01-01 00:02:09 2020-01-02 03:30:30 2020-01-01 00:02:09.000 2020-01-02 03:30:30.000 129 99030 49579.5 4957950 129 99030 49579.5 4957950 -32440 32495 5015.02 501502 -128 127 -0.02 -2 -13 102 10003 99913 0.03903 300.03903 150.03903 15153.94294 0.03903 300.03903 150.03903 15153.94255 0.03903 300.03903 150.03902999999994 15153.94203 2020-01-01 2020-01-02 2020-01-01 00:00:13 2020-01-02 03:45:13 2020-01-01 00:00:13.000 2020-01-02 03:45:13.000 13 99913 49963 5046263 13 99913 49963 5046263 -32556 32379 4542.009900990099 458743 -126 125 -0.0891089108910891 -9 +128 101 10118 99029 0.38438 297.38438 148.88438 14888.43843 0.38438 297.3844 148.88438 14888.43862 0.38438 297.38438 148.88438 14888.43800 2020-01-01 2020-01-02 2020-01-01 00:02:08 2020-01-02 03:30:29 2020-01-01 00:02:08.000 2020-01-02 03:30:29.000 128 99029 49578.5 4957850 128 99029 49578.5 4957850 -32441 32494 5014.02 501402 -128 127 1.54 154 +129 101 10119 99030 0.38738 297.38738 148.88738 14888.73873 0.38738 297.3874 148.88738 14888.73894 0.38738 297.38738 148.88738 14888.73800 2020-01-01 2020-01-02 2020-01-01 00:02:09 2020-01-02 03:30:30 2020-01-01 00:02:09.000 2020-01-02 03:30:30.000 129 99030 49579.5 4957950 129 99030 49579.5 4957950 -32440 32495 5015.02 501502 -128 127 -0.02 -2 +13 102 10003 99913 0.03903 300.03903 150.03903 15153.94294 0.03903 300.03903 150.03903 15153.94255 0.03903 300.03903 150.03903 15153.94203 2020-01-01 2020-01-02 2020-01-01 00:00:13 2020-01-02 03:45:13 2020-01-01 00:00:13.000 2020-01-02 03:45:13.000 13 99913 49963 5046263 13 99913 49963 5046263 -32556 32379 4542.009900990099 458743 -126 125 -0.0891089108910891 -9 130 101 10120 99031 0.39039 297.39039 148.89039 14889.03903 0.39039 297.39038 148.89039 14889.03909 0.39039 297.39039 148.89039 14889.03900 2020-01-01 2020-01-02 2020-01-01 00:02:10 2020-01-02 03:30:31 2020-01-01 00:02:10.000 2020-01-02 03:30:31.000 130 99031 49580.5 4958050 130 99031 49580.5 4958050 -32439 32496 5016.02 501602 -128 123 -1.58 -158 -131 101 10121 99032 0.39339 297.39339 148.89339 14889.33933 0.39339 297.3934 148.89339 14889.33921 0.39339 297.39339 148.89338999999998 14889.33900 2020-01-01 2020-01-02 2020-01-01 00:02:11 2020-01-02 03:30:32 2020-01-01 00:02:11.000 2020-01-02 03:30:32.000 131 99032 49581.5 4958150 131 99032 49581.5 4958150 -32438 32497 5017.02 501702 -127 124 -0.58 -58 -132 101 10122 99033 0.39639 297.39639 148.89639 14889.63963 0.39639 297.3964 148.89639 14889.63936 0.39639 297.39639 148.8963899999999 14889.63900 2020-01-01 2020-01-02 2020-01-01 00:02:12 2020-01-02 03:30:33 2020-01-01 00:02:12.000 2020-01-02 03:30:33.000 132 99033 49582.5 4958250 132 99033 49582.5 4958250 -32437 32498 5018.02 501802 -126 125 0.42 42 -133 101 10123 99034 0.39939 297.39939 148.89939 14889.93993 0.39939 297.3994 148.8994 14889.94009 0.39939 297.39939 148.89938999999998 14889.93900 2020-01-01 2020-01-02 2020-01-01 00:02:13 2020-01-02 03:30:34 2020-01-01 00:02:13.000 2020-01-02 03:30:34.000 133 99034 49583.5 4958350 133 99034 49583.5 4958350 -32436 32499 5019.02 501902 -125 126 1.42 142 -134 101 10124 99035 0.4024 297.4024 148.9024 14890.24024 0.4024 297.4024 148.9024 14890.24041 0.40240 297.40240 148.90240000000009 14890.24000 2020-01-01 2020-01-02 2020-01-01 00:02:14 2020-01-02 03:30:35 2020-01-01 00:02:14.000 2020-01-02 03:30:35.000 134 99035 49584.5 4958450 134 99035 49584.5 4958450 -32435 32500 5020.02 502002 -124 127 2.42 242 -135 101 10125 99036 0.4054 297.4054 148.9054 14890.54054 0.4054 297.4054 148.9054 14890.54059 0.40540 297.40540 148.90539999999993 14890.54000 2020-01-01 2020-01-02 2020-01-01 00:02:15 2020-01-02 03:30:36 2020-01-01 00:02:15.000 2020-01-02 03:30:36.000 135 99036 49585.5 4958550 135 99036 49585.5 4958550 -32434 32501 5021.02 502102 -128 127 0.86 86 -136 101 10126 99037 0.4084 297.4084 148.9084 14890.84084 0.4084 297.40842 148.9084 14890.84068 0.40840 297.40840 148.90840000000003 14890.84000 2020-01-01 2020-01-02 2020-01-01 00:02:16 2020-01-02 03:30:37 2020-01-01 00:02:16.000 2020-01-02 03:30:37.000 136 99037 49586.5 4958650 136 99037 49586.5 4958650 -32433 32502 5022.02 502202 -128 123 -0.7 -70 -137 101 10127 99038 0.41141 297.41141 148.91141 14891.14114 0.41141 297.4114 148.9114 14891.14099 0.41141 297.41141 148.91141000000005 14891.14100 2020-01-01 2020-01-02 2020-01-01 00:02:17 2020-01-02 03:30:38 2020-01-01 00:02:17.000 2020-01-02 03:30:38.000 137 99038 49587.5 4958750 137 99038 49587.5 4958750 -32432 32503 5023.02 502302 -127 124 0.3 30 -138 101 10128 99039 0.41441 297.41441 148.91441 14891.44144 0.41441 297.41443 148.91441 14891.44157 0.41441 297.41441 148.91440999999995 14891.44100 2020-01-01 2020-01-02 2020-01-01 00:02:18 2020-01-02 03:30:39 2020-01-01 00:02:18.000 2020-01-02 03:30:39.000 138 99039 49588.5 4958850 138 99039 49588.5 4958850 -32431 32504 5024.02 502402 -126 125 1.3 130 +131 101 10121 99032 0.39339 297.39339 148.89339 14889.33933 0.39339 297.3934 148.89339 14889.33921 0.39339 297.39339 148.89339 14889.33900 2020-01-01 2020-01-02 2020-01-01 00:02:11 2020-01-02 03:30:32 2020-01-01 00:02:11.000 2020-01-02 03:30:32.000 131 99032 49581.5 4958150 131 99032 49581.5 4958150 -32438 32497 5017.02 501702 -127 124 -0.58 -58 +132 101 10122 99033 0.39639 297.39639 148.89639 14889.63963 0.39639 297.3964 148.89639 14889.63936 0.39639 297.39639 148.89639 14889.63900 2020-01-01 2020-01-02 2020-01-01 00:02:12 2020-01-02 03:30:33 2020-01-01 00:02:12.000 2020-01-02 03:30:33.000 132 99033 49582.5 4958250 132 99033 49582.5 4958250 -32437 32498 5018.02 501802 -126 125 0.42 42 +133 101 10123 99034 0.39939 297.39939 148.89939 14889.93993 0.39939 297.3994 148.8994 14889.94009 0.39939 297.39939 148.89939 14889.93900 2020-01-01 2020-01-02 2020-01-01 00:02:13 2020-01-02 03:30:34 2020-01-01 00:02:13.000 2020-01-02 03:30:34.000 133 99034 49583.5 4958350 133 99034 49583.5 4958350 -32436 32499 5019.02 501902 -125 126 1.42 142 +134 101 10124 99035 0.4024 297.4024 148.9024 14890.24024 0.4024 297.4024 148.9024 14890.24041 0.40240 297.40240 148.9024 14890.24000 2020-01-01 2020-01-02 2020-01-01 00:02:14 2020-01-02 03:30:35 2020-01-01 00:02:14.000 2020-01-02 03:30:35.000 134 99035 49584.5 4958450 134 99035 49584.5 4958450 -32435 32500 5020.02 502002 -124 127 2.42 242 +135 101 10125 99036 0.4054 297.4054 148.9054 14890.54054 0.4054 297.4054 148.9054 14890.54059 0.40540 297.40540 148.90540000000001 14890.54000 2020-01-01 2020-01-02 2020-01-01 00:02:15 2020-01-02 03:30:36 2020-01-01 00:02:15.000 2020-01-02 03:30:36.000 135 99036 49585.5 4958550 135 99036 49585.5 4958550 -32434 32501 5021.02 502102 -128 127 0.86 86 +136 101 10126 99037 0.4084 297.4084 148.9084 14890.84084 0.4084 297.40842 148.9084 14890.84068 0.40840 297.40840 148.9084 14890.84000 2020-01-01 2020-01-02 2020-01-01 00:02:16 2020-01-02 03:30:37 2020-01-01 00:02:16.000 2020-01-02 03:30:37.000 136 99037 49586.5 4958650 136 99037 49586.5 4958650 -32433 32502 5022.02 502202 -128 123 -0.7 -70 +137 101 10127 99038 0.41141 297.41141 148.91141 14891.14114 0.41141 297.4114 148.9114 14891.14099 0.41141 297.41141 148.91141 14891.14100 2020-01-01 2020-01-02 2020-01-01 00:02:17 2020-01-02 03:30:38 2020-01-01 00:02:17.000 2020-01-02 03:30:38.000 137 99038 49587.5 4958750 137 99038 49587.5 4958750 -32432 32503 5023.02 502302 -127 124 0.3 30 +138 101 10128 99039 0.41441 297.41441 148.91441 14891.44144 0.41441 297.41443 148.91441 14891.44157 0.41441 297.41441 148.91441 14891.44100 2020-01-01 2020-01-02 2020-01-01 00:02:18 2020-01-02 03:30:39 2020-01-01 00:02:18.000 2020-01-02 03:30:39.000 138 99039 49588.5 4958850 138 99039 49588.5 4958850 -32431 32504 5024.02 502402 -126 125 1.3 130 139 101 10129 99040 0.41741 297.41741 148.91741 14891.74174 0.41741 297.41742 148.91741 14891.74188 0.41741 297.41741 148.91741 14891.74100 2020-01-01 2020-01-02 2020-01-01 00:02:19 2020-01-02 03:30:40 2020-01-01 00:02:19.000 2020-01-02 03:30:40.000 139 99040 49589.5 4958950 139 99040 49589.5 4958950 -32430 32505 5025.02 502502 -125 126 2.3 230 -14 102 10004 99914 0.04204 300.04204 150.04204 15154.24624 0.04204 300.04205 150.04204 15154.2463 0.04204 300.04204 150.04204000000004 15154.24604 2020-01-01 2020-01-02 2020-01-01 00:00:14 2020-01-02 03:45:14 2020-01-01 00:00:14.000 2020-01-02 03:45:14.000 14 99914 49964 5046364 14 99914 49964 5046364 -32555 32380 4543.009900990099 458844 -125 126 0.9108910891089109 92 -140 101 10130 99041 0.42042 297.42042 148.92042 14892.04204 0.42042 297.4204 148.92042 14892.04206 0.42042 297.42042 148.9204200000001 14892.04200 2020-01-01 2020-01-02 2020-01-01 00:02:20 2020-01-02 03:30:41 2020-01-01 00:02:20.000 2020-01-02 03:30:41.000 140 99041 49590.5 4959050 140 99041 49590.5 4959050 -32429 32506 5026.02 502602 -124 127 3.3 330 -141 101 10131 99042 0.42342 297.42342 148.92342 14892.34234 0.42342 297.42343 148.92342 14892.34215 0.42342 297.42342 148.92341999999994 14892.34200 2020-01-01 2020-01-02 2020-01-01 00:02:21 2020-01-02 03:30:42 2020-01-01 00:02:21.000 2020-01-02 03:30:42.000 141 99042 49591.5 4959150 141 99042 49591.5 4959150 -32428 32507 5027.02 502702 -128 127 1.74 174 -142 101 10132 99043 0.42642 297.42642 148.92642 14892.64264 0.42642 297.42642 148.92642 14892.64246 0.42642 297.42642 148.9264199999999 14892.64200 2020-01-01 2020-01-02 2020-01-01 00:02:22 2020-01-02 03:30:43 2020-01-01 00:02:22.000 2020-01-02 03:30:43.000 142 99043 49592.5 4959250 142 99043 49592.5 4959250 -32427 32508 5028.02 502802 -128 123 0.18 18 -143 101 10133 99044 0.42942 297.42942 148.92942 14892.94294 0.42942 297.42944 148.92943 14892.94304 0.42942 297.42942 148.9294200000001 14892.94200 2020-01-01 2020-01-02 2020-01-01 00:02:23 2020-01-02 03:30:44 2020-01-01 00:02:23.000 2020-01-02 03:30:44.000 143 99044 49593.5 4959350 143 99044 49593.5 4959350 -32426 32509 5029.02 502902 -127 124 1.18 118 +14 102 10004 99914 0.04204 300.04204 150.04204 15154.24624 0.04204 300.04205 150.04204 15154.2463 0.04204 300.04204 150.04204 15154.24604 2020-01-01 2020-01-02 2020-01-01 00:00:14 2020-01-02 03:45:14 2020-01-01 00:00:14.000 2020-01-02 03:45:14.000 14 99914 49964 5046364 14 99914 49964 5046364 -32555 32380 4543.009900990099 458844 -125 126 0.9108910891089109 92 +140 101 10130 99041 0.42042 297.42042 148.92042 14892.04204 0.42042 297.4204 148.92042 14892.04206 0.42042 297.42042 148.92042 14892.04200 2020-01-01 2020-01-02 2020-01-01 00:02:20 2020-01-02 03:30:41 2020-01-01 00:02:20.000 2020-01-02 03:30:41.000 140 99041 49590.5 4959050 140 99041 49590.5 4959050 -32429 32506 5026.02 502602 -124 127 3.3 330 +141 101 10131 99042 0.42342 297.42342 148.92342 14892.34234 0.42342 297.42343 148.92342 14892.34215 0.42342 297.42342 148.92342 14892.34200 2020-01-01 2020-01-02 2020-01-01 00:02:21 2020-01-02 03:30:42 2020-01-01 00:02:21.000 2020-01-02 03:30:42.000 141 99042 49591.5 4959150 141 99042 49591.5 4959150 -32428 32507 5027.02 502702 -128 127 1.74 174 +142 101 10132 99043 0.42642 297.42642 148.92642 14892.64264 0.42642 297.42642 148.92642 14892.64246 0.42642 297.42642 148.92642 14892.64200 2020-01-01 2020-01-02 2020-01-01 00:02:22 2020-01-02 03:30:43 2020-01-01 00:02:22.000 2020-01-02 03:30:43.000 142 99043 49592.5 4959250 142 99043 49592.5 4959250 -32427 32508 5028.02 502802 -128 123 0.18 18 +143 101 10133 99044 0.42942 297.42942 148.92942 14892.94294 0.42942 297.42944 148.92943 14892.94304 0.42942 297.42942 148.92942 14892.94200 2020-01-01 2020-01-02 2020-01-01 00:02:23 2020-01-02 03:30:44 2020-01-01 00:02:23.000 2020-01-02 03:30:44.000 143 99044 49593.5 4959350 143 99044 49593.5 4959350 -32426 32509 5029.02 502902 -127 124 1.18 118 144 101 10134 99045 0.43243 297.43243 148.93243 14893.24324 0.43243 297.43243 148.93243 14893.24338 0.43243 297.43243 148.93243 14893.24300 2020-01-01 2020-01-02 2020-01-01 00:02:24 2020-01-02 03:30:45 2020-01-01 00:02:24.000 2020-01-02 03:30:45.000 144 99045 49594.5 4959450 144 99045 49594.5 4959450 -32425 32510 5030.02 503002 -126 125 2.18 218 -145 101 10135 99046 0.43543 297.43543 148.93543 14893.54354 0.43543 297.43542 148.93543 14893.54354 0.43543 297.43543 148.93542999999983 14893.54300 2020-01-01 2020-01-02 2020-01-01 00:02:25 2020-01-02 03:30:46 2020-01-01 00:02:25.000 2020-01-02 03:30:46.000 145 99046 49595.5 4959550 145 99046 49595.5 4959550 -32424 32511 5031.02 503102 -125 126 3.18 318 -146 101 10136 99047 0.43843 297.43843 148.93843 14893.84384 0.43843 297.43845 148.93844 14893.84427 0.43843 297.43843 148.93843000000007 14893.84300 2020-01-01 2020-01-02 2020-01-01 00:02:26 2020-01-02 03:30:47 2020-01-01 00:02:26.000 2020-01-02 03:30:47.000 146 99047 49596.5 4959650 146 99047 49596.5 4959650 -32423 32512 5032.02 503202 -124 127 4.18 418 +145 101 10135 99046 0.43543 297.43543 148.93543 14893.54354 0.43543 297.43542 148.93543 14893.54354 0.43543 297.43543 148.93543 14893.54300 2020-01-01 2020-01-02 2020-01-01 00:02:25 2020-01-02 03:30:46 2020-01-01 00:02:25.000 2020-01-02 03:30:46.000 145 99046 49595.5 4959550 145 99046 49595.5 4959550 -32424 32511 5031.02 503102 -125 126 3.18 318 +146 101 10136 99047 0.43843 297.43843 148.93843 14893.84384 0.43843 297.43845 148.93844 14893.84427 0.43843 297.43843 148.93843 14893.84300 2020-01-01 2020-01-02 2020-01-01 00:02:26 2020-01-02 03:30:47 2020-01-01 00:02:26.000 2020-01-02 03:30:47.000 146 99047 49596.5 4959650 146 99047 49596.5 4959650 -32423 32512 5032.02 503202 -124 127 4.18 418 147 101 10137 99048 0.44144 297.44144 148.94144 14894.14414 0.44144 297.44144 148.94143 14894.14392 0.44144 297.44144 148.94144 14894.14400 2020-01-01 2020-01-02 2020-01-01 00:02:27 2020-01-02 03:30:48 2020-01-01 00:02:27.000 2020-01-02 03:30:48.000 147 99048 49597.5 4959750 147 99048 49597.5 4959750 -32422 32513 5033.02 503302 -128 127 2.62 262 148 101 10138 99049 0.44444 297.44444 148.94444 14894.44444 0.44444 297.44446 148.94444 14894.4445 0.44444 297.44444 148.94444 14894.44400 2020-01-01 2020-01-02 2020-01-01 00:02:28 2020-01-02 03:30:49 2020-01-01 00:02:28.000 2020-01-02 03:30:49.000 148 99049 49598.5 4959850 148 99049 49598.5 4959850 -32421 32514 5034.02 503402 -128 127 1.06 106 -149 101 10139 99050 0.44744 297.44744 148.94744 14894.74474 0.44744 297.44745 148.94744 14894.74485 0.44744 297.44744 148.94743999999994 14894.74400 2020-01-01 2020-01-02 2020-01-01 00:02:29 2020-01-02 03:30:50 2020-01-01 00:02:29.000 2020-01-02 03:30:50.000 149 99050 49599.5 4959950 149 99050 49599.5 4959950 -32420 32515 5035.02 503502 -128 124 -0.5 -50 -15 102 10005 99915 0.04504 300.04504 150.04504 15154.54954 0.04504 300.04504 150.04504 15154.54945 0.04504 300.04504 150.04504000000006 15154.54904 2020-01-01 2020-01-02 2020-01-01 00:00:15 2020-01-02 03:45:15 2020-01-01 00:00:15.000 2020-01-02 03:45:15.000 15 99915 49965 5046465 15 99915 49965 5046465 -32554 32381 4544.009900990099 458945 -124 127 1.9108910891089108 193 -150 101 10140 99051 0.45045 297.45045 148.95045 14895.04504 0.45045 297.45044 148.95045 14895.04501 0.45045 297.45045 148.95045000000005 14895.04500 2020-01-01 2020-01-02 2020-01-01 00:02:30 2020-01-02 03:30:51 2020-01-01 00:02:30.000 2020-01-02 03:30:51.000 150 99051 49600.5 4960050 150 99051 49600.5 4960050 -32419 32516 5036.02 503602 -127 125 0.5 50 -151 101 10141 99052 0.45345 297.45345 148.95345 14895.34534 0.45345 297.45346 148.95345 14895.34574 0.45345 297.45345 148.9534500000001 14895.34500 2020-01-01 2020-01-02 2020-01-01 00:02:31 2020-01-02 03:30:52 2020-01-01 00:02:31.000 2020-01-02 03:30:52.000 151 99052 49601.5 4960150 151 99052 49601.5 4960150 -32418 32517 5037.02 503702 -126 126 1.5 150 -152 101 10142 99053 0.45645 297.45645 148.95645 14895.64564 0.45645 297.45645 148.95645 14895.6454 0.45645 297.45645 148.9564499999999 14895.64500 2020-01-01 2020-01-02 2020-01-01 00:02:32 2020-01-02 03:30:53 2020-01-01 00:02:32.000 2020-01-02 03:30:53.000 152 99053 49602.5 4960250 152 99053 49602.5 4960250 -32417 32518 5038.02 503802 -125 127 2.5 250 -153 101 10143 99054 0.45945 297.45945 148.95945 14895.94594 0.45945 297.45947 148.95946 14895.94601 0.45945 297.45945 148.95945000000006 14895.94500 2020-01-01 2020-01-02 2020-01-01 00:02:33 2020-01-02 03:30:54 2020-01-01 00:02:33.000 2020-01-02 03:30:54.000 153 99054 49603.5 4960350 153 99054 49603.5 4960350 -32416 32519 5039.02 503902 -128 127 0.94 94 -154 101 10144 99055 0.46246 297.46246 148.96246 14896.24624 0.46246 297.46246 148.96246 14896.24633 0.46246 297.46246 148.96246000000008 14896.24600 2020-01-01 2020-01-02 2020-01-01 00:02:34 2020-01-02 03:30:55 2020-01-01 00:02:34.000 2020-01-02 03:30:55.000 154 99055 49604.5 4960450 154 99055 49604.5 4960450 -32415 32520 5040.02 504002 -128 127 -0.62 -62 -155 101 10145 99056 0.46546 297.46546 148.96546 14896.54654 0.46546 297.46545 148.96546 14896.54647 0.46546 297.46546 148.96545999999998 14896.54600 2020-01-01 2020-01-02 2020-01-01 00:02:35 2020-01-02 03:30:56 2020-01-01 00:02:35.000 2020-01-02 03:30:56.000 155 99056 49605.5 4960550 155 99056 49605.5 4960550 -32414 32521 5041.02 504102 -128 123 -2.18 -218 -156 101 10146 99057 0.46846 297.46846 148.96846 14896.84684 0.46846 297.46848 148.96847 14896.84721 0.46846 297.46846 148.96846000000002 14896.84600 2020-01-01 2020-01-02 2020-01-01 00:02:36 2020-01-02 03:30:57 2020-01-01 00:02:36.000 2020-01-02 03:30:57.000 156 99057 49606.5 4960650 156 99057 49606.5 4960650 -32413 32522 5042.02 504202 -127 124 -1.18 -118 -157 101 10147 99058 0.47147 297.47147 148.97147 14897.14714 0.47147 297.47147 148.97146 14897.14687 0.47147 297.47147 148.97147000000012 14897.14700 2020-01-01 2020-01-02 2020-01-01 00:02:37 2020-01-02 03:30:58 2020-01-01 00:02:37.000 2020-01-02 03:30:58.000 157 99058 49607.5 4960750 157 99058 49607.5 4960750 -32412 32523 5043.02 504302 -126 125 -0.18 -18 -158 101 10148 99059 0.47447 297.47447 148.97447 14897.44744 0.47447 297.4745 148.97447 14897.44748 0.47447 297.47447 148.97446999999994 14897.44700 2020-01-01 2020-01-02 2020-01-01 00:02:38 2020-01-02 03:30:59 2020-01-01 00:02:38.000 2020-01-02 03:30:59.000 158 99059 49608.5 4960850 158 99059 49608.5 4960850 -32411 32524 5044.02 504402 -125 126 0.82 82 -159 101 10149 99060 0.47747 297.47747 148.97747 14897.74774 0.47747 297.47748 148.97747 14897.74779 0.47747 297.47747 148.97746999999993 14897.74700 2020-01-01 2020-01-02 2020-01-01 00:02:39 2020-01-02 03:31:00 2020-01-01 00:02:39.000 2020-01-02 03:31:00.000 159 99060 49609.5 4960950 159 99060 49609.5 4960950 -32410 32525 5045.02 504502 -124 127 1.82 182 -16 102 10006 99916 0.04804 300.04804 150.04804 15154.85285 0.04804 300.04803 150.04804 15154.85279 0.04804 300.04804 150.0480399999999 15154.85204 2020-01-01 2020-01-02 2020-01-01 00:00:16 2020-01-02 03:45:16 2020-01-01 00:00:16.000 2020-01-02 03:45:16.000 16 99916 49966 5046566 16 99916 49966 5046566 -32553 32382 4545.009900990099 459046 -128 127 0.37623762376237624 38 -160 101 10150 99061 0.48048 297.48048 148.98048 14898.04804 0.48048 297.48047 148.98048 14898.0481 0.48048 297.48048 148.98048000000009 14898.04800 2020-01-01 2020-01-02 2020-01-01 00:02:40 2020-01-02 03:31:01 2020-01-01 00:02:40.000 2020-01-02 03:31:01.000 160 99061 49610.5 4961050 160 99061 49610.5 4961050 -32409 32526 5046.02 504602 -128 127 0.26 26 -161 101 10151 99062 0.48348 297.48348 148.98348 14898.34834 0.48348 297.4835 148.98348 14898.34868 0.48348 297.48348 148.98348000000004 14898.34800 2020-01-01 2020-01-02 2020-01-01 00:02:41 2020-01-02 03:31:02 2020-01-01 00:02:41.000 2020-01-02 03:31:02.000 161 99062 49611.5 4961150 161 99062 49611.5 4961150 -32408 32527 5047.02 504702 -128 123 -1.3 -130 -162 101 10152 99063 0.48648 297.48648 148.98648 14898.64864 0.48648 297.48648 148.98648 14898.64837 0.48648 297.48648 148.98647999999986 14898.64800 2020-01-01 2020-01-02 2020-01-01 00:02:42 2020-01-02 03:31:03 2020-01-01 00:02:42.000 2020-01-02 03:31:03.000 162 99063 49612.5 4961250 162 99063 49612.5 4961250 -32407 32528 5048.02 504802 -127 124 -0.3 -30 -163 101 10153 99064 0.48948 297.48948 148.98948 14898.94894 0.48948 297.4895 148.98948 14898.94895 0.48948 297.48948 148.98948000000007 14898.94800 2020-01-01 2020-01-02 2020-01-01 00:02:43 2020-01-02 03:31:04 2020-01-01 00:02:43.000 2020-01-02 03:31:04.000 163 99064 49613.5 4961350 163 99064 49613.5 4961350 -32406 32529 5049.02 504902 -126 125 0.7 70 +149 101 10139 99050 0.44744 297.44744 148.94744 14894.74474 0.44744 297.44745 148.94744 14894.74485 0.44744 297.44744 148.94744 14894.74400 2020-01-01 2020-01-02 2020-01-01 00:02:29 2020-01-02 03:30:50 2020-01-01 00:02:29.000 2020-01-02 03:30:50.000 149 99050 49599.5 4959950 149 99050 49599.5 4959950 -32420 32515 5035.02 503502 -128 124 -0.5 -50 +15 102 10005 99915 0.04504 300.04504 150.04504 15154.54954 0.04504 300.04504 150.04504 15154.54945 0.04504 300.04504 150.04504 15154.54904 2020-01-01 2020-01-02 2020-01-01 00:00:15 2020-01-02 03:45:15 2020-01-01 00:00:15.000 2020-01-02 03:45:15.000 15 99915 49965 5046465 15 99915 49965 5046465 -32554 32381 4544.009900990099 458945 -124 127 1.9108910891089108 193 +150 101 10140 99051 0.45045 297.45045 148.95045 14895.04504 0.45045 297.45044 148.95045 14895.04501 0.45045 297.45045 148.95045 14895.04500 2020-01-01 2020-01-02 2020-01-01 00:02:30 2020-01-02 03:30:51 2020-01-01 00:02:30.000 2020-01-02 03:30:51.000 150 99051 49600.5 4960050 150 99051 49600.5 4960050 -32419 32516 5036.02 503602 -127 125 0.5 50 +151 101 10141 99052 0.45345 297.45345 148.95345 14895.34534 0.45345 297.45346 148.95345 14895.34574 0.45345 297.45345 148.95345 14895.34500 2020-01-01 2020-01-02 2020-01-01 00:02:31 2020-01-02 03:30:52 2020-01-01 00:02:31.000 2020-01-02 03:30:52.000 151 99052 49601.5 4960150 151 99052 49601.5 4960150 -32418 32517 5037.02 503702 -126 126 1.5 150 +152 101 10142 99053 0.45645 297.45645 148.95645 14895.64564 0.45645 297.45645 148.95645 14895.6454 0.45645 297.45645 148.95645000000002 14895.64500 2020-01-01 2020-01-02 2020-01-01 00:02:32 2020-01-02 03:30:53 2020-01-01 00:02:32.000 2020-01-02 03:30:53.000 152 99053 49602.5 4960250 152 99053 49602.5 4960250 -32417 32518 5038.02 503802 -125 127 2.5 250 +153 101 10143 99054 0.45945 297.45945 148.95945 14895.94594 0.45945 297.45947 148.95946 14895.94601 0.45945 297.45945 148.95945 14895.94500 2020-01-01 2020-01-02 2020-01-01 00:02:33 2020-01-02 03:30:54 2020-01-01 00:02:33.000 2020-01-02 03:30:54.000 153 99054 49603.5 4960350 153 99054 49603.5 4960350 -32416 32519 5039.02 503902 -128 127 0.94 94 +154 101 10144 99055 0.46246 297.46246 148.96246 14896.24624 0.46246 297.46246 148.96246 14896.24633 0.46246 297.46246 148.96246 14896.24600 2020-01-01 2020-01-02 2020-01-01 00:02:34 2020-01-02 03:30:55 2020-01-01 00:02:34.000 2020-01-02 03:30:55.000 154 99055 49604.5 4960450 154 99055 49604.5 4960450 -32415 32520 5040.02 504002 -128 127 -0.62 -62 +155 101 10145 99056 0.46546 297.46546 148.96546 14896.54654 0.46546 297.46545 148.96546 14896.54647 0.46546 297.46546 148.96546 14896.54600 2020-01-01 2020-01-02 2020-01-01 00:02:35 2020-01-02 03:30:56 2020-01-01 00:02:35.000 2020-01-02 03:30:56.000 155 99056 49605.5 4960550 155 99056 49605.5 4960550 -32414 32521 5041.02 504102 -128 123 -2.18 -218 +156 101 10146 99057 0.46846 297.46846 148.96846 14896.84684 0.46846 297.46848 148.96847 14896.84721 0.46846 297.46846 148.96846 14896.84600 2020-01-01 2020-01-02 2020-01-01 00:02:36 2020-01-02 03:30:57 2020-01-01 00:02:36.000 2020-01-02 03:30:57.000 156 99057 49606.5 4960650 156 99057 49606.5 4960650 -32413 32522 5042.02 504202 -127 124 -1.18 -118 +157 101 10147 99058 0.47147 297.47147 148.97147 14897.14714 0.47147 297.47147 148.97146 14897.14687 0.47147 297.47147 148.97147 14897.14700 2020-01-01 2020-01-02 2020-01-01 00:02:37 2020-01-02 03:30:58 2020-01-01 00:02:37.000 2020-01-02 03:30:58.000 157 99058 49607.5 4960750 157 99058 49607.5 4960750 -32412 32523 5043.02 504302 -126 125 -0.18 -18 +158 101 10148 99059 0.47447 297.47447 148.97447 14897.44744 0.47447 297.4745 148.97447 14897.44748 0.47447 297.47447 148.97447 14897.44700 2020-01-01 2020-01-02 2020-01-01 00:02:38 2020-01-02 03:30:59 2020-01-01 00:02:38.000 2020-01-02 03:30:59.000 158 99059 49608.5 4960850 158 99059 49608.5 4960850 -32411 32524 5044.02 504402 -125 126 0.82 82 +159 101 10149 99060 0.47747 297.47747 148.97747 14897.74774 0.47747 297.47748 148.97747 14897.74779 0.47747 297.47747 148.97746999999998 14897.74700 2020-01-01 2020-01-02 2020-01-01 00:02:39 2020-01-02 03:31:00 2020-01-01 00:02:39.000 2020-01-02 03:31:00.000 159 99060 49609.5 4960950 159 99060 49609.5 4960950 -32410 32525 5045.02 504502 -124 127 1.82 182 +16 102 10006 99916 0.04804 300.04804 150.04804 15154.85285 0.04804 300.04803 150.04804 15154.85279 0.04804 300.04804 150.04804 15154.85204 2020-01-01 2020-01-02 2020-01-01 00:00:16 2020-01-02 03:45:16 2020-01-01 00:00:16.000 2020-01-02 03:45:16.000 16 99916 49966 5046566 16 99916 49966 5046566 -32553 32382 4545.009900990099 459046 -128 127 0.37623762376237624 38 +160 101 10150 99061 0.48048 297.48048 148.98048 14898.04804 0.48048 297.48047 148.98048 14898.0481 0.48048 297.48048 148.98048 14898.04800 2020-01-01 2020-01-02 2020-01-01 00:02:40 2020-01-02 03:31:01 2020-01-01 00:02:40.000 2020-01-02 03:31:01.000 160 99061 49610.5 4961050 160 99061 49610.5 4961050 -32409 32526 5046.02 504602 -128 127 0.26 26 +161 101 10151 99062 0.48348 297.48348 148.98348 14898.34834 0.48348 297.4835 148.98348 14898.34868 0.48348 297.48348 148.98348 14898.34800 2020-01-01 2020-01-02 2020-01-01 00:02:41 2020-01-02 03:31:02 2020-01-01 00:02:41.000 2020-01-02 03:31:02.000 161 99062 49611.5 4961150 161 99062 49611.5 4961150 -32408 32527 5047.02 504702 -128 123 -1.3 -130 +162 101 10152 99063 0.48648 297.48648 148.98648 14898.64864 0.48648 297.48648 148.98648 14898.64837 0.48648 297.48648 148.98648 14898.64800 2020-01-01 2020-01-02 2020-01-01 00:02:42 2020-01-02 03:31:03 2020-01-01 00:02:42.000 2020-01-02 03:31:03.000 162 99063 49612.5 4961250 162 99063 49612.5 4961250 -32407 32528 5048.02 504802 -127 124 -0.3 -30 +163 101 10153 99064 0.48948 297.48948 148.98948 14898.94894 0.48948 297.4895 148.98948 14898.94895 0.48948 297.48948 148.98948000000001 14898.94800 2020-01-01 2020-01-02 2020-01-01 00:02:43 2020-01-02 03:31:04 2020-01-01 00:02:43.000 2020-01-02 03:31:04.000 163 99064 49613.5 4961350 163 99064 49613.5 4961350 -32406 32529 5049.02 504902 -126 125 0.7 70 164 101 10154 99065 0.49249 297.49249 148.99249 14899.24924 0.49249 297.4925 148.99249 14899.24926 0.49249 297.49249 148.99249 14899.24900 2020-01-01 2020-01-02 2020-01-01 00:02:44 2020-01-02 03:31:05 2020-01-01 00:02:44.000 2020-01-02 03:31:05.000 164 99065 49614.5 4961450 164 99065 49614.5 4961450 -32405 32530 5050.02 505002 -125 126 1.7 170 -165 101 10155 99066 0.49549 297.49549 148.99549 14899.54954 0.49549 297.49548 148.99549 14899.54957 0.49549 297.49549 148.99548999999996 14899.54900 2020-01-01 2020-01-02 2020-01-01 00:02:45 2020-01-02 03:31:06 2020-01-01 00:02:45.000 2020-01-02 03:31:06.000 165 99066 49615.5 4961550 165 99066 49615.5 4961550 -32404 32531 5051.02 505102 -124 127 2.7 270 -166 101 10156 99067 0.49849 297.49849 148.99849 14899.84984 0.49849 297.4985 148.9985 14899.85015 0.49849 297.49849 148.99848999999998 14899.84900 2020-01-01 2020-01-02 2020-01-01 00:02:46 2020-01-02 03:31:07 2020-01-01 00:02:46.000 2020-01-02 03:31:07.000 166 99067 49616.5 4961650 166 99067 49616.5 4961650 -32403 32532 5052.02 505202 -128 127 1.14 114 -167 101 10157 99068 0.5015 297.5015 149.0015 14900.15015 0.5015 297.5015 149.00149 14900.14984 0.50150 297.50150 149.00150000000008 14900.15000 2020-01-01 2020-01-02 2020-01-01 00:02:47 2020-01-02 03:31:08 2020-01-01 00:02:47.000 2020-01-02 03:31:08.000 167 99068 49617.5 4961750 167 99068 49617.5 4961750 -32402 32533 5053.02 505302 -128 123 -0.42 -42 -168 101 10158 99069 0.5045 297.5045 149.0045 14900.45045 0.5045 297.50452 149.0045 14900.45042 0.50450 297.50450 149.0044999999999 14900.45000 2020-01-01 2020-01-02 2020-01-01 00:02:48 2020-01-02 03:31:09 2020-01-01 00:02:48.000 2020-01-02 03:31:09.000 168 99069 49618.5 4961850 168 99069 49618.5 4961850 -32401 32534 5054.02 505402 -127 124 0.58 58 -169 101 10159 99070 0.5075 297.5075 149.0075 14900.75075 0.5075 297.5075 149.0075 14900.75073 0.50750 297.50750 149.00749999999994 14900.75000 2020-01-01 2020-01-02 2020-01-01 00:02:49 2020-01-02 03:31:10 2020-01-01 00:02:49.000 2020-01-02 03:31:10.000 169 99070 49619.5 4961950 169 99070 49619.5 4961950 -32400 32535 5055.02 505502 -126 125 1.58 158 +165 101 10155 99066 0.49549 297.49549 148.99549 14899.54954 0.49549 297.49548 148.99549 14899.54957 0.49549 297.49549 148.99549000000002 14899.54900 2020-01-01 2020-01-02 2020-01-01 00:02:45 2020-01-02 03:31:06 2020-01-01 00:02:45.000 2020-01-02 03:31:06.000 165 99066 49615.5 4961550 165 99066 49615.5 4961550 -32404 32531 5051.02 505102 -124 127 2.7 270 +166 101 10156 99067 0.49849 297.49849 148.99849 14899.84984 0.49849 297.4985 148.9985 14899.85015 0.49849 297.49849 148.99849 14899.84900 2020-01-01 2020-01-02 2020-01-01 00:02:46 2020-01-02 03:31:07 2020-01-01 00:02:46.000 2020-01-02 03:31:07.000 166 99067 49616.5 4961650 166 99067 49616.5 4961650 -32403 32532 5052.02 505202 -128 127 1.14 114 +167 101 10157 99068 0.5015 297.5015 149.0015 14900.15015 0.5015 297.5015 149.00149 14900.14984 0.50150 297.50150 149.0015 14900.15000 2020-01-01 2020-01-02 2020-01-01 00:02:47 2020-01-02 03:31:08 2020-01-01 00:02:47.000 2020-01-02 03:31:08.000 167 99068 49617.5 4961750 167 99068 49617.5 4961750 -32402 32533 5053.02 505302 -128 123 -0.42 -42 +168 101 10158 99069 0.5045 297.5045 149.0045 14900.45045 0.5045 297.50452 149.0045 14900.45042 0.50450 297.50450 149.0045 14900.45000 2020-01-01 2020-01-02 2020-01-01 00:02:48 2020-01-02 03:31:09 2020-01-01 00:02:48.000 2020-01-02 03:31:09.000 168 99069 49618.5 4961850 168 99069 49618.5 4961850 -32401 32534 5054.02 505402 -127 124 0.58 58 +169 101 10159 99070 0.5075 297.5075 149.0075 14900.75075 0.5075 297.5075 149.0075 14900.75073 0.50750 297.50750 149.0075 14900.75000 2020-01-01 2020-01-02 2020-01-01 00:02:49 2020-01-02 03:31:10 2020-01-01 00:02:49.000 2020-01-02 03:31:10.000 169 99070 49619.5 4961950 169 99070 49619.5 4961950 -32400 32535 5055.02 505502 -126 125 1.58 158 17 102 10007 99917 0.05105 300.05105 150.05105 15155.15615 0.05105 300.05106 150.05105 15155.15638 0.05105 300.05105 150.05105 15155.15605 2020-01-01 2020-01-02 2020-01-01 00:00:17 2020-01-02 03:45:17 2020-01-01 00:00:17.000 2020-01-02 03:45:17.000 17 99917 49967 5046667 17 99917 49967 5046667 -32552 32383 4546.009900990099 459147 -128 127 -1.1584158415841583 -117 -170 101 10160 99071 0.51051 297.51051 149.01051 14901.05105 0.51051 297.5105 149.01051 14901.05104 0.51051 297.51051 149.01051000000004 14901.05100 2020-01-01 2020-01-02 2020-01-01 00:02:50 2020-01-02 03:31:11 2020-01-01 00:02:50.000 2020-01-02 03:31:11.000 170 99071 49620.5 4962050 170 99071 49620.5 4962050 -32399 32536 5056.02 505602 -125 126 2.58 258 +170 101 10160 99071 0.51051 297.51051 149.01051 14901.05105 0.51051 297.5105 149.01051 14901.05104 0.51051 297.51051 149.01050999999998 14901.05100 2020-01-01 2020-01-02 2020-01-01 00:02:50 2020-01-02 03:31:11 2020-01-01 00:02:50.000 2020-01-02 03:31:11.000 170 99071 49620.5 4962050 170 99071 49620.5 4962050 -32399 32536 5056.02 505602 -125 126 2.58 258 171 101 10161 99072 0.51351 297.51351 149.01351 14901.35135 0.51351 297.51352 149.01351 14901.35162 0.51351 297.51351 149.01351 14901.35100 2020-01-01 2020-01-02 2020-01-01 00:02:51 2020-01-02 03:31:12 2020-01-01 00:02:51.000 2020-01-02 03:31:12.000 171 99072 49621.5 4962150 171 99072 49621.5 4962150 -32398 32537 5057.02 505702 -124 127 3.58 358 -172 101 10162 99073 0.51651 297.51651 149.01651 14901.65165 0.51651 297.5165 149.01651 14901.65131 0.51651 297.51651 149.01650999999998 14901.65100 2020-01-01 2020-01-02 2020-01-01 00:02:52 2020-01-02 03:31:13 2020-01-01 00:02:52.000 2020-01-02 03:31:13.000 172 99073 49622.5 4962250 172 99073 49622.5 4962250 -32397 32538 5058.02 505802 -128 127 2.02 202 -173 101 10163 99074 0.51951 297.51951 149.01951 14901.95195 0.51951 297.51953 149.01951 14901.95189 0.51951 297.51951 149.01951000000005 14901.95100 2020-01-01 2020-01-02 2020-01-01 00:02:53 2020-01-02 03:31:14 2020-01-01 00:02:53.000 2020-01-02 03:31:14.000 173 99074 49623.5 4962350 173 99074 49623.5 4962350 -32396 32539 5059.02 505902 -128 127 0.46 46 -174 101 10164 99075 0.52252 297.52252 149.02252 14902.25225 0.52252 297.52252 149.02252 14902.2522 0.52252 297.52252 149.02252000000016 14902.25200 2020-01-01 2020-01-02 2020-01-01 00:02:54 2020-01-02 03:31:15 2020-01-01 00:02:54.000 2020-01-02 03:31:15.000 174 99075 49624.5 4962450 174 99075 49624.5 4962450 -32395 32540 5060.02 506002 -128 124 -1.1 -110 -175 101 10165 99076 0.52552 297.52552 149.02552 14902.55255 0.52552 297.5255 149.02552 14902.55251 0.52552 297.52552 149.02551999999997 14902.55200 2020-01-01 2020-01-02 2020-01-01 00:02:55 2020-01-02 03:31:16 2020-01-01 00:02:55.000 2020-01-02 03:31:16.000 175 99076 49625.5 4962550 175 99076 49625.5 4962550 -32394 32541 5061.02 506102 -127 125 -0.1 -10 -176 101 10166 99077 0.52852 297.52852 149.02852 14902.85285 0.52852 297.52853 149.02853 14902.85312 0.52852 297.52852 149.02851999999993 14902.85200 2020-01-01 2020-01-02 2020-01-01 00:02:56 2020-01-02 03:31:17 2020-01-01 00:02:56.000 2020-01-02 03:31:17.000 176 99077 49626.5 4962650 176 99077 49626.5 4962650 -32393 32542 5062.02 506202 -126 126 0.9 90 -177 101 10167 99078 0.53153 297.53153 149.03153 14903.15315 0.53153 297.53152 149.03152 14903.15278 0.53153 297.53153 149.0315300000001 14903.15300 2020-01-01 2020-01-02 2020-01-01 00:02:57 2020-01-02 03:31:18 2020-01-01 00:02:57.000 2020-01-02 03:31:18.000 177 99078 49627.5 4962750 177 99078 49627.5 4962750 -32392 32543 5063.02 506302 -125 127 1.9 190 -178 101 10168 99079 0.53453 297.53453 149.03453 14903.45345 0.53453 297.53455 149.03453 14903.45352 0.53453 297.53453 149.03453000000005 14903.45300 2020-01-01 2020-01-02 2020-01-01 00:02:58 2020-01-02 03:31:19 2020-01-01 00:02:58.000 2020-01-02 03:31:19.000 178 99079 49628.5 4962850 178 99079 49628.5 4962850 -32391 32544 5064.02 506402 -128 127 0.34 34 -179 101 10169 99080 0.53753 297.53753 149.03753 14903.75375 0.53753 297.53754 149.03753 14903.75366 0.53753 297.53753 149.0375299999999 14903.75300 2020-01-01 2020-01-02 2020-01-01 00:02:59 2020-01-02 03:31:20 2020-01-01 00:02:59.000 2020-01-02 03:31:20.000 179 99080 49629.5 4962950 179 99080 49629.5 4962950 -32390 32545 5065.02 506502 -128 127 -1.22 -122 -18 102 10008 99918 0.05405 300.05405 150.05405 15155.45945 0.05405 300.05405 150.05404 15155.45903 0.05405 300.05405 150.05405000000007 15155.45905 2020-01-01 2020-01-02 2020-01-01 00:00:18 2020-01-02 03:45:18 2020-01-01 00:00:18.000 2020-01-02 03:45:18.000 18 99918 49968 5046768 18 99918 49968 5046768 -32551 32384 4547.009900990099 459248 -128 124 -2.6930693069306932 -272 +172 101 10162 99073 0.51651 297.51651 149.01651 14901.65165 0.51651 297.5165 149.01651 14901.65131 0.51651 297.51651 149.01651 14901.65100 2020-01-01 2020-01-02 2020-01-01 00:02:52 2020-01-02 03:31:13 2020-01-01 00:02:52.000 2020-01-02 03:31:13.000 172 99073 49622.5 4962250 172 99073 49622.5 4962250 -32397 32538 5058.02 505802 -128 127 2.02 202 +173 101 10163 99074 0.51951 297.51951 149.01951 14901.95195 0.51951 297.51953 149.01951 14901.95189 0.51951 297.51951 149.01951 14901.95100 2020-01-01 2020-01-02 2020-01-01 00:02:53 2020-01-02 03:31:14 2020-01-01 00:02:53.000 2020-01-02 03:31:14.000 173 99074 49623.5 4962350 173 99074 49623.5 4962350 -32396 32539 5059.02 505902 -128 127 0.46 46 +174 101 10164 99075 0.52252 297.52252 149.02252 14902.25225 0.52252 297.52252 149.02252 14902.2522 0.52252 297.52252 149.02252000000001 14902.25200 2020-01-01 2020-01-02 2020-01-01 00:02:54 2020-01-02 03:31:15 2020-01-01 00:02:54.000 2020-01-02 03:31:15.000 174 99075 49624.5 4962450 174 99075 49624.5 4962450 -32395 32540 5060.02 506002 -128 124 -1.1 -110 +175 101 10165 99076 0.52552 297.52552 149.02552 14902.55255 0.52552 297.5255 149.02552 14902.55251 0.52552 297.52552 149.02552 14902.55200 2020-01-01 2020-01-02 2020-01-01 00:02:55 2020-01-02 03:31:16 2020-01-01 00:02:55.000 2020-01-02 03:31:16.000 175 99076 49625.5 4962550 175 99076 49625.5 4962550 -32394 32541 5061.02 506102 -127 125 -0.1 -10 +176 101 10166 99077 0.52852 297.52852 149.02852 14902.85285 0.52852 297.52853 149.02853 14902.85312 0.52852 297.52852 149.02852000000001 14902.85200 2020-01-01 2020-01-02 2020-01-01 00:02:56 2020-01-02 03:31:17 2020-01-01 00:02:56.000 2020-01-02 03:31:17.000 176 99077 49626.5 4962650 176 99077 49626.5 4962650 -32393 32542 5062.02 506202 -126 126 0.9 90 +177 101 10167 99078 0.53153 297.53153 149.03153 14903.15315 0.53153 297.53152 149.03152 14903.15278 0.53153 297.53153 149.03153 14903.15300 2020-01-01 2020-01-02 2020-01-01 00:02:57 2020-01-02 03:31:18 2020-01-01 00:02:57.000 2020-01-02 03:31:18.000 177 99078 49627.5 4962750 177 99078 49627.5 4962750 -32392 32543 5063.02 506302 -125 127 1.9 190 +178 101 10168 99079 0.53453 297.53453 149.03453 14903.45345 0.53453 297.53455 149.03453 14903.45352 0.53453 297.53453 149.03453 14903.45300 2020-01-01 2020-01-02 2020-01-01 00:02:58 2020-01-02 03:31:19 2020-01-01 00:02:58.000 2020-01-02 03:31:19.000 178 99079 49628.5 4962850 178 99079 49628.5 4962850 -32391 32544 5064.02 506402 -128 127 0.34 34 +179 101 10169 99080 0.53753 297.53753 149.03753 14903.75375 0.53753 297.53754 149.03753 14903.75366 0.53753 297.53753 149.03753 14903.75300 2020-01-01 2020-01-02 2020-01-01 00:02:59 2020-01-02 03:31:20 2020-01-01 00:02:59.000 2020-01-02 03:31:20.000 179 99080 49629.5 4962950 179 99080 49629.5 4962950 -32390 32545 5065.02 506502 -128 127 -1.22 -122 +18 102 10008 99918 0.05405 300.05405 150.05405 15155.45945 0.05405 300.05405 150.05404 15155.45903 0.05405 300.05405 150.05405 15155.45905 2020-01-01 2020-01-02 2020-01-01 00:00:18 2020-01-02 03:45:18 2020-01-01 00:00:18.000 2020-01-02 03:45:18.000 18 99918 49968 5046768 18 99918 49968 5046768 -32551 32384 4547.009900990099 459248 -128 124 -2.6930693069306932 -272 180 101 10170 99081 0.54054 297.54054 149.04054 14904.05405 0.54054 297.54053 149.04053 14904.05398 0.54054 297.54054 149.04054 14904.05400 2020-01-01 2020-01-02 2020-01-01 00:03:00 2020-01-02 03:31:21 2020-01-01 00:03:00.000 2020-01-02 03:31:21.000 180 99081 49630.5 4963050 180 99081 49630.5 4963050 -32389 32546 5066.02 506602 -128 123 -2.78 -278 -181 101 10171 99082 0.54354 297.54354 149.04354 14904.35435 0.54354 297.54355 149.04354 14904.35459 0.54354 297.54354 149.04354000000004 14904.35400 2020-01-01 2020-01-02 2020-01-01 00:03:01 2020-01-02 03:31:22 2020-01-01 00:03:01.000 2020-01-02 03:31:22.000 181 99082 49631.5 4963150 181 99082 49631.5 4963150 -32388 32547 5067.02 506702 -127 124 -1.78 -178 -182 101 10172 99083 0.54654 297.54654 149.04654 14904.65465 0.54654 297.54654 149.04654 14904.65425 0.54654 297.54654 149.04653999999994 14904.65400 2020-01-01 2020-01-02 2020-01-01 00:03:02 2020-01-02 03:31:23 2020-01-01 00:03:02.000 2020-01-02 03:31:23.000 182 99083 49632.5 4963250 182 99083 49632.5 4963250 -32387 32548 5068.02 506802 -126 125 -0.78 -78 +181 101 10171 99082 0.54354 297.54354 149.04354 14904.35435 0.54354 297.54355 149.04354 14904.35459 0.54354 297.54354 149.04354 14904.35400 2020-01-01 2020-01-02 2020-01-01 00:03:01 2020-01-02 03:31:22 2020-01-01 00:03:01.000 2020-01-02 03:31:22.000 181 99082 49631.5 4963150 181 99082 49631.5 4963150 -32388 32547 5067.02 506702 -127 124 -1.78 -178 +182 101 10172 99083 0.54654 297.54654 149.04654 14904.65465 0.54654 297.54654 149.04654 14904.65425 0.54654 297.54654 149.04654 14904.65400 2020-01-01 2020-01-02 2020-01-01 00:03:02 2020-01-02 03:31:23 2020-01-01 00:03:02.000 2020-01-02 03:31:23.000 182 99083 49632.5 4963250 182 99083 49632.5 4963250 -32387 32548 5068.02 506802 -126 125 -0.78 -78 183 101 10173 99084 0.54954 297.54954 149.04954 14904.95495 0.54954 297.54956 149.04954 14904.95498 0.54954 297.54954 149.04954 14904.95400 2020-01-01 2020-01-02 2020-01-01 00:03:03 2020-01-02 03:31:24 2020-01-01 00:03:03.000 2020-01-02 03:31:24.000 183 99084 49633.5 4963350 183 99084 49633.5 4963350 -32386 32549 5069.02 506902 -125 126 0.22 22 -184 101 10174 99085 0.55255 297.55255 149.05255 14905.25525 0.55255 297.55255 149.05255 14905.25514 0.55255 297.55255 149.05255000000005 14905.25500 2020-01-01 2020-01-02 2020-01-01 00:03:04 2020-01-02 03:31:25 2020-01-01 00:03:04.000 2020-01-02 03:31:25.000 184 99085 49634.5 4963450 184 99085 49634.5 4963450 -32385 32550 5070.02 507002 -124 127 1.22 122 -185 101 10175 99086 0.55555 297.55555 149.05555 14905.55555 0.55555 297.55554 149.05555 14905.55549 0.55555 297.55555 149.05554999999993 14905.55500 2020-01-01 2020-01-02 2020-01-01 00:03:05 2020-01-02 03:31:26 2020-01-01 00:03:05.000 2020-01-02 03:31:26.000 185 99086 49635.5 4963550 185 99086 49635.5 4963550 -32384 32551 5071.02 507102 -128 127 -0.34 -34 -186 101 10176 99087 0.55855 297.55855 149.05855 14905.85585 0.55855 297.55856 149.05856 14905.85607 0.55855 297.55855 149.05854999999997 14905.85500 2020-01-01 2020-01-02 2020-01-01 00:03:06 2020-01-02 03:31:27 2020-01-01 00:03:06.000 2020-01-02 03:31:27.000 186 99087 49636.5 4963650 186 99087 49636.5 4963650 -32383 32552 5072.02 507202 -128 123 -1.9 -190 -187 101 10177 99088 0.56156 297.56156 149.06156 14906.15615 0.56156 297.56155 149.06155 14906.15572 0.56156 297.56156 149.06156000000007 14906.15600 2020-01-01 2020-01-02 2020-01-01 00:03:07 2020-01-02 03:31:28 2020-01-01 00:03:07.000 2020-01-02 03:31:28.000 187 99088 49637.5 4963750 187 99088 49637.5 4963750 -32382 32553 5073.02 507302 -127 124 -0.9 -90 +184 101 10174 99085 0.55255 297.55255 149.05255 14905.25525 0.55255 297.55255 149.05255 14905.25514 0.55255 297.55255 149.05255 14905.25500 2020-01-01 2020-01-02 2020-01-01 00:03:04 2020-01-02 03:31:25 2020-01-01 00:03:04.000 2020-01-02 03:31:25.000 184 99085 49634.5 4963450 184 99085 49634.5 4963450 -32385 32550 5070.02 507002 -124 127 1.22 122 +185 101 10175 99086 0.55555 297.55555 149.05555 14905.55555 0.55555 297.55554 149.05555 14905.55549 0.55555 297.55555 149.05555 14905.55500 2020-01-01 2020-01-02 2020-01-01 00:03:05 2020-01-02 03:31:26 2020-01-01 00:03:05.000 2020-01-02 03:31:26.000 185 99086 49635.5 4963550 185 99086 49635.5 4963550 -32384 32551 5071.02 507102 -128 127 -0.34 -34 +186 101 10176 99087 0.55855 297.55855 149.05855 14905.85585 0.55855 297.55856 149.05856 14905.85607 0.55855 297.55855 149.05855 14905.85500 2020-01-01 2020-01-02 2020-01-01 00:03:06 2020-01-02 03:31:27 2020-01-01 00:03:06.000 2020-01-02 03:31:27.000 186 99087 49636.5 4963650 186 99087 49636.5 4963650 -32383 32552 5072.02 507202 -128 123 -1.9 -190 +187 101 10177 99088 0.56156 297.56156 149.06156 14906.15615 0.56156 297.56155 149.06155 14906.15572 0.56156 297.56156 149.06156000000001 14906.15600 2020-01-01 2020-01-02 2020-01-01 00:03:07 2020-01-02 03:31:28 2020-01-01 00:03:07.000 2020-01-02 03:31:28.000 187 99088 49637.5 4963750 187 99088 49637.5 4963750 -32382 32553 5073.02 507302 -127 124 -0.9 -90 188 101 10178 99089 0.56456 297.56456 149.06456 14906.45645 0.56456 297.56458 149.06456 14906.45645 0.56456 297.56456 149.06456 14906.45600 2020-01-01 2020-01-02 2020-01-01 00:03:08 2020-01-02 03:31:29 2020-01-01 00:03:08.000 2020-01-02 03:31:29.000 188 99089 49638.5 4963850 188 99089 49638.5 4963850 -32381 32554 5074.02 507402 -126 125 0.1 10 -189 101 10179 99090 0.56756 297.56756 149.06756 14906.75675 0.56756 297.56757 149.06756 14906.75661 0.56756 297.56756 149.06756000000001 14906.75600 2020-01-01 2020-01-02 2020-01-01 00:03:09 2020-01-02 03:31:30 2020-01-01 00:03:09.000 2020-01-02 03:31:30.000 189 99090 49639.5 4963950 189 99090 49639.5 4963950 -32380 32555 5075.02 507502 -125 126 1.1 110 -19 102 10009 99919 0.05705 300.05705 150.05705 15155.76276 0.05705 300.05707 150.05705 15155.76279 0.05705 300.05705 150.05704999999998 15155.76205 2020-01-01 2020-01-02 2020-01-01 00:00:19 2020-01-02 03:45:19 2020-01-01 00:00:19.000 2020-01-02 03:45:19.000 19 99919 49969 5046869 19 99919 49969 5046869 -32550 32385 4548.009900990099 459349 -127 125 -1.693069306930693 -171 -190 101 10180 99091 0.57057 297.57057 149.07057 14907.05705 0.57057 297.57056 149.07056 14907.05695 0.57057 297.57057 149.07056999999995 14907.05700 2020-01-01 2020-01-02 2020-01-01 00:03:10 2020-01-02 03:31:31 2020-01-01 00:03:10.000 2020-01-02 03:31:31.000 190 99091 49640.5 4964050 190 99091 49640.5 4964050 -32379 32556 5076.02 507602 -124 127 2.1 210 +189 101 10179 99090 0.56756 297.56756 149.06756 14906.75675 0.56756 297.56757 149.06756 14906.75661 0.56756 297.56756 149.06756 14906.75600 2020-01-01 2020-01-02 2020-01-01 00:03:09 2020-01-02 03:31:30 2020-01-01 00:03:09.000 2020-01-02 03:31:30.000 189 99090 49639.5 4963950 189 99090 49639.5 4963950 -32380 32555 5075.02 507502 -125 126 1.1 110 +19 102 10009 99919 0.05705 300.05705 150.05705 15155.76276 0.05705 300.05707 150.05705 15155.76279 0.05705 300.05705 150.05705 15155.76205 2020-01-01 2020-01-02 2020-01-01 00:00:19 2020-01-02 03:45:19 2020-01-01 00:00:19.000 2020-01-02 03:45:19.000 19 99919 49969 5046869 19 99919 49969 5046869 -32550 32385 4548.009900990099 459349 -127 125 -1.693069306930693 -171 +190 101 10180 99091 0.57057 297.57057 149.07057 14907.05705 0.57057 297.57056 149.07056 14907.05695 0.57057 297.57057 149.07057 14907.05700 2020-01-01 2020-01-02 2020-01-01 00:03:10 2020-01-02 03:31:31 2020-01-01 00:03:10.000 2020-01-02 03:31:31.000 190 99091 49640.5 4964050 190 99091 49640.5 4964050 -32379 32556 5076.02 507602 -124 127 2.1 210 191 101 10181 99092 0.57357 297.57357 149.07357 14907.35735 0.57357 297.57358 149.07357 14907.35753 0.57357 297.57357 149.07357 14907.35700 2020-01-01 2020-01-02 2020-01-01 00:03:11 2020-01-02 03:31:32 2020-01-01 00:03:11.000 2020-01-02 03:31:32.000 191 99092 49641.5 4964150 191 99092 49641.5 4964150 -32378 32557 5077.02 507702 -128 127 0.54 54 -192 101 10182 99093 0.57657 297.57657 149.07657 14907.65765 0.57657 297.57657 149.07657 14907.65784 0.57657 297.57657 149.07656999999998 14907.65700 2020-01-01 2020-01-02 2020-01-01 00:03:12 2020-01-02 03:31:33 2020-01-01 00:03:12.000 2020-01-02 03:31:33.000 192 99093 49642.5 4964250 192 99093 49642.5 4964250 -32377 32558 5078.02 507802 -128 123 -1.02 -102 -193 101 10183 99094 0.57957 297.57957 149.07957 14907.95795 0.57957 297.5796 149.07957 14907.95793 0.57957 297.57957 149.07956999999993 14907.95700 2020-01-01 2020-01-02 2020-01-01 00:03:13 2020-01-02 03:31:34 2020-01-01 00:03:13.000 2020-01-02 03:31:34.000 193 99094 49643.5 4964350 193 99094 49643.5 4964350 -32376 32559 5079.02 507902 -127 124 -0.02 -2 -194 101 10184 99095 0.58258 297.58258 149.08258 14908.25825 0.58258 297.58258 149.08258 14908.25811 0.58258 297.58258 149.0825800000001 14908.25800 2020-01-01 2020-01-02 2020-01-01 00:03:14 2020-01-02 03:31:35 2020-01-01 00:03:14.000 2020-01-02 03:31:35.000 194 99095 49644.5 4964450 194 99095 49644.5 4964450 -32375 32560 5080.02 508002 -126 125 0.98 98 -195 101 10185 99096 0.58558 297.58558 149.08558 14908.55855 0.58558 297.58557 149.08558 14908.55842 0.58558 297.58558 149.08558000000008 14908.55800 2020-01-01 2020-01-02 2020-01-01 00:03:15 2020-01-02 03:31:36 2020-01-01 00:03:15.000 2020-01-02 03:31:36.000 195 99096 49645.5 4964550 195 99096 49645.5 4964550 -32374 32561 5081.02 508102 -125 126 1.98 198 -196 101 10186 99097 0.58858 297.58858 149.08858 14908.85885 0.58858 297.5886 149.08859 14908.859 0.58858 297.58858 149.08857999999992 14908.85800 2020-01-01 2020-01-02 2020-01-01 00:03:16 2020-01-02 03:31:37 2020-01-01 00:03:16.000 2020-01-02 03:31:37.000 196 99097 49646.5 4964650 196 99097 49646.5 4964650 -32373 32562 5082.02 508202 -124 127 2.98 298 +192 101 10182 99093 0.57657 297.57657 149.07657 14907.65765 0.57657 297.57657 149.07657 14907.65784 0.57657 297.57657 149.07657 14907.65700 2020-01-01 2020-01-02 2020-01-01 00:03:12 2020-01-02 03:31:33 2020-01-01 00:03:12.000 2020-01-02 03:31:33.000 192 99093 49642.5 4964250 192 99093 49642.5 4964250 -32377 32558 5078.02 507802 -128 123 -1.02 -102 +193 101 10183 99094 0.57957 297.57957 149.07957 14907.95795 0.57957 297.5796 149.07957 14907.95793 0.57957 297.57957 149.07957 14907.95700 2020-01-01 2020-01-02 2020-01-01 00:03:13 2020-01-02 03:31:34 2020-01-01 00:03:13.000 2020-01-02 03:31:34.000 193 99094 49643.5 4964350 193 99094 49643.5 4964350 -32376 32559 5079.02 507902 -127 124 -0.02 -2 +194 101 10184 99095 0.58258 297.58258 149.08258 14908.25825 0.58258 297.58258 149.08258 14908.25811 0.58258 297.58258 149.08258 14908.25800 2020-01-01 2020-01-02 2020-01-01 00:03:14 2020-01-02 03:31:35 2020-01-01 00:03:14.000 2020-01-02 03:31:35.000 194 99095 49644.5 4964450 194 99095 49644.5 4964450 -32375 32560 5080.02 508002 -126 125 0.98 98 +195 101 10185 99096 0.58558 297.58558 149.08558 14908.55855 0.58558 297.58557 149.08558 14908.55842 0.58558 297.58558 149.08558000000002 14908.55800 2020-01-01 2020-01-02 2020-01-01 00:03:15 2020-01-02 03:31:36 2020-01-01 00:03:15.000 2020-01-02 03:31:36.000 195 99096 49645.5 4964550 195 99096 49645.5 4964550 -32374 32561 5081.02 508102 -125 126 1.98 198 +196 101 10186 99097 0.58858 297.58858 149.08858 14908.85885 0.58858 297.5886 149.08859 14908.859 0.58858 297.58858 149.08858 14908.85800 2020-01-01 2020-01-02 2020-01-01 00:03:16 2020-01-02 03:31:37 2020-01-01 00:03:16.000 2020-01-02 03:31:37.000 196 99097 49646.5 4964650 196 99097 49646.5 4964650 -32373 32562 5082.02 508202 -124 127 2.98 298 197 101 10187 99098 0.59159 297.59159 149.09159 14909.15915 0.59159 297.59158 149.09159 14909.15931 0.59159 297.59159 149.09159 14909.15900 2020-01-01 2020-01-02 2020-01-01 00:03:17 2020-01-02 03:31:38 2020-01-01 00:03:17.000 2020-01-02 03:31:38.000 197 99098 49647.5 4964750 197 99098 49647.5 4964750 -32372 32563 5083.02 508302 -128 127 1.42 142 -198 101 10188 99099 0.59459 297.59459 149.09459 14909.45945 0.59459 297.5946 149.09459 14909.4594 0.59459 297.59459 149.09459000000004 14909.45900 2020-01-01 2020-01-02 2020-01-01 00:03:18 2020-01-02 03:31:39 2020-01-01 00:03:18.000 2020-01-02 03:31:39.000 198 99099 49648.5 4964850 198 99099 49648.5 4964850 -32371 32564 5084.02 508402 -128 127 -0.14 -14 -199 101 10189 99100 0.59759 297.59759 149.09759 14909.75975 0.59759 297.5976 149.09759 14909.75958 0.59759 297.59759 149.09758999999997 14909.75900 2020-01-01 2020-01-02 2020-01-01 00:03:19 2020-01-02 03:31:40 2020-01-01 00:03:19.000 2020-01-02 03:31:40.000 199 99100 49649.5 4964950 199 99100 49649.5 4964950 -32370 32565 5085.02 508502 -128 124 -1.7 -170 -2 102 1001 9992 0.006 300.006 150.006 15150.6066 0.006 300.006 150.006 15150.6069 0.00600 300.00600 150.00599999999997 15150.60600 2020-01-01 2020-01-02 2020-01-01 00:00:02 2020-01-02 03:45:02 2020-01-01 00:00:02.000 2020-01-02 03:45:02.000 2 99902 49952 5045152 2 99902 49952 5045152 -32567 32368 4531.009900990099 457632 -125 126 -0.9504950495049505 -96 -20 102 10010 99920 0.06006 300.06006 150.06006 15156.06606 0.06006 300.06006 150.06005 15156.06593 0.06006 300.06006 150.0600599999999 15156.06606 2020-01-01 2020-01-02 2020-01-01 00:00:20 2020-01-02 03:45:20 2020-01-01 00:00:20.000 2020-01-02 03:45:20.000 20 99920 49970 5046970 20 99920 49970 5046970 -32549 32386 4549.009900990099 459450 -126 126 -0.693069306930693 -70 -200 101 10190 99101 0.6006 297.6006 149.1006 14910.06006 0.6006 297.6006 149.10059 14910.0599 0.60060 297.60060 149.1005999999999 14910.06000 2020-01-01 2020-01-02 2020-01-01 00:03:20 2020-01-02 03:31:41 2020-01-01 00:03:20.000 2020-01-02 03:31:41.000 200 99101 49650.5 4965050 200 99101 49650.5 4965050 -32369 32566 5086.02 508602 -127 125 -0.7 -70 -201 101 10191 99102 0.6036 297.6036 149.1036 14910.36036 0.6036 297.6036 149.1036 14910.36063 0.60360 297.60360 149.10360000000009 14910.36000 2020-01-01 2020-01-02 2020-01-01 00:03:21 2020-01-02 03:31:42 2020-01-01 00:03:21.000 2020-01-02 03:31:42.000 201 99102 49651.5 4965150 201 99102 49651.5 4965150 -32368 32567 5087.02 508702 -126 126 0.3 30 -202 101 10192 99103 0.6066 297.6066 149.1066 14910.66066 0.6066 297.6066 149.1066 14910.66078 0.60660 297.60660 149.10659999999993 14910.66000 2020-01-01 2020-01-02 2020-01-01 00:03:22 2020-01-02 03:31:43 2020-01-01 00:03:22.000 2020-01-02 03:31:43.000 202 99103 49652.5 4965250 202 99103 49652.5 4965250 -32367 32568 5088.02 508802 -125 127 1.3 130 -203 101 10193 99104 0.6096 297.6096 149.1096 14910.96096 0.6096 297.60962 149.1096 14910.9609 0.60960 297.60960 149.1095999999999 14910.96000 2020-01-01 2020-01-02 2020-01-01 00:03:23 2020-01-02 03:31:44 2020-01-01 00:03:23.000 2020-01-02 03:31:44.000 203 99104 49653.5 4965350 203 99104 49653.5 4965350 -32366 32569 5089.02 508902 -128 127 -0.26 -26 -204 101 10194 99105 0.61261 297.61261 149.11261 14911.26126 0.61261 297.6126 149.11261 14911.26105 0.61261 297.61261 149.11261000000005 14911.26100 2020-01-01 2020-01-02 2020-01-01 00:03:24 2020-01-02 03:31:45 2020-01-01 00:03:24.000 2020-01-02 03:31:45.000 204 99105 49654.5 4965450 204 99105 49654.5 4965450 -32365 32570 5090.02 509002 -128 127 -1.82 -182 -205 101 10195 99106 0.61561 297.61561 149.11561 14911.56156 0.61561 297.6156 149.11561 14911.56137 0.61561 297.61561 149.11561000000003 14911.56100 2020-01-01 2020-01-02 2020-01-01 00:03:25 2020-01-02 03:31:46 2020-01-01 00:03:25.000 2020-01-02 03:31:46.000 205 99106 49655.5 4965550 205 99106 49655.5 4965550 -32364 32571 5091.02 509102 -128 123 -3.38 -338 -206 101 10196 99107 0.61861 297.61861 149.11861 14911.86186 0.61861 297.61862 149.11862 14911.8621 0.61861 297.61861 149.11860999999985 14911.86100 2020-01-01 2020-01-02 2020-01-01 00:03:26 2020-01-02 03:31:47 2020-01-01 00:03:26.000 2020-01-02 03:31:47.000 206 99107 49656.5 4965650 206 99107 49656.5 4965650 -32363 32572 5092.02 509202 -127 124 -2.38 -238 -207 101 10197 99108 0.62162 297.62162 149.12162 14912.16216 0.62162 297.6216 149.12162 14912.16225 0.62162 297.62162 149.12161999999992 14912.16200 2020-01-01 2020-01-02 2020-01-01 00:03:27 2020-01-02 03:31:48 2020-01-01 00:03:27.000 2020-01-02 03:31:48.000 207 99108 49657.5 4965750 207 99108 49657.5 4965750 -32362 32573 5093.02 509302 -126 125 -1.38 -138 +198 101 10188 99099 0.59459 297.59459 149.09459 14909.45945 0.59459 297.5946 149.09459 14909.4594 0.59459 297.59459 149.09459 14909.45900 2020-01-01 2020-01-02 2020-01-01 00:03:18 2020-01-02 03:31:39 2020-01-01 00:03:18.000 2020-01-02 03:31:39.000 198 99099 49648.5 4964850 198 99099 49648.5 4964850 -32371 32564 5084.02 508402 -128 127 -0.14 -14 +199 101 10189 99100 0.59759 297.59759 149.09759 14909.75975 0.59759 297.5976 149.09759 14909.75958 0.59759 297.59759 149.09759 14909.75900 2020-01-01 2020-01-02 2020-01-01 00:03:19 2020-01-02 03:31:40 2020-01-01 00:03:19.000 2020-01-02 03:31:40.000 199 99100 49649.5 4964950 199 99100 49649.5 4964950 -32370 32565 5085.02 508502 -128 124 -1.7 -170 +2 102 1001 9992 0.006 300.006 150.006 15150.6066 0.006 300.006 150.006 15150.6069 0.00600 300.00600 150.006 15150.60600 2020-01-01 2020-01-02 2020-01-01 00:00:02 2020-01-02 03:45:02 2020-01-01 00:00:02.000 2020-01-02 03:45:02.000 2 99902 49952 5045152 2 99902 49952 5045152 -32567 32368 4531.009900990099 457632 -125 126 -0.9504950495049505 -96 +20 102 10010 99920 0.06006 300.06006 150.06006 15156.06606 0.06006 300.06006 150.06005 15156.06593 0.06006 300.06006 150.06006 15156.06606 2020-01-01 2020-01-02 2020-01-01 00:00:20 2020-01-02 03:45:20 2020-01-01 00:00:20.000 2020-01-02 03:45:20.000 20 99920 49970 5046970 20 99920 49970 5046970 -32549 32386 4549.009900990099 459450 -126 126 -0.693069306930693 -70 +200 101 10190 99101 0.6006 297.6006 149.1006 14910.06006 0.6006 297.6006 149.10059 14910.0599 0.60060 297.60060 149.1006 14910.06000 2020-01-01 2020-01-02 2020-01-01 00:03:20 2020-01-02 03:31:41 2020-01-01 00:03:20.000 2020-01-02 03:31:41.000 200 99101 49650.5 4965050 200 99101 49650.5 4965050 -32369 32566 5086.02 508602 -127 125 -0.7 -70 +201 101 10191 99102 0.6036 297.6036 149.1036 14910.36036 0.6036 297.6036 149.1036 14910.36063 0.60360 297.60360 149.1036 14910.36000 2020-01-01 2020-01-02 2020-01-01 00:03:21 2020-01-02 03:31:42 2020-01-01 00:03:21.000 2020-01-02 03:31:42.000 201 99102 49651.5 4965150 201 99102 49651.5 4965150 -32368 32567 5087.02 508702 -126 126 0.3 30 +202 101 10192 99103 0.6066 297.6066 149.1066 14910.66066 0.6066 297.6066 149.1066 14910.66078 0.60660 297.60660 149.1066 14910.66000 2020-01-01 2020-01-02 2020-01-01 00:03:22 2020-01-02 03:31:43 2020-01-01 00:03:22.000 2020-01-02 03:31:43.000 202 99103 49652.5 4965250 202 99103 49652.5 4965250 -32367 32568 5088.02 508802 -125 127 1.3 130 +203 101 10193 99104 0.6096 297.6096 149.1096 14910.96096 0.6096 297.60962 149.1096 14910.9609 0.60960 297.60960 149.1096 14910.96000 2020-01-01 2020-01-02 2020-01-01 00:03:23 2020-01-02 03:31:44 2020-01-01 00:03:23.000 2020-01-02 03:31:44.000 203 99104 49653.5 4965350 203 99104 49653.5 4965350 -32366 32569 5089.02 508902 -128 127 -0.26 -26 +204 101 10194 99105 0.61261 297.61261 149.11261 14911.26126 0.61261 297.6126 149.11261 14911.26105 0.61261 297.61261 149.11261000000002 14911.26100 2020-01-01 2020-01-02 2020-01-01 00:03:24 2020-01-02 03:31:45 2020-01-01 00:03:24.000 2020-01-02 03:31:45.000 204 99105 49654.5 4965450 204 99105 49654.5 4965450 -32365 32570 5090.02 509002 -128 127 -1.82 -182 +205 101 10195 99106 0.61561 297.61561 149.11561 14911.56156 0.61561 297.6156 149.11561 14911.56137 0.61561 297.61561 149.11561 14911.56100 2020-01-01 2020-01-02 2020-01-01 00:03:25 2020-01-02 03:31:46 2020-01-01 00:03:25.000 2020-01-02 03:31:46.000 205 99106 49655.5 4965550 205 99106 49655.5 4965550 -32364 32571 5091.02 509102 -128 123 -3.38 -338 +206 101 10196 99107 0.61861 297.61861 149.11861 14911.86186 0.61861 297.61862 149.11862 14911.8621 0.61861 297.61861 149.11861000000002 14911.86100 2020-01-01 2020-01-02 2020-01-01 00:03:26 2020-01-02 03:31:47 2020-01-01 00:03:26.000 2020-01-02 03:31:47.000 206 99107 49656.5 4965650 206 99107 49656.5 4965650 -32363 32572 5092.02 509202 -127 124 -2.38 -238 +207 101 10197 99108 0.62162 297.62162 149.12162 14912.16216 0.62162 297.6216 149.12162 14912.16225 0.62162 297.62162 149.12162 14912.16200 2020-01-01 2020-01-02 2020-01-01 00:03:27 2020-01-02 03:31:48 2020-01-01 00:03:27.000 2020-01-02 03:31:48.000 207 99108 49657.5 4965750 207 99108 49657.5 4965750 -32362 32573 5093.02 509302 -126 125 -1.38 -138 208 101 10198 99109 0.62462 297.62462 149.12462 14912.46246 0.62462 297.62463 149.12462 14912.46237 0.62462 297.62462 149.12462 14912.46200 2020-01-01 2020-01-02 2020-01-01 00:03:28 2020-01-02 03:31:49 2020-01-01 00:03:28.000 2020-01-02 03:31:49.000 208 99109 49658.5 4965850 208 99109 49658.5 4965850 -32361 32574 5094.02 509402 -125 126 -0.38 -38 -209 101 10199 99110 0.62762 297.62762 149.12762 14912.76276 0.62762 297.62762 149.12762 14912.76253 0.62762 297.62762 149.12761999999998 14912.76200 2020-01-01 2020-01-02 2020-01-01 00:03:29 2020-01-02 03:31:50 2020-01-01 00:03:29.000 2020-01-02 03:31:50.000 209 99110 49659.5 4965950 209 99110 49659.5 4965950 -32360 32575 5095.02 509502 -124 127 0.62 62 -21 102 10011 99921 0.06306 300.06306 150.06306 15156.36936 0.06306 300.06305 150.06306 15156.36927 0.06306 300.06306 150.06306000000012 15156.36906 2020-01-01 2020-01-02 2020-01-01 00:00:21 2020-01-02 03:45:21 2020-01-01 00:00:21.000 2020-01-02 03:45:21.000 21 99921 49971 5047071 21 99921 49971 5047071 -32548 32387 4550.009900990099 459551 -125 127 0.3069306930693069 31 -210 101 10200 99111 0.63063 297.63063 149.13063 14913.06306 0.63063 297.63065 149.13063 14913.06326 0.63063 297.63063 149.13062999999994 14913.06300 2020-01-01 2020-01-02 2020-01-01 00:03:30 2020-01-02 03:31:51 2020-01-01 00:03:30.000 2020-01-02 03:31:51.000 210 99111 49660.5 4966050 210 99111 49660.5 4966050 -32359 32576 5096.02 509602 -128 127 -0.94 -94 -211 101 10201 99112 0.63363 297.63363 149.13363 14913.36336 0.63363 297.63364 149.13363 14913.36357 0.63363 297.63363 149.13363000000004 14913.36300 2020-01-01 2020-01-02 2020-01-01 00:03:31 2020-01-02 03:31:52 2020-01-01 00:03:31.000 2020-01-02 03:31:52.000 211 99112 49661.5 4966150 211 99112 49661.5 4966150 -32358 32577 5097.02 509702 -128 123 -2.5 -250 -212 101 10202 99113 0.63663 297.63663 149.13663 14913.66366 0.63663 297.63663 149.13663 14913.66372 0.63663 297.63663 149.13663000000008 14913.66300 2020-01-01 2020-01-02 2020-01-01 00:03:32 2020-01-02 03:31:53 2020-01-01 00:03:32.000 2020-01-02 03:31:53.000 212 99113 49662.5 4966250 212 99113 49662.5 4966250 -32357 32578 5098.02 509802 -127 124 -1.5 -150 -213 101 10203 99114 0.63963 297.63963 149.13963 14913.96396 0.63963 297.63965 149.13963 14913.96384 0.63963 297.63963 149.1396299999999 14913.96300 2020-01-01 2020-01-02 2020-01-01 00:03:33 2020-01-02 03:31:54 2020-01-01 00:03:33.000 2020-01-02 03:31:54.000 213 99114 49663.5 4966350 213 99114 49663.5 4966350 -32356 32579 5099.02 509902 -126 125 -0.5 -50 +209 101 10199 99110 0.62762 297.62762 149.12762 14912.76276 0.62762 297.62762 149.12762 14912.76253 0.62762 297.62762 149.12762 14912.76200 2020-01-01 2020-01-02 2020-01-01 00:03:29 2020-01-02 03:31:50 2020-01-01 00:03:29.000 2020-01-02 03:31:50.000 209 99110 49659.5 4965950 209 99110 49659.5 4965950 -32360 32575 5095.02 509502 -124 127 0.62 62 +21 102 10011 99921 0.06306 300.06306 150.06306 15156.36936 0.06306 300.06305 150.06306 15156.36927 0.06306 300.06306 150.06306 15156.36906 2020-01-01 2020-01-02 2020-01-01 00:00:21 2020-01-02 03:45:21 2020-01-01 00:00:21.000 2020-01-02 03:45:21.000 21 99921 49971 5047071 21 99921 49971 5047071 -32548 32387 4550.009900990099 459551 -125 127 0.3069306930693069 31 +210 101 10200 99111 0.63063 297.63063 149.13063 14913.06306 0.63063 297.63065 149.13063 14913.06326 0.63063 297.63063 149.13063 14913.06300 2020-01-01 2020-01-02 2020-01-01 00:03:30 2020-01-02 03:31:51 2020-01-01 00:03:30.000 2020-01-02 03:31:51.000 210 99111 49660.5 4966050 210 99111 49660.5 4966050 -32359 32576 5096.02 509602 -128 127 -0.94 -94 +211 101 10201 99112 0.63363 297.63363 149.13363 14913.36336 0.63363 297.63364 149.13363 14913.36357 0.63363 297.63363 149.13362999999998 14913.36300 2020-01-01 2020-01-02 2020-01-01 00:03:31 2020-01-02 03:31:52 2020-01-01 00:03:31.000 2020-01-02 03:31:52.000 211 99112 49661.5 4966150 211 99112 49661.5 4966150 -32358 32577 5097.02 509702 -128 123 -2.5 -250 +212 101 10202 99113 0.63663 297.63663 149.13663 14913.66366 0.63663 297.63663 149.13663 14913.66372 0.63663 297.63663 149.13663 14913.66300 2020-01-01 2020-01-02 2020-01-01 00:03:32 2020-01-02 03:31:53 2020-01-01 00:03:32.000 2020-01-02 03:31:53.000 212 99113 49662.5 4966250 212 99113 49662.5 4966250 -32357 32578 5098.02 509802 -127 124 -1.5 -150 +213 101 10203 99114 0.63963 297.63963 149.13963 14913.96396 0.63963 297.63965 149.13963 14913.96384 0.63963 297.63963 149.13963 14913.96300 2020-01-01 2020-01-02 2020-01-01 00:03:33 2020-01-02 03:31:54 2020-01-01 00:03:33.000 2020-01-02 03:31:54.000 213 99114 49663.5 4966350 213 99114 49663.5 4966350 -32356 32579 5099.02 509902 -126 125 -0.5 -50 214 101 10204 99115 0.64264 297.64264 149.14264 14914.26426 0.64264 297.64264 149.14263 14914.26399 0.64264 297.64264 149.14264 14914.26400 2020-01-01 2020-01-02 2020-01-01 00:03:34 2020-01-02 03:31:55 2020-01-01 00:03:34.000 2020-01-02 03:31:55.000 214 99115 49664.5 4966450 214 99115 49664.5 4966450 -32355 32580 5100.02 510002 -125 126 0.5 50 -215 101 10205 99116 0.64564 297.64564 149.14564 14914.56456 0.64564 297.64566 149.14564 14914.56473 0.64564 297.64564 149.14564000000007 14914.56400 2020-01-01 2020-01-02 2020-01-01 00:03:35 2020-01-02 03:31:56 2020-01-01 00:03:35.000 2020-01-02 03:31:56.000 215 99116 49665.5 4966550 215 99116 49665.5 4966550 -32354 32581 5101.02 510102 -124 127 1.5 150 -216 101 10206 99117 0.64864 297.64864 149.14864 14914.86486 0.64864 297.64865 149.14865 14914.86504 0.64864 297.64864 149.14863999999997 14914.86400 2020-01-01 2020-01-02 2020-01-01 00:03:36 2020-01-02 03:31:57 2020-01-01 00:03:36.000 2020-01-02 03:31:57.000 216 99117 49666.5 4966650 216 99117 49666.5 4966650 -32353 32582 5102.02 510202 -128 127 -0.06 -6 -217 101 10207 99118 0.65165 297.65165 149.15165 14915.16516 0.65165 297.65164 149.15165 14915.16523 0.65165 297.65165 149.1516499999999 14915.16500 2020-01-01 2020-01-02 2020-01-01 00:03:37 2020-01-02 03:31:58 2020-01-01 00:03:37.000 2020-01-02 03:31:58.000 217 99118 49667.5 4966750 217 99118 49667.5 4966750 -32352 32583 5103.02 510302 -128 123 -1.62 -162 -218 101 10208 99119 0.65465 297.65465 149.15465 14915.46546 0.65465 297.65466 149.15465 14915.46531 0.65465 297.65465 149.15465000000012 14915.46500 2020-01-01 2020-01-02 2020-01-01 00:03:38 2020-01-02 03:31:59 2020-01-01 00:03:38.000 2020-01-02 03:31:59.000 218 99119 49668.5 4966850 218 99119 49668.5 4966850 -32351 32584 5104.02 510402 -127 124 -0.62 -62 -219 101 10209 99120 0.65765 297.65765 149.15765 14915.76576 0.65765 297.65765 149.15765 14915.76562 0.65765 297.65765 149.15764999999993 14915.76500 2020-01-01 2020-01-02 2020-01-01 00:03:39 2020-01-02 03:32:00 2020-01-01 00:03:39.000 2020-01-02 03:32:00.000 219 99120 49669.5 4966950 219 99120 49669.5 4966950 -32350 32585 5105.02 510502 -126 125 0.38 38 -22 102 10012 99922 0.06606 300.06606 150.06606 15156.67267 0.06606 300.06607 150.06606 15156.67287 0.06606 300.06606 150.06605999999996 15156.67206 2020-01-01 2020-01-02 2020-01-01 00:00:22 2020-01-02 03:45:22 2020-01-01 00:00:22.000 2020-01-02 03:45:22.000 22 99922 49972 5047172 22 99922 49972 5047172 -32547 32388 4551.009900990099 459652 -128 127 -1.2277227722772277 -124 -220 101 10210 99121 0.66066 297.66066 149.16066 14916.06606 0.66066 297.66068 149.16066 14916.06619 0.66066 297.66066 149.16065999999984 14916.06600 2020-01-01 2020-01-02 2020-01-01 00:03:40 2020-01-02 03:32:01 2020-01-01 00:03:40.000 2020-01-02 03:32:01.000 220 99121 49670.5 4967050 220 99121 49670.5 4967050 -32349 32586 5106.02 510602 -125 126 1.38 138 -221 101 10211 99122 0.66366 297.66366 149.16366 14916.36636 0.66366 297.66367 149.16366 14916.36651 0.66366 297.66366 149.16366000000008 14916.36600 2020-01-01 2020-01-02 2020-01-01 00:03:41 2020-01-02 03:32:02 2020-01-01 00:03:41.000 2020-01-02 03:32:02.000 221 99122 49671.5 4967150 221 99122 49671.5 4967150 -32348 32587 5107.02 510702 -124 127 2.38 238 -222 101 10212 99123 0.66666 297.66666 149.16666 14916.66666 0.66666 297.66666 149.16666 14916.6667 0.66666 297.66666 149.16666000000004 14916.66600 2020-01-01 2020-01-02 2020-01-01 00:03:42 2020-01-02 03:32:03 2020-01-01 00:03:42.000 2020-01-02 03:32:03.000 222 99123 49672.5 4967250 222 99123 49672.5 4967250 -32347 32588 5108.02 510802 -128 127 0.82 82 -223 101 10213 99124 0.66966 297.66966 149.16966 14916.96696 0.66966 297.66968 149.16966 14916.96678 0.66966 297.66966 149.16965999999985 14916.96600 2020-01-01 2020-01-02 2020-01-01 00:03:43 2020-01-02 03:32:04 2020-01-01 00:03:43.000 2020-01-02 03:32:04.000 223 99124 49673.5 4967350 223 99124 49673.5 4967350 -32346 32589 5109.02 510902 -128 127 -0.74 -74 -224 101 10214 99125 0.67267 297.67267 149.17267 14917.26726 0.67267 297.67267 149.17267 14917.26709 0.67267 297.67267 149.17266999999995 14917.26700 2020-01-01 2020-01-02 2020-01-01 00:03:44 2020-01-02 03:32:05 2020-01-01 00:03:44.000 2020-01-02 03:32:05.000 224 99125 49674.5 4967450 224 99125 49674.5 4967450 -32345 32590 5110.02 511002 -128 124 -2.3 -230 +215 101 10205 99116 0.64564 297.64564 149.14564 14914.56456 0.64564 297.64566 149.14564 14914.56473 0.64564 297.64564 149.14564000000001 14914.56400 2020-01-01 2020-01-02 2020-01-01 00:03:35 2020-01-02 03:31:56 2020-01-01 00:03:35.000 2020-01-02 03:31:56.000 215 99116 49665.5 4966550 215 99116 49665.5 4966550 -32354 32581 5101.02 510102 -124 127 1.5 150 +216 101 10206 99117 0.64864 297.64864 149.14864 14914.86486 0.64864 297.64865 149.14865 14914.86504 0.64864 297.64864 149.14864 14914.86400 2020-01-01 2020-01-02 2020-01-01 00:03:36 2020-01-02 03:31:57 2020-01-01 00:03:36.000 2020-01-02 03:31:57.000 216 99117 49666.5 4966650 216 99117 49666.5 4966650 -32353 32582 5102.02 510202 -128 127 -0.06 -6 +217 101 10207 99118 0.65165 297.65165 149.15165 14915.16516 0.65165 297.65164 149.15165 14915.16523 0.65165 297.65165 149.15165000000002 14915.16500 2020-01-01 2020-01-02 2020-01-01 00:03:37 2020-01-02 03:31:58 2020-01-01 00:03:37.000 2020-01-02 03:31:58.000 217 99118 49667.5 4966750 217 99118 49667.5 4966750 -32352 32583 5103.02 510302 -128 123 -1.62 -162 +218 101 10208 99119 0.65465 297.65465 149.15465 14915.46546 0.65465 297.65466 149.15465 14915.46531 0.65465 297.65465 149.15465 14915.46500 2020-01-01 2020-01-02 2020-01-01 00:03:38 2020-01-02 03:31:59 2020-01-01 00:03:38.000 2020-01-02 03:31:59.000 218 99119 49668.5 4966850 218 99119 49668.5 4966850 -32351 32584 5104.02 510402 -127 124 -0.62 -62 +219 101 10209 99120 0.65765 297.65765 149.15765 14915.76576 0.65765 297.65765 149.15765 14915.76562 0.65765 297.65765 149.15765 14915.76500 2020-01-01 2020-01-02 2020-01-01 00:03:39 2020-01-02 03:32:00 2020-01-01 00:03:39.000 2020-01-02 03:32:00.000 219 99120 49669.5 4966950 219 99120 49669.5 4966950 -32350 32585 5105.02 510502 -126 125 0.38 38 +22 102 10012 99922 0.06606 300.06606 150.06606 15156.67267 0.06606 300.06607 150.06606 15156.67287 0.06606 300.06606 150.06606000000002 15156.67206 2020-01-01 2020-01-02 2020-01-01 00:00:22 2020-01-02 03:45:22 2020-01-01 00:00:22.000 2020-01-02 03:45:22.000 22 99922 49972 5047172 22 99922 49972 5047172 -32547 32388 4551.009900990099 459652 -128 127 -1.2277227722772277 -124 +220 101 10210 99121 0.66066 297.66066 149.16066 14916.06606 0.66066 297.66068 149.16066 14916.06619 0.66066 297.66066 149.16066 14916.06600 2020-01-01 2020-01-02 2020-01-01 00:03:40 2020-01-02 03:32:01 2020-01-01 00:03:40.000 2020-01-02 03:32:01.000 220 99121 49670.5 4967050 220 99121 49670.5 4967050 -32349 32586 5106.02 510602 -125 126 1.38 138 +221 101 10211 99122 0.66366 297.66366 149.16366 14916.36636 0.66366 297.66367 149.16366 14916.36651 0.66366 297.66366 149.16366 14916.36600 2020-01-01 2020-01-02 2020-01-01 00:03:41 2020-01-02 03:32:02 2020-01-01 00:03:41.000 2020-01-02 03:32:02.000 221 99122 49671.5 4967150 221 99122 49671.5 4967150 -32348 32587 5107.02 510702 -124 127 2.38 238 +222 101 10212 99123 0.66666 297.66666 149.16666 14916.66666 0.66666 297.66666 149.16666 14916.6667 0.66666 297.66666 149.16665999999998 14916.66600 2020-01-01 2020-01-02 2020-01-01 00:03:42 2020-01-02 03:32:03 2020-01-01 00:03:42.000 2020-01-02 03:32:03.000 222 99123 49672.5 4967250 222 99123 49672.5 4967250 -32347 32588 5108.02 510802 -128 127 0.82 82 +223 101 10213 99124 0.66966 297.66966 149.16966 14916.96696 0.66966 297.66968 149.16966 14916.96678 0.66966 297.66966 149.16966 14916.96600 2020-01-01 2020-01-02 2020-01-01 00:03:43 2020-01-02 03:32:04 2020-01-01 00:03:43.000 2020-01-02 03:32:04.000 223 99124 49673.5 4967350 223 99124 49673.5 4967350 -32346 32589 5109.02 510902 -128 127 -0.74 -74 +224 101 10214 99125 0.67267 297.67267 149.17267 14917.26726 0.67267 297.67267 149.17267 14917.26709 0.67267 297.67267 149.17267 14917.26700 2020-01-01 2020-01-02 2020-01-01 00:03:44 2020-01-02 03:32:05 2020-01-01 00:03:44.000 2020-01-02 03:32:05.000 224 99125 49674.5 4967450 224 99125 49674.5 4967450 -32345 32590 5110.02 511002 -128 124 -2.3 -230 225 101 10215 99126 0.67567 297.67567 149.17567 14917.56756 0.67567 297.6757 149.17567 14917.56767 0.67567 297.67567 149.17567 14917.56700 2020-01-01 2020-01-02 2020-01-01 00:03:45 2020-01-02 03:32:06 2020-01-01 00:03:45.000 2020-01-02 03:32:06.000 225 99126 49675.5 4967550 225 99126 49675.5 4967550 -32344 32591 5111.02 511102 -127 125 -1.3 -130 -226 101 10216 99127 0.67867 297.67867 149.17867 14917.86786 0.67867 297.67868 149.17868 14917.86802 0.67867 297.67867 149.17866999999998 14917.86700 2020-01-01 2020-01-02 2020-01-01 00:03:46 2020-01-02 03:32:07 2020-01-01 00:03:46.000 2020-01-02 03:32:07.000 226 99127 49676.5 4967650 226 99127 49676.5 4967650 -32343 32592 5112.02 511202 -126 126 -0.3 -30 -227 101 10217 99128 0.68168 297.68168 149.18168 14918.16816 0.68168 297.68167 149.18168 14918.16817 0.68168 297.68168 149.18167999999991 14918.16800 2020-01-01 2020-01-02 2020-01-01 00:03:47 2020-01-02 03:32:08 2020-01-01 00:03:47.000 2020-01-02 03:32:08.000 227 99128 49677.5 4967750 227 99128 49677.5 4967750 -32342 32593 5113.02 511302 -125 127 0.7 70 -228 101 10218 99129 0.68468 297.68468 149.18468 14918.46846 0.68468 297.6847 149.18468 14918.46825 0.68468 297.68468 149.18468000000007 14918.46800 2020-01-01 2020-01-02 2020-01-01 00:03:48 2020-01-02 03:32:09 2020-01-01 00:03:48.000 2020-01-02 03:32:09.000 228 99129 49678.5 4967850 228 99129 49678.5 4967850 -32341 32594 5114.02 511402 -128 127 -0.86 -86 -229 101 10219 99130 0.68768 297.68768 149.18768 14918.76876 0.68768 297.68768 149.18768 14918.76855 0.68768 297.68768 149.1876799999999 14918.76800 2020-01-01 2020-01-02 2020-01-01 00:03:49 2020-01-02 03:32:10 2020-01-01 00:03:49.000 2020-01-02 03:32:10.000 229 99130 49679.5 4967950 229 99130 49679.5 4967950 -32340 32595 5115.02 511502 -128 127 -2.42 -242 -23 102 10013 99923 0.06906 300.06906 150.06906 15156.97597 0.06906 300.06906 150.06907 15156.97617 0.06906 300.06906 150.06905999999992 15156.97506 2020-01-01 2020-01-02 2020-01-01 00:00:23 2020-01-02 03:45:23 2020-01-01 00:00:23.000 2020-01-02 03:45:23.000 23 99923 49973 5047273 23 99923 49973 5047273 -32546 32389 4552.009900990099 459753 -128 127 -2.762376237623762 -279 +226 101 10216 99127 0.67867 297.67867 149.17867 14917.86786 0.67867 297.67868 149.17868 14917.86802 0.67867 297.67867 149.17867 14917.86700 2020-01-01 2020-01-02 2020-01-01 00:03:46 2020-01-02 03:32:07 2020-01-01 00:03:46.000 2020-01-02 03:32:07.000 226 99127 49676.5 4967650 226 99127 49676.5 4967650 -32343 32592 5112.02 511202 -126 126 -0.3 -30 +227 101 10217 99128 0.68168 297.68168 149.18168 14918.16816 0.68168 297.68167 149.18168 14918.16817 0.68168 297.68168 149.18168 14918.16800 2020-01-01 2020-01-02 2020-01-01 00:03:47 2020-01-02 03:32:08 2020-01-01 00:03:47.000 2020-01-02 03:32:08.000 227 99128 49677.5 4967750 227 99128 49677.5 4967750 -32342 32593 5113.02 511302 -125 127 0.7 70 +228 101 10218 99129 0.68468 297.68468 149.18468 14918.46846 0.68468 297.6847 149.18468 14918.46825 0.68468 297.68468 149.18468000000001 14918.46800 2020-01-01 2020-01-02 2020-01-01 00:03:48 2020-01-02 03:32:09 2020-01-01 00:03:48.000 2020-01-02 03:32:09.000 228 99129 49678.5 4967850 228 99129 49678.5 4967850 -32341 32594 5114.02 511402 -128 127 -0.86 -86 +229 101 10219 99130 0.68768 297.68768 149.18768 14918.76876 0.68768 297.68768 149.18768 14918.76855 0.68768 297.68768 149.18768 14918.76800 2020-01-01 2020-01-02 2020-01-01 00:03:49 2020-01-02 03:32:10 2020-01-01 00:03:49.000 2020-01-02 03:32:10.000 229 99130 49679.5 4967950 229 99130 49679.5 4967950 -32340 32595 5115.02 511502 -128 127 -2.42 -242 +23 102 10013 99923 0.06906 300.06906 150.06906 15156.97597 0.06906 300.06906 150.06907 15156.97617 0.06906 300.06906 150.06906 15156.97506 2020-01-01 2020-01-02 2020-01-01 00:00:23 2020-01-02 03:45:23 2020-01-01 00:00:23.000 2020-01-02 03:45:23.000 23 99923 49973 5047273 23 99923 49973 5047273 -32546 32389 4552.009900990099 459753 -128 127 -2.762376237623762 -279 230 101 10220 99131 0.69069 297.69069 149.19069 14919.06906 0.69069 297.6907 149.19069 14919.06914 0.69069 297.69069 149.19069 14919.06900 2020-01-01 2020-01-02 2020-01-01 00:03:50 2020-01-02 03:32:11 2020-01-01 00:03:50.000 2020-01-02 03:32:11.000 230 99131 49680.5 4968050 230 99131 49680.5 4968050 -32339 32596 5116.02 511602 -128 123 -3.98 -398 -231 101 10221 99132 0.69369 297.69369 149.19369 14919.36936 0.69369 297.6937 149.19369 14919.36949 0.69369 297.69369 149.19369000000003 14919.36900 2020-01-01 2020-01-02 2020-01-01 00:03:51 2020-01-02 03:32:12 2020-01-01 00:03:51.000 2020-01-02 03:32:12.000 231 99132 49681.5 4968150 231 99132 49681.5 4968150 -32338 32597 5117.02 511702 -127 124 -2.98 -298 -232 101 10222 99133 0.69669 297.69669 149.19669 14919.66966 0.69669 297.6967 149.19669 14919.66964 0.69669 297.69669 149.19669000000002 14919.66900 2020-01-01 2020-01-02 2020-01-01 00:03:52 2020-01-02 03:32:13 2020-01-01 00:03:52.000 2020-01-02 03:32:13.000 232 99133 49682.5 4968250 232 99133 49682.5 4968250 -32337 32598 5118.02 511802 -126 125 -1.98 -198 -233 101 10223 99134 0.69969 297.69969 149.19969 14919.96996 0.69969 297.6997 149.1997 14919.97037 0.69969 297.69969 149.19968999999998 14919.96900 2020-01-01 2020-01-02 2020-01-01 00:03:53 2020-01-02 03:32:14 2020-01-01 00:03:53.000 2020-01-02 03:32:14.000 233 99134 49683.5 4968350 233 99134 49683.5 4968350 -32336 32599 5119.02 511902 -125 126 -0.98 -98 -234 101 10224 99135 0.7027 297.7027 149.2027 14920.27027 0.7027 297.7027 149.2027 14920.27003 0.70270 297.70270 149.2026999999999 14920.27000 2020-01-01 2020-01-02 2020-01-01 00:03:54 2020-01-02 03:32:15 2020-01-01 00:03:54.000 2020-01-02 03:32:15.000 234 99135 49684.5 4968450 234 99135 49684.5 4968450 -32335 32600 5120.02 512002 -124 127 0.02 2 -235 101 10225 99136 0.7057 297.7057 149.2057 14920.57057 0.7057 297.70572 149.2057 14920.57065 0.70570 297.70570 149.20570000000015 14920.57000 2020-01-01 2020-01-02 2020-01-01 00:03:55 2020-01-02 03:32:16 2020-01-01 00:03:55.000 2020-01-02 03:32:16.000 235 99136 49685.5 4968550 235 99136 49685.5 4968550 -32334 32601 5121.02 512102 -128 127 -1.54 -154 -236 101 10226 99137 0.7087 297.7087 149.2087 14920.87087 0.7087 297.7087 149.2087 14920.87095 0.70870 297.70870 149.20869999999996 14920.87000 2020-01-01 2020-01-02 2020-01-01 00:03:56 2020-01-02 03:32:17 2020-01-01 00:03:56.000 2020-01-02 03:32:17.000 236 99137 49686.5 4968650 236 99137 49686.5 4968650 -32333 32602 5122.02 512202 -128 123 -3.1 -310 -237 101 10227 99138 0.71171 297.71171 149.21171 14921.17117 0.71171 297.7117 149.21171 14921.17111 0.71171 297.71171 149.21170999999987 14921.17100 2020-01-01 2020-01-02 2020-01-01 00:03:57 2020-01-02 03:32:18 2020-01-01 00:03:57.000 2020-01-02 03:32:18.000 237 99138 49687.5 4968750 237 99138 49687.5 4968750 -32332 32603 5123.02 512302 -127 124 -2.1 -210 -238 101 10228 99139 0.71471 297.71471 149.21471 14921.47147 0.71471 297.71472 149.21471 14921.47184 0.71471 297.71471 149.21471000000008 14921.47100 2020-01-01 2020-01-02 2020-01-01 00:03:58 2020-01-02 03:32:19 2020-01-01 00:03:58.000 2020-01-02 03:32:19.000 238 99139 49688.5 4968850 238 99139 49688.5 4968850 -32331 32604 5124.02 512402 -126 125 -1.1 -110 -239 101 10229 99140 0.71771 297.71771 149.21771 14921.77177 0.71771 297.7177 149.21771 14921.7715 0.71771 297.71771 149.21771000000007 14921.77100 2020-01-01 2020-01-02 2020-01-01 00:03:59 2020-01-02 03:32:20 2020-01-01 00:03:59.000 2020-01-02 03:32:20.000 239 99140 49689.5 4968950 239 99140 49689.5 4968950 -32330 32605 5125.02 512502 -125 126 -0.1 -10 -24 102 10014 99924 0.07207 300.07207 150.07207 15157.27927 0.07207 300.07208 150.07207 15157.27928 0.07207 300.07207 150.07207000000008 15157.27907 2020-01-01 2020-01-02 2020-01-01 00:00:24 2020-01-02 03:45:24 2020-01-01 00:00:24.000 2020-01-02 03:45:24.000 24 99924 49974 5047374 24 99924 49974 5047374 -32545 32390 4553.009900990099 459854 -128 123 -4.297029702970297 -434 -240 101 10230 99141 0.72072 297.72072 149.22072 14922.07207 0.72072 297.72073 149.22072 14922.07211 0.72072 297.72072 149.22071999999994 14922.07200 2020-01-01 2020-01-02 2020-01-01 00:04:00 2020-01-02 03:32:21 2020-01-01 00:04:00.000 2020-01-02 03:32:21.000 240 99141 49690.5 4969050 240 99141 49690.5 4969050 -32329 32606 5126.02 512602 -124 127 0.9 90 +231 101 10221 99132 0.69369 297.69369 149.19369 14919.36936 0.69369 297.6937 149.19369 14919.36949 0.69369 297.69369 149.19369 14919.36900 2020-01-01 2020-01-02 2020-01-01 00:03:51 2020-01-02 03:32:12 2020-01-01 00:03:51.000 2020-01-02 03:32:12.000 231 99132 49681.5 4968150 231 99132 49681.5 4968150 -32338 32597 5117.02 511702 -127 124 -2.98 -298 +232 101 10222 99133 0.69669 297.69669 149.19669 14919.66966 0.69669 297.6967 149.19669 14919.66964 0.69669 297.69669 149.19669 14919.66900 2020-01-01 2020-01-02 2020-01-01 00:03:52 2020-01-02 03:32:13 2020-01-01 00:03:52.000 2020-01-02 03:32:13.000 232 99133 49682.5 4968250 232 99133 49682.5 4968250 -32337 32598 5118.02 511802 -126 125 -1.98 -198 +233 101 10223 99134 0.69969 297.69969 149.19969 14919.96996 0.69969 297.6997 149.1997 14919.97037 0.69969 297.69969 149.19969 14919.96900 2020-01-01 2020-01-02 2020-01-01 00:03:53 2020-01-02 03:32:14 2020-01-01 00:03:53.000 2020-01-02 03:32:14.000 233 99134 49683.5 4968350 233 99134 49683.5 4968350 -32336 32599 5119.02 511902 -125 126 -0.98 -98 +234 101 10224 99135 0.7027 297.7027 149.2027 14920.27027 0.7027 297.7027 149.2027 14920.27003 0.70270 297.70270 149.2027 14920.27000 2020-01-01 2020-01-02 2020-01-01 00:03:54 2020-01-02 03:32:15 2020-01-01 00:03:54.000 2020-01-02 03:32:15.000 234 99135 49684.5 4968450 234 99135 49684.5 4968450 -32335 32600 5120.02 512002 -124 127 0.02 2 +235 101 10225 99136 0.7057 297.7057 149.2057 14920.57057 0.7057 297.70572 149.2057 14920.57065 0.70570 297.70570 149.2057 14920.57000 2020-01-01 2020-01-02 2020-01-01 00:03:55 2020-01-02 03:32:16 2020-01-01 00:03:55.000 2020-01-02 03:32:16.000 235 99136 49685.5 4968550 235 99136 49685.5 4968550 -32334 32601 5121.02 512102 -128 127 -1.54 -154 +236 101 10226 99137 0.7087 297.7087 149.2087 14920.87087 0.7087 297.7087 149.2087 14920.87095 0.70870 297.70870 149.20870000000002 14920.87000 2020-01-01 2020-01-02 2020-01-01 00:03:56 2020-01-02 03:32:17 2020-01-01 00:03:56.000 2020-01-02 03:32:17.000 236 99137 49686.5 4968650 236 99137 49686.5 4968650 -32333 32602 5122.02 512202 -128 123 -3.1 -310 +237 101 10227 99138 0.71171 297.71171 149.21171 14921.17117 0.71171 297.7117 149.21171 14921.17111 0.71171 297.71171 149.21171 14921.17100 2020-01-01 2020-01-02 2020-01-01 00:03:57 2020-01-02 03:32:18 2020-01-01 00:03:57.000 2020-01-02 03:32:18.000 237 99138 49687.5 4968750 237 99138 49687.5 4968750 -32332 32603 5123.02 512302 -127 124 -2.1 -210 +238 101 10228 99139 0.71471 297.71471 149.21471 14921.47147 0.71471 297.71472 149.21471 14921.47184 0.71471 297.71471 149.21471 14921.47100 2020-01-01 2020-01-02 2020-01-01 00:03:58 2020-01-02 03:32:19 2020-01-01 00:03:58.000 2020-01-02 03:32:19.000 238 99139 49688.5 4968850 238 99139 49688.5 4968850 -32331 32604 5124.02 512402 -126 125 -1.1 -110 +239 101 10229 99140 0.71771 297.71771 149.21771 14921.77177 0.71771 297.7177 149.21771 14921.7715 0.71771 297.71771 149.21771 14921.77100 2020-01-01 2020-01-02 2020-01-01 00:03:59 2020-01-02 03:32:20 2020-01-01 00:03:59.000 2020-01-02 03:32:20.000 239 99140 49689.5 4968950 239 99140 49689.5 4968950 -32330 32605 5125.02 512502 -125 126 -0.1 -10 +24 102 10014 99924 0.07207 300.07207 150.07207 15157.27927 0.07207 300.07208 150.07207 15157.27928 0.07207 300.07207 150.07207 15157.27907 2020-01-01 2020-01-02 2020-01-01 00:00:24 2020-01-02 03:45:24 2020-01-01 00:00:24.000 2020-01-02 03:45:24.000 24 99924 49974 5047374 24 99924 49974 5047374 -32545 32390 4553.009900990099 459854 -128 123 -4.297029702970297 -434 +240 101 10230 99141 0.72072 297.72072 149.22072 14922.07207 0.72072 297.72073 149.22072 14922.07211 0.72072 297.72072 149.22072 14922.07200 2020-01-01 2020-01-02 2020-01-01 00:04:00 2020-01-02 03:32:21 2020-01-01 00:04:00.000 2020-01-02 03:32:21.000 240 99141 49690.5 4969050 240 99141 49690.5 4969050 -32329 32606 5126.02 512602 -124 127 0.9 90 241 101 10231 99142 0.72372 297.72372 149.22372 14922.37237 0.72372 297.72372 149.22372 14922.37243 0.72372 297.72372 149.22372 14922.37200 2020-01-01 2020-01-02 2020-01-01 00:04:01 2020-01-02 03:32:22 2020-01-01 00:04:01.000 2020-01-02 03:32:22.000 241 99142 49691.5 4969150 241 99142 49691.5 4969150 -32328 32607 5127.02 512702 -128 127 -0.66 -66 -242 101 10232 99143 0.72672 297.72672 149.22672 14922.67267 0.72672 297.7267 149.22672 14922.67273 0.72672 297.72672 149.22672000000003 14922.67200 2020-01-01 2020-01-02 2020-01-01 00:04:02 2020-01-02 03:32:23 2020-01-01 00:04:02.000 2020-01-02 03:32:23.000 242 99143 49692.5 4969250 242 99143 49692.5 4969250 -32327 32608 5128.02 512802 -128 123 -2.22 -222 -243 101 10233 99144 0.72972 297.72972 149.22972 14922.97297 0.72972 297.72974 149.22973 14922.97332 0.72972 297.72972 149.22971999999993 14922.97200 2020-01-01 2020-01-02 2020-01-01 00:04:03 2020-01-02 03:32:24 2020-01-01 00:04:03.000 2020-01-02 03:32:24.000 243 99144 49693.5 4969350 243 99144 49693.5 4969350 -32326 32609 5129.02 512902 -127 124 -1.22 -122 -244 101 10234 99145 0.73273 297.73273 149.23273 14923.27327 0.73273 297.73273 149.23272 14923.27297 0.73273 297.73273 149.23272999999995 14923.27300 2020-01-01 2020-01-02 2020-01-01 00:04:04 2020-01-02 03:32:25 2020-01-01 00:04:04.000 2020-01-02 03:32:25.000 244 99145 49694.5 4969450 244 99145 49694.5 4969450 -32325 32610 5130.02 513002 -126 125 -0.22 -22 -245 101 10235 99146 0.73573 297.73573 149.23573 14923.57357 0.73573 297.73575 149.23573 14923.57358 0.73573 297.73573 149.2357300000001 14923.57300 2020-01-01 2020-01-02 2020-01-01 00:04:05 2020-01-02 03:32:26 2020-01-01 00:04:05.000 2020-01-02 03:32:26.000 245 99146 49695.5 4969550 245 99146 49695.5 4969550 -32324 32611 5131.02 513102 -125 126 0.78 78 -246 101 10236 99147 0.73873 297.73873 149.23873 14923.87387 0.73873 297.73874 149.23873 14923.8739 0.73873 297.73873 149.23872999999992 14923.87300 2020-01-01 2020-01-02 2020-01-01 00:04:06 2020-01-02 03:32:27 2020-01-01 00:04:06.000 2020-01-02 03:32:27.000 246 99147 49696.5 4969650 246 99147 49696.5 4969650 -32323 32612 5132.02 513202 -124 127 1.78 178 -247 101 10237 99148 0.74174 297.74174 149.24174 14924.17417 0.74174 297.74173 149.24174 14924.1742 0.74174 297.74174 149.24174 14924.17400 2020-01-01 2020-01-02 2020-01-01 00:04:07 2020-01-02 03:32:28 2020-01-01 00:04:07.000 2020-01-02 03:32:28.000 247 99148 49697.5 4969750 247 99148 49697.5 4969750 -32322 32613 5133.02 513302 -128 127 0.22 22 -248 101 10238 99149 0.74474 297.74474 149.24474 14924.47447 0.74474 297.74475 149.24474 14924.47478 0.74474 297.74474 149.24474000000006 14924.47400 2020-01-01 2020-01-02 2020-01-01 00:04:08 2020-01-02 03:32:29 2020-01-01 00:04:08.000 2020-01-02 03:32:29.000 248 99149 49698.5 4969850 248 99149 49698.5 4969850 -32321 32614 5134.02 513402 -128 127 -1.34 -134 -249 101 10239 99150 0.74774 297.74774 149.24774 14924.77477 0.74774 297.74774 149.24774 14924.77447 0.74774 297.74774 149.24774000000002 14924.77400 2020-01-01 2020-01-02 2020-01-01 00:04:09 2020-01-02 03:32:30 2020-01-01 00:04:09.000 2020-01-02 03:32:30.000 249 99150 49699.5 4969950 249 99150 49699.5 4969950 -32320 32615 5135.02 513502 -128 124 -2.9 -290 -25 102 10015 99925 0.07507 300.07507 150.07507 15157.58258 0.07507 300.07507 150.07507 15157.58241 0.07507 300.07507 150.07507000000004 15157.58207 2020-01-01 2020-01-02 2020-01-01 00:00:25 2020-01-02 03:45:25 2020-01-01 00:00:25.000 2020-01-02 03:45:25.000 25 99925 49975 5047475 25 99925 49975 5047475 -32544 32391 4554.009900990099 459955 -127 124 -3.297029702970297 -333 -250 101 10240 99151 0.75075 297.75075 149.25075 14925.07507 0.75075 297.75076 149.25075 14925.07506 0.75075 297.75075 149.25074999999998 14925.07500 2020-01-01 2020-01-02 2020-01-01 00:04:10 2020-01-02 03:32:31 2020-01-01 00:04:10.000 2020-01-02 03:32:31.000 250 99151 49700.5 4970050 250 99151 49700.5 4970050 -32319 32616 5136.02 513602 -127 125 -1.9 -190 -251 101 10241 99152 0.75375 297.75375 149.25375 14925.37537 0.75375 297.75375 149.25375 14925.37536 0.75375 297.75375 149.25374999999994 14925.37500 2020-01-01 2020-01-02 2020-01-01 00:04:11 2020-01-02 03:32:32 2020-01-01 00:04:11.000 2020-01-02 03:32:32.000 251 99152 49701.5 4970150 251 99152 49701.5 4970150 -32318 32617 5137.02 513702 -126 126 -0.9 -90 -252 101 10242 99153 0.75675 297.75675 149.25675 14925.67567 0.75675 297.75674 149.25675 14925.67567 0.75675 297.75675 149.25675000000015 14925.67500 2020-01-01 2020-01-02 2020-01-01 00:04:12 2020-01-02 03:32:33 2020-01-01 00:04:12.000 2020-01-02 03:32:33.000 252 99153 49702.5 4970250 252 99153 49702.5 4970250 -32317 32618 5138.02 513802 -125 127 0.1 10 -253 101 10243 99154 0.75975 297.75975 149.25975 14925.97597 0.75975 297.75977 149.25976 14925.97625 0.75975 297.75975 149.25974999999997 14925.97500 2020-01-01 2020-01-02 2020-01-01 00:04:13 2020-01-02 03:32:34 2020-01-01 00:04:13.000 2020-01-02 03:32:34.000 253 99154 49703.5 4970350 253 99154 49703.5 4970350 -32316 32619 5139.02 513902 -128 127 -1.46 -146 -254 101 10244 99155 0.76276 297.76276 149.26276 14926.27627 0.76276 297.76276 149.26275 14926.27594 0.76276 297.76276 149.2627599999999 14926.27600 2020-01-01 2020-01-02 2020-01-01 00:04:14 2020-01-02 03:32:35 2020-01-01 00:04:14.000 2020-01-02 03:32:35.000 254 99155 49704.5 4970450 254 99155 49704.5 4970450 -32315 32620 5140.02 514002 -128 127 -3.02 -302 -255 101 10245 99156 0.76576 297.76576 149.26576 14926.57657 0.76576 297.76578 149.26576 14926.57652 0.76576 297.76576 149.2657600000001 14926.57600 2020-01-01 2020-01-02 2020-01-01 00:04:15 2020-01-02 03:32:36 2020-01-01 00:04:15.000 2020-01-02 03:32:36.000 255 99156 49705.5 4970550 255 99156 49705.5 4970550 -32314 32621 5141.02 514102 -128 123 -4.58 -458 -256 101 10246 99157 0.76876 297.76876 149.26876 14926.87687 0.76876 297.76877 149.26876 14926.87683 0.76876 297.76876 149.26876000000007 14926.87600 2020-01-01 2020-01-02 2020-01-01 00:04:16 2020-01-02 03:32:37 2020-01-01 00:04:16.000 2020-01-02 03:32:37.000 256 99157 49706.5 4970650 256 99157 49706.5 4970650 -32313 32622 5142.02 514202 -127 124 -3.58 -358 -257 101 10247 99158 0.77177 297.77177 149.27177 14927.17717 0.77177 297.77176 149.27177 14927.17714 0.77177 297.77177 149.27176999999995 14927.17700 2020-01-01 2020-01-02 2020-01-01 00:04:17 2020-01-02 03:32:38 2020-01-01 00:04:17.000 2020-01-02 03:32:38.000 257 99158 49707.5 4970750 257 99158 49707.5 4970750 -32312 32623 5143.02 514302 -126 125 -2.58 -258 -258 101 10248 99159 0.77477 297.77477 149.27477 14927.47747 0.77477 297.77478 149.27477 14927.47776 0.77477 297.77477 149.27477 14927.47700 2020-01-01 2020-01-02 2020-01-01 00:04:18 2020-01-02 03:32:39 2020-01-01 00:04:18.000 2020-01-02 03:32:39.000 258 99159 49708.5 4970850 258 99159 49708.5 4970850 -32311 32624 5144.02 514402 -125 126 -1.58 -158 -259 101 10249 99160 0.77777 297.77777 149.27777 14927.77777 0.77777 297.77777 149.27777 14927.77742 0.77777 297.77777 149.27777000000003 14927.77700 2020-01-01 2020-01-02 2020-01-01 00:04:19 2020-01-02 03:32:40 2020-01-01 00:04:19.000 2020-01-02 03:32:40.000 259 99160 49709.5 4970950 259 99160 49709.5 4970950 -32310 32625 5145.02 514502 -124 127 -0.58 -58 -26 102 10016 99926 0.07807 300.07807 150.07807 15157.88588 0.07807 300.07806 150.07807 15157.88575 0.07807 300.07807 150.07806999999985 15157.88507 2020-01-01 2020-01-02 2020-01-01 00:00:26 2020-01-02 03:45:26 2020-01-01 00:00:26.000 2020-01-02 03:45:26.000 26 99926 49976 5047576 26 99926 49976 5047576 -32543 32392 4555.009900990099 460056 -126 125 -2.297029702970297 -232 -260 101 10250 99161 0.78078 297.78078 149.28078 14928.07807 0.78078 297.7808 149.28077 14928.07799 0.78078 297.78078 149.2807799999999 14928.07800 2020-01-01 2020-01-02 2020-01-01 00:04:20 2020-01-02 03:32:41 2020-01-01 00:04:20.000 2020-01-02 03:32:41.000 260 99161 49710.5 4971050 260 99161 49710.5 4971050 -32309 32626 5146.02 514602 -128 127 -2.14 -214 -261 101 10251 99162 0.78378 297.78378 149.28378 14928.37837 0.78378 297.78378 149.28378 14928.3783 0.78378 297.78378 149.28377999999992 14928.37800 2020-01-01 2020-01-02 2020-01-01 00:04:21 2020-01-02 03:32:42 2020-01-01 00:04:21.000 2020-01-02 03:32:42.000 261 99162 49711.5 4971150 261 99162 49711.5 4971150 -32308 32627 5147.02 514702 -128 123 -3.7 -370 -262 101 10252 99163 0.78678 297.78678 149.28678 14928.67867 0.78678 297.78677 149.28678 14928.67861 0.78678 297.78678 149.28678000000008 14928.67800 2020-01-01 2020-01-02 2020-01-01 00:04:22 2020-01-02 03:32:43 2020-01-01 00:04:22.000 2020-01-02 03:32:43.000 262 99163 49712.5 4971250 262 99163 49712.5 4971250 -32307 32628 5148.02 514802 -127 124 -2.7 -270 -263 101 10253 99164 0.78978 297.78978 149.28978 14928.97897 0.78978 297.7898 149.28979 14928.97923 0.78978 297.78978 149.28977999999992 14928.97800 2020-01-01 2020-01-02 2020-01-01 00:04:23 2020-01-02 03:32:44 2020-01-01 00:04:23.000 2020-01-02 03:32:44.000 263 99164 49713.5 4971350 263 99164 49713.5 4971350 -32306 32629 5149.02 514902 -126 125 -1.7 -170 -264 101 10254 99165 0.79279 297.79279 149.29279 14929.27927 0.79279 297.7928 149.29278 14929.27888 0.79279 297.79279 149.29279000000002 14929.27900 2020-01-01 2020-01-02 2020-01-01 00:04:24 2020-01-02 03:32:45 2020-01-01 00:04:24.000 2020-01-02 03:32:45.000 264 99165 49714.5 4971450 264 99165 49714.5 4971450 -32305 32630 5150.02 515002 -125 126 -0.7 -70 -265 101 10255 99166 0.79579 297.79579 149.29579 14929.57957 0.79579 297.7958 149.29579 14929.57962 0.79579 297.79579 149.29579000000007 14929.57900 2020-01-01 2020-01-02 2020-01-01 00:04:25 2020-01-02 03:32:46 2020-01-01 00:04:25.000 2020-01-02 03:32:46.000 265 99166 49715.5 4971550 265 99166 49715.5 4971550 -32304 32631 5151.02 515102 -124 127 0.3 30 -266 101 10256 99167 0.79879 297.79879 149.29879 14929.87987 0.79879 297.7988 149.29879 14929.87977 0.79879 297.79879 149.29879000000003 14929.87900 2020-01-01 2020-01-02 2020-01-01 00:04:26 2020-01-02 03:32:47 2020-01-01 00:04:26.000 2020-01-02 03:32:47.000 266 99167 49716.5 4971650 266 99167 49716.5 4971650 -32303 32632 5152.02 515202 -128 127 -1.26 -126 -267 101 10257 99168 0.8018 297.8018 149.3018 14930.18018 0.8018 297.8018 149.3018 14930.18012 0.80180 297.80180 149.3018 14930.18000 2020-01-01 2020-01-02 2020-01-01 00:04:27 2020-01-02 03:32:48 2020-01-01 00:04:27.000 2020-01-02 03:32:48.000 267 99168 49717.5 4971750 267 99168 49717.5 4971750 -32302 32633 5153.02 515302 -128 123 -2.82 -282 -268 101 10258 99169 0.8048 297.8048 149.3048 14930.48048 0.8048 297.8048 149.3048 14930.4807 0.80480 297.80480 149.30479999999994 14930.48000 2020-01-01 2020-01-02 2020-01-01 00:04:28 2020-01-02 03:32:49 2020-01-01 00:04:28.000 2020-01-02 03:32:49.000 268 99169 49718.5 4971850 268 99169 49718.5 4971850 -32301 32634 5154.02 515402 -127 124 -1.82 -182 +242 101 10232 99143 0.72672 297.72672 149.22672 14922.67267 0.72672 297.7267 149.22672 14922.67273 0.72672 297.72672 149.22672 14922.67200 2020-01-01 2020-01-02 2020-01-01 00:04:02 2020-01-02 03:32:23 2020-01-01 00:04:02.000 2020-01-02 03:32:23.000 242 99143 49692.5 4969250 242 99143 49692.5 4969250 -32327 32608 5128.02 512802 -128 123 -2.22 -222 +243 101 10233 99144 0.72972 297.72972 149.22972 14922.97297 0.72972 297.72974 149.22973 14922.97332 0.72972 297.72972 149.22972 14922.97200 2020-01-01 2020-01-02 2020-01-01 00:04:03 2020-01-02 03:32:24 2020-01-01 00:04:03.000 2020-01-02 03:32:24.000 243 99144 49693.5 4969350 243 99144 49693.5 4969350 -32326 32609 5129.02 512902 -127 124 -1.22 -122 +244 101 10234 99145 0.73273 297.73273 149.23273 14923.27327 0.73273 297.73273 149.23272 14923.27297 0.73273 297.73273 149.23273 14923.27300 2020-01-01 2020-01-02 2020-01-01 00:04:04 2020-01-02 03:32:25 2020-01-01 00:04:04.000 2020-01-02 03:32:25.000 244 99145 49694.5 4969450 244 99145 49694.5 4969450 -32325 32610 5130.02 513002 -126 125 -0.22 -22 +245 101 10235 99146 0.73573 297.73573 149.23573 14923.57357 0.73573 297.73575 149.23573 14923.57358 0.73573 297.73573 149.23573 14923.57300 2020-01-01 2020-01-02 2020-01-01 00:04:05 2020-01-02 03:32:26 2020-01-01 00:04:05.000 2020-01-02 03:32:26.000 245 99146 49695.5 4969550 245 99146 49695.5 4969550 -32324 32611 5131.02 513102 -125 126 0.78 78 +246 101 10236 99147 0.73873 297.73873 149.23873 14923.87387 0.73873 297.73874 149.23873 14923.8739 0.73873 297.73873 149.23873 14923.87300 2020-01-01 2020-01-02 2020-01-01 00:04:06 2020-01-02 03:32:27 2020-01-01 00:04:06.000 2020-01-02 03:32:27.000 246 99147 49696.5 4969650 246 99147 49696.5 4969650 -32323 32612 5132.02 513202 -124 127 1.78 178 +247 101 10237 99148 0.74174 297.74174 149.24174 14924.17417 0.74174 297.74173 149.24174 14924.1742 0.74174 297.74174 149.24174000000002 14924.17400 2020-01-01 2020-01-02 2020-01-01 00:04:07 2020-01-02 03:32:28 2020-01-01 00:04:07.000 2020-01-02 03:32:28.000 247 99148 49697.5 4969750 247 99148 49697.5 4969750 -32322 32613 5133.02 513302 -128 127 0.22 22 +248 101 10238 99149 0.74474 297.74474 149.24474 14924.47447 0.74474 297.74475 149.24474 14924.47478 0.74474 297.74474 149.24474 14924.47400 2020-01-01 2020-01-02 2020-01-01 00:04:08 2020-01-02 03:32:29 2020-01-01 00:04:08.000 2020-01-02 03:32:29.000 248 99149 49698.5 4969850 248 99149 49698.5 4969850 -32321 32614 5134.02 513402 -128 127 -1.34 -134 +249 101 10239 99150 0.74774 297.74774 149.24774 14924.77477 0.74774 297.74774 149.24774 14924.77447 0.74774 297.74774 149.24774 14924.77400 2020-01-01 2020-01-02 2020-01-01 00:04:09 2020-01-02 03:32:30 2020-01-01 00:04:09.000 2020-01-02 03:32:30.000 249 99150 49699.5 4969950 249 99150 49699.5 4969950 -32320 32615 5135.02 513502 -128 124 -2.9 -290 +25 102 10015 99925 0.07507 300.07507 150.07507 15157.58258 0.07507 300.07507 150.07507 15157.58241 0.07507 300.07507 150.07507 15157.58207 2020-01-01 2020-01-02 2020-01-01 00:00:25 2020-01-02 03:45:25 2020-01-01 00:00:25.000 2020-01-02 03:45:25.000 25 99925 49975 5047475 25 99925 49975 5047475 -32544 32391 4554.009900990099 459955 -127 124 -3.297029702970297 -333 +250 101 10240 99151 0.75075 297.75075 149.25075 14925.07507 0.75075 297.75076 149.25075 14925.07506 0.75075 297.75075 149.25075 14925.07500 2020-01-01 2020-01-02 2020-01-01 00:04:10 2020-01-02 03:32:31 2020-01-01 00:04:10.000 2020-01-02 03:32:31.000 250 99151 49700.5 4970050 250 99151 49700.5 4970050 -32319 32616 5136.02 513602 -127 125 -1.9 -190 +251 101 10241 99152 0.75375 297.75375 149.25375 14925.37537 0.75375 297.75375 149.25375 14925.37536 0.75375 297.75375 149.25375 14925.37500 2020-01-01 2020-01-02 2020-01-01 00:04:11 2020-01-02 03:32:32 2020-01-01 00:04:11.000 2020-01-02 03:32:32.000 251 99152 49701.5 4970150 251 99152 49701.5 4970150 -32318 32617 5137.02 513702 -126 126 -0.9 -90 +252 101 10242 99153 0.75675 297.75675 149.25675 14925.67567 0.75675 297.75674 149.25675 14925.67567 0.75675 297.75675 149.25674999999998 14925.67500 2020-01-01 2020-01-02 2020-01-01 00:04:12 2020-01-02 03:32:33 2020-01-01 00:04:12.000 2020-01-02 03:32:33.000 252 99153 49702.5 4970250 252 99153 49702.5 4970250 -32317 32618 5138.02 513802 -125 127 0.1 10 +253 101 10243 99154 0.75975 297.75975 149.25975 14925.97597 0.75975 297.75977 149.25976 14925.97625 0.75975 297.75975 149.25975 14925.97500 2020-01-01 2020-01-02 2020-01-01 00:04:13 2020-01-02 03:32:34 2020-01-01 00:04:13.000 2020-01-02 03:32:34.000 253 99154 49703.5 4970350 253 99154 49703.5 4970350 -32316 32619 5139.02 513902 -128 127 -1.46 -146 +254 101 10244 99155 0.76276 297.76276 149.26276 14926.27627 0.76276 297.76276 149.26275 14926.27594 0.76276 297.76276 149.26276 14926.27600 2020-01-01 2020-01-02 2020-01-01 00:04:14 2020-01-02 03:32:35 2020-01-01 00:04:14.000 2020-01-02 03:32:35.000 254 99155 49704.5 4970450 254 99155 49704.5 4970450 -32315 32620 5140.02 514002 -128 127 -3.02 -302 +255 101 10245 99156 0.76576 297.76576 149.26576 14926.57657 0.76576 297.76578 149.26576 14926.57652 0.76576 297.76576 149.26576 14926.57600 2020-01-01 2020-01-02 2020-01-01 00:04:15 2020-01-02 03:32:36 2020-01-01 00:04:15.000 2020-01-02 03:32:36.000 255 99156 49705.5 4970550 255 99156 49705.5 4970550 -32314 32621 5141.02 514102 -128 123 -4.58 -458 +256 101 10246 99157 0.76876 297.76876 149.26876 14926.87687 0.76876 297.76877 149.26876 14926.87683 0.76876 297.76876 149.26876000000001 14926.87600 2020-01-01 2020-01-02 2020-01-01 00:04:16 2020-01-02 03:32:37 2020-01-01 00:04:16.000 2020-01-02 03:32:37.000 256 99157 49706.5 4970650 256 99157 49706.5 4970650 -32313 32622 5142.02 514202 -127 124 -3.58 -358 +257 101 10247 99158 0.77177 297.77177 149.27177 14927.17717 0.77177 297.77176 149.27177 14927.17714 0.77177 297.77177 149.27177 14927.17700 2020-01-01 2020-01-02 2020-01-01 00:04:17 2020-01-02 03:32:38 2020-01-01 00:04:17.000 2020-01-02 03:32:38.000 257 99158 49707.5 4970750 257 99158 49707.5 4970750 -32312 32623 5143.02 514302 -126 125 -2.58 -258 +258 101 10248 99159 0.77477 297.77477 149.27477 14927.47747 0.77477 297.77478 149.27477 14927.47776 0.77477 297.77477 149.27477000000002 14927.47700 2020-01-01 2020-01-02 2020-01-01 00:04:18 2020-01-02 03:32:39 2020-01-01 00:04:18.000 2020-01-02 03:32:39.000 258 99159 49708.5 4970850 258 99159 49708.5 4970850 -32311 32624 5144.02 514402 -125 126 -1.58 -158 +259 101 10249 99160 0.77777 297.77777 149.27777 14927.77777 0.77777 297.77777 149.27777 14927.77742 0.77777 297.77777 149.27777 14927.77700 2020-01-01 2020-01-02 2020-01-01 00:04:19 2020-01-02 03:32:40 2020-01-01 00:04:19.000 2020-01-02 03:32:40.000 259 99160 49709.5 4970950 259 99160 49709.5 4970950 -32310 32625 5145.02 514502 -124 127 -0.58 -58 +26 102 10016 99926 0.07807 300.07807 150.07807 15157.88588 0.07807 300.07806 150.07807 15157.88575 0.07807 300.07807 150.07807 15157.88507 2020-01-01 2020-01-02 2020-01-01 00:00:26 2020-01-02 03:45:26 2020-01-01 00:00:26.000 2020-01-02 03:45:26.000 26 99926 49976 5047576 26 99926 49976 5047576 -32543 32392 4555.009900990099 460056 -126 125 -2.297029702970297 -232 +260 101 10250 99161 0.78078 297.78078 149.28078 14928.07807 0.78078 297.7808 149.28077 14928.07799 0.78078 297.78078 149.28078 14928.07800 2020-01-01 2020-01-02 2020-01-01 00:04:20 2020-01-02 03:32:41 2020-01-01 00:04:20.000 2020-01-02 03:32:41.000 260 99161 49710.5 4971050 260 99161 49710.5 4971050 -32309 32626 5146.02 514602 -128 127 -2.14 -214 +261 101 10251 99162 0.78378 297.78378 149.28378 14928.37837 0.78378 297.78378 149.28378 14928.3783 0.78378 297.78378 149.28378 14928.37800 2020-01-01 2020-01-02 2020-01-01 00:04:21 2020-01-02 03:32:42 2020-01-01 00:04:21.000 2020-01-02 03:32:42.000 261 99162 49711.5 4971150 261 99162 49711.5 4971150 -32308 32627 5147.02 514702 -128 123 -3.7 -370 +262 101 10252 99163 0.78678 297.78678 149.28678 14928.67867 0.78678 297.78677 149.28678 14928.67861 0.78678 297.78678 149.28678 14928.67800 2020-01-01 2020-01-02 2020-01-01 00:04:22 2020-01-02 03:32:43 2020-01-01 00:04:22.000 2020-01-02 03:32:43.000 262 99163 49712.5 4971250 262 99163 49712.5 4971250 -32307 32628 5148.02 514802 -127 124 -2.7 -270 +263 101 10253 99164 0.78978 297.78978 149.28978 14928.97897 0.78978 297.7898 149.28979 14928.97923 0.78978 297.78978 149.28977999999998 14928.97800 2020-01-01 2020-01-02 2020-01-01 00:04:23 2020-01-02 03:32:44 2020-01-01 00:04:23.000 2020-01-02 03:32:44.000 263 99164 49713.5 4971350 263 99164 49713.5 4971350 -32306 32629 5149.02 514902 -126 125 -1.7 -170 +264 101 10254 99165 0.79279 297.79279 149.29279 14929.27927 0.79279 297.7928 149.29278 14929.27888 0.79279 297.79279 149.29279 14929.27900 2020-01-01 2020-01-02 2020-01-01 00:04:24 2020-01-02 03:32:45 2020-01-01 00:04:24.000 2020-01-02 03:32:45.000 264 99165 49714.5 4971450 264 99165 49714.5 4971450 -32305 32630 5150.02 515002 -125 126 -0.7 -70 +265 101 10255 99166 0.79579 297.79579 149.29579 14929.57957 0.79579 297.7958 149.29579 14929.57962 0.79579 297.79579 149.29579 14929.57900 2020-01-01 2020-01-02 2020-01-01 00:04:25 2020-01-02 03:32:46 2020-01-01 00:04:25.000 2020-01-02 03:32:46.000 265 99166 49715.5 4971550 265 99166 49715.5 4971550 -32304 32631 5151.02 515102 -124 127 0.3 30 +266 101 10256 99167 0.79879 297.79879 149.29879 14929.87987 0.79879 297.7988 149.29879 14929.87977 0.79879 297.79879 149.29879 14929.87900 2020-01-01 2020-01-02 2020-01-01 00:04:26 2020-01-02 03:32:47 2020-01-01 00:04:26.000 2020-01-02 03:32:47.000 266 99167 49716.5 4971650 266 99167 49716.5 4971650 -32303 32632 5152.02 515202 -128 127 -1.26 -126 +267 101 10257 99168 0.8018 297.8018 149.3018 14930.18018 0.8018 297.8018 149.3018 14930.18012 0.80180 297.80180 149.30180000000001 14930.18000 2020-01-01 2020-01-02 2020-01-01 00:04:27 2020-01-02 03:32:48 2020-01-01 00:04:27.000 2020-01-02 03:32:48.000 267 99168 49717.5 4971750 267 99168 49717.5 4971750 -32302 32633 5153.02 515302 -128 123 -2.82 -282 +268 101 10258 99169 0.8048 297.8048 149.3048 14930.48048 0.8048 297.8048 149.3048 14930.4807 0.80480 297.80480 149.3048 14930.48000 2020-01-01 2020-01-02 2020-01-01 00:04:28 2020-01-02 03:32:49 2020-01-01 00:04:28.000 2020-01-02 03:32:49.000 268 99169 49718.5 4971850 268 99169 49718.5 4971850 -32301 32634 5154.02 515402 -127 124 -1.82 -182 269 101 10259 99170 0.8078 297.8078 149.3078 14930.78078 0.8078 297.8078 149.3078 14930.78035 0.80780 297.80780 149.30780000000001 14930.78000 2020-01-01 2020-01-02 2020-01-01 00:04:29 2020-01-02 03:32:50 2020-01-01 00:04:29.000 2020-01-02 03:32:50.000 269 99170 49719.5 4971950 269 99170 49719.5 4971950 -32300 32635 5155.02 515502 -126 125 -0.82 -82 -27 102 10017 99927 0.08108 300.08108 150.08108 15158.18918 0.08108 300.0811 150.08108 15158.18936 0.08108 300.08108 150.08107999999996 15158.18908 2020-01-01 2020-01-02 2020-01-01 00:00:27 2020-01-02 03:45:27 2020-01-01 00:00:27.000 2020-01-02 03:45:27.000 27 99927 49977 5047677 27 99927 49977 5047677 -32542 32393 4556.009900990099 460157 -125 126 -1.297029702970297 -131 -270 101 10260 99171 0.81081 297.81081 149.31081 14931.08108 0.81081 297.81082 149.31081 14931.08109 0.81081 297.81081 149.31081000000023 14931.08100 2020-01-01 2020-01-02 2020-01-01 00:04:30 2020-01-02 03:32:51 2020-01-01 00:04:30.000 2020-01-02 03:32:51.000 270 99171 49720.5 4972050 270 99171 49720.5 4972050 -32299 32636 5156.02 515602 -125 126 0.18 18 -271 101 10261 99172 0.81381 297.81381 149.31381 14931.38138 0.81381 297.8138 149.31381 14931.38124 0.81381 297.81381 149.3138099999998 14931.38100 2020-01-01 2020-01-02 2020-01-01 00:04:31 2020-01-02 03:32:52 2020-01-01 00:04:31.000 2020-01-02 03:32:52.000 271 99172 49721.5 4972150 271 99172 49721.5 4972150 -32298 32637 5157.02 515702 -124 127 1.18 118 -272 101 10262 99173 0.81681 297.81681 149.31681 14931.68168 0.81681 297.8168 149.31681 14931.68159 0.81681 297.81681 149.31681000000015 14931.68100 2020-01-01 2020-01-02 2020-01-01 00:04:32 2020-01-02 03:32:53 2020-01-01 00:04:32.000 2020-01-02 03:32:53.000 272 99173 49722.5 4972250 272 99173 49722.5 4972250 -32297 32638 5158.02 515802 -128 127 -0.38 -38 -273 101 10263 99174 0.81981 297.81981 149.31981 14931.98198 0.81981 297.81982 149.31982 14931.98217 0.81981 297.81981 149.31981000000025 14931.98100 2020-01-01 2020-01-02 2020-01-01 00:04:33 2020-01-02 03:32:54 2020-01-01 00:04:33.000 2020-01-02 03:32:54.000 273 99174 49723.5 4972350 273 99174 49723.5 4972350 -32296 32639 5159.02 515902 -128 127 -1.94 -194 -274 101 10264 99175 0.82282 297.82282 149.32282 14932.28228 0.82282 297.8228 149.32282 14932.28247 0.82282 297.82282 149.3228199999999 14932.28200 2020-01-01 2020-01-02 2020-01-01 00:04:34 2020-01-02 03:32:55 2020-01-01 00:04:34.000 2020-01-02 03:32:55.000 274 99175 49724.5 4972450 274 99175 49724.5 4972450 -32295 32640 5160.02 516002 -128 124 -3.5 -350 -275 101 10265 99176 0.82582 297.82582 149.32582 14932.58258 0.82582 297.82584 149.32582 14932.58256 0.82582 297.82582 149.32582000000002 14932.58200 2020-01-01 2020-01-02 2020-01-01 00:04:35 2020-01-02 03:32:56 2020-01-01 00:04:35.000 2020-01-02 03:32:56.000 275 99176 49725.5 4972550 275 99176 49725.5 4972550 -32294 32641 5161.02 516102 -127 125 -2.5 -250 -276 101 10266 99177 0.82882 297.82882 149.32882 14932.88288 0.82882 297.82883 149.32882 14932.88275 0.82882 297.82882 149.32882000000015 14932.88200 2020-01-01 2020-01-02 2020-01-01 00:04:36 2020-01-02 03:32:57 2020-01-01 00:04:36.000 2020-01-02 03:32:57.000 276 99177 49726.5 4972650 276 99177 49726.5 4972650 -32293 32642 5162.02 516202 -126 126 -1.5 -150 -277 101 10267 99178 0.83183 297.83183 149.33183 14933.18318 0.83183 297.83182 149.33183 14933.18305 0.83183 297.83183 149.3318299999998 14933.18300 2020-01-01 2020-01-02 2020-01-01 00:04:37 2020-01-02 03:32:58 2020-01-01 00:04:37.000 2020-01-02 03:32:58.000 277 99178 49727.5 4972750 277 99178 49727.5 4972750 -32292 32643 5163.02 516302 -125 127 -0.5 -50 -278 101 10268 99179 0.83483 297.83483 149.33483 14933.48348 0.83483 297.83484 149.33483 14933.48364 0.83483 297.83483 149.3348299999998 14933.48300 2020-01-01 2020-01-02 2020-01-01 00:04:38 2020-01-02 03:32:59 2020-01-01 00:04:38.000 2020-01-02 03:32:59.000 278 99179 49728.5 4972850 278 99179 49728.5 4972850 -32291 32644 5164.02 516402 -128 127 -2.06 -206 -279 101 10269 99180 0.83783 297.83783 149.33783 14933.78378 0.83783 297.83783 149.33783 14933.78394 0.83783 297.83783 149.33783000000025 14933.78300 2020-01-01 2020-01-02 2020-01-01 00:04:39 2020-01-02 03:33:00 2020-01-01 00:04:39.000 2020-01-02 03:33:00.000 279 99180 49729.5 4972950 279 99180 49729.5 4972950 -32290 32645 5165.02 516502 -128 127 -3.62 -362 -28 102 10018 99928 0.08408 300.08408 150.08408 15158.49249 0.08408 300.08408 150.08408 15158.49265 0.08408 300.08408 150.08408000000003 15158.49208 2020-01-01 2020-01-02 2020-01-01 00:00:28 2020-01-02 03:45:28 2020-01-01 00:00:28.000 2020-01-02 03:45:28.000 28 99928 49978 5047778 28 99928 49978 5047778 -32541 32394 4557.009900990099 460258 -124 127 -0.297029702970297 -30 -280 101 10270 99181 0.84084 297.84084 149.34084 14934.08408 0.84084 297.84085 149.34084 14934.08403 0.84084 297.84084 149.34084000000016 14934.08400 2020-01-01 2020-01-02 2020-01-01 00:04:40 2020-01-02 03:33:01 2020-01-01 00:04:40.000 2020-01-02 03:33:01.000 280 99181 49730.5 4973050 280 99181 49730.5 4973050 -32289 32646 5166.02 516602 -128 123 -5.18 -518 -281 101 10271 99182 0.84384 297.84384 149.34384 14934.38438 0.84384 297.84384 149.34384 14934.38421 0.84384 297.84384 149.3438399999997 14934.38400 2020-01-01 2020-01-02 2020-01-01 00:04:41 2020-01-02 03:33:02 2020-01-01 00:04:41.000 2020-01-02 03:33:02.000 281 99182 49731.5 4973150 281 99182 49731.5 4973150 -32288 32647 5167.02 516702 -127 124 -4.18 -418 -282 101 10272 99183 0.84684 297.84684 149.34684 14934.68468 0.84684 297.84683 149.34684 14934.68453 0.84684 297.84684 149.34684000000013 14934.68400 2020-01-01 2020-01-02 2020-01-01 00:04:42 2020-01-02 03:33:03 2020-01-01 00:04:42.000 2020-01-02 03:33:03.000 282 99183 49732.5 4973250 282 99183 49732.5 4973250 -32287 32648 5168.02 516802 -126 125 -3.18 -318 -283 101 10273 99184 0.84984 297.84984 149.34984 14934.98498 0.84984 297.84985 149.34985 14934.98526 0.84984 297.84984 149.34984000000014 14934.98400 2020-01-01 2020-01-02 2020-01-01 00:04:43 2020-01-02 03:33:04 2020-01-01 00:04:43.000 2020-01-02 03:33:04.000 283 99184 49733.5 4973350 283 99184 49733.5 4973350 -32286 32649 5169.02 516902 -125 126 -2.18 -218 -284 101 10274 99185 0.85285 297.85285 149.35285 14935.28528 0.85285 297.85284 149.35285 14935.28542 0.85285 297.85285 149.3528499999999 14935.28500 2020-01-01 2020-01-02 2020-01-01 00:04:44 2020-01-02 03:33:05 2020-01-01 00:04:44.000 2020-01-02 03:33:05.000 284 99185 49734.5 4973450 284 99185 49734.5 4973450 -32285 32650 5170.02 517002 -124 127 -1.18 -118 -285 101 10275 99186 0.85585 297.85585 149.35585 14935.58558 0.85585 297.85587 149.35585 14935.5855 0.85585 297.85585 149.35584999999995 14935.58500 2020-01-01 2020-01-02 2020-01-01 00:04:45 2020-01-02 03:33:06 2020-01-01 00:04:45.000 2020-01-02 03:33:06.000 285 99186 49735.5 4973550 285 99186 49735.5 4973550 -32284 32651 5171.02 517102 -128 127 -2.74 -274 -286 101 10276 99187 0.85885 297.85885 149.35885 14935.88588 0.85885 297.85886 149.35885 14935.88568 0.85885 297.85885 149.35885000000005 14935.88500 2020-01-01 2020-01-02 2020-01-01 00:04:46 2020-01-02 03:33:07 2020-01-01 00:04:46.000 2020-01-02 03:33:07.000 286 99187 49736.5 4973650 286 99187 49736.5 4973650 -32283 32652 5172.02 517202 -128 123 -4.3 -430 -287 101 10277 99188 0.86186 297.86186 149.36186 14936.18618 0.86186 297.86185 149.36186 14936.186 0.86186 297.86186 149.36186000000026 14936.18600 2020-01-01 2020-01-02 2020-01-01 00:04:47 2020-01-02 03:33:08 2020-01-01 00:04:47.000 2020-01-02 03:33:08.000 287 99188 49737.5 4973750 287 99188 49737.5 4973750 -32282 32653 5173.02 517302 -127 124 -3.3 -330 -288 101 10278 99189 0.86486 297.86486 149.36486 14936.48648 0.86486 297.86487 149.36486 14936.48673 0.86486 297.86486 149.36485999999982 14936.48600 2020-01-01 2020-01-02 2020-01-01 00:04:48 2020-01-02 03:33:09 2020-01-01 00:04:48.000 2020-01-02 03:33:09.000 288 99189 49738.5 4973850 288 99189 49738.5 4973850 -32281 32654 5174.02 517402 -126 125 -2.3 -230 -289 101 10279 99190 0.86786 297.86786 149.36786 14936.78678 0.86786 297.86786 149.36786 14936.78688 0.86786 297.86786 149.36786000000018 14936.78600 2020-01-01 2020-01-02 2020-01-01 00:04:49 2020-01-02 03:33:10 2020-01-01 00:04:49.000 2020-01-02 03:33:10.000 289 99190 49739.5 4973950 289 99190 49739.5 4973950 -32280 32655 5175.02 517502 -125 126 -1.3 -130 -29 102 10019 99929 0.08708 300.08708 150.08708 15158.79579 0.08708 300.0871 150.08708 15158.79576 0.08708 300.08708 150.08707999999987 15158.79508 2020-01-01 2020-01-02 2020-01-01 00:00:29 2020-01-02 03:45:29 2020-01-01 00:00:29.000 2020-01-02 03:45:29.000 29 99929 49979 5047879 29 99929 49979 5047879 -32540 32395 4558.009900990099 460359 -128 127 -1.8316831683168318 -185 -290 101 10280 99191 0.87087 297.87087 149.37087 14937.08708 0.87087 297.87088 149.37087 14937.087 0.87087 297.87087 149.37087000000017 14937.08700 2020-01-01 2020-01-02 2020-01-01 00:04:50 2020-01-02 03:33:11 2020-01-01 00:04:50.000 2020-01-02 03:33:11.000 290 99191 49740.5 4974050 290 99191 49740.5 4974050 -32279 32656 5176.02 517602 -124 127 -0.3 -30 -291 101 10281 99192 0.87387 297.87387 149.37387 14937.38738 0.87387 297.87387 149.37387 14937.38716 0.87387 297.87387 149.37386999999993 14937.38700 2020-01-01 2020-01-02 2020-01-01 00:04:51 2020-01-02 03:33:12 2020-01-01 00:04:51.000 2020-01-02 03:33:12.000 291 99192 49741.5 4974150 291 99192 49741.5 4974150 -32278 32657 5177.02 517702 -128 127 -1.86 -186 -292 101 10282 99193 0.87687 297.87687 149.37687 14937.68768 0.87687 297.8769 149.37687 14937.68789 0.87687 297.87687 149.37687000000003 14937.68700 2020-01-01 2020-01-02 2020-01-01 00:04:52 2020-01-02 03:33:13 2020-01-01 00:04:52.000 2020-01-02 03:33:13.000 292 99193 49742.5 4974250 292 99193 49742.5 4974250 -32277 32658 5178.02 517802 -128 123 -3.42 -342 -293 101 10283 99194 0.87987 297.87987 149.37987 14937.98798 0.87987 297.87988 149.37988 14937.9882 0.87987 297.87987 149.37987000000012 14937.98700 2020-01-01 2020-01-02 2020-01-01 00:04:53 2020-01-02 03:33:14 2020-01-01 00:04:53.000 2020-01-02 03:33:14.000 293 99194 49743.5 4974350 293 99194 49743.5 4974350 -32276 32659 5179.02 517902 -127 124 -2.42 -242 -294 101 10284 99195 0.88288 297.88288 149.38288 14938.28828 0.88288 297.88287 149.38288 14938.28835 0.88288 297.88288 149.38287999999983 14938.28800 2020-01-01 2020-01-02 2020-01-01 00:04:54 2020-01-02 03:33:15 2020-01-01 00:04:54.000 2020-01-02 03:33:15.000 294 99195 49744.5 4974450 294 99195 49744.5 4974450 -32275 32660 5180.02 518002 -126 125 -1.42 -142 -295 101 10285 99196 0.88588 297.88588 149.38588 14938.58858 0.88588 297.8859 149.38588 14938.58847 0.88588 297.88588 149.38587999999984 14938.58800 2020-01-01 2020-01-02 2020-01-01 00:04:55 2020-01-02 03:33:16 2020-01-01 00:04:55.000 2020-01-02 03:33:16.000 295 99196 49745.5 4974550 295 99196 49745.5 4974550 -32274 32661 5181.02 518102 -125 126 -0.42 -42 -296 101 10286 99197 0.88888 297.88888 149.38888 14938.88888 0.88888 297.8889 149.38888 14938.88863 0.88888 297.88888 149.38888000000028 14938.88800 2020-01-01 2020-01-02 2020-01-01 00:04:56 2020-01-02 03:33:17 2020-01-01 00:04:56.000 2020-01-02 03:33:17.000 296 99197 49746.5 4974650 296 99197 49746.5 4974650 -32273 32662 5182.02 518202 -124 127 0.58 58 -297 101 10287 99198 0.89189 297.89189 149.39189 14939.18918 0.89189 297.8919 149.39189 14939.18936 0.89189 297.89189 149.39189000000016 14939.18900 2020-01-01 2020-01-02 2020-01-01 00:04:57 2020-01-02 03:33:18 2020-01-01 00:04:57.000 2020-01-02 03:33:18.000 297 99198 49747.5 4974750 297 99198 49747.5 4974750 -32272 32663 5183.02 518302 -128 127 -0.98 -98 -298 101 10288 99199 0.89489 297.89489 149.39489 14939.48948 0.89489 297.8949 149.39489 14939.48967 0.89489 297.89489 149.39488999999972 14939.48900 2020-01-01 2020-01-02 2020-01-01 00:04:58 2020-01-02 03:33:19 2020-01-01 00:04:58.000 2020-01-02 03:33:19.000 298 99199 49748.5 4974850 298 99199 49748.5 4974850 -32271 32664 5184.02 518402 -128 127 -2.54 -254 -299 101 10289 99200 0.89789 297.89789 149.39789 14939.78978 0.89789 297.8979 149.39789 14939.78986 0.89789 297.89789 149.39789000000016 14939.78900 2020-01-01 2020-01-02 2020-01-01 00:04:59 2020-01-02 03:33:20 2020-01-01 00:04:59.000 2020-01-02 03:33:20.000 299 99200 49749.5 4974950 299 99200 49749.5 4974950 -32270 32665 5185.02 518502 -128 124 -4.1 -410 -3 102 1002 9993 0.009 300.009 150.009 15150.9099 0.009 300.009 150.009 15150.90958 0.00900 300.00900 150.00900000000001 15150.90900 2020-01-01 2020-01-02 2020-01-01 00:00:03 2020-01-02 03:45:03 2020-01-01 00:00:03.000 2020-01-02 03:45:03.000 3 99903 49953 5045253 3 99903 49953 5045253 -32566 32369 4532.009900990099 457733 -124 127 0.04950495049504951 5 -30 102 10020 99930 0.09009 300.09009 150.09009 15159.09909 0.09009 300.0901 150.09008 15159.09894 0.09009 300.09009 150.0900899999998 15159.09909 2020-01-01 2020-01-02 2020-01-01 00:00:30 2020-01-02 03:45:30 2020-01-01 00:00:30.000 2020-01-02 03:45:30.000 30 99930 49980 5047980 30 99930 49980 5047980 -32539 32396 4559.009900990099 460460 -128 123 -3.366336633663366 -340 -300 101 10290 99201 0.9009 297.9009 149.4009 14940.09009 0.9009 297.9009 149.40089 14940.08995 0.90090 297.90090 149.40090000000006 14940.09000 2020-01-01 2020-01-02 2020-01-01 00:05:00 2020-01-02 03:33:21 2020-01-01 00:05:00.000 2020-01-02 03:33:21.000 300 99201 49750.5 4975050 300 99201 49750.5 4975050 -32269 32666 5186.02 518602 -127 125 -3.1 -310 -301 101 10291 99202 0.9039 297.9039 149.4039 14940.39039 0.9039 297.9039 149.4039 14940.39009 0.90390 297.90390 149.40389999999988 14940.39000 2020-01-01 2020-01-02 2020-01-01 00:05:01 2020-01-02 03:33:22 2020-01-01 00:05:01.000 2020-01-02 03:33:22.000 301 99202 49751.5 4975150 301 99202 49751.5 4975150 -32268 32667 5187.02 518702 -126 126 -2.1 -210 -302 101 10292 99203 0.9069 297.9069 149.4069 14940.69069 0.9069 297.90692 149.4069 14940.69083 0.90690 297.90690 149.40689999999998 14940.69000 2020-01-01 2020-01-02 2020-01-01 00:05:02 2020-01-02 03:33:23 2020-01-01 00:05:02.000 2020-01-02 03:33:23.000 302 99203 49752.5 4975250 302 99203 49752.5 4975250 -32267 32668 5188.02 518802 -125 127 -1.1 -110 -303 101 10293 99204 0.9099 297.9099 149.4099 14940.99099 0.9099 297.9099 149.40991 14940.99114 0.90990 297.90990 149.40990000000008 14940.99000 2020-01-01 2020-01-02 2020-01-01 00:05:03 2020-01-02 03:33:24 2020-01-01 00:05:03.000 2020-01-02 03:33:24.000 303 99204 49753.5 4975350 303 99204 49753.5 4975350 -32266 32669 5189.02 518902 -128 127 -2.66 -266 -304 101 10294 99205 0.91291 297.91291 149.41291 14941.29129 0.91291 297.9129 149.41291 14941.29133 0.91291 297.91291 149.41290999999973 14941.29100 2020-01-01 2020-01-02 2020-01-01 00:05:04 2020-01-02 03:33:25 2020-01-01 00:05:04.000 2020-01-02 03:33:25.000 304 99205 49754.5 4975450 304 99205 49754.5 4975450 -32265 32670 5190.02 519002 -128 127 -4.22 -422 -305 101 10295 99206 0.91591 297.91591 149.41591 14941.59159 0.91591 297.91592 149.41591 14941.59141 0.91591 297.91591 149.41590999999983 14941.59100 2020-01-01 2020-01-02 2020-01-01 00:05:05 2020-01-02 03:33:26 2020-01-01 00:05:05.000 2020-01-02 03:33:26.000 305 99206 49755.5 4975550 305 99206 49755.5 4975550 -32264 32671 5191.02 519102 -128 123 -5.78 -578 -306 101 10296 99207 0.91891 297.91891 149.41891 14941.89189 0.91891 297.9189 149.41891 14941.89172 0.91891 297.91891 149.41891000000018 14941.89100 2020-01-01 2020-01-02 2020-01-01 00:05:06 2020-01-02 03:33:27 2020-01-01 00:05:06.000 2020-01-02 03:33:27.000 306 99207 49756.5 4975650 306 99207 49756.5 4975650 -32263 32672 5192.02 519202 -127 124 -4.78 -478 -307 101 10297 99208 0.92192 297.92192 149.42192 14942.19219 0.92192 297.92194 149.42192 14942.1923 0.92192 297.92192 149.42192000000009 14942.19200 2020-01-01 2020-01-02 2020-01-01 00:05:07 2020-01-02 03:33:28 2020-01-01 00:05:07.000 2020-01-02 03:33:28.000 307 99208 49757.5 4975750 307 99208 49757.5 4975750 -32262 32673 5193.02 519302 -126 125 -3.78 -378 -308 101 10298 99209 0.92492 297.92492 149.42492 14942.49249 0.92492 297.92493 149.42492 14942.49265 0.92492 297.92492 149.42491999999996 14942.49200 2020-01-01 2020-01-02 2020-01-01 00:05:08 2020-01-02 03:33:29 2020-01-01 00:05:08.000 2020-01-02 03:33:29.000 308 99209 49758.5 4975850 308 99209 49758.5 4975850 -32261 32674 5194.02 519402 -125 126 -2.78 -278 -309 101 10299 99210 0.92792 297.92792 149.42792 14942.79279 0.92792 297.92792 149.42792 14942.7928 0.92792 297.92792 149.42792000000006 14942.79200 2020-01-01 2020-01-02 2020-01-01 00:05:09 2020-01-02 03:33:30 2020-01-01 00:05:09.000 2020-01-02 03:33:30.000 309 99210 49759.5 4975950 309 99210 49759.5 4975950 -32260 32675 5195.02 519502 -124 127 -1.78 -178 -31 102 10021 99931 0.09309 300.09309 150.09309 15159.4024 0.09309 300.09308 150.09309 15159.40224 0.09309 300.09309 150.0930900000002 15159.40209 2020-01-01 2020-01-02 2020-01-01 00:00:31 2020-01-02 03:45:31 2020-01-01 00:00:31.000 2020-01-02 03:45:31.000 31 99931 49981 5048081 31 99931 49981 5048081 -32538 32397 4560.009900990099 460561 -127 124 -2.366336633663366 -239 -310 101 10300 99211 0.93093 297.93093 149.43093 14943.09309 0.93093 297.93094 149.43092 14943.09288 0.93093 297.93093 149.4309300000003 14943.09300 2020-01-01 2020-01-02 2020-01-01 00:05:10 2020-01-02 03:33:31 2020-01-01 00:05:10.000 2020-01-02 03:33:31.000 310 99211 49760.5 4976050 310 99211 49760.5 4976050 -32259 32676 5196.02 519602 -128 127 -3.34 -334 -311 101 10301 99212 0.93393 297.93393 149.43393 14943.39339 0.93393 297.93393 149.43393 14943.39319 0.93393 297.93393 149.43392999999983 14943.39300 2020-01-01 2020-01-02 2020-01-01 00:05:11 2020-01-02 03:33:32 2020-01-01 00:05:11.000 2020-01-02 03:33:32.000 311 99212 49761.5 4976150 311 99212 49761.5 4976150 -32258 32677 5197.02 519702 -128 123 -4.9 -490 -312 101 10302 99213 0.93693 297.93693 149.43693 14943.69369 0.93693 297.93695 149.43693 14943.69377 0.93693 297.93693 149.43692999999988 14943.69300 2020-01-01 2020-01-02 2020-01-01 00:05:12 2020-01-02 03:33:33 2020-01-01 00:05:12.000 2020-01-02 03:33:33.000 312 99213 49762.5 4976250 312 99213 49762.5 4976250 -32257 32678 5198.02 519802 -127 124 -3.9 -390 -313 101 10303 99214 0.93993 297.93993 149.43993 14943.99399 0.93993 297.93994 149.43994 14943.99412 0.93993 297.93993 149.4399300000003 14943.99300 2020-01-01 2020-01-02 2020-01-01 00:05:13 2020-01-02 03:33:34 2020-01-01 00:05:13.000 2020-01-02 03:33:34.000 313 99214 49763.5 4976350 313 99214 49763.5 4976350 -32256 32679 5199.02 519902 -126 125 -2.9 -290 -314 101 10304 99215 0.94294 297.94294 149.44294 14944.29429 0.94294 297.94293 149.44294 14944.29427 0.94294 297.94294 149.4429400000002 14944.29400 2020-01-01 2020-01-02 2020-01-01 00:05:14 2020-01-02 03:33:35 2020-01-01 00:05:14.000 2020-01-02 03:33:35.000 314 99215 49764.5 4976450 314 99215 49764.5 4976450 -32255 32680 5200.02 520002 -125 126 -1.9 -190 -315 101 10305 99216 0.94594 297.94594 149.44594 14944.59459 0.94594 297.94595 149.44595 14944.595 0.94594 297.94594 149.44593999999975 14944.59400 2020-01-01 2020-01-02 2020-01-01 00:05:15 2020-01-02 03:33:36 2020-01-01 00:05:15.000 2020-01-02 03:33:36.000 315 99216 49765.5 4976550 315 99216 49765.5 4976550 -32254 32681 5201.02 520102 -124 127 -0.9 -90 -316 101 10306 99217 0.94894 297.94894 149.44894 14944.89489 0.94894 297.94894 149.44894 14944.89466 0.94894 297.94894 149.4489400000002 14944.89400 2020-01-01 2020-01-02 2020-01-01 00:05:16 2020-01-02 03:33:37 2020-01-01 00:05:16.000 2020-01-02 03:33:37.000 316 99217 49766.5 4976650 316 99217 49766.5 4976650 -32253 32682 5202.02 520202 -128 127 -2.46 -246 -317 101 10307 99218 0.95195 297.95195 149.45195 14945.19519 0.95195 297.95197 149.45195 14945.19524 0.95195 297.95195 149.4519500000001 14945.19500 2020-01-01 2020-01-02 2020-01-01 00:05:17 2020-01-02 03:33:38 2020-01-01 00:05:17.000 2020-01-02 03:33:38.000 317 99218 49767.5 4976750 317 99218 49767.5 4976750 -32252 32683 5203.02 520302 -128 123 -4.02 -402 -318 101 10308 99219 0.95495 297.95495 149.45495 14945.49549 0.95495 297.95496 149.45495 14945.49558 0.95495 297.95495 149.45494999999985 14945.49500 2020-01-01 2020-01-02 2020-01-01 00:05:18 2020-01-02 03:33:39 2020-01-01 00:05:18.000 2020-01-02 03:33:39.000 318 99219 49768.5 4976850 318 99219 49768.5 4976850 -32251 32684 5204.02 520402 -127 124 -3.02 -302 -319 101 10309 99220 0.95795 297.95795 149.45795 14945.79579 0.95795 297.95795 149.45795 14945.79574 0.95795 297.95795 149.45794999999998 14945.79500 2020-01-01 2020-01-02 2020-01-01 00:05:19 2020-01-02 03:33:40 2020-01-01 00:05:19.000 2020-01-02 03:33:40.000 319 99220 49769.5 4976950 319 99220 49769.5 4976950 -32250 32685 5205.02 520502 -126 125 -2.02 -202 -32 102 10022 99932 0.09609 300.09609 150.09609 15159.7057 0.09609 300.0961 150.09609 15159.706 0.09609 300.09609 150.09608999999972 15159.70509 2020-01-01 2020-01-02 2020-01-01 00:00:32 2020-01-02 03:45:32 2020-01-01 00:00:32.000 2020-01-02 03:45:32.000 32 99932 49982 5048182 32 99932 49982 5048182 -32537 32398 4561.009900990099 460662 -126 125 -1.3663366336633664 -138 -320 101 10310 99221 0.96096 297.96096 149.46096 14946.09609 0.96096 297.96097 149.46096 14946.09647 0.96096 297.96096 149.4609600000002 14946.09600 2020-01-01 2020-01-02 2020-01-01 00:05:20 2020-01-02 03:33:41 2020-01-01 00:05:20.000 2020-01-02 03:33:41.000 320 99221 49770.5 4977050 320 99221 49770.5 4977050 -32249 32686 5206.02 520602 -125 126 -1.02 -102 -321 101 10311 99222 0.96396 297.96396 149.46396 14946.39639 0.96396 297.96396 149.46396 14946.39613 0.96396 297.96396 149.46395999999976 14946.39600 2020-01-01 2020-01-02 2020-01-01 00:05:21 2020-01-02 03:33:42 2020-01-01 00:05:21.000 2020-01-02 03:33:42.000 321 99222 49771.5 4977150 321 99222 49771.5 4977150 -32248 32687 5207.02 520702 -124 127 -0.02 -2 -322 101 10312 99223 0.96696 297.96696 149.46696 14946.69669 0.96696 297.96698 149.46696 14946.69674 0.96696 297.96696 149.46695999999986 14946.69600 2020-01-01 2020-01-02 2020-01-01 00:05:22 2020-01-02 03:33:43 2020-01-01 00:05:22.000 2020-01-02 03:33:43.000 322 99223 49772.5 4977250 322 99223 49772.5 4977250 -32247 32688 5208.02 520802 -128 127 -1.58 -158 -323 101 10313 99224 0.96996 297.96996 149.46996 14946.99699 0.96997 297.96997 149.46997 14946.99706 0.96996 297.96996 149.4699600000002 14946.99600 2020-01-01 2020-01-02 2020-01-01 00:05:23 2020-01-02 03:33:44 2020-01-01 00:05:23.000 2020-01-02 03:33:44.000 323 99224 49773.5 4977350 323 99224 49773.5 4977350 -32246 32689 5209.02 520902 -128 123 -3.14 -314 -324 101 10314 99225 0.97297 297.97297 149.47297 14947.29729 0.97297 297.97296 149.47297 14947.29737 0.97297 297.97297 149.47297000000012 14947.29700 2020-01-01 2020-01-02 2020-01-01 00:05:24 2020-01-02 03:33:45 2020-01-01 00:05:24.000 2020-01-02 03:33:45.000 324 99225 49774.5 4977450 324 99225 49774.5 4977450 -32245 32690 5210.02 521002 -127 124 -2.14 -214 +27 102 10017 99927 0.08108 300.08108 150.08108 15158.18918 0.08108 300.0811 150.08108 15158.18936 0.08108 300.08108 150.08108000000001 15158.18908 2020-01-01 2020-01-02 2020-01-01 00:00:27 2020-01-02 03:45:27 2020-01-01 00:00:27.000 2020-01-02 03:45:27.000 27 99927 49977 5047677 27 99927 49977 5047677 -32542 32393 4556.009900990099 460157 -125 126 -1.297029702970297 -131 +270 101 10260 99171 0.81081 297.81081 149.31081 14931.08108 0.81081 297.81082 149.31081 14931.08109 0.81081 297.81081 149.31081 14931.08100 2020-01-01 2020-01-02 2020-01-01 00:04:30 2020-01-02 03:32:51 2020-01-01 00:04:30.000 2020-01-02 03:32:51.000 270 99171 49720.5 4972050 270 99171 49720.5 4972050 -32299 32636 5156.02 515602 -125 126 0.18 18 +271 101 10261 99172 0.81381 297.81381 149.31381 14931.38138 0.81381 297.8138 149.31381 14931.38124 0.81381 297.81381 149.31381 14931.38100 2020-01-01 2020-01-02 2020-01-01 00:04:31 2020-01-02 03:32:52 2020-01-01 00:04:31.000 2020-01-02 03:32:52.000 271 99172 49721.5 4972150 271 99172 49721.5 4972150 -32298 32637 5157.02 515702 -124 127 1.18 118 +272 101 10262 99173 0.81681 297.81681 149.31681 14931.68168 0.81681 297.8168 149.31681 14931.68159 0.81681 297.81681 149.31681 14931.68100 2020-01-01 2020-01-02 2020-01-01 00:04:32 2020-01-02 03:32:53 2020-01-01 00:04:32.000 2020-01-02 03:32:53.000 272 99173 49722.5 4972250 272 99173 49722.5 4972250 -32297 32638 5158.02 515802 -128 127 -0.38 -38 +273 101 10263 99174 0.81981 297.81981 149.31981 14931.98198 0.81981 297.81982 149.31982 14931.98217 0.81981 297.81981 149.31981 14931.98100 2020-01-01 2020-01-02 2020-01-01 00:04:33 2020-01-02 03:32:54 2020-01-01 00:04:33.000 2020-01-02 03:32:54.000 273 99174 49723.5 4972350 273 99174 49723.5 4972350 -32296 32639 5159.02 515902 -128 127 -1.94 -194 +274 101 10264 99175 0.82282 297.82282 149.32282 14932.28228 0.82282 297.8228 149.32282 14932.28247 0.82282 297.82282 149.32281999999998 14932.28200 2020-01-01 2020-01-02 2020-01-01 00:04:34 2020-01-02 03:32:55 2020-01-01 00:04:34.000 2020-01-02 03:32:55.000 274 99175 49724.5 4972450 274 99175 49724.5 4972450 -32295 32640 5160.02 516002 -128 124 -3.5 -350 +275 101 10265 99176 0.82582 297.82582 149.32582 14932.58258 0.82582 297.82584 149.32582 14932.58256 0.82582 297.82582 149.32582 14932.58200 2020-01-01 2020-01-02 2020-01-01 00:04:35 2020-01-02 03:32:56 2020-01-01 00:04:35.000 2020-01-02 03:32:56.000 275 99176 49725.5 4972550 275 99176 49725.5 4972550 -32294 32641 5161.02 516102 -127 125 -2.5 -250 +276 101 10266 99177 0.82882 297.82882 149.32882 14932.88288 0.82882 297.82883 149.32882 14932.88275 0.82882 297.82882 149.32882 14932.88200 2020-01-01 2020-01-02 2020-01-01 00:04:36 2020-01-02 03:32:57 2020-01-01 00:04:36.000 2020-01-02 03:32:57.000 276 99177 49726.5 4972650 276 99177 49726.5 4972650 -32293 32642 5162.02 516202 -126 126 -1.5 -150 +277 101 10267 99178 0.83183 297.83183 149.33183 14933.18318 0.83183 297.83182 149.33183 14933.18305 0.83183 297.83183 149.33183 14933.18300 2020-01-01 2020-01-02 2020-01-01 00:04:37 2020-01-02 03:32:58 2020-01-01 00:04:37.000 2020-01-02 03:32:58.000 277 99178 49727.5 4972750 277 99178 49727.5 4972750 -32292 32643 5163.02 516302 -125 127 -0.5 -50 +278 101 10268 99179 0.83483 297.83483 149.33483 14933.48348 0.83483 297.83484 149.33483 14933.48364 0.83483 297.83483 149.33483 14933.48300 2020-01-01 2020-01-02 2020-01-01 00:04:38 2020-01-02 03:32:59 2020-01-01 00:04:38.000 2020-01-02 03:32:59.000 278 99179 49728.5 4972850 278 99179 49728.5 4972850 -32291 32644 5164.02 516402 -128 127 -2.06 -206 +279 101 10269 99180 0.83783 297.83783 149.33783 14933.78378 0.83783 297.83783 149.33783 14933.78394 0.83783 297.83783 149.33783 14933.78300 2020-01-01 2020-01-02 2020-01-01 00:04:39 2020-01-02 03:33:00 2020-01-01 00:04:39.000 2020-01-02 03:33:00.000 279 99180 49729.5 4972950 279 99180 49729.5 4972950 -32290 32645 5165.02 516502 -128 127 -3.62 -362 +28 102 10018 99928 0.08408 300.08408 150.08408 15158.49249 0.08408 300.08408 150.08408 15158.49265 0.08408 300.08408 150.08408 15158.49208 2020-01-01 2020-01-02 2020-01-01 00:00:28 2020-01-02 03:45:28 2020-01-01 00:00:28.000 2020-01-02 03:45:28.000 28 99928 49978 5047778 28 99928 49978 5047778 -32541 32394 4557.009900990099 460258 -124 127 -0.297029702970297 -30 +280 101 10270 99181 0.84084 297.84084 149.34084 14934.08408 0.84084 297.84085 149.34084 14934.08403 0.84084 297.84084 149.34084000000001 14934.08400 2020-01-01 2020-01-02 2020-01-01 00:04:40 2020-01-02 03:33:01 2020-01-01 00:04:40.000 2020-01-02 03:33:01.000 280 99181 49730.5 4973050 280 99181 49730.5 4973050 -32289 32646 5166.02 516602 -128 123 -5.18 -518 +281 101 10271 99182 0.84384 297.84384 149.34384 14934.38438 0.84384 297.84384 149.34384 14934.38421 0.84384 297.84384 149.34384 14934.38400 2020-01-01 2020-01-02 2020-01-01 00:04:41 2020-01-02 03:33:02 2020-01-01 00:04:41.000 2020-01-02 03:33:02.000 281 99182 49731.5 4973150 281 99182 49731.5 4973150 -32288 32647 5167.02 516702 -127 124 -4.18 -418 +282 101 10272 99183 0.84684 297.84684 149.34684 14934.68468 0.84684 297.84683 149.34684 14934.68453 0.84684 297.84684 149.34684 14934.68400 2020-01-01 2020-01-02 2020-01-01 00:04:42 2020-01-02 03:33:03 2020-01-01 00:04:42.000 2020-01-02 03:33:03.000 282 99183 49732.5 4973250 282 99183 49732.5 4973250 -32287 32648 5168.02 516802 -126 125 -3.18 -318 +283 101 10273 99184 0.84984 297.84984 149.34984 14934.98498 0.84984 297.84985 149.34985 14934.98526 0.84984 297.84984 149.34984 14934.98400 2020-01-01 2020-01-02 2020-01-01 00:04:43 2020-01-02 03:33:04 2020-01-01 00:04:43.000 2020-01-02 03:33:04.000 283 99184 49733.5 4973350 283 99184 49733.5 4973350 -32286 32649 5169.02 516902 -125 126 -2.18 -218 +284 101 10274 99185 0.85285 297.85285 149.35285 14935.28528 0.85285 297.85284 149.35285 14935.28542 0.85285 297.85285 149.35285 14935.28500 2020-01-01 2020-01-02 2020-01-01 00:04:44 2020-01-02 03:33:05 2020-01-01 00:04:44.000 2020-01-02 03:33:05.000 284 99185 49734.5 4973450 284 99185 49734.5 4973450 -32285 32650 5170.02 517002 -124 127 -1.18 -118 +285 101 10275 99186 0.85585 297.85585 149.35585 14935.58558 0.85585 297.85587 149.35585 14935.5855 0.85585 297.85585 149.35585 14935.58500 2020-01-01 2020-01-02 2020-01-01 00:04:45 2020-01-02 03:33:06 2020-01-01 00:04:45.000 2020-01-02 03:33:06.000 285 99186 49735.5 4973550 285 99186 49735.5 4973550 -32284 32651 5171.02 517102 -128 127 -2.74 -274 +286 101 10276 99187 0.85885 297.85885 149.35885 14935.88588 0.85885 297.85886 149.35885 14935.88568 0.85885 297.85885 149.35885 14935.88500 2020-01-01 2020-01-02 2020-01-01 00:04:46 2020-01-02 03:33:07 2020-01-01 00:04:46.000 2020-01-02 03:33:07.000 286 99187 49736.5 4973650 286 99187 49736.5 4973650 -32283 32652 5172.02 517202 -128 123 -4.3 -430 +287 101 10277 99188 0.86186 297.86186 149.36186 14936.18618 0.86186 297.86185 149.36186 14936.186 0.86186 297.86186 149.36186 14936.18600 2020-01-01 2020-01-02 2020-01-01 00:04:47 2020-01-02 03:33:08 2020-01-01 00:04:47.000 2020-01-02 03:33:08.000 287 99188 49737.5 4973750 287 99188 49737.5 4973750 -32282 32653 5173.02 517302 -127 124 -3.3 -330 +288 101 10278 99189 0.86486 297.86486 149.36486 14936.48648 0.86486 297.86487 149.36486 14936.48673 0.86486 297.86486 149.36486000000002 14936.48600 2020-01-01 2020-01-02 2020-01-01 00:04:48 2020-01-02 03:33:09 2020-01-01 00:04:48.000 2020-01-02 03:33:09.000 288 99189 49738.5 4973850 288 99189 49738.5 4973850 -32281 32654 5174.02 517402 -126 125 -2.3 -230 +289 101 10279 99190 0.86786 297.86786 149.36786 14936.78678 0.86786 297.86786 149.36786 14936.78688 0.86786 297.86786 149.36786 14936.78600 2020-01-01 2020-01-02 2020-01-01 00:04:49 2020-01-02 03:33:10 2020-01-01 00:04:49.000 2020-01-02 03:33:10.000 289 99190 49739.5 4973950 289 99190 49739.5 4973950 -32280 32655 5175.02 517502 -125 126 -1.3 -130 +29 102 10019 99929 0.08708 300.08708 150.08708 15158.79579 0.08708 300.0871 150.08708 15158.79576 0.08708 300.08708 150.08708 15158.79508 2020-01-01 2020-01-02 2020-01-01 00:00:29 2020-01-02 03:45:29 2020-01-01 00:00:29.000 2020-01-02 03:45:29.000 29 99929 49979 5047879 29 99929 49979 5047879 -32540 32395 4558.009900990099 460359 -128 127 -1.8316831683168318 -185 +290 101 10280 99191 0.87087 297.87087 149.37087 14937.08708 0.87087 297.87088 149.37087 14937.087 0.87087 297.87087 149.37087 14937.08700 2020-01-01 2020-01-02 2020-01-01 00:04:50 2020-01-02 03:33:11 2020-01-01 00:04:50.000 2020-01-02 03:33:11.000 290 99191 49740.5 4974050 290 99191 49740.5 4974050 -32279 32656 5176.02 517602 -124 127 -0.3 -30 +291 101 10281 99192 0.87387 297.87387 149.37387 14937.38738 0.87387 297.87387 149.37387 14937.38716 0.87387 297.87387 149.37387 14937.38700 2020-01-01 2020-01-02 2020-01-01 00:04:51 2020-01-02 03:33:12 2020-01-01 00:04:51.000 2020-01-02 03:33:12.000 291 99192 49741.5 4974150 291 99192 49741.5 4974150 -32278 32657 5177.02 517702 -128 127 -1.86 -186 +292 101 10282 99193 0.87687 297.87687 149.37687 14937.68768 0.87687 297.8769 149.37687 14937.68789 0.87687 297.87687 149.37687 14937.68700 2020-01-01 2020-01-02 2020-01-01 00:04:52 2020-01-02 03:33:13 2020-01-01 00:04:52.000 2020-01-02 03:33:13.000 292 99193 49742.5 4974250 292 99193 49742.5 4974250 -32277 32658 5178.02 517802 -128 123 -3.42 -342 +293 101 10283 99194 0.87987 297.87987 149.37987 14937.98798 0.87987 297.87988 149.37988 14937.9882 0.87987 297.87987 149.37986999999998 14937.98700 2020-01-01 2020-01-02 2020-01-01 00:04:53 2020-01-02 03:33:14 2020-01-01 00:04:53.000 2020-01-02 03:33:14.000 293 99194 49743.5 4974350 293 99194 49743.5 4974350 -32276 32659 5179.02 517902 -127 124 -2.42 -242 +294 101 10284 99195 0.88288 297.88288 149.38288 14938.28828 0.88288 297.88287 149.38288 14938.28835 0.88288 297.88288 149.38288 14938.28800 2020-01-01 2020-01-02 2020-01-01 00:04:54 2020-01-02 03:33:15 2020-01-01 00:04:54.000 2020-01-02 03:33:15.000 294 99195 49744.5 4974450 294 99195 49744.5 4974450 -32275 32660 5180.02 518002 -126 125 -1.42 -142 +295 101 10285 99196 0.88588 297.88588 149.38588 14938.58858 0.88588 297.8859 149.38588 14938.58847 0.88588 297.88588 149.38588 14938.58800 2020-01-01 2020-01-02 2020-01-01 00:04:55 2020-01-02 03:33:16 2020-01-01 00:04:55.000 2020-01-02 03:33:16.000 295 99196 49745.5 4974550 295 99196 49745.5 4974550 -32274 32661 5181.02 518102 -125 126 -0.42 -42 +296 101 10286 99197 0.88888 297.88888 149.38888 14938.88888 0.88888 297.8889 149.38888 14938.88863 0.88888 297.88888 149.38888 14938.88800 2020-01-01 2020-01-02 2020-01-01 00:04:56 2020-01-02 03:33:17 2020-01-01 00:04:56.000 2020-01-02 03:33:17.000 296 99197 49746.5 4974650 296 99197 49746.5 4974650 -32273 32662 5182.02 518202 -124 127 0.58 58 +297 101 10287 99198 0.89189 297.89189 149.39189 14939.18918 0.89189 297.8919 149.39189 14939.18936 0.89189 297.89189 149.39189 14939.18900 2020-01-01 2020-01-02 2020-01-01 00:04:57 2020-01-02 03:33:18 2020-01-01 00:04:57.000 2020-01-02 03:33:18.000 297 99198 49747.5 4974750 297 99198 49747.5 4974750 -32272 32663 5183.02 518302 -128 127 -0.98 -98 +298 101 10288 99199 0.89489 297.89489 149.39489 14939.48948 0.89489 297.8949 149.39489 14939.48967 0.89489 297.89489 149.39489 14939.48900 2020-01-01 2020-01-02 2020-01-01 00:04:58 2020-01-02 03:33:19 2020-01-01 00:04:58.000 2020-01-02 03:33:19.000 298 99199 49748.5 4974850 298 99199 49748.5 4974850 -32271 32664 5184.02 518402 -128 127 -2.54 -254 +299 101 10289 99200 0.89789 297.89789 149.39789 14939.78978 0.89789 297.8979 149.39789 14939.78986 0.89789 297.89789 149.39789000000002 14939.78900 2020-01-01 2020-01-02 2020-01-01 00:04:59 2020-01-02 03:33:20 2020-01-01 00:04:59.000 2020-01-02 03:33:20.000 299 99200 49749.5 4974950 299 99200 49749.5 4974950 -32270 32665 5185.02 518502 -128 124 -4.1 -410 +3 102 1002 9993 0.009 300.009 150.009 15150.9099 0.009 300.009 150.009 15150.90958 0.00900 300.00900 150.009 15150.90900 2020-01-01 2020-01-02 2020-01-01 00:00:03 2020-01-02 03:45:03 2020-01-01 00:00:03.000 2020-01-02 03:45:03.000 3 99903 49953 5045253 3 99903 49953 5045253 -32566 32369 4532.009900990099 457733 -124 127 0.04950495049504951 5 +30 102 10020 99930 0.09009 300.09009 150.09009 15159.09909 0.09009 300.0901 150.09008 15159.09894 0.09009 300.09009 150.09009 15159.09909 2020-01-01 2020-01-02 2020-01-01 00:00:30 2020-01-02 03:45:30 2020-01-01 00:00:30.000 2020-01-02 03:45:30.000 30 99930 49980 5047980 30 99930 49980 5047980 -32539 32396 4559.009900990099 460460 -128 123 -3.366336633663366 -340 +300 101 10290 99201 0.9009 297.9009 149.4009 14940.09009 0.9009 297.9009 149.40089 14940.08995 0.90090 297.90090 149.4009 14940.09000 2020-01-01 2020-01-02 2020-01-01 00:05:00 2020-01-02 03:33:21 2020-01-01 00:05:00.000 2020-01-02 03:33:21.000 300 99201 49750.5 4975050 300 99201 49750.5 4975050 -32269 32666 5186.02 518602 -127 125 -3.1 -310 +301 101 10291 99202 0.9039 297.9039 149.4039 14940.39039 0.9039 297.9039 149.4039 14940.39009 0.90390 297.90390 149.4039 14940.39000 2020-01-01 2020-01-02 2020-01-01 00:05:01 2020-01-02 03:33:22 2020-01-01 00:05:01.000 2020-01-02 03:33:22.000 301 99202 49751.5 4975150 301 99202 49751.5 4975150 -32268 32667 5187.02 518702 -126 126 -2.1 -210 +302 101 10292 99203 0.9069 297.9069 149.4069 14940.69069 0.9069 297.90692 149.4069 14940.69083 0.90690 297.90690 149.4069 14940.69000 2020-01-01 2020-01-02 2020-01-01 00:05:02 2020-01-02 03:33:23 2020-01-01 00:05:02.000 2020-01-02 03:33:23.000 302 99203 49752.5 4975250 302 99203 49752.5 4975250 -32267 32668 5188.02 518802 -125 127 -1.1 -110 +303 101 10293 99204 0.9099 297.9099 149.4099 14940.99099 0.9099 297.9099 149.40991 14940.99114 0.90990 297.90990 149.4099 14940.99000 2020-01-01 2020-01-02 2020-01-01 00:05:03 2020-01-02 03:33:24 2020-01-01 00:05:03.000 2020-01-02 03:33:24.000 303 99204 49753.5 4975350 303 99204 49753.5 4975350 -32266 32669 5189.02 518902 -128 127 -2.66 -266 +304 101 10294 99205 0.91291 297.91291 149.41291 14941.29129 0.91291 297.9129 149.41291 14941.29133 0.91291 297.91291 149.41290999999998 14941.29100 2020-01-01 2020-01-02 2020-01-01 00:05:04 2020-01-02 03:33:25 2020-01-01 00:05:04.000 2020-01-02 03:33:25.000 304 99205 49754.5 4975450 304 99205 49754.5 4975450 -32265 32670 5190.02 519002 -128 127 -4.22 -422 +305 101 10295 99206 0.91591 297.91591 149.41591 14941.59159 0.91591 297.91592 149.41591 14941.59141 0.91591 297.91591 149.41591 14941.59100 2020-01-01 2020-01-02 2020-01-01 00:05:05 2020-01-02 03:33:26 2020-01-01 00:05:05.000 2020-01-02 03:33:26.000 305 99206 49755.5 4975550 305 99206 49755.5 4975550 -32264 32671 5191.02 519102 -128 123 -5.78 -578 +306 101 10296 99207 0.91891 297.91891 149.41891 14941.89189 0.91891 297.9189 149.41891 14941.89172 0.91891 297.91891 149.41890999999998 14941.89100 2020-01-01 2020-01-02 2020-01-01 00:05:06 2020-01-02 03:33:27 2020-01-01 00:05:06.000 2020-01-02 03:33:27.000 306 99207 49756.5 4975650 306 99207 49756.5 4975650 -32263 32672 5192.02 519202 -127 124 -4.78 -478 +307 101 10297 99208 0.92192 297.92192 149.42192 14942.19219 0.92192 297.92194 149.42192 14942.1923 0.92192 297.92192 149.42192 14942.19200 2020-01-01 2020-01-02 2020-01-01 00:05:07 2020-01-02 03:33:28 2020-01-01 00:05:07.000 2020-01-02 03:33:28.000 307 99208 49757.5 4975750 307 99208 49757.5 4975750 -32262 32673 5193.02 519302 -126 125 -3.78 -378 +308 101 10298 99209 0.92492 297.92492 149.42492 14942.49249 0.92492 297.92493 149.42492 14942.49265 0.92492 297.92492 149.42492000000001 14942.49200 2020-01-01 2020-01-02 2020-01-01 00:05:08 2020-01-02 03:33:29 2020-01-01 00:05:08.000 2020-01-02 03:33:29.000 308 99209 49758.5 4975850 308 99209 49758.5 4975850 -32261 32674 5194.02 519402 -125 126 -2.78 -278 +309 101 10299 99210 0.92792 297.92792 149.42792 14942.79279 0.92792 297.92792 149.42792 14942.7928 0.92792 297.92792 149.42792 14942.79200 2020-01-01 2020-01-02 2020-01-01 00:05:09 2020-01-02 03:33:30 2020-01-01 00:05:09.000 2020-01-02 03:33:30.000 309 99210 49759.5 4975950 309 99210 49759.5 4975950 -32260 32675 5195.02 519502 -124 127 -1.78 -178 +31 102 10021 99931 0.09309 300.09309 150.09309 15159.4024 0.09309 300.09308 150.09309 15159.40224 0.09309 300.09309 150.09309 15159.40209 2020-01-01 2020-01-02 2020-01-01 00:00:31 2020-01-02 03:45:31 2020-01-01 00:00:31.000 2020-01-02 03:45:31.000 31 99931 49981 5048081 31 99931 49981 5048081 -32538 32397 4560.009900990099 460561 -127 124 -2.366336633663366 -239 +310 101 10300 99211 0.93093 297.93093 149.43093 14943.09309 0.93093 297.93094 149.43092 14943.09288 0.93093 297.93093 149.43093000000002 14943.09300 2020-01-01 2020-01-02 2020-01-01 00:05:10 2020-01-02 03:33:31 2020-01-01 00:05:10.000 2020-01-02 03:33:31.000 310 99211 49760.5 4976050 310 99211 49760.5 4976050 -32259 32676 5196.02 519602 -128 127 -3.34 -334 +311 101 10301 99212 0.93393 297.93393 149.43393 14943.39339 0.93393 297.93393 149.43393 14943.39319 0.93393 297.93393 149.43393 14943.39300 2020-01-01 2020-01-02 2020-01-01 00:05:11 2020-01-02 03:33:32 2020-01-01 00:05:11.000 2020-01-02 03:33:32.000 311 99212 49761.5 4976150 311 99212 49761.5 4976150 -32258 32677 5197.02 519702 -128 123 -4.9 -490 +312 101 10302 99213 0.93693 297.93693 149.43693 14943.69369 0.93693 297.93695 149.43693 14943.69377 0.93693 297.93693 149.43693 14943.69300 2020-01-01 2020-01-02 2020-01-01 00:05:12 2020-01-02 03:33:33 2020-01-01 00:05:12.000 2020-01-02 03:33:33.000 312 99213 49762.5 4976250 312 99213 49762.5 4976250 -32257 32678 5198.02 519802 -127 124 -3.9 -390 +313 101 10303 99214 0.93993 297.93993 149.43993 14943.99399 0.93993 297.93994 149.43994 14943.99412 0.93993 297.93993 149.43993 14943.99300 2020-01-01 2020-01-02 2020-01-01 00:05:13 2020-01-02 03:33:34 2020-01-01 00:05:13.000 2020-01-02 03:33:34.000 313 99214 49763.5 4976350 313 99214 49763.5 4976350 -32256 32679 5199.02 519902 -126 125 -2.9 -290 +314 101 10304 99215 0.94294 297.94294 149.44294 14944.29429 0.94294 297.94293 149.44294 14944.29427 0.94294 297.94294 149.44294 14944.29400 2020-01-01 2020-01-02 2020-01-01 00:05:14 2020-01-02 03:33:35 2020-01-01 00:05:14.000 2020-01-02 03:33:35.000 314 99215 49764.5 4976450 314 99215 49764.5 4976450 -32255 32680 5200.02 520002 -125 126 -1.9 -190 +315 101 10305 99216 0.94594 297.94594 149.44594 14944.59459 0.94594 297.94595 149.44595 14944.595 0.94594 297.94594 149.44593999999998 14944.59400 2020-01-01 2020-01-02 2020-01-01 00:05:15 2020-01-02 03:33:36 2020-01-01 00:05:15.000 2020-01-02 03:33:36.000 315 99216 49765.5 4976550 315 99216 49765.5 4976550 -32254 32681 5201.02 520102 -124 127 -0.9 -90 +316 101 10306 99217 0.94894 297.94894 149.44894 14944.89489 0.94894 297.94894 149.44894 14944.89466 0.94894 297.94894 149.44894 14944.89400 2020-01-01 2020-01-02 2020-01-01 00:05:16 2020-01-02 03:33:37 2020-01-01 00:05:16.000 2020-01-02 03:33:37.000 316 99217 49766.5 4976650 316 99217 49766.5 4976650 -32253 32682 5202.02 520202 -128 127 -2.46 -246 +317 101 10307 99218 0.95195 297.95195 149.45195 14945.19519 0.95195 297.95197 149.45195 14945.19524 0.95195 297.95195 149.45195 14945.19500 2020-01-01 2020-01-02 2020-01-01 00:05:17 2020-01-02 03:33:38 2020-01-01 00:05:17.000 2020-01-02 03:33:38.000 317 99218 49767.5 4976750 317 99218 49767.5 4976750 -32252 32683 5203.02 520302 -128 123 -4.02 -402 +318 101 10308 99219 0.95495 297.95495 149.45495 14945.49549 0.95495 297.95496 149.45495 14945.49558 0.95495 297.95495 149.45495 14945.49500 2020-01-01 2020-01-02 2020-01-01 00:05:18 2020-01-02 03:33:39 2020-01-01 00:05:18.000 2020-01-02 03:33:39.000 318 99219 49768.5 4976850 318 99219 49768.5 4976850 -32251 32684 5204.02 520402 -127 124 -3.02 -302 +319 101 10309 99220 0.95795 297.95795 149.45795 14945.79579 0.95795 297.95795 149.45795 14945.79574 0.95795 297.95795 149.45795 14945.79500 2020-01-01 2020-01-02 2020-01-01 00:05:19 2020-01-02 03:33:40 2020-01-01 00:05:19.000 2020-01-02 03:33:40.000 319 99220 49769.5 4976950 319 99220 49769.5 4976950 -32250 32685 5205.02 520502 -126 125 -2.02 -202 +32 102 10022 99932 0.09609 300.09609 150.09609 15159.7057 0.09609 300.0961 150.09609 15159.706 0.09609 300.09609 150.09609 15159.70509 2020-01-01 2020-01-02 2020-01-01 00:00:32 2020-01-02 03:45:32 2020-01-01 00:00:32.000 2020-01-02 03:45:32.000 32 99932 49982 5048182 32 99932 49982 5048182 -32537 32398 4561.009900990099 460662 -126 125 -1.3663366336633664 -138 +320 101 10310 99221 0.96096 297.96096 149.46096 14946.09609 0.96096 297.96097 149.46096 14946.09647 0.96096 297.96096 149.46096 14946.09600 2020-01-01 2020-01-02 2020-01-01 00:05:20 2020-01-02 03:33:41 2020-01-01 00:05:20.000 2020-01-02 03:33:41.000 320 99221 49770.5 4977050 320 99221 49770.5 4977050 -32249 32686 5206.02 520602 -125 126 -1.02 -102 +321 101 10311 99222 0.96396 297.96396 149.46396 14946.39639 0.96396 297.96396 149.46396 14946.39613 0.96396 297.96396 149.46396000000001 14946.39600 2020-01-01 2020-01-02 2020-01-01 00:05:21 2020-01-02 03:33:42 2020-01-01 00:05:21.000 2020-01-02 03:33:42.000 321 99222 49771.5 4977150 321 99222 49771.5 4977150 -32248 32687 5207.02 520702 -124 127 -0.02 -2 +322 101 10312 99223 0.96696 297.96696 149.46696 14946.69669 0.96696 297.96698 149.46696 14946.69674 0.96696 297.96696 149.46696 14946.69600 2020-01-01 2020-01-02 2020-01-01 00:05:22 2020-01-02 03:33:43 2020-01-01 00:05:22.000 2020-01-02 03:33:43.000 322 99223 49772.5 4977250 322 99223 49772.5 4977250 -32247 32688 5208.02 520802 -128 127 -1.58 -158 +323 101 10313 99224 0.96996 297.96996 149.46996 14946.99699 0.96997 297.96997 149.46997 14946.99706 0.96996 297.96996 149.46996 14946.99600 2020-01-01 2020-01-02 2020-01-01 00:05:23 2020-01-02 03:33:44 2020-01-01 00:05:23.000 2020-01-02 03:33:44.000 323 99224 49773.5 4977350 323 99224 49773.5 4977350 -32246 32689 5209.02 520902 -128 123 -3.14 -314 +324 101 10314 99225 0.97297 297.97297 149.47297 14947.29729 0.97297 297.97296 149.47297 14947.29737 0.97297 297.97297 149.47297 14947.29700 2020-01-01 2020-01-02 2020-01-01 00:05:24 2020-01-02 03:33:45 2020-01-01 00:05:24.000 2020-01-02 03:33:45.000 324 99225 49774.5 4977450 324 99225 49774.5 4977450 -32245 32690 5210.02 521002 -127 124 -2.14 -214 325 101 10315 99226 0.97597 297.97597 149.47597 14947.59759 0.97597 297.97598 149.47597 14947.59794 0.97597 297.97597 149.47597 14947.59700 2020-01-01 2020-01-02 2020-01-01 00:05:25 2020-01-02 03:33:46 2020-01-01 00:05:25.000 2020-01-02 03:33:46.000 325 99226 49775.5 4977550 325 99226 49775.5 4977550 -32244 32691 5211.02 521102 -126 125 -1.14 -114 -326 101 10316 99227 0.97897 297.97897 149.47897 14947.89789 0.97897 297.97897 149.47897 14947.8976 0.97897 297.97897 149.4789700000001 14947.89700 2020-01-01 2020-01-02 2020-01-01 00:05:26 2020-01-02 03:33:47 2020-01-01 00:05:26.000 2020-01-02 03:33:47.000 326 99227 49776.5 4977650 326 99227 49776.5 4977650 -32243 32692 5212.02 521202 -125 126 -0.14 -14 +326 101 10316 99227 0.97897 297.97897 149.47897 14947.89789 0.97897 297.97897 149.47897 14947.8976 0.97897 297.97897 149.47897 14947.89700 2020-01-01 2020-01-02 2020-01-01 00:05:26 2020-01-02 03:33:47 2020-01-01 00:05:26.000 2020-01-02 03:33:47.000 326 99227 49776.5 4977650 326 99227 49776.5 4977650 -32243 32692 5212.02 521202 -125 126 -0.14 -14 327 101 10317 99228 0.98198 297.98198 149.48198 14948.19819 0.98198 297.982 149.48198 14948.19821 0.98198 297.98198 149.48198 14948.19800 2020-01-01 2020-01-02 2020-01-01 00:05:27 2020-01-02 03:33:48 2020-01-01 00:05:27.000 2020-01-02 03:33:48.000 327 99228 49777.5 4977750 327 99228 49777.5 4977750 -32242 32693 5213.02 521302 -124 127 0.86 86 -328 101 10318 99229 0.98498 297.98498 149.48498 14948.49849 0.98498 297.985 149.48498 14948.49853 0.98498 297.98498 149.48497999999987 14948.49800 2020-01-01 2020-01-02 2020-01-01 00:05:28 2020-01-02 03:33:49 2020-01-01 00:05:28.000 2020-01-02 03:33:49.000 328 99229 49778.5 4977850 328 99229 49778.5 4977850 -32241 32694 5214.02 521402 -128 127 -0.7 -70 -329 101 10319 99230 0.98798 297.98798 149.48798 14948.79879 0.98798 297.98798 149.48798 14948.79883 0.98798 297.98798 149.4879799999999 14948.79800 2020-01-01 2020-01-02 2020-01-01 00:05:29 2020-01-02 03:33:50 2020-01-01 00:05:29.000 2020-01-02 03:33:50.000 329 99230 49779.5 4977950 329 99230 49779.5 4977950 -32240 32695 5215.02 521502 -128 127 -2.26 -226 -33 102 10023 99933 0.09909 300.09909 150.09909 15160.009 0.09909 300.0991 150.0991 15160.00913 0.09909 300.09909 150.09908999999982 15160.00809 2020-01-01 2020-01-02 2020-01-01 00:00:33 2020-01-02 03:45:33 2020-01-01 00:00:33.000 2020-01-02 03:45:33.000 33 99933 49983 5048283 33 99933 49983 5048283 -32536 32399 4562.009900990099 460763 -125 126 -0.36633663366336633 -37 -330 101 10320 99231 0.99099 297.99099 149.49099 14949.09909 0.99099 297.991 149.49099 14949.09941 0.99099 297.99099 149.49099000000012 14949.09900 2020-01-01 2020-01-02 2020-01-01 00:05:30 2020-01-02 03:33:51 2020-01-01 00:05:30.000 2020-01-02 03:33:51.000 330 99231 49780.5 4978050 330 99231 49780.5 4978050 -32239 32696 5216.02 521602 -128 123 -3.82 -382 -331 101 10321 99232 0.99399 297.99399 149.49399 14949.39939 0.99399 297.994 149.49399 14949.39911 0.99399 297.99399 149.49399000000022 14949.39900 2020-01-01 2020-01-02 2020-01-01 00:05:31 2020-01-02 03:33:52 2020-01-01 00:05:31.000 2020-01-02 03:33:52.000 331 99232 49781.5 4978150 331 99232 49781.5 4978150 -32238 32697 5217.02 521702 -127 124 -2.82 -282 -332 101 10322 99233 0.99699 297.99699 149.49699 14949.69969 0.99699 297.997 149.49699 14949.69969 0.99699 297.99699 149.49698999999978 14949.69900 2020-01-01 2020-01-02 2020-01-01 00:05:32 2020-01-02 03:33:53 2020-01-01 00:05:32.000 2020-01-02 03:33:53.000 332 99233 49782.5 4978250 332 99233 49782.5 4978250 -32237 32698 5218.02 521802 -126 125 -1.82 -182 +328 101 10318 99229 0.98498 297.98498 149.48498 14948.49849 0.98498 297.985 149.48498 14948.49853 0.98498 297.98498 149.48498 14948.49800 2020-01-01 2020-01-02 2020-01-01 00:05:28 2020-01-02 03:33:49 2020-01-01 00:05:28.000 2020-01-02 03:33:49.000 328 99229 49778.5 4977850 328 99229 49778.5 4977850 -32241 32694 5214.02 521402 -128 127 -0.7 -70 +329 101 10319 99230 0.98798 297.98798 149.48798 14948.79879 0.98798 297.98798 149.48798 14948.79883 0.98798 297.98798 149.48798 14948.79800 2020-01-01 2020-01-02 2020-01-01 00:05:29 2020-01-02 03:33:50 2020-01-01 00:05:29.000 2020-01-02 03:33:50.000 329 99230 49779.5 4977950 329 99230 49779.5 4977950 -32240 32695 5215.02 521502 -128 127 -2.26 -226 +33 102 10023 99933 0.09909 300.09909 150.09909 15160.009 0.09909 300.0991 150.0991 15160.00913 0.09909 300.09909 150.09909 15160.00809 2020-01-01 2020-01-02 2020-01-01 00:00:33 2020-01-02 03:45:33 2020-01-01 00:00:33.000 2020-01-02 03:45:33.000 33 99933 49983 5048283 33 99933 49983 5048283 -32536 32399 4562.009900990099 460763 -125 126 -0.36633663366336633 -37 +330 101 10320 99231 0.99099 297.99099 149.49099 14949.09909 0.99099 297.991 149.49099 14949.09941 0.99099 297.99099 149.49099 14949.09900 2020-01-01 2020-01-02 2020-01-01 00:05:30 2020-01-02 03:33:51 2020-01-01 00:05:30.000 2020-01-02 03:33:51.000 330 99231 49780.5 4978050 330 99231 49780.5 4978050 -32239 32696 5216.02 521602 -128 123 -3.82 -382 +331 101 10321 99232 0.99399 297.99399 149.49399 14949.39939 0.99399 297.994 149.49399 14949.39911 0.99399 297.99399 149.49399 14949.39900 2020-01-01 2020-01-02 2020-01-01 00:05:31 2020-01-02 03:33:52 2020-01-01 00:05:31.000 2020-01-02 03:33:52.000 331 99232 49781.5 4978150 331 99232 49781.5 4978150 -32238 32697 5217.02 521702 -127 124 -2.82 -282 +332 101 10322 99233 0.99699 297.99699 149.49699 14949.69969 0.99699 297.997 149.49699 14949.69969 0.99699 297.99699 149.49699 14949.69900 2020-01-01 2020-01-02 2020-01-01 00:05:32 2020-01-02 03:33:53 2020-01-01 00:05:32.000 2020-01-02 03:33:53.000 332 99233 49782.5 4978250 332 99233 49782.5 4978250 -32237 32698 5218.02 521802 -126 125 -1.82 -182 333 101 10323 99234 1 298 149.5 14950 1 298 149.5 14950 1.00000 298.00000 149.5 14950.00000 2020-01-01 2020-01-02 2020-01-01 00:05:33 2020-01-02 03:33:54 2020-01-01 00:05:33.000 2020-01-02 03:33:54.000 333 99234 49783.5 4978350 333 99234 49783.5 4978350 -32236 32699 5219.02 521902 -125 126 -0.82 -82 -334 101 10324 99235 1.003 298.003 149.503 14950.3003 1.003 298.003 149.503 14950.30029 1.00300 298.00300 149.50300000000013 14950.30000 2020-01-01 2020-01-02 2020-01-01 00:05:34 2020-01-02 03:33:55 2020-01-01 00:05:34.000 2020-01-02 03:33:55.000 334 99235 49784.5 4978450 334 99235 49784.5 4978450 -32235 32700 5220.02 522002 -124 127 0.18 18 -335 101 10325 99236 1.006 298.006 149.506 14950.6006 1.006 298.006 149.506 14950.60088 1.00600 298.00600 149.5059999999999 14950.60000 2020-01-01 2020-01-02 2020-01-01 00:05:35 2020-01-02 03:33:56 2020-01-01 00:05:35.000 2020-01-02 03:33:56.000 335 99236 49785.5 4978550 335 99236 49785.5 4978550 -32234 32701 5221.02 522102 -128 127 -1.38 -138 -336 101 10326 99237 1.009 298.009 149.509 14950.9009 1.009 298.009 149.509 14950.90057 1.00900 298.00900 149.50900000000001 14950.90000 2020-01-01 2020-01-02 2020-01-01 00:05:36 2020-01-02 03:33:57 2020-01-01 00:05:36.000 2020-01-02 03:33:57.000 336 99237 49786.5 4978650 336 99237 49786.5 4978650 -32233 32702 5222.02 522202 -128 123 -2.94 -294 -337 101 10327 99238 1.01201 298.01201 149.51201 14951.2012 1.01201 298.01202 149.51201 14951.20117 1.01201 298.01201 149.51201000000023 14951.20100 2020-01-01 2020-01-02 2020-01-01 00:05:37 2020-01-02 03:33:58 2020-01-01 00:05:37.000 2020-01-02 03:33:58.000 337 99238 49787.5 4978750 337 99238 49787.5 4978750 -32232 32703 5223.02 522302 -127 124 -1.94 -194 -338 101 10328 99239 1.01501 298.01501 149.51501 14951.5015 1.01501 298.015 149.51501 14951.50146 1.01501 298.01501 149.5150099999998 14951.50100 2020-01-01 2020-01-02 2020-01-01 00:05:38 2020-01-02 03:33:59 2020-01-01 00:05:38.000 2020-01-02 03:33:59.000 338 99239 49788.5 4978850 338 99239 49788.5 4978850 -32231 32704 5224.02 522402 -126 125 -0.94 -94 -339 101 10329 99240 1.01801 298.01801 149.51801 14951.8018 1.01801 298.018 149.51801 14951.80177 1.01801 298.01801 149.5180099999998 14951.80100 2020-01-01 2020-01-02 2020-01-01 00:05:39 2020-01-02 03:34:00 2020-01-01 00:05:39.000 2020-01-02 03:34:00.000 339 99240 49789.5 4978950 339 99240 49789.5 4978950 -32230 32705 5225.02 522502 -125 126 0.06 6 -34 102 10024 99934 0.1021 300.1021 150.1021 15160.31231 0.1021 300.1021 150.1021 15160.31224 0.10210 300.10210 150.10210000000004 15160.31210 2020-01-01 2020-01-02 2020-01-01 00:00:34 2020-01-02 03:45:34 2020-01-01 00:00:34.000 2020-01-02 03:45:34.000 34 99934 49984 5048384 34 99934 49984 5048384 -32535 32400 4563.009900990099 460864 -124 127 0.6336633663366337 64 -340 101 10330 99241 1.02102 298.02102 149.52102 14952.1021 1.02102 298.02103 149.52102 14952.10239 1.02102 298.02102 149.5210200000001 14952.10200 2020-01-01 2020-01-02 2020-01-01 00:05:40 2020-01-02 03:34:01 2020-01-01 00:05:40.000 2020-01-02 03:34:01.000 340 99241 49790.5 4979050 340 99241 49790.5 4979050 -32229 32706 5226.02 522602 -124 127 1.06 106 -341 101 10331 99242 1.02402 298.02402 149.52402 14952.4024 1.02402 298.02402 149.52402 14952.40205 1.02402 298.02402 149.52402000000015 14952.40200 2020-01-01 2020-01-02 2020-01-01 00:05:41 2020-01-02 03:34:02 2020-01-01 00:05:41.000 2020-01-02 03:34:02.000 341 99242 49791.5 4979150 341 99242 49791.5 4979150 -32228 32707 5227.02 522702 -128 127 -0.5 -50 -342 101 10332 99243 1.02702 298.02702 149.52702 14952.7027 1.02702 298.02704 149.52702 14952.70264 1.02702 298.02702 149.5270199999997 14952.70200 2020-01-01 2020-01-02 2020-01-01 00:05:42 2020-01-02 03:34:03 2020-01-01 00:05:42.000 2020-01-02 03:34:03.000 342 99243 49792.5 4979250 342 99243 49792.5 4979250 -32227 32708 5228.02 522802 -128 123 -2.06 -206 -343 101 10333 99244 1.03003 298.03003 149.53003 14953.003 1.03003 298.03003 149.53002 14953.00293 1.03003 298.03003 149.53002999999993 14953.00300 2020-01-01 2020-01-02 2020-01-01 00:05:43 2020-01-02 03:34:04 2020-01-01 00:05:43.000 2020-01-02 03:34:04.000 343 99244 49793.5 4979350 343 99244 49793.5 4979350 -32226 32709 5229.02 522902 -127 124 -1.06 -106 -344 101 10334 99245 1.03303 298.03303 149.53303 14953.3033 1.03303 298.03302 149.53303 14953.30323 1.03303 298.03303 149.53303000000002 14953.30300 2020-01-01 2020-01-02 2020-01-01 00:05:44 2020-01-02 03:34:05 2020-01-01 00:05:44.000 2020-01-02 03:34:05.000 344 99245 49794.5 4979450 344 99245 49794.5 4979450 -32225 32710 5230.02 523002 -126 125 -0.06 -6 -345 101 10335 99246 1.03603 298.03603 149.53603 14953.6036 1.03603 298.03604 149.53603 14953.60386 1.03603 298.03603 149.5360299999999 14953.60300 2020-01-01 2020-01-02 2020-01-01 00:05:45 2020-01-02 03:34:06 2020-01-01 00:05:45.000 2020-01-02 03:34:06.000 345 99246 49795.5 4979550 345 99246 49795.5 4979550 -32224 32711 5231.02 523102 -125 126 0.94 94 -346 101 10336 99247 1.03903 298.03903 149.53903 14953.9039 1.03903 298.03903 149.53903 14953.90352 1.03903 298.03903 149.53902999999994 14953.90300 2020-01-01 2020-01-02 2020-01-01 00:05:46 2020-01-02 03:34:07 2020-01-01 00:05:46.000 2020-01-02 03:34:07.000 346 99247 49796.5 4979650 346 99247 49796.5 4979650 -32223 32712 5232.02 523202 -124 127 1.94 194 -347 101 10337 99248 1.04204 298.04204 149.54204 14954.2042 1.04204 298.04205 149.54204 14954.20427 1.04204 298.04204 149.54204000000016 14954.20400 2020-01-01 2020-01-02 2020-01-01 00:05:47 2020-01-02 03:34:08 2020-01-01 00:05:47.000 2020-01-02 03:34:08.000 347 99248 49797.5 4979750 347 99248 49797.5 4979750 -32222 32713 5233.02 523302 -128 127 0.38 38 -348 101 10338 99249 1.04504 298.04504 149.54504 14954.5045 1.04504 298.04504 149.54504 14954.50441 1.04504 298.04504 149.54504000000026 14954.50400 2020-01-01 2020-01-02 2020-01-01 00:05:48 2020-01-02 03:34:09 2020-01-01 00:05:48.000 2020-01-02 03:34:09.000 348 99249 49798.5 4979850 348 99249 49798.5 4979850 -32221 32714 5234.02 523402 -128 123 -1.18 -118 -349 101 10339 99250 1.04804 298.04804 149.54804 14954.8048 1.04804 298.04803 149.54804 14954.80474 1.04804 298.04804 149.54803999999982 14954.80400 2020-01-01 2020-01-02 2020-01-01 00:05:49 2020-01-02 03:34:10 2020-01-01 00:05:49.000 2020-01-02 03:34:10.000 349 99250 49799.5 4979950 349 99250 49799.5 4979950 -32220 32715 5235.02 523502 -127 124 -0.18 -18 -35 102 10025 99935 0.1051 300.1051 150.1051 15160.61561 0.1051 300.1051 150.1051 15160.61542 0.10510 300.10510 150.10510000000014 15160.61510 2020-01-01 2020-01-02 2020-01-01 00:00:35 2020-01-02 03:45:35 2020-01-01 00:00:35.000 2020-01-02 03:45:35.000 35 99935 49985 5048485 35 99935 49985 5048485 -32534 32401 4564.009900990099 460965 -128 127 -0.900990099009901 -91 +334 101 10324 99235 1.003 298.003 149.503 14950.3003 1.003 298.003 149.503 14950.30029 1.00300 298.00300 149.503 14950.30000 2020-01-01 2020-01-02 2020-01-01 00:05:34 2020-01-02 03:33:55 2020-01-01 00:05:34.000 2020-01-02 03:33:55.000 334 99235 49784.5 4978450 334 99235 49784.5 4978450 -32235 32700 5220.02 522002 -124 127 0.18 18 +335 101 10325 99236 1.006 298.006 149.506 14950.6006 1.006 298.006 149.506 14950.60088 1.00600 298.00600 149.506 14950.60000 2020-01-01 2020-01-02 2020-01-01 00:05:35 2020-01-02 03:33:56 2020-01-01 00:05:35.000 2020-01-02 03:33:56.000 335 99236 49785.5 4978550 335 99236 49785.5 4978550 -32234 32701 5221.02 522102 -128 127 -1.38 -138 +336 101 10326 99237 1.009 298.009 149.509 14950.9009 1.009 298.009 149.509 14950.90057 1.00900 298.00900 149.509 14950.90000 2020-01-01 2020-01-02 2020-01-01 00:05:36 2020-01-02 03:33:57 2020-01-01 00:05:36.000 2020-01-02 03:33:57.000 336 99237 49786.5 4978650 336 99237 49786.5 4978650 -32233 32702 5222.02 522202 -128 123 -2.94 -294 +337 101 10327 99238 1.01201 298.01201 149.51201 14951.2012 1.01201 298.01202 149.51201 14951.20117 1.01201 298.01201 149.51201 14951.20100 2020-01-01 2020-01-02 2020-01-01 00:05:37 2020-01-02 03:33:58 2020-01-01 00:05:37.000 2020-01-02 03:33:58.000 337 99238 49787.5 4978750 337 99238 49787.5 4978750 -32232 32703 5223.02 522302 -127 124 -1.94 -194 +338 101 10328 99239 1.01501 298.01501 149.51501 14951.5015 1.01501 298.015 149.51501 14951.50146 1.01501 298.01501 149.51501 14951.50100 2020-01-01 2020-01-02 2020-01-01 00:05:38 2020-01-02 03:33:59 2020-01-01 00:05:38.000 2020-01-02 03:33:59.000 338 99239 49788.5 4978850 338 99239 49788.5 4978850 -32231 32704 5224.02 522402 -126 125 -0.94 -94 +339 101 10329 99240 1.01801 298.01801 149.51801 14951.8018 1.01801 298.018 149.51801 14951.80177 1.01801 298.01801 149.51801 14951.80100 2020-01-01 2020-01-02 2020-01-01 00:05:39 2020-01-02 03:34:00 2020-01-01 00:05:39.000 2020-01-02 03:34:00.000 339 99240 49789.5 4978950 339 99240 49789.5 4978950 -32230 32705 5225.02 522502 -125 126 0.06 6 +34 102 10024 99934 0.1021 300.1021 150.1021 15160.31231 0.1021 300.1021 150.1021 15160.31224 0.10210 300.10210 150.10209999999998 15160.31210 2020-01-01 2020-01-02 2020-01-01 00:00:34 2020-01-02 03:45:34 2020-01-01 00:00:34.000 2020-01-02 03:45:34.000 34 99934 49984 5048384 34 99934 49984 5048384 -32535 32400 4563.009900990099 460864 -124 127 0.6336633663366337 64 +340 101 10330 99241 1.02102 298.02102 149.52102 14952.1021 1.02102 298.02103 149.52102 14952.10239 1.02102 298.02102 149.52102000000002 14952.10200 2020-01-01 2020-01-02 2020-01-01 00:05:40 2020-01-02 03:34:01 2020-01-01 00:05:40.000 2020-01-02 03:34:01.000 340 99241 49790.5 4979050 340 99241 49790.5 4979050 -32229 32706 5226.02 522602 -124 127 1.06 106 +341 101 10331 99242 1.02402 298.02402 149.52402 14952.4024 1.02402 298.02402 149.52402 14952.40205 1.02402 298.02402 149.52402 14952.40200 2020-01-01 2020-01-02 2020-01-01 00:05:41 2020-01-02 03:34:02 2020-01-01 00:05:41.000 2020-01-02 03:34:02.000 341 99242 49791.5 4979150 341 99242 49791.5 4979150 -32228 32707 5227.02 522702 -128 127 -0.5 -50 +342 101 10332 99243 1.02702 298.02702 149.52702 14952.7027 1.02702 298.02704 149.52702 14952.70264 1.02702 298.02702 149.52702 14952.70200 2020-01-01 2020-01-02 2020-01-01 00:05:42 2020-01-02 03:34:03 2020-01-01 00:05:42.000 2020-01-02 03:34:03.000 342 99243 49792.5 4979250 342 99243 49792.5 4979250 -32227 32708 5228.02 522802 -128 123 -2.06 -206 +343 101 10333 99244 1.03003 298.03003 149.53003 14953.003 1.03003 298.03003 149.53002 14953.00293 1.03003 298.03003 149.53003 14953.00300 2020-01-01 2020-01-02 2020-01-01 00:05:43 2020-01-02 03:34:04 2020-01-01 00:05:43.000 2020-01-02 03:34:04.000 343 99244 49793.5 4979350 343 99244 49793.5 4979350 -32226 32709 5229.02 522902 -127 124 -1.06 -106 +344 101 10334 99245 1.03303 298.03303 149.53303 14953.3033 1.03303 298.03302 149.53303 14953.30323 1.03303 298.03303 149.53303 14953.30300 2020-01-01 2020-01-02 2020-01-01 00:05:44 2020-01-02 03:34:05 2020-01-01 00:05:44.000 2020-01-02 03:34:05.000 344 99245 49794.5 4979450 344 99245 49794.5 4979450 -32225 32710 5230.02 523002 -126 125 -0.06 -6 +345 101 10335 99246 1.03603 298.03603 149.53603 14953.6036 1.03603 298.03604 149.53603 14953.60386 1.03603 298.03603 149.53602999999998 14953.60300 2020-01-01 2020-01-02 2020-01-01 00:05:45 2020-01-02 03:34:06 2020-01-01 00:05:45.000 2020-01-02 03:34:06.000 345 99246 49795.5 4979550 345 99246 49795.5 4979550 -32224 32711 5231.02 523102 -125 126 0.94 94 +346 101 10336 99247 1.03903 298.03903 149.53903 14953.9039 1.03903 298.03903 149.53903 14953.90352 1.03903 298.03903 149.53903 14953.90300 2020-01-01 2020-01-02 2020-01-01 00:05:46 2020-01-02 03:34:07 2020-01-01 00:05:46.000 2020-01-02 03:34:07.000 346 99247 49796.5 4979650 346 99247 49796.5 4979650 -32223 32712 5232.02 523202 -124 127 1.94 194 +347 101 10337 99248 1.04204 298.04204 149.54204 14954.2042 1.04204 298.04205 149.54204 14954.20427 1.04204 298.04204 149.54204 14954.20400 2020-01-01 2020-01-02 2020-01-01 00:05:47 2020-01-02 03:34:08 2020-01-01 00:05:47.000 2020-01-02 03:34:08.000 347 99248 49797.5 4979750 347 99248 49797.5 4979750 -32222 32713 5233.02 523302 -128 127 0.38 38 +348 101 10338 99249 1.04504 298.04504 149.54504 14954.5045 1.04504 298.04504 149.54504 14954.50441 1.04504 298.04504 149.54504 14954.50400 2020-01-01 2020-01-02 2020-01-01 00:05:48 2020-01-02 03:34:09 2020-01-01 00:05:48.000 2020-01-02 03:34:09.000 348 99249 49798.5 4979850 348 99249 49798.5 4979850 -32221 32714 5234.02 523402 -128 123 -1.18 -118 +349 101 10339 99250 1.04804 298.04804 149.54804 14954.8048 1.04804 298.04803 149.54804 14954.80474 1.04804 298.04804 149.54804000000001 14954.80400 2020-01-01 2020-01-02 2020-01-01 00:05:49 2020-01-02 03:34:10 2020-01-01 00:05:49.000 2020-01-02 03:34:10.000 349 99250 49799.5 4979950 349 99250 49799.5 4979950 -32220 32715 5235.02 523502 -127 124 -0.18 -18 +35 102 10025 99935 0.1051 300.1051 150.1051 15160.61561 0.1051 300.1051 150.1051 15160.61542 0.10510 300.10510 150.10510000000002 15160.61510 2020-01-01 2020-01-02 2020-01-01 00:00:35 2020-01-02 03:45:35 2020-01-01 00:00:35.000 2020-01-02 03:45:35.000 35 99935 49985 5048485 35 99935 49985 5048485 -32534 32401 4564.009900990099 460965 -128 127 -0.900990099009901 -91 350 101 10340 99251 1.05105 298.05105 149.55105 14955.1051 1.05105 298.05106 149.55105 14955.10532 1.05105 298.05105 149.55105 14955.10500 2020-01-01 2020-01-02 2020-01-01 00:05:50 2020-01-02 03:34:11 2020-01-01 00:05:50.000 2020-01-02 03:34:11.000 350 99251 49800.5 4980050 350 99251 49800.5 4980050 -32219 32716 5236.02 523602 -126 125 0.82 82 -351 101 10341 99252 1.05405 298.05405 149.55405 14955.4054 1.05405 298.05405 149.55404 14955.40499 1.05405 298.05405 149.55405000000016 14955.40500 2020-01-01 2020-01-02 2020-01-01 00:05:51 2020-01-02 03:34:12 2020-01-01 00:05:51.000 2020-01-02 03:34:12.000 351 99252 49801.5 4980150 351 99252 49801.5 4980150 -32218 32717 5237.02 523702 -125 126 1.82 182 -352 101 10342 99253 1.05705 298.05705 149.55705 14955.7057 1.05705 298.05707 149.55705 14955.70574 1.05705 298.05705 149.5570500000001 14955.70500 2020-01-01 2020-01-02 2020-01-01 00:05:52 2020-01-02 03:34:13 2020-01-01 00:05:52.000 2020-01-02 03:34:13.000 352 99253 49802.5 4980250 352 99253 49802.5 4980250 -32217 32718 5238.02 523802 -124 127 2.82 282 -353 101 10343 99254 1.06006 298.06006 149.56006 14956.006 1.06006 298.06006 149.56005 14956.00587 1.06006 298.06006 149.5600599999999 14956.00600 2020-01-01 2020-01-02 2020-01-01 00:05:53 2020-01-02 03:34:14 2020-01-01 00:05:53.000 2020-01-02 03:34:14.000 353 99254 49803.5 4980350 353 99254 49803.5 4980350 -32216 32719 5239.02 523902 -128 127 1.26 126 -354 101 10344 99255 1.06306 298.06306 149.56306 14956.3063 1.06306 298.06305 149.56306 14956.3062 1.06306 298.06306 149.56306000000012 14956.30600 2020-01-01 2020-01-02 2020-01-01 00:05:54 2020-01-02 03:34:15 2020-01-01 00:05:54.000 2020-01-02 03:34:15.000 354 99255 49804.5 4980450 354 99255 49804.5 4980450 -32215 32720 5240.02 524002 -128 127 -0.3 -30 -355 101 10345 99256 1.06606 298.06606 149.56606 14956.6066 1.06606 298.06607 149.56606 14956.6068 1.06606 298.06606 149.56605999999994 14956.60600 2020-01-01 2020-01-02 2020-01-01 00:05:55 2020-01-02 03:34:16 2020-01-01 00:05:55.000 2020-01-02 03:34:16.000 355 99256 49805.5 4980550 355 99256 49805.5 4980550 -32214 32721 5241.02 524102 -128 123 -1.86 -186 -356 101 10346 99257 1.06906 298.06906 149.56906 14956.9069 1.06906 298.06906 149.56907 14956.90709 1.06906 298.06906 149.56905999999992 14956.90600 2020-01-01 2020-01-02 2020-01-01 00:05:56 2020-01-02 03:34:17 2020-01-01 00:05:56.000 2020-01-02 03:34:17.000 356 99257 49806.5 4980650 356 99257 49806.5 4980650 -32213 32722 5242.02 524202 -127 124 -0.86 -86 -357 101 10347 99258 1.07207 298.07207 149.57207 14957.2072 1.07207 298.07208 149.57207 14957.20721 1.07207 298.07207 149.57207000000008 14957.20700 2020-01-01 2020-01-02 2020-01-01 00:05:57 2020-01-02 03:34:18 2020-01-01 00:05:57.000 2020-01-02 03:34:18.000 357 99258 49807.5 4980750 357 99258 49807.5 4980750 -32212 32723 5243.02 524302 -126 125 0.14 14 -358 101 10348 99259 1.07507 298.07507 149.57507 14957.5075 1.07507 298.07507 149.57507 14957.50734 1.07507 298.07507 149.57507000000004 14957.50700 2020-01-01 2020-01-02 2020-01-01 00:05:58 2020-01-02 03:34:19 2020-01-01 00:05:58.000 2020-01-02 03:34:19.000 358 99259 49808.5 4980850 358 99259 49808.5 4980850 -32211 32724 5244.02 524402 -125 126 1.14 114 -359 101 10349 99260 1.07807 298.07807 149.57807 14957.8078 1.07807 298.07806 149.57807 14957.80767 1.07807 298.07807 149.57806999999985 14957.80700 2020-01-01 2020-01-02 2020-01-01 00:05:59 2020-01-02 03:34:20 2020-01-01 00:05:59.000 2020-01-02 03:34:20.000 359 99260 49809.5 4980950 359 99260 49809.5 4980950 -32210 32725 5245.02 524502 -124 127 2.14 214 -36 102 10026 99936 0.1081 300.1081 150.1081 15160.91891 0.1081 300.1081 150.1081 15160.91873 0.10810 300.10810 150.10809999999998 15160.91810 2020-01-01 2020-01-02 2020-01-01 00:00:36 2020-01-02 03:45:36 2020-01-01 00:00:36.000 2020-01-02 03:45:36.000 36 99936 49986 5048586 36 99936 49986 5048586 -32533 32402 4565.009900990099 461066 -128 123 -2.4356435643564356 -246 -360 101 10350 99261 1.08108 298.08108 149.58108 14958.1081 1.08108 298.0811 149.58108 14958.10827 1.08108 298.08108 149.58107999999996 14958.10800 2020-01-01 2020-01-02 2020-01-01 00:06:00 2020-01-02 03:34:21 2020-01-01 00:06:00.000 2020-01-02 03:34:21.000 360 99261 49810.5 4981050 360 99261 49810.5 4981050 -32209 32726 5246.02 524602 -128 127 0.58 58 +351 101 10341 99252 1.05405 298.05405 149.55405 14955.4054 1.05405 298.05405 149.55404 14955.40499 1.05405 298.05405 149.55405000000002 14955.40500 2020-01-01 2020-01-02 2020-01-01 00:05:51 2020-01-02 03:34:12 2020-01-01 00:05:51.000 2020-01-02 03:34:12.000 351 99252 49801.5 4980150 351 99252 49801.5 4980150 -32218 32717 5237.02 523702 -125 126 1.82 182 +352 101 10342 99253 1.05705 298.05705 149.55705 14955.7057 1.05705 298.05707 149.55705 14955.70574 1.05705 298.05705 149.55705 14955.70500 2020-01-01 2020-01-02 2020-01-01 00:05:52 2020-01-02 03:34:13 2020-01-01 00:05:52.000 2020-01-02 03:34:13.000 352 99253 49802.5 4980250 352 99253 49802.5 4980250 -32217 32718 5238.02 523802 -124 127 2.82 282 +353 101 10343 99254 1.06006 298.06006 149.56006 14956.006 1.06006 298.06006 149.56005 14956.00587 1.06006 298.06006 149.56006 14956.00600 2020-01-01 2020-01-02 2020-01-01 00:05:53 2020-01-02 03:34:14 2020-01-01 00:05:53.000 2020-01-02 03:34:14.000 353 99254 49803.5 4980350 353 99254 49803.5 4980350 -32216 32719 5239.02 523902 -128 127 1.26 126 +354 101 10344 99255 1.06306 298.06306 149.56306 14956.3063 1.06306 298.06305 149.56306 14956.3062 1.06306 298.06306 149.56306 14956.30600 2020-01-01 2020-01-02 2020-01-01 00:05:54 2020-01-02 03:34:15 2020-01-01 00:05:54.000 2020-01-02 03:34:15.000 354 99255 49804.5 4980450 354 99255 49804.5 4980450 -32215 32720 5240.02 524002 -128 127 -0.3 -30 +355 101 10345 99256 1.06606 298.06606 149.56606 14956.6066 1.06606 298.06607 149.56606 14956.6068 1.06606 298.06606 149.56606 14956.60600 2020-01-01 2020-01-02 2020-01-01 00:05:55 2020-01-02 03:34:16 2020-01-01 00:05:55.000 2020-01-02 03:34:16.000 355 99256 49805.5 4980550 355 99256 49805.5 4980550 -32214 32721 5241.02 524102 -128 123 -1.86 -186 +356 101 10346 99257 1.06906 298.06906 149.56906 14956.9069 1.06906 298.06906 149.56907 14956.90709 1.06906 298.06906 149.56906 14956.90600 2020-01-01 2020-01-02 2020-01-01 00:05:56 2020-01-02 03:34:17 2020-01-01 00:05:56.000 2020-01-02 03:34:17.000 356 99257 49806.5 4980650 356 99257 49806.5 4980650 -32213 32722 5242.02 524202 -127 124 -0.86 -86 +357 101 10347 99258 1.07207 298.07207 149.57207 14957.2072 1.07207 298.07208 149.57207 14957.20721 1.07207 298.07207 149.57207 14957.20700 2020-01-01 2020-01-02 2020-01-01 00:05:57 2020-01-02 03:34:18 2020-01-01 00:05:57.000 2020-01-02 03:34:18.000 357 99258 49807.5 4980750 357 99258 49807.5 4980750 -32212 32723 5243.02 524302 -126 125 0.14 14 +358 101 10348 99259 1.07507 298.07507 149.57507 14957.5075 1.07507 298.07507 149.57507 14957.50734 1.07507 298.07507 149.57506999999998 14957.50700 2020-01-01 2020-01-02 2020-01-01 00:05:58 2020-01-02 03:34:19 2020-01-01 00:05:58.000 2020-01-02 03:34:19.000 358 99259 49808.5 4980850 358 99259 49808.5 4980850 -32211 32724 5244.02 524402 -125 126 1.14 114 +359 101 10349 99260 1.07807 298.07807 149.57807 14957.8078 1.07807 298.07806 149.57807 14957.80767 1.07807 298.07807 149.57807 14957.80700 2020-01-01 2020-01-02 2020-01-01 00:05:59 2020-01-02 03:34:20 2020-01-01 00:05:59.000 2020-01-02 03:34:20.000 359 99260 49809.5 4980950 359 99260 49809.5 4980950 -32210 32725 5245.02 524502 -124 127 2.14 214 +36 102 10026 99936 0.1081 300.1081 150.1081 15160.91891 0.1081 300.1081 150.1081 15160.91873 0.10810 300.10810 150.1081 15160.91810 2020-01-01 2020-01-02 2020-01-01 00:00:36 2020-01-02 03:45:36 2020-01-01 00:00:36.000 2020-01-02 03:45:36.000 36 99936 49986 5048586 36 99936 49986 5048586 -32533 32402 4565.009900990099 461066 -128 123 -2.4356435643564356 -246 +360 101 10350 99261 1.08108 298.08108 149.58108 14958.1081 1.08108 298.0811 149.58108 14958.10827 1.08108 298.08108 149.58108000000001 14958.10800 2020-01-01 2020-01-02 2020-01-01 00:06:00 2020-01-02 03:34:21 2020-01-01 00:06:00.000 2020-01-02 03:34:21.000 360 99261 49810.5 4981050 360 99261 49810.5 4981050 -32209 32726 5246.02 524602 -128 127 0.58 58 361 101 10351 99262 1.08408 298.08408 149.58408 14958.4084 1.08408 298.08408 149.58408 14958.40856 1.08408 298.08408 149.58408 14958.40800 2020-01-01 2020-01-02 2020-01-01 00:06:01 2020-01-02 03:34:22 2020-01-01 00:06:01.000 2020-01-02 03:34:22.000 361 99262 49811.5 4981150 361 99262 49811.5 4981150 -32208 32727 5247.02 524702 -128 123 -0.98 -98 -362 101 10352 99263 1.08708 298.08708 149.58708 14958.7087 1.08708 298.0871 149.58708 14958.70868 1.08708 298.08708 149.58707999999996 14958.70800 2020-01-01 2020-01-02 2020-01-01 00:06:02 2020-01-02 03:34:23 2020-01-01 00:06:02.000 2020-01-02 03:34:23.000 362 99263 49812.5 4981250 362 99263 49812.5 4981250 -32207 32728 5248.02 524802 -127 124 0.02 2 -363 101 10353 99264 1.09009 298.09009 149.59009 14959.009 1.09009 298.0901 149.59008 14959.00884 1.09009 298.09009 149.59008999999992 14959.00900 2020-01-01 2020-01-02 2020-01-01 00:06:03 2020-01-02 03:34:24 2020-01-01 00:06:03.000 2020-01-02 03:34:24.000 363 99264 49813.5 4981350 363 99264 49813.5 4981350 -32206 32729 5249.02 524902 -126 125 1.02 102 -364 101 10354 99265 1.09309 298.09309 149.59309 14959.3093 1.09309 298.09308 149.59309 14959.30915 1.09309 298.09309 149.59309000000007 14959.30900 2020-01-01 2020-01-02 2020-01-01 00:06:04 2020-01-02 03:34:25 2020-01-01 00:06:04.000 2020-01-02 03:34:25.000 364 99265 49814.5 4981450 364 99265 49814.5 4981450 -32205 32730 5250.02 525002 -125 126 2.02 202 -365 101 10355 99266 1.09609 298.09609 149.59609 14959.6096 1.09609 298.0961 149.59609 14959.6099 1.09609 298.09609 149.5960899999999 14959.60900 2020-01-01 2020-01-02 2020-01-01 00:06:05 2020-01-02 03:34:26 2020-01-01 00:06:05.000 2020-01-02 03:34:26.000 365 99266 49815.5 4981550 365 99266 49815.5 4981550 -32204 32731 5251.02 525102 -124 127 3.02 302 -366 101 10356 99267 1.09909 298.09909 149.59909 14959.9099 1.09909 298.0991 149.5991 14959.91003 1.09909 298.09909 149.59908999999993 14959.90900 2020-01-01 2020-01-02 2020-01-01 00:06:06 2020-01-02 03:34:27 2020-01-01 00:06:06.000 2020-01-02 03:34:27.000 366 99267 49816.5 4981650 366 99267 49816.5 4981650 -32203 32732 5252.02 525202 -128 127 1.46 146 -367 101 10357 99268 1.1021 298.1021 149.6021 14960.21021 1.1021 298.1021 149.6021 14960.21015 1.10210 298.10210 149.60210000000004 14960.21000 2020-01-01 2020-01-02 2020-01-01 00:06:07 2020-01-02 03:34:28 2020-01-01 00:06:07.000 2020-01-02 03:34:28.000 367 99268 49817.5 4981750 367 99268 49817.5 4981750 -32202 32733 5253.02 525302 -128 123 -0.1 -10 +362 101 10352 99263 1.08708 298.08708 149.58708 14958.7087 1.08708 298.0871 149.58708 14958.70868 1.08708 298.08708 149.58708000000001 14958.70800 2020-01-01 2020-01-02 2020-01-01 00:06:02 2020-01-02 03:34:23 2020-01-01 00:06:02.000 2020-01-02 03:34:23.000 362 99263 49812.5 4981250 362 99263 49812.5 4981250 -32207 32728 5248.02 524802 -127 124 0.02 2 +363 101 10353 99264 1.09009 298.09009 149.59009 14959.009 1.09009 298.0901 149.59008 14959.00884 1.09009 298.09009 149.59009 14959.00900 2020-01-01 2020-01-02 2020-01-01 00:06:03 2020-01-02 03:34:24 2020-01-01 00:06:03.000 2020-01-02 03:34:24.000 363 99264 49813.5 4981350 363 99264 49813.5 4981350 -32206 32729 5249.02 524902 -126 125 1.02 102 +364 101 10354 99265 1.09309 298.09309 149.59309 14959.3093 1.09309 298.09308 149.59309 14959.30915 1.09309 298.09309 149.59309 14959.30900 2020-01-01 2020-01-02 2020-01-01 00:06:04 2020-01-02 03:34:25 2020-01-01 00:06:04.000 2020-01-02 03:34:25.000 364 99265 49814.5 4981450 364 99265 49814.5 4981450 -32205 32730 5250.02 525002 -125 126 2.02 202 +365 101 10355 99266 1.09609 298.09609 149.59609 14959.6096 1.09609 298.0961 149.59609 14959.6099 1.09609 298.09609 149.59609 14959.60900 2020-01-01 2020-01-02 2020-01-01 00:06:05 2020-01-02 03:34:26 2020-01-01 00:06:05.000 2020-01-02 03:34:26.000 365 99266 49815.5 4981550 365 99266 49815.5 4981550 -32204 32731 5251.02 525102 -124 127 3.02 302 +366 101 10356 99267 1.09909 298.09909 149.59909 14959.9099 1.09909 298.0991 149.5991 14959.91003 1.09909 298.09909 149.59909 14959.90900 2020-01-01 2020-01-02 2020-01-01 00:06:06 2020-01-02 03:34:27 2020-01-01 00:06:06.000 2020-01-02 03:34:27.000 366 99267 49816.5 4981650 366 99267 49816.5 4981650 -32203 32732 5252.02 525202 -128 127 1.46 146 +367 101 10357 99268 1.1021 298.1021 149.6021 14960.21021 1.1021 298.1021 149.6021 14960.21015 1.10210 298.10210 149.60209999999998 14960.21000 2020-01-01 2020-01-02 2020-01-01 00:06:07 2020-01-02 03:34:28 2020-01-01 00:06:07.000 2020-01-02 03:34:28.000 367 99268 49817.5 4981750 367 99268 49817.5 4981750 -32202 32733 5253.02 525302 -128 123 -0.1 -10 368 101 10358 99269 1.1051 298.1051 149.6051 14960.51051 1.1051 298.1051 149.6051 14960.51031 1.10510 298.10510 149.6051 14960.51000 2020-01-01 2020-01-02 2020-01-01 00:06:08 2020-01-02 03:34:29 2020-01-01 00:06:08.000 2020-01-02 03:34:29.000 368 99269 49818.5 4981850 368 99269 49818.5 4981850 -32201 32734 5254.02 525402 -127 124 0.9 90 -369 101 10359 99270 1.1081 298.1081 149.6081 14960.81081 1.1081 298.1081 149.6081 14960.81062 1.10810 298.10810 149.60809999999998 14960.81000 2020-01-01 2020-01-02 2020-01-01 00:06:09 2020-01-02 03:34:30 2020-01-01 00:06:09.000 2020-01-02 03:34:30.000 369 99270 49819.5 4981950 369 99270 49819.5 4981950 -32200 32735 5255.02 525502 -126 125 1.9 190 -37 102 10027 99937 0.11111 300.11111 150.11111 15161.22222 0.11111 300.1111 150.11111 15161.22248 0.11111 300.11111 150.1111099999999 15161.22211 2020-01-01 2020-01-02 2020-01-01 00:00:37 2020-01-02 03:45:37 2020-01-01 00:00:37.000 2020-01-02 03:45:37.000 37 99937 49987 5048687 37 99937 49987 5048687 -32532 32403 4566.009900990099 461167 -127 124 -1.4356435643564356 -145 -370 101 10360 99271 1.11111 298.11111 149.61111 14961.11111 1.11111 298.1111 149.61111 14961.11137 1.11111 298.11111 149.6111099999999 14961.11100 2020-01-01 2020-01-02 2020-01-01 00:06:10 2020-01-02 03:34:31 2020-01-01 00:06:10.000 2020-01-02 03:34:31.000 370 99271 49820.5 4982050 370 99271 49820.5 4982050 -32199 32736 5256.02 525602 -125 126 2.9 290 -371 101 10361 99272 1.11411 298.11411 149.61411 14961.41141 1.11411 298.1141 149.61411 14961.4115 1.11411 298.11411 149.61411000000015 14961.41100 2020-01-01 2020-01-02 2020-01-01 00:06:11 2020-01-02 03:34:32 2020-01-01 00:06:11.000 2020-01-02 03:34:32.000 371 99272 49821.5 4982150 371 99272 49821.5 4982150 -32198 32737 5257.02 525702 -124 127 3.9 390 -372 101 10362 99273 1.11711 298.11711 149.61711 14961.71171 1.11711 298.11713 149.61711 14961.71165 1.11711 298.11711 149.61710999999997 14961.71100 2020-01-01 2020-01-02 2020-01-01 00:06:12 2020-01-02 03:34:33 2020-01-01 00:06:12.000 2020-01-02 03:34:33.000 372 99273 49822.5 4982250 372 99273 49822.5 4982250 -32197 32738 5258.02 525802 -128 127 2.34 234 -373 101 10363 99274 1.12012 298.12012 149.62012 14962.01201 1.12012 298.12012 149.62011 14962.01179 1.12012 298.12012 149.62011999999987 14962.01200 2020-01-01 2020-01-02 2020-01-01 00:06:13 2020-01-02 03:34:34 2020-01-01 00:06:13.000 2020-01-02 03:34:34.000 373 99274 49823.5 4982350 373 99274 49823.5 4982350 -32196 32739 5259.02 525902 -128 123 0.78 78 -374 101 10364 99275 1.12312 298.12312 149.62312 14962.31231 1.12312 298.1231 149.62312 14962.31208 1.12312 298.12312 149.62312000000009 14962.31200 2020-01-01 2020-01-02 2020-01-01 00:06:14 2020-01-02 03:34:35 2020-01-01 00:06:14.000 2020-01-02 03:34:35.000 374 99275 49824.5 4982450 374 99275 49824.5 4982450 -32195 32740 5260.02 526002 -127 124 1.78 178 -375 101 10365 99276 1.12612 298.12612 149.62612 14962.61261 1.12612 298.12613 149.62612 14962.61283 1.12612 298.12612 149.62612000000004 14962.61200 2020-01-01 2020-01-02 2020-01-01 00:06:15 2020-01-02 03:34:36 2020-01-01 00:06:15.000 2020-01-02 03:34:36.000 375 99276 49825.5 4982550 375 99276 49825.5 4982550 -32194 32741 5261.02 526102 -126 125 2.78 278 -376 101 10366 99277 1.12912 298.12912 149.62912 14962.91291 1.12912 298.12912 149.62912 14962.91297 1.12912 298.12912 149.6291199999999 14962.91200 2020-01-01 2020-01-02 2020-01-01 00:06:16 2020-01-02 03:34:37 2020-01-01 00:06:16.000 2020-01-02 03:34:37.000 376 99277 49826.5 4982650 376 99277 49826.5 4982650 -32193 32742 5262.02 526202 -125 126 3.78 378 +369 101 10359 99270 1.1081 298.1081 149.6081 14960.81081 1.1081 298.1081 149.6081 14960.81062 1.10810 298.10810 149.6081 14960.81000 2020-01-01 2020-01-02 2020-01-01 00:06:09 2020-01-02 03:34:30 2020-01-01 00:06:09.000 2020-01-02 03:34:30.000 369 99270 49819.5 4981950 369 99270 49819.5 4981950 -32200 32735 5255.02 525502 -126 125 1.9 190 +37 102 10027 99937 0.11111 300.11111 150.11111 15161.22222 0.11111 300.1111 150.11111 15161.22248 0.11111 300.11111 150.11111 15161.22211 2020-01-01 2020-01-02 2020-01-01 00:00:37 2020-01-02 03:45:37 2020-01-01 00:00:37.000 2020-01-02 03:45:37.000 37 99937 49987 5048687 37 99937 49987 5048687 -32532 32403 4566.009900990099 461167 -127 124 -1.4356435643564356 -145 +370 101 10360 99271 1.11111 298.11111 149.61111 14961.11111 1.11111 298.1111 149.61111 14961.11137 1.11111 298.11111 149.61111 14961.11100 2020-01-01 2020-01-02 2020-01-01 00:06:10 2020-01-02 03:34:31 2020-01-01 00:06:10.000 2020-01-02 03:34:31.000 370 99271 49820.5 4982050 370 99271 49820.5 4982050 -32199 32736 5256.02 525602 -125 126 2.9 290 +371 101 10361 99272 1.11411 298.11411 149.61411 14961.41141 1.11411 298.1141 149.61411 14961.4115 1.11411 298.11411 149.61411 14961.41100 2020-01-01 2020-01-02 2020-01-01 00:06:11 2020-01-02 03:34:32 2020-01-01 00:06:11.000 2020-01-02 03:34:32.000 371 99272 49821.5 4982150 371 99272 49821.5 4982150 -32198 32737 5257.02 525702 -124 127 3.9 390 +372 101 10362 99273 1.11711 298.11711 149.61711 14961.71171 1.11711 298.11713 149.61711 14961.71165 1.11711 298.11711 149.61711 14961.71100 2020-01-01 2020-01-02 2020-01-01 00:06:12 2020-01-02 03:34:33 2020-01-01 00:06:12.000 2020-01-02 03:34:33.000 372 99273 49822.5 4982250 372 99273 49822.5 4982250 -32197 32738 5258.02 525802 -128 127 2.34 234 +373 101 10363 99274 1.12012 298.12012 149.62012 14962.01201 1.12012 298.12012 149.62011 14962.01179 1.12012 298.12012 149.62012000000001 14962.01200 2020-01-01 2020-01-02 2020-01-01 00:06:13 2020-01-02 03:34:34 2020-01-01 00:06:13.000 2020-01-02 03:34:34.000 373 99274 49823.5 4982350 373 99274 49823.5 4982350 -32196 32739 5259.02 525902 -128 123 0.78 78 +374 101 10364 99275 1.12312 298.12312 149.62312 14962.31231 1.12312 298.1231 149.62312 14962.31208 1.12312 298.12312 149.62312 14962.31200 2020-01-01 2020-01-02 2020-01-01 00:06:14 2020-01-02 03:34:35 2020-01-01 00:06:14.000 2020-01-02 03:34:35.000 374 99275 49824.5 4982450 374 99275 49824.5 4982450 -32195 32740 5260.02 526002 -127 124 1.78 178 +375 101 10365 99276 1.12612 298.12612 149.62612 14962.61261 1.12612 298.12613 149.62612 14962.61283 1.12612 298.12612 149.62612 14962.61200 2020-01-01 2020-01-02 2020-01-01 00:06:15 2020-01-02 03:34:36 2020-01-01 00:06:15.000 2020-01-02 03:34:36.000 375 99276 49825.5 4982550 375 99276 49825.5 4982550 -32194 32741 5261.02 526102 -126 125 2.78 278 +376 101 10366 99277 1.12912 298.12912 149.62912 14962.91291 1.12912 298.12912 149.62912 14962.91297 1.12912 298.12912 149.62912 14962.91200 2020-01-01 2020-01-02 2020-01-01 00:06:16 2020-01-02 03:34:37 2020-01-01 00:06:16.000 2020-01-02 03:34:37.000 376 99277 49826.5 4982650 376 99277 49826.5 4982650 -32193 32742 5262.02 526202 -125 126 3.78 378 377 101 10367 99278 1.13213 298.13213 149.63213 14963.21321 1.13213 298.13214 149.63213 14963.21312 1.13213 298.13213 149.63213 14963.21300 2020-01-01 2020-01-02 2020-01-01 00:06:17 2020-01-02 03:34:38 2020-01-01 00:06:17.000 2020-01-02 03:34:38.000 377 99278 49827.5 4982750 377 99278 49827.5 4982750 -32192 32743 5263.02 526302 -124 127 4.78 478 -378 101 10368 99279 1.13513 298.13513 149.63513 14963.51351 1.13513 298.13513 149.63513 14963.51326 1.13513 298.13513 149.63513000000003 14963.51300 2020-01-01 2020-01-02 2020-01-01 00:06:18 2020-01-02 03:34:39 2020-01-01 00:06:18.000 2020-01-02 03:34:39.000 378 99279 49828.5 4982850 378 99279 49828.5 4982850 -32191 32744 5264.02 526402 -128 127 3.22 322 -379 101 10369 99280 1.13813 298.13813 149.63813 14963.81381 1.13813 298.13815 149.63814 14963.81401 1.13813 298.13813 149.63812999999993 14963.81300 2020-01-01 2020-01-02 2020-01-01 00:06:19 2020-01-02 03:34:40 2020-01-01 00:06:19.000 2020-01-02 03:34:40.000 379 99280 49829.5 4982950 379 99280 49829.5 4982950 -32190 32745 5265.02 526502 -128 127 1.66 166 -38 102 10028 99938 0.11411 300.11411 150.11411 15161.52552 0.11411 300.1141 150.11411 15161.52562 0.11411 300.11411 150.11411000000012 15161.52511 2020-01-01 2020-01-02 2020-01-01 00:00:38 2020-01-02 03:45:38 2020-01-01 00:00:38.000 2020-01-02 03:45:38.000 38 99938 49988 5048788 38 99938 49988 5048788 -32531 32404 4567.009900990099 461268 -126 125 -0.43564356435643564 -44 -380 101 10370 99281 1.14114 298.14114 149.64114 14964.11411 1.14114 298.14114 149.64114 14964.11431 1.14114 298.14114 149.64113999999995 14964.11400 2020-01-01 2020-01-02 2020-01-01 00:06:20 2020-01-02 03:34:41 2020-01-01 00:06:20.000 2020-01-02 03:34:41.000 380 99281 49830.5 4983050 380 99281 49830.5 4983050 -32189 32746 5266.02 526602 -128 124 0.1 10 -381 101 10371 99282 1.14414 298.14414 149.64414 14964.41441 1.14414 298.14413 149.64414 14964.41448 1.14414 298.14414 149.64414000000005 14964.41400 2020-01-01 2020-01-02 2020-01-01 00:06:21 2020-01-02 03:34:42 2020-01-01 00:06:21.000 2020-01-02 03:34:42.000 381 99282 49831.5 4983150 381 99282 49831.5 4983150 -32188 32747 5267.02 526702 -127 125 1.1 110 -382 101 10372 99283 1.14714 298.14714 149.64714 14964.71471 1.14714 298.14716 149.64714 14964.71459 1.14714 298.14714 149.64713999999992 14964.71400 2020-01-01 2020-01-02 2020-01-01 00:06:22 2020-01-02 03:34:43 2020-01-01 00:06:22.000 2020-01-02 03:34:43.000 382 99283 49832.5 4983250 382 99283 49832.5 4983250 -32187 32748 5268.02 526802 -126 126 2.1 210 +378 101 10368 99279 1.13513 298.13513 149.63513 14963.51351 1.13513 298.13513 149.63513 14963.51326 1.13513 298.13513 149.63513 14963.51300 2020-01-01 2020-01-02 2020-01-01 00:06:18 2020-01-02 03:34:39 2020-01-01 00:06:18.000 2020-01-02 03:34:39.000 378 99279 49828.5 4982850 378 99279 49828.5 4982850 -32191 32744 5264.02 526402 -128 127 3.22 322 +379 101 10369 99280 1.13813 298.13813 149.63813 14963.81381 1.13813 298.13815 149.63814 14963.81401 1.13813 298.13813 149.63813 14963.81300 2020-01-01 2020-01-02 2020-01-01 00:06:19 2020-01-02 03:34:40 2020-01-01 00:06:19.000 2020-01-02 03:34:40.000 379 99280 49829.5 4982950 379 99280 49829.5 4982950 -32190 32745 5265.02 526502 -128 127 1.66 166 +38 102 10028 99938 0.11411 300.11411 150.11411 15161.52552 0.11411 300.1141 150.11411 15161.52562 0.11411 300.11411 150.11411 15161.52511 2020-01-01 2020-01-02 2020-01-01 00:00:38 2020-01-02 03:45:38 2020-01-01 00:00:38.000 2020-01-02 03:45:38.000 38 99938 49988 5048788 38 99938 49988 5048788 -32531 32404 4567.009900990099 461268 -126 125 -0.43564356435643564 -44 +380 101 10370 99281 1.14114 298.14114 149.64114 14964.11411 1.14114 298.14114 149.64114 14964.11431 1.14114 298.14114 149.64114 14964.11400 2020-01-01 2020-01-02 2020-01-01 00:06:20 2020-01-02 03:34:41 2020-01-01 00:06:20.000 2020-01-02 03:34:41.000 380 99281 49830.5 4983050 380 99281 49830.5 4983050 -32189 32746 5266.02 526602 -128 124 0.1 10 +381 101 10371 99282 1.14414 298.14414 149.64414 14964.41441 1.14414 298.14413 149.64414 14964.41448 1.14414 298.14414 149.64414 14964.41400 2020-01-01 2020-01-02 2020-01-01 00:06:21 2020-01-02 03:34:42 2020-01-01 00:06:21.000 2020-01-02 03:34:42.000 381 99282 49831.5 4983150 381 99282 49831.5 4983150 -32188 32747 5267.02 526702 -127 125 1.1 110 +382 101 10372 99283 1.14714 298.14714 149.64714 14964.71471 1.14714 298.14716 149.64714 14964.71459 1.14714 298.14714 149.64714 14964.71400 2020-01-01 2020-01-02 2020-01-01 00:06:22 2020-01-02 03:34:43 2020-01-01 00:06:22.000 2020-01-02 03:34:43.000 382 99283 49832.5 4983250 382 99283 49832.5 4983250 -32187 32748 5268.02 526802 -126 126 2.1 210 383 101 10373 99284 1.15015 298.15015 149.65015 14965.01501 1.15015 298.15015 149.65014 14965.01472 1.15015 298.15015 149.65015 14965.01500 2020-01-01 2020-01-02 2020-01-01 00:06:23 2020-01-02 03:34:44 2020-01-01 00:06:23.000 2020-01-02 03:34:44.000 383 99284 49833.5 4983350 383 99284 49833.5 4983350 -32186 32749 5269.02 526902 -125 127 3.1 310 -384 101 10374 99285 1.15315 298.15315 149.65315 14965.31531 1.15315 298.15317 149.65315 14965.31547 1.15315 298.15315 149.65315000000007 14965.31500 2020-01-01 2020-01-02 2020-01-01 00:06:24 2020-01-02 03:34:45 2020-01-01 00:06:24.000 2020-01-02 03:34:45.000 384 99285 49834.5 4983450 384 99285 49834.5 4983450 -32185 32750 5270.02 527002 -128 127 1.54 154 -385 101 10375 99286 1.15615 298.15615 149.65615 14965.61561 1.15615 298.15616 149.65615 14965.61578 1.15615 298.15615 149.65615000000003 14965.61500 2020-01-01 2020-01-02 2020-01-01 00:06:25 2020-01-02 03:34:46 2020-01-01 00:06:25.000 2020-01-02 03:34:46.000 385 99286 49835.5 4983550 385 99286 49835.5 4983550 -32184 32751 5271.02 527102 -128 127 -0.02 -2 +384 101 10374 99285 1.15315 298.15315 149.65315 14965.31531 1.15315 298.15317 149.65315 14965.31547 1.15315 298.15315 149.65315 14965.31500 2020-01-01 2020-01-02 2020-01-01 00:06:24 2020-01-02 03:34:45 2020-01-01 00:06:24.000 2020-01-02 03:34:45.000 384 99285 49834.5 4983450 384 99285 49834.5 4983450 -32185 32750 5270.02 527002 -128 127 1.54 154 +385 101 10375 99286 1.15615 298.15615 149.65615 14965.61561 1.15615 298.15616 149.65615 14965.61578 1.15615 298.15615 149.65615 14965.61500 2020-01-01 2020-01-02 2020-01-01 00:06:25 2020-01-02 03:34:46 2020-01-01 00:06:25.000 2020-01-02 03:34:46.000 385 99286 49835.5 4983550 385 99286 49835.5 4983550 -32184 32751 5271.02 527102 -128 127 -0.02 -2 386 101 10376 99287 1.15915 298.15915 149.65915 14965.91591 1.15915 298.15915 149.65915 14965.91594 1.15915 298.15915 149.65915 14965.91500 2020-01-01 2020-01-02 2020-01-01 00:06:26 2020-01-02 03:34:47 2020-01-01 00:06:26.000 2020-01-02 03:34:47.000 386 99287 49836.5 4983650 386 99287 49836.5 4983650 -32183 32752 5272.02 527202 -128 123 -1.58 -158 -387 101 10377 99288 1.16216 298.16216 149.66216 14966.21621 1.16216 298.16217 149.66216 14966.21606 1.16216 298.16216 149.66215999999994 14966.21600 2020-01-01 2020-01-02 2020-01-01 00:06:27 2020-01-02 03:34:48 2020-01-01 00:06:27.000 2020-01-02 03:34:48.000 387 99288 49837.5 4983750 387 99288 49837.5 4983750 -32182 32753 5273.02 527302 -127 124 -0.58 -58 +387 101 10377 99288 1.16216 298.16216 149.66216 14966.21621 1.16216 298.16217 149.66216 14966.21606 1.16216 298.16216 149.66216 14966.21600 2020-01-01 2020-01-02 2020-01-01 00:06:27 2020-01-02 03:34:48 2020-01-01 00:06:27.000 2020-01-02 03:34:48.000 387 99288 49837.5 4983750 387 99288 49837.5 4983750 -32182 32753 5273.02 527302 -127 124 -0.58 -58 388 101 10378 99289 1.16516 298.16516 149.66516 14966.51651 1.16516 298.16516 149.66516 14966.51636 1.16516 298.16516 149.66516 14966.51600 2020-01-01 2020-01-02 2020-01-01 00:06:28 2020-01-02 03:34:49 2020-01-01 00:06:28.000 2020-01-02 03:34:49.000 388 99289 49838.5 4983850 388 99289 49838.5 4983850 -32181 32754 5274.02 527402 -126 125 0.42 42 -389 101 10379 99290 1.16816 298.16816 149.66816 14966.81681 1.16816 298.16818 149.66816 14966.81695 1.16816 298.16816 149.66815999999997 14966.81600 2020-01-01 2020-01-02 2020-01-01 00:06:29 2020-01-02 03:34:50 2020-01-01 00:06:29.000 2020-01-02 03:34:50.000 389 99290 49839.5 4983950 389 99290 49839.5 4983950 -32180 32755 5275.02 527502 -125 126 1.42 142 -39 102 10029 99939 0.11711 300.11711 150.11711 15161.82882 0.11711 300.11713 150.11711 15161.82876 0.11711 300.11711 150.11710999999997 15161.82811 2020-01-01 2020-01-02 2020-01-01 00:00:39 2020-01-02 03:45:39 2020-01-01 00:00:39.000 2020-01-02 03:45:39.000 39 99939 49989 5048889 39 99939 49989 5048889 -32530 32405 4568.009900990099 461369 -125 126 0.5643564356435643 57 -390 101 10380 99291 1.17117 298.17117 149.67117 14967.11711 1.17117 298.17117 149.67117 14967.11725 1.17117 298.17117 149.6711699999999 14967.11700 2020-01-01 2020-01-02 2020-01-01 00:06:30 2020-01-02 03:34:51 2020-01-01 00:06:30.000 2020-01-02 03:34:51.000 390 99291 49840.5 4984050 390 99291 49840.5 4984050 -32179 32756 5276.02 527602 -124 127 2.42 242 -391 101 10381 99292 1.17417 298.17417 149.67417 14967.41741 1.17417 298.17416 149.67417 14967.41741 1.17417 298.17417 149.67417000000012 14967.41700 2020-01-01 2020-01-02 2020-01-01 00:06:31 2020-01-02 03:34:52 2020-01-01 00:06:31.000 2020-01-02 03:34:52.000 391 99292 49841.5 4984150 391 99292 49841.5 4984150 -32178 32757 5277.02 527702 -128 127 0.86 86 -392 101 10382 99293 1.17717 298.17717 149.67717 14967.71771 1.17717 298.1772 149.67717 14967.71753 1.17717 298.17717 149.67717000000007 14967.71700 2020-01-01 2020-01-02 2020-01-01 00:06:32 2020-01-02 03:34:53 2020-01-01 00:06:32.000 2020-01-02 03:34:53.000 392 99293 49842.5 4984250 392 99293 49842.5 4984250 -32177 32758 5278.02 527802 -128 123 -0.7 -70 -393 101 10383 99294 1.18018 298.18018 149.68018 14968.01801 1.18018 298.18018 149.68017 14968.01782 1.18018 298.18018 149.68017999999995 14968.01800 2020-01-01 2020-01-02 2020-01-01 00:06:33 2020-01-02 03:34:54 2020-01-01 00:06:33.000 2020-01-02 03:34:54.000 393 99294 49843.5 4984350 393 99294 49843.5 4984350 -32176 32759 5279.02 527902 -127 124 0.3 30 +389 101 10379 99290 1.16816 298.16816 149.66816 14966.81681 1.16816 298.16818 149.66816 14966.81695 1.16816 298.16816 149.66816 14966.81600 2020-01-01 2020-01-02 2020-01-01 00:06:29 2020-01-02 03:34:50 2020-01-01 00:06:29.000 2020-01-02 03:34:50.000 389 99290 49839.5 4983950 389 99290 49839.5 4983950 -32180 32755 5275.02 527502 -125 126 1.42 142 +39 102 10029 99939 0.11711 300.11711 150.11711 15161.82882 0.11711 300.11713 150.11711 15161.82876 0.11711 300.11711 150.11711 15161.82811 2020-01-01 2020-01-02 2020-01-01 00:00:39 2020-01-02 03:45:39 2020-01-01 00:00:39.000 2020-01-02 03:45:39.000 39 99939 49989 5048889 39 99939 49989 5048889 -32530 32405 4568.009900990099 461369 -125 126 0.5643564356435643 57 +390 101 10380 99291 1.17117 298.17117 149.67117 14967.11711 1.17117 298.17117 149.67117 14967.11725 1.17117 298.17117 149.67117 14967.11700 2020-01-01 2020-01-02 2020-01-01 00:06:30 2020-01-02 03:34:51 2020-01-01 00:06:30.000 2020-01-02 03:34:51.000 390 99291 49840.5 4984050 390 99291 49840.5 4984050 -32179 32756 5276.02 527602 -124 127 2.42 242 +391 101 10381 99292 1.17417 298.17417 149.67417 14967.41741 1.17417 298.17416 149.67417 14967.41741 1.17417 298.17417 149.67417 14967.41700 2020-01-01 2020-01-02 2020-01-01 00:06:31 2020-01-02 03:34:52 2020-01-01 00:06:31.000 2020-01-02 03:34:52.000 391 99292 49841.5 4984150 391 99292 49841.5 4984150 -32178 32757 5277.02 527702 -128 127 0.86 86 +392 101 10382 99293 1.17717 298.17717 149.67717 14967.71771 1.17717 298.1772 149.67717 14967.71753 1.17717 298.17717 149.67717000000002 14967.71700 2020-01-01 2020-01-02 2020-01-01 00:06:32 2020-01-02 03:34:53 2020-01-01 00:06:32.000 2020-01-02 03:34:53.000 392 99293 49842.5 4984250 392 99293 49842.5 4984250 -32177 32758 5278.02 527802 -128 123 -0.7 -70 +393 101 10383 99294 1.18018 298.18018 149.68018 14968.01801 1.18018 298.18018 149.68017 14968.01782 1.18018 298.18018 149.68018 14968.01800 2020-01-01 2020-01-02 2020-01-01 00:06:33 2020-01-02 03:34:54 2020-01-01 00:06:33.000 2020-01-02 03:34:54.000 393 99294 49843.5 4984350 393 99294 49843.5 4984350 -32176 32759 5279.02 527902 -127 124 0.3 30 394 101 10384 99295 1.18318 298.18318 149.68318 14968.31831 1.18318 298.1832 149.68318 14968.31842 1.18318 298.18318 149.68318 14968.31800 2020-01-01 2020-01-02 2020-01-01 00:06:34 2020-01-02 03:34:55 2020-01-01 00:06:34.000 2020-01-02 03:34:55.000 394 99295 49844.5 4984450 394 99295 49844.5 4984450 -32175 32760 5280.02 528002 -126 125 1.3 130 -395 101 10385 99296 1.18618 298.18618 149.68618 14968.61861 1.18618 298.1862 149.68618 14968.61875 1.18618 298.18618 149.68618000000004 14968.61800 2020-01-01 2020-01-02 2020-01-01 00:06:35 2020-01-02 03:34:56 2020-01-01 00:06:35.000 2020-01-02 03:34:56.000 395 99296 49845.5 4984550 395 99296 49845.5 4984550 -32174 32761 5281.02 528102 -125 126 2.3 230 -396 101 10386 99297 1.18918 298.18918 149.68918 14968.91891 1.18918 298.18918 149.68918 14968.91889 1.18918 298.18918 149.68917999999996 14968.91800 2020-01-01 2020-01-02 2020-01-01 00:06:36 2020-01-02 03:34:57 2020-01-01 00:06:36.000 2020-01-02 03:34:57.000 396 99297 49846.5 4984650 396 99297 49846.5 4984650 -32173 32762 5282.02 528202 -124 127 3.3 330 -397 101 10387 99298 1.19219 298.19219 149.69219 14969.21921 1.19219 298.1922 149.69219 14969.21964 1.19219 298.19219 149.69218999999993 14969.21900 2020-01-01 2020-01-02 2020-01-01 00:06:37 2020-01-02 03:34:58 2020-01-01 00:06:37.000 2020-01-02 03:34:58.000 397 99298 49847.5 4984750 397 99298 49847.5 4984750 -32172 32763 5283.02 528302 -128 127 1.74 174 -398 101 10388 99299 1.19519 298.19519 149.69519 14969.51951 1.19519 298.1952 149.69519 14969.51929 1.19519 298.19519 149.69519000000008 14969.51900 2020-01-01 2020-01-02 2020-01-01 00:06:38 2020-01-02 03:34:59 2020-01-01 00:06:38.000 2020-01-02 03:34:59.000 398 99299 49848.5 4984850 398 99299 49848.5 4984850 -32171 32764 5284.02 528402 -128 123 0.18 18 -399 101 10389 99300 1.19819 298.19819 149.69819 14969.81981 1.19819 298.1982 149.69819 14969.81989 1.19819 298.19819 149.69818999999993 14969.81900 2020-01-01 2020-01-02 2020-01-01 00:06:39 2020-01-02 03:35:00 2020-01-01 00:06:39.000 2020-01-02 03:35:00.000 399 99300 49849.5 4984950 399 99300 49849.5 4984950 -32170 32765 5285.02 528502 -127 124 1.18 118 -4 102 1003 9994 0.01201 300.01201 150.01201 15151.21321 0.01201 300.01202 150.01201 15151.21318 0.01201 300.01201 150.0120100000001 15151.21301 2020-01-01 2020-01-02 2020-01-01 00:00:04 2020-01-02 03:45:04 2020-01-01 00:00:04.000 2020-01-02 03:45:04.000 4 99904 49954 5045354 4 99904 49954 5045354 -32565 32370 4533.009900990099 457834 -128 127 -1.4851485148514851 -150 -40 102 10030 99940 0.12012 300.12012 150.12012 15162.13213 0.12012 300.12012 150.12011 15162.13191 0.12012 300.12012 150.12011999999987 15162.13212 2020-01-01 2020-01-02 2020-01-01 00:00:40 2020-01-02 03:45:40 2020-01-01 00:00:40.000 2020-01-02 03:45:40.000 40 99940 49990 5048990 40 99940 49990 5048990 -32529 32406 4569.009900990099 461470 -124 127 1.5643564356435644 158 +395 101 10385 99296 1.18618 298.18618 149.68618 14968.61861 1.18618 298.1862 149.68618 14968.61875 1.18618 298.18618 149.68618 14968.61800 2020-01-01 2020-01-02 2020-01-01 00:06:35 2020-01-02 03:34:56 2020-01-01 00:06:35.000 2020-01-02 03:34:56.000 395 99296 49845.5 4984550 395 99296 49845.5 4984550 -32174 32761 5281.02 528102 -125 126 2.3 230 +396 101 10386 99297 1.18918 298.18918 149.68918 14968.91891 1.18918 298.18918 149.68918 14968.91889 1.18918 298.18918 149.68918 14968.91800 2020-01-01 2020-01-02 2020-01-01 00:06:36 2020-01-02 03:34:57 2020-01-01 00:06:36.000 2020-01-02 03:34:57.000 396 99297 49846.5 4984650 396 99297 49846.5 4984650 -32173 32762 5282.02 528202 -124 127 3.3 330 +397 101 10387 99298 1.19219 298.19219 149.69219 14969.21921 1.19219 298.1922 149.69219 14969.21964 1.19219 298.19219 149.69218999999998 14969.21900 2020-01-01 2020-01-02 2020-01-01 00:06:37 2020-01-02 03:34:58 2020-01-01 00:06:37.000 2020-01-02 03:34:58.000 397 99298 49847.5 4984750 397 99298 49847.5 4984750 -32172 32763 5283.02 528302 -128 127 1.74 174 +398 101 10388 99299 1.19519 298.19519 149.69519 14969.51951 1.19519 298.1952 149.69519 14969.51929 1.19519 298.19519 149.69519 14969.51900 2020-01-01 2020-01-02 2020-01-01 00:06:38 2020-01-02 03:34:59 2020-01-01 00:06:38.000 2020-01-02 03:34:59.000 398 99299 49848.5 4984850 398 99299 49848.5 4984850 -32171 32764 5284.02 528402 -128 123 0.18 18 +399 101 10389 99300 1.19819 298.19819 149.69819 14969.81981 1.19819 298.1982 149.69819 14969.81989 1.19819 298.19819 149.69818999999998 14969.81900 2020-01-01 2020-01-02 2020-01-01 00:06:39 2020-01-02 03:35:00 2020-01-01 00:06:39.000 2020-01-02 03:35:00.000 399 99300 49849.5 4984950 399 99300 49849.5 4984950 -32170 32765 5285.02 528502 -127 124 1.18 118 +4 102 1003 9994 0.01201 300.01201 150.01201 15151.21321 0.01201 300.01202 150.01201 15151.21318 0.01201 300.01201 150.01201 15151.21301 2020-01-01 2020-01-02 2020-01-01 00:00:04 2020-01-02 03:45:04 2020-01-01 00:00:04.000 2020-01-02 03:45:04.000 4 99904 49954 5045354 4 99904 49954 5045354 -32565 32370 4533.009900990099 457834 -128 127 -1.4851485148514851 -150 +40 102 10030 99940 0.12012 300.12012 150.12012 15162.13213 0.12012 300.12012 150.12011 15162.13191 0.12012 300.12012 150.12012000000001 15162.13212 2020-01-01 2020-01-02 2020-01-01 00:00:40 2020-01-02 03:45:40 2020-01-01 00:00:40.000 2020-01-02 03:45:40.000 40 99940 49990 5048990 40 99940 49990 5048990 -32529 32406 4569.009900990099 461470 -124 127 1.5643564356435644 158 400 101 10390 99301 1.2012 298.2012 149.7012 14970.12012 1.2012 298.2012 149.7012 14970.12022 1.20120 298.20120 149.7012 14970.12000 2020-01-01 2020-01-02 2020-01-01 00:06:40 2020-01-02 03:35:01 2020-01-01 00:06:40.000 2020-01-02 03:35:01.000 400 99301 49850.5 4985050 400 99301 49850.5 4985050 -32169 32766 5286.02 528602 -126 125 2.18 218 -401 101 10391 99302 1.2042 298.2042 149.7042 14970.42042 1.2042 298.2042 149.7042 14970.42035 1.20420 298.20420 149.70420000000004 14970.42000 2020-01-01 2020-01-02 2020-01-01 00:06:41 2020-01-02 03:35:02 2020-01-01 00:06:41.000 2020-01-02 03:35:02.000 401 99302 49851.5 4985150 401 99302 49851.5 4985150 -32168 32767 5287.02 528702 -125 126 3.18 318 -402 101 10392 99303 1.2072 298.2072 149.7072 14970.72072 1.2072 298.2072 149.70721 14970.72111 1.20720 298.20720 149.70720000000003 14970.72000 2020-01-01 2020-01-02 2020-01-01 00:06:42 2020-01-02 03:35:03 2020-01-01 00:06:42.000 2020-01-02 03:35:03.000 402 99303 49852.5 4985250 402 99303 49852.5 4985250 -32768 32370 4632.66 463266 -124 127 4.18 418 -403 101 10393 99304 1.21021 298.21021 149.71021 14971.02102 1.21021 298.2102 149.7102 14971.02077 1.21021 298.21021 149.71020999999996 14971.02100 2020-01-01 2020-01-02 2020-01-01 00:06:43 2020-01-02 03:35:04 2020-01-01 00:06:43.000 2020-01-02 03:35:04.000 403 99304 49853.5 4985350 403 99304 49853.5 4985350 -32767 32371 4633.66 463366 -128 127 2.62 262 -404 101 10394 99305 1.21321 298.21321 149.71321 14971.32132 1.21321 298.21323 149.71321 14971.32139 1.21321 298.21321 149.71320999999992 14971.32100 2020-01-01 2020-01-02 2020-01-01 00:06:44 2020-01-02 03:35:05 2020-01-01 00:06:44.000 2020-01-02 03:35:05.000 404 99305 49854.5 4985450 404 99305 49854.5 4985450 -32766 32372 4634.66 463466 -128 127 1.06 106 +401 101 10391 99302 1.2042 298.2042 149.7042 14970.42042 1.2042 298.2042 149.7042 14970.42035 1.20420 298.20420 149.70420000000001 14970.42000 2020-01-01 2020-01-02 2020-01-01 00:06:41 2020-01-02 03:35:02 2020-01-01 00:06:41.000 2020-01-02 03:35:02.000 401 99302 49851.5 4985150 401 99302 49851.5 4985150 -32168 32767 5287.02 528702 -125 126 3.18 318 +402 101 10392 99303 1.2072 298.2072 149.7072 14970.72072 1.2072 298.2072 149.70721 14970.72111 1.20720 298.20720 149.7072 14970.72000 2020-01-01 2020-01-02 2020-01-01 00:06:42 2020-01-02 03:35:03 2020-01-01 00:06:42.000 2020-01-02 03:35:03.000 402 99303 49852.5 4985250 402 99303 49852.5 4985250 -32768 32370 4632.66 463266 -124 127 4.18 418 +403 101 10393 99304 1.21021 298.21021 149.71021 14971.02102 1.21021 298.2102 149.7102 14971.02077 1.21021 298.21021 149.71021000000002 14971.02100 2020-01-01 2020-01-02 2020-01-01 00:06:43 2020-01-02 03:35:04 2020-01-01 00:06:43.000 2020-01-02 03:35:04.000 403 99304 49853.5 4985350 403 99304 49853.5 4985350 -32767 32371 4633.66 463366 -128 127 2.62 262 +404 101 10394 99305 1.21321 298.21321 149.71321 14971.32132 1.21321 298.21323 149.71321 14971.32139 1.21321 298.21321 149.71321 14971.32100 2020-01-01 2020-01-02 2020-01-01 00:06:44 2020-01-02 03:35:05 2020-01-01 00:06:44.000 2020-01-02 03:35:05.000 404 99305 49854.5 4985450 404 99305 49854.5 4985450 -32766 32372 4634.66 463466 -128 127 1.06 106 405 101 10395 99306 1.21621 298.21621 149.71621 14971.62162 1.21621 298.21622 149.71621 14971.62169 1.21621 298.21621 149.71621 14971.62100 2020-01-01 2020-01-02 2020-01-01 00:06:45 2020-01-02 03:35:06 2020-01-01 00:06:45.000 2020-01-02 03:35:06.000 405 99306 49855.5 4985550 405 99306 49855.5 4985550 -32765 32373 4635.66 463566 -128 124 -0.5 -50 -406 101 10396 99307 1.21921 298.21921 149.71921 14971.92192 1.21921 298.2192 149.71921 14971.92199 1.21921 298.21921 149.71920999999998 14971.92100 2020-01-01 2020-01-02 2020-01-01 00:06:46 2020-01-02 03:35:07 2020-01-01 00:06:46.000 2020-01-02 03:35:07.000 406 99307 49856.5 4985650 406 99307 49856.5 4985650 -32764 32374 4636.66 463666 -127 125 0.5 50 -407 101 10397 99308 1.22222 298.22222 149.72222 14972.22222 1.22222 298.22223 149.72222 14972.22257 1.22222 298.22222 149.72221999999994 14972.22200 2020-01-01 2020-01-02 2020-01-01 00:06:47 2020-01-02 03:35:08 2020-01-01 00:06:47.000 2020-01-02 03:35:08.000 407 99308 49857.5 4985750 407 99308 49857.5 4985750 -32763 32375 4637.66 463766 -126 126 1.5 150 -408 101 10398 99309 1.22522 298.22522 149.72522 14972.52252 1.22522 298.22522 149.72522 14972.52224 1.22522 298.22522 149.72522000000004 14972.52200 2020-01-01 2020-01-02 2020-01-01 00:06:48 2020-01-02 03:35:09 2020-01-01 00:06:48.000 2020-01-02 03:35:09.000 408 99309 49858.5 4985850 408 99309 49858.5 4985850 -32762 32376 4638.66 463866 -125 127 2.5 250 -409 101 10399 99310 1.22822 298.22822 149.72822 14972.82282 1.22822 298.22824 149.72822 14972.82286 1.22822 298.22822 149.7282200000001 14972.82200 2020-01-01 2020-01-02 2020-01-01 00:06:49 2020-01-02 03:35:10 2020-01-01 00:06:49.000 2020-01-02 03:35:10.000 409 99310 49859.5 4985950 409 99310 49859.5 4985950 -32761 32377 4639.66 463966 -128 127 0.94 94 -41 102 10031 99941 0.12312 300.12312 150.12312 15162.43543 0.12312 300.1231 150.12312 15162.43521 0.12312 300.12312 150.12312000000009 15162.43512 2020-01-01 2020-01-02 2020-01-01 00:00:41 2020-01-02 03:45:41 2020-01-01 00:00:41.000 2020-01-02 03:45:41.000 41 99941 49991 5049091 41 99941 49991 5049091 -32528 32407 4570.009900990099 461571 -128 127 0.0297029702970297 3 -410 101 10400 99311 1.23123 298.23123 149.73123 14973.12312 1.23123 298.23123 149.73123 14973.12316 1.23123 298.23123 149.73122999999995 14973.12300 2020-01-01 2020-01-02 2020-01-01 00:06:50 2020-01-02 03:35:11 2020-01-01 00:06:50.000 2020-01-02 03:35:11.000 410 99311 49860.5 4986050 410 99311 49860.5 4986050 -32760 32378 4640.66 464066 -128 127 -0.62 -62 +406 101 10396 99307 1.21921 298.21921 149.71921 14971.92192 1.21921 298.2192 149.71921 14971.92199 1.21921 298.21921 149.71921 14971.92100 2020-01-01 2020-01-02 2020-01-01 00:06:46 2020-01-02 03:35:07 2020-01-01 00:06:46.000 2020-01-02 03:35:07.000 406 99307 49856.5 4985650 406 99307 49856.5 4985650 -32764 32374 4636.66 463666 -127 125 0.5 50 +407 101 10397 99308 1.22222 298.22222 149.72222 14972.22222 1.22222 298.22223 149.72222 14972.22257 1.22222 298.22222 149.72222 14972.22200 2020-01-01 2020-01-02 2020-01-01 00:06:47 2020-01-02 03:35:08 2020-01-01 00:06:47.000 2020-01-02 03:35:08.000 407 99308 49857.5 4985750 407 99308 49857.5 4985750 -32763 32375 4637.66 463766 -126 126 1.5 150 +408 101 10398 99309 1.22522 298.22522 149.72522 14972.52252 1.22522 298.22522 149.72522 14972.52224 1.22522 298.22522 149.72522 14972.52200 2020-01-01 2020-01-02 2020-01-01 00:06:48 2020-01-02 03:35:09 2020-01-01 00:06:48.000 2020-01-02 03:35:09.000 408 99309 49858.5 4985850 408 99309 49858.5 4985850 -32762 32376 4638.66 463866 -125 127 2.5 250 +409 101 10399 99310 1.22822 298.22822 149.72822 14972.82282 1.22822 298.22824 149.72822 14972.82286 1.22822 298.22822 149.72822 14972.82200 2020-01-01 2020-01-02 2020-01-01 00:06:49 2020-01-02 03:35:10 2020-01-01 00:06:49.000 2020-01-02 03:35:10.000 409 99310 49859.5 4985950 409 99310 49859.5 4985950 -32761 32377 4639.66 463966 -128 127 0.94 94 +41 102 10031 99941 0.12312 300.12312 150.12312 15162.43543 0.12312 300.1231 150.12312 15162.43521 0.12312 300.12312 150.12312 15162.43512 2020-01-01 2020-01-02 2020-01-01 00:00:41 2020-01-02 03:45:41 2020-01-01 00:00:41.000 2020-01-02 03:45:41.000 41 99941 49991 5049091 41 99941 49991 5049091 -32528 32407 4570.009900990099 461571 -128 127 0.0297029702970297 3 +410 101 10400 99311 1.23123 298.23123 149.73123 14973.12312 1.23123 298.23123 149.73123 14973.12316 1.23123 298.23123 149.73122999999998 14973.12300 2020-01-01 2020-01-02 2020-01-01 00:06:50 2020-01-02 03:35:11 2020-01-01 00:06:50.000 2020-01-02 03:35:11.000 410 99311 49860.5 4986050 410 99311 49860.5 4986050 -32760 32378 4640.66 464066 -128 127 -0.62 -62 411 101 10401 99312 1.23423 298.23423 149.73423 14973.42342 1.23423 298.23422 149.73423 14973.42345 1.23423 298.23423 149.73423 14973.42300 2020-01-01 2020-01-02 2020-01-01 00:06:51 2020-01-02 03:35:12 2020-01-01 00:06:51.000 2020-01-02 03:35:12.000 411 99312 49861.5 4986150 411 99312 49861.5 4986150 -32759 32379 4641.66 464166 -128 123 -2.18 -218 -412 101 10402 99313 1.23723 298.23723 149.73723 14973.72372 1.23723 298.23724 149.73724 14973.72405 1.23723 298.23723 149.73723000000007 14973.72300 2020-01-01 2020-01-02 2020-01-01 00:06:52 2020-01-02 03:35:13 2020-01-01 00:06:52.000 2020-01-02 03:35:13.000 412 99313 49862.5 4986250 412 99313 49862.5 4986250 -32758 32380 4642.66 464266 -127 124 -1.18 -118 -413 101 10403 99314 1.24024 298.24024 149.74024 14974.02402 1.24024 298.24023 149.74023 14974.02374 1.24024 298.24024 149.74023999999991 14974.02400 2020-01-01 2020-01-02 2020-01-01 00:06:53 2020-01-02 03:35:14 2020-01-01 00:06:53.000 2020-01-02 03:35:14.000 413 99314 49863.5 4986350 413 99314 49863.5 4986350 -32757 32381 4643.66 464366 -126 125 -0.18 -18 -414 101 10404 99315 1.24324 298.24324 149.74324 14974.32432 1.24324 298.24326 149.74324 14974.32433 1.24324 298.24324 149.7432399999999 14974.32400 2020-01-01 2020-01-02 2020-01-01 00:06:54 2020-01-02 03:35:15 2020-01-01 00:06:54.000 2020-01-02 03:35:15.000 414 99315 49864.5 4986450 414 99315 49864.5 4986450 -32756 32382 4644.66 464466 -125 126 0.82 82 -415 101 10405 99316 1.24624 298.24624 149.74624 14974.62462 1.24624 298.24625 149.74624 14974.62463 1.24624 298.24624 149.7462400000001 14974.62400 2020-01-01 2020-01-02 2020-01-01 00:06:55 2020-01-02 03:35:16 2020-01-01 00:06:55.000 2020-01-02 03:35:16.000 415 99316 49865.5 4986550 415 99316 49865.5 4986550 -32755 32383 4645.66 464566 -124 127 1.82 182 -416 101 10406 99317 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74923999999993 14974.92400 2020-01-01 2020-01-02 2020-01-01 00:06:56 2020-01-02 03:35:17 2020-01-01 00:06:56.000 2020-01-02 03:35:17.000 416 99317 49866.5 4986650 416 99317 49866.5 4986650 -32754 32384 4646.66 464666 -128 127 0.26 26 -417 101 10407 99318 1.25225 298.25225 149.75225 14975.22522 1.25225 298.25226 149.75225 14975.22552 1.25225 298.25225 149.75224999999983 14975.22500 2020-01-01 2020-01-02 2020-01-01 00:06:57 2020-01-02 03:35:18 2020-01-01 00:06:57.000 2020-01-02 03:35:18.000 417 99318 49867.5 4986750 417 99318 49867.5 4986750 -32753 32385 4647.66 464766 -128 123 -1.3 -130 -418 101 10408 99319 1.25525 298.25525 149.75525 14975.52552 1.25525 298.25525 149.75525 14975.52521 1.25525 298.25525 149.75525000000007 14975.52500 2020-01-01 2020-01-02 2020-01-01 00:06:58 2020-01-02 03:35:19 2020-01-01 00:06:58.000 2020-01-02 03:35:19.000 418 99319 49868.5 4986850 418 99319 49868.5 4986850 -32752 32386 4648.66 464866 -127 124 -0.3 -30 -419 101 10409 99320 1.25825 298.25825 149.75825 14975.82582 1.25825 298.25827 149.75825 14975.8258 1.25825 298.25825 149.75825000000006 14975.82500 2020-01-01 2020-01-02 2020-01-01 00:06:59 2020-01-02 03:35:20 2020-01-01 00:06:59.000 2020-01-02 03:35:20.000 419 99320 49869.5 4986950 419 99320 49869.5 4986950 -32751 32387 4649.66 464966 -126 125 0.7 70 -42 102 10032 99942 0.12612 300.12612 150.12612 15162.73873 0.12612 300.12613 150.12612 15162.73896 0.12612 300.12612 150.12612000000004 15162.73812 2020-01-01 2020-01-02 2020-01-01 00:00:42 2020-01-02 03:45:42 2020-01-01 00:00:42.000 2020-01-02 03:45:42.000 42 99942 49992 5049192 42 99942 49992 5049192 -32527 32408 4571.009900990099 461672 -128 127 -1.504950495049505 -152 +412 101 10402 99313 1.23723 298.23723 149.73723 14973.72372 1.23723 298.23724 149.73724 14973.72405 1.23723 298.23723 149.73723 14973.72300 2020-01-01 2020-01-02 2020-01-01 00:06:52 2020-01-02 03:35:13 2020-01-01 00:06:52.000 2020-01-02 03:35:13.000 412 99313 49862.5 4986250 412 99313 49862.5 4986250 -32758 32380 4642.66 464266 -127 124 -1.18 -118 +413 101 10403 99314 1.24024 298.24024 149.74024 14974.02402 1.24024 298.24023 149.74023 14974.02374 1.24024 298.24024 149.74024 14974.02400 2020-01-01 2020-01-02 2020-01-01 00:06:53 2020-01-02 03:35:14 2020-01-01 00:06:53.000 2020-01-02 03:35:14.000 413 99314 49863.5 4986350 413 99314 49863.5 4986350 -32757 32381 4643.66 464366 -126 125 -0.18 -18 +414 101 10404 99315 1.24324 298.24324 149.74324 14974.32432 1.24324 298.24326 149.74324 14974.32433 1.24324 298.24324 149.74324000000001 14974.32400 2020-01-01 2020-01-02 2020-01-01 00:06:54 2020-01-02 03:35:15 2020-01-01 00:06:54.000 2020-01-02 03:35:15.000 414 99315 49864.5 4986450 414 99315 49864.5 4986450 -32756 32382 4644.66 464466 -125 126 0.82 82 +415 101 10405 99316 1.24624 298.24624 149.74624 14974.62462 1.24624 298.24625 149.74624 14974.62463 1.24624 298.24624 149.74624 14974.62400 2020-01-01 2020-01-02 2020-01-01 00:06:55 2020-01-02 03:35:16 2020-01-01 00:06:55.000 2020-01-02 03:35:16.000 415 99316 49865.5 4986550 415 99316 49865.5 4986550 -32755 32383 4645.66 464566 -124 127 1.82 182 +416 101 10406 99317 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924000000001 14974.92400 2020-01-01 2020-01-02 2020-01-01 00:06:56 2020-01-02 03:35:17 2020-01-01 00:06:56.000 2020-01-02 03:35:17.000 416 99317 49866.5 4986650 416 99317 49866.5 4986650 -32754 32384 4646.66 464666 -128 127 0.26 26 +417 101 10407 99318 1.25225 298.25225 149.75225 14975.22522 1.25225 298.25226 149.75225 14975.22552 1.25225 298.25225 149.75225 14975.22500 2020-01-01 2020-01-02 2020-01-01 00:06:57 2020-01-02 03:35:18 2020-01-01 00:06:57.000 2020-01-02 03:35:18.000 417 99318 49867.5 4986750 417 99318 49867.5 4986750 -32753 32385 4647.66 464766 -128 123 -1.3 -130 +418 101 10408 99319 1.25525 298.25525 149.75525 14975.52552 1.25525 298.25525 149.75525 14975.52521 1.25525 298.25525 149.75525 14975.52500 2020-01-01 2020-01-02 2020-01-01 00:06:58 2020-01-02 03:35:19 2020-01-01 00:06:58.000 2020-01-02 03:35:19.000 418 99319 49868.5 4986850 418 99319 49868.5 4986850 -32752 32386 4648.66 464866 -127 124 -0.3 -30 +419 101 10409 99320 1.25825 298.25825 149.75825 14975.82582 1.25825 298.25827 149.75825 14975.8258 1.25825 298.25825 149.75825 14975.82500 2020-01-01 2020-01-02 2020-01-01 00:06:59 2020-01-02 03:35:20 2020-01-01 00:06:59.000 2020-01-02 03:35:20.000 419 99320 49869.5 4986950 419 99320 49869.5 4986950 -32751 32387 4649.66 464966 -126 125 0.7 70 +42 102 10032 99942 0.12612 300.12612 150.12612 15162.73873 0.12612 300.12613 150.12612 15162.73896 0.12612 300.12612 150.12612 15162.73812 2020-01-01 2020-01-02 2020-01-01 00:00:42 2020-01-02 03:45:42 2020-01-01 00:00:42.000 2020-01-02 03:45:42.000 42 99942 49992 5049192 42 99942 49992 5049192 -32527 32408 4571.009900990099 461672 -128 127 -1.504950495049505 -152 420 101 10410 99321 1.26126 298.26126 149.76126 14976.12612 1.26126 298.26126 149.76126 14976.12609 1.26126 298.26126 149.76126 14976.12600 2020-01-01 2020-01-02 2020-01-01 00:07:00 2020-01-02 03:35:21 2020-01-01 00:07:00.000 2020-01-02 03:35:21.000 420 99321 49870.5 4987050 420 99321 49870.5 4987050 -32750 32388 4650.66 465066 -125 126 1.7 170 -421 101 10411 99322 1.26426 298.26426 149.76426 14976.42642 1.26426 298.26425 149.76426 14976.4264 1.26426 298.26426 149.76425999999995 14976.42600 2020-01-01 2020-01-02 2020-01-01 00:07:01 2020-01-02 03:35:22 2020-01-01 00:07:01.000 2020-01-02 03:35:22.000 421 99322 49871.5 4987150 421 99322 49871.5 4987150 -32749 32389 4651.66 465166 -124 127 2.7 270 -422 101 10412 99323 1.26726 298.26726 149.76726 14976.72672 1.26726 298.26727 149.76727 14976.72702 1.26726 298.26726 149.76726000000002 14976.72600 2020-01-01 2020-01-02 2020-01-01 00:07:02 2020-01-02 03:35:23 2020-01-01 00:07:02.000 2020-01-02 03:35:23.000 422 99323 49872.5 4987250 422 99323 49872.5 4987250 -32748 32390 4652.66 465266 -128 127 1.14 114 -423 101 10413 99324 1.27027 298.27027 149.77027 14977.02702 1.27027 298.27026 149.77026 14977.02667 1.27027 298.27027 149.77027000000012 14977.02700 2020-01-01 2020-01-02 2020-01-01 00:07:03 2020-01-02 03:35:24 2020-01-01 00:07:03.000 2020-01-02 03:35:24.000 423 99324 49873.5 4987350 423 99324 49873.5 4987350 -32747 32391 4653.66 465366 -128 123 -0.42 -42 -424 101 10414 99325 1.27327 298.27327 149.77327 14977.32732 1.27327 298.2733 149.77327 14977.32727 1.27327 298.27327 149.7732699999999 14977.32700 2020-01-01 2020-01-02 2020-01-01 00:07:04 2020-01-02 03:35:25 2020-01-01 00:07:04.000 2020-01-02 03:35:25.000 424 99325 49874.5 4987450 424 99325 49874.5 4987450 -32746 32392 4654.66 465466 -127 124 0.58 58 -425 101 10415 99326 1.27627 298.27627 149.77627 14977.62762 1.27627 298.27628 149.77627 14977.62756 1.27627 298.27627 149.77627000000007 14977.62700 2020-01-01 2020-01-02 2020-01-01 00:07:05 2020-01-02 03:35:26 2020-01-01 00:07:05.000 2020-01-02 03:35:26.000 425 99326 49875.5 4987550 425 99326 49875.5 4987550 -32745 32393 4655.66 465566 -126 125 1.58 158 -426 101 10416 99327 1.27927 298.27927 149.77927 14977.92792 1.27927 298.27927 149.77927 14977.92787 1.27927 298.27927 149.77926999999988 14977.92700 2020-01-01 2020-01-02 2020-01-01 00:07:06 2020-01-02 03:35:27 2020-01-01 00:07:06.000 2020-01-02 03:35:27.000 426 99327 49876.5 4987650 426 99327 49876.5 4987650 -32744 32394 4656.66 465666 -125 126 2.58 258 -427 101 10417 99328 1.28228 298.28228 149.78228 14978.22822 1.28228 298.2823 149.78228 14978.22849 1.28228 298.28228 149.78227999999996 14978.22800 2020-01-01 2020-01-02 2020-01-01 00:07:07 2020-01-02 03:35:28 2020-01-01 00:07:07.000 2020-01-02 03:35:28.000 427 99328 49877.5 4987750 427 99328 49877.5 4987750 -32743 32395 4657.66 465766 -124 127 3.58 358 -428 101 10418 99329 1.28528 298.28528 149.78528 14978.52852 1.28528 298.28528 149.78528 14978.52815 1.28528 298.28528 149.78528000000003 14978.52800 2020-01-01 2020-01-02 2020-01-01 00:07:08 2020-01-02 03:35:29 2020-01-01 00:07:08.000 2020-01-02 03:35:29.000 428 99329 49878.5 4987850 428 99329 49878.5 4987850 -32742 32396 4658.66 465866 -128 127 2.02 202 -429 101 10419 99330 1.28828 298.28828 149.78828 14978.82882 1.28828 298.2883 149.78828 14978.8289 1.28828 298.28828 149.78828000000001 14978.82800 2020-01-01 2020-01-02 2020-01-01 00:07:09 2020-01-02 03:35:30 2020-01-01 00:07:09.000 2020-01-02 03:35:30.000 429 99330 49879.5 4987950 429 99330 49879.5 4987950 -32741 32397 4659.66 465966 -128 127 0.46 46 -43 102 10033 99943 0.12912 300.12912 150.12912 15163.04204 0.12912 300.12912 150.12912 15163.04211 0.12912 300.12912 150.1291199999999 15163.04112 2020-01-01 2020-01-02 2020-01-01 00:00:43 2020-01-02 03:45:43 2020-01-01 00:00:43.000 2020-01-02 03:45:43.000 43 99943 49993 5049293 43 99943 49993 5049293 -32526 32409 4572.009900990099 461773 -128 124 -3.0396039603960396 -307 -430 101 10420 99331 1.29129 298.29129 149.79129 14979.12912 1.29129 298.2913 149.79129 14979.12904 1.29129 298.29129 149.79128999999995 14979.12900 2020-01-01 2020-01-02 2020-01-01 00:07:10 2020-01-02 03:35:31 2020-01-01 00:07:10.000 2020-01-02 03:35:31.000 430 99331 49880.5 4988050 430 99331 49880.5 4988050 -32740 32398 4660.66 466066 -128 124 -1.1 -110 -431 101 10421 99332 1.29429 298.29429 149.79429 14979.42942 1.29429 298.29428 149.79429 14979.42933 1.29429 298.29429 149.79428999999993 14979.42900 2020-01-01 2020-01-02 2020-01-01 00:07:11 2020-01-02 03:35:32 2020-01-01 00:07:11.000 2020-01-02 03:35:32.000 431 99332 49881.5 4988150 431 99332 49881.5 4988150 -32739 32399 4661.66 466166 -127 125 -0.1 -10 -432 101 10422 99333 1.29729 298.29729 149.79729 14979.72972 1.29729 298.2973 149.79729 14979.72996 1.29729 298.29729 149.79729000000015 14979.72900 2020-01-01 2020-01-02 2020-01-01 00:07:12 2020-01-02 03:35:33 2020-01-01 00:07:12.000 2020-01-02 03:35:33.000 432 99333 49882.5 4988250 432 99333 49882.5 4988250 -32738 32400 4662.66 466266 -126 126 0.9 90 -433 101 10423 99334 1.3003 298.3003 149.8003 14980.03003 1.3003 298.3003 149.80029 14980.02962 1.30030 298.30030 149.80030000000005 14980.03000 2020-01-01 2020-01-02 2020-01-01 00:07:13 2020-01-02 03:35:34 2020-01-01 00:07:13.000 2020-01-02 03:35:34.000 433 99334 49883.5 4988350 433 99334 49883.5 4988350 -32737 32401 4663.66 466366 -125 127 1.9 190 -434 101 10424 99335 1.3033 298.3033 149.8033 14980.33033 1.3033 298.3033 149.8033 14980.33037 1.30330 298.30330 149.80329999999987 14980.33000 2020-01-01 2020-01-02 2020-01-01 00:07:14 2020-01-02 03:35:35 2020-01-01 00:07:14.000 2020-01-02 03:35:35.000 434 99335 49884.5 4988450 434 99335 49884.5 4988450 -32736 32402 4664.66 466466 -128 127 0.34 34 -435 101 10425 99336 1.3063 298.3063 149.8063 14980.63063 1.3063 298.3063 149.8063 14980.63051 1.30630 298.30630 149.80630000000008 14980.63000 2020-01-01 2020-01-02 2020-01-01 00:07:15 2020-01-02 03:35:36 2020-01-01 00:07:15.000 2020-01-02 03:35:36.000 435 99336 49885.5 4988550 435 99336 49885.5 4988550 -32735 32403 4665.66 466566 -128 127 -1.22 -122 -436 101 10426 99337 1.3093 298.3093 149.8093 14980.93093 1.3093 298.3093 149.8093 14980.93084 1.30930 298.30930 149.80930000000006 14980.93000 2020-01-01 2020-01-02 2020-01-01 00:07:16 2020-01-02 03:35:37 2020-01-01 00:07:16.000 2020-01-02 03:35:37.000 436 99337 49886.5 4988650 436 99337 49886.5 4988650 -32734 32404 4666.66 466666 -128 123 -2.78 -278 +421 101 10411 99322 1.26426 298.26426 149.76426 14976.42642 1.26426 298.26425 149.76426 14976.4264 1.26426 298.26426 149.76426 14976.42600 2020-01-01 2020-01-02 2020-01-01 00:07:01 2020-01-02 03:35:22 2020-01-01 00:07:01.000 2020-01-02 03:35:22.000 421 99322 49871.5 4987150 421 99322 49871.5 4987150 -32749 32389 4651.66 465166 -124 127 2.7 270 +422 101 10412 99323 1.26726 298.26726 149.76726 14976.72672 1.26726 298.26727 149.76727 14976.72702 1.26726 298.26726 149.76726 14976.72600 2020-01-01 2020-01-02 2020-01-01 00:07:02 2020-01-02 03:35:23 2020-01-01 00:07:02.000 2020-01-02 03:35:23.000 422 99323 49872.5 4987250 422 99323 49872.5 4987250 -32748 32390 4652.66 465266 -128 127 1.14 114 +423 101 10413 99324 1.27027 298.27027 149.77027 14977.02702 1.27027 298.27026 149.77026 14977.02667 1.27027 298.27027 149.77027 14977.02700 2020-01-01 2020-01-02 2020-01-01 00:07:03 2020-01-02 03:35:24 2020-01-01 00:07:03.000 2020-01-02 03:35:24.000 423 99324 49873.5 4987350 423 99324 49873.5 4987350 -32747 32391 4653.66 465366 -128 123 -0.42 -42 +424 101 10414 99325 1.27327 298.27327 149.77327 14977.32732 1.27327 298.2733 149.77327 14977.32727 1.27327 298.27327 149.77327 14977.32700 2020-01-01 2020-01-02 2020-01-01 00:07:04 2020-01-02 03:35:25 2020-01-01 00:07:04.000 2020-01-02 03:35:25.000 424 99325 49874.5 4987450 424 99325 49874.5 4987450 -32746 32392 4654.66 465466 -127 124 0.58 58 +425 101 10415 99326 1.27627 298.27627 149.77627 14977.62762 1.27627 298.27628 149.77627 14977.62756 1.27627 298.27627 149.77627 14977.62700 2020-01-01 2020-01-02 2020-01-01 00:07:05 2020-01-02 03:35:26 2020-01-01 00:07:05.000 2020-01-02 03:35:26.000 425 99326 49875.5 4987550 425 99326 49875.5 4987550 -32745 32393 4655.66 465566 -126 125 1.58 158 +426 101 10416 99327 1.27927 298.27927 149.77927 14977.92792 1.27927 298.27927 149.77927 14977.92787 1.27927 298.27927 149.77927 14977.92700 2020-01-01 2020-01-02 2020-01-01 00:07:06 2020-01-02 03:35:27 2020-01-01 00:07:06.000 2020-01-02 03:35:27.000 426 99327 49876.5 4987650 426 99327 49876.5 4987650 -32744 32394 4656.66 465666 -125 126 2.58 258 +427 101 10417 99328 1.28228 298.28228 149.78228 14978.22822 1.28228 298.2823 149.78228 14978.22849 1.28228 298.28228 149.78228 14978.22800 2020-01-01 2020-01-02 2020-01-01 00:07:07 2020-01-02 03:35:28 2020-01-01 00:07:07.000 2020-01-02 03:35:28.000 427 99328 49877.5 4987750 427 99328 49877.5 4987750 -32743 32395 4657.66 465766 -124 127 3.58 358 +428 101 10418 99329 1.28528 298.28528 149.78528 14978.52852 1.28528 298.28528 149.78528 14978.52815 1.28528 298.28528 149.78528 14978.52800 2020-01-01 2020-01-02 2020-01-01 00:07:08 2020-01-02 03:35:29 2020-01-01 00:07:08.000 2020-01-02 03:35:29.000 428 99329 49878.5 4987850 428 99329 49878.5 4987850 -32742 32396 4658.66 465866 -128 127 2.02 202 +429 101 10419 99330 1.28828 298.28828 149.78828 14978.82882 1.28828 298.2883 149.78828 14978.8289 1.28828 298.28828 149.78828 14978.82800 2020-01-01 2020-01-02 2020-01-01 00:07:09 2020-01-02 03:35:30 2020-01-01 00:07:09.000 2020-01-02 03:35:30.000 429 99330 49879.5 4987950 429 99330 49879.5 4987950 -32741 32397 4659.66 465966 -128 127 0.46 46 +43 102 10033 99943 0.12912 300.12912 150.12912 15163.04204 0.12912 300.12912 150.12912 15163.04211 0.12912 300.12912 150.12912 15163.04112 2020-01-01 2020-01-02 2020-01-01 00:00:43 2020-01-02 03:45:43 2020-01-01 00:00:43.000 2020-01-02 03:45:43.000 43 99943 49993 5049293 43 99943 49993 5049293 -32526 32409 4572.009900990099 461773 -128 124 -3.0396039603960396 -307 +430 101 10420 99331 1.29129 298.29129 149.79129 14979.12912 1.29129 298.2913 149.79129 14979.12904 1.29129 298.29129 149.79129 14979.12900 2020-01-01 2020-01-02 2020-01-01 00:07:10 2020-01-02 03:35:31 2020-01-01 00:07:10.000 2020-01-02 03:35:31.000 430 99331 49880.5 4988050 430 99331 49880.5 4988050 -32740 32398 4660.66 466066 -128 124 -1.1 -110 +431 101 10421 99332 1.29429 298.29429 149.79429 14979.42942 1.29429 298.29428 149.79429 14979.42933 1.29429 298.29429 149.79429 14979.42900 2020-01-01 2020-01-02 2020-01-01 00:07:11 2020-01-02 03:35:32 2020-01-01 00:07:11.000 2020-01-02 03:35:32.000 431 99332 49881.5 4988150 431 99332 49881.5 4988150 -32739 32399 4661.66 466166 -127 125 -0.1 -10 +432 101 10422 99333 1.29729 298.29729 149.79729 14979.72972 1.29729 298.2973 149.79729 14979.72996 1.29729 298.29729 149.79729 14979.72900 2020-01-01 2020-01-02 2020-01-01 00:07:12 2020-01-02 03:35:33 2020-01-01 00:07:12.000 2020-01-02 03:35:33.000 432 99333 49882.5 4988250 432 99333 49882.5 4988250 -32738 32400 4662.66 466266 -126 126 0.9 90 +433 101 10423 99334 1.3003 298.3003 149.8003 14980.03003 1.3003 298.3003 149.80029 14980.02962 1.30030 298.30030 149.8003 14980.03000 2020-01-01 2020-01-02 2020-01-01 00:07:13 2020-01-02 03:35:34 2020-01-01 00:07:13.000 2020-01-02 03:35:34.000 433 99334 49883.5 4988350 433 99334 49883.5 4988350 -32737 32401 4663.66 466366 -125 127 1.9 190 +434 101 10424 99335 1.3033 298.3033 149.8033 14980.33033 1.3033 298.3033 149.8033 14980.33037 1.30330 298.30330 149.8033 14980.33000 2020-01-01 2020-01-02 2020-01-01 00:07:14 2020-01-02 03:35:35 2020-01-01 00:07:14.000 2020-01-02 03:35:35.000 434 99335 49884.5 4988450 434 99335 49884.5 4988450 -32736 32402 4664.66 466466 -128 127 0.34 34 +435 101 10425 99336 1.3063 298.3063 149.8063 14980.63063 1.3063 298.3063 149.8063 14980.63051 1.30630 298.30630 149.8063 14980.63000 2020-01-01 2020-01-02 2020-01-01 00:07:15 2020-01-02 03:35:36 2020-01-01 00:07:15.000 2020-01-02 03:35:36.000 435 99336 49885.5 4988550 435 99336 49885.5 4988550 -32735 32403 4665.66 466566 -128 127 -1.22 -122 +436 101 10426 99337 1.3093 298.3093 149.8093 14980.93093 1.3093 298.3093 149.8093 14980.93084 1.30930 298.30930 149.8093 14980.93000 2020-01-01 2020-01-02 2020-01-01 00:07:16 2020-01-02 03:35:37 2020-01-01 00:07:16.000 2020-01-02 03:35:37.000 436 99337 49886.5 4988650 436 99337 49886.5 4988650 -32734 32404 4666.66 466666 -128 123 -2.78 -278 437 101 10427 99338 1.31231 298.31231 149.81231 14981.23123 1.31231 298.31232 149.81231 14981.23143 1.31231 298.31231 149.81231 14981.23100 2020-01-01 2020-01-02 2020-01-01 00:07:17 2020-01-02 03:35:38 2020-01-01 00:07:17.000 2020-01-02 03:35:38.000 437 99338 49887.5 4988750 437 99338 49887.5 4988750 -32733 32405 4667.66 466766 -127 124 -1.78 -178 -438 101 10428 99339 1.31531 298.31531 149.81531 14981.53153 1.31531 298.3153 149.81531 14981.53173 1.31531 298.31531 149.81530999999998 14981.53100 2020-01-01 2020-01-02 2020-01-01 00:07:18 2020-01-02 03:35:39 2020-01-01 00:07:18.000 2020-01-02 03:35:39.000 438 99339 49888.5 4988850 438 99339 49888.5 4988850 -32732 32406 4668.66 466866 -126 125 -0.78 -78 -439 101 10429 99340 1.31831 298.31831 149.81831 14981.83183 1.31831 298.31833 149.81831 14981.83184 1.31831 298.31831 149.81831000000008 14981.83100 2020-01-01 2020-01-02 2020-01-01 00:07:19 2020-01-02 03:35:40 2020-01-01 00:07:19.000 2020-01-02 03:35:40.000 439 99340 49889.5 4988950 439 99340 49889.5 4988950 -32731 32407 4669.66 466966 -125 126 0.22 22 +438 101 10428 99339 1.31531 298.31531 149.81531 14981.53153 1.31531 298.3153 149.81531 14981.53173 1.31531 298.31531 149.81531 14981.53100 2020-01-01 2020-01-02 2020-01-01 00:07:18 2020-01-02 03:35:39 2020-01-01 00:07:18.000 2020-01-02 03:35:39.000 438 99339 49888.5 4988850 438 99339 49888.5 4988850 -32732 32406 4668.66 466866 -126 125 -0.78 -78 +439 101 10429 99340 1.31831 298.31831 149.81831 14981.83183 1.31831 298.31833 149.81831 14981.83184 1.31831 298.31831 149.81831 14981.83100 2020-01-01 2020-01-02 2020-01-01 00:07:19 2020-01-02 03:35:40 2020-01-01 00:07:19.000 2020-01-02 03:35:40.000 439 99340 49889.5 4988950 439 99340 49889.5 4988950 -32731 32407 4669.66 466966 -125 126 0.22 22 44 102 10034 99944 0.13213 300.13213 150.13213 15163.34534 0.13213 300.13214 150.13213 15163.34525 0.13213 300.13213 150.13213 15163.34513 2020-01-01 2020-01-02 2020-01-01 00:00:44 2020-01-02 03:45:44 2020-01-01 00:00:44.000 2020-01-02 03:45:44.000 44 99944 49994 5049394 44 99944 49994 5049394 -32525 32410 4573.009900990099 461874 -127 125 -2.0396039603960396 -206 -440 101 10430 99341 1.32132 298.32132 149.82132 14982.13213 1.32132 298.32132 149.82131 14982.13197 1.32132 298.32132 149.82131999999973 14982.13200 2020-01-01 2020-01-02 2020-01-01 00:07:20 2020-01-02 03:35:41 2020-01-01 00:07:20.000 2020-01-02 03:35:41.000 440 99341 49890.5 4989050 440 99341 49890.5 4989050 -32730 32408 4670.66 467066 -124 127 1.22 122 -441 101 10431 99342 1.32432 298.32432 149.82432 14982.43243 1.32432 298.3243 149.82432 14982.4323 1.32432 298.32432 149.82431999999983 14982.43200 2020-01-01 2020-01-02 2020-01-01 00:07:21 2020-01-02 03:35:42 2020-01-01 00:07:21.000 2020-01-02 03:35:42.000 441 99342 49891.5 4989150 441 99342 49891.5 4989150 -32729 32409 4671.66 467166 -128 127 -0.34 -34 -442 101 10432 99343 1.32732 298.32732 149.82732 14982.73273 1.32732 298.32733 149.82732 14982.7329 1.32732 298.32732 149.82732000000019 14982.73200 2020-01-01 2020-01-02 2020-01-01 00:07:22 2020-01-02 03:35:43 2020-01-01 00:07:22.000 2020-01-02 03:35:43.000 442 99343 49892.5 4989250 442 99343 49892.5 4989250 -32728 32410 4672.66 467266 -128 123 -1.9 -190 -443 101 10433 99344 1.33033 298.33033 149.83033 14983.03303 1.33033 298.33032 149.83033 14983.03319 1.33033 298.33033 149.8303300000001 14983.03300 2020-01-01 2020-01-02 2020-01-01 00:07:23 2020-01-02 03:35:44 2020-01-01 00:07:23.000 2020-01-02 03:35:44.000 443 99344 49893.5 4989350 443 99344 49893.5 4989350 -32727 32411 4673.66 467366 -127 124 -0.9 -90 -444 101 10434 99345 1.33333 298.33333 149.83333 14983.33333 1.33333 298.33334 149.83333 14983.33331 1.33333 298.33333 149.83332999999996 14983.33300 2020-01-01 2020-01-02 2020-01-01 00:07:24 2020-01-02 03:35:45 2020-01-01 00:07:24.000 2020-01-02 03:35:45.000 444 99345 49894.5 4989450 444 99345 49894.5 4989450 -32726 32412 4674.66 467466 -126 125 0.1 10 -445 101 10435 99346 1.33633 298.33633 149.83633 14983.63363 1.33633 298.33633 149.83633 14983.63348 1.33633 298.33633 149.83633000000006 14983.63300 2020-01-01 2020-01-02 2020-01-01 00:07:25 2020-01-02 03:35:46 2020-01-01 00:07:25.000 2020-01-02 03:35:46.000 445 99346 49895.5 4989550 445 99346 49895.5 4989550 -32725 32413 4675.66 467566 -125 126 1.1 110 -446 101 10436 99347 1.33933 298.33933 149.83933 14983.93393 1.33933 298.33932 149.83933 14983.93378 1.33933 298.33933 149.8393300000001 14983.93300 2020-01-01 2020-01-02 2020-01-01 00:07:26 2020-01-02 03:35:47 2020-01-01 00:07:26.000 2020-01-02 03:35:47.000 446 99347 49896.5 4989650 446 99347 49896.5 4989650 -32724 32414 4676.66 467666 -124 127 2.1 210 -447 101 10437 99348 1.34234 298.34234 149.84234 14984.23423 1.34234 298.34235 149.84234 14984.23437 1.34234 298.34234 149.84233999999984 14984.23400 2020-01-01 2020-01-02 2020-01-01 00:07:27 2020-01-02 03:35:48 2020-01-01 00:07:27.000 2020-01-02 03:35:48.000 447 99348 49897.5 4989750 447 99348 49897.5 4989750 -32723 32415 4677.66 467766 -128 127 0.54 54 -448 101 10438 99349 1.34534 298.34534 149.84534 14984.53453 1.34534 298.34534 149.84534 14984.53466 1.34534 298.34534 149.84533999999988 14984.53400 2020-01-01 2020-01-02 2020-01-01 00:07:28 2020-01-02 03:35:49 2020-01-01 00:07:28.000 2020-01-02 03:35:49.000 448 99349 49898.5 4989850 448 99349 49898.5 4989850 -32722 32416 4678.66 467866 -128 123 -1.02 -102 -449 101 10439 99350 1.34834 298.34834 149.84834 14984.83483 1.34834 298.34836 149.84834 14984.83478 1.34834 298.34834 149.8483400000001 14984.83400 2020-01-01 2020-01-02 2020-01-01 00:07:29 2020-01-02 03:35:50 2020-01-01 00:07:29.000 2020-01-02 03:35:50.000 449 99350 49899.5 4989950 449 99350 49899.5 4989950 -32721 32417 4679.66 467966 -127 124 -0.02 -2 -45 102 10035 99945 0.13513 300.13513 150.13513 15163.64864 0.13513 300.13513 150.13513 15163.64839 0.13513 300.13513 150.13513000000006 15163.64813 2020-01-01 2020-01-02 2020-01-01 00:00:45 2020-01-02 03:45:45 2020-01-01 00:00:45.000 2020-01-02 03:45:45.000 45 99945 49995 5049495 45 99945 49995 5049495 -32524 32411 4574.009900990099 461975 -126 126 -1.0396039603960396 -105 -450 101 10440 99351 1.35135 298.35135 149.85135 14985.13513 1.35135 298.35135 149.85134 14985.13495 1.35135 298.35135 149.8513500000002 14985.13500 2020-01-01 2020-01-02 2020-01-01 00:07:30 2020-01-02 03:35:51 2020-01-01 00:07:30.000 2020-01-02 03:35:51.000 450 99351 49900.5 4990050 450 99351 49900.5 4990050 -32720 32418 4680.66 468066 -126 125 0.98 98 -451 101 10441 99352 1.35435 298.35435 149.85435 14985.43543 1.35435 298.35434 149.85435 14985.43525 1.35435 298.35435 149.85434999999976 14985.43500 2020-01-01 2020-01-02 2020-01-01 00:07:31 2020-01-02 03:35:52 2020-01-01 00:07:31.000 2020-01-02 03:35:52.000 451 99352 49901.5 4990150 451 99352 49901.5 4990150 -32719 32419 4681.66 468166 -125 126 1.98 198 -452 101 10442 99353 1.35735 298.35735 149.85735 14985.73573 1.35735 298.35736 149.85736 14985.736 1.35735 298.35735 149.8573500000002 14985.73500 2020-01-01 2020-01-02 2020-01-01 00:07:32 2020-01-02 03:35:53 2020-01-01 00:07:32.000 2020-01-02 03:35:53.000 452 99353 49902.5 4990250 452 99353 49902.5 4990250 -32718 32420 4682.66 468266 -124 127 2.98 298 -453 101 10443 99354 1.36036 298.36036 149.86036 14986.03603 1.36036 298.36035 149.86036 14986.03614 1.36036 298.36036 149.8603600000001 14986.03600 2020-01-01 2020-01-02 2020-01-01 00:07:33 2020-01-02 03:35:54 2020-01-01 00:07:33.000 2020-01-02 03:35:54.000 453 99354 49903.5 4990350 453 99354 49903.5 4990350 -32717 32421 4683.66 468366 -128 127 1.42 142 -454 101 10444 99355 1.36336 298.36336 149.86336 14986.33633 1.36336 298.36337 149.86336 14986.33629 1.36336 298.36336 149.86335999999986 14986.33600 2020-01-01 2020-01-02 2020-01-01 00:07:34 2020-01-02 03:35:55 2020-01-01 00:07:34.000 2020-01-02 03:35:55.000 454 99355 49904.5 4990450 454 99355 49904.5 4990450 -32716 32422 4684.66 468466 -128 127 -0.14 -14 -455 101 10445 99356 1.36636 298.36636 149.86636 14986.63663 1.36636 298.36636 149.86636 14986.63641 1.36636 298.36636 149.86636 14986.63600 2020-01-01 2020-01-02 2020-01-01 00:07:35 2020-01-02 03:35:56 2020-01-01 00:07:35.000 2020-01-02 03:35:56.000 455 99356 49905.5 4990550 455 99356 49905.5 4990550 -32715 32423 4685.66 468566 -128 124 -1.7 -170 -456 101 10446 99357 1.36936 298.36936 149.86936 14986.93693 1.36936 298.36935 149.86936 14986.93672 1.36936 298.36936 149.8693600000001 14986.93600 2020-01-01 2020-01-02 2020-01-01 00:07:36 2020-01-02 03:35:57 2020-01-01 00:07:36.000 2020-01-02 03:35:57.000 456 99357 49906.5 4990650 456 99357 49906.5 4990650 -32714 32424 4686.66 468666 -127 125 -0.7 -70 -457 101 10447 99358 1.37237 298.37237 149.87237 14987.23723 1.37237 298.37238 149.87237 14987.23747 1.37237 298.37237 149.87236999999976 14987.23700 2020-01-01 2020-01-02 2020-01-01 00:07:37 2020-01-02 03:35:58 2020-01-01 00:07:37.000 2020-01-02 03:35:58.000 457 99358 49907.5 4990750 457 99358 49907.5 4990750 -32713 32425 4687.66 468766 -126 126 0.3 30 -458 101 10448 99359 1.37537 298.37537 149.87537 14987.53753 1.37537 298.37537 149.87537 14987.5376 1.37537 298.37537 149.87536999999986 14987.53700 2020-01-01 2020-01-02 2020-01-01 00:07:38 2020-01-02 03:35:59 2020-01-01 00:07:38.000 2020-01-02 03:35:59.000 458 99359 49908.5 4990850 458 99359 49908.5 4990850 -32712 32426 4688.66 468866 -125 127 1.3 130 -459 101 10449 99360 1.37837 298.37837 149.87837 14987.83783 1.37837 298.3784 149.87837 14987.83775 1.37837 298.37837 149.87837000000022 14987.83700 2020-01-01 2020-01-02 2020-01-01 00:07:39 2020-01-02 03:36:00 2020-01-01 00:07:39.000 2020-01-02 03:36:00.000 459 99360 49909.5 4990950 459 99360 49909.5 4990950 -32711 32427 4689.66 468966 -128 127 -0.26 -26 -46 102 10036 99946 0.13813 300.13813 150.13813 15163.95195 0.13813 300.13815 150.13814 15163.95214 0.13813 300.13813 150.13812999999988 15163.95113 2020-01-01 2020-01-02 2020-01-01 00:00:46 2020-01-02 03:45:46 2020-01-01 00:00:46.000 2020-01-02 03:45:46.000 46 99946 49996 5049596 46 99946 49996 5049596 -32523 32412 4575.009900990099 462076 -125 127 -0.039603960396039604 -4 -460 101 10450 99361 1.38138 298.38138 149.88138 14988.13813 1.38138 298.38138 149.88137 14988.13789 1.38138 298.38138 149.88138000000012 14988.13800 2020-01-01 2020-01-02 2020-01-01 00:07:40 2020-01-02 03:36:01 2020-01-01 00:07:40.000 2020-01-02 03:36:01.000 460 99361 49910.5 4991050 460 99361 49910.5 4991050 -32710 32428 4690.66 469066 -128 127 -1.82 -182 +440 101 10430 99341 1.32132 298.32132 149.82132 14982.13213 1.32132 298.32132 149.82131 14982.13197 1.32132 298.32132 149.82132 14982.13200 2020-01-01 2020-01-02 2020-01-01 00:07:20 2020-01-02 03:35:41 2020-01-01 00:07:20.000 2020-01-02 03:35:41.000 440 99341 49890.5 4989050 440 99341 49890.5 4989050 -32730 32408 4670.66 467066 -124 127 1.22 122 +441 101 10431 99342 1.32432 298.32432 149.82432 14982.43243 1.32432 298.3243 149.82432 14982.4323 1.32432 298.32432 149.82432 14982.43200 2020-01-01 2020-01-02 2020-01-01 00:07:21 2020-01-02 03:35:42 2020-01-01 00:07:21.000 2020-01-02 03:35:42.000 441 99342 49891.5 4989150 441 99342 49891.5 4989150 -32729 32409 4671.66 467166 -128 127 -0.34 -34 +442 101 10432 99343 1.32732 298.32732 149.82732 14982.73273 1.32732 298.32733 149.82732 14982.7329 1.32732 298.32732 149.82732 14982.73200 2020-01-01 2020-01-02 2020-01-01 00:07:22 2020-01-02 03:35:43 2020-01-01 00:07:22.000 2020-01-02 03:35:43.000 442 99343 49892.5 4989250 442 99343 49892.5 4989250 -32728 32410 4672.66 467266 -128 123 -1.9 -190 +443 101 10433 99344 1.33033 298.33033 149.83033 14983.03303 1.33033 298.33032 149.83033 14983.03319 1.33033 298.33033 149.83033 14983.03300 2020-01-01 2020-01-02 2020-01-01 00:07:23 2020-01-02 03:35:44 2020-01-01 00:07:23.000 2020-01-02 03:35:44.000 443 99344 49893.5 4989350 443 99344 49893.5 4989350 -32727 32411 4673.66 467366 -127 124 -0.9 -90 +444 101 10434 99345 1.33333 298.33333 149.83333 14983.33333 1.33333 298.33334 149.83333 14983.33331 1.33333 298.33333 149.83333000000002 14983.33300 2020-01-01 2020-01-02 2020-01-01 00:07:24 2020-01-02 03:35:45 2020-01-01 00:07:24.000 2020-01-02 03:35:45.000 444 99345 49894.5 4989450 444 99345 49894.5 4989450 -32726 32412 4674.66 467466 -126 125 0.1 10 +445 101 10435 99346 1.33633 298.33633 149.83633 14983.63363 1.33633 298.33633 149.83633 14983.63348 1.33633 298.33633 149.83633 14983.63300 2020-01-01 2020-01-02 2020-01-01 00:07:25 2020-01-02 03:35:46 2020-01-01 00:07:25.000 2020-01-02 03:35:46.000 445 99346 49895.5 4989550 445 99346 49895.5 4989550 -32725 32413 4675.66 467566 -125 126 1.1 110 +446 101 10436 99347 1.33933 298.33933 149.83933 14983.93393 1.33933 298.33932 149.83933 14983.93378 1.33933 298.33933 149.83933000000002 14983.93300 2020-01-01 2020-01-02 2020-01-01 00:07:26 2020-01-02 03:35:47 2020-01-01 00:07:26.000 2020-01-02 03:35:47.000 446 99347 49896.5 4989650 446 99347 49896.5 4989650 -32724 32414 4676.66 467666 -124 127 2.1 210 +447 101 10437 99348 1.34234 298.34234 149.84234 14984.23423 1.34234 298.34235 149.84234 14984.23437 1.34234 298.34234 149.84234 14984.23400 2020-01-01 2020-01-02 2020-01-01 00:07:27 2020-01-02 03:35:48 2020-01-01 00:07:27.000 2020-01-02 03:35:48.000 447 99348 49897.5 4989750 447 99348 49897.5 4989750 -32723 32415 4677.66 467766 -128 127 0.54 54 +448 101 10438 99349 1.34534 298.34534 149.84534 14984.53453 1.34534 298.34534 149.84534 14984.53466 1.34534 298.34534 149.84534 14984.53400 2020-01-01 2020-01-02 2020-01-01 00:07:28 2020-01-02 03:35:49 2020-01-01 00:07:28.000 2020-01-02 03:35:49.000 448 99349 49898.5 4989850 448 99349 49898.5 4989850 -32722 32416 4678.66 467866 -128 123 -1.02 -102 +449 101 10439 99350 1.34834 298.34834 149.84834 14984.83483 1.34834 298.34836 149.84834 14984.83478 1.34834 298.34834 149.84834 14984.83400 2020-01-01 2020-01-02 2020-01-01 00:07:29 2020-01-02 03:35:50 2020-01-01 00:07:29.000 2020-01-02 03:35:50.000 449 99350 49899.5 4989950 449 99350 49899.5 4989950 -32721 32417 4679.66 467966 -127 124 -0.02 -2 +45 102 10035 99945 0.13513 300.13513 150.13513 15163.64864 0.13513 300.13513 150.13513 15163.64839 0.13513 300.13513 150.13513 15163.64813 2020-01-01 2020-01-02 2020-01-01 00:00:45 2020-01-02 03:45:45 2020-01-01 00:00:45.000 2020-01-02 03:45:45.000 45 99945 49995 5049495 45 99945 49995 5049495 -32524 32411 4574.009900990099 461975 -126 126 -1.0396039603960396 -105 +450 101 10440 99351 1.35135 298.35135 149.85135 14985.13513 1.35135 298.35135 149.85134 14985.13495 1.35135 298.35135 149.85135 14985.13500 2020-01-01 2020-01-02 2020-01-01 00:07:30 2020-01-02 03:35:51 2020-01-01 00:07:30.000 2020-01-02 03:35:51.000 450 99351 49900.5 4990050 450 99351 49900.5 4990050 -32720 32418 4680.66 468066 -126 125 0.98 98 +451 101 10441 99352 1.35435 298.35435 149.85435 14985.43543 1.35435 298.35434 149.85435 14985.43525 1.35435 298.35435 149.85434999999998 14985.43500 2020-01-01 2020-01-02 2020-01-01 00:07:31 2020-01-02 03:35:52 2020-01-01 00:07:31.000 2020-01-02 03:35:52.000 451 99352 49901.5 4990150 451 99352 49901.5 4990150 -32719 32419 4681.66 468166 -125 126 1.98 198 +452 101 10442 99353 1.35735 298.35735 149.85735 14985.73573 1.35735 298.35736 149.85736 14985.736 1.35735 298.35735 149.85735 14985.73500 2020-01-01 2020-01-02 2020-01-01 00:07:32 2020-01-02 03:35:53 2020-01-01 00:07:32.000 2020-01-02 03:35:53.000 452 99353 49902.5 4990250 452 99353 49902.5 4990250 -32718 32420 4682.66 468266 -124 127 2.98 298 +453 101 10443 99354 1.36036 298.36036 149.86036 14986.03603 1.36036 298.36035 149.86036 14986.03614 1.36036 298.36036 149.86036000000001 14986.03600 2020-01-01 2020-01-02 2020-01-01 00:07:33 2020-01-02 03:35:54 2020-01-01 00:07:33.000 2020-01-02 03:35:54.000 453 99354 49903.5 4990350 453 99354 49903.5 4990350 -32717 32421 4683.66 468366 -128 127 1.42 142 +454 101 10444 99355 1.36336 298.36336 149.86336 14986.33633 1.36336 298.36337 149.86336 14986.33629 1.36336 298.36336 149.86336 14986.33600 2020-01-01 2020-01-02 2020-01-01 00:07:34 2020-01-02 03:35:55 2020-01-01 00:07:34.000 2020-01-02 03:35:55.000 454 99355 49904.5 4990450 454 99355 49904.5 4990450 -32716 32422 4684.66 468466 -128 127 -0.14 -14 +455 101 10445 99356 1.36636 298.36636 149.86636 14986.63663 1.36636 298.36636 149.86636 14986.63641 1.36636 298.36636 149.86636000000001 14986.63600 2020-01-01 2020-01-02 2020-01-01 00:07:35 2020-01-02 03:35:56 2020-01-01 00:07:35.000 2020-01-02 03:35:56.000 455 99356 49905.5 4990550 455 99356 49905.5 4990550 -32715 32423 4685.66 468566 -128 124 -1.7 -170 +456 101 10446 99357 1.36936 298.36936 149.86936 14986.93693 1.36936 298.36935 149.86936 14986.93672 1.36936 298.36936 149.86936 14986.93600 2020-01-01 2020-01-02 2020-01-01 00:07:36 2020-01-02 03:35:57 2020-01-01 00:07:36.000 2020-01-02 03:35:57.000 456 99357 49906.5 4990650 456 99357 49906.5 4990650 -32714 32424 4686.66 468666 -127 125 -0.7 -70 +457 101 10447 99358 1.37237 298.37237 149.87237 14987.23723 1.37237 298.37238 149.87237 14987.23747 1.37237 298.37237 149.87237 14987.23700 2020-01-01 2020-01-02 2020-01-01 00:07:37 2020-01-02 03:35:58 2020-01-01 00:07:37.000 2020-01-02 03:35:58.000 457 99358 49907.5 4990750 457 99358 49907.5 4990750 -32713 32425 4687.66 468766 -126 126 0.3 30 +458 101 10448 99359 1.37537 298.37537 149.87537 14987.53753 1.37537 298.37537 149.87537 14987.5376 1.37537 298.37537 149.87537 14987.53700 2020-01-01 2020-01-02 2020-01-01 00:07:38 2020-01-02 03:35:59 2020-01-01 00:07:38.000 2020-01-02 03:35:59.000 458 99359 49908.5 4990850 458 99359 49908.5 4990850 -32712 32426 4688.66 468866 -125 127 1.3 130 +459 101 10449 99360 1.37837 298.37837 149.87837 14987.83783 1.37837 298.3784 149.87837 14987.83775 1.37837 298.37837 149.87837 14987.83700 2020-01-01 2020-01-02 2020-01-01 00:07:39 2020-01-02 03:36:00 2020-01-01 00:07:39.000 2020-01-02 03:36:00.000 459 99360 49909.5 4990950 459 99360 49909.5 4990950 -32711 32427 4689.66 468966 -128 127 -0.26 -26 +46 102 10036 99946 0.13813 300.13813 150.13813 15163.95195 0.13813 300.13815 150.13814 15163.95214 0.13813 300.13813 150.13813 15163.95113 2020-01-01 2020-01-02 2020-01-01 00:00:46 2020-01-02 03:45:46 2020-01-01 00:00:46.000 2020-01-02 03:45:46.000 46 99946 49996 5049596 46 99946 49996 5049596 -32523 32412 4575.009900990099 462076 -125 127 -0.039603960396039604 -4 +460 101 10450 99361 1.38138 298.38138 149.88138 14988.13813 1.38138 298.38138 149.88137 14988.13789 1.38138 298.38138 149.88138 14988.13800 2020-01-01 2020-01-02 2020-01-01 00:07:40 2020-01-02 03:36:01 2020-01-01 00:07:40.000 2020-01-02 03:36:01.000 460 99361 49910.5 4991050 460 99361 49910.5 4991050 -32710 32428 4690.66 469066 -128 127 -1.82 -182 461 101 10451 99362 1.38438 298.38438 149.88438 14988.43843 1.38438 298.3844 149.88438 14988.43864 1.38438 298.38438 149.88438 14988.43800 2020-01-01 2020-01-02 2020-01-01 00:07:41 2020-01-02 03:36:02 2020-01-01 00:07:41.000 2020-01-02 03:36:02.000 461 99362 49911.5 4991150 461 99362 49911.5 4991150 -32709 32429 4691.66 469166 -128 123 -3.38 -338 -462 101 10452 99363 1.38738 298.38738 149.88738 14988.73873 1.38738 298.3874 149.88738 14988.73894 1.38738 298.38738 149.8873800000001 14988.73800 2020-01-01 2020-01-02 2020-01-01 00:07:42 2020-01-02 03:36:03 2020-01-01 00:07:42.000 2020-01-02 03:36:03.000 462 99363 49912.5 4991250 462 99363 49912.5 4991250 -32708 32430 4692.66 469266 -127 124 -2.38 -238 +462 101 10452 99363 1.38738 298.38738 149.88738 14988.73873 1.38738 298.3874 149.88738 14988.73894 1.38738 298.38738 149.88738 14988.73800 2020-01-01 2020-01-02 2020-01-01 00:07:42 2020-01-02 03:36:03 2020-01-01 00:07:42.000 2020-01-02 03:36:03.000 462 99363 49912.5 4991250 462 99363 49912.5 4991250 -32708 32430 4692.66 469266 -127 124 -2.38 -238 463 101 10453 99364 1.39039 298.39039 149.89039 14989.03903 1.39039 298.39038 149.89039 14989.03907 1.39039 298.39039 149.89039 14989.03900 2020-01-01 2020-01-02 2020-01-01 00:07:43 2020-01-02 03:36:04 2020-01-01 00:07:43.000 2020-01-02 03:36:04.000 463 99364 49913.5 4991350 463 99364 49913.5 4991350 -32707 32431 4693.66 469366 -126 125 -1.38 -138 -464 101 10454 99365 1.39339 298.39339 149.89339 14989.33933 1.39339 298.3934 149.89339 14989.33922 1.39339 298.39339 149.89338999999987 14989.33900 2020-01-01 2020-01-02 2020-01-01 00:07:44 2020-01-02 03:36:05 2020-01-01 00:07:44.000 2020-01-02 03:36:05.000 464 99365 49914.5 4991450 464 99365 49914.5 4991450 -32706 32432 4694.66 469466 -125 126 -0.38 -38 -465 101 10455 99366 1.39639 298.39639 149.89639 14989.63963 1.39639 298.3964 149.89639 14989.63936 1.39639 298.39639 149.8963899999999 14989.63900 2020-01-01 2020-01-02 2020-01-01 00:07:45 2020-01-02 03:36:06 2020-01-01 00:07:45.000 2020-01-02 03:36:06.000 465 99366 49915.5 4991550 465 99366 49915.5 4991550 -32705 32433 4695.66 469566 -124 127 0.62 62 +464 101 10454 99365 1.39339 298.39339 149.89339 14989.33933 1.39339 298.3934 149.89339 14989.33922 1.39339 298.39339 149.89339 14989.33900 2020-01-01 2020-01-02 2020-01-01 00:07:44 2020-01-02 03:36:05 2020-01-01 00:07:44.000 2020-01-02 03:36:05.000 464 99365 49914.5 4991450 464 99365 49914.5 4991450 -32706 32432 4694.66 469466 -125 126 -0.38 -38 +465 101 10455 99366 1.39639 298.39639 149.89639 14989.63963 1.39639 298.3964 149.89639 14989.63936 1.39639 298.39639 149.89639 14989.63900 2020-01-01 2020-01-02 2020-01-01 00:07:45 2020-01-02 03:36:06 2020-01-01 00:07:45.000 2020-01-02 03:36:06.000 465 99366 49915.5 4991550 465 99366 49915.5 4991550 -32705 32433 4695.66 469566 -124 127 0.62 62 466 101 10456 99367 1.39939 298.39939 149.89939 14989.93993 1.39939 298.3994 149.8994 14989.94011 1.39939 298.39939 149.89939 14989.93900 2020-01-01 2020-01-02 2020-01-01 00:07:46 2020-01-02 03:36:07 2020-01-01 00:07:46.000 2020-01-02 03:36:07.000 466 99367 49916.5 4991650 466 99367 49916.5 4991650 -32704 32434 4696.66 469666 -128 127 -0.94 -94 -467 101 10457 99368 1.4024 298.4024 149.9024 14990.24024 1.4024 298.4024 149.9024 14990.24041 1.40240 298.40240 149.90240000000023 14990.24000 2020-01-01 2020-01-02 2020-01-01 00:07:47 2020-01-02 03:36:08 2020-01-01 00:07:47.000 2020-01-02 03:36:08.000 467 99368 49917.5 4991750 467 99368 49917.5 4991750 -32703 32435 4697.66 469766 -128 123 -2.5 -250 -468 101 10458 99369 1.4054 298.4054 149.9054 14990.54054 1.4054 298.4054 149.9054 14990.54058 1.40540 298.40540 149.9053999999998 14990.54000 2020-01-01 2020-01-02 2020-01-01 00:07:48 2020-01-02 03:36:09 2020-01-01 00:07:48.000 2020-01-02 03:36:09.000 468 99369 49918.5 4991850 468 99369 49918.5 4991850 -32702 32436 4698.66 469866 -127 124 -1.5 -150 -469 101 10459 99370 1.4084 298.4084 149.9084 14990.84084 1.4084 298.40842 149.9084 14990.8407 1.40840 298.40840 149.90840000000014 14990.84000 2020-01-01 2020-01-02 2020-01-01 00:07:49 2020-01-02 03:36:10 2020-01-01 00:07:49.000 2020-01-02 03:36:10.000 469 99370 49919.5 4991950 469 99370 49919.5 4991950 -32701 32437 4699.66 469966 -126 125 -0.5 -50 -47 102 10037 99947 0.14114 300.14114 150.14114 15164.25525 0.14114 300.14114 150.14114 15164.25545 0.14114 300.14114 150.14113999999984 15164.25514 2020-01-01 2020-01-02 2020-01-01 00:00:47 2020-01-02 03:45:47 2020-01-01 00:00:47.000 2020-01-02 03:45:47.000 47 99947 49997 5049697 47 99947 49997 5049697 -32522 32413 4576.009900990099 462177 -128 127 -1.5742574257425743 -159 -470 101 10460 99371 1.41141 298.41141 149.91141 14991.14114 1.41141 298.4114 149.9114 14991.14099 1.41141 298.41141 149.91141000000013 14991.14100 2020-01-01 2020-01-02 2020-01-01 00:07:50 2020-01-02 03:36:11 2020-01-01 00:07:50.000 2020-01-02 03:36:11.000 470 99371 49920.5 4992050 470 99371 49920.5 4992050 -32700 32438 4700.66 470066 -125 126 0.5 50 -471 101 10461 99372 1.41441 298.41441 149.91441 14991.44144 1.41441 298.41443 149.91441 14991.44159 1.41441 298.41441 149.9144099999999 14991.44100 2020-01-01 2020-01-02 2020-01-01 00:07:51 2020-01-02 03:36:12 2020-01-01 00:07:51.000 2020-01-02 03:36:12.000 471 99372 49921.5 4992150 471 99372 49921.5 4992150 -32699 32439 4701.66 470166 -124 127 1.5 150 -472 101 10462 99373 1.41741 298.41741 149.91741 14991.74174 1.41741 298.41742 149.91741 14991.74188 1.41741 298.41741 149.91741000000002 14991.74100 2020-01-01 2020-01-02 2020-01-01 00:07:52 2020-01-02 03:36:13 2020-01-01 00:07:52.000 2020-01-02 03:36:13.000 472 99373 49922.5 4992250 472 99373 49922.5 4992250 -32698 32440 4702.66 470266 -128 127 -0.06 -6 -473 101 10463 99374 1.42042 298.42042 149.92042 14992.04204 1.42042 298.4204 149.92042 14992.04204 1.42042 298.42042 149.92042000000023 14992.04200 2020-01-01 2020-01-02 2020-01-01 00:07:53 2020-01-02 03:36:14 2020-01-01 00:07:53.000 2020-01-02 03:36:14.000 473 99374 49923.5 4992350 473 99374 49923.5 4992350 -32697 32441 4703.66 470366 -128 123 -1.62 -162 -474 101 10464 99375 1.42342 298.42342 149.92342 14992.34234 1.42342 298.42343 149.92342 14992.34216 1.42342 298.42342 149.9234199999998 14992.34200 2020-01-01 2020-01-02 2020-01-01 00:07:54 2020-01-02 03:36:15 2020-01-01 00:07:54.000 2020-01-02 03:36:15.000 474 99375 49924.5 4992450 474 99375 49924.5 4992450 -32696 32442 4704.66 470466 -127 124 -0.62 -62 -475 101 10465 99376 1.42642 298.42642 149.92642 14992.64264 1.42642 298.42642 149.92642 14992.64246 1.42642 298.42642 149.9264199999998 14992.64200 2020-01-01 2020-01-02 2020-01-01 00:07:55 2020-01-02 03:36:16 2020-01-01 00:07:55.000 2020-01-02 03:36:16.000 475 99376 49925.5 4992550 475 99376 49925.5 4992550 -32695 32443 4705.66 470566 -126 125 0.38 38 -476 101 10466 99377 1.42942 298.42942 149.92942 14992.94294 1.42942 298.42944 149.92943 14992.94305 1.42942 298.42942 149.92942000000025 14992.94200 2020-01-01 2020-01-02 2020-01-01 00:07:56 2020-01-02 03:36:17 2020-01-01 00:07:56.000 2020-01-02 03:36:17.000 476 99377 49926.5 4992650 476 99377 49926.5 4992650 -32694 32444 4706.66 470666 -125 126 1.38 138 -477 101 10467 99378 1.43243 298.43243 149.93243 14993.24324 1.43243 298.43243 149.93243 14993.24338 1.43243 298.43243 149.93243000000015 14993.24300 2020-01-01 2020-01-02 2020-01-01 00:07:57 2020-01-02 03:36:18 2020-01-01 00:07:57.000 2020-01-02 03:36:18.000 477 99378 49927.5 4992750 477 99378 49927.5 4992750 -32693 32445 4707.66 470766 -124 127 2.38 238 -478 101 10468 99379 1.43543 298.43543 149.93543 14993.54354 1.43543 298.43542 149.93543 14993.54352 1.43543 298.43543 149.93542999999968 14993.54300 2020-01-01 2020-01-02 2020-01-01 00:07:58 2020-01-02 03:36:19 2020-01-01 00:07:58.000 2020-01-02 03:36:19.000 478 99379 49928.5 4992850 478 99379 49928.5 4992850 -32692 32446 4708.66 470866 -128 127 0.82 82 -479 101 10469 99380 1.43843 298.43843 149.93843 14993.84384 1.43843 298.43845 149.93844 14993.84427 1.43843 298.43843 149.93843000000012 14993.84300 2020-01-01 2020-01-02 2020-01-01 00:07:59 2020-01-02 03:36:20 2020-01-01 00:07:59.000 2020-01-02 03:36:20.000 479 99380 49929.5 4992950 479 99380 49929.5 4992950 -32691 32447 4709.66 470966 -128 127 -0.74 -74 -48 102 10038 99948 0.14414 300.14414 150.14414 15164.55855 0.14414 300.14413 150.14414 15164.55863 0.14414 300.14414 150.14414000000022 15164.55814 2020-01-01 2020-01-02 2020-01-01 00:00:48 2020-01-02 03:45:48 2020-01-01 00:00:48.000 2020-01-02 03:45:48.000 48 99948 49998 5049798 48 99948 49998 5049798 -32521 32414 4577.009900990099 462278 -128 127 -3.108910891089109 -314 -480 101 10470 99381 1.44144 298.44144 149.94144 14994.14414 1.44144 298.44144 149.94143 14994.14392 1.44144 298.44144 149.94144000000003 14994.14400 2020-01-01 2020-01-02 2020-01-01 00:08:00 2020-01-02 03:36:21 2020-01-01 00:08:00.000 2020-01-02 03:36:21.000 480 99381 49930.5 4993050 480 99381 49930.5 4993050 -32690 32448 4710.66 471066 -128 124 -2.3 -230 -481 101 10471 99382 1.44444 298.44444 149.94444 14994.44444 1.44444 298.44446 149.94444 14994.44452 1.44444 298.44444 149.9444399999999 14994.44400 2020-01-01 2020-01-02 2020-01-01 00:08:01 2020-01-02 03:36:22 2020-01-01 00:08:01.000 2020-01-02 03:36:22.000 481 99382 49931.5 4993150 481 99382 49931.5 4993150 -32689 32449 4711.66 471166 -127 125 -1.3 -130 -482 101 10472 99383 1.44744 298.44744 149.94744 14994.74474 1.44744 298.44745 149.94744 14994.74485 1.44744 298.44744 149.94743999999994 14994.74400 2020-01-01 2020-01-02 2020-01-01 00:08:02 2020-01-02 03:36:23 2020-01-01 00:08:02.000 2020-01-02 03:36:23.000 482 99383 49932.5 4993250 482 99383 49932.5 4993250 -32688 32450 4712.66 471266 -126 126 -0.3 -30 -483 101 10473 99384 1.45045 298.45045 149.95045 14995.04504 1.45045 298.45044 149.95044 14995.04499 1.45045 298.45045 149.95045000000016 14995.04500 2020-01-01 2020-01-02 2020-01-01 00:08:03 2020-01-02 03:36:24 2020-01-01 00:08:03.000 2020-01-02 03:36:24.000 483 99384 49933.5 4993350 483 99384 49933.5 4993350 -32687 32451 4713.66 471366 -125 127 0.7 70 -484 101 10474 99385 1.45345 298.45345 149.95345 14995.34534 1.45345 298.45346 149.95345 14995.34574 1.45345 298.45345 149.95345000000026 14995.34500 2020-01-01 2020-01-02 2020-01-01 00:08:04 2020-01-02 03:36:25 2020-01-01 00:08:04.000 2020-01-02 03:36:25.000 484 99385 49934.5 4993450 484 99385 49934.5 4993450 -32686 32452 4714.66 471466 -128 127 -0.86 -86 -485 101 10475 99386 1.45645 298.45645 149.95645 14995.64564 1.45645 298.45645 149.95645 14995.6454 1.45645 298.45645 149.95644999999982 14995.64500 2020-01-01 2020-01-02 2020-01-01 00:08:05 2020-01-02 03:36:26 2020-01-01 00:08:05.000 2020-01-02 03:36:26.000 485 99386 49935.5 4993550 485 99386 49935.5 4993550 -32685 32453 4715.66 471566 -128 127 -2.42 -242 -486 101 10476 99387 1.45945 298.45945 149.95945 14995.94594 1.45945 298.45947 149.95946 14995.94602 1.45945 298.45945 149.95945000000017 14995.94500 2020-01-01 2020-01-02 2020-01-01 00:08:06 2020-01-02 03:36:27 2020-01-01 00:08:06.000 2020-01-02 03:36:27.000 486 99387 49936.5 4993650 486 99387 49936.5 4993650 -32684 32454 4716.66 471666 -128 123 -3.98 -398 -487 101 10477 99388 1.46246 298.46246 149.96246 14996.24624 1.46246 298.46246 149.96246 14996.24633 1.46246 298.46246 149.96246000000014 14996.24600 2020-01-01 2020-01-02 2020-01-01 00:08:07 2020-01-02 03:36:28 2020-01-01 00:08:07.000 2020-01-02 03:36:28.000 487 99388 49937.5 4993750 487 99388 49937.5 4993750 -32683 32455 4717.66 471766 -127 124 -2.98 -298 -488 101 10478 99389 1.46546 298.46546 149.96546 14996.54654 1.46546 298.46545 149.96546 14996.54645 1.46546 298.46546 149.96545999999995 14996.54600 2020-01-01 2020-01-02 2020-01-01 00:08:08 2020-01-02 03:36:29 2020-01-01 00:08:08.000 2020-01-02 03:36:29.000 488 99389 49938.5 4993850 488 99389 49938.5 4993850 -32682 32456 4718.66 471866 -126 125 -1.98 -198 -489 101 10479 99390 1.46846 298.46846 149.96846 14996.84684 1.46846 298.46848 149.96847 14996.84721 1.46846 298.46846 149.96846000000002 14996.84600 2020-01-01 2020-01-02 2020-01-01 00:08:09 2020-01-02 03:36:30 2020-01-01 00:08:09.000 2020-01-02 03:36:30.000 489 99390 49939.5 4993950 489 99390 49939.5 4993950 -32681 32457 4719.66 471966 -125 126 -0.98 -98 -49 102 10039 99949 0.14714 300.14714 150.14714 15164.86186 0.14714 300.14716 150.14714 15164.86173 0.14714 300.14714 150.14713999999972 15164.86114 2020-01-01 2020-01-02 2020-01-01 00:00:49 2020-01-02 03:45:49 2020-01-01 00:00:49.000 2020-01-02 03:45:49.000 49 99949 49999 5049899 49 99949 49999 5049899 -32520 32415 4578.009900990099 462379 -128 123 -4.643564356435643 -469 -490 101 10480 99391 1.47147 298.47147 149.97147 14997.14714 1.47147 298.47147 149.97146 14997.14687 1.47147 298.47147 149.9714700000003 14997.14700 2020-01-01 2020-01-02 2020-01-01 00:08:10 2020-01-02 03:36:31 2020-01-01 00:08:10.000 2020-01-02 03:36:31.000 490 99391 49940.5 4994050 490 99391 49940.5 4994050 -32680 32458 4720.66 472066 -124 127 0.02 2 -491 101 10481 99392 1.47447 298.47447 149.97447 14997.44744 1.47447 298.4745 149.97447 14997.44749 1.47447 298.47447 149.9744699999998 14997.44700 2020-01-01 2020-01-02 2020-01-01 00:08:11 2020-01-02 03:36:32 2020-01-01 00:08:11.000 2020-01-02 03:36:32.000 491 99392 49941.5 4994150 491 99392 49941.5 4994150 -32679 32459 4721.66 472166 -128 127 -1.54 -154 -492 101 10482 99393 1.47747 298.47747 149.97747 14997.74774 1.47747 298.47748 149.97747 14997.74779 1.47747 298.47747 149.97746999999984 14997.74700 2020-01-01 2020-01-02 2020-01-01 00:08:12 2020-01-02 03:36:33 2020-01-01 00:08:12.000 2020-01-02 03:36:33.000 492 99393 49942.5 4994250 492 99393 49942.5 4994250 -32678 32460 4722.66 472266 -128 123 -3.1 -310 -493 101 10483 99394 1.48048 298.48048 149.98048 14998.04804 1.48048 298.48047 149.98048 14998.04809 1.48048 298.48048 149.98048000000014 14998.04800 2020-01-01 2020-01-02 2020-01-01 00:08:13 2020-01-02 03:36:34 2020-01-01 00:08:13.000 2020-01-02 03:36:34.000 493 99394 49943.5 4994350 493 99394 49943.5 4994350 -32677 32461 4723.66 472366 -127 124 -2.1 -210 -494 101 10484 99395 1.48348 298.48348 149.98348 14998.34834 1.48348 298.4835 149.98348 14998.34868 1.48348 298.48348 149.98348000000018 14998.34800 2020-01-01 2020-01-02 2020-01-01 00:08:14 2020-01-02 03:36:35 2020-01-01 00:08:14.000 2020-01-02 03:36:35.000 494 99395 49944.5 4994450 494 99395 49944.5 4994450 -32676 32462 4724.66 472466 -126 125 -1.1 -110 -495 100 10485 99396 1.48648 298.48648 149.98648 14998.64864 1.48648 298.48648 149.98648 14998.64837 1.48648 298.48648 149.9864799999997 14998.64800 2020-01-01 2020-01-02 2020-01-01 00:08:15 2020-01-02 03:36:36 2020-01-01 00:08:15.000 2020-01-02 03:36:36.000 495 99396 49945.5 4994550 495 99396 49945.5 4994550 -32675 32463 4725.66 472566 -125 126 -0.1 -10 -496 100 10486 99397 1.48948 298.48948 149.98948 14998.94894 1.48948 298.4895 149.98948 14998.94896 1.48948 298.48948 149.98948000000016 14998.94800 2020-01-01 2020-01-02 2020-01-01 00:08:16 2020-01-02 03:36:37 2020-01-01 00:08:16.000 2020-01-02 03:36:37.000 496 99397 49946.5 4994650 496 99397 49946.5 4994650 -32674 32464 4726.66 472666 -124 127 0.9 90 -497 100 10487 99398 1.49249 298.49249 149.99249 14999.24924 1.49249 298.4925 149.99249 14999.24926 1.49249 298.49249 149.99249000000003 14999.24900 2020-01-01 2020-01-02 2020-01-01 00:08:17 2020-01-02 03:36:38 2020-01-01 00:08:17.000 2020-01-02 03:36:38.000 497 99398 49947.5 4994750 497 99398 49947.5 4994750 -32673 32465 4727.66 472766 -128 127 -0.66 -66 -498 100 10488 99399 1.49549 298.49549 149.99549 14999.54954 1.49549 298.49548 149.99549 14999.54956 1.49549 298.49549 149.99548999999988 14999.54900 2020-01-01 2020-01-02 2020-01-01 00:08:18 2020-01-02 03:36:39 2020-01-01 00:08:18.000 2020-01-02 03:36:39.000 498 99399 49948.5 4994850 498 99399 49948.5 4994850 -32672 32466 4728.66 472866 -128 123 -2.22 -222 -499 100 10489 99400 1.49849 298.49849 149.99849 14999.84984 1.49849 298.4985 149.9985 14999.85015 1.49849 298.49849 149.99848999999998 14999.84900 2020-01-01 2020-01-02 2020-01-01 00:08:19 2020-01-02 03:36:40 2020-01-01 00:08:19.000 2020-01-02 03:36:40.000 499 99400 49949.5 4994950 499 99400 49949.5 4994950 -32671 32467 4729.66 472966 -127 124 -1.22 -122 -5 102 1004 9995 0.01501 300.01501 150.01501 15151.51651 0.01501 300.015 150.01501 15151.51648 0.01501 300.01501 150.01500999999976 15151.51601 2020-01-01 2020-01-02 2020-01-01 00:00:05 2020-01-02 03:45:05 2020-01-01 00:00:05.000 2020-01-02 03:45:05.000 5 99905 49955 5045455 5 99905 49955 5045455 -32564 32371 4534.009900990099 457935 -128 123 -3.01980198019802 -305 +467 101 10457 99368 1.4024 298.4024 149.9024 14990.24024 1.4024 298.4024 149.9024 14990.24041 1.40240 298.40240 149.9024 14990.24000 2020-01-01 2020-01-02 2020-01-01 00:07:47 2020-01-02 03:36:08 2020-01-01 00:07:47.000 2020-01-02 03:36:08.000 467 99368 49917.5 4991750 467 99368 49917.5 4991750 -32703 32435 4697.66 469766 -128 123 -2.5 -250 +468 101 10458 99369 1.4054 298.4054 149.9054 14990.54054 1.4054 298.4054 149.9054 14990.54058 1.40540 298.40540 149.90540000000001 14990.54000 2020-01-01 2020-01-02 2020-01-01 00:07:48 2020-01-02 03:36:09 2020-01-01 00:07:48.000 2020-01-02 03:36:09.000 468 99369 49918.5 4991850 468 99369 49918.5 4991850 -32702 32436 4698.66 469866 -127 124 -1.5 -150 +469 101 10459 99370 1.4084 298.4084 149.9084 14990.84084 1.4084 298.40842 149.9084 14990.8407 1.40840 298.40840 149.9084 14990.84000 2020-01-01 2020-01-02 2020-01-01 00:07:49 2020-01-02 03:36:10 2020-01-01 00:07:49.000 2020-01-02 03:36:10.000 469 99370 49919.5 4991950 469 99370 49919.5 4991950 -32701 32437 4699.66 469966 -126 125 -0.5 -50 +47 102 10037 99947 0.14114 300.14114 150.14114 15164.25525 0.14114 300.14114 150.14114 15164.25545 0.14114 300.14114 150.14114 15164.25514 2020-01-01 2020-01-02 2020-01-01 00:00:47 2020-01-02 03:45:47 2020-01-01 00:00:47.000 2020-01-02 03:45:47.000 47 99947 49997 5049697 47 99947 49997 5049697 -32522 32413 4576.009900990099 462177 -128 127 -1.5742574257425743 -159 +470 101 10460 99371 1.41141 298.41141 149.91141 14991.14114 1.41141 298.4114 149.9114 14991.14099 1.41141 298.41141 149.91141 14991.14100 2020-01-01 2020-01-02 2020-01-01 00:07:50 2020-01-02 03:36:11 2020-01-01 00:07:50.000 2020-01-02 03:36:11.000 470 99371 49920.5 4992050 470 99371 49920.5 4992050 -32700 32438 4700.66 470066 -125 126 0.5 50 +471 101 10461 99372 1.41441 298.41441 149.91441 14991.44144 1.41441 298.41443 149.91441 14991.44159 1.41441 298.41441 149.91441 14991.44100 2020-01-01 2020-01-02 2020-01-01 00:07:51 2020-01-02 03:36:12 2020-01-01 00:07:51.000 2020-01-02 03:36:12.000 471 99372 49921.5 4992150 471 99372 49921.5 4992150 -32699 32439 4701.66 470166 -124 127 1.5 150 +472 101 10462 99373 1.41741 298.41741 149.91741 14991.74174 1.41741 298.41742 149.91741 14991.74188 1.41741 298.41741 149.91741 14991.74100 2020-01-01 2020-01-02 2020-01-01 00:07:52 2020-01-02 03:36:13 2020-01-01 00:07:52.000 2020-01-02 03:36:13.000 472 99373 49922.5 4992250 472 99373 49922.5 4992250 -32698 32440 4702.66 470266 -128 127 -0.06 -6 +473 101 10463 99374 1.42042 298.42042 149.92042 14992.04204 1.42042 298.4204 149.92042 14992.04204 1.42042 298.42042 149.92042 14992.04200 2020-01-01 2020-01-02 2020-01-01 00:07:53 2020-01-02 03:36:14 2020-01-01 00:07:53.000 2020-01-02 03:36:14.000 473 99374 49923.5 4992350 473 99374 49923.5 4992350 -32697 32441 4703.66 470366 -128 123 -1.62 -162 +474 101 10464 99375 1.42342 298.42342 149.92342 14992.34234 1.42342 298.42343 149.92342 14992.34216 1.42342 298.42342 149.92342 14992.34200 2020-01-01 2020-01-02 2020-01-01 00:07:54 2020-01-02 03:36:15 2020-01-01 00:07:54.000 2020-01-02 03:36:15.000 474 99375 49924.5 4992450 474 99375 49924.5 4992450 -32696 32442 4704.66 470466 -127 124 -0.62 -62 +475 101 10465 99376 1.42642 298.42642 149.92642 14992.64264 1.42642 298.42642 149.92642 14992.64246 1.42642 298.42642 149.92642 14992.64200 2020-01-01 2020-01-02 2020-01-01 00:07:55 2020-01-02 03:36:16 2020-01-01 00:07:55.000 2020-01-02 03:36:16.000 475 99376 49925.5 4992550 475 99376 49925.5 4992550 -32695 32443 4705.66 470566 -126 125 0.38 38 +476 101 10466 99377 1.42942 298.42942 149.92942 14992.94294 1.42942 298.42944 149.92943 14992.94305 1.42942 298.42942 149.92942 14992.94200 2020-01-01 2020-01-02 2020-01-01 00:07:56 2020-01-02 03:36:17 2020-01-01 00:07:56.000 2020-01-02 03:36:17.000 476 99377 49926.5 4992650 476 99377 49926.5 4992650 -32694 32444 4706.66 470666 -125 126 1.38 138 +477 101 10467 99378 1.43243 298.43243 149.93243 14993.24324 1.43243 298.43243 149.93243 14993.24338 1.43243 298.43243 149.93243 14993.24300 2020-01-01 2020-01-02 2020-01-01 00:07:57 2020-01-02 03:36:18 2020-01-01 00:07:57.000 2020-01-02 03:36:18.000 477 99378 49927.5 4992750 477 99378 49927.5 4992750 -32693 32445 4707.66 470766 -124 127 2.38 238 +478 101 10468 99379 1.43543 298.43543 149.93543 14993.54354 1.43543 298.43542 149.93543 14993.54352 1.43543 298.43543 149.93543 14993.54300 2020-01-01 2020-01-02 2020-01-01 00:07:58 2020-01-02 03:36:19 2020-01-01 00:07:58.000 2020-01-02 03:36:19.000 478 99379 49928.5 4992850 478 99379 49928.5 4992850 -32692 32446 4708.66 470866 -128 127 0.82 82 +479 101 10469 99380 1.43843 298.43843 149.93843 14993.84384 1.43843 298.43845 149.93844 14993.84427 1.43843 298.43843 149.93843 14993.84300 2020-01-01 2020-01-02 2020-01-01 00:07:59 2020-01-02 03:36:20 2020-01-01 00:07:59.000 2020-01-02 03:36:20.000 479 99380 49929.5 4992950 479 99380 49929.5 4992950 -32691 32447 4709.66 470966 -128 127 -0.74 -74 +48 102 10038 99948 0.14414 300.14414 150.14414 15164.55855 0.14414 300.14413 150.14414 15164.55863 0.14414 300.14414 150.14414 15164.55814 2020-01-01 2020-01-02 2020-01-01 00:00:48 2020-01-02 03:45:48 2020-01-01 00:00:48.000 2020-01-02 03:45:48.000 48 99948 49998 5049798 48 99948 49998 5049798 -32521 32414 4577.009900990099 462278 -128 127 -3.108910891089109 -314 +480 101 10470 99381 1.44144 298.44144 149.94144 14994.14414 1.44144 298.44144 149.94143 14994.14392 1.44144 298.44144 149.94144 14994.14400 2020-01-01 2020-01-02 2020-01-01 00:08:00 2020-01-02 03:36:21 2020-01-01 00:08:00.000 2020-01-02 03:36:21.000 480 99381 49930.5 4993050 480 99381 49930.5 4993050 -32690 32448 4710.66 471066 -128 124 -2.3 -230 +481 101 10471 99382 1.44444 298.44444 149.94444 14994.44444 1.44444 298.44446 149.94444 14994.44452 1.44444 298.44444 149.94444 14994.44400 2020-01-01 2020-01-02 2020-01-01 00:08:01 2020-01-02 03:36:22 2020-01-01 00:08:01.000 2020-01-02 03:36:22.000 481 99382 49931.5 4993150 481 99382 49931.5 4993150 -32689 32449 4711.66 471166 -127 125 -1.3 -130 +482 101 10472 99383 1.44744 298.44744 149.94744 14994.74474 1.44744 298.44745 149.94744 14994.74485 1.44744 298.44744 149.94744 14994.74400 2020-01-01 2020-01-02 2020-01-01 00:08:02 2020-01-02 03:36:23 2020-01-01 00:08:02.000 2020-01-02 03:36:23.000 482 99383 49932.5 4993250 482 99383 49932.5 4993250 -32688 32450 4712.66 471266 -126 126 -0.3 -30 +483 101 10473 99384 1.45045 298.45045 149.95045 14995.04504 1.45045 298.45044 149.95044 14995.04499 1.45045 298.45045 149.95045 14995.04500 2020-01-01 2020-01-02 2020-01-01 00:08:03 2020-01-02 03:36:24 2020-01-01 00:08:03.000 2020-01-02 03:36:24.000 483 99384 49933.5 4993350 483 99384 49933.5 4993350 -32687 32451 4713.66 471366 -125 127 0.7 70 +484 101 10474 99385 1.45345 298.45345 149.95345 14995.34534 1.45345 298.45346 149.95345 14995.34574 1.45345 298.45345 149.95345 14995.34500 2020-01-01 2020-01-02 2020-01-01 00:08:04 2020-01-02 03:36:25 2020-01-01 00:08:04.000 2020-01-02 03:36:25.000 484 99385 49934.5 4993450 484 99385 49934.5 4993450 -32686 32452 4714.66 471466 -128 127 -0.86 -86 +485 101 10475 99386 1.45645 298.45645 149.95645 14995.64564 1.45645 298.45645 149.95645 14995.6454 1.45645 298.45645 149.95645000000002 14995.64500 2020-01-01 2020-01-02 2020-01-01 00:08:05 2020-01-02 03:36:26 2020-01-01 00:08:05.000 2020-01-02 03:36:26.000 485 99386 49935.5 4993550 485 99386 49935.5 4993550 -32685 32453 4715.66 471566 -128 127 -2.42 -242 +486 101 10476 99387 1.45945 298.45945 149.95945 14995.94594 1.45945 298.45947 149.95946 14995.94602 1.45945 298.45945 149.95945 14995.94500 2020-01-01 2020-01-02 2020-01-01 00:08:06 2020-01-02 03:36:27 2020-01-01 00:08:06.000 2020-01-02 03:36:27.000 486 99387 49936.5 4993650 486 99387 49936.5 4993650 -32684 32454 4716.66 471666 -128 123 -3.98 -398 +487 101 10477 99388 1.46246 298.46246 149.96246 14996.24624 1.46246 298.46246 149.96246 14996.24633 1.46246 298.46246 149.96246 14996.24600 2020-01-01 2020-01-02 2020-01-01 00:08:07 2020-01-02 03:36:28 2020-01-01 00:08:07.000 2020-01-02 03:36:28.000 487 99388 49937.5 4993750 487 99388 49937.5 4993750 -32683 32455 4717.66 471766 -127 124 -2.98 -298 +488 101 10478 99389 1.46546 298.46546 149.96546 14996.54654 1.46546 298.46545 149.96546 14996.54645 1.46546 298.46546 149.96546 14996.54600 2020-01-01 2020-01-02 2020-01-01 00:08:08 2020-01-02 03:36:29 2020-01-01 00:08:08.000 2020-01-02 03:36:29.000 488 99389 49938.5 4993850 488 99389 49938.5 4993850 -32682 32456 4718.66 471866 -126 125 -1.98 -198 +489 101 10479 99390 1.46846 298.46846 149.96846 14996.84684 1.46846 298.46848 149.96847 14996.84721 1.46846 298.46846 149.96846 14996.84600 2020-01-01 2020-01-02 2020-01-01 00:08:09 2020-01-02 03:36:30 2020-01-01 00:08:09.000 2020-01-02 03:36:30.000 489 99390 49939.5 4993950 489 99390 49939.5 4993950 -32681 32457 4719.66 471966 -125 126 -0.98 -98 +49 102 10039 99949 0.14714 300.14714 150.14714 15164.86186 0.14714 300.14716 150.14714 15164.86173 0.14714 300.14714 150.14714 15164.86114 2020-01-01 2020-01-02 2020-01-01 00:00:49 2020-01-02 03:45:49 2020-01-01 00:00:49.000 2020-01-02 03:45:49.000 49 99949 49999 5049899 49 99949 49999 5049899 -32520 32415 4578.009900990099 462379 -128 123 -4.643564356435643 -469 +490 101 10480 99391 1.47147 298.47147 149.97147 14997.14714 1.47147 298.47147 149.97146 14997.14687 1.47147 298.47147 149.97147 14997.14700 2020-01-01 2020-01-02 2020-01-01 00:08:10 2020-01-02 03:36:31 2020-01-01 00:08:10.000 2020-01-02 03:36:31.000 490 99391 49940.5 4994050 490 99391 49940.5 4994050 -32680 32458 4720.66 472066 -124 127 0.02 2 +491 101 10481 99392 1.47447 298.47447 149.97447 14997.44744 1.47447 298.4745 149.97447 14997.44749 1.47447 298.47447 149.97447 14997.44700 2020-01-01 2020-01-02 2020-01-01 00:08:11 2020-01-02 03:36:32 2020-01-01 00:08:11.000 2020-01-02 03:36:32.000 491 99392 49941.5 4994150 491 99392 49941.5 4994150 -32679 32459 4721.66 472166 -128 127 -1.54 -154 +492 101 10482 99393 1.47747 298.47747 149.97747 14997.74774 1.47747 298.47748 149.97747 14997.74779 1.47747 298.47747 149.97746999999998 14997.74700 2020-01-01 2020-01-02 2020-01-01 00:08:12 2020-01-02 03:36:33 2020-01-01 00:08:12.000 2020-01-02 03:36:33.000 492 99393 49942.5 4994250 492 99393 49942.5 4994250 -32678 32460 4722.66 472266 -128 123 -3.1 -310 +493 101 10483 99394 1.48048 298.48048 149.98048 14998.04804 1.48048 298.48047 149.98048 14998.04809 1.48048 298.48048 149.98048 14998.04800 2020-01-01 2020-01-02 2020-01-01 00:08:13 2020-01-02 03:36:34 2020-01-01 00:08:13.000 2020-01-02 03:36:34.000 493 99394 49943.5 4994350 493 99394 49943.5 4994350 -32677 32461 4723.66 472366 -127 124 -2.1 -210 +494 101 10484 99395 1.48348 298.48348 149.98348 14998.34834 1.48348 298.4835 149.98348 14998.34868 1.48348 298.48348 149.98348 14998.34800 2020-01-01 2020-01-02 2020-01-01 00:08:14 2020-01-02 03:36:35 2020-01-01 00:08:14.000 2020-01-02 03:36:35.000 494 99395 49944.5 4994450 494 99395 49944.5 4994450 -32676 32462 4724.66 472466 -126 125 -1.1 -110 +495 100 10485 99396 1.48648 298.48648 149.98648 14998.64864 1.48648 298.48648 149.98648 14998.64837 1.48648 298.48648 149.98648 14998.64800 2020-01-01 2020-01-02 2020-01-01 00:08:15 2020-01-02 03:36:36 2020-01-01 00:08:15.000 2020-01-02 03:36:36.000 495 99396 49945.5 4994550 495 99396 49945.5 4994550 -32675 32463 4725.66 472566 -125 126 -0.1 -10 +496 100 10486 99397 1.48948 298.48948 149.98948 14998.94894 1.48948 298.4895 149.98948 14998.94896 1.48948 298.48948 149.98948000000001 14998.94800 2020-01-01 2020-01-02 2020-01-01 00:08:16 2020-01-02 03:36:37 2020-01-01 00:08:16.000 2020-01-02 03:36:37.000 496 99397 49946.5 4994650 496 99397 49946.5 4994650 -32674 32464 4726.66 472666 -124 127 0.9 90 +497 100 10487 99398 1.49249 298.49249 149.99249 14999.24924 1.49249 298.4925 149.99249 14999.24926 1.49249 298.49249 149.99249 14999.24900 2020-01-01 2020-01-02 2020-01-01 00:08:17 2020-01-02 03:36:38 2020-01-01 00:08:17.000 2020-01-02 03:36:38.000 497 99398 49947.5 4994750 497 99398 49947.5 4994750 -32673 32465 4727.66 472766 -128 127 -0.66 -66 +498 100 10488 99399 1.49549 298.49549 149.99549 14999.54954 1.49549 298.49548 149.99549 14999.54956 1.49549 298.49549 149.99549000000002 14999.54900 2020-01-01 2020-01-02 2020-01-01 00:08:18 2020-01-02 03:36:39 2020-01-01 00:08:18.000 2020-01-02 03:36:39.000 498 99399 49948.5 4994850 498 99399 49948.5 4994850 -32672 32466 4728.66 472866 -128 123 -2.22 -222 +499 100 10489 99400 1.49849 298.49849 149.99849 14999.84984 1.49849 298.4985 149.9985 14999.85015 1.49849 298.49849 149.99849 14999.84900 2020-01-01 2020-01-02 2020-01-01 00:08:19 2020-01-02 03:36:40 2020-01-01 00:08:19.000 2020-01-02 03:36:40.000 499 99400 49949.5 4994950 499 99400 49949.5 4994950 -32671 32467 4729.66 472966 -127 124 -1.22 -122 +5 102 1004 9995 0.01501 300.01501 150.01501 15151.51651 0.01501 300.015 150.01501 15151.51648 0.01501 300.01501 150.01501 15151.51601 2020-01-01 2020-01-02 2020-01-01 00:00:05 2020-01-02 03:45:05 2020-01-01 00:00:05.000 2020-01-02 03:45:05.000 5 99905 49955 5045455 5 99905 49955 5045455 -32564 32371 4534.009900990099 457935 -128 123 -3.01980198019802 -305 50 102 10040 99950 0.15015 300.15015 150.15015 15165.16516 0.15015 300.15015 150.15014 15165.16487 0.15015 300.15015 150.15015 15165.16515 2020-01-01 2020-01-02 2020-01-01 00:00:50 2020-01-02 03:45:50 2020-01-01 00:00:50.000 2020-01-02 03:45:50.000 50 99950 50000 5050000 50 99950 50000 5050000 -32519 32416 4579.009900990099 462480 -127 124 -3.6435643564356437 -368 -500 100 10490 99401 1.5015 298.5015 150.0015 15000.15015 1.5015 298.5015 150.00149 15000.14984 1.50150 298.50150 150.0015000000002 15000.15000 2020-01-01 2020-01-02 2020-01-01 00:08:20 2020-01-02 03:36:41 2020-01-01 00:08:20.000 2020-01-02 03:36:41.000 500 99401 49950.5 4995050 500 99401 49950.5 4995050 -32670 32468 4730.66 473066 -126 125 -0.22 -22 -501 100 10491 99402 1.5045 298.5045 150.0045 15000.45045 1.5045 298.50452 150.0045 15000.45043 1.50450 298.50450 150.00449999999972 15000.45000 2020-01-01 2020-01-02 2020-01-01 00:08:21 2020-01-02 03:36:42 2020-01-01 00:08:21.000 2020-01-02 03:36:42.000 501 99402 49951.5 4995150 501 99402 49951.5 4995150 -32669 32469 4731.66 473166 -125 126 0.78 78 -502 100 10492 99403 1.5075 298.5075 150.0075 15000.75075 1.5075 298.5075 150.0075 15000.75073 1.50750 298.50750 150.00749999999982 15000.75000 2020-01-01 2020-01-02 2020-01-01 00:08:22 2020-01-02 03:36:43 2020-01-01 00:08:22.000 2020-01-02 03:36:43.000 502 99403 49952.5 4995250 502 99403 49952.5 4995250 -32668 32470 4732.66 473266 -124 127 1.78 178 -503 100 10493 99404 1.51051 298.51051 150.01051 15001.05105 1.51051 298.5105 150.01051 15001.05103 1.51051 298.51051 150.01051000000004 15001.05100 2020-01-01 2020-01-02 2020-01-01 00:08:23 2020-01-02 03:36:44 2020-01-01 00:08:23.000 2020-01-02 03:36:44.000 503 99404 49953.5 4995350 503 99404 49953.5 4995350 -32667 32471 4733.66 473366 -128 127 0.22 22 -504 100 10494 99405 1.51351 298.51351 150.01351 15001.35135 1.51351 298.51352 150.01351 15001.35162 1.51351 298.51351 150.01351000000014 15001.35100 2020-01-01 2020-01-02 2020-01-01 00:08:24 2020-01-02 03:36:45 2020-01-01 00:08:24.000 2020-01-02 03:36:45.000 504 99405 49954.5 4995450 504 99405 49954.5 4995450 -32666 32472 4734.66 473466 -128 127 -1.34 -134 -505 100 10495 99406 1.51651 298.51651 150.01651 15001.65165 1.51651 298.5165 150.01651 15001.65131 1.51651 298.51651 150.01650999999998 15001.65100 2020-01-01 2020-01-02 2020-01-01 00:08:25 2020-01-02 03:36:46 2020-01-01 00:08:25.000 2020-01-02 03:36:46.000 505 99406 49955.5 4995550 505 99406 49955.5 4995550 -32665 32473 4735.66 473566 -128 124 -2.9 -290 -506 100 10496 99407 1.51951 298.51951 150.01951 15001.95195 1.51951 298.51953 150.01951 15001.9519 1.51951 298.51951 150.01951000000005 15001.95100 2020-01-01 2020-01-02 2020-01-01 00:08:26 2020-01-02 03:36:47 2020-01-01 00:08:26.000 2020-01-02 03:36:47.000 506 99407 49956.5 4995650 506 99407 49956.5 4995650 -32664 32474 4736.66 473666 -127 125 -1.9 -190 -507 100 10497 99408 1.52252 298.52252 150.02252 15002.25225 1.52252 298.52252 150.02252 15002.2522 1.52252 298.52252 150.02252000000033 15002.25200 2020-01-01 2020-01-02 2020-01-01 00:08:27 2020-01-02 03:36:48 2020-01-01 00:08:27.000 2020-01-02 03:36:48.000 507 99408 49957.5 4995750 507 99408 49957.5 4995750 -32663 32475 4737.66 473766 -126 126 -0.9 -90 -508 100 10498 99409 1.52552 298.52552 150.02552 15002.55255 1.52552 298.5255 150.02552 15002.5525 1.52552 298.52552 150.02551999999983 15002.55200 2020-01-01 2020-01-02 2020-01-01 00:08:28 2020-01-02 03:36:49 2020-01-01 00:08:28.000 2020-01-02 03:36:49.000 508 99409 49958.5 4995850 508 99409 49958.5 4995850 -32662 32476 4738.66 473866 -125 127 0.1 10 -509 100 10499 99410 1.52852 298.52852 150.02852 15002.85285 1.52852 298.52853 150.02853 15002.85312 1.52852 298.52852 150.02851999999987 15002.85200 2020-01-01 2020-01-02 2020-01-01 00:08:29 2020-01-02 03:36:50 2020-01-01 00:08:29.000 2020-01-02 03:36:50.000 509 99410 49959.5 4995950 509 99410 49959.5 4995950 -32661 32477 4739.66 473966 -128 127 -1.46 -146 -51 102 10041 99951 0.15315 300.15315 150.15315 15165.46846 0.15315 300.15317 150.15315 15165.46863 0.15315 300.15315 150.15315000000007 15165.46815 2020-01-01 2020-01-02 2020-01-01 00:00:51 2020-01-02 03:45:51 2020-01-01 00:00:51.000 2020-01-02 03:45:51.000 51 99951 50001 5050101 51 99951 50001 5050101 -32518 32417 4580.009900990099 462581 -126 125 -2.6435643564356437 -267 -510 100 10500 99411 1.53153 298.53153 150.03153 15003.15315 1.53153 298.53152 150.03152 15003.15278 1.53153 298.53153 150.03153000000017 15003.15300 2020-01-01 2020-01-02 2020-01-01 00:08:30 2020-01-02 03:36:51 2020-01-01 00:08:30.000 2020-01-02 03:36:51.000 510 99411 49960.5 4996050 510 99411 49960.5 4996050 -32660 32478 4740.66 474066 -128 127 -3.02 -302 -511 100 10501 99412 1.53453 298.53453 150.03453 15003.45345 1.53453 298.53455 150.03453 15003.45354 1.53453 298.53453 150.03453000000022 15003.45300 2020-01-01 2020-01-02 2020-01-01 00:08:31 2020-01-02 03:36:52 2020-01-01 00:08:31.000 2020-01-02 03:36:52.000 511 99412 49961.5 4996150 511 99412 49961.5 4996150 -32659 32479 4741.66 474166 -128 123 -4.58 -458 -512 100 10502 99413 1.53753 298.53753 150.03753 15003.75375 1.53753 298.53754 150.03753 15003.75366 1.53753 298.53753 150.03752999999972 15003.75300 2020-01-01 2020-01-02 2020-01-01 00:08:32 2020-01-02 03:36:53 2020-01-01 00:08:32.000 2020-01-02 03:36:53.000 512 99413 49962.5 4996250 512 99413 49962.5 4996250 -32658 32480 4742.66 474266 -127 124 -3.58 -358 +500 100 10490 99401 1.5015 298.5015 150.0015 15000.15015 1.5015 298.5015 150.00149 15000.14984 1.50150 298.50150 150.0015 15000.15000 2020-01-01 2020-01-02 2020-01-01 00:08:20 2020-01-02 03:36:41 2020-01-01 00:08:20.000 2020-01-02 03:36:41.000 500 99401 49950.5 4995050 500 99401 49950.5 4995050 -32670 32468 4730.66 473066 -126 125 -0.22 -22 +501 100 10491 99402 1.5045 298.5045 150.0045 15000.45045 1.5045 298.50452 150.0045 15000.45043 1.50450 298.50450 150.0045 15000.45000 2020-01-01 2020-01-02 2020-01-01 00:08:21 2020-01-02 03:36:42 2020-01-01 00:08:21.000 2020-01-02 03:36:42.000 501 99402 49951.5 4995150 501 99402 49951.5 4995150 -32669 32469 4731.66 473166 -125 126 0.78 78 +502 100 10492 99403 1.5075 298.5075 150.0075 15000.75075 1.5075 298.5075 150.0075 15000.75073 1.50750 298.50750 150.0075 15000.75000 2020-01-01 2020-01-02 2020-01-01 00:08:22 2020-01-02 03:36:43 2020-01-01 00:08:22.000 2020-01-02 03:36:43.000 502 99403 49952.5 4995250 502 99403 49952.5 4995250 -32668 32470 4732.66 473266 -124 127 1.78 178 +503 100 10493 99404 1.51051 298.51051 150.01051 15001.05105 1.51051 298.5105 150.01051 15001.05103 1.51051 298.51051 150.01050999999998 15001.05100 2020-01-01 2020-01-02 2020-01-01 00:08:23 2020-01-02 03:36:44 2020-01-01 00:08:23.000 2020-01-02 03:36:44.000 503 99404 49953.5 4995350 503 99404 49953.5 4995350 -32667 32471 4733.66 473366 -128 127 0.22 22 +504 100 10494 99405 1.51351 298.51351 150.01351 15001.35135 1.51351 298.51352 150.01351 15001.35162 1.51351 298.51351 150.01351 15001.35100 2020-01-01 2020-01-02 2020-01-01 00:08:24 2020-01-02 03:36:45 2020-01-01 00:08:24.000 2020-01-02 03:36:45.000 504 99405 49954.5 4995450 504 99405 49954.5 4995450 -32666 32472 4734.66 473466 -128 127 -1.34 -134 +505 100 10495 99406 1.51651 298.51651 150.01651 15001.65165 1.51651 298.5165 150.01651 15001.65131 1.51651 298.51651 150.01651 15001.65100 2020-01-01 2020-01-02 2020-01-01 00:08:25 2020-01-02 03:36:46 2020-01-01 00:08:25.000 2020-01-02 03:36:46.000 505 99406 49955.5 4995550 505 99406 49955.5 4995550 -32665 32473 4735.66 473566 -128 124 -2.9 -290 +506 100 10496 99407 1.51951 298.51951 150.01951 15001.95195 1.51951 298.51953 150.01951 15001.9519 1.51951 298.51951 150.01951 15001.95100 2020-01-01 2020-01-02 2020-01-01 00:08:26 2020-01-02 03:36:47 2020-01-01 00:08:26.000 2020-01-02 03:36:47.000 506 99407 49956.5 4995650 506 99407 49956.5 4995650 -32664 32474 4736.66 473666 -127 125 -1.9 -190 +507 100 10497 99408 1.52252 298.52252 150.02252 15002.25225 1.52252 298.52252 150.02252 15002.2522 1.52252 298.52252 150.02252000000001 15002.25200 2020-01-01 2020-01-02 2020-01-01 00:08:27 2020-01-02 03:36:48 2020-01-01 00:08:27.000 2020-01-02 03:36:48.000 507 99408 49957.5 4995750 507 99408 49957.5 4995750 -32663 32475 4737.66 473766 -126 126 -0.9 -90 +508 100 10498 99409 1.52552 298.52552 150.02552 15002.55255 1.52552 298.5255 150.02552 15002.5525 1.52552 298.52552 150.02552 15002.55200 2020-01-01 2020-01-02 2020-01-01 00:08:28 2020-01-02 03:36:49 2020-01-01 00:08:28.000 2020-01-02 03:36:49.000 508 99409 49958.5 4995850 508 99409 49958.5 4995850 -32662 32476 4738.66 473866 -125 127 0.1 10 +509 100 10499 99410 1.52852 298.52852 150.02852 15002.85285 1.52852 298.52853 150.02853 15002.85312 1.52852 298.52852 150.02852000000001 15002.85200 2020-01-01 2020-01-02 2020-01-01 00:08:29 2020-01-02 03:36:50 2020-01-01 00:08:29.000 2020-01-02 03:36:50.000 509 99410 49959.5 4995950 509 99410 49959.5 4995950 -32661 32477 4739.66 473966 -128 127 -1.46 -146 +51 102 10041 99951 0.15315 300.15315 150.15315 15165.46846 0.15315 300.15317 150.15315 15165.46863 0.15315 300.15315 150.15315 15165.46815 2020-01-01 2020-01-02 2020-01-01 00:00:51 2020-01-02 03:45:51 2020-01-01 00:00:51.000 2020-01-02 03:45:51.000 51 99951 50001 5050101 51 99951 50001 5050101 -32518 32417 4580.009900990099 462581 -126 125 -2.6435643564356437 -267 +510 100 10500 99411 1.53153 298.53153 150.03153 15003.15315 1.53153 298.53152 150.03152 15003.15278 1.53153 298.53153 150.03153 15003.15300 2020-01-01 2020-01-02 2020-01-01 00:08:30 2020-01-02 03:36:51 2020-01-01 00:08:30.000 2020-01-02 03:36:51.000 510 99411 49960.5 4996050 510 99411 49960.5 4996050 -32660 32478 4740.66 474066 -128 127 -3.02 -302 +511 100 10501 99412 1.53453 298.53453 150.03453 15003.45345 1.53453 298.53455 150.03453 15003.45354 1.53453 298.53453 150.03453 15003.45300 2020-01-01 2020-01-02 2020-01-01 00:08:31 2020-01-02 03:36:52 2020-01-01 00:08:31.000 2020-01-02 03:36:52.000 511 99412 49961.5 4996150 511 99412 49961.5 4996150 -32659 32479 4741.66 474166 -128 123 -4.58 -458 +512 100 10502 99413 1.53753 298.53753 150.03753 15003.75375 1.53753 298.53754 150.03753 15003.75366 1.53753 298.53753 150.03753 15003.75300 2020-01-01 2020-01-02 2020-01-01 00:08:32 2020-01-02 03:36:53 2020-01-01 00:08:32.000 2020-01-02 03:36:53.000 512 99413 49962.5 4996250 512 99413 49962.5 4996250 -32658 32480 4742.66 474266 -127 124 -3.58 -358 513 100 10503 99414 1.54054 298.54054 150.04054 15004.05405 1.54054 298.54053 150.04053 15004.05397 1.54054 298.54054 150.04054 15004.05400 2020-01-01 2020-01-02 2020-01-01 00:08:33 2020-01-02 03:36:54 2020-01-01 00:08:33.000 2020-01-02 03:36:54.000 513 99414 49963.5 4996350 513 99414 49963.5 4996350 -32657 32481 4743.66 474366 -126 125 -2.58 -258 -514 100 10504 99415 1.54354 298.54354 150.04354 15004.35435 1.54354 298.54355 150.04354 15004.35459 1.54354 298.54354 150.04354000000006 15004.35400 2020-01-01 2020-01-02 2020-01-01 00:08:34 2020-01-02 03:36:55 2020-01-01 00:08:34.000 2020-01-02 03:36:55.000 514 99415 49964.5 4996450 514 99415 49964.5 4996450 -32656 32482 4744.66 474466 -125 126 -1.58 -158 -515 100 10505 99416 1.54654 298.54654 150.04654 15004.65465 1.54654 298.54654 150.04654 15004.65425 1.54654 298.54654 150.04653999999988 15004.65400 2020-01-01 2020-01-02 2020-01-01 00:08:35 2020-01-02 03:36:56 2020-01-01 00:08:35.000 2020-01-02 03:36:56.000 515 99416 49965.5 4996550 515 99416 49965.5 4996550 -32655 32483 4745.66 474566 -124 127 -0.58 -58 -516 100 10506 99417 1.54954 298.54954 150.04954 15004.95495 1.54954 298.54956 150.04955 15004.955 1.54954 298.54954 150.04953999999998 15004.95400 2020-01-01 2020-01-02 2020-01-01 00:08:36 2020-01-02 03:36:57 2020-01-01 00:08:36.000 2020-01-02 03:36:57.000 516 99417 49966.5 4996650 516 99417 49966.5 4996650 -32654 32484 4746.66 474666 -128 127 -2.14 -214 -517 100 10507 99418 1.55255 298.55255 150.05255 15005.25525 1.55255 298.55255 150.05255 15005.25514 1.55255 298.55255 150.05255000000022 15005.25500 2020-01-01 2020-01-02 2020-01-01 00:08:37 2020-01-02 03:36:58 2020-01-01 00:08:37.000 2020-01-02 03:36:58.000 517 99418 49967.5 4996750 517 99418 49967.5 4996750 -32653 32485 4747.66 474766 -128 123 -3.7 -370 -518 100 10508 99419 1.55555 298.55555 150.05555 15005.55555 1.55555 298.55554 150.05555 15005.55547 1.55555 298.55555 150.05554999999973 15005.55500 2020-01-01 2020-01-02 2020-01-01 00:08:38 2020-01-02 03:36:59 2020-01-01 00:08:38.000 2020-01-02 03:36:59.000 518 99419 49968.5 4996850 518 99419 49968.5 4996850 -32652 32486 4748.66 474866 -127 124 -2.7 -270 -519 100 10509 99420 1.55855 298.55855 150.05855 15005.85585 1.55855 298.55856 150.05856 15005.85607 1.55855 298.55855 150.05854999999985 15005.85500 2020-01-01 2020-01-02 2020-01-01 00:08:39 2020-01-02 03:37:00 2020-01-01 00:08:39.000 2020-01-02 03:37:00.000 519 99420 49969.5 4996950 519 99420 49969.5 4996950 -32651 32487 4749.66 474966 -126 125 -1.7 -170 -52 102 10042 99952 0.15615 300.15615 150.15615 15165.77177 0.15615 300.15616 150.15615 15165.77193 0.15615 300.15615 150.1561500000001 15165.77115 2020-01-01 2020-01-02 2020-01-01 00:00:52 2020-01-02 03:45:52 2020-01-01 00:00:52.000 2020-01-02 03:45:52.000 52 99952 50002 5050202 52 99952 50002 5050202 -32517 32418 4581.009900990099 462682 -125 126 -1.6435643564356435 -166 -520 100 10510 99421 1.56156 298.56156 150.06156 15006.15615 1.56156 298.56155 150.06155 15006.15572 1.56156 298.56156 150.06156000000007 15006.15600 2020-01-01 2020-01-02 2020-01-01 00:08:40 2020-01-02 03:37:01 2020-01-01 00:08:40.000 2020-01-02 03:37:01.000 520 99421 49970.5 4997050 520 99421 49970.5 4997050 -32650 32488 4750.66 475066 -125 126 -0.7 -70 -521 100 10511 99422 1.56456 298.56456 150.06456 15006.45645 1.56456 298.56458 150.06456 15006.45647 1.56456 298.56456 150.0645600000001 15006.45600 2020-01-01 2020-01-02 2020-01-01 00:08:41 2020-01-02 03:37:02 2020-01-01 00:08:41.000 2020-01-02 03:37:02.000 521 99422 49971.5 4997150 521 99422 49971.5 4997150 -32649 32489 4751.66 475166 -124 127 0.3 30 -522 100 10512 99423 1.56756 298.56756 150.06756 15006.75675 1.56756 298.56757 150.06756 15006.75661 1.56756 298.56756 150.06756000000004 15006.75600 2020-01-01 2020-01-02 2020-01-01 00:08:42 2020-01-02 03:37:03 2020-01-01 00:08:42.000 2020-01-02 03:37:03.000 522 99423 49972.5 4997250 522 99423 49972.5 4997250 -32648 32490 4752.66 475266 -128 127 -1.26 -126 -523 100 10513 99424 1.57057 298.57057 150.07057 15007.05705 1.57057 298.57056 150.07056 15007.05694 1.57057 298.57057 150.07056999999992 15007.05700 2020-01-01 2020-01-02 2020-01-01 00:08:43 2020-01-02 03:37:04 2020-01-01 00:08:43.000 2020-01-02 03:37:04.000 523 99424 49973.5 4997350 523 99424 49973.5 4997350 -32647 32491 4753.66 475366 -128 123 -2.82 -282 -524 100 10514 99425 1.57357 298.57357 150.07357 15007.35735 1.57357 298.57358 150.07357 15007.35753 1.57357 298.57357 150.07356999999996 15007.35700 2020-01-01 2020-01-02 2020-01-01 00:08:44 2020-01-02 03:37:05 2020-01-01 00:08:44.000 2020-01-02 03:37:05.000 524 99425 49974.5 4997450 524 99425 49974.5 4997450 -32646 32492 4754.66 475466 -127 124 -1.82 -182 -525 100 10515 99426 1.57657 298.57657 150.07657 15007.65765 1.57657 298.57657 150.07657 15007.65783 1.57657 298.57657 150.07656999999998 15007.65700 2020-01-01 2020-01-02 2020-01-01 00:08:45 2020-01-02 03:37:06 2020-01-01 00:08:45.000 2020-01-02 03:37:06.000 525 99426 49975.5 4997550 525 99426 49975.5 4997550 -32645 32493 4755.66 475566 -126 125 -0.82 -82 -526 100 10516 99427 1.57957 298.57957 150.07957 15007.95795 1.57957 298.5796 150.07957 15007.95795 1.57957 298.57957 150.07956999999993 15007.95700 2020-01-01 2020-01-02 2020-01-01 00:08:46 2020-01-02 03:37:07 2020-01-01 00:08:46.000 2020-01-02 03:37:07.000 526 99427 49976.5 4997650 526 99427 49976.5 4997650 -32644 32494 4756.66 475666 -125 126 0.18 18 -527 100 10517 99428 1.58258 298.58258 150.08258 15008.25825 1.58258 298.58258 150.08258 15008.25811 1.58258 298.58258 150.0825800000001 15008.25800 2020-01-01 2020-01-02 2020-01-01 00:08:47 2020-01-02 03:37:08 2020-01-01 00:08:47.000 2020-01-02 03:37:08.000 527 99428 49977.5 4997750 527 99428 49977.5 4997750 -32643 32495 4757.66 475766 -124 127 1.18 118 -528 100 10518 99429 1.58558 298.58558 150.08558 15008.55855 1.58558 298.58557 150.08558 15008.5584 1.58558 298.58558 150.08558000000005 15008.55800 2020-01-01 2020-01-02 2020-01-01 00:08:48 2020-01-02 03:37:09 2020-01-01 00:08:48.000 2020-01-02 03:37:09.000 528 99429 49978.5 4997850 528 99429 49978.5 4997850 -32642 32496 4758.66 475866 -128 127 -0.38 -38 -529 100 10519 99430 1.58858 298.58858 150.08858 15008.85885 1.58858 298.5886 150.08859 15008.859 1.58858 298.58858 150.08857999999992 15008.85800 2020-01-01 2020-01-02 2020-01-01 00:08:49 2020-01-02 03:37:10 2020-01-01 00:08:49.000 2020-01-02 03:37:10.000 529 99430 49979.5 4997950 529 99430 49979.5 4997950 -32641 32497 4759.66 475966 -128 127 -1.94 -194 +514 100 10504 99415 1.54354 298.54354 150.04354 15004.35435 1.54354 298.54355 150.04354 15004.35459 1.54354 298.54354 150.04354 15004.35400 2020-01-01 2020-01-02 2020-01-01 00:08:34 2020-01-02 03:36:55 2020-01-01 00:08:34.000 2020-01-02 03:36:55.000 514 99415 49964.5 4996450 514 99415 49964.5 4996450 -32656 32482 4744.66 474466 -125 126 -1.58 -158 +515 100 10505 99416 1.54654 298.54654 150.04654 15004.65465 1.54654 298.54654 150.04654 15004.65425 1.54654 298.54654 150.04654 15004.65400 2020-01-01 2020-01-02 2020-01-01 00:08:35 2020-01-02 03:36:56 2020-01-01 00:08:35.000 2020-01-02 03:36:56.000 515 99416 49965.5 4996550 515 99416 49965.5 4996550 -32655 32483 4745.66 474566 -124 127 -0.58 -58 +516 100 10506 99417 1.54954 298.54954 150.04954 15004.95495 1.54954 298.54956 150.04955 15004.955 1.54954 298.54954 150.04954 15004.95400 2020-01-01 2020-01-02 2020-01-01 00:08:36 2020-01-02 03:36:57 2020-01-01 00:08:36.000 2020-01-02 03:36:57.000 516 99417 49966.5 4996650 516 99417 49966.5 4996650 -32654 32484 4746.66 474666 -128 127 -2.14 -214 +517 100 10507 99418 1.55255 298.55255 150.05255 15005.25525 1.55255 298.55255 150.05255 15005.25514 1.55255 298.55255 150.05255 15005.25500 2020-01-01 2020-01-02 2020-01-01 00:08:37 2020-01-02 03:36:58 2020-01-01 00:08:37.000 2020-01-02 03:36:58.000 517 99418 49967.5 4996750 517 99418 49967.5 4996750 -32653 32485 4747.66 474766 -128 123 -3.7 -370 +518 100 10508 99419 1.55555 298.55555 150.05555 15005.55555 1.55555 298.55554 150.05555 15005.55547 1.55555 298.55555 150.05555 15005.55500 2020-01-01 2020-01-02 2020-01-01 00:08:38 2020-01-02 03:36:59 2020-01-01 00:08:38.000 2020-01-02 03:36:59.000 518 99419 49968.5 4996850 518 99419 49968.5 4996850 -32652 32486 4748.66 474866 -127 124 -2.7 -270 +519 100 10509 99420 1.55855 298.55855 150.05855 15005.85585 1.55855 298.55856 150.05856 15005.85607 1.55855 298.55855 150.05855 15005.85500 2020-01-01 2020-01-02 2020-01-01 00:08:39 2020-01-02 03:37:00 2020-01-01 00:08:39.000 2020-01-02 03:37:00.000 519 99420 49969.5 4996950 519 99420 49969.5 4996950 -32651 32487 4749.66 474966 -126 125 -1.7 -170 +52 102 10042 99952 0.15615 300.15615 150.15615 15165.77177 0.15615 300.15616 150.15615 15165.77193 0.15615 300.15615 150.15615 15165.77115 2020-01-01 2020-01-02 2020-01-01 00:00:52 2020-01-02 03:45:52 2020-01-01 00:00:52.000 2020-01-02 03:45:52.000 52 99952 50002 5050202 52 99952 50002 5050202 -32517 32418 4581.009900990099 462682 -125 126 -1.6435643564356435 -166 +520 100 10510 99421 1.56156 298.56156 150.06156 15006.15615 1.56156 298.56155 150.06155 15006.15572 1.56156 298.56156 150.06156000000001 15006.15600 2020-01-01 2020-01-02 2020-01-01 00:08:40 2020-01-02 03:37:01 2020-01-01 00:08:40.000 2020-01-02 03:37:01.000 520 99421 49970.5 4997050 520 99421 49970.5 4997050 -32650 32488 4750.66 475066 -125 126 -0.7 -70 +521 100 10511 99422 1.56456 298.56456 150.06456 15006.45645 1.56456 298.56458 150.06456 15006.45647 1.56456 298.56456 150.06456 15006.45600 2020-01-01 2020-01-02 2020-01-01 00:08:41 2020-01-02 03:37:02 2020-01-01 00:08:41.000 2020-01-02 03:37:02.000 521 99422 49971.5 4997150 521 99422 49971.5 4997150 -32649 32489 4751.66 475166 -124 127 0.3 30 +522 100 10512 99423 1.56756 298.56756 150.06756 15006.75675 1.56756 298.56757 150.06756 15006.75661 1.56756 298.56756 150.06756 15006.75600 2020-01-01 2020-01-02 2020-01-01 00:08:42 2020-01-02 03:37:03 2020-01-01 00:08:42.000 2020-01-02 03:37:03.000 522 99423 49972.5 4997250 522 99423 49972.5 4997250 -32648 32490 4752.66 475266 -128 127 -1.26 -126 +523 100 10513 99424 1.57057 298.57057 150.07057 15007.05705 1.57057 298.57056 150.07056 15007.05694 1.57057 298.57057 150.07057 15007.05700 2020-01-01 2020-01-02 2020-01-01 00:08:43 2020-01-02 03:37:04 2020-01-01 00:08:43.000 2020-01-02 03:37:04.000 523 99424 49973.5 4997350 523 99424 49973.5 4997350 -32647 32491 4753.66 475366 -128 123 -2.82 -282 +524 100 10514 99425 1.57357 298.57357 150.07357 15007.35735 1.57357 298.57358 150.07357 15007.35753 1.57357 298.57357 150.07357 15007.35700 2020-01-01 2020-01-02 2020-01-01 00:08:44 2020-01-02 03:37:05 2020-01-01 00:08:44.000 2020-01-02 03:37:05.000 524 99425 49974.5 4997450 524 99425 49974.5 4997450 -32646 32492 4754.66 475466 -127 124 -1.82 -182 +525 100 10515 99426 1.57657 298.57657 150.07657 15007.65765 1.57657 298.57657 150.07657 15007.65783 1.57657 298.57657 150.07657 15007.65700 2020-01-01 2020-01-02 2020-01-01 00:08:45 2020-01-02 03:37:06 2020-01-01 00:08:45.000 2020-01-02 03:37:06.000 525 99426 49975.5 4997550 525 99426 49975.5 4997550 -32645 32493 4755.66 475566 -126 125 -0.82 -82 +526 100 10516 99427 1.57957 298.57957 150.07957 15007.95795 1.57957 298.5796 150.07957 15007.95795 1.57957 298.57957 150.07957 15007.95700 2020-01-01 2020-01-02 2020-01-01 00:08:46 2020-01-02 03:37:07 2020-01-01 00:08:46.000 2020-01-02 03:37:07.000 526 99427 49976.5 4997650 526 99427 49976.5 4997650 -32644 32494 4756.66 475666 -125 126 0.18 18 +527 100 10517 99428 1.58258 298.58258 150.08258 15008.25825 1.58258 298.58258 150.08258 15008.25811 1.58258 298.58258 150.08258 15008.25800 2020-01-01 2020-01-02 2020-01-01 00:08:47 2020-01-02 03:37:08 2020-01-01 00:08:47.000 2020-01-02 03:37:08.000 527 99428 49977.5 4997750 527 99428 49977.5 4997750 -32643 32495 4757.66 475766 -124 127 1.18 118 +528 100 10518 99429 1.58558 298.58558 150.08558 15008.55855 1.58558 298.58557 150.08558 15008.5584 1.58558 298.58558 150.08558000000002 15008.55800 2020-01-01 2020-01-02 2020-01-01 00:08:48 2020-01-02 03:37:09 2020-01-01 00:08:48.000 2020-01-02 03:37:09.000 528 99429 49978.5 4997850 528 99429 49978.5 4997850 -32642 32496 4758.66 475866 -128 127 -0.38 -38 +529 100 10519 99430 1.58858 298.58858 150.08858 15008.85885 1.58858 298.5886 150.08859 15008.859 1.58858 298.58858 150.08858 15008.85800 2020-01-01 2020-01-02 2020-01-01 00:08:49 2020-01-02 03:37:10 2020-01-01 00:08:49.000 2020-01-02 03:37:10.000 529 99430 49979.5 4997950 529 99430 49979.5 4997950 -32641 32497 4759.66 475966 -128 127 -1.94 -194 53 102 10043 99953 0.15915 300.15915 150.15915 15166.07507 0.15915 300.15915 150.15915 15166.07511 0.15915 300.15915 150.15915 15166.07415 2020-01-01 2020-01-02 2020-01-01 00:00:53 2020-01-02 03:45:53 2020-01-01 00:00:53.000 2020-01-02 03:45:53.000 53 99953 50003 5050303 53 99953 50003 5050303 -32516 32419 4582.009900990099 462783 -124 127 -0.6435643564356436 -65 530 100 10520 99431 1.59159 298.59159 150.09159 15009.15915 1.59159 298.59158 150.09159 15009.15929 1.59159 298.59159 150.09159 15009.15900 2020-01-01 2020-01-02 2020-01-01 00:08:50 2020-01-02 03:37:11 2020-01-01 00:08:50.000 2020-01-02 03:37:11.000 530 99431 49980.5 4998050 530 99431 49980.5 4998050 -32640 32498 4760.66 476066 -128 124 -3.5 -350 -531 100 10521 99432 1.59459 298.59459 150.09459 15009.45945 1.59459 298.5946 150.09459 15009.45941 1.59459 298.59459 150.09459000000004 15009.45900 2020-01-01 2020-01-02 2020-01-01 00:08:51 2020-01-02 03:37:12 2020-01-01 00:08:51.000 2020-01-02 03:37:12.000 531 99432 49981.5 4998150 531 99432 49981.5 4998150 -32639 32499 4761.66 476166 -127 125 -2.5 -250 -532 100 10522 99433 1.59759 298.59759 150.09759 15009.75975 1.59759 298.5976 150.09759 15009.75958 1.59759 298.59759 150.09758999999997 15009.75900 2020-01-01 2020-01-02 2020-01-01 00:08:52 2020-01-02 03:37:13 2020-01-01 00:08:52.000 2020-01-02 03:37:13.000 532 99433 49982.5 4998250 532 99433 49982.5 4998250 -32638 32500 4762.66 476266 -126 126 -1.5 -150 -533 100 10523 99434 1.6006 298.6006 150.1006 15010.06006 1.6006 298.6006 150.10059 15010.05988 1.60060 298.60060 150.1005999999999 15010.06000 2020-01-01 2020-01-02 2020-01-01 00:08:53 2020-01-02 03:37:14 2020-01-01 00:08:53.000 2020-01-02 03:37:14.000 533 99434 49983.5 4998350 533 99434 49983.5 4998350 -32637 32501 4763.66 476366 -125 127 -0.5 -50 -534 100 10524 99435 1.6036 298.6036 150.1036 15010.36036 1.6036 298.6036 150.1036 15010.36063 1.60360 298.60360 150.10360000000009 15010.36000 2020-01-01 2020-01-02 2020-01-01 00:08:54 2020-01-02 03:37:15 2020-01-01 00:08:54.000 2020-01-02 03:37:15.000 534 99435 49984.5 4998450 534 99435 49984.5 4998450 -32636 32502 4764.66 476466 -128 127 -2.06 -206 -535 100 10525 99436 1.6066 298.6066 150.1066 15010.66066 1.6066 298.6066 150.1066 15010.66077 1.60660 298.60660 150.10659999999993 15010.66000 2020-01-01 2020-01-02 2020-01-01 00:08:55 2020-01-02 03:37:16 2020-01-01 00:08:55.000 2020-01-02 03:37:16.000 535 99436 49985.5 4998550 535 99436 49985.5 4998550 -32635 32503 4765.66 476566 -128 127 -3.62 -362 -536 100 10526 99437 1.6096 298.6096 150.1096 15010.96096 1.6096 298.60962 150.1096 15010.96092 1.60960 298.60960 150.1095999999999 15010.96000 2020-01-01 2020-01-02 2020-01-01 00:08:56 2020-01-02 03:37:17 2020-01-01 00:08:56.000 2020-01-02 03:37:17.000 536 99437 49986.5 4998650 536 99437 49986.5 4998650 -32634 32504 4766.66 476666 -128 123 -5.18 -518 -537 100 10527 99438 1.61261 298.61261 150.11261 15011.26126 1.61261 298.6126 150.11261 15011.26105 1.61261 298.61261 150.11261000000007 15011.26100 2020-01-01 2020-01-02 2020-01-01 00:08:57 2020-01-02 03:37:18 2020-01-01 00:08:57.000 2020-01-02 03:37:18.000 537 99438 49987.5 4998750 537 99438 49987.5 4998750 -32633 32505 4767.66 476766 -127 124 -4.18 -418 -538 100 10528 99439 1.61561 298.61561 150.11561 15011.56156 1.61561 298.6156 150.11561 15011.56135 1.61561 298.61561 150.11561000000003 15011.56100 2020-01-01 2020-01-02 2020-01-01 00:08:58 2020-01-02 03:37:19 2020-01-01 00:08:58.000 2020-01-02 03:37:19.000 538 99439 49988.5 4998850 538 99439 49988.5 4998850 -32632 32506 4768.66 476866 -126 125 -3.18 -318 -539 100 10529 99440 1.61861 298.61861 150.11861 15011.86186 1.61861 298.61862 150.11862 15011.8621 1.61861 298.61861 150.11860999999988 15011.86100 2020-01-01 2020-01-02 2020-01-01 00:08:59 2020-01-02 03:37:20 2020-01-01 00:08:59.000 2020-01-02 03:37:20.000 539 99440 49989.5 4998950 539 99440 49989.5 4998950 -32631 32507 4769.66 476966 -125 126 -2.18 -218 -54 102 10044 99954 0.16216 300.16216 150.16216 15166.37837 0.16216 300.16217 150.16216 15166.37822 0.16216 300.16216 150.16215999999994 15166.37816 2020-01-01 2020-01-02 2020-01-01 00:00:54 2020-01-02 03:45:54 2020-01-01 00:00:54.000 2020-01-02 03:45:54.000 54 99954 50004 5050404 54 99954 50004 5050404 -32515 32420 4583.009900990099 462884 -128 127 -2.1782178217821784 -220 -540 100 10530 99441 1.62162 298.62162 150.12162 15012.16216 1.62162 298.6216 150.12162 15012.16224 1.62162 298.62162 150.12161999999992 15012.16200 2020-01-01 2020-01-02 2020-01-01 00:09:00 2020-01-02 03:37:21 2020-01-01 00:09:00.000 2020-01-02 03:37:21.000 540 99441 49990.5 4999050 540 99441 49990.5 4999050 -32630 32508 4770.66 477066 -124 127 -1.18 -118 +531 100 10521 99432 1.59459 298.59459 150.09459 15009.45945 1.59459 298.5946 150.09459 15009.45941 1.59459 298.59459 150.09459 15009.45900 2020-01-01 2020-01-02 2020-01-01 00:08:51 2020-01-02 03:37:12 2020-01-01 00:08:51.000 2020-01-02 03:37:12.000 531 99432 49981.5 4998150 531 99432 49981.5 4998150 -32639 32499 4761.66 476166 -127 125 -2.5 -250 +532 100 10522 99433 1.59759 298.59759 150.09759 15009.75975 1.59759 298.5976 150.09759 15009.75958 1.59759 298.59759 150.09759 15009.75900 2020-01-01 2020-01-02 2020-01-01 00:08:52 2020-01-02 03:37:13 2020-01-01 00:08:52.000 2020-01-02 03:37:13.000 532 99433 49982.5 4998250 532 99433 49982.5 4998250 -32638 32500 4762.66 476266 -126 126 -1.5 -150 +533 100 10523 99434 1.6006 298.6006 150.1006 15010.06006 1.6006 298.6006 150.10059 15010.05988 1.60060 298.60060 150.1006 15010.06000 2020-01-01 2020-01-02 2020-01-01 00:08:53 2020-01-02 03:37:14 2020-01-01 00:08:53.000 2020-01-02 03:37:14.000 533 99434 49983.5 4998350 533 99434 49983.5 4998350 -32637 32501 4763.66 476366 -125 127 -0.5 -50 +534 100 10524 99435 1.6036 298.6036 150.1036 15010.36036 1.6036 298.6036 150.1036 15010.36063 1.60360 298.60360 150.1036 15010.36000 2020-01-01 2020-01-02 2020-01-01 00:08:54 2020-01-02 03:37:15 2020-01-01 00:08:54.000 2020-01-02 03:37:15.000 534 99435 49984.5 4998450 534 99435 49984.5 4998450 -32636 32502 4764.66 476466 -128 127 -2.06 -206 +535 100 10525 99436 1.6066 298.6066 150.1066 15010.66066 1.6066 298.6066 150.1066 15010.66077 1.60660 298.60660 150.1066 15010.66000 2020-01-01 2020-01-02 2020-01-01 00:08:55 2020-01-02 03:37:16 2020-01-01 00:08:55.000 2020-01-02 03:37:16.000 535 99436 49985.5 4998550 535 99436 49985.5 4998550 -32635 32503 4765.66 476566 -128 127 -3.62 -362 +536 100 10526 99437 1.6096 298.6096 150.1096 15010.96096 1.6096 298.60962 150.1096 15010.96092 1.60960 298.60960 150.1096 15010.96000 2020-01-01 2020-01-02 2020-01-01 00:08:56 2020-01-02 03:37:17 2020-01-01 00:08:56.000 2020-01-02 03:37:17.000 536 99437 49986.5 4998650 536 99437 49986.5 4998650 -32634 32504 4766.66 476666 -128 123 -5.18 -518 +537 100 10527 99438 1.61261 298.61261 150.11261 15011.26126 1.61261 298.6126 150.11261 15011.26105 1.61261 298.61261 150.11261000000002 15011.26100 2020-01-01 2020-01-02 2020-01-01 00:08:57 2020-01-02 03:37:18 2020-01-01 00:08:57.000 2020-01-02 03:37:18.000 537 99438 49987.5 4998750 537 99438 49987.5 4998750 -32633 32505 4767.66 476766 -127 124 -4.18 -418 +538 100 10528 99439 1.61561 298.61561 150.11561 15011.56156 1.61561 298.6156 150.11561 15011.56135 1.61561 298.61561 150.11561 15011.56100 2020-01-01 2020-01-02 2020-01-01 00:08:58 2020-01-02 03:37:19 2020-01-01 00:08:58.000 2020-01-02 03:37:19.000 538 99439 49988.5 4998850 538 99439 49988.5 4998850 -32632 32506 4768.66 476866 -126 125 -3.18 -318 +539 100 10529 99440 1.61861 298.61861 150.11861 15011.86186 1.61861 298.61862 150.11862 15011.8621 1.61861 298.61861 150.11861000000002 15011.86100 2020-01-01 2020-01-02 2020-01-01 00:08:59 2020-01-02 03:37:20 2020-01-01 00:08:59.000 2020-01-02 03:37:20.000 539 99440 49989.5 4998950 539 99440 49989.5 4998950 -32631 32507 4769.66 476966 -125 126 -2.18 -218 +54 102 10044 99954 0.16216 300.16216 150.16216 15166.37837 0.16216 300.16217 150.16216 15166.37822 0.16216 300.16216 150.16216 15166.37816 2020-01-01 2020-01-02 2020-01-01 00:00:54 2020-01-02 03:45:54 2020-01-01 00:00:54.000 2020-01-02 03:45:54.000 54 99954 50004 5050404 54 99954 50004 5050404 -32515 32420 4583.009900990099 462884 -128 127 -2.1782178217821784 -220 +540 100 10530 99441 1.62162 298.62162 150.12162 15012.16216 1.62162 298.6216 150.12162 15012.16224 1.62162 298.62162 150.12162 15012.16200 2020-01-01 2020-01-02 2020-01-01 00:09:00 2020-01-02 03:37:21 2020-01-01 00:09:00.000 2020-01-02 03:37:21.000 540 99441 49990.5 4999050 540 99441 49990.5 4999050 -32630 32508 4770.66 477066 -124 127 -1.18 -118 541 100 10531 99442 1.62462 298.62462 150.12462 15012.46246 1.62462 298.62463 150.12462 15012.46239 1.62462 298.62462 150.12462 15012.46200 2020-01-01 2020-01-02 2020-01-01 00:09:01 2020-01-02 03:37:22 2020-01-01 00:09:01.000 2020-01-02 03:37:22.000 541 99442 49991.5 4999150 541 99442 49991.5 4999150 -32629 32509 4771.66 477166 -128 127 -2.74 -274 -542 100 10532 99443 1.62762 298.62762 150.12762 15012.76276 1.62762 298.62762 150.12762 15012.76252 1.62762 298.62762 150.12761999999998 15012.76200 2020-01-01 2020-01-02 2020-01-01 00:09:02 2020-01-02 03:37:23 2020-01-01 00:09:02.000 2020-01-02 03:37:23.000 542 99443 49992.5 4999250 542 99443 49992.5 4999250 -32628 32510 4772.66 477266 -128 123 -4.3 -430 -543 100 10533 99444 1.63063 298.63063 150.13063 15013.06306 1.63063 298.63065 150.13063 15013.06327 1.63063 298.63063 150.13062999999994 15013.06300 2020-01-01 2020-01-02 2020-01-01 00:09:03 2020-01-02 03:37:24 2020-01-01 00:09:03.000 2020-01-02 03:37:24.000 543 99444 49993.5 4999350 543 99444 49993.5 4999350 -32627 32511 4773.66 477366 -127 124 -3.3 -330 -544 100 10534 99445 1.63363 298.63363 150.13363 15013.36336 1.63363 298.63364 150.13363 15013.36358 1.63363 298.63363 150.13363000000004 15013.36300 2020-01-01 2020-01-02 2020-01-01 00:09:04 2020-01-02 03:37:25 2020-01-01 00:09:04.000 2020-01-02 03:37:25.000 544 99445 49994.5 4999450 544 99445 49994.5 4999450 -32626 32512 4774.66 477466 -126 125 -2.3 -230 -545 100 10535 99446 1.63663 298.63663 150.13663 15013.66366 1.63663 298.63663 150.13663 15013.6637 1.63663 298.63663 150.13663000000008 15013.66300 2020-01-01 2020-01-02 2020-01-01 00:09:05 2020-01-02 03:37:26 2020-01-01 00:09:05.000 2020-01-02 03:37:26.000 545 99446 49995.5 4999550 545 99446 49995.5 4999550 -32625 32513 4775.66 477566 -125 126 -1.3 -130 -546 100 10536 99447 1.63963 298.63963 150.13963 15013.96396 1.63963 298.63965 150.13963 15013.96385 1.63963 298.63963 150.1396299999999 15013.96300 2020-01-01 2020-01-02 2020-01-01 00:09:06 2020-01-02 03:37:27 2020-01-01 00:09:06.000 2020-01-02 03:37:27.000 546 99447 49996.5 4999650 546 99447 49996.5 4999650 -32624 32514 4776.66 477666 -124 127 -0.3 -30 +542 100 10532 99443 1.62762 298.62762 150.12762 15012.76276 1.62762 298.62762 150.12762 15012.76252 1.62762 298.62762 150.12762 15012.76200 2020-01-01 2020-01-02 2020-01-01 00:09:02 2020-01-02 03:37:23 2020-01-01 00:09:02.000 2020-01-02 03:37:23.000 542 99443 49992.5 4999250 542 99443 49992.5 4999250 -32628 32510 4772.66 477266 -128 123 -4.3 -430 +543 100 10533 99444 1.63063 298.63063 150.13063 15013.06306 1.63063 298.63065 150.13063 15013.06327 1.63063 298.63063 150.13063 15013.06300 2020-01-01 2020-01-02 2020-01-01 00:09:03 2020-01-02 03:37:24 2020-01-01 00:09:03.000 2020-01-02 03:37:24.000 543 99444 49993.5 4999350 543 99444 49993.5 4999350 -32627 32511 4773.66 477366 -127 124 -3.3 -330 +544 100 10534 99445 1.63363 298.63363 150.13363 15013.36336 1.63363 298.63364 150.13363 15013.36358 1.63363 298.63363 150.13362999999998 15013.36300 2020-01-01 2020-01-02 2020-01-01 00:09:04 2020-01-02 03:37:25 2020-01-01 00:09:04.000 2020-01-02 03:37:25.000 544 99445 49994.5 4999450 544 99445 49994.5 4999450 -32626 32512 4774.66 477466 -126 125 -2.3 -230 +545 100 10535 99446 1.63663 298.63663 150.13663 15013.66366 1.63663 298.63663 150.13663 15013.6637 1.63663 298.63663 150.13663 15013.66300 2020-01-01 2020-01-02 2020-01-01 00:09:05 2020-01-02 03:37:26 2020-01-01 00:09:05.000 2020-01-02 03:37:26.000 545 99446 49995.5 4999550 545 99446 49995.5 4999550 -32625 32513 4775.66 477566 -125 126 -1.3 -130 +546 100 10536 99447 1.63963 298.63963 150.13963 15013.96396 1.63963 298.63965 150.13963 15013.96385 1.63963 298.63963 150.13963 15013.96300 2020-01-01 2020-01-02 2020-01-01 00:09:06 2020-01-02 03:37:27 2020-01-01 00:09:06.000 2020-01-02 03:37:27.000 546 99447 49996.5 4999650 546 99447 49996.5 4999650 -32624 32514 4776.66 477666 -124 127 -0.3 -30 547 100 10537 99448 1.64264 298.64264 150.14264 15014.26426 1.64264 298.64264 150.14263 15014.26399 1.64264 298.64264 150.14264 15014.26400 2020-01-01 2020-01-02 2020-01-01 00:09:07 2020-01-02 03:37:28 2020-01-01 00:09:07.000 2020-01-02 03:37:28.000 547 99448 49997.5 4999750 547 99448 49997.5 4999750 -32623 32515 4777.66 477766 -128 127 -1.86 -186 -548 100 10538 99449 1.64564 298.64564 150.14564 15014.56456 1.64564 298.64566 150.14564 15014.56474 1.64564 298.64564 150.14564000000007 15014.56400 2020-01-01 2020-01-02 2020-01-01 00:09:08 2020-01-02 03:37:29 2020-01-01 00:09:08.000 2020-01-02 03:37:29.000 548 99449 49998.5 4999850 548 99449 49998.5 4999850 -32622 32516 4778.66 477866 -128 123 -3.42 -342 -549 100 10539 99450 1.64864 298.64864 150.14864 15014.86486 1.64864 298.64865 150.14865 15014.86504 1.64864 298.64864 150.14863999999997 15014.86400 2020-01-01 2020-01-02 2020-01-01 00:09:09 2020-01-02 03:37:30 2020-01-01 00:09:09.000 2020-01-02 03:37:30.000 549 99450 49999.5 4999950 549 99450 49999.5 4999950 -32621 32517 4779.66 477966 -127 124 -2.42 -242 -55 102 10045 99955 0.16516 300.16516 150.16516 15166.68168 0.16516 300.16516 150.16516 15166.68151 0.16516 300.16516 150.16515999999996 15166.68116 2020-01-01 2020-01-02 2020-01-01 00:00:55 2020-01-02 03:45:55 2020-01-01 00:00:55.000 2020-01-02 03:45:55.000 55 99955 50005 5050505 55 99955 50005 5050505 -32514 32421 4584.009900990099 462985 -128 123 -3.712871287128713 -375 -550 100 10540 99451 1.65165 298.65165 150.15165 15015.16516 1.65165 298.65164 150.15165 15015.16521 1.65165 298.65165 150.1516499999999 15015.16500 2020-01-01 2020-01-02 2020-01-01 00:09:10 2020-01-02 03:37:31 2020-01-01 00:09:10.000 2020-01-02 03:37:31.000 550 99451 50000.5 5000050 550 99451 50000.5 5000050 -32620 32518 4780.66 478066 -126 125 -1.42 -142 -551 100 10541 99452 1.65465 298.65465 150.15465 15015.46546 1.65465 298.65466 150.15465 15015.46533 1.65465 298.65465 150.15465000000012 15015.46500 2020-01-01 2020-01-02 2020-01-01 00:09:11 2020-01-02 03:37:32 2020-01-01 00:09:11.000 2020-01-02 03:37:32.000 551 99452 50001.5 5000150 551 99452 50001.5 5000150 -32619 32519 4781.66 478166 -125 126 -0.42 -42 -552 100 10542 99453 1.65765 298.65765 150.15765 15015.76576 1.65765 298.65765 150.15765 15015.76562 1.65765 298.65765 150.15764999999996 15015.76500 2020-01-01 2020-01-02 2020-01-01 00:09:12 2020-01-02 03:37:33 2020-01-01 00:09:12.000 2020-01-02 03:37:33.000 552 99453 50002.5 5000250 552 99453 50002.5 5000250 -32618 32520 4782.66 478266 -124 127 0.58 58 -553 100 10543 99454 1.66066 298.66066 150.16066 15016.06606 1.66066 298.66068 150.16066 15016.06621 1.66066 298.66066 150.16065999999984 15016.06600 2020-01-01 2020-01-02 2020-01-01 00:09:13 2020-01-02 03:37:34 2020-01-01 00:09:13.000 2020-01-02 03:37:34.000 553 99454 50003.5 5000350 553 99454 50003.5 5000350 -32617 32521 4783.66 478366 -128 127 -0.98 -98 -554 100 10544 99455 1.66366 298.66366 150.16366 15016.36636 1.66366 298.66367 150.16366 15016.36651 1.66366 298.66366 150.16366000000008 15016.36600 2020-01-01 2020-01-02 2020-01-01 00:09:14 2020-01-02 03:37:35 2020-01-01 00:09:14.000 2020-01-02 03:37:35.000 554 99455 50004.5 5000450 554 99455 50004.5 5000450 -32616 32522 4784.66 478466 -128 127 -2.54 -254 -555 100 10545 99456 1.66666 298.66666 150.16666 15016.66666 1.66666 298.66666 150.16666 15016.66668 1.66666 298.66666 150.16666000000004 15016.66600 2020-01-01 2020-01-02 2020-01-01 00:09:15 2020-01-02 03:37:36 2020-01-01 00:09:15.000 2020-01-02 03:37:36.000 555 99456 50005.5 5000550 555 99456 50005.5 5000550 -32615 32523 4785.66 478566 -128 124 -4.1 -410 -556 100 10546 99457 1.66966 298.66966 150.16966 15016.96696 1.66966 298.66968 150.16966 15016.9668 1.66966 298.66966 150.16965999999985 15016.96600 2020-01-01 2020-01-02 2020-01-01 00:09:16 2020-01-02 03:37:37 2020-01-01 00:09:16.000 2020-01-02 03:37:37.000 556 99457 50006.5 5000650 556 99457 50006.5 5000650 -32614 32524 4786.66 478666 -127 125 -3.1 -310 -557 100 10547 99458 1.67267 298.67267 150.17267 15017.26726 1.67267 298.67267 150.17267 15017.26709 1.67267 298.67267 150.17266999999995 15017.26700 2020-01-01 2020-01-02 2020-01-01 00:09:17 2020-01-02 03:37:38 2020-01-01 00:09:17.000 2020-01-02 03:37:38.000 557 99458 50007.5 5000750 557 99458 50007.5 5000750 -32613 32525 4787.66 478766 -126 126 -2.1 -210 +548 100 10538 99449 1.64564 298.64564 150.14564 15014.56456 1.64564 298.64566 150.14564 15014.56474 1.64564 298.64564 150.14564000000001 15014.56400 2020-01-01 2020-01-02 2020-01-01 00:09:08 2020-01-02 03:37:29 2020-01-01 00:09:08.000 2020-01-02 03:37:29.000 548 99449 49998.5 4999850 548 99449 49998.5 4999850 -32622 32516 4778.66 477866 -128 123 -3.42 -342 +549 100 10539 99450 1.64864 298.64864 150.14864 15014.86486 1.64864 298.64865 150.14865 15014.86504 1.64864 298.64864 150.14864 15014.86400 2020-01-01 2020-01-02 2020-01-01 00:09:09 2020-01-02 03:37:30 2020-01-01 00:09:09.000 2020-01-02 03:37:30.000 549 99450 49999.5 4999950 549 99450 49999.5 4999950 -32621 32517 4779.66 477966 -127 124 -2.42 -242 +55 102 10045 99955 0.16516 300.16516 150.16516 15166.68168 0.16516 300.16516 150.16516 15166.68151 0.16516 300.16516 150.16516000000001 15166.68116 2020-01-01 2020-01-02 2020-01-01 00:00:55 2020-01-02 03:45:55 2020-01-01 00:00:55.000 2020-01-02 03:45:55.000 55 99955 50005 5050505 55 99955 50005 5050505 -32514 32421 4584.009900990099 462985 -128 123 -3.712871287128713 -375 +550 100 10540 99451 1.65165 298.65165 150.15165 15015.16516 1.65165 298.65164 150.15165 15015.16521 1.65165 298.65165 150.15165000000002 15015.16500 2020-01-01 2020-01-02 2020-01-01 00:09:10 2020-01-02 03:37:31 2020-01-01 00:09:10.000 2020-01-02 03:37:31.000 550 99451 50000.5 5000050 550 99451 50000.5 5000050 -32620 32518 4780.66 478066 -126 125 -1.42 -142 +551 100 10541 99452 1.65465 298.65465 150.15465 15015.46546 1.65465 298.65466 150.15465 15015.46533 1.65465 298.65465 150.15465 15015.46500 2020-01-01 2020-01-02 2020-01-01 00:09:11 2020-01-02 03:37:32 2020-01-01 00:09:11.000 2020-01-02 03:37:32.000 551 99452 50001.5 5000150 551 99452 50001.5 5000150 -32619 32519 4781.66 478166 -125 126 -0.42 -42 +552 100 10542 99453 1.65765 298.65765 150.15765 15015.76576 1.65765 298.65765 150.15765 15015.76562 1.65765 298.65765 150.15765 15015.76500 2020-01-01 2020-01-02 2020-01-01 00:09:12 2020-01-02 03:37:33 2020-01-01 00:09:12.000 2020-01-02 03:37:33.000 552 99453 50002.5 5000250 552 99453 50002.5 5000250 -32618 32520 4782.66 478266 -124 127 0.58 58 +553 100 10543 99454 1.66066 298.66066 150.16066 15016.06606 1.66066 298.66068 150.16066 15016.06621 1.66066 298.66066 150.16066 15016.06600 2020-01-01 2020-01-02 2020-01-01 00:09:13 2020-01-02 03:37:34 2020-01-01 00:09:13.000 2020-01-02 03:37:34.000 553 99454 50003.5 5000350 553 99454 50003.5 5000350 -32617 32521 4783.66 478366 -128 127 -0.98 -98 +554 100 10544 99455 1.66366 298.66366 150.16366 15016.36636 1.66366 298.66367 150.16366 15016.36651 1.66366 298.66366 150.16366 15016.36600 2020-01-01 2020-01-02 2020-01-01 00:09:14 2020-01-02 03:37:35 2020-01-01 00:09:14.000 2020-01-02 03:37:35.000 554 99455 50004.5 5000450 554 99455 50004.5 5000450 -32616 32522 4784.66 478466 -128 127 -2.54 -254 +555 100 10545 99456 1.66666 298.66666 150.16666 15016.66666 1.66666 298.66666 150.16666 15016.66668 1.66666 298.66666 150.16665999999998 15016.66600 2020-01-01 2020-01-02 2020-01-01 00:09:15 2020-01-02 03:37:36 2020-01-01 00:09:15.000 2020-01-02 03:37:36.000 555 99456 50005.5 5000550 555 99456 50005.5 5000550 -32615 32523 4785.66 478566 -128 124 -4.1 -410 +556 100 10546 99457 1.66966 298.66966 150.16966 15016.96696 1.66966 298.66968 150.16966 15016.9668 1.66966 298.66966 150.16966 15016.96600 2020-01-01 2020-01-02 2020-01-01 00:09:16 2020-01-02 03:37:37 2020-01-01 00:09:16.000 2020-01-02 03:37:37.000 556 99457 50006.5 5000650 556 99457 50006.5 5000650 -32614 32524 4786.66 478666 -127 125 -3.1 -310 +557 100 10547 99458 1.67267 298.67267 150.17267 15017.26726 1.67267 298.67267 150.17267 15017.26709 1.67267 298.67267 150.17267 15017.26700 2020-01-01 2020-01-02 2020-01-01 00:09:17 2020-01-02 03:37:38 2020-01-01 00:09:17.000 2020-01-02 03:37:38.000 557 99458 50007.5 5000750 557 99458 50007.5 5000750 -32613 32525 4787.66 478766 -126 126 -2.1 -210 558 100 10548 99459 1.67567 298.67567 150.17567 15017.56756 1.67567 298.6757 150.17567 15017.56769 1.67567 298.67567 150.17567 15017.56700 2020-01-01 2020-01-02 2020-01-01 00:09:18 2020-01-02 03:37:39 2020-01-01 00:09:18.000 2020-01-02 03:37:39.000 558 99459 50008.5 5000850 558 99459 50008.5 5000850 -32612 32526 4788.66 478866 -125 127 -1.1 -110 -559 100 10549 99460 1.67867 298.67867 150.17867 15017.86786 1.67867 298.67868 150.17868 15017.86802 1.67867 298.67867 150.17866999999998 15017.86700 2020-01-01 2020-01-02 2020-01-01 00:09:19 2020-01-02 03:37:40 2020-01-01 00:09:19.000 2020-01-02 03:37:40.000 559 99460 50009.5 5000950 559 99460 50009.5 5000950 -32611 32527 4789.66 478966 -128 127 -2.66 -266 -56 102 10046 99956 0.16816 300.16816 150.16816 15166.98498 0.16816 300.16818 150.16816 15166.98512 0.16816 300.16816 150.16815999999997 15166.98416 2020-01-01 2020-01-02 2020-01-01 00:00:56 2020-01-02 03:45:56 2020-01-01 00:00:56.000 2020-01-02 03:45:56.000 56 99956 50006 5050606 56 99956 50006 5050606 -32513 32422 4585.009900990099 463086 -127 124 -2.712871287128713 -274 -560 100 10550 99461 1.68168 298.68168 150.18168 15018.16816 1.68168 298.68167 150.18168 15018.16815 1.68168 298.68168 150.18167999999991 15018.16800 2020-01-01 2020-01-02 2020-01-01 00:09:20 2020-01-02 03:37:41 2020-01-01 00:09:20.000 2020-01-02 03:37:41.000 560 99461 50010.5 5001050 560 99461 50010.5 5001050 -32610 32528 4790.66 479066 -128 127 -4.22 -422 -561 100 10551 99462 1.68468 298.68468 150.18468 15018.46846 1.68468 298.6847 150.18468 15018.46826 1.68468 298.68468 150.18468000000007 15018.46800 2020-01-01 2020-01-02 2020-01-01 00:09:21 2020-01-02 03:37:42 2020-01-01 00:09:21.000 2020-01-02 03:37:42.000 561 99462 50011.5 5001150 561 99462 50011.5 5001150 -32609 32529 4791.66 479166 -128 123 -5.78 -578 -562 100 10552 99463 1.68768 298.68768 150.18768 15018.76876 1.68768 298.68768 150.18768 15018.76856 1.68768 298.68768 150.18767999999992 15018.76800 2020-01-01 2020-01-02 2020-01-01 00:09:22 2020-01-02 03:37:43 2020-01-01 00:09:22.000 2020-01-02 03:37:43.000 562 99463 50012.5 5001250 562 99463 50012.5 5001250 -32608 32530 4792.66 479266 -127 124 -4.78 -478 +559 100 10549 99460 1.67867 298.67867 150.17867 15017.86786 1.67867 298.67868 150.17868 15017.86802 1.67867 298.67867 150.17867 15017.86700 2020-01-01 2020-01-02 2020-01-01 00:09:19 2020-01-02 03:37:40 2020-01-01 00:09:19.000 2020-01-02 03:37:40.000 559 99460 50009.5 5000950 559 99460 50009.5 5000950 -32611 32527 4789.66 478966 -128 127 -2.66 -266 +56 102 10046 99956 0.16816 300.16816 150.16816 15166.98498 0.16816 300.16818 150.16816 15166.98512 0.16816 300.16816 150.16816 15166.98416 2020-01-01 2020-01-02 2020-01-01 00:00:56 2020-01-02 03:45:56 2020-01-01 00:00:56.000 2020-01-02 03:45:56.000 56 99956 50006 5050606 56 99956 50006 5050606 -32513 32422 4585.009900990099 463086 -127 124 -2.712871287128713 -274 +560 100 10550 99461 1.68168 298.68168 150.18168 15018.16816 1.68168 298.68167 150.18168 15018.16815 1.68168 298.68168 150.18168 15018.16800 2020-01-01 2020-01-02 2020-01-01 00:09:20 2020-01-02 03:37:41 2020-01-01 00:09:20.000 2020-01-02 03:37:41.000 560 99461 50010.5 5001050 560 99461 50010.5 5001050 -32610 32528 4790.66 479066 -128 127 -4.22 -422 +561 100 10551 99462 1.68468 298.68468 150.18468 15018.46846 1.68468 298.6847 150.18468 15018.46826 1.68468 298.68468 150.18468000000001 15018.46800 2020-01-01 2020-01-02 2020-01-01 00:09:21 2020-01-02 03:37:42 2020-01-01 00:09:21.000 2020-01-02 03:37:42.000 561 99462 50011.5 5001150 561 99462 50011.5 5001150 -32609 32529 4791.66 479166 -128 123 -5.78 -578 +562 100 10552 99463 1.68768 298.68768 150.18768 15018.76876 1.68768 298.68768 150.18768 15018.76856 1.68768 298.68768 150.18768 15018.76800 2020-01-01 2020-01-02 2020-01-01 00:09:22 2020-01-02 03:37:43 2020-01-01 00:09:22.000 2020-01-02 03:37:43.000 562 99463 50012.5 5001250 562 99463 50012.5 5001250 -32608 32530 4792.66 479266 -127 124 -4.78 -478 563 100 10553 99464 1.69069 298.69069 150.19069 15019.06906 1.69069 298.6907 150.19069 15019.06915 1.69069 298.69069 150.19069 15019.06900 2020-01-01 2020-01-02 2020-01-01 00:09:23 2020-01-02 03:37:44 2020-01-01 00:09:23.000 2020-01-02 03:37:44.000 563 99464 50013.5 5001350 563 99464 50013.5 5001350 -32607 32531 4793.66 479366 -126 125 -3.78 -378 -564 100 10554 99465 1.69369 298.69369 150.19369 15019.36936 1.69369 298.6937 150.19369 15019.36948 1.69369 298.69369 150.19369000000003 15019.36900 2020-01-01 2020-01-02 2020-01-01 00:09:24 2020-01-02 03:37:45 2020-01-01 00:09:24.000 2020-01-02 03:37:45.000 564 99465 50014.5 5001450 564 99465 50014.5 5001450 -32606 32532 4794.66 479466 -125 126 -2.78 -278 -565 100 10555 99466 1.69669 298.69669 150.19669 15019.66966 1.69669 298.6967 150.19669 15019.66962 1.69669 298.69669 150.19669000000002 15019.66900 2020-01-01 2020-01-02 2020-01-01 00:09:25 2020-01-02 03:37:46 2020-01-01 00:09:25.000 2020-01-02 03:37:46.000 565 99466 50015.5 5001550 565 99466 50015.5 5001550 -32605 32533 4795.66 479566 -124 127 -1.78 -178 +564 100 10554 99465 1.69369 298.69369 150.19369 15019.36936 1.69369 298.6937 150.19369 15019.36948 1.69369 298.69369 150.19369 15019.36900 2020-01-01 2020-01-02 2020-01-01 00:09:24 2020-01-02 03:37:45 2020-01-01 00:09:24.000 2020-01-02 03:37:45.000 564 99465 50014.5 5001450 564 99465 50014.5 5001450 -32606 32532 4794.66 479466 -125 126 -2.78 -278 +565 100 10555 99466 1.69669 298.69669 150.19669 15019.66966 1.69669 298.6967 150.19669 15019.66962 1.69669 298.69669 150.19669 15019.66900 2020-01-01 2020-01-02 2020-01-01 00:09:25 2020-01-02 03:37:46 2020-01-01 00:09:25.000 2020-01-02 03:37:46.000 565 99466 50015.5 5001550 565 99466 50015.5 5001550 -32605 32533 4795.66 479566 -124 127 -1.78 -178 566 100 10556 99467 1.69969 298.69969 150.19969 15019.96996 1.69969 298.6997 150.1997 15019.97037 1.69969 298.69969 150.19969 15019.96900 2020-01-01 2020-01-02 2020-01-01 00:09:26 2020-01-02 03:37:47 2020-01-01 00:09:26.000 2020-01-02 03:37:47.000 566 99467 50016.5 5001650 566 99467 50016.5 5001650 -32604 32534 4796.66 479666 -128 127 -3.34 -334 -567 100 10557 99468 1.7027 298.7027 150.2027 15020.27027 1.7027 298.7027 150.2027 15020.27003 1.70270 298.70270 150.2026999999999 15020.27000 2020-01-01 2020-01-02 2020-01-01 00:09:27 2020-01-02 03:37:48 2020-01-01 00:09:27.000 2020-01-02 03:37:48.000 567 99468 50017.5 5001750 567 99468 50017.5 5001750 -32603 32535 4797.66 479766 -128 123 -4.9 -490 -568 100 10558 99469 1.7057 298.7057 150.2057 15020.57057 1.7057 298.70572 150.2057 15020.57066 1.70570 298.70570 150.20570000000015 15020.57000 2020-01-01 2020-01-02 2020-01-01 00:09:28 2020-01-02 03:37:49 2020-01-01 00:09:28.000 2020-01-02 03:37:49.000 568 99469 50018.5 5001850 568 99469 50018.5 5001850 -32602 32536 4798.66 479866 -127 124 -3.9 -390 -569 100 10559 99470 1.7087 298.7087 150.2087 15020.87087 1.7087 298.7087 150.2087 15020.87095 1.70870 298.70870 150.20869999999996 15020.87000 2020-01-01 2020-01-02 2020-01-01 00:09:29 2020-01-02 03:37:50 2020-01-01 00:09:29.000 2020-01-02 03:37:50.000 569 99470 50019.5 5001950 569 99470 50019.5 5001950 -32601 32537 4799.66 479966 -126 125 -2.9 -290 -57 102 10047 99957 0.17117 300.17117 150.17117 15167.28828 0.17117 300.17117 150.17117 15167.28841 0.17117 300.17117 150.1711699999999 15167.28817 2020-01-01 2020-01-02 2020-01-01 00:00:57 2020-01-02 03:45:57 2020-01-01 00:00:57.000 2020-01-02 03:45:57.000 57 99957 50007 5050707 57 99957 50007 5050707 -32512 32423 4586.009900990099 463187 -126 125 -1.7128712871287128 -173 -570 100 10560 99471 1.71171 298.71171 150.21171 15021.17117 1.71171 298.7117 150.21171 15021.17109 1.71171 298.71171 150.21170999999987 15021.17100 2020-01-01 2020-01-02 2020-01-01 00:09:30 2020-01-02 03:37:51 2020-01-01 00:09:30.000 2020-01-02 03:37:51.000 570 99471 50020.5 5002050 570 99471 50020.5 5002050 -32600 32538 4800.66 480066 -125 126 -1.9 -190 -571 100 10561 99472 1.71471 298.71471 150.21471 15021.47147 1.71471 298.71472 150.21471 15021.47184 1.71471 298.71471 150.21471000000008 15021.47100 2020-01-01 2020-01-02 2020-01-01 00:09:31 2020-01-02 03:37:52 2020-01-01 00:09:31.000 2020-01-02 03:37:52.000 571 99472 50021.5 5002150 571 99472 50021.5 5002150 -32599 32539 4801.66 480166 -124 127 -0.9 -90 -572 100 10562 99473 1.71771 298.71771 150.21771 15021.77177 1.71771 298.7177 150.21771 15021.7715 1.71771 298.71771 150.21771000000004 15021.77100 2020-01-01 2020-01-02 2020-01-01 00:09:32 2020-01-02 03:37:53 2020-01-01 00:09:32.000 2020-01-02 03:37:53.000 572 99473 50022.5 5002250 572 99473 50022.5 5002250 -32598 32540 4802.66 480266 -128 127 -2.46 -246 -573 100 10563 99474 1.72072 298.72072 150.22072 15022.07207 1.72072 298.72073 150.22072 15022.07212 1.72072 298.72072 150.22071999999991 15022.07200 2020-01-01 2020-01-02 2020-01-01 00:09:33 2020-01-02 03:37:54 2020-01-01 00:09:33.000 2020-01-02 03:37:54.000 573 99474 50023.5 5002350 573 99474 50023.5 5002350 -32597 32541 4803.66 480366 -128 123 -4.02 -402 +567 100 10557 99468 1.7027 298.7027 150.2027 15020.27027 1.7027 298.7027 150.2027 15020.27003 1.70270 298.70270 150.2027 15020.27000 2020-01-01 2020-01-02 2020-01-01 00:09:27 2020-01-02 03:37:48 2020-01-01 00:09:27.000 2020-01-02 03:37:48.000 567 99468 50017.5 5001750 567 99468 50017.5 5001750 -32603 32535 4797.66 479766 -128 123 -4.9 -490 +568 100 10558 99469 1.7057 298.7057 150.2057 15020.57057 1.7057 298.70572 150.2057 15020.57066 1.70570 298.70570 150.2057 15020.57000 2020-01-01 2020-01-02 2020-01-01 00:09:28 2020-01-02 03:37:49 2020-01-01 00:09:28.000 2020-01-02 03:37:49.000 568 99469 50018.5 5001850 568 99469 50018.5 5001850 -32602 32536 4798.66 479866 -127 124 -3.9 -390 +569 100 10559 99470 1.7087 298.7087 150.2087 15020.87087 1.7087 298.7087 150.2087 15020.87095 1.70870 298.70870 150.20870000000002 15020.87000 2020-01-01 2020-01-02 2020-01-01 00:09:29 2020-01-02 03:37:50 2020-01-01 00:09:29.000 2020-01-02 03:37:50.000 569 99470 50019.5 5001950 569 99470 50019.5 5001950 -32601 32537 4799.66 479966 -126 125 -2.9 -290 +57 102 10047 99957 0.17117 300.17117 150.17117 15167.28828 0.17117 300.17117 150.17117 15167.28841 0.17117 300.17117 150.17117 15167.28817 2020-01-01 2020-01-02 2020-01-01 00:00:57 2020-01-02 03:45:57 2020-01-01 00:00:57.000 2020-01-02 03:45:57.000 57 99957 50007 5050707 57 99957 50007 5050707 -32512 32423 4586.009900990099 463187 -126 125 -1.7128712871287128 -173 +570 100 10560 99471 1.71171 298.71171 150.21171 15021.17117 1.71171 298.7117 150.21171 15021.17109 1.71171 298.71171 150.21171 15021.17100 2020-01-01 2020-01-02 2020-01-01 00:09:30 2020-01-02 03:37:51 2020-01-01 00:09:30.000 2020-01-02 03:37:51.000 570 99471 50020.5 5002050 570 99471 50020.5 5002050 -32600 32538 4800.66 480066 -125 126 -1.9 -190 +571 100 10561 99472 1.71471 298.71471 150.21471 15021.47147 1.71471 298.71472 150.21471 15021.47184 1.71471 298.71471 150.21471 15021.47100 2020-01-01 2020-01-02 2020-01-01 00:09:31 2020-01-02 03:37:52 2020-01-01 00:09:31.000 2020-01-02 03:37:52.000 571 99472 50021.5 5002150 571 99472 50021.5 5002150 -32599 32539 4801.66 480166 -124 127 -0.9 -90 +572 100 10562 99473 1.71771 298.71771 150.21771 15021.77177 1.71771 298.7177 150.21771 15021.7715 1.71771 298.71771 150.21771 15021.77100 2020-01-01 2020-01-02 2020-01-01 00:09:32 2020-01-02 03:37:53 2020-01-01 00:09:32.000 2020-01-02 03:37:53.000 572 99473 50022.5 5002250 572 99473 50022.5 5002250 -32598 32540 4802.66 480266 -128 127 -2.46 -246 +573 100 10563 99474 1.72072 298.72072 150.22072 15022.07207 1.72072 298.72073 150.22072 15022.07212 1.72072 298.72072 150.22072 15022.07200 2020-01-01 2020-01-02 2020-01-01 00:09:33 2020-01-02 03:37:54 2020-01-01 00:09:33.000 2020-01-02 03:37:54.000 573 99474 50023.5 5002350 573 99474 50023.5 5002350 -32597 32541 4803.66 480366 -128 123 -4.02 -402 574 100 10564 99475 1.72372 298.72372 150.22372 15022.37237 1.72372 298.72372 150.22372 15022.37243 1.72372 298.72372 150.22372 15022.37200 2020-01-01 2020-01-02 2020-01-01 00:09:34 2020-01-02 03:37:55 2020-01-01 00:09:34.000 2020-01-02 03:37:55.000 574 99475 50024.5 5002450 574 99475 50024.5 5002450 -32596 32542 4804.66 480466 -127 124 -3.02 -302 -575 100 10565 99476 1.72672 298.72672 150.22672 15022.67267 1.72672 298.7267 150.22672 15022.67272 1.72672 298.72672 150.22672000000003 15022.67200 2020-01-01 2020-01-02 2020-01-01 00:09:35 2020-01-02 03:37:56 2020-01-01 00:09:35.000 2020-01-02 03:37:56.000 575 99476 50025.5 5002550 575 99476 50025.5 5002550 -32595 32543 4805.66 480566 -126 125 -2.02 -202 -576 100 10566 99477 1.72972 298.72972 150.22972 15022.97297 1.72972 298.72974 150.22973 15022.97332 1.72972 298.72972 150.22971999999993 15022.97200 2020-01-01 2020-01-02 2020-01-01 00:09:36 2020-01-02 03:37:57 2020-01-01 00:09:36.000 2020-01-02 03:37:57.000 576 99477 50026.5 5002650 576 99477 50026.5 5002650 -32594 32544 4806.66 480666 -125 126 -1.02 -102 -577 100 10567 99478 1.73273 298.73273 150.23273 15023.27327 1.73273 298.73273 150.23272 15023.27297 1.73273 298.73273 150.23272999999995 15023.27300 2020-01-01 2020-01-02 2020-01-01 00:09:37 2020-01-02 03:37:58 2020-01-01 00:09:37.000 2020-01-02 03:37:58.000 577 99478 50027.5 5002750 577 99478 50027.5 5002750 -32593 32545 4807.66 480766 -124 127 -0.02 -2 -578 100 10568 99479 1.73573 298.73573 150.23573 15023.57357 1.73573 298.73575 150.23573 15023.57359 1.73573 298.73573 150.2357300000001 15023.57300 2020-01-01 2020-01-02 2020-01-01 00:09:38 2020-01-02 03:37:59 2020-01-01 00:09:38.000 2020-01-02 03:37:59.000 578 99479 50028.5 5002850 578 99479 50028.5 5002850 -32592 32546 4808.66 480866 -128 127 -1.58 -158 -579 100 10569 99480 1.73873 298.73873 150.23873 15023.87387 1.73873 298.73874 150.23873 15023.8739 1.73873 298.73873 150.23872999999992 15023.87300 2020-01-01 2020-01-02 2020-01-01 00:09:39 2020-01-02 03:38:00 2020-01-01 00:09:39.000 2020-01-02 03:38:00.000 579 99480 50029.5 5002950 579 99480 50029.5 5002950 -32591 32547 4809.66 480966 -128 123 -3.14 -314 -58 102 10048 99958 0.17417 300.17417 150.17417 15167.59159 0.17417 300.17416 150.17417 15167.59159 0.17417 300.17417 150.17417000000012 15167.59117 2020-01-01 2020-01-02 2020-01-01 00:00:58 2020-01-02 03:45:58 2020-01-01 00:00:58.000 2020-01-02 03:45:58.000 58 99958 50008 5050808 58 99958 50008 5050808 -32511 32424 4587.009900990099 463288 -125 126 -0.7128712871287128 -72 -580 100 10570 99481 1.74174 298.74174 150.24174 15024.17417 1.74174 298.74173 150.24174 15024.17419 1.74174 298.74174 150.24174 15024.17400 2020-01-01 2020-01-02 2020-01-01 00:09:40 2020-01-02 03:38:01 2020-01-01 00:09:40.000 2020-01-02 03:38:01.000 580 99481 50030.5 5003050 580 99481 50030.5 5003050 -32590 32548 4810.66 481066 -127 124 -2.14 -214 -581 100 10571 99482 1.74474 298.74474 150.24474 15024.47447 1.74474 298.74475 150.24474 15024.47478 1.74474 298.74474 150.24474000000006 15024.47400 2020-01-01 2020-01-02 2020-01-01 00:09:41 2020-01-02 03:38:02 2020-01-01 00:09:41.000 2020-01-02 03:38:02.000 581 99482 50031.5 5003150 581 99482 50031.5 5003150 -32589 32549 4811.66 481166 -126 125 -1.14 -114 -582 100 10572 99483 1.74774 298.74774 150.24774 15024.77477 1.74774 298.74774 150.24774 15024.77447 1.74774 298.74774 150.24774000000002 15024.77400 2020-01-01 2020-01-02 2020-01-01 00:09:42 2020-01-02 03:38:03 2020-01-01 00:09:42.000 2020-01-02 03:38:03.000 582 99483 50032.5 5003250 582 99483 50032.5 5003250 -32588 32550 4812.66 481266 -125 126 -0.14 -14 -583 100 10573 99484 1.75075 298.75075 150.25075 15025.07507 1.75075 298.75076 150.25075 15025.07507 1.75075 298.75075 150.25074999999995 15025.07500 2020-01-01 2020-01-02 2020-01-01 00:09:43 2020-01-02 03:38:04 2020-01-01 00:09:43.000 2020-01-02 03:38:04.000 583 99484 50033.5 5003350 583 99484 50033.5 5003350 -32587 32551 4813.66 481366 -124 127 0.86 86 -584 100 10574 99485 1.75375 298.75375 150.25375 15025.37537 1.75375 298.75375 150.25375 15025.37536 1.75375 298.75375 150.25374999999994 15025.37500 2020-01-01 2020-01-02 2020-01-01 00:09:44 2020-01-02 03:38:05 2020-01-01 00:09:44.000 2020-01-02 03:38:05.000 584 99485 50034.5 5003450 584 99485 50034.5 5003450 -32586 32552 4814.66 481466 -128 127 -0.7 -70 -585 100 10575 99486 1.75675 298.75675 150.25675 15025.67567 1.75675 298.75674 150.25675 15025.67566 1.75675 298.75675 150.25675000000012 15025.67500 2020-01-01 2020-01-02 2020-01-01 00:09:45 2020-01-02 03:38:06 2020-01-01 00:09:45.000 2020-01-02 03:38:06.000 585 99486 50035.5 5003550 585 99486 50035.5 5003550 -32585 32553 4815.66 481566 -128 127 -2.26 -226 -586 100 10576 99487 1.75975 298.75975 150.25975 15025.97597 1.75975 298.75977 150.25976 15025.97625 1.75975 298.75975 150.25974999999997 15025.97500 2020-01-01 2020-01-02 2020-01-01 00:09:46 2020-01-02 03:38:07 2020-01-01 00:09:46.000 2020-01-02 03:38:07.000 586 99487 50036.5 5003650 586 99487 50036.5 5003650 -32584 32554 4816.66 481666 -128 123 -3.82 -382 -587 100 10577 99488 1.76276 298.76276 150.26276 15026.27627 1.76276 298.76276 150.26275 15026.27594 1.76276 298.76276 150.2627599999999 15026.27600 2020-01-01 2020-01-02 2020-01-01 00:09:47 2020-01-02 03:38:08 2020-01-01 00:09:47.000 2020-01-02 03:38:08.000 587 99488 50037.5 5003750 587 99488 50037.5 5003750 -32583 32555 4817.66 481766 -127 124 -2.82 -282 -588 100 10578 99489 1.76576 298.76576 150.26576 15026.57657 1.76576 298.76578 150.26576 15026.57654 1.76576 298.76576 150.2657600000001 15026.57600 2020-01-01 2020-01-02 2020-01-01 00:09:48 2020-01-02 03:38:09 2020-01-01 00:09:48.000 2020-01-02 03:38:09.000 588 99489 50038.5 5003850 588 99489 50038.5 5003850 -32582 32556 4818.66 481866 -126 125 -1.82 -182 -589 100 10579 99490 1.76876 298.76876 150.26876 15026.87687 1.76876 298.76877 150.26876 15026.87683 1.76876 298.76876 150.26876000000007 15026.87600 2020-01-01 2020-01-02 2020-01-01 00:09:49 2020-01-02 03:38:10 2020-01-01 00:09:49.000 2020-01-02 03:38:10.000 589 99490 50039.5 5003950 589 99490 50039.5 5003950 -32581 32557 4819.66 481966 -125 126 -0.82 -82 -59 102 10049 99959 0.17717 300.17717 150.17717 15167.89489 0.17717 300.1772 150.17717 15167.8947 0.17717 300.17717 150.17717000000007 15167.89417 2020-01-01 2020-01-02 2020-01-01 00:00:59 2020-01-02 03:45:59 2020-01-01 00:00:59.000 2020-01-02 03:45:59.000 59 99959 50009 5050909 59 99959 50009 5050909 -32510 32425 4588.009900990099 463389 -124 127 0.2871287128712871 29 -590 100 10580 99491 1.77177 298.77177 150.27177 15027.17717 1.77177 298.77176 150.27177 15027.17713 1.77177 298.77177 150.27176999999995 15027.17700 2020-01-01 2020-01-02 2020-01-01 00:09:50 2020-01-02 03:38:11 2020-01-01 00:09:50.000 2020-01-02 03:38:11.000 590 99491 50040.5 5004050 590 99491 50040.5 5004050 -32580 32558 4820.66 482066 -124 127 0.18 18 -591 100 10581 99492 1.77477 298.77477 150.27477 15027.47747 1.77477 298.77478 150.27477 15027.47775 1.77477 298.77477 150.27477 15027.47700 2020-01-01 2020-01-02 2020-01-01 00:09:51 2020-01-02 03:38:12 2020-01-01 00:09:51.000 2020-01-02 03:38:12.000 591 99492 50041.5 5004150 591 99492 50041.5 5004150 -32579 32559 4821.66 482166 -128 127 -1.38 -138 -592 100 10582 99493 1.77777 298.77777 150.27777 15027.77777 1.77777 298.77777 150.27777 15027.77742 1.77777 298.77777 150.27777000000003 15027.77700 2020-01-01 2020-01-02 2020-01-01 00:09:52 2020-01-02 03:38:13 2020-01-01 00:09:52.000 2020-01-02 03:38:13.000 592 99493 50042.5 5004250 592 99493 50042.5 5004250 -32578 32560 4822.66 482266 -128 123 -2.94 -294 -593 100 10583 99494 1.78078 298.78078 150.28078 15028.07807 1.78078 298.7808 150.28078 15028.078 1.78078 298.78078 150.28077999999994 15028.07800 2020-01-01 2020-01-02 2020-01-01 00:09:53 2020-01-02 03:38:14 2020-01-01 00:09:53.000 2020-01-02 03:38:14.000 593 99494 50043.5 5004350 593 99494 50043.5 5004350 -32577 32561 4823.66 482366 -127 124 -1.94 -194 -594 100 10584 99495 1.78378 298.78378 150.28378 15028.37837 1.78378 298.78378 150.28378 15028.3783 1.78378 298.78378 150.28377999999992 15028.37800 2020-01-01 2020-01-02 2020-01-01 00:09:54 2020-01-02 03:38:15 2020-01-01 00:09:54.000 2020-01-02 03:38:15.000 594 99495 50044.5 5004450 594 99495 50044.5 5004450 -32576 32562 4824.66 482466 -126 125 -0.94 -94 -595 100 10585 99496 1.78678 298.78678 150.28678 15028.67867 1.78678 298.78677 150.28678 15028.6786 1.78678 298.78678 150.28678000000008 15028.67800 2020-01-01 2020-01-02 2020-01-01 00:09:55 2020-01-02 03:38:16 2020-01-01 00:09:55.000 2020-01-02 03:38:16.000 595 99496 50045.5 5004550 595 99496 50045.5 5004550 -32575 32563 4825.66 482566 -125 126 0.06 6 -596 100 10586 99497 1.78978 298.78978 150.28978 15028.97897 1.78978 298.7898 150.28979 15028.97922 1.78978 298.78978 150.28977999999992 15028.97800 2020-01-01 2020-01-02 2020-01-01 00:09:56 2020-01-02 03:38:17 2020-01-01 00:09:56.000 2020-01-02 03:38:17.000 596 99497 50046.5 5004650 596 99497 50046.5 5004650 -32574 32564 4826.66 482666 -124 127 1.06 106 -597 100 10587 99498 1.79279 298.79279 150.29279 15029.27927 1.79279 298.7928 150.29278 15029.27888 1.79279 298.79279 150.29279000000002 15029.27900 2020-01-01 2020-01-02 2020-01-01 00:09:57 2020-01-02 03:38:18 2020-01-01 00:09:57.000 2020-01-02 03:38:18.000 597 99498 50047.5 5004750 597 99498 50047.5 5004750 -32573 32565 4827.66 482766 -128 127 -0.5 -50 -598 100 10588 99499 1.79579 298.79579 150.29579 15029.57957 1.79579 298.7958 150.29579 15029.57964 1.79579 298.79579 150.29579000000007 15029.57900 2020-01-01 2020-01-02 2020-01-01 00:09:58 2020-01-02 03:38:19 2020-01-01 00:09:58.000 2020-01-02 03:38:19.000 598 99499 50048.5 5004850 598 99499 50048.5 5004850 -32572 32566 4828.66 482866 -128 123 -2.06 -206 -599 100 10589 99500 1.79879 298.79879 150.29879 15029.87987 1.79879 298.7988 150.29879 15029.87977 1.79879 298.79879 150.29879000000003 15029.87900 2020-01-01 2020-01-02 2020-01-01 00:09:59 2020-01-02 03:38:20 2020-01-01 00:09:59.000 2020-01-02 03:38:20.000 599 99500 50049.5 5004950 599 99500 50049.5 5004950 -32571 32567 4829.66 482966 -127 124 -1.06 -106 -6 102 1005 9996 0.01801 300.01801 150.01801 15151.81981 0.01801 300.018 150.01801 15151.81978 0.01801 300.01801 150.0180099999999 15151.81901 2020-01-01 2020-01-02 2020-01-01 00:00:06 2020-01-02 03:45:06 2020-01-01 00:00:06.000 2020-01-02 03:45:06.000 6 99906 49956 5045556 6 99906 49956 5045556 -32563 32372 4535.009900990099 458036 -127 124 -2.01980198019802 -204 -60 102 10050 99960 0.18018 300.18018 150.18018 15168.19819 0.18018 300.18018 150.18017 15168.198 0.18018 300.18018 150.18017999999995 15168.19818 2020-01-01 2020-01-02 2020-01-01 00:01:00 2020-01-02 03:46:00 2020-01-01 00:01:00.000 2020-01-02 03:46:00.000 60 99960 50010 5051010 60 99960 50010 5051010 -32509 32426 4589.009900990099 463490 -128 127 -1.2475247524752475 -126 -600 100 10590 99501 1.8018 298.8018 150.3018 15030.18018 1.8018 298.8018 150.3018 15030.1801 1.80180 298.80180 150.3018 15030.18000 2020-01-01 2020-01-02 2020-01-01 00:10:00 2020-01-02 03:38:21 2020-01-01 00:10:00.000 2020-01-02 03:38:21.000 600 99501 50050.5 5005050 600 99501 50050.5 5005050 -32570 32568 4830.66 483066 -126 125 -0.06 -6 -601 100 10591 99502 1.8048 298.8048 150.3048 15030.48048 1.8048 298.8048 150.3048 15030.4807 1.80480 298.80480 150.30479999999991 15030.48000 2020-01-01 2020-01-02 2020-01-01 00:10:01 2020-01-02 03:38:22 2020-01-01 00:10:01.000 2020-01-02 03:38:22.000 601 99502 50051.5 5005150 601 99502 50051.5 5005150 -32569 32569 4831.66 483166 -125 126 0.94 94 -602 100 10592 99503 1.8078 298.8078 150.3078 15030.78078 1.8078 298.8078 150.3078 15030.78035 1.80780 298.80780 150.3078 15030.78000 2020-01-01 2020-01-02 2020-01-01 00:10:02 2020-01-02 03:38:23 2020-01-01 00:10:02.000 2020-01-02 03:38:23.000 602 99503 50052.5 5005250 602 99503 50052.5 5005250 -32568 32570 4832.66 483266 -124 127 1.94 194 -603 100 10593 99504 1.81081 298.81081 150.31081 15031.08108 1.81081 298.81082 150.31081 15031.0811 1.81081 298.81081 150.31081000000006 15031.08100 2020-01-01 2020-01-02 2020-01-01 00:10:03 2020-01-02 03:38:24 2020-01-01 00:10:03.000 2020-01-02 03:38:24.000 603 99504 50053.5 5005350 603 99504 50053.5 5005350 -32567 32571 4833.66 483366 -128 127 0.38 38 -604 100 10594 99505 1.81381 298.81381 150.31381 15031.38138 1.81381 298.8138 150.31381 15031.38124 1.81381 298.81381 150.31380999999993 15031.38100 2020-01-01 2020-01-02 2020-01-01 00:10:04 2020-01-02 03:38:25 2020-01-01 00:10:04.000 2020-01-02 03:38:25.000 604 99505 50054.5 5005450 604 99505 50054.5 5005450 -32566 32572 4834.66 483466 -128 123 -1.18 -118 -605 100 10595 99506 1.81681 298.81681 150.31681 15031.68168 1.81681 298.8168 150.31681 15031.68157 1.81681 298.81681 150.31681000000003 15031.68100 2020-01-01 2020-01-02 2020-01-01 00:10:05 2020-01-02 03:38:26 2020-01-01 00:10:05.000 2020-01-02 03:38:26.000 605 99506 50055.5 5005550 605 99506 50055.5 5005550 -32565 32573 4835.66 483566 -127 124 -0.18 -18 -606 100 10596 99507 1.81981 298.81981 150.31981 15031.98198 1.81981 298.81982 150.31982 15031.98217 1.81981 298.81981 150.31981000000007 15031.98100 2020-01-01 2020-01-02 2020-01-01 00:10:06 2020-01-02 03:38:27 2020-01-01 00:10:06.000 2020-01-02 03:38:27.000 606 99507 50056.5 5005650 606 99507 50056.5 5005650 -32564 32574 4836.66 483666 -126 125 0.82 82 +575 100 10565 99476 1.72672 298.72672 150.22672 15022.67267 1.72672 298.7267 150.22672 15022.67272 1.72672 298.72672 150.22672 15022.67200 2020-01-01 2020-01-02 2020-01-01 00:09:35 2020-01-02 03:37:56 2020-01-01 00:09:35.000 2020-01-02 03:37:56.000 575 99476 50025.5 5002550 575 99476 50025.5 5002550 -32595 32543 4805.66 480566 -126 125 -2.02 -202 +576 100 10566 99477 1.72972 298.72972 150.22972 15022.97297 1.72972 298.72974 150.22973 15022.97332 1.72972 298.72972 150.22972 15022.97200 2020-01-01 2020-01-02 2020-01-01 00:09:36 2020-01-02 03:37:57 2020-01-01 00:09:36.000 2020-01-02 03:37:57.000 576 99477 50026.5 5002650 576 99477 50026.5 5002650 -32594 32544 4806.66 480666 -125 126 -1.02 -102 +577 100 10567 99478 1.73273 298.73273 150.23273 15023.27327 1.73273 298.73273 150.23272 15023.27297 1.73273 298.73273 150.23273 15023.27300 2020-01-01 2020-01-02 2020-01-01 00:09:37 2020-01-02 03:37:58 2020-01-01 00:09:37.000 2020-01-02 03:37:58.000 577 99478 50027.5 5002750 577 99478 50027.5 5002750 -32593 32545 4807.66 480766 -124 127 -0.02 -2 +578 100 10568 99479 1.73573 298.73573 150.23573 15023.57357 1.73573 298.73575 150.23573 15023.57359 1.73573 298.73573 150.23573 15023.57300 2020-01-01 2020-01-02 2020-01-01 00:09:38 2020-01-02 03:37:59 2020-01-01 00:09:38.000 2020-01-02 03:37:59.000 578 99479 50028.5 5002850 578 99479 50028.5 5002850 -32592 32546 4808.66 480866 -128 127 -1.58 -158 +579 100 10569 99480 1.73873 298.73873 150.23873 15023.87387 1.73873 298.73874 150.23873 15023.8739 1.73873 298.73873 150.23873 15023.87300 2020-01-01 2020-01-02 2020-01-01 00:09:39 2020-01-02 03:38:00 2020-01-01 00:09:39.000 2020-01-02 03:38:00.000 579 99480 50029.5 5002950 579 99480 50029.5 5002950 -32591 32547 4809.66 480966 -128 123 -3.14 -314 +58 102 10048 99958 0.17417 300.17417 150.17417 15167.59159 0.17417 300.17416 150.17417 15167.59159 0.17417 300.17417 150.17417 15167.59117 2020-01-01 2020-01-02 2020-01-01 00:00:58 2020-01-02 03:45:58 2020-01-01 00:00:58.000 2020-01-02 03:45:58.000 58 99958 50008 5050808 58 99958 50008 5050808 -32511 32424 4587.009900990099 463288 -125 126 -0.7128712871287128 -72 +580 100 10570 99481 1.74174 298.74174 150.24174 15024.17417 1.74174 298.74173 150.24174 15024.17419 1.74174 298.74174 150.24174000000002 15024.17400 2020-01-01 2020-01-02 2020-01-01 00:09:40 2020-01-02 03:38:01 2020-01-01 00:09:40.000 2020-01-02 03:38:01.000 580 99481 50030.5 5003050 580 99481 50030.5 5003050 -32590 32548 4810.66 481066 -127 124 -2.14 -214 +581 100 10571 99482 1.74474 298.74474 150.24474 15024.47447 1.74474 298.74475 150.24474 15024.47478 1.74474 298.74474 150.24474 15024.47400 2020-01-01 2020-01-02 2020-01-01 00:09:41 2020-01-02 03:38:02 2020-01-01 00:09:41.000 2020-01-02 03:38:02.000 581 99482 50031.5 5003150 581 99482 50031.5 5003150 -32589 32549 4811.66 481166 -126 125 -1.14 -114 +582 100 10572 99483 1.74774 298.74774 150.24774 15024.77477 1.74774 298.74774 150.24774 15024.77447 1.74774 298.74774 150.24774 15024.77400 2020-01-01 2020-01-02 2020-01-01 00:09:42 2020-01-02 03:38:03 2020-01-01 00:09:42.000 2020-01-02 03:38:03.000 582 99483 50032.5 5003250 582 99483 50032.5 5003250 -32588 32550 4812.66 481266 -125 126 -0.14 -14 +583 100 10573 99484 1.75075 298.75075 150.25075 15025.07507 1.75075 298.75076 150.25075 15025.07507 1.75075 298.75075 150.25075 15025.07500 2020-01-01 2020-01-02 2020-01-01 00:09:43 2020-01-02 03:38:04 2020-01-01 00:09:43.000 2020-01-02 03:38:04.000 583 99484 50033.5 5003350 583 99484 50033.5 5003350 -32587 32551 4813.66 481366 -124 127 0.86 86 +584 100 10574 99485 1.75375 298.75375 150.25375 15025.37537 1.75375 298.75375 150.25375 15025.37536 1.75375 298.75375 150.25375 15025.37500 2020-01-01 2020-01-02 2020-01-01 00:09:44 2020-01-02 03:38:05 2020-01-01 00:09:44.000 2020-01-02 03:38:05.000 584 99485 50034.5 5003450 584 99485 50034.5 5003450 -32586 32552 4814.66 481466 -128 127 -0.7 -70 +585 100 10575 99486 1.75675 298.75675 150.25675 15025.67567 1.75675 298.75674 150.25675 15025.67566 1.75675 298.75675 150.25674999999998 15025.67500 2020-01-01 2020-01-02 2020-01-01 00:09:45 2020-01-02 03:38:06 2020-01-01 00:09:45.000 2020-01-02 03:38:06.000 585 99486 50035.5 5003550 585 99486 50035.5 5003550 -32585 32553 4815.66 481566 -128 127 -2.26 -226 +586 100 10576 99487 1.75975 298.75975 150.25975 15025.97597 1.75975 298.75977 150.25976 15025.97625 1.75975 298.75975 150.25975 15025.97500 2020-01-01 2020-01-02 2020-01-01 00:09:46 2020-01-02 03:38:07 2020-01-01 00:09:46.000 2020-01-02 03:38:07.000 586 99487 50036.5 5003650 586 99487 50036.5 5003650 -32584 32554 4816.66 481666 -128 123 -3.82 -382 +587 100 10577 99488 1.76276 298.76276 150.26276 15026.27627 1.76276 298.76276 150.26275 15026.27594 1.76276 298.76276 150.26276 15026.27600 2020-01-01 2020-01-02 2020-01-01 00:09:47 2020-01-02 03:38:08 2020-01-01 00:09:47.000 2020-01-02 03:38:08.000 587 99488 50037.5 5003750 587 99488 50037.5 5003750 -32583 32555 4817.66 481766 -127 124 -2.82 -282 +588 100 10578 99489 1.76576 298.76576 150.26576 15026.57657 1.76576 298.76578 150.26576 15026.57654 1.76576 298.76576 150.26576 15026.57600 2020-01-01 2020-01-02 2020-01-01 00:09:48 2020-01-02 03:38:09 2020-01-01 00:09:48.000 2020-01-02 03:38:09.000 588 99489 50038.5 5003850 588 99489 50038.5 5003850 -32582 32556 4818.66 481866 -126 125 -1.82 -182 +589 100 10579 99490 1.76876 298.76876 150.26876 15026.87687 1.76876 298.76877 150.26876 15026.87683 1.76876 298.76876 150.26876000000001 15026.87600 2020-01-01 2020-01-02 2020-01-01 00:09:49 2020-01-02 03:38:10 2020-01-01 00:09:49.000 2020-01-02 03:38:10.000 589 99490 50039.5 5003950 589 99490 50039.5 5003950 -32581 32557 4819.66 481966 -125 126 -0.82 -82 +59 102 10049 99959 0.17717 300.17717 150.17717 15167.89489 0.17717 300.1772 150.17717 15167.8947 0.17717 300.17717 150.17717 15167.89417 2020-01-01 2020-01-02 2020-01-01 00:00:59 2020-01-02 03:45:59 2020-01-01 00:00:59.000 2020-01-02 03:45:59.000 59 99959 50009 5050909 59 99959 50009 5050909 -32510 32425 4588.009900990099 463389 -124 127 0.2871287128712871 29 +590 100 10580 99491 1.77177 298.77177 150.27177 15027.17717 1.77177 298.77176 150.27177 15027.17713 1.77177 298.77177 150.27177 15027.17700 2020-01-01 2020-01-02 2020-01-01 00:09:50 2020-01-02 03:38:11 2020-01-01 00:09:50.000 2020-01-02 03:38:11.000 590 99491 50040.5 5004050 590 99491 50040.5 5004050 -32580 32558 4820.66 482066 -124 127 0.18 18 +591 100 10581 99492 1.77477 298.77477 150.27477 15027.47747 1.77477 298.77478 150.27477 15027.47775 1.77477 298.77477 150.27477000000002 15027.47700 2020-01-01 2020-01-02 2020-01-01 00:09:51 2020-01-02 03:38:12 2020-01-01 00:09:51.000 2020-01-02 03:38:12.000 591 99492 50041.5 5004150 591 99492 50041.5 5004150 -32579 32559 4821.66 482166 -128 127 -1.38 -138 +592 100 10582 99493 1.77777 298.77777 150.27777 15027.77777 1.77777 298.77777 150.27777 15027.77742 1.77777 298.77777 150.27777 15027.77700 2020-01-01 2020-01-02 2020-01-01 00:09:52 2020-01-02 03:38:13 2020-01-01 00:09:52.000 2020-01-02 03:38:13.000 592 99493 50042.5 5004250 592 99493 50042.5 5004250 -32578 32560 4822.66 482266 -128 123 -2.94 -294 +593 100 10583 99494 1.78078 298.78078 150.28078 15028.07807 1.78078 298.7808 150.28078 15028.078 1.78078 298.78078 150.28078 15028.07800 2020-01-01 2020-01-02 2020-01-01 00:09:53 2020-01-02 03:38:14 2020-01-01 00:09:53.000 2020-01-02 03:38:14.000 593 99494 50043.5 5004350 593 99494 50043.5 5004350 -32577 32561 4823.66 482366 -127 124 -1.94 -194 +594 100 10584 99495 1.78378 298.78378 150.28378 15028.37837 1.78378 298.78378 150.28378 15028.3783 1.78378 298.78378 150.28378 15028.37800 2020-01-01 2020-01-02 2020-01-01 00:09:54 2020-01-02 03:38:15 2020-01-01 00:09:54.000 2020-01-02 03:38:15.000 594 99495 50044.5 5004450 594 99495 50044.5 5004450 -32576 32562 4824.66 482466 -126 125 -0.94 -94 +595 100 10585 99496 1.78678 298.78678 150.28678 15028.67867 1.78678 298.78677 150.28678 15028.6786 1.78678 298.78678 150.28678 15028.67800 2020-01-01 2020-01-02 2020-01-01 00:09:55 2020-01-02 03:38:16 2020-01-01 00:09:55.000 2020-01-02 03:38:16.000 595 99496 50045.5 5004550 595 99496 50045.5 5004550 -32575 32563 4825.66 482566 -125 126 0.06 6 +596 100 10586 99497 1.78978 298.78978 150.28978 15028.97897 1.78978 298.7898 150.28979 15028.97922 1.78978 298.78978 150.28977999999998 15028.97800 2020-01-01 2020-01-02 2020-01-01 00:09:56 2020-01-02 03:38:17 2020-01-01 00:09:56.000 2020-01-02 03:38:17.000 596 99497 50046.5 5004650 596 99497 50046.5 5004650 -32574 32564 4826.66 482666 -124 127 1.06 106 +597 100 10587 99498 1.79279 298.79279 150.29279 15029.27927 1.79279 298.7928 150.29278 15029.27888 1.79279 298.79279 150.29279 15029.27900 2020-01-01 2020-01-02 2020-01-01 00:09:57 2020-01-02 03:38:18 2020-01-01 00:09:57.000 2020-01-02 03:38:18.000 597 99498 50047.5 5004750 597 99498 50047.5 5004750 -32573 32565 4827.66 482766 -128 127 -0.5 -50 +598 100 10588 99499 1.79579 298.79579 150.29579 15029.57957 1.79579 298.7958 150.29579 15029.57964 1.79579 298.79579 150.29579 15029.57900 2020-01-01 2020-01-02 2020-01-01 00:09:58 2020-01-02 03:38:19 2020-01-01 00:09:58.000 2020-01-02 03:38:19.000 598 99499 50048.5 5004850 598 99499 50048.5 5004850 -32572 32566 4828.66 482866 -128 123 -2.06 -206 +599 100 10589 99500 1.79879 298.79879 150.29879 15029.87987 1.79879 298.7988 150.29879 15029.87977 1.79879 298.79879 150.29879 15029.87900 2020-01-01 2020-01-02 2020-01-01 00:09:59 2020-01-02 03:38:20 2020-01-01 00:09:59.000 2020-01-02 03:38:20.000 599 99500 50049.5 5004950 599 99500 50049.5 5004950 -32571 32567 4829.66 482966 -127 124 -1.06 -106 +6 102 1005 9996 0.01801 300.01801 150.01801 15151.81981 0.01801 300.018 150.01801 15151.81978 0.01801 300.01801 150.01801 15151.81901 2020-01-01 2020-01-02 2020-01-01 00:00:06 2020-01-02 03:45:06 2020-01-01 00:00:06.000 2020-01-02 03:45:06.000 6 99906 49956 5045556 6 99906 49956 5045556 -32563 32372 4535.009900990099 458036 -127 124 -2.01980198019802 -204 +60 102 10050 99960 0.18018 300.18018 150.18018 15168.19819 0.18018 300.18018 150.18017 15168.198 0.18018 300.18018 150.18018 15168.19818 2020-01-01 2020-01-02 2020-01-01 00:01:00 2020-01-02 03:46:00 2020-01-01 00:01:00.000 2020-01-02 03:46:00.000 60 99960 50010 5051010 60 99960 50010 5051010 -32509 32426 4589.009900990099 463490 -128 127 -1.2475247524752475 -126 +600 100 10590 99501 1.8018 298.8018 150.3018 15030.18018 1.8018 298.8018 150.3018 15030.1801 1.80180 298.80180 150.30180000000001 15030.18000 2020-01-01 2020-01-02 2020-01-01 00:10:00 2020-01-02 03:38:21 2020-01-01 00:10:00.000 2020-01-02 03:38:21.000 600 99501 50050.5 5005050 600 99501 50050.5 5005050 -32570 32568 4830.66 483066 -126 125 -0.06 -6 +601 100 10591 99502 1.8048 298.8048 150.3048 15030.48048 1.8048 298.8048 150.3048 15030.4807 1.80480 298.80480 150.3048 15030.48000 2020-01-01 2020-01-02 2020-01-01 00:10:01 2020-01-02 03:38:22 2020-01-01 00:10:01.000 2020-01-02 03:38:22.000 601 99502 50051.5 5005150 601 99502 50051.5 5005150 -32569 32569 4831.66 483166 -125 126 0.94 94 +602 100 10592 99503 1.8078 298.8078 150.3078 15030.78078 1.8078 298.8078 150.3078 15030.78035 1.80780 298.80780 150.30780000000001 15030.78000 2020-01-01 2020-01-02 2020-01-01 00:10:02 2020-01-02 03:38:23 2020-01-01 00:10:02.000 2020-01-02 03:38:23.000 602 99503 50052.5 5005250 602 99503 50052.5 5005250 -32568 32570 4832.66 483266 -124 127 1.94 194 +603 100 10593 99504 1.81081 298.81081 150.31081 15031.08108 1.81081 298.81082 150.31081 15031.0811 1.81081 298.81081 150.31081 15031.08100 2020-01-01 2020-01-02 2020-01-01 00:10:03 2020-01-02 03:38:24 2020-01-01 00:10:03.000 2020-01-02 03:38:24.000 603 99504 50053.5 5005350 603 99504 50053.5 5005350 -32567 32571 4833.66 483366 -128 127 0.38 38 +604 100 10594 99505 1.81381 298.81381 150.31381 15031.38138 1.81381 298.8138 150.31381 15031.38124 1.81381 298.81381 150.31381 15031.38100 2020-01-01 2020-01-02 2020-01-01 00:10:04 2020-01-02 03:38:25 2020-01-01 00:10:04.000 2020-01-02 03:38:25.000 604 99505 50054.5 5005450 604 99505 50054.5 5005450 -32566 32572 4834.66 483466 -128 123 -1.18 -118 +605 100 10595 99506 1.81681 298.81681 150.31681 15031.68168 1.81681 298.8168 150.31681 15031.68157 1.81681 298.81681 150.31681 15031.68100 2020-01-01 2020-01-02 2020-01-01 00:10:05 2020-01-02 03:38:26 2020-01-01 00:10:05.000 2020-01-02 03:38:26.000 605 99506 50055.5 5005550 605 99506 50055.5 5005550 -32565 32573 4835.66 483566 -127 124 -0.18 -18 +606 100 10596 99507 1.81981 298.81981 150.31981 15031.98198 1.81981 298.81982 150.31982 15031.98217 1.81981 298.81981 150.31981 15031.98100 2020-01-01 2020-01-02 2020-01-01 00:10:06 2020-01-02 03:38:27 2020-01-01 00:10:06.000 2020-01-02 03:38:27.000 606 99507 50056.5 5005650 606 99507 50056.5 5005650 -32564 32574 4836.66 483666 -126 125 0.82 82 607 100 10597 99508 1.82282 298.82282 150.32282 15032.28228 1.82282 298.8228 150.32282 15032.28246 1.82282 298.82282 150.32281999999998 15032.28200 2020-01-01 2020-01-02 2020-01-01 00:10:07 2020-01-02 03:38:28 2020-01-01 00:10:07.000 2020-01-02 03:38:28.000 607 99508 50057.5 5005750 607 99508 50057.5 5005750 -32563 32575 4837.66 483766 -125 126 1.82 182 -608 100 10598 99509 1.82582 298.82582 150.32582 15032.58258 1.82582 298.82584 150.32582 15032.58258 1.82582 298.82582 150.32582000000002 15032.58200 2020-01-01 2020-01-02 2020-01-01 00:10:08 2020-01-02 03:38:29 2020-01-01 00:10:08.000 2020-01-02 03:38:29.000 608 99509 50058.5 5005850 608 99509 50058.5 5005850 -32562 32576 4838.66 483866 -124 127 2.82 282 -609 100 10599 99510 1.82882 298.82882 150.32882 15032.88288 1.82882 298.82883 150.32882 15032.88274 1.82882 298.82882 150.32882000000012 15032.88200 2020-01-01 2020-01-02 2020-01-01 00:10:09 2020-01-02 03:38:30 2020-01-01 00:10:09.000 2020-01-02 03:38:30.000 609 99510 50059.5 5005950 609 99510 50059.5 5005950 -32561 32577 4839.66 483966 -128 127 1.26 126 +608 100 10598 99509 1.82582 298.82582 150.32582 15032.58258 1.82582 298.82584 150.32582 15032.58258 1.82582 298.82582 150.32582 15032.58200 2020-01-01 2020-01-02 2020-01-01 00:10:08 2020-01-02 03:38:29 2020-01-01 00:10:08.000 2020-01-02 03:38:29.000 608 99509 50058.5 5005850 608 99509 50058.5 5005850 -32562 32576 4838.66 483866 -124 127 2.82 282 +609 100 10599 99510 1.82882 298.82882 150.32882 15032.88288 1.82882 298.82883 150.32882 15032.88274 1.82882 298.82882 150.32882 15032.88200 2020-01-01 2020-01-02 2020-01-01 00:10:09 2020-01-02 03:38:30 2020-01-01 00:10:09.000 2020-01-02 03:38:30.000 609 99510 50059.5 5005950 609 99510 50059.5 5005950 -32561 32577 4839.66 483966 -128 127 1.26 126 61 102 10051 99961 0.18318 300.18318 150.18318 15168.5015 0.18318 300.1832 150.18318 15168.5016 0.18318 300.18318 150.18318 15168.50118 2020-01-01 2020-01-02 2020-01-01 00:01:01 2020-01-02 03:46:01 2020-01-01 00:01:01.000 2020-01-02 03:46:01.000 61 99961 50011 5051111 61 99961 50011 5051111 -32508 32427 4590.009900990099 463591 -128 123 -2.782178217821782 -281 -610 100 10600 99511 1.83183 298.83183 150.33183 15033.18318 1.83183 298.83182 150.33183 15033.18304 1.83183 298.83183 150.33182999999977 15033.18300 2020-01-01 2020-01-02 2020-01-01 00:10:10 2020-01-02 03:38:31 2020-01-01 00:10:10.000 2020-01-02 03:38:31.000 610 99511 50060.5 5006050 610 99511 50060.5 5006050 -32560 32578 4840.66 484066 -128 127 -0.3 -30 -611 100 10601 99512 1.83483 298.83483 150.33483 15033.48348 1.83483 298.83484 150.33483 15033.48363 1.83483 298.83483 150.3348299999998 15033.48300 2020-01-01 2020-01-02 2020-01-01 00:10:11 2020-01-02 03:38:32 2020-01-01 00:10:11.000 2020-01-02 03:38:32.000 611 99512 50061.5 5006150 611 99512 50061.5 5006150 -32559 32579 4841.66 484166 -128 123 -1.86 -186 -612 100 10602 99513 1.83783 298.83783 150.33783 15033.78378 1.83783 298.83783 150.33783 15033.78393 1.83783 298.83783 150.33783000000028 15033.78300 2020-01-01 2020-01-02 2020-01-01 00:10:12 2020-01-02 03:38:33 2020-01-01 00:10:12.000 2020-01-02 03:38:33.000 612 99513 50062.5 5006250 612 99513 50062.5 5006250 -32558 32580 4842.66 484266 -127 124 -0.86 -86 -613 100 10603 99514 1.84084 298.84084 150.34084 15034.08408 1.84084 298.84085 150.34084 15034.08405 1.84084 298.84084 150.34084000000016 15034.08400 2020-01-01 2020-01-02 2020-01-01 00:10:13 2020-01-02 03:38:34 2020-01-01 00:10:13.000 2020-01-02 03:38:34.000 613 99514 50063.5 5006350 613 99514 50063.5 5006350 -32557 32581 4843.66 484366 -126 125 0.14 14 -614 100 10604 99515 1.84384 298.84384 150.34384 15034.38438 1.84384 298.84384 150.34384 15034.38421 1.84384 298.84384 150.34383999999966 15034.38400 2020-01-01 2020-01-02 2020-01-01 00:10:14 2020-01-02 03:38:35 2020-01-01 00:10:14.000 2020-01-02 03:38:35.000 614 99515 50064.5 5006450 614 99515 50064.5 5006450 -32556 32582 4844.66 484466 -125 126 1.14 114 -615 100 10605 99516 1.84684 298.84684 150.34684 15034.68468 1.84684 298.84683 150.34684 15034.68452 1.84684 298.84684 150.34684000000013 15034.68400 2020-01-01 2020-01-02 2020-01-01 00:10:15 2020-01-02 03:38:36 2020-01-01 00:10:15.000 2020-01-02 03:38:36.000 615 99516 50065.5 5006550 615 99516 50065.5 5006550 -32555 32583 4845.66 484566 -124 127 2.14 214 -616 100 10606 99517 1.84984 298.84984 150.34984 15034.98498 1.84984 298.84985 150.34985 15034.98527 1.84984 298.84984 150.34984000000017 15034.98400 2020-01-01 2020-01-02 2020-01-01 00:10:16 2020-01-02 03:38:37 2020-01-01 00:10:16.000 2020-01-02 03:38:37.000 616 99517 50066.5 5006650 616 99517 50066.5 5006650 -32554 32584 4846.66 484666 -128 127 0.58 58 -617 100 10607 99518 1.85285 298.85285 150.35285 15035.28528 1.85285 298.85284 150.35285 15035.2854 1.85285 298.85285 150.3528499999999 15035.28500 2020-01-01 2020-01-02 2020-01-01 00:10:17 2020-01-02 03:38:38 2020-01-01 00:10:17.000 2020-01-02 03:38:38.000 617 99518 50067.5 5006750 617 99518 50067.5 5006750 -32553 32585 4847.66 484766 -128 123 -0.98 -98 -618 100 10608 99519 1.85585 298.85585 150.35585 15035.58558 1.85585 298.85587 150.35585 15035.58551 1.85585 298.85585 150.35584999999995 15035.58500 2020-01-01 2020-01-02 2020-01-01 00:10:18 2020-01-02 03:38:39 2020-01-01 00:10:18.000 2020-01-02 03:38:39.000 618 99519 50068.5 5006850 618 99519 50068.5 5006850 -32552 32586 4848.66 484866 -127 124 0.02 2 -619 100 10609 99520 1.85885 298.85885 150.35885 15035.88588 1.85885 298.85886 150.35885 15035.88568 1.85885 298.85885 150.35885000000002 15035.88500 2020-01-01 2020-01-02 2020-01-01 00:10:19 2020-01-02 03:38:40 2020-01-01 00:10:19.000 2020-01-02 03:38:40.000 619 99520 50069.5 5006950 619 99520 50069.5 5006950 -32551 32587 4849.66 484966 -126 125 1.02 102 -62 102 10052 99962 0.18618 300.18618 150.18618 15168.8048 0.18618 300.1862 150.18618 15168.80494 0.18618 300.18618 150.1861800000001 15168.80418 2020-01-01 2020-01-02 2020-01-01 00:01:02 2020-01-02 03:46:02 2020-01-01 00:01:02.000 2020-01-02 03:46:02.000 62 99962 50012 5051212 62 99962 50012 5051212 -32507 32428 4591.009900990099 463692 -127 124 -1.7821782178217822 -180 -620 100 10610 99521 1.86186 298.86186 150.36186 15036.18618 1.86186 298.86185 150.36185 15036.18598 1.86186 298.86186 150.36186000000026 15036.18600 2020-01-01 2020-01-02 2020-01-01 00:10:20 2020-01-02 03:38:41 2020-01-01 00:10:20.000 2020-01-02 03:38:41.000 620 99521 50070.5 5007050 620 99521 50070.5 5007050 -32550 32588 4850.66 485066 -125 126 2.02 202 -621 100 10611 99522 1.86486 298.86486 150.36486 15036.48648 1.86486 298.86487 150.36486 15036.48673 1.86486 298.86486 150.3648599999998 15036.48600 2020-01-01 2020-01-02 2020-01-01 00:10:21 2020-01-02 03:38:42 2020-01-01 00:10:21.000 2020-01-02 03:38:42.000 621 99522 50071.5 5007150 621 99522 50071.5 5007150 -32549 32589 4851.66 485166 -124 127 3.02 302 -622 100 10612 99523 1.86786 298.86786 150.36786 15036.78678 1.86786 298.86786 150.36786 15036.78687 1.86786 298.86786 150.36786000000018 15036.78600 2020-01-01 2020-01-02 2020-01-01 00:10:22 2020-01-02 03:38:43 2020-01-01 00:10:22.000 2020-01-02 03:38:43.000 622 99523 50072.5 5007250 622 99523 50072.5 5007250 -32548 32590 4852.66 485266 -128 127 1.46 146 -623 100 10613 99524 1.87087 298.87087 150.37087 15037.08708 1.87087 298.87088 150.37087 15037.08702 1.87087 298.87087 150.37087000000014 15037.08700 2020-01-01 2020-01-02 2020-01-01 00:10:23 2020-01-02 03:38:44 2020-01-01 00:10:23.000 2020-01-02 03:38:44.000 623 99524 50073.5 5007350 623 99524 50073.5 5007350 -32547 32591 4853.66 485366 -128 123 -0.1 -10 -624 100 10614 99525 1.87387 298.87387 150.37387 15037.38738 1.87387 298.87387 150.37387 15037.38716 1.87387 298.87387 150.37386999999995 15037.38700 2020-01-01 2020-01-02 2020-01-01 00:10:24 2020-01-02 03:38:45 2020-01-01 00:10:24.000 2020-01-02 03:38:45.000 624 99525 50074.5 5007450 624 99525 50074.5 5007450 -32546 32592 4854.66 485466 -127 124 0.9 90 -625 100 10615 99526 1.87687 298.87687 150.37687 15037.68768 1.87687 298.8769 150.37687 15037.68791 1.87687 298.87687 150.37687000000003 15037.68700 2020-01-01 2020-01-02 2020-01-01 00:10:25 2020-01-02 03:38:46 2020-01-01 00:10:25.000 2020-01-02 03:38:46.000 625 99526 50075.5 5007550 625 99526 50075.5 5007550 -32545 32593 4855.66 485566 -126 125 1.9 190 -626 100 10616 99527 1.87987 298.87987 150.37987 15037.98798 1.87987 298.87988 150.37988 15037.9882 1.87987 298.87987 150.37987000000012 15037.98700 2020-01-01 2020-01-02 2020-01-01 00:10:26 2020-01-02 03:38:47 2020-01-01 00:10:26.000 2020-01-02 03:38:47.000 626 99527 50076.5 5007650 626 99527 50076.5 5007650 -32544 32594 4856.66 485666 -125 126 2.9 290 -627 100 10617 99528 1.88288 298.88288 150.38288 15038.28828 1.88288 298.88287 150.38288 15038.28834 1.88288 298.88288 150.3828799999998 15038.28800 2020-01-01 2020-01-02 2020-01-01 00:10:27 2020-01-02 03:38:48 2020-01-01 00:10:27.000 2020-01-02 03:38:48.000 627 99528 50077.5 5007750 627 99528 50077.5 5007750 -32543 32595 4857.66 485766 -124 127 3.9 390 -628 100 10618 99529 1.88588 298.88588 150.38588 15038.58858 1.88588 298.8859 150.38588 15038.58849 1.88588 298.88588 150.38587999999984 15038.58800 2020-01-01 2020-01-02 2020-01-01 00:10:28 2020-01-02 03:38:49 2020-01-01 00:10:28.000 2020-01-02 03:38:49.000 628 99529 50078.5 5007850 628 99529 50078.5 5007850 -32542 32596 4858.66 485866 -128 127 2.34 234 -629 100 10619 99530 1.88888 298.88888 150.38888 15038.88888 1.88888 298.8889 150.38888 15038.88862 1.88888 298.88888 150.3888800000003 15038.88800 2020-01-01 2020-01-02 2020-01-01 00:10:29 2020-01-02 03:38:50 2020-01-01 00:10:29.000 2020-01-02 03:38:50.000 629 99530 50079.5 5007950 629 99530 50079.5 5007950 -32541 32597 4859.66 485966 -128 123 0.78 78 -63 102 10053 99963 0.18918 300.18918 150.18918 15169.1081 0.18918 300.18918 150.18918 15169.10808 0.18918 300.18918 150.1891799999999 15169.10718 2020-01-01 2020-01-02 2020-01-01 00:01:03 2020-01-02 03:46:03 2020-01-01 00:01:03.000 2020-01-02 03:46:03.000 63 99963 50013 5051313 63 99963 50013 5051313 -32506 32429 4592.009900990099 463793 -126 125 -0.7821782178217822 -79 -630 100 10620 99531 1.89189 298.89189 150.39189 15039.18918 1.89189 298.8919 150.39189 15039.18937 1.89189 298.89189 150.3918900000002 15039.18900 2020-01-01 2020-01-02 2020-01-01 00:10:30 2020-01-02 03:38:51 2020-01-01 00:10:30.000 2020-01-02 03:38:51.000 630 99531 50080.5 5008050 630 99531 50080.5 5008050 -32540 32598 4860.66 486066 -127 124 1.78 178 -631 100 10621 99532 1.89489 298.89489 150.39489 15039.48948 1.89489 298.8949 150.39489 15039.48968 1.89489 298.89489 150.3948899999997 15039.48900 2020-01-01 2020-01-02 2020-01-01 00:10:31 2020-01-02 03:38:52 2020-01-01 00:10:31.000 2020-01-02 03:38:52.000 631 99532 50081.5 5008150 631 99532 50081.5 5008150 -32539 32599 4861.66 486166 -126 125 2.78 278 -632 100 10622 99533 1.89789 298.89789 150.39789 15039.78978 1.89789 298.8979 150.39789 15039.78984 1.89789 298.89789 150.39789000000016 15039.78900 2020-01-01 2020-01-02 2020-01-01 00:10:32 2020-01-02 03:38:53 2020-01-01 00:10:32.000 2020-01-02 03:38:53.000 632 99533 50082.5 5008250 632 99533 50082.5 5008250 -32538 32600 4862.66 486266 -125 126 3.78 378 -633 100 10623 99534 1.9009 298.9009 150.4009 15040.09009 1.9009 298.9009 150.40089 15040.08996 1.90090 298.90090 150.40090000000004 15040.09000 2020-01-01 2020-01-02 2020-01-01 00:10:33 2020-01-02 03:38:54 2020-01-01 00:10:33.000 2020-01-02 03:38:54.000 633 99534 50083.5 5008350 633 99534 50083.5 5008350 -32537 32601 4863.66 486366 -124 127 4.78 478 -634 100 10624 99535 1.9039 298.9039 150.4039 15040.39039 1.9039 298.9039 150.4039 15040.39009 1.90390 298.90390 150.40389999999988 15040.39000 2020-01-01 2020-01-02 2020-01-01 00:10:34 2020-01-02 03:38:55 2020-01-01 00:10:34.000 2020-01-02 03:38:55.000 634 99535 50084.5 5008450 634 99535 50084.5 5008450 -32536 32602 4864.66 486466 -128 127 3.22 322 -635 100 10625 99536 1.9069 298.9069 150.4069 15040.69069 1.9069 298.90692 150.4069 15040.69084 1.90690 298.90690 150.40689999999998 15040.69000 2020-01-01 2020-01-02 2020-01-01 00:10:35 2020-01-02 03:38:56 2020-01-01 00:10:35.000 2020-01-02 03:38:56.000 635 99536 50085.5 5008550 635 99536 50085.5 5008550 -32535 32603 4865.66 486566 -128 127 1.66 166 -636 100 10626 99537 1.9099 298.9099 150.4099 15040.99099 1.90991 298.9099 150.40991 15040.99115 1.90990 298.90990 150.40990000000005 15040.99000 2020-01-01 2020-01-02 2020-01-01 00:10:36 2020-01-02 03:38:57 2020-01-01 00:10:36.000 2020-01-02 03:38:57.000 636 99537 50086.5 5008650 636 99537 50086.5 5008650 -32534 32604 4866.66 486666 -128 124 0.1 10 -637 100 10627 99538 1.91291 298.91291 150.41291 15041.29129 1.91291 298.9129 150.41291 15041.29131 1.91291 298.91291 150.41290999999973 15041.29100 2020-01-01 2020-01-02 2020-01-01 00:10:37 2020-01-02 03:38:58 2020-01-01 00:10:37.000 2020-01-02 03:38:58.000 637 99538 50087.5 5008750 637 99538 50087.5 5008750 -32533 32605 4867.66 486766 -127 125 1.1 110 -638 100 10628 99539 1.91591 298.91591 150.41591 15041.59159 1.91591 298.91592 150.41591 15041.59143 1.91591 298.91591 150.41590999999983 15041.59100 2020-01-01 2020-01-02 2020-01-01 00:10:38 2020-01-02 03:38:59 2020-01-01 00:10:38.000 2020-01-02 03:38:59.000 638 99539 50088.5 5008850 638 99539 50088.5 5008850 -32532 32606 4868.66 486866 -126 126 2.1 210 -639 100 10629 99540 1.91891 298.91891 150.41891 15041.89189 1.91891 298.9189 150.41891 15041.89172 1.91891 298.91891 150.4189100000002 15041.89100 2020-01-01 2020-01-02 2020-01-01 00:10:39 2020-01-02 03:39:00 2020-01-01 00:10:39.000 2020-01-02 03:39:00.000 639 99540 50089.5 5008950 639 99540 50089.5 5008950 -32531 32607 4869.66 486966 -125 127 3.1 310 -64 102 10054 99964 0.19219 300.19219 150.19219 15169.41141 0.19219 300.1922 150.19219 15169.41184 0.19219 300.19219 150.19218999999984 15169.41119 2020-01-01 2020-01-02 2020-01-01 00:01:04 2020-01-02 03:46:04 2020-01-01 00:01:04.000 2020-01-02 03:46:04.000 64 99964 50014 5051414 64 99964 50014 5051414 -32505 32430 4593.009900990099 463894 -125 126 0.21782178217821782 22 -640 100 10630 99541 1.92192 298.92192 150.42192 15042.19219 1.92192 298.92194 150.42192 15042.19232 1.92192 298.92192 150.42192000000009 15042.19200 2020-01-01 2020-01-02 2020-01-01 00:10:40 2020-01-02 03:39:01 2020-01-01 00:10:40.000 2020-01-02 03:39:01.000 640 99541 50090.5 5009050 640 99541 50090.5 5009050 -32530 32608 4870.66 487066 -128 127 1.54 154 -641 100 10631 99542 1.92492 298.92492 150.42492 15042.49249 1.92492 298.92493 150.42492 15042.49265 1.92492 298.92492 150.42492 15042.49200 2020-01-01 2020-01-02 2020-01-01 00:10:41 2020-01-02 03:39:02 2020-01-01 00:10:41.000 2020-01-02 03:39:02.000 641 99542 50091.5 5009150 641 99542 50091.5 5009150 -32529 32609 4871.66 487166 -128 127 -0.02 -2 -642 100 10632 99543 1.92792 298.92792 150.42792 15042.79279 1.92792 298.92792 150.42792 15042.79278 1.92792 298.92792 150.42792000000006 15042.79200 2020-01-01 2020-01-02 2020-01-01 00:10:42 2020-01-02 03:39:03 2020-01-01 00:10:42.000 2020-01-02 03:39:03.000 642 99543 50092.5 5009250 642 99543 50092.5 5009250 -32528 32610 4872.66 487266 -128 123 -1.58 -158 -643 100 10633 99544 1.93093 298.93093 150.43093 15043.09309 1.93093 298.93094 150.43092 15043.0929 1.93093 298.93093 150.43093000000033 15043.09300 2020-01-01 2020-01-02 2020-01-01 00:10:43 2020-01-02 03:39:04 2020-01-01 00:10:43.000 2020-01-02 03:39:04.000 643 99544 50093.5 5009350 643 99544 50093.5 5009350 -32527 32611 4873.66 487366 -127 124 -0.58 -58 -644 100 10634 99545 1.93393 298.93393 150.43393 15043.39339 1.93393 298.93393 150.43393 15043.39319 1.93393 298.93393 150.43392999999983 15043.39300 2020-01-01 2020-01-02 2020-01-01 00:10:44 2020-01-02 03:39:05 2020-01-01 00:10:44.000 2020-01-02 03:39:05.000 644 99545 50094.5 5009450 644 99545 50094.5 5009450 -32526 32612 4874.66 487466 -126 125 0.42 42 -645 100 10635 99546 1.93693 298.93693 150.43693 15043.69369 1.93693 298.93695 150.43693 15043.69379 1.93693 298.93693 150.43692999999988 15043.69300 2020-01-01 2020-01-02 2020-01-01 00:10:45 2020-01-02 03:39:06 2020-01-01 00:10:45.000 2020-01-02 03:39:06.000 645 99546 50095.5 5009550 645 99546 50095.5 5009550 -32525 32613 4875.66 487566 -125 126 1.42 142 -646 100 10636 99547 1.93993 298.93993 150.43993 15043.99399 1.93994 298.93994 150.43994 15043.99412 1.93993 298.93993 150.43993000000023 15043.99300 2020-01-01 2020-01-02 2020-01-01 00:10:46 2020-01-02 03:39:07 2020-01-01 00:10:46.000 2020-01-02 03:39:07.000 646 99547 50096.5 5009650 646 99547 50096.5 5009650 -32524 32614 4876.66 487666 -124 127 2.42 242 -647 100 10637 99548 1.94294 298.94294 150.44294 15044.29429 1.94294 298.94293 150.44294 15044.29425 1.94294 298.94294 150.44294000000022 15044.29400 2020-01-01 2020-01-02 2020-01-01 00:10:47 2020-01-02 03:39:08 2020-01-01 00:10:47.000 2020-01-02 03:39:08.000 647 99548 50097.5 5009750 647 99548 50097.5 5009750 -32523 32615 4877.66 487766 -128 127 0.86 86 -648 100 10638 99549 1.94594 298.94594 150.44594 15044.59459 1.94594 298.94595 150.44595 15044.595 1.94594 298.94594 150.44593999999972 15044.59400 2020-01-01 2020-01-02 2020-01-01 00:10:48 2020-01-02 03:39:09 2020-01-01 00:10:48.000 2020-01-02 03:39:09.000 648 99549 50098.5 5009850 648 99549 50098.5 5009850 -32522 32616 4878.66 487866 -128 123 -0.7 -70 -649 100 10639 99550 1.94894 298.94894 150.44894 15044.89489 1.94894 298.94894 150.44894 15044.89467 1.94894 298.94894 150.4489400000002 15044.89400 2020-01-01 2020-01-02 2020-01-01 00:10:49 2020-01-02 03:39:10 2020-01-01 00:10:49.000 2020-01-02 03:39:10.000 649 99550 50099.5 5009950 649 99550 50099.5 5009950 -32521 32617 4879.66 487966 -127 124 0.3 30 -65 102 10055 99965 0.19519 300.19519 150.19519 15169.71471 0.19519 300.1952 150.19519 15169.71448 0.19519 300.19519 150.19519000000025 15169.71419 2020-01-01 2020-01-02 2020-01-01 00:01:05 2020-01-02 03:46:05 2020-01-01 00:01:05.000 2020-01-02 03:46:05.000 65 99965 50015 5051515 65 99965 50015 5051515 -32504 32431 4594.009900990099 463995 -124 127 1.2178217821782178 123 -650 100 10640 99551 1.95195 298.95195 150.45195 15045.19519 1.95195 298.95197 150.45195 15045.19525 1.95195 298.95195 150.45195000000007 15045.19500 2020-01-01 2020-01-02 2020-01-01 00:10:50 2020-01-02 03:39:11 2020-01-01 00:10:50.000 2020-01-02 03:39:11.000 650 99551 50100.5 5010050 650 99551 50100.5 5010050 -32520 32618 4880.66 488066 -126 125 1.3 130 -651 100 10641 99552 1.95495 298.95495 150.45495 15045.49549 1.95495 298.95496 150.45495 15045.49558 1.95495 298.95495 150.45494999999988 15045.49500 2020-01-01 2020-01-02 2020-01-01 00:10:51 2020-01-02 03:39:12 2020-01-01 00:10:51.000 2020-01-02 03:39:12.000 651 99552 50101.5 5010150 651 99552 50101.5 5010150 -32519 32619 4881.66 488166 -125 126 2.3 230 -652 100 10642 99553 1.95795 298.95795 150.45795 15045.79579 1.95795 298.95795 150.45795 15045.79572 1.95795 298.95795 150.45794999999998 15045.79500 2020-01-01 2020-01-02 2020-01-01 00:10:52 2020-01-02 03:39:13 2020-01-01 00:10:52.000 2020-01-02 03:39:13.000 652 99553 50102.5 5010250 652 99553 50102.5 5010250 -32518 32620 4882.66 488266 -124 127 3.3 330 -653 100 10643 99554 1.96096 298.96096 150.46096 15046.09609 1.96096 298.96097 150.46096 15046.09647 1.96096 298.96096 150.46096000000023 15046.09600 2020-01-01 2020-01-02 2020-01-01 00:10:53 2020-01-02 03:39:14 2020-01-01 00:10:53.000 2020-01-02 03:39:14.000 653 99554 50103.5 5010350 653 99554 50103.5 5010350 -32517 32621 4883.66 488366 -128 127 1.74 174 -654 100 10644 99555 1.96396 298.96396 150.46396 15046.39639 1.96396 298.96396 150.46396 15046.39613 1.96396 298.96396 150.46395999999973 15046.39600 2020-01-01 2020-01-02 2020-01-01 00:10:54 2020-01-02 03:39:15 2020-01-01 00:10:54.000 2020-01-02 03:39:15.000 654 99555 50104.5 5010450 654 99555 50104.5 5010450 -32516 32622 4884.66 488466 -128 123 0.18 18 -655 100 10645 99556 1.96696 298.96696 150.46696 15046.69669 1.96696 298.96698 150.46696 15046.69676 1.96696 298.96696 150.46695999999983 15046.69600 2020-01-01 2020-01-02 2020-01-01 00:10:55 2020-01-02 03:39:16 2020-01-01 00:10:55.000 2020-01-02 03:39:16.000 655 99556 50105.5 5010550 655 99556 50105.5 5010550 -32515 32623 4885.66 488566 -127 124 1.18 118 -656 100 10646 99557 1.96996 298.96996 150.46996 15046.99699 1.96997 298.96997 150.46997 15046.99706 1.96996 298.96996 150.46996000000024 15046.99600 2020-01-01 2020-01-02 2020-01-01 00:10:56 2020-01-02 03:39:17 2020-01-01 00:10:56.000 2020-01-02 03:39:17.000 656 99557 50106.5 5010650 656 99557 50106.5 5010650 -32514 32624 4886.66 488666 -126 125 2.18 218 -657 100 10647 99558 1.97297 298.97297 150.47297 15047.29729 1.97297 298.97296 150.47297 15047.29735 1.97297 298.97297 150.47297000000012 15047.29700 2020-01-01 2020-01-02 2020-01-01 00:10:57 2020-01-02 03:39:18 2020-01-01 00:10:57.000 2020-01-02 03:39:18.000 657 99558 50107.5 5010750 657 99558 50107.5 5010750 -32513 32625 4887.66 488766 -125 126 3.18 318 -658 100 10648 99559 1.97597 298.97597 150.47597 15047.59759 1.97597 298.97598 150.47597 15047.59794 1.97597 298.97597 150.47597000000002 15047.59700 2020-01-01 2020-01-02 2020-01-01 00:10:58 2020-01-02 03:39:19 2020-01-01 00:10:58.000 2020-01-02 03:39:19.000 658 99559 50108.5 5010850 658 99559 50108.5 5010850 -32512 32626 4888.66 488866 -124 127 4.18 418 -659 100 10649 99560 1.97897 298.97897 150.47897 15047.89789 1.97897 298.97897 150.47897 15047.8976 1.97897 298.97897 150.4789700000001 15047.89700 2020-01-01 2020-01-02 2020-01-01 00:10:59 2020-01-02 03:39:20 2020-01-01 00:10:59.000 2020-01-02 03:39:20.000 659 99560 50109.5 5010950 659 99560 50109.5 5010950 -32511 32627 4889.66 488966 -128 127 2.62 262 -66 102 10056 99966 0.19819 300.19819 150.19819 15170.01801 0.19819 300.1982 150.19819 15170.01808 0.19819 300.19819 150.19818999999976 15170.01719 2020-01-01 2020-01-02 2020-01-01 00:01:06 2020-01-02 03:46:06 2020-01-01 00:01:06.000 2020-01-02 03:46:06.000 66 99966 50016 5051616 66 99966 50016 5051616 -32503 32432 4595.009900990099 464096 -128 127 -0.31683168316831684 -32 -660 100 10650 99561 1.98198 298.98198 150.48198 15048.19819 1.98198 298.982 150.48198 15048.19822 1.98198 298.98198 150.48197999999996 15048.19800 2020-01-01 2020-01-02 2020-01-01 00:11:00 2020-01-02 03:39:21 2020-01-01 00:11:00.000 2020-01-02 03:39:21.000 660 99561 50110.5 5011050 660 99561 50110.5 5011050 -32510 32628 4890.66 489066 -128 127 1.06 106 -661 100 10651 99562 1.98498 298.98498 150.48498 15048.49849 1.98498 298.985 150.48498 15048.49853 1.98498 298.98498 150.48497999999987 15048.49800 2020-01-01 2020-01-02 2020-01-01 00:11:01 2020-01-02 03:39:22 2020-01-01 00:11:01.000 2020-01-02 03:39:22.000 661 99562 50111.5 5011150 661 99562 50111.5 5011150 -32509 32629 4891.66 489166 -128 124 -0.5 -50 -662 100 10652 99563 1.98798 298.98798 150.48798 15048.79879 1.98798 298.98798 150.48798 15048.79882 1.98798 298.98798 150.4879799999999 15048.79800 2020-01-01 2020-01-02 2020-01-01 00:11:02 2020-01-02 03:39:23 2020-01-01 00:11:02.000 2020-01-02 03:39:23.000 662 99563 50112.5 5011250 662 99563 50112.5 5011250 -32508 32630 4892.66 489266 -127 125 0.5 50 -663 100 10653 99564 1.99099 298.99099 150.49099 15049.09909 1.99099 298.991 150.49099 15049.09942 1.99099 298.99099 150.49099000000015 15049.09900 2020-01-01 2020-01-02 2020-01-01 00:11:03 2020-01-02 03:39:24 2020-01-01 00:11:03.000 2020-01-02 03:39:24.000 663 99564 50113.5 5011350 663 99564 50113.5 5011350 -32507 32631 4893.66 489366 -126 126 1.5 150 -664 100 10654 99565 1.99399 298.99399 150.49399 15049.39939 1.99399 298.994 150.49399 15049.39911 1.99399 298.99399 150.49399000000025 15049.39900 2020-01-01 2020-01-02 2020-01-01 00:11:04 2020-01-02 03:39:25 2020-01-01 00:11:04.000 2020-01-02 03:39:25.000 664 99565 50114.5 5011450 664 99565 50114.5 5011450 -32506 32632 4894.66 489466 -125 127 2.5 250 -665 100 10655 99566 1.99699 298.99699 150.49699 15049.69969 1.99699 298.997 150.49699 15049.6997 1.99699 298.99699 150.49698999999976 15049.69900 2020-01-01 2020-01-02 2020-01-01 00:11:05 2020-01-02 03:39:26 2020-01-01 00:11:05.000 2020-01-02 03:39:26.000 665 99566 50115.5 5011550 665 99566 50115.5 5011550 -32505 32633 4895.66 489566 -128 127 0.94 94 +610 100 10600 99511 1.83183 298.83183 150.33183 15033.18318 1.83183 298.83182 150.33183 15033.18304 1.83183 298.83183 150.33183 15033.18300 2020-01-01 2020-01-02 2020-01-01 00:10:10 2020-01-02 03:38:31 2020-01-01 00:10:10.000 2020-01-02 03:38:31.000 610 99511 50060.5 5006050 610 99511 50060.5 5006050 -32560 32578 4840.66 484066 -128 127 -0.3 -30 +611 100 10601 99512 1.83483 298.83483 150.33483 15033.48348 1.83483 298.83484 150.33483 15033.48363 1.83483 298.83483 150.33483 15033.48300 2020-01-01 2020-01-02 2020-01-01 00:10:11 2020-01-02 03:38:32 2020-01-01 00:10:11.000 2020-01-02 03:38:32.000 611 99512 50061.5 5006150 611 99512 50061.5 5006150 -32559 32579 4841.66 484166 -128 123 -1.86 -186 +612 100 10602 99513 1.83783 298.83783 150.33783 15033.78378 1.83783 298.83783 150.33783 15033.78393 1.83783 298.83783 150.33783 15033.78300 2020-01-01 2020-01-02 2020-01-01 00:10:12 2020-01-02 03:38:33 2020-01-01 00:10:12.000 2020-01-02 03:38:33.000 612 99513 50062.5 5006250 612 99513 50062.5 5006250 -32558 32580 4842.66 484266 -127 124 -0.86 -86 +613 100 10603 99514 1.84084 298.84084 150.34084 15034.08408 1.84084 298.84085 150.34084 15034.08405 1.84084 298.84084 150.34084000000001 15034.08400 2020-01-01 2020-01-02 2020-01-01 00:10:13 2020-01-02 03:38:34 2020-01-01 00:10:13.000 2020-01-02 03:38:34.000 613 99514 50063.5 5006350 613 99514 50063.5 5006350 -32557 32581 4843.66 484366 -126 125 0.14 14 +614 100 10604 99515 1.84384 298.84384 150.34384 15034.38438 1.84384 298.84384 150.34384 15034.38421 1.84384 298.84384 150.34384 15034.38400 2020-01-01 2020-01-02 2020-01-01 00:10:14 2020-01-02 03:38:35 2020-01-01 00:10:14.000 2020-01-02 03:38:35.000 614 99515 50064.5 5006450 614 99515 50064.5 5006450 -32556 32582 4844.66 484466 -125 126 1.14 114 +615 100 10605 99516 1.84684 298.84684 150.34684 15034.68468 1.84684 298.84683 150.34684 15034.68452 1.84684 298.84684 150.34684 15034.68400 2020-01-01 2020-01-02 2020-01-01 00:10:15 2020-01-02 03:38:36 2020-01-01 00:10:15.000 2020-01-02 03:38:36.000 615 99516 50065.5 5006550 615 99516 50065.5 5006550 -32555 32583 4845.66 484566 -124 127 2.14 214 +616 100 10606 99517 1.84984 298.84984 150.34984 15034.98498 1.84984 298.84985 150.34985 15034.98527 1.84984 298.84984 150.34984 15034.98400 2020-01-01 2020-01-02 2020-01-01 00:10:16 2020-01-02 03:38:37 2020-01-01 00:10:16.000 2020-01-02 03:38:37.000 616 99517 50066.5 5006650 616 99517 50066.5 5006650 -32554 32584 4846.66 484666 -128 127 0.58 58 +617 100 10607 99518 1.85285 298.85285 150.35285 15035.28528 1.85285 298.85284 150.35285 15035.2854 1.85285 298.85285 150.35285 15035.28500 2020-01-01 2020-01-02 2020-01-01 00:10:17 2020-01-02 03:38:38 2020-01-01 00:10:17.000 2020-01-02 03:38:38.000 617 99518 50067.5 5006750 617 99518 50067.5 5006750 -32553 32585 4847.66 484766 -128 123 -0.98 -98 +618 100 10608 99519 1.85585 298.85585 150.35585 15035.58558 1.85585 298.85587 150.35585 15035.58551 1.85585 298.85585 150.35585 15035.58500 2020-01-01 2020-01-02 2020-01-01 00:10:18 2020-01-02 03:38:39 2020-01-01 00:10:18.000 2020-01-02 03:38:39.000 618 99519 50068.5 5006850 618 99519 50068.5 5006850 -32552 32586 4848.66 484866 -127 124 0.02 2 +619 100 10609 99520 1.85885 298.85885 150.35885 15035.88588 1.85885 298.85886 150.35885 15035.88568 1.85885 298.85885 150.35885 15035.88500 2020-01-01 2020-01-02 2020-01-01 00:10:19 2020-01-02 03:38:40 2020-01-01 00:10:19.000 2020-01-02 03:38:40.000 619 99520 50069.5 5006950 619 99520 50069.5 5006950 -32551 32587 4849.66 484966 -126 125 1.02 102 +62 102 10052 99962 0.18618 300.18618 150.18618 15168.8048 0.18618 300.1862 150.18618 15168.80494 0.18618 300.18618 150.18617999999998 15168.80418 2020-01-01 2020-01-02 2020-01-01 00:01:02 2020-01-02 03:46:02 2020-01-01 00:01:02.000 2020-01-02 03:46:02.000 62 99962 50012 5051212 62 99962 50012 5051212 -32507 32428 4591.009900990099 463692 -127 124 -1.7821782178217822 -180 +620 100 10610 99521 1.86186 298.86186 150.36186 15036.18618 1.86186 298.86185 150.36185 15036.18598 1.86186 298.86186 150.36186 15036.18600 2020-01-01 2020-01-02 2020-01-01 00:10:20 2020-01-02 03:38:41 2020-01-01 00:10:20.000 2020-01-02 03:38:41.000 620 99521 50070.5 5007050 620 99521 50070.5 5007050 -32550 32588 4850.66 485066 -125 126 2.02 202 +621 100 10611 99522 1.86486 298.86486 150.36486 15036.48648 1.86486 298.86487 150.36486 15036.48673 1.86486 298.86486 150.36486000000002 15036.48600 2020-01-01 2020-01-02 2020-01-01 00:10:21 2020-01-02 03:38:42 2020-01-01 00:10:21.000 2020-01-02 03:38:42.000 621 99522 50071.5 5007150 621 99522 50071.5 5007150 -32549 32589 4851.66 485166 -124 127 3.02 302 +622 100 10612 99523 1.86786 298.86786 150.36786 15036.78678 1.86786 298.86786 150.36786 15036.78687 1.86786 298.86786 150.36786 15036.78600 2020-01-01 2020-01-02 2020-01-01 00:10:22 2020-01-02 03:38:43 2020-01-01 00:10:22.000 2020-01-02 03:38:43.000 622 99523 50072.5 5007250 622 99523 50072.5 5007250 -32548 32590 4852.66 485266 -128 127 1.46 146 +623 100 10613 99524 1.87087 298.87087 150.37087 15037.08708 1.87087 298.87088 150.37087 15037.08702 1.87087 298.87087 150.37087 15037.08700 2020-01-01 2020-01-02 2020-01-01 00:10:23 2020-01-02 03:38:44 2020-01-01 00:10:23.000 2020-01-02 03:38:44.000 623 99524 50073.5 5007350 623 99524 50073.5 5007350 -32547 32591 4853.66 485366 -128 123 -0.1 -10 +624 100 10614 99525 1.87387 298.87387 150.37387 15037.38738 1.87387 298.87387 150.37387 15037.38716 1.87387 298.87387 150.37387 15037.38700 2020-01-01 2020-01-02 2020-01-01 00:10:24 2020-01-02 03:38:45 2020-01-01 00:10:24.000 2020-01-02 03:38:45.000 624 99525 50074.5 5007450 624 99525 50074.5 5007450 -32546 32592 4854.66 485466 -127 124 0.9 90 +625 100 10615 99526 1.87687 298.87687 150.37687 15037.68768 1.87687 298.8769 150.37687 15037.68791 1.87687 298.87687 150.37687 15037.68700 2020-01-01 2020-01-02 2020-01-01 00:10:25 2020-01-02 03:38:46 2020-01-01 00:10:25.000 2020-01-02 03:38:46.000 625 99526 50075.5 5007550 625 99526 50075.5 5007550 -32545 32593 4855.66 485566 -126 125 1.9 190 +626 100 10616 99527 1.87987 298.87987 150.37987 15037.98798 1.87987 298.87988 150.37988 15037.9882 1.87987 298.87987 150.37986999999998 15037.98700 2020-01-01 2020-01-02 2020-01-01 00:10:26 2020-01-02 03:38:47 2020-01-01 00:10:26.000 2020-01-02 03:38:47.000 626 99527 50076.5 5007650 626 99527 50076.5 5007650 -32544 32594 4856.66 485666 -125 126 2.9 290 +627 100 10617 99528 1.88288 298.88288 150.38288 15038.28828 1.88288 298.88287 150.38288 15038.28834 1.88288 298.88288 150.38288 15038.28800 2020-01-01 2020-01-02 2020-01-01 00:10:27 2020-01-02 03:38:48 2020-01-01 00:10:27.000 2020-01-02 03:38:48.000 627 99528 50077.5 5007750 627 99528 50077.5 5007750 -32543 32595 4857.66 485766 -124 127 3.9 390 +628 100 10618 99529 1.88588 298.88588 150.38588 15038.58858 1.88588 298.8859 150.38588 15038.58849 1.88588 298.88588 150.38588 15038.58800 2020-01-01 2020-01-02 2020-01-01 00:10:28 2020-01-02 03:38:49 2020-01-01 00:10:28.000 2020-01-02 03:38:49.000 628 99529 50078.5 5007850 628 99529 50078.5 5007850 -32542 32596 4858.66 485866 -128 127 2.34 234 +629 100 10619 99530 1.88888 298.88888 150.38888 15038.88888 1.88888 298.8889 150.38888 15038.88862 1.88888 298.88888 150.38888 15038.88800 2020-01-01 2020-01-02 2020-01-01 00:10:29 2020-01-02 03:38:50 2020-01-01 00:10:29.000 2020-01-02 03:38:50.000 629 99530 50079.5 5007950 629 99530 50079.5 5007950 -32541 32597 4859.66 485966 -128 123 0.78 78 +63 102 10053 99963 0.18918 300.18918 150.18918 15169.1081 0.18918 300.18918 150.18918 15169.10808 0.18918 300.18918 150.18918000000002 15169.10718 2020-01-01 2020-01-02 2020-01-01 00:01:03 2020-01-02 03:46:03 2020-01-01 00:01:03.000 2020-01-02 03:46:03.000 63 99963 50013 5051313 63 99963 50013 5051313 -32506 32429 4592.009900990099 463793 -126 125 -0.7821782178217822 -79 +630 100 10620 99531 1.89189 298.89189 150.39189 15039.18918 1.89189 298.8919 150.39189 15039.18937 1.89189 298.89189 150.39189 15039.18900 2020-01-01 2020-01-02 2020-01-01 00:10:30 2020-01-02 03:38:51 2020-01-01 00:10:30.000 2020-01-02 03:38:51.000 630 99531 50080.5 5008050 630 99531 50080.5 5008050 -32540 32598 4860.66 486066 -127 124 1.78 178 +631 100 10621 99532 1.89489 298.89489 150.39489 15039.48948 1.89489 298.8949 150.39489 15039.48968 1.89489 298.89489 150.39489 15039.48900 2020-01-01 2020-01-02 2020-01-01 00:10:31 2020-01-02 03:38:52 2020-01-01 00:10:31.000 2020-01-02 03:38:52.000 631 99532 50081.5 5008150 631 99532 50081.5 5008150 -32539 32599 4861.66 486166 -126 125 2.78 278 +632 100 10622 99533 1.89789 298.89789 150.39789 15039.78978 1.89789 298.8979 150.39789 15039.78984 1.89789 298.89789 150.39789000000002 15039.78900 2020-01-01 2020-01-02 2020-01-01 00:10:32 2020-01-02 03:38:53 2020-01-01 00:10:32.000 2020-01-02 03:38:53.000 632 99533 50082.5 5008250 632 99533 50082.5 5008250 -32538 32600 4862.66 486266 -125 126 3.78 378 +633 100 10623 99534 1.9009 298.9009 150.4009 15040.09009 1.9009 298.9009 150.40089 15040.08996 1.90090 298.90090 150.4009 15040.09000 2020-01-01 2020-01-02 2020-01-01 00:10:33 2020-01-02 03:38:54 2020-01-01 00:10:33.000 2020-01-02 03:38:54.000 633 99534 50083.5 5008350 633 99534 50083.5 5008350 -32537 32601 4863.66 486366 -124 127 4.78 478 +634 100 10624 99535 1.9039 298.9039 150.4039 15040.39039 1.9039 298.9039 150.4039 15040.39009 1.90390 298.90390 150.4039 15040.39000 2020-01-01 2020-01-02 2020-01-01 00:10:34 2020-01-02 03:38:55 2020-01-01 00:10:34.000 2020-01-02 03:38:55.000 634 99535 50084.5 5008450 634 99535 50084.5 5008450 -32536 32602 4864.66 486466 -128 127 3.22 322 +635 100 10625 99536 1.9069 298.9069 150.4069 15040.69069 1.9069 298.90692 150.4069 15040.69084 1.90690 298.90690 150.4069 15040.69000 2020-01-01 2020-01-02 2020-01-01 00:10:35 2020-01-02 03:38:56 2020-01-01 00:10:35.000 2020-01-02 03:38:56.000 635 99536 50085.5 5008550 635 99536 50085.5 5008550 -32535 32603 4865.66 486566 -128 127 1.66 166 +636 100 10626 99537 1.9099 298.9099 150.4099 15040.99099 1.90991 298.9099 150.40991 15040.99115 1.90990 298.90990 150.4099 15040.99000 2020-01-01 2020-01-02 2020-01-01 00:10:36 2020-01-02 03:38:57 2020-01-01 00:10:36.000 2020-01-02 03:38:57.000 636 99537 50086.5 5008650 636 99537 50086.5 5008650 -32534 32604 4866.66 486666 -128 124 0.1 10 +637 100 10627 99538 1.91291 298.91291 150.41291 15041.29129 1.91291 298.9129 150.41291 15041.29131 1.91291 298.91291 150.41290999999998 15041.29100 2020-01-01 2020-01-02 2020-01-01 00:10:37 2020-01-02 03:38:58 2020-01-01 00:10:37.000 2020-01-02 03:38:58.000 637 99538 50087.5 5008750 637 99538 50087.5 5008750 -32533 32605 4867.66 486766 -127 125 1.1 110 +638 100 10628 99539 1.91591 298.91591 150.41591 15041.59159 1.91591 298.91592 150.41591 15041.59143 1.91591 298.91591 150.41591 15041.59100 2020-01-01 2020-01-02 2020-01-01 00:10:38 2020-01-02 03:38:59 2020-01-01 00:10:38.000 2020-01-02 03:38:59.000 638 99539 50088.5 5008850 638 99539 50088.5 5008850 -32532 32606 4868.66 486866 -126 126 2.1 210 +639 100 10629 99540 1.91891 298.91891 150.41891 15041.89189 1.91891 298.9189 150.41891 15041.89172 1.91891 298.91891 150.41890999999998 15041.89100 2020-01-01 2020-01-02 2020-01-01 00:10:39 2020-01-02 03:39:00 2020-01-01 00:10:39.000 2020-01-02 03:39:00.000 639 99540 50089.5 5008950 639 99540 50089.5 5008950 -32531 32607 4869.66 486966 -125 127 3.1 310 +64 102 10054 99964 0.19219 300.19219 150.19219 15169.41141 0.19219 300.1922 150.19219 15169.41184 0.19219 300.19219 150.19219 15169.41119 2020-01-01 2020-01-02 2020-01-01 00:01:04 2020-01-02 03:46:04 2020-01-01 00:01:04.000 2020-01-02 03:46:04.000 64 99964 50014 5051414 64 99964 50014 5051414 -32505 32430 4593.009900990099 463894 -125 126 0.21782178217821782 22 +640 100 10630 99541 1.92192 298.92192 150.42192 15042.19219 1.92192 298.92194 150.42192 15042.19232 1.92192 298.92192 150.42192 15042.19200 2020-01-01 2020-01-02 2020-01-01 00:10:40 2020-01-02 03:39:01 2020-01-01 00:10:40.000 2020-01-02 03:39:01.000 640 99541 50090.5 5009050 640 99541 50090.5 5009050 -32530 32608 4870.66 487066 -128 127 1.54 154 +641 100 10631 99542 1.92492 298.92492 150.42492 15042.49249 1.92492 298.92493 150.42492 15042.49265 1.92492 298.92492 150.42492000000001 15042.49200 2020-01-01 2020-01-02 2020-01-01 00:10:41 2020-01-02 03:39:02 2020-01-01 00:10:41.000 2020-01-02 03:39:02.000 641 99542 50091.5 5009150 641 99542 50091.5 5009150 -32529 32609 4871.66 487166 -128 127 -0.02 -2 +642 100 10632 99543 1.92792 298.92792 150.42792 15042.79279 1.92792 298.92792 150.42792 15042.79278 1.92792 298.92792 150.42792 15042.79200 2020-01-01 2020-01-02 2020-01-01 00:10:42 2020-01-02 03:39:03 2020-01-01 00:10:42.000 2020-01-02 03:39:03.000 642 99543 50092.5 5009250 642 99543 50092.5 5009250 -32528 32610 4872.66 487266 -128 123 -1.58 -158 +643 100 10633 99544 1.93093 298.93093 150.43093 15043.09309 1.93093 298.93094 150.43092 15043.0929 1.93093 298.93093 150.43093000000002 15043.09300 2020-01-01 2020-01-02 2020-01-01 00:10:43 2020-01-02 03:39:04 2020-01-01 00:10:43.000 2020-01-02 03:39:04.000 643 99544 50093.5 5009350 643 99544 50093.5 5009350 -32527 32611 4873.66 487366 -127 124 -0.58 -58 +644 100 10634 99545 1.93393 298.93393 150.43393 15043.39339 1.93393 298.93393 150.43393 15043.39319 1.93393 298.93393 150.43393 15043.39300 2020-01-01 2020-01-02 2020-01-01 00:10:44 2020-01-02 03:39:05 2020-01-01 00:10:44.000 2020-01-02 03:39:05.000 644 99545 50094.5 5009450 644 99545 50094.5 5009450 -32526 32612 4874.66 487466 -126 125 0.42 42 +645 100 10635 99546 1.93693 298.93693 150.43693 15043.69369 1.93693 298.93695 150.43693 15043.69379 1.93693 298.93693 150.43693 15043.69300 2020-01-01 2020-01-02 2020-01-01 00:10:45 2020-01-02 03:39:06 2020-01-01 00:10:45.000 2020-01-02 03:39:06.000 645 99546 50095.5 5009550 645 99546 50095.5 5009550 -32525 32613 4875.66 487566 -125 126 1.42 142 +646 100 10636 99547 1.93993 298.93993 150.43993 15043.99399 1.93994 298.93994 150.43994 15043.99412 1.93993 298.93993 150.43993 15043.99300 2020-01-01 2020-01-02 2020-01-01 00:10:46 2020-01-02 03:39:07 2020-01-01 00:10:46.000 2020-01-02 03:39:07.000 646 99547 50096.5 5009650 646 99547 50096.5 5009650 -32524 32614 4876.66 487666 -124 127 2.42 242 +647 100 10637 99548 1.94294 298.94294 150.44294 15044.29429 1.94294 298.94293 150.44294 15044.29425 1.94294 298.94294 150.44294 15044.29400 2020-01-01 2020-01-02 2020-01-01 00:10:47 2020-01-02 03:39:08 2020-01-01 00:10:47.000 2020-01-02 03:39:08.000 647 99548 50097.5 5009750 647 99548 50097.5 5009750 -32523 32615 4877.66 487766 -128 127 0.86 86 +648 100 10638 99549 1.94594 298.94594 150.44594 15044.59459 1.94594 298.94595 150.44595 15044.595 1.94594 298.94594 150.44593999999998 15044.59400 2020-01-01 2020-01-02 2020-01-01 00:10:48 2020-01-02 03:39:09 2020-01-01 00:10:48.000 2020-01-02 03:39:09.000 648 99549 50098.5 5009850 648 99549 50098.5 5009850 -32522 32616 4878.66 487866 -128 123 -0.7 -70 +649 100 10639 99550 1.94894 298.94894 150.44894 15044.89489 1.94894 298.94894 150.44894 15044.89467 1.94894 298.94894 150.44894 15044.89400 2020-01-01 2020-01-02 2020-01-01 00:10:49 2020-01-02 03:39:10 2020-01-01 00:10:49.000 2020-01-02 03:39:10.000 649 99550 50099.5 5009950 649 99550 50099.5 5009950 -32521 32617 4879.66 487966 -127 124 0.3 30 +65 102 10055 99965 0.19519 300.19519 150.19519 15169.71471 0.19519 300.1952 150.19519 15169.71448 0.19519 300.19519 150.19519 15169.71419 2020-01-01 2020-01-02 2020-01-01 00:01:05 2020-01-02 03:46:05 2020-01-01 00:01:05.000 2020-01-02 03:46:05.000 65 99965 50015 5051515 65 99965 50015 5051515 -32504 32431 4594.009900990099 463995 -124 127 1.2178217821782178 123 +650 100 10640 99551 1.95195 298.95195 150.45195 15045.19519 1.95195 298.95197 150.45195 15045.19525 1.95195 298.95195 150.45195 15045.19500 2020-01-01 2020-01-02 2020-01-01 00:10:50 2020-01-02 03:39:11 2020-01-01 00:10:50.000 2020-01-02 03:39:11.000 650 99551 50100.5 5010050 650 99551 50100.5 5010050 -32520 32618 4880.66 488066 -126 125 1.3 130 +651 100 10641 99552 1.95495 298.95495 150.45495 15045.49549 1.95495 298.95496 150.45495 15045.49558 1.95495 298.95495 150.45495 15045.49500 2020-01-01 2020-01-02 2020-01-01 00:10:51 2020-01-02 03:39:12 2020-01-01 00:10:51.000 2020-01-02 03:39:12.000 651 99552 50101.5 5010150 651 99552 50101.5 5010150 -32519 32619 4881.66 488166 -125 126 2.3 230 +652 100 10642 99553 1.95795 298.95795 150.45795 15045.79579 1.95795 298.95795 150.45795 15045.79572 1.95795 298.95795 150.45795 15045.79500 2020-01-01 2020-01-02 2020-01-01 00:10:52 2020-01-02 03:39:13 2020-01-01 00:10:52.000 2020-01-02 03:39:13.000 652 99553 50102.5 5010250 652 99553 50102.5 5010250 -32518 32620 4882.66 488266 -124 127 3.3 330 +653 100 10643 99554 1.96096 298.96096 150.46096 15046.09609 1.96096 298.96097 150.46096 15046.09647 1.96096 298.96096 150.46096 15046.09600 2020-01-01 2020-01-02 2020-01-01 00:10:53 2020-01-02 03:39:14 2020-01-01 00:10:53.000 2020-01-02 03:39:14.000 653 99554 50103.5 5010350 653 99554 50103.5 5010350 -32517 32621 4883.66 488366 -128 127 1.74 174 +654 100 10644 99555 1.96396 298.96396 150.46396 15046.39639 1.96396 298.96396 150.46396 15046.39613 1.96396 298.96396 150.46396000000001 15046.39600 2020-01-01 2020-01-02 2020-01-01 00:10:54 2020-01-02 03:39:15 2020-01-01 00:10:54.000 2020-01-02 03:39:15.000 654 99555 50104.5 5010450 654 99555 50104.5 5010450 -32516 32622 4884.66 488466 -128 123 0.18 18 +655 100 10645 99556 1.96696 298.96696 150.46696 15046.69669 1.96696 298.96698 150.46696 15046.69676 1.96696 298.96696 150.46696 15046.69600 2020-01-01 2020-01-02 2020-01-01 00:10:55 2020-01-02 03:39:16 2020-01-01 00:10:55.000 2020-01-02 03:39:16.000 655 99556 50105.5 5010550 655 99556 50105.5 5010550 -32515 32623 4885.66 488566 -127 124 1.18 118 +656 100 10646 99557 1.96996 298.96996 150.46996 15046.99699 1.96997 298.96997 150.46997 15046.99706 1.96996 298.96996 150.46996 15046.99600 2020-01-01 2020-01-02 2020-01-01 00:10:56 2020-01-02 03:39:17 2020-01-01 00:10:56.000 2020-01-02 03:39:17.000 656 99557 50106.5 5010650 656 99557 50106.5 5010650 -32514 32624 4886.66 488666 -126 125 2.18 218 +657 100 10647 99558 1.97297 298.97297 150.47297 15047.29729 1.97297 298.97296 150.47297 15047.29735 1.97297 298.97297 150.47297 15047.29700 2020-01-01 2020-01-02 2020-01-01 00:10:57 2020-01-02 03:39:18 2020-01-01 00:10:57.000 2020-01-02 03:39:18.000 657 99558 50107.5 5010750 657 99558 50107.5 5010750 -32513 32625 4887.66 488766 -125 126 3.18 318 +658 100 10648 99559 1.97597 298.97597 150.47597 15047.59759 1.97597 298.97598 150.47597 15047.59794 1.97597 298.97597 150.47597 15047.59700 2020-01-01 2020-01-02 2020-01-01 00:10:58 2020-01-02 03:39:19 2020-01-01 00:10:58.000 2020-01-02 03:39:19.000 658 99559 50108.5 5010850 658 99559 50108.5 5010850 -32512 32626 4888.66 488866 -124 127 4.18 418 +659 100 10649 99560 1.97897 298.97897 150.47897 15047.89789 1.97897 298.97897 150.47897 15047.8976 1.97897 298.97897 150.47897 15047.89700 2020-01-01 2020-01-02 2020-01-01 00:10:59 2020-01-02 03:39:20 2020-01-01 00:10:59.000 2020-01-02 03:39:20.000 659 99560 50109.5 5010950 659 99560 50109.5 5010950 -32511 32627 4889.66 488966 -128 127 2.62 262 +66 102 10056 99966 0.19819 300.19819 150.19819 15170.01801 0.19819 300.1982 150.19819 15170.01808 0.19819 300.19819 150.19819 15170.01719 2020-01-01 2020-01-02 2020-01-01 00:01:06 2020-01-02 03:46:06 2020-01-01 00:01:06.000 2020-01-02 03:46:06.000 66 99966 50016 5051616 66 99966 50016 5051616 -32503 32432 4595.009900990099 464096 -128 127 -0.31683168316831684 -32 +660 100 10650 99561 1.98198 298.98198 150.48198 15048.19819 1.98198 298.982 150.48198 15048.19822 1.98198 298.98198 150.48198 15048.19800 2020-01-01 2020-01-02 2020-01-01 00:11:00 2020-01-02 03:39:21 2020-01-01 00:11:00.000 2020-01-02 03:39:21.000 660 99561 50110.5 5011050 660 99561 50110.5 5011050 -32510 32628 4890.66 489066 -128 127 1.06 106 +661 100 10651 99562 1.98498 298.98498 150.48498 15048.49849 1.98498 298.985 150.48498 15048.49853 1.98498 298.98498 150.48498 15048.49800 2020-01-01 2020-01-02 2020-01-01 00:11:01 2020-01-02 03:39:22 2020-01-01 00:11:01.000 2020-01-02 03:39:22.000 661 99562 50111.5 5011150 661 99562 50111.5 5011150 -32509 32629 4891.66 489166 -128 124 -0.5 -50 +662 100 10652 99563 1.98798 298.98798 150.48798 15048.79879 1.98798 298.98798 150.48798 15048.79882 1.98798 298.98798 150.48798 15048.79800 2020-01-01 2020-01-02 2020-01-01 00:11:02 2020-01-02 03:39:23 2020-01-01 00:11:02.000 2020-01-02 03:39:23.000 662 99563 50112.5 5011250 662 99563 50112.5 5011250 -32508 32630 4892.66 489266 -127 125 0.5 50 +663 100 10653 99564 1.99099 298.99099 150.49099 15049.09909 1.99099 298.991 150.49099 15049.09942 1.99099 298.99099 150.49099 15049.09900 2020-01-01 2020-01-02 2020-01-01 00:11:03 2020-01-02 03:39:24 2020-01-01 00:11:03.000 2020-01-02 03:39:24.000 663 99564 50113.5 5011350 663 99564 50113.5 5011350 -32507 32631 4893.66 489366 -126 126 1.5 150 +664 100 10654 99565 1.99399 298.99399 150.49399 15049.39939 1.99399 298.994 150.49399 15049.39911 1.99399 298.99399 150.49399 15049.39900 2020-01-01 2020-01-02 2020-01-01 00:11:04 2020-01-02 03:39:25 2020-01-01 00:11:04.000 2020-01-02 03:39:25.000 664 99565 50114.5 5011450 664 99565 50114.5 5011450 -32506 32632 4894.66 489466 -125 127 2.5 250 +665 100 10655 99566 1.99699 298.99699 150.49699 15049.69969 1.99699 298.997 150.49699 15049.6997 1.99699 298.99699 150.49699 15049.69900 2020-01-01 2020-01-02 2020-01-01 00:11:05 2020-01-02 03:39:26 2020-01-01 00:11:05.000 2020-01-02 03:39:26.000 665 99566 50115.5 5011550 665 99566 50115.5 5011550 -32505 32633 4895.66 489566 -128 127 0.94 94 666 100 10656 99567 2 299 150.5 15050 2 299 150.5 15050 2.00000 299.00000 150.5 15050.00000 2020-01-01 2020-01-02 2020-01-01 00:11:06 2020-01-02 03:39:27 2020-01-01 00:11:06.000 2020-01-02 03:39:27.000 666 99567 50116.5 5011650 666 99567 50116.5 5011650 -32504 32634 4896.66 489666 -128 127 -0.62 -62 -667 100 10657 99568 2.003 299.003 150.503 15050.3003 2.003 299.003 150.503 15050.30029 2.00300 299.00300 150.5030000000001 15050.30000 2020-01-01 2020-01-02 2020-01-01 00:11:07 2020-01-02 03:39:28 2020-01-01 00:11:07.000 2020-01-02 03:39:28.000 667 99568 50117.5 5011750 667 99568 50117.5 5011750 -32503 32635 4897.66 489766 -128 123 -2.18 -218 -668 100 10658 99569 2.006 299.006 150.506 15050.6006 2.006 299.006 150.506 15050.60089 2.00600 299.00600 150.50599999999991 15050.60000 2020-01-01 2020-01-02 2020-01-01 00:11:08 2020-01-02 03:39:29 2020-01-01 00:11:08.000 2020-01-02 03:39:29.000 668 99569 50118.5 5011850 668 99569 50118.5 5011850 -32502 32636 4898.66 489866 -127 124 -1.18 -118 -669 100 10659 99570 2.009 299.009 150.509 15050.9009 2.009 299.009 150.509 15050.90057 2.00900 299.00900 150.50900000000001 15050.90000 2020-01-01 2020-01-02 2020-01-01 00:11:09 2020-01-02 03:39:30 2020-01-01 00:11:09.000 2020-01-02 03:39:30.000 669 99570 50119.5 5011950 669 99570 50119.5 5011950 -32501 32637 4899.66 489966 -126 125 -0.18 -18 -67 102 10057 99967 0.2012 300.2012 150.2012 15170.32132 0.2012 300.2012 150.2012 15170.32142 0.20120 300.20120 150.20120000000003 15170.32120 2020-01-01 2020-01-02 2020-01-01 00:01:07 2020-01-02 03:46:07 2020-01-01 00:01:07.000 2020-01-02 03:46:07.000 67 99967 50017 5051717 67 99967 50017 5051717 -32502 32433 4596.009900990099 464197 -128 127 -1.8514851485148516 -187 -670 100 10660 99571 2.01201 299.01201 150.51201 15051.2012 2.01201 299.01202 150.51201 15051.20117 2.01201 299.01201 150.51201000000026 15051.20100 2020-01-01 2020-01-02 2020-01-01 00:11:10 2020-01-02 03:39:31 2020-01-01 00:11:10.000 2020-01-02 03:39:31.000 670 99571 50120.5 5012050 670 99571 50120.5 5012050 -32500 32638 4900.66 490066 -125 126 0.82 82 -671 100 10661 99572 2.01501 299.01501 150.51501 15051.5015 2.01501 299.015 150.51501 15051.50146 2.01501 299.01501 150.51500999999976 15051.50100 2020-01-01 2020-01-02 2020-01-01 00:11:11 2020-01-02 03:39:32 2020-01-01 00:11:11.000 2020-01-02 03:39:32.000 671 99572 50121.5 5012150 671 99572 50121.5 5012150 -32499 32639 4901.66 490166 -124 127 1.82 182 -672 100 10662 99573 2.01801 299.01801 150.51801 15051.8018 2.01801 299.018 150.51801 15051.80176 2.01801 299.01801 150.5180099999998 15051.80100 2020-01-01 2020-01-02 2020-01-01 00:11:12 2020-01-02 03:39:33 2020-01-01 00:11:12.000 2020-01-02 03:39:33.000 672 99573 50122.5 5012250 672 99573 50122.5 5012250 -32498 32640 4902.66 490266 -128 127 0.26 26 -673 100 10663 99574 2.02102 299.02102 150.52102 15052.1021 2.02102 299.02103 150.52102 15052.1024 2.02102 299.02102 150.5210200000001 15052.10200 2020-01-01 2020-01-02 2020-01-01 00:11:13 2020-01-02 03:39:34 2020-01-01 00:11:13.000 2020-01-02 03:39:34.000 673 99574 50123.5 5012350 673 99574 50123.5 5012350 -32497 32641 4903.66 490366 -128 123 -1.3 -130 -674 100 10664 99575 2.02402 299.02402 150.52402 15052.4024 2.02402 299.02402 150.52402 15052.40204 2.02402 299.02402 150.52402000000015 15052.40200 2020-01-01 2020-01-02 2020-01-01 00:11:14 2020-01-02 03:39:35 2020-01-01 00:11:14.000 2020-01-02 03:39:35.000 674 99575 50124.5 5012450 674 99575 50124.5 5012450 -32496 32642 4904.66 490466 -127 124 -0.3 -30 -675 100 10665 99576 2.02702 299.02702 150.52702 15052.7027 2.02702 299.02704 150.52702 15052.70264 2.02702 299.02702 150.52701999999977 15052.70200 2020-01-01 2020-01-02 2020-01-01 00:11:15 2020-01-02 03:39:36 2020-01-01 00:11:15.000 2020-01-02 03:39:36.000 675 99576 50125.5 5012550 675 99576 50125.5 5012550 -32495 32643 4905.66 490566 -126 125 0.7 70 -676 100 10666 99577 2.03003 299.03003 150.53003 15053.003 2.03003 299.03003 150.53002 15053.00293 2.03003 299.03003 150.53002999999993 15053.00300 2020-01-01 2020-01-02 2020-01-01 00:11:16 2020-01-02 03:39:37 2020-01-01 00:11:16.000 2020-01-02 03:39:37.000 676 99577 50126.5 5012650 676 99577 50126.5 5012650 -32494 32644 4906.66 490666 -125 126 1.7 170 +667 100 10657 99568 2.003 299.003 150.503 15050.3003 2.003 299.003 150.503 15050.30029 2.00300 299.00300 150.503 15050.30000 2020-01-01 2020-01-02 2020-01-01 00:11:07 2020-01-02 03:39:28 2020-01-01 00:11:07.000 2020-01-02 03:39:28.000 667 99568 50117.5 5011750 667 99568 50117.5 5011750 -32503 32635 4897.66 489766 -128 123 -2.18 -218 +668 100 10658 99569 2.006 299.006 150.506 15050.6006 2.006 299.006 150.506 15050.60089 2.00600 299.00600 150.506 15050.60000 2020-01-01 2020-01-02 2020-01-01 00:11:08 2020-01-02 03:39:29 2020-01-01 00:11:08.000 2020-01-02 03:39:29.000 668 99569 50118.5 5011850 668 99569 50118.5 5011850 -32502 32636 4898.66 489866 -127 124 -1.18 -118 +669 100 10659 99570 2.009 299.009 150.509 15050.9009 2.009 299.009 150.509 15050.90057 2.00900 299.00900 150.509 15050.90000 2020-01-01 2020-01-02 2020-01-01 00:11:09 2020-01-02 03:39:30 2020-01-01 00:11:09.000 2020-01-02 03:39:30.000 669 99570 50119.5 5011950 669 99570 50119.5 5011950 -32501 32637 4899.66 489966 -126 125 -0.18 -18 +67 102 10057 99967 0.2012 300.2012 150.2012 15170.32132 0.2012 300.2012 150.2012 15170.32142 0.20120 300.20120 150.2012 15170.32120 2020-01-01 2020-01-02 2020-01-01 00:01:07 2020-01-02 03:46:07 2020-01-01 00:01:07.000 2020-01-02 03:46:07.000 67 99967 50017 5051717 67 99967 50017 5051717 -32502 32433 4596.009900990099 464197 -128 127 -1.8514851485148516 -187 +670 100 10660 99571 2.01201 299.01201 150.51201 15051.2012 2.01201 299.01202 150.51201 15051.20117 2.01201 299.01201 150.51201 15051.20100 2020-01-01 2020-01-02 2020-01-01 00:11:10 2020-01-02 03:39:31 2020-01-01 00:11:10.000 2020-01-02 03:39:31.000 670 99571 50120.5 5012050 670 99571 50120.5 5012050 -32500 32638 4900.66 490066 -125 126 0.82 82 +671 100 10661 99572 2.01501 299.01501 150.51501 15051.5015 2.01501 299.015 150.51501 15051.50146 2.01501 299.01501 150.51501 15051.50100 2020-01-01 2020-01-02 2020-01-01 00:11:11 2020-01-02 03:39:32 2020-01-01 00:11:11.000 2020-01-02 03:39:32.000 671 99572 50121.5 5012150 671 99572 50121.5 5012150 -32499 32639 4901.66 490166 -124 127 1.82 182 +672 100 10662 99573 2.01801 299.01801 150.51801 15051.8018 2.01801 299.018 150.51801 15051.80176 2.01801 299.01801 150.51801 15051.80100 2020-01-01 2020-01-02 2020-01-01 00:11:12 2020-01-02 03:39:33 2020-01-01 00:11:12.000 2020-01-02 03:39:33.000 672 99573 50122.5 5012250 672 99573 50122.5 5012250 -32498 32640 4902.66 490266 -128 127 0.26 26 +673 100 10663 99574 2.02102 299.02102 150.52102 15052.1021 2.02102 299.02103 150.52102 15052.1024 2.02102 299.02102 150.52102000000002 15052.10200 2020-01-01 2020-01-02 2020-01-01 00:11:13 2020-01-02 03:39:34 2020-01-01 00:11:13.000 2020-01-02 03:39:34.000 673 99574 50123.5 5012350 673 99574 50123.5 5012350 -32497 32641 4903.66 490366 -128 123 -1.3 -130 +674 100 10664 99575 2.02402 299.02402 150.52402 15052.4024 2.02402 299.02402 150.52402 15052.40204 2.02402 299.02402 150.52402 15052.40200 2020-01-01 2020-01-02 2020-01-01 00:11:14 2020-01-02 03:39:35 2020-01-01 00:11:14.000 2020-01-02 03:39:35.000 674 99575 50124.5 5012450 674 99575 50124.5 5012450 -32496 32642 4904.66 490466 -127 124 -0.3 -30 +675 100 10665 99576 2.02702 299.02702 150.52702 15052.7027 2.02702 299.02704 150.52702 15052.70264 2.02702 299.02702 150.52702 15052.70200 2020-01-01 2020-01-02 2020-01-01 00:11:15 2020-01-02 03:39:36 2020-01-01 00:11:15.000 2020-01-02 03:39:36.000 675 99576 50125.5 5012550 675 99576 50125.5 5012550 -32495 32643 4905.66 490566 -126 125 0.7 70 +676 100 10666 99577 2.03003 299.03003 150.53003 15053.003 2.03003 299.03003 150.53002 15053.00293 2.03003 299.03003 150.53003 15053.00300 2020-01-01 2020-01-02 2020-01-01 00:11:16 2020-01-02 03:39:37 2020-01-01 00:11:16.000 2020-01-02 03:39:37.000 676 99577 50126.5 5012650 676 99577 50126.5 5012650 -32494 32644 4906.66 490666 -125 126 1.7 170 677 100 10667 99578 2.03303 299.03303 150.53303 15053.3033 2.03303 299.03302 150.53303 15053.30323 2.03303 299.03303 150.53303 15053.30300 2020-01-01 2020-01-02 2020-01-01 00:11:17 2020-01-02 03:39:38 2020-01-01 00:11:17.000 2020-01-02 03:39:38.000 677 99578 50127.5 5012750 677 99578 50127.5 5012750 -32493 32645 4907.66 490766 -124 127 2.7 270 -678 100 10668 99579 2.03603 299.03603 150.53603 15053.6036 2.03603 299.03604 150.53603 15053.60387 2.03603 299.03603 150.5360299999999 15053.60300 2020-01-01 2020-01-02 2020-01-01 00:11:18 2020-01-02 03:39:39 2020-01-01 00:11:18.000 2020-01-02 03:39:39.000 678 99579 50128.5 5012850 678 99579 50128.5 5012850 -32492 32646 4908.66 490866 -128 127 1.14 114 -679 100 10669 99580 2.03903 299.03903 150.53903 15053.9039 2.03903 299.03903 150.53903 15053.90351 2.03903 299.03903 150.53902999999994 15053.90300 2020-01-01 2020-01-02 2020-01-01 00:11:19 2020-01-02 03:39:40 2020-01-01 00:11:19.000 2020-01-02 03:39:40.000 679 99580 50129.5 5012950 679 99580 50129.5 5012950 -32491 32647 4909.66 490966 -128 123 -0.42 -42 -68 102 10058 99968 0.2042 300.2042 150.2042 15170.62462 0.2042 300.2042 150.2042 15170.62457 0.20420 300.20420 150.2042000000001 15170.62420 2020-01-01 2020-01-02 2020-01-01 00:01:08 2020-01-02 03:46:08 2020-01-01 00:01:08.000 2020-01-02 03:46:08.000 68 99968 50018 5051818 68 99968 50018 5051818 -32501 32434 4597.009900990099 464298 -128 124 -3.386138613861386 -342 -680 100 10670 99581 2.04204 299.04204 150.54204 15054.2042 2.04204 299.04205 150.54204 15054.20426 2.04204 299.04204 150.54204000000016 15054.20400 2020-01-01 2020-01-02 2020-01-01 00:11:20 2020-01-02 03:39:41 2020-01-01 00:11:20.000 2020-01-02 03:39:41.000 680 99581 50130.5 5013050 680 99581 50130.5 5013050 -32490 32648 4910.66 491066 -127 124 0.58 58 -681 100 10671 99582 2.04504 299.04504 150.54504 15054.5045 2.04504 299.04504 150.54504 15054.5044 2.04504 299.04504 150.54504000000026 15054.50400 2020-01-01 2020-01-02 2020-01-01 00:11:21 2020-01-02 03:39:42 2020-01-01 00:11:21.000 2020-01-02 03:39:42.000 681 99582 50131.5 5013150 681 99582 50131.5 5013150 -32489 32649 4911.66 491166 -126 125 1.58 158 -682 100 10672 99583 2.04804 299.04804 150.54804 15054.8048 2.04804 299.04803 150.54804 15054.80474 2.04804 299.04804 150.5480399999998 15054.80400 2020-01-01 2020-01-02 2020-01-01 00:11:22 2020-01-02 03:39:43 2020-01-01 00:11:22.000 2020-01-02 03:39:43.000 682 99583 50132.5 5013250 682 99583 50132.5 5013250 -32488 32650 4912.66 491266 -125 126 2.58 258 +678 100 10668 99579 2.03603 299.03603 150.53603 15053.6036 2.03603 299.03604 150.53603 15053.60387 2.03603 299.03603 150.53602999999998 15053.60300 2020-01-01 2020-01-02 2020-01-01 00:11:18 2020-01-02 03:39:39 2020-01-01 00:11:18.000 2020-01-02 03:39:39.000 678 99579 50128.5 5012850 678 99579 50128.5 5012850 -32492 32646 4908.66 490866 -128 127 1.14 114 +679 100 10669 99580 2.03903 299.03903 150.53903 15053.9039 2.03903 299.03903 150.53903 15053.90351 2.03903 299.03903 150.53903 15053.90300 2020-01-01 2020-01-02 2020-01-01 00:11:19 2020-01-02 03:39:40 2020-01-01 00:11:19.000 2020-01-02 03:39:40.000 679 99580 50129.5 5012950 679 99580 50129.5 5012950 -32491 32647 4909.66 490966 -128 123 -0.42 -42 +68 102 10058 99968 0.2042 300.2042 150.2042 15170.62462 0.2042 300.2042 150.2042 15170.62457 0.20420 300.20420 150.20420000000001 15170.62420 2020-01-01 2020-01-02 2020-01-01 00:01:08 2020-01-02 03:46:08 2020-01-01 00:01:08.000 2020-01-02 03:46:08.000 68 99968 50018 5051818 68 99968 50018 5051818 -32501 32434 4597.009900990099 464298 -128 124 -3.386138613861386 -342 +680 100 10670 99581 2.04204 299.04204 150.54204 15054.2042 2.04204 299.04205 150.54204 15054.20426 2.04204 299.04204 150.54204 15054.20400 2020-01-01 2020-01-02 2020-01-01 00:11:20 2020-01-02 03:39:41 2020-01-01 00:11:20.000 2020-01-02 03:39:41.000 680 99581 50130.5 5013050 680 99581 50130.5 5013050 -32490 32648 4910.66 491066 -127 124 0.58 58 +681 100 10671 99582 2.04504 299.04504 150.54504 15054.5045 2.04504 299.04504 150.54504 15054.5044 2.04504 299.04504 150.54504 15054.50400 2020-01-01 2020-01-02 2020-01-01 00:11:21 2020-01-02 03:39:42 2020-01-01 00:11:21.000 2020-01-02 03:39:42.000 681 99582 50131.5 5013150 681 99582 50131.5 5013150 -32489 32649 4911.66 491166 -126 125 1.58 158 +682 100 10672 99583 2.04804 299.04804 150.54804 15054.8048 2.04804 299.04803 150.54804 15054.80474 2.04804 299.04804 150.54804000000001 15054.80400 2020-01-01 2020-01-02 2020-01-01 00:11:22 2020-01-02 03:39:43 2020-01-01 00:11:22.000 2020-01-02 03:39:43.000 682 99583 50132.5 5013250 682 99583 50132.5 5013250 -32488 32650 4912.66 491266 -125 126 2.58 258 683 100 10673 99584 2.05105 299.05105 150.55105 15055.1051 2.05105 299.05106 150.55105 15055.10533 2.05105 299.05105 150.55105 15055.10500 2020-01-01 2020-01-02 2020-01-01 00:11:23 2020-01-02 03:39:44 2020-01-01 00:11:23.000 2020-01-02 03:39:44.000 683 99584 50133.5 5013350 683 99584 50133.5 5013350 -32487 32651 4913.66 491366 -124 127 3.58 358 -684 100 10674 99585 2.05405 299.05405 150.55405 15055.4054 2.05405 299.05405 150.55404 15055.40498 2.05405 299.05405 150.55405000000013 15055.40500 2020-01-01 2020-01-02 2020-01-01 00:11:24 2020-01-02 03:39:45 2020-01-01 00:11:24.000 2020-01-02 03:39:45.000 684 99585 50134.5 5013450 684 99585 50134.5 5013450 -32486 32652 4914.66 491466 -128 127 2.02 202 -685 100 10675 99586 2.05705 299.05705 150.55705 15055.7057 2.05705 299.05707 150.55705 15055.70573 2.05705 299.05705 150.55704999999995 15055.70500 2020-01-01 2020-01-02 2020-01-01 00:11:25 2020-01-02 03:39:46 2020-01-01 00:11:25.000 2020-01-02 03:39:46.000 685 99586 50135.5 5013550 685 99586 50135.5 5013550 -32485 32653 4915.66 491566 -128 127 0.46 46 -686 100 10676 99587 2.06006 299.06006 150.56006 15056.006 2.06006 299.06006 150.56005 15056.00587 2.06006 299.06006 150.56005999999982 15056.00600 2020-01-01 2020-01-02 2020-01-01 00:11:26 2020-01-02 03:39:47 2020-01-01 00:11:26.000 2020-01-02 03:39:47.000 686 99587 50136.5 5013650 686 99587 50136.5 5013650 -32484 32654 4916.66 491666 -128 124 -1.1 -110 -687 100 10677 99588 2.06306 299.06306 150.56306 15056.3063 2.06306 299.06305 150.56306 15056.30621 2.06306 299.06306 150.5630600000003 15056.30600 2020-01-01 2020-01-02 2020-01-01 00:11:27 2020-01-02 03:39:48 2020-01-01 00:11:27.000 2020-01-02 03:39:48.000 687 99588 50137.5 5013750 687 99588 50137.5 5013750 -32483 32655 4917.66 491766 -127 125 -0.1 -10 -688 100 10678 99589 2.06606 299.06606 150.56606 15056.6066 2.06606 299.06607 150.56606 15056.60681 2.06606 299.06606 150.5660599999998 15056.60600 2020-01-01 2020-01-02 2020-01-01 00:11:28 2020-01-02 03:39:49 2020-01-01 00:11:28.000 2020-01-02 03:39:49.000 688 99589 50138.5 5013850 688 99589 50138.5 5013850 -32482 32656 4918.66 491866 -126 126 0.9 90 -689 100 10679 99590 2.06906 299.06906 150.56906 15056.9069 2.06906 299.06906 150.56907 15056.9071 2.06906 299.06906 150.56905999999984 15056.90600 2020-01-01 2020-01-02 2020-01-01 00:11:29 2020-01-02 03:39:50 2020-01-01 00:11:29.000 2020-01-02 03:39:50.000 689 99590 50139.5 5013950 689 99590 50139.5 5013950 -32481 32657 4919.66 491966 -125 127 1.9 190 -69 102 10059 99969 0.2072 300.2072 150.2072 15170.92792 0.2072 300.2072 150.20721 15170.92832 0.20720 300.20720 150.20720000000014 15170.92720 2020-01-01 2020-01-02 2020-01-01 00:01:09 2020-01-02 03:46:09 2020-01-01 00:01:09.000 2020-01-02 03:46:09.000 69 99969 50019 5051919 69 99969 50019 5051919 -32500 32435 4598.009900990099 464399 -127 125 -2.386138613861386 -241 -690 100 10680 99591 2.07207 299.07207 150.57207 15057.2072 2.07207 299.07208 150.57207 15057.2072 2.07207 299.07207 150.57207000000014 15057.20700 2020-01-01 2020-01-02 2020-01-01 00:11:30 2020-01-02 03:39:51 2020-01-01 00:11:30.000 2020-01-02 03:39:51.000 690 99591 50140.5 5014050 690 99591 50140.5 5014050 -32480 32658 4920.66 492066 -128 127 0.34 34 -691 100 10681 99592 2.07507 299.07507 150.57507 15057.5075 2.07507 299.07507 150.57507 15057.50734 2.07507 299.07507 150.57507000000018 15057.50700 2020-01-01 2020-01-02 2020-01-01 00:11:31 2020-01-02 03:39:52 2020-01-01 00:11:31.000 2020-01-02 03:39:52.000 691 99592 50141.5 5014150 691 99592 50141.5 5014150 -32479 32659 4921.66 492166 -128 127 -1.22 -122 -692 100 10682 99593 2.07807 299.07807 150.57807 15057.8078 2.07807 299.07806 150.57807 15057.80767 2.07807 299.07807 150.57806999999968 15057.80700 2020-01-01 2020-01-02 2020-01-01 00:11:32 2020-01-02 03:39:53 2020-01-01 00:11:32.000 2020-01-02 03:39:53.000 692 99593 50142.5 5014250 692 99593 50142.5 5014250 -32478 32660 4922.66 492266 -128 123 -2.78 -278 -693 100 10683 99594 2.08108 299.08108 150.58108 15058.1081 2.08108 299.0811 150.58108 15058.10827 2.08108 299.08108 150.58107999999996 15058.10800 2020-01-01 2020-01-02 2020-01-01 00:11:33 2020-01-02 03:39:54 2020-01-01 00:11:33.000 2020-01-02 03:39:54.000 693 99594 50143.5 5014350 693 99594 50143.5 5014350 -32477 32661 4923.66 492366 -127 124 -1.78 -178 -694 100 10684 99595 2.08408 299.08408 150.58408 15058.4084 2.08408 299.08408 150.58408 15058.40857 2.08408 299.08408 150.58407999999986 15058.40800 2020-01-01 2020-01-02 2020-01-01 00:11:34 2020-01-02 03:39:55 2020-01-01 00:11:34.000 2020-01-02 03:39:55.000 694 99595 50144.5 5014450 694 99595 50144.5 5014450 -32476 32662 4924.66 492466 -126 125 -0.78 -78 -695 100 10685 99596 2.08708 299.08708 150.58708 15058.7087 2.08708 299.0871 150.58708 15058.70867 2.08708 299.08708 150.58707999999996 15058.70800 2020-01-01 2020-01-02 2020-01-01 00:11:35 2020-01-02 03:39:56 2020-01-01 00:11:35.000 2020-01-02 03:39:56.000 695 99596 50145.5 5014550 695 99596 50145.5 5014550 -32475 32663 4925.66 492566 -125 126 0.22 22 -696 100 10686 99597 2.09009 299.09009 150.59009 15059.009 2.09009 299.0901 150.59008 15059.00885 2.09009 299.09009 150.59008999999992 15059.00900 2020-01-01 2020-01-02 2020-01-01 00:11:36 2020-01-02 03:39:57 2020-01-01 00:11:36.000 2020-01-02 03:39:57.000 696 99597 50146.5 5014650 696 99597 50146.5 5014650 -32474 32664 4926.66 492666 -124 127 1.22 122 -697 100 10687 99598 2.09309 299.09309 150.59309 15059.3093 2.09309 299.09308 150.59309 15059.30915 2.09309 299.09309 150.59309000000007 15059.30900 2020-01-01 2020-01-02 2020-01-01 00:11:37 2020-01-02 03:39:58 2020-01-01 00:11:37.000 2020-01-02 03:39:58.000 697 99598 50147.5 5014750 697 99598 50147.5 5014750 -32473 32665 4927.66 492766 -128 127 -0.34 -34 -698 100 10688 99599 2.09609 299.09609 150.59609 15059.6096 2.09609 299.0961 150.59609 15059.6099 2.09609 299.09609 150.59608999999992 15059.60900 2020-01-01 2020-01-02 2020-01-01 00:11:38 2020-01-02 03:39:59 2020-01-01 00:11:38.000 2020-01-02 03:39:59.000 698 99599 50148.5 5014850 698 99599 50148.5 5014850 -32472 32666 4928.66 492866 -128 123 -1.9 -190 -699 100 10689 99600 2.09909 299.09909 150.59909 15059.9099 2.09909 299.0991 150.5991 15059.91003 2.09909 299.09909 150.59908999999993 15059.90900 2020-01-01 2020-01-02 2020-01-01 00:11:39 2020-01-02 03:40:00 2020-01-01 00:11:39.000 2020-01-02 03:40:00.000 699 99600 50149.5 5014950 699 99600 50149.5 5014950 -32471 32667 4929.66 492966 -127 124 -0.9 -90 -7 102 1006 9997 0.02102 300.02102 150.02102 15152.12312 0.02102 300.02103 150.02102 15152.12342 0.02102 300.02102 150.02102000000005 15152.12302 2020-01-01 2020-01-02 2020-01-01 00:00:07 2020-01-02 03:45:07 2020-01-01 00:00:07.000 2020-01-02 03:45:07.000 7 99907 49957 5045657 7 99907 49957 5045657 -32562 32373 4536.009900990099 458137 -126 125 -1.0198019801980198 -103 +684 100 10674 99585 2.05405 299.05405 150.55405 15055.4054 2.05405 299.05405 150.55404 15055.40498 2.05405 299.05405 150.55405000000002 15055.40500 2020-01-01 2020-01-02 2020-01-01 00:11:24 2020-01-02 03:39:45 2020-01-01 00:11:24.000 2020-01-02 03:39:45.000 684 99585 50134.5 5013450 684 99585 50134.5 5013450 -32486 32652 4914.66 491466 -128 127 2.02 202 +685 100 10675 99586 2.05705 299.05705 150.55705 15055.7057 2.05705 299.05707 150.55705 15055.70573 2.05705 299.05705 150.55705 15055.70500 2020-01-01 2020-01-02 2020-01-01 00:11:25 2020-01-02 03:39:46 2020-01-01 00:11:25.000 2020-01-02 03:39:46.000 685 99586 50135.5 5013550 685 99586 50135.5 5013550 -32485 32653 4915.66 491566 -128 127 0.46 46 +686 100 10676 99587 2.06006 299.06006 150.56006 15056.006 2.06006 299.06006 150.56005 15056.00587 2.06006 299.06006 150.56006 15056.00600 2020-01-01 2020-01-02 2020-01-01 00:11:26 2020-01-02 03:39:47 2020-01-01 00:11:26.000 2020-01-02 03:39:47.000 686 99587 50136.5 5013650 686 99587 50136.5 5013650 -32484 32654 4916.66 491666 -128 124 -1.1 -110 +687 100 10677 99588 2.06306 299.06306 150.56306 15056.3063 2.06306 299.06305 150.56306 15056.30621 2.06306 299.06306 150.56306 15056.30600 2020-01-01 2020-01-02 2020-01-01 00:11:27 2020-01-02 03:39:48 2020-01-01 00:11:27.000 2020-01-02 03:39:48.000 687 99588 50137.5 5013750 687 99588 50137.5 5013750 -32483 32655 4917.66 491766 -127 125 -0.1 -10 +688 100 10678 99589 2.06606 299.06606 150.56606 15056.6066 2.06606 299.06607 150.56606 15056.60681 2.06606 299.06606 150.56606 15056.60600 2020-01-01 2020-01-02 2020-01-01 00:11:28 2020-01-02 03:39:49 2020-01-01 00:11:28.000 2020-01-02 03:39:49.000 688 99589 50138.5 5013850 688 99589 50138.5 5013850 -32482 32656 4918.66 491866 -126 126 0.9 90 +689 100 10679 99590 2.06906 299.06906 150.56906 15056.9069 2.06906 299.06906 150.56907 15056.9071 2.06906 299.06906 150.56906 15056.90600 2020-01-01 2020-01-02 2020-01-01 00:11:29 2020-01-02 03:39:50 2020-01-01 00:11:29.000 2020-01-02 03:39:50.000 689 99590 50139.5 5013950 689 99590 50139.5 5013950 -32481 32657 4919.66 491966 -125 127 1.9 190 +69 102 10059 99969 0.2072 300.2072 150.2072 15170.92792 0.2072 300.2072 150.20721 15170.92832 0.20720 300.20720 150.2072 15170.92720 2020-01-01 2020-01-02 2020-01-01 00:01:09 2020-01-02 03:46:09 2020-01-01 00:01:09.000 2020-01-02 03:46:09.000 69 99969 50019 5051919 69 99969 50019 5051919 -32500 32435 4598.009900990099 464399 -127 125 -2.386138613861386 -241 +690 100 10680 99591 2.07207 299.07207 150.57207 15057.2072 2.07207 299.07208 150.57207 15057.2072 2.07207 299.07207 150.57207 15057.20700 2020-01-01 2020-01-02 2020-01-01 00:11:30 2020-01-02 03:39:51 2020-01-01 00:11:30.000 2020-01-02 03:39:51.000 690 99591 50140.5 5014050 690 99591 50140.5 5014050 -32480 32658 4920.66 492066 -128 127 0.34 34 +691 100 10681 99592 2.07507 299.07507 150.57507 15057.5075 2.07507 299.07507 150.57507 15057.50734 2.07507 299.07507 150.57506999999998 15057.50700 2020-01-01 2020-01-02 2020-01-01 00:11:31 2020-01-02 03:39:52 2020-01-01 00:11:31.000 2020-01-02 03:39:52.000 691 99592 50141.5 5014150 691 99592 50141.5 5014150 -32479 32659 4921.66 492166 -128 127 -1.22 -122 +692 100 10682 99593 2.07807 299.07807 150.57807 15057.8078 2.07807 299.07806 150.57807 15057.80767 2.07807 299.07807 150.57807 15057.80700 2020-01-01 2020-01-02 2020-01-01 00:11:32 2020-01-02 03:39:53 2020-01-01 00:11:32.000 2020-01-02 03:39:53.000 692 99593 50142.5 5014250 692 99593 50142.5 5014250 -32478 32660 4922.66 492266 -128 123 -2.78 -278 +693 100 10683 99594 2.08108 299.08108 150.58108 15058.1081 2.08108 299.0811 150.58108 15058.10827 2.08108 299.08108 150.58108000000001 15058.10800 2020-01-01 2020-01-02 2020-01-01 00:11:33 2020-01-02 03:39:54 2020-01-01 00:11:33.000 2020-01-02 03:39:54.000 693 99594 50143.5 5014350 693 99594 50143.5 5014350 -32477 32661 4923.66 492366 -127 124 -1.78 -178 +694 100 10684 99595 2.08408 299.08408 150.58408 15058.4084 2.08408 299.08408 150.58408 15058.40857 2.08408 299.08408 150.58408 15058.40800 2020-01-01 2020-01-02 2020-01-01 00:11:34 2020-01-02 03:39:55 2020-01-01 00:11:34.000 2020-01-02 03:39:55.000 694 99595 50144.5 5014450 694 99595 50144.5 5014450 -32476 32662 4924.66 492466 -126 125 -0.78 -78 +695 100 10685 99596 2.08708 299.08708 150.58708 15058.7087 2.08708 299.0871 150.58708 15058.70867 2.08708 299.08708 150.58708000000001 15058.70800 2020-01-01 2020-01-02 2020-01-01 00:11:35 2020-01-02 03:39:56 2020-01-01 00:11:35.000 2020-01-02 03:39:56.000 695 99596 50145.5 5014550 695 99596 50145.5 5014550 -32475 32663 4925.66 492566 -125 126 0.22 22 +696 100 10686 99597 2.09009 299.09009 150.59009 15059.009 2.09009 299.0901 150.59008 15059.00885 2.09009 299.09009 150.59009 15059.00900 2020-01-01 2020-01-02 2020-01-01 00:11:36 2020-01-02 03:39:57 2020-01-01 00:11:36.000 2020-01-02 03:39:57.000 696 99597 50146.5 5014650 696 99597 50146.5 5014650 -32474 32664 4926.66 492666 -124 127 1.22 122 +697 100 10687 99598 2.09309 299.09309 150.59309 15059.3093 2.09309 299.09308 150.59309 15059.30915 2.09309 299.09309 150.59309 15059.30900 2020-01-01 2020-01-02 2020-01-01 00:11:37 2020-01-02 03:39:58 2020-01-01 00:11:37.000 2020-01-02 03:39:58.000 697 99598 50147.5 5014750 697 99598 50147.5 5014750 -32473 32665 4927.66 492766 -128 127 -0.34 -34 +698 100 10688 99599 2.09609 299.09609 150.59609 15059.6096 2.09609 299.0961 150.59609 15059.6099 2.09609 299.09609 150.59609 15059.60900 2020-01-01 2020-01-02 2020-01-01 00:11:38 2020-01-02 03:39:59 2020-01-01 00:11:38.000 2020-01-02 03:39:59.000 698 99599 50148.5 5014850 698 99599 50148.5 5014850 -32472 32666 4928.66 492866 -128 123 -1.9 -190 +699 100 10689 99600 2.09909 299.09909 150.59909 15059.9099 2.09909 299.0991 150.5991 15059.91003 2.09909 299.09909 150.59909 15059.90900 2020-01-01 2020-01-02 2020-01-01 00:11:39 2020-01-02 03:40:00 2020-01-01 00:11:39.000 2020-01-02 03:40:00.000 699 99600 50149.5 5014950 699 99600 50149.5 5014950 -32471 32667 4929.66 492966 -127 124 -0.9 -90 +7 102 1006 9997 0.02102 300.02102 150.02102 15152.12312 0.02102 300.02103 150.02102 15152.12342 0.02102 300.02102 150.02102000000002 15152.12302 2020-01-01 2020-01-02 2020-01-01 00:00:07 2020-01-02 03:45:07 2020-01-01 00:00:07.000 2020-01-02 03:45:07.000 7 99907 49957 5045657 7 99907 49957 5045657 -32562 32373 4536.009900990099 458137 -126 125 -1.0198019801980198 -103 70 102 10060 99970 0.21021 300.21021 150.21021 15171.23123 0.21021 300.2102 150.2102 15171.23097 0.21021 300.21021 150.21021 15171.23121 2020-01-01 2020-01-02 2020-01-01 00:01:10 2020-01-02 03:46:10 2020-01-01 00:01:10.000 2020-01-02 03:46:10.000 70 99970 50020 5052020 70 99970 50020 5052020 -32499 32436 4599.009900990099 464500 -126 126 -1.386138613861386 -140 -700 100 10690 99601 2.1021 299.1021 150.6021 15060.21021 2.1021 299.1021 150.6021 15060.21014 2.10210 299.10210 150.60210000000004 15060.21000 2020-01-01 2020-01-02 2020-01-01 00:11:40 2020-01-02 03:40:01 2020-01-01 00:11:40.000 2020-01-02 03:40:01.000 700 99601 50150.5 5015050 700 99601 50150.5 5015050 -32470 32668 4930.66 493066 -126 125 0.1 10 +700 100 10690 99601 2.1021 299.1021 150.6021 15060.21021 2.1021 299.1021 150.6021 15060.21014 2.10210 299.10210 150.60209999999998 15060.21000 2020-01-01 2020-01-02 2020-01-01 00:11:40 2020-01-02 03:40:01 2020-01-01 00:11:40.000 2020-01-02 03:40:01.000 700 99601 50150.5 5015050 700 99601 50150.5 5015050 -32470 32668 4930.66 493066 -126 125 0.1 10 701 100 10691 99602 2.1051 299.1051 150.6051 15060.51051 2.1051 299.1051 150.6051 15060.51031 2.10510 299.10510 150.6051 15060.51000 2020-01-01 2020-01-02 2020-01-01 00:11:41 2020-01-02 03:40:02 2020-01-01 00:11:41.000 2020-01-02 03:40:02.000 701 99602 50151.5 5015150 701 99602 50151.5 5015150 -32469 32669 4931.66 493166 -125 126 1.1 110 702 100 10692 99603 2.1081 299.1081 150.6081 15060.81081 2.1081 299.1081 150.6081 15060.81062 2.10810 299.10810 150.6081 15060.81000 2020-01-01 2020-01-02 2020-01-01 00:11:42 2020-01-02 03:40:03 2020-01-01 00:11:42.000 2020-01-02 03:40:03.000 702 99603 50152.5 5015250 702 99603 50152.5 5015250 -32468 32670 4932.66 493266 -124 127 2.1 210 -703 100 10693 99604 2.11111 299.11111 150.61111 15061.11111 2.11111 299.1111 150.61111 15061.11137 2.11111 299.11111 150.6111099999999 15061.11100 2020-01-01 2020-01-02 2020-01-01 00:11:43 2020-01-02 03:40:04 2020-01-01 00:11:43.000 2020-01-02 03:40:04.000 703 99604 50153.5 5015350 703 99604 50153.5 5015350 -32467 32671 4933.66 493366 -128 127 0.54 54 -704 100 10694 99605 2.11411 299.11411 150.61411 15061.41141 2.11411 299.1141 150.61411 15061.41151 2.11411 299.11411 150.61411000000015 15061.41100 2020-01-01 2020-01-02 2020-01-01 00:11:44 2020-01-02 03:40:05 2020-01-01 00:11:44.000 2020-01-02 03:40:05.000 704 99605 50154.5 5015450 704 99605 50154.5 5015450 -32466 32672 4934.66 493466 -128 123 -1.02 -102 -705 100 10695 99606 2.11711 299.11711 150.61711 15061.71171 2.11711 299.11713 150.61711 15061.71165 2.11711 299.11711 150.61710999999997 15061.71100 2020-01-01 2020-01-02 2020-01-01 00:11:45 2020-01-02 03:40:06 2020-01-01 00:11:45.000 2020-01-02 03:40:06.000 705 99606 50155.5 5015550 705 99606 50155.5 5015550 -32465 32673 4935.66 493566 -127 124 -0.02 -2 -706 100 10696 99607 2.12012 299.12012 150.62012 15062.01201 2.12012 299.12012 150.62011 15062.01179 2.12012 299.12012 150.62011999999987 15062.01200 2020-01-01 2020-01-02 2020-01-01 00:11:46 2020-01-02 03:40:07 2020-01-01 00:11:46.000 2020-01-02 03:40:07.000 706 99607 50156.5 5015650 706 99607 50156.5 5015650 -32464 32674 4936.66 493666 -126 125 0.98 98 -707 100 10697 99608 2.12312 299.12312 150.62312 15062.31231 2.12312 299.1231 150.62312 15062.31208 2.12312 299.12312 150.62312000000009 15062.31200 2020-01-01 2020-01-02 2020-01-01 00:11:47 2020-01-02 03:40:08 2020-01-01 00:11:47.000 2020-01-02 03:40:08.000 707 99608 50157.5 5015750 707 99608 50157.5 5015750 -32463 32675 4937.66 493766 -125 126 1.98 198 -708 100 10698 99609 2.12612 299.12612 150.62612 15062.61261 2.12612 299.12613 150.62612 15062.61283 2.12612 299.12612 150.62612000000004 15062.61200 2020-01-01 2020-01-02 2020-01-01 00:11:48 2020-01-02 03:40:09 2020-01-01 00:11:48.000 2020-01-02 03:40:09.000 708 99609 50158.5 5015850 708 99609 50158.5 5015850 -32462 32676 4938.66 493866 -124 127 2.98 298 -709 100 10699 99610 2.12912 299.12912 150.62912 15062.91291 2.12912 299.12912 150.62912 15062.91298 2.12912 299.12912 150.6291199999999 15062.91200 2020-01-01 2020-01-02 2020-01-01 00:11:49 2020-01-02 03:40:10 2020-01-01 00:11:49.000 2020-01-02 03:40:10.000 709 99610 50159.5 5015950 709 99610 50159.5 5015950 -32461 32677 4939.66 493966 -128 127 1.42 142 -71 102 10061 99971 0.21321 300.21321 150.21321 15171.53453 0.21321 300.21323 150.21321 15171.5346 0.21321 300.21321 150.21320999999992 15171.53421 2020-01-01 2020-01-02 2020-01-01 00:01:11 2020-01-02 03:46:11 2020-01-01 00:01:11.000 2020-01-02 03:46:11.000 71 99971 50021 5052121 71 99971 50021 5052121 -32498 32437 4600.009900990099 464601 -125 127 -0.38613861386138615 -39 +703 100 10693 99604 2.11111 299.11111 150.61111 15061.11111 2.11111 299.1111 150.61111 15061.11137 2.11111 299.11111 150.61111 15061.11100 2020-01-01 2020-01-02 2020-01-01 00:11:43 2020-01-02 03:40:04 2020-01-01 00:11:43.000 2020-01-02 03:40:04.000 703 99604 50153.5 5015350 703 99604 50153.5 5015350 -32467 32671 4933.66 493366 -128 127 0.54 54 +704 100 10694 99605 2.11411 299.11411 150.61411 15061.41141 2.11411 299.1141 150.61411 15061.41151 2.11411 299.11411 150.61411 15061.41100 2020-01-01 2020-01-02 2020-01-01 00:11:44 2020-01-02 03:40:05 2020-01-01 00:11:44.000 2020-01-02 03:40:05.000 704 99605 50154.5 5015450 704 99605 50154.5 5015450 -32466 32672 4934.66 493466 -128 123 -1.02 -102 +705 100 10695 99606 2.11711 299.11711 150.61711 15061.71171 2.11711 299.11713 150.61711 15061.71165 2.11711 299.11711 150.61711 15061.71100 2020-01-01 2020-01-02 2020-01-01 00:11:45 2020-01-02 03:40:06 2020-01-01 00:11:45.000 2020-01-02 03:40:06.000 705 99606 50155.5 5015550 705 99606 50155.5 5015550 -32465 32673 4935.66 493566 -127 124 -0.02 -2 +706 100 10696 99607 2.12012 299.12012 150.62012 15062.01201 2.12012 299.12012 150.62011 15062.01179 2.12012 299.12012 150.62012000000001 15062.01200 2020-01-01 2020-01-02 2020-01-01 00:11:46 2020-01-02 03:40:07 2020-01-01 00:11:46.000 2020-01-02 03:40:07.000 706 99607 50156.5 5015650 706 99607 50156.5 5015650 -32464 32674 4936.66 493666 -126 125 0.98 98 +707 100 10697 99608 2.12312 299.12312 150.62312 15062.31231 2.12312 299.1231 150.62312 15062.31208 2.12312 299.12312 150.62312 15062.31200 2020-01-01 2020-01-02 2020-01-01 00:11:47 2020-01-02 03:40:08 2020-01-01 00:11:47.000 2020-01-02 03:40:08.000 707 99608 50157.5 5015750 707 99608 50157.5 5015750 -32463 32675 4937.66 493766 -125 126 1.98 198 +708 100 10698 99609 2.12612 299.12612 150.62612 15062.61261 2.12612 299.12613 150.62612 15062.61283 2.12612 299.12612 150.62612 15062.61200 2020-01-01 2020-01-02 2020-01-01 00:11:48 2020-01-02 03:40:09 2020-01-01 00:11:48.000 2020-01-02 03:40:09.000 708 99609 50158.5 5015850 708 99609 50158.5 5015850 -32462 32676 4938.66 493866 -124 127 2.98 298 +709 100 10699 99610 2.12912 299.12912 150.62912 15062.91291 2.12912 299.12912 150.62912 15062.91298 2.12912 299.12912 150.62912 15062.91200 2020-01-01 2020-01-02 2020-01-01 00:11:49 2020-01-02 03:40:10 2020-01-01 00:11:49.000 2020-01-02 03:40:10.000 709 99610 50159.5 5015950 709 99610 50159.5 5015950 -32461 32677 4939.66 493966 -128 127 1.42 142 +71 102 10061 99971 0.21321 300.21321 150.21321 15171.53453 0.21321 300.21323 150.21321 15171.5346 0.21321 300.21321 150.21321 15171.53421 2020-01-01 2020-01-02 2020-01-01 00:01:11 2020-01-02 03:46:11 2020-01-01 00:01:11.000 2020-01-02 03:46:11.000 71 99971 50021 5052121 71 99971 50021 5052121 -32498 32437 4600.009900990099 464601 -125 127 -0.38613861386138615 -39 710 100 10700 99611 2.13213 299.13213 150.63213 15063.21321 2.13213 299.13214 150.63213 15063.21311 2.13213 299.13213 150.63213 15063.21300 2020-01-01 2020-01-02 2020-01-01 00:11:50 2020-01-02 03:40:11 2020-01-01 00:11:50.000 2020-01-02 03:40:11.000 710 99611 50160.5 5016050 710 99611 50160.5 5016050 -32460 32678 4940.66 494066 -128 127 -0.14 -14 -711 100 10701 99612 2.13513 299.13513 150.63513 15063.51351 2.13513 299.13513 150.63513 15063.51325 2.13513 299.13513 150.63513000000003 15063.51300 2020-01-01 2020-01-02 2020-01-01 00:11:51 2020-01-02 03:40:12 2020-01-01 00:11:51.000 2020-01-02 03:40:12.000 711 99612 50161.5 5016150 711 99612 50161.5 5016150 -32459 32679 4941.66 494166 -128 124 -1.7 -170 -712 100 10702 99613 2.13813 299.13813 150.63813 15063.81381 2.13813 299.13815 150.63814 15063.81401 2.13813 299.13813 150.63812999999993 15063.81300 2020-01-01 2020-01-02 2020-01-01 00:11:52 2020-01-02 03:40:13 2020-01-01 00:11:52.000 2020-01-02 03:40:13.000 712 99613 50162.5 5016250 712 99613 50162.5 5016250 -32458 32680 4942.66 494266 -127 125 -0.7 -70 -713 100 10703 99614 2.14114 299.14114 150.64114 15064.11411 2.14114 299.14114 150.64114 15064.11431 2.14114 299.14114 150.64113999999995 15064.11400 2020-01-01 2020-01-02 2020-01-01 00:11:53 2020-01-02 03:40:14 2020-01-01 00:11:53.000 2020-01-02 03:40:14.000 713 99614 50163.5 5016350 713 99614 50163.5 5016350 -32457 32681 4943.66 494366 -126 126 0.3 30 -714 100 10704 99615 2.14414 299.14414 150.64414 15064.41441 2.14414 299.14413 150.64414 15064.41448 2.14414 299.14414 150.64414000000005 15064.41400 2020-01-01 2020-01-02 2020-01-01 00:11:54 2020-01-02 03:40:15 2020-01-01 00:11:54.000 2020-01-02 03:40:15.000 714 99615 50164.5 5016450 714 99615 50164.5 5016450 -32456 32682 4944.66 494466 -125 127 1.3 130 -715 100 10705 99616 2.14714 299.14714 150.64714 15064.71471 2.14714 299.14716 150.64714 15064.71458 2.14714 299.14714 150.64713999999992 15064.71400 2020-01-01 2020-01-02 2020-01-01 00:11:55 2020-01-02 03:40:16 2020-01-01 00:11:55.000 2020-01-02 03:40:16.000 715 99616 50165.5 5016550 715 99616 50165.5 5016550 -32455 32683 4945.66 494566 -128 127 -0.26 -26 +711 100 10701 99612 2.13513 299.13513 150.63513 15063.51351 2.13513 299.13513 150.63513 15063.51325 2.13513 299.13513 150.63513 15063.51300 2020-01-01 2020-01-02 2020-01-01 00:11:51 2020-01-02 03:40:12 2020-01-01 00:11:51.000 2020-01-02 03:40:12.000 711 99612 50161.5 5016150 711 99612 50161.5 5016150 -32459 32679 4941.66 494166 -128 124 -1.7 -170 +712 100 10702 99613 2.13813 299.13813 150.63813 15063.81381 2.13813 299.13815 150.63814 15063.81401 2.13813 299.13813 150.63813 15063.81300 2020-01-01 2020-01-02 2020-01-01 00:11:52 2020-01-02 03:40:13 2020-01-01 00:11:52.000 2020-01-02 03:40:13.000 712 99613 50162.5 5016250 712 99613 50162.5 5016250 -32458 32680 4942.66 494266 -127 125 -0.7 -70 +713 100 10703 99614 2.14114 299.14114 150.64114 15064.11411 2.14114 299.14114 150.64114 15064.11431 2.14114 299.14114 150.64114 15064.11400 2020-01-01 2020-01-02 2020-01-01 00:11:53 2020-01-02 03:40:14 2020-01-01 00:11:53.000 2020-01-02 03:40:14.000 713 99614 50163.5 5016350 713 99614 50163.5 5016350 -32457 32681 4943.66 494366 -126 126 0.3 30 +714 100 10704 99615 2.14414 299.14414 150.64414 15064.41441 2.14414 299.14413 150.64414 15064.41448 2.14414 299.14414 150.64414 15064.41400 2020-01-01 2020-01-02 2020-01-01 00:11:54 2020-01-02 03:40:15 2020-01-01 00:11:54.000 2020-01-02 03:40:15.000 714 99615 50164.5 5016450 714 99615 50164.5 5016450 -32456 32682 4944.66 494466 -125 127 1.3 130 +715 100 10705 99616 2.14714 299.14714 150.64714 15064.71471 2.14714 299.14716 150.64714 15064.71458 2.14714 299.14714 150.64714 15064.71400 2020-01-01 2020-01-02 2020-01-01 00:11:55 2020-01-02 03:40:16 2020-01-01 00:11:55.000 2020-01-02 03:40:16.000 715 99616 50165.5 5016550 715 99616 50165.5 5016550 -32455 32683 4945.66 494566 -128 127 -0.26 -26 716 100 10706 99617 2.15015 299.15015 150.65015 15065.01501 2.15015 299.15015 150.65014 15065.01472 2.15015 299.15015 150.65015 15065.01500 2020-01-01 2020-01-02 2020-01-01 00:11:56 2020-01-02 03:40:17 2020-01-01 00:11:56.000 2020-01-02 03:40:17.000 716 99617 50166.5 5016650 716 99617 50166.5 5016650 -32454 32684 4946.66 494666 -128 127 -1.82 -182 -717 100 10707 99618 2.15315 299.15315 150.65315 15065.31531 2.15315 299.15317 150.65315 15065.31547 2.15315 299.15315 150.65315000000007 15065.31500 2020-01-01 2020-01-02 2020-01-01 00:11:57 2020-01-02 03:40:18 2020-01-01 00:11:57.000 2020-01-02 03:40:18.000 717 99618 50167.5 5016750 717 99618 50167.5 5016750 -32453 32685 4947.66 494766 -128 123 -3.38 -338 -718 100 10708 99619 2.15615 299.15615 150.65615 15065.61561 2.15615 299.15616 150.65615 15065.61578 2.15615 299.15615 150.65615000000003 15065.61500 2020-01-01 2020-01-02 2020-01-01 00:11:58 2020-01-02 03:40:19 2020-01-01 00:11:58.000 2020-01-02 03:40:19.000 718 99619 50168.5 5016850 718 99619 50168.5 5016850 -32452 32686 4948.66 494866 -127 124 -2.38 -238 +717 100 10707 99618 2.15315 299.15315 150.65315 15065.31531 2.15315 299.15317 150.65315 15065.31547 2.15315 299.15315 150.65315 15065.31500 2020-01-01 2020-01-02 2020-01-01 00:11:57 2020-01-02 03:40:18 2020-01-01 00:11:57.000 2020-01-02 03:40:18.000 717 99618 50167.5 5016750 717 99618 50167.5 5016750 -32453 32685 4947.66 494766 -128 123 -3.38 -338 +718 100 10708 99619 2.15615 299.15615 150.65615 15065.61561 2.15615 299.15616 150.65615 15065.61578 2.15615 299.15615 150.65615 15065.61500 2020-01-01 2020-01-02 2020-01-01 00:11:58 2020-01-02 03:40:19 2020-01-01 00:11:58.000 2020-01-02 03:40:19.000 718 99619 50168.5 5016850 718 99619 50168.5 5016850 -32452 32686 4948.66 494866 -127 124 -2.38 -238 719 100 10709 99620 2.15915 299.15915 150.65915 15065.91591 2.15915 299.15915 150.65915 15065.91595 2.15915 299.15915 150.65915 15065.91500 2020-01-01 2020-01-02 2020-01-01 00:11:59 2020-01-02 03:40:20 2020-01-01 00:11:59.000 2020-01-02 03:40:20.000 719 99620 50169.5 5016950 719 99620 50169.5 5016950 -32451 32687 4949.66 494966 -126 125 -1.38 -138 72 102 10062 99972 0.21621 300.21621 150.21621 15171.83783 0.21621 300.21622 150.21621 15171.83791 0.21621 300.21621 150.21621 15171.83721 2020-01-01 2020-01-02 2020-01-01 00:01:12 2020-01-02 03:46:12 2020-01-01 00:01:12.000 2020-01-02 03:46:12.000 72 99972 50022 5052222 72 99972 50022 5052222 -32497 32438 4601.009900990099 464702 -128 127 -1.9207920792079207 -194 -720 100 10710 99621 2.16216 299.16216 150.66216 15066.21621 2.16216 299.16217 150.66216 15066.21606 2.16216 299.16216 150.66215999999994 15066.21600 2020-01-01 2020-01-02 2020-01-01 00:12:00 2020-01-02 03:40:21 2020-01-01 00:12:00.000 2020-01-02 03:40:21.000 720 99621 50170.5 5017050 720 99621 50170.5 5017050 -32450 32688 4950.66 495066 -125 126 -0.38 -38 -721 100 10711 99622 2.16516 299.16516 150.66516 15066.51651 2.16516 299.16516 150.66516 15066.51635 2.16516 299.16516 150.66515999999996 15066.51600 2020-01-01 2020-01-02 2020-01-01 00:12:01 2020-01-02 03:40:22 2020-01-01 00:12:01.000 2020-01-02 03:40:22.000 721 99622 50171.5 5017150 721 99622 50171.5 5017150 -32449 32689 4951.66 495166 -124 127 0.62 62 -722 100 10712 99623 2.16816 299.16816 150.66816 15066.81681 2.16816 299.16818 150.66816 15066.81695 2.16816 299.16816 150.66815999999997 15066.81600 2020-01-01 2020-01-02 2020-01-01 00:12:02 2020-01-02 03:40:23 2020-01-01 00:12:02.000 2020-01-02 03:40:23.000 722 99623 50172.5 5017250 722 99623 50172.5 5017250 -32448 32690 4952.66 495266 -128 127 -0.94 -94 -723 100 10713 99624 2.17117 299.17117 150.67117 15067.11711 2.17117 299.17117 150.67117 15067.11724 2.17117 299.17117 150.6711699999999 15067.11700 2020-01-01 2020-01-02 2020-01-01 00:12:03 2020-01-02 03:40:24 2020-01-01 00:12:03.000 2020-01-02 03:40:24.000 723 99624 50173.5 5017350 723 99624 50173.5 5017350 -32447 32691 4953.66 495366 -128 123 -2.5 -250 -724 100 10714 99625 2.17417 299.17417 150.67417 15067.41741 2.17417 299.17416 150.67417 15067.41742 2.17417 299.17417 150.67417000000012 15067.41700 2020-01-01 2020-01-02 2020-01-01 00:12:04 2020-01-02 03:40:25 2020-01-01 00:12:04.000 2020-01-02 03:40:25.000 724 99625 50174.5 5017450 724 99625 50174.5 5017450 -32446 32692 4954.66 495466 -127 124 -1.5 -150 -725 100 10715 99626 2.17717 299.17717 150.67717 15067.71771 2.17717 299.1772 150.67717 15067.71752 2.17717 299.17717 150.67717000000007 15067.71700 2020-01-01 2020-01-02 2020-01-01 00:12:05 2020-01-02 03:40:26 2020-01-01 00:12:05.000 2020-01-02 03:40:26.000 725 99626 50175.5 5017550 725 99626 50175.5 5017550 -32445 32693 4955.66 495566 -126 125 -0.5 -50 -726 100 10716 99627 2.18018 299.18018 150.68018 15068.01801 2.18018 299.18018 150.68017 15068.01782 2.18018 299.18018 150.68017999999995 15068.01800 2020-01-01 2020-01-02 2020-01-01 00:12:06 2020-01-02 03:40:27 2020-01-01 00:12:06.000 2020-01-02 03:40:27.000 726 99627 50176.5 5017650 726 99627 50176.5 5017650 -32444 32694 4956.66 495666 -125 126 0.5 50 +720 100 10710 99621 2.16216 299.16216 150.66216 15066.21621 2.16216 299.16217 150.66216 15066.21606 2.16216 299.16216 150.66216 15066.21600 2020-01-01 2020-01-02 2020-01-01 00:12:00 2020-01-02 03:40:21 2020-01-01 00:12:00.000 2020-01-02 03:40:21.000 720 99621 50170.5 5017050 720 99621 50170.5 5017050 -32450 32688 4950.66 495066 -125 126 -0.38 -38 +721 100 10711 99622 2.16516 299.16516 150.66516 15066.51651 2.16516 299.16516 150.66516 15066.51635 2.16516 299.16516 150.66516 15066.51600 2020-01-01 2020-01-02 2020-01-01 00:12:01 2020-01-02 03:40:22 2020-01-01 00:12:01.000 2020-01-02 03:40:22.000 721 99622 50171.5 5017150 721 99622 50171.5 5017150 -32449 32689 4951.66 495166 -124 127 0.62 62 +722 100 10712 99623 2.16816 299.16816 150.66816 15066.81681 2.16816 299.16818 150.66816 15066.81695 2.16816 299.16816 150.66816 15066.81600 2020-01-01 2020-01-02 2020-01-01 00:12:02 2020-01-02 03:40:23 2020-01-01 00:12:02.000 2020-01-02 03:40:23.000 722 99623 50172.5 5017250 722 99623 50172.5 5017250 -32448 32690 4952.66 495266 -128 127 -0.94 -94 +723 100 10713 99624 2.17117 299.17117 150.67117 15067.11711 2.17117 299.17117 150.67117 15067.11724 2.17117 299.17117 150.67117 15067.11700 2020-01-01 2020-01-02 2020-01-01 00:12:03 2020-01-02 03:40:24 2020-01-01 00:12:03.000 2020-01-02 03:40:24.000 723 99624 50173.5 5017350 723 99624 50173.5 5017350 -32447 32691 4953.66 495366 -128 123 -2.5 -250 +724 100 10714 99625 2.17417 299.17417 150.67417 15067.41741 2.17417 299.17416 150.67417 15067.41742 2.17417 299.17417 150.67417 15067.41700 2020-01-01 2020-01-02 2020-01-01 00:12:04 2020-01-02 03:40:25 2020-01-01 00:12:04.000 2020-01-02 03:40:25.000 724 99625 50174.5 5017450 724 99625 50174.5 5017450 -32446 32692 4954.66 495466 -127 124 -1.5 -150 +725 100 10715 99626 2.17717 299.17717 150.67717 15067.71771 2.17717 299.1772 150.67717 15067.71752 2.17717 299.17717 150.67717000000002 15067.71700 2020-01-01 2020-01-02 2020-01-01 00:12:05 2020-01-02 03:40:26 2020-01-01 00:12:05.000 2020-01-02 03:40:26.000 725 99626 50175.5 5017550 725 99626 50175.5 5017550 -32445 32693 4955.66 495566 -126 125 -0.5 -50 +726 100 10716 99627 2.18018 299.18018 150.68018 15068.01801 2.18018 299.18018 150.68017 15068.01782 2.18018 299.18018 150.68018 15068.01800 2020-01-01 2020-01-02 2020-01-01 00:12:06 2020-01-02 03:40:27 2020-01-01 00:12:06.000 2020-01-02 03:40:27.000 726 99627 50176.5 5017650 726 99627 50176.5 5017650 -32444 32694 4956.66 495666 -125 126 0.5 50 727 100 10717 99628 2.18318 299.18318 150.68318 15068.31831 2.18318 299.1832 150.68318 15068.31842 2.18318 299.18318 150.68318 15068.31800 2020-01-01 2020-01-02 2020-01-01 00:12:07 2020-01-02 03:40:28 2020-01-01 00:12:07.000 2020-01-02 03:40:28.000 727 99628 50177.5 5017750 727 99628 50177.5 5017750 -32443 32695 4957.66 495766 -124 127 1.5 150 -728 100 10718 99629 2.18618 299.18618 150.68618 15068.61861 2.18618 299.1862 150.68618 15068.61875 2.18618 299.18618 150.68618000000004 15068.61800 2020-01-01 2020-01-02 2020-01-01 00:12:08 2020-01-02 03:40:29 2020-01-01 00:12:08.000 2020-01-02 03:40:29.000 728 99629 50178.5 5017850 728 99629 50178.5 5017850 -32442 32696 4958.66 495866 -128 127 -0.06 -6 -729 100 10719 99630 2.18918 299.18918 150.68918 15068.91891 2.18918 299.18918 150.68918 15068.91889 2.18918 299.18918 150.68917999999996 15068.91800 2020-01-01 2020-01-02 2020-01-01 00:12:09 2020-01-02 03:40:30 2020-01-01 00:12:09.000 2020-01-02 03:40:30.000 729 99630 50179.5 5017950 729 99630 50179.5 5017950 -32441 32697 4959.66 495966 -128 123 -1.62 -162 +728 100 10718 99629 2.18618 299.18618 150.68618 15068.61861 2.18618 299.1862 150.68618 15068.61875 2.18618 299.18618 150.68618 15068.61800 2020-01-01 2020-01-02 2020-01-01 00:12:08 2020-01-02 03:40:29 2020-01-01 00:12:08.000 2020-01-02 03:40:29.000 728 99629 50178.5 5017850 728 99629 50178.5 5017850 -32442 32696 4958.66 495866 -128 127 -0.06 -6 +729 100 10719 99630 2.18918 299.18918 150.68918 15068.91891 2.18918 299.18918 150.68918 15068.91889 2.18918 299.18918 150.68918 15068.91800 2020-01-01 2020-01-02 2020-01-01 00:12:09 2020-01-02 03:40:30 2020-01-01 00:12:09.000 2020-01-02 03:40:30.000 729 99630 50179.5 5017950 729 99630 50179.5 5017950 -32441 32697 4959.66 495966 -128 123 -1.62 -162 73 102 10063 99973 0.21921 300.21921 150.21921 15172.14114 0.21921 300.2192 150.21921 15172.14121 0.21921 300.21921 150.21921 15172.14021 2020-01-01 2020-01-02 2020-01-01 00:01:13 2020-01-02 03:46:13 2020-01-01 00:01:13.000 2020-01-02 03:46:13.000 73 99973 50023 5052323 73 99973 50023 5052323 -32496 32439 4602.009900990099 464803 -128 127 -3.4554455445544554 -349 -730 100 10720 99631 2.19219 299.19219 150.69219 15069.21921 2.19219 299.1922 150.69219 15069.21965 2.19219 299.19219 150.69218999999993 15069.21900 2020-01-01 2020-01-02 2020-01-01 00:12:10 2020-01-02 03:40:31 2020-01-01 00:12:10.000 2020-01-02 03:40:31.000 730 99631 50180.5 5018050 730 99631 50180.5 5018050 -32440 32698 4960.66 496066 -127 124 -0.62 -62 -731 100 10721 99632 2.19519 299.19519 150.69519 15069.51951 2.19519 299.1952 150.69519 15069.51928 2.19519 299.19519 150.69519000000008 15069.51900 2020-01-01 2020-01-02 2020-01-01 00:12:11 2020-01-02 03:40:32 2020-01-01 00:12:11.000 2020-01-02 03:40:32.000 731 99632 50181.5 5018150 731 99632 50181.5 5018150 -32439 32699 4961.66 496166 -126 125 0.38 38 -732 100 10722 99633 2.19819 299.19819 150.69819 15069.81981 2.19819 299.1982 150.69819 15069.81988 2.19819 299.19819 150.69818999999993 15069.81900 2020-01-01 2020-01-02 2020-01-01 00:12:12 2020-01-02 03:40:33 2020-01-01 00:12:12.000 2020-01-02 03:40:33.000 732 99633 50182.5 5018250 732 99633 50182.5 5018250 -32438 32700 4962.66 496266 -125 126 1.38 138 -733 100 10723 99634 2.2012 299.2012 150.7012 15070.12012 2.2012 299.2012 150.7012 15070.12022 2.20120 299.20120 150.70120000000003 15070.12000 2020-01-01 2020-01-02 2020-01-01 00:12:13 2020-01-02 03:40:34 2020-01-01 00:12:13.000 2020-01-02 03:40:34.000 733 99634 50183.5 5018350 733 99634 50183.5 5018350 -32437 32701 4963.66 496366 -124 127 2.38 238 -734 100 10724 99635 2.2042 299.2042 150.7042 15070.42042 2.2042 299.2042 150.7042 15070.42036 2.20420 299.20420 150.70420000000007 15070.42000 2020-01-01 2020-01-02 2020-01-01 00:12:14 2020-01-02 03:40:35 2020-01-01 00:12:14.000 2020-01-02 03:40:35.000 734 99635 50184.5 5018450 734 99635 50184.5 5018450 -32436 32702 4964.66 496466 -128 127 0.82 82 -735 100 10725 99636 2.2072 299.2072 150.7072 15070.72072 2.2072 299.2072 150.70721 15070.72111 2.20720 299.20720 150.70720000000003 15070.72000 2020-01-01 2020-01-02 2020-01-01 00:12:15 2020-01-02 03:40:36 2020-01-01 00:12:15.000 2020-01-02 03:40:36.000 735 99636 50185.5 5018550 735 99636 50185.5 5018550 -32435 32703 4965.66 496566 -128 127 -0.74 -74 -736 100 10726 99637 2.21021 299.21021 150.71021 15071.02102 2.21021 299.2102 150.7102 15071.02076 2.21021 299.21021 150.71021 15071.02100 2020-01-01 2020-01-02 2020-01-01 00:12:16 2020-01-02 03:40:37 2020-01-01 00:12:16.000 2020-01-02 03:40:37.000 736 99637 50186.5 5018650 736 99637 50186.5 5018650 -32434 32704 4966.66 496666 -128 124 -2.3 -230 -737 100 10727 99638 2.21321 299.21321 150.71321 15071.32132 2.21321 299.21323 150.71321 15071.32139 2.21321 299.21321 150.71320999999992 15071.32100 2020-01-01 2020-01-02 2020-01-01 00:12:17 2020-01-02 03:40:38 2020-01-01 00:12:17.000 2020-01-02 03:40:38.000 737 99638 50187.5 5018750 737 99638 50187.5 5018750 -32433 32705 4967.66 496766 -127 125 -1.3 -130 +730 100 10720 99631 2.19219 299.19219 150.69219 15069.21921 2.19219 299.1922 150.69219 15069.21965 2.19219 299.19219 150.69218999999998 15069.21900 2020-01-01 2020-01-02 2020-01-01 00:12:10 2020-01-02 03:40:31 2020-01-01 00:12:10.000 2020-01-02 03:40:31.000 730 99631 50180.5 5018050 730 99631 50180.5 5018050 -32440 32698 4960.66 496066 -127 124 -0.62 -62 +731 100 10721 99632 2.19519 299.19519 150.69519 15069.51951 2.19519 299.1952 150.69519 15069.51928 2.19519 299.19519 150.69519 15069.51900 2020-01-01 2020-01-02 2020-01-01 00:12:11 2020-01-02 03:40:32 2020-01-01 00:12:11.000 2020-01-02 03:40:32.000 731 99632 50181.5 5018150 731 99632 50181.5 5018150 -32439 32699 4961.66 496166 -126 125 0.38 38 +732 100 10722 99633 2.19819 299.19819 150.69819 15069.81981 2.19819 299.1982 150.69819 15069.81988 2.19819 299.19819 150.69818999999998 15069.81900 2020-01-01 2020-01-02 2020-01-01 00:12:12 2020-01-02 03:40:33 2020-01-01 00:12:12.000 2020-01-02 03:40:33.000 732 99633 50182.5 5018250 732 99633 50182.5 5018250 -32438 32700 4962.66 496266 -125 126 1.38 138 +733 100 10723 99634 2.2012 299.2012 150.7012 15070.12012 2.2012 299.2012 150.7012 15070.12022 2.20120 299.20120 150.7012 15070.12000 2020-01-01 2020-01-02 2020-01-01 00:12:13 2020-01-02 03:40:34 2020-01-01 00:12:13.000 2020-01-02 03:40:34.000 733 99634 50183.5 5018350 733 99634 50183.5 5018350 -32437 32701 4963.66 496366 -124 127 2.38 238 +734 100 10724 99635 2.2042 299.2042 150.7042 15070.42042 2.2042 299.2042 150.7042 15070.42036 2.20420 299.20420 150.70420000000001 15070.42000 2020-01-01 2020-01-02 2020-01-01 00:12:14 2020-01-02 03:40:35 2020-01-01 00:12:14.000 2020-01-02 03:40:35.000 734 99635 50184.5 5018450 734 99635 50184.5 5018450 -32436 32702 4964.66 496466 -128 127 0.82 82 +735 100 10725 99636 2.2072 299.2072 150.7072 15070.72072 2.2072 299.2072 150.70721 15070.72111 2.20720 299.20720 150.7072 15070.72000 2020-01-01 2020-01-02 2020-01-01 00:12:15 2020-01-02 03:40:36 2020-01-01 00:12:15.000 2020-01-02 03:40:36.000 735 99636 50185.5 5018550 735 99636 50185.5 5018550 -32435 32703 4965.66 496566 -128 127 -0.74 -74 +736 100 10726 99637 2.21021 299.21021 150.71021 15071.02102 2.21021 299.2102 150.7102 15071.02076 2.21021 299.21021 150.71021000000002 15071.02100 2020-01-01 2020-01-02 2020-01-01 00:12:16 2020-01-02 03:40:37 2020-01-01 00:12:16.000 2020-01-02 03:40:37.000 736 99637 50186.5 5018650 736 99637 50186.5 5018650 -32434 32704 4966.66 496666 -128 124 -2.3 -230 +737 100 10727 99638 2.21321 299.21321 150.71321 15071.32132 2.21321 299.21323 150.71321 15071.32139 2.21321 299.21321 150.71321 15071.32100 2020-01-01 2020-01-02 2020-01-01 00:12:17 2020-01-02 03:40:38 2020-01-01 00:12:17.000 2020-01-02 03:40:38.000 737 99638 50187.5 5018750 737 99638 50187.5 5018750 -32433 32705 4967.66 496766 -127 125 -1.3 -130 738 100 10728 99639 2.21621 299.21621 150.71621 15071.62162 2.21621 299.21622 150.71621 15071.62169 2.21621 299.21621 150.71621 15071.62100 2020-01-01 2020-01-02 2020-01-01 00:12:18 2020-01-02 03:40:39 2020-01-01 00:12:18.000 2020-01-02 03:40:39.000 738 99639 50188.5 5018850 738 99639 50188.5 5018850 -32432 32706 4968.66 496866 -126 126 -0.3 -30 -739 100 10729 99640 2.21921 299.21921 150.71921 15071.92192 2.21921 299.2192 150.71921 15071.92199 2.21921 299.21921 150.71920999999998 15071.92100 2020-01-01 2020-01-02 2020-01-01 00:12:19 2020-01-02 03:40:40 2020-01-01 00:12:19.000 2020-01-02 03:40:40.000 739 99640 50189.5 5018950 739 99640 50189.5 5018950 -32431 32707 4969.66 496966 -125 127 0.7 70 -74 102 10064 99974 0.22222 300.22222 150.22222 15172.44444 0.22222 300.22223 150.22222 15172.4448 0.22222 300.22222 150.22221999999994 15172.44422 2020-01-01 2020-01-02 2020-01-01 00:01:14 2020-01-02 03:46:14 2020-01-01 00:01:14.000 2020-01-02 03:46:14.000 74 99974 50024 5052424 74 99974 50024 5052424 -32495 32440 4603.009900990099 464904 -128 123 -4.99009900990099 -504 -740 100 10730 99641 2.22222 299.22222 150.72222 15072.22222 2.22222 299.22223 150.72222 15072.22258 2.22222 299.22222 150.72221999999994 15072.22200 2020-01-01 2020-01-02 2020-01-01 00:12:20 2020-01-02 03:40:41 2020-01-01 00:12:20.000 2020-01-02 03:40:41.000 740 99641 50190.5 5019050 740 99641 50190.5 5019050 -32430 32708 4970.66 497066 -128 127 -0.86 -86 -741 100 10731 99642 2.22522 299.22522 150.72522 15072.52252 2.22522 299.22522 150.72522 15072.52223 2.22522 299.22522 150.72522000000004 15072.52200 2020-01-01 2020-01-02 2020-01-01 00:12:21 2020-01-02 03:40:42 2020-01-01 00:12:21.000 2020-01-02 03:40:42.000 741 99642 50191.5 5019150 741 99642 50191.5 5019150 -32429 32709 4971.66 497166 -128 127 -2.42 -242 -742 100 10732 99643 2.22822 299.22822 150.72822 15072.82282 2.22822 299.22824 150.72822 15072.82286 2.22822 299.22822 150.72822000000008 15072.82200 2020-01-01 2020-01-02 2020-01-01 00:12:22 2020-01-02 03:40:43 2020-01-01 00:12:22.000 2020-01-02 03:40:43.000 742 99643 50192.5 5019250 742 99643 50192.5 5019250 -32428 32710 4972.66 497266 -128 123 -3.98 -398 +739 100 10729 99640 2.21921 299.21921 150.71921 15071.92192 2.21921 299.2192 150.71921 15071.92199 2.21921 299.21921 150.71921 15071.92100 2020-01-01 2020-01-02 2020-01-01 00:12:19 2020-01-02 03:40:40 2020-01-01 00:12:19.000 2020-01-02 03:40:40.000 739 99640 50189.5 5018950 739 99640 50189.5 5018950 -32431 32707 4969.66 496966 -125 127 0.7 70 +74 102 10064 99974 0.22222 300.22222 150.22222 15172.44444 0.22222 300.22223 150.22222 15172.4448 0.22222 300.22222 150.22222 15172.44422 2020-01-01 2020-01-02 2020-01-01 00:01:14 2020-01-02 03:46:14 2020-01-01 00:01:14.000 2020-01-02 03:46:14.000 74 99974 50024 5052424 74 99974 50024 5052424 -32495 32440 4603.009900990099 464904 -128 123 -4.99009900990099 -504 +740 100 10730 99641 2.22222 299.22222 150.72222 15072.22222 2.22222 299.22223 150.72222 15072.22258 2.22222 299.22222 150.72222 15072.22200 2020-01-01 2020-01-02 2020-01-01 00:12:20 2020-01-02 03:40:41 2020-01-01 00:12:20.000 2020-01-02 03:40:41.000 740 99641 50190.5 5019050 740 99641 50190.5 5019050 -32430 32708 4970.66 497066 -128 127 -0.86 -86 +741 100 10731 99642 2.22522 299.22522 150.72522 15072.52252 2.22522 299.22522 150.72522 15072.52223 2.22522 299.22522 150.72522 15072.52200 2020-01-01 2020-01-02 2020-01-01 00:12:21 2020-01-02 03:40:42 2020-01-01 00:12:21.000 2020-01-02 03:40:42.000 741 99642 50191.5 5019150 741 99642 50191.5 5019150 -32429 32709 4971.66 497166 -128 127 -2.42 -242 +742 100 10732 99643 2.22822 299.22822 150.72822 15072.82282 2.22822 299.22824 150.72822 15072.82286 2.22822 299.22822 150.72822 15072.82200 2020-01-01 2020-01-02 2020-01-01 00:12:22 2020-01-02 03:40:43 2020-01-01 00:12:22.000 2020-01-02 03:40:43.000 742 99643 50192.5 5019250 742 99643 50192.5 5019250 -32428 32710 4972.66 497266 -128 123 -3.98 -398 743 100 10733 99644 2.23123 299.23123 150.73123 15073.12312 2.23123 299.23123 150.73123 15073.12316 2.23123 299.23123 150.73122999999998 15073.12300 2020-01-01 2020-01-02 2020-01-01 00:12:23 2020-01-02 03:40:44 2020-01-01 00:12:23.000 2020-01-02 03:40:44.000 743 99644 50193.5 5019350 743 99644 50193.5 5019350 -32427 32711 4973.66 497366 -127 124 -2.98 -298 744 100 10734 99645 2.23423 299.23423 150.73423 15073.42342 2.23423 299.23422 150.73423 15073.42345 2.23423 299.23423 150.73423 15073.42300 2020-01-01 2020-01-02 2020-01-01 00:12:24 2020-01-02 03:40:45 2020-01-01 00:12:24.000 2020-01-02 03:40:45.000 744 99645 50194.5 5019450 744 99645 50194.5 5019450 -32426 32712 4974.66 497466 -126 125 -1.98 -198 -745 100 10735 99646 2.23723 299.23723 150.73723 15073.72372 2.23723 299.23724 150.73724 15073.72405 2.23723 299.23723 150.73723000000007 15073.72300 2020-01-01 2020-01-02 2020-01-01 00:12:25 2020-01-02 03:40:46 2020-01-01 00:12:25.000 2020-01-02 03:40:46.000 745 99646 50195.5 5019550 745 99646 50195.5 5019550 -32425 32713 4975.66 497566 -125 126 -0.98 -98 -746 100 10736 99647 2.24024 299.24024 150.74024 15074.02402 2.24024 299.24023 150.74023 15074.02373 2.24024 299.24024 150.74023999999994 15074.02400 2020-01-01 2020-01-02 2020-01-01 00:12:26 2020-01-02 03:40:47 2020-01-01 00:12:26.000 2020-01-02 03:40:47.000 746 99647 50196.5 5019650 746 99647 50196.5 5019650 -32424 32714 4976.66 497666 -124 127 0.02 2 -747 100 10737 99648 2.24324 299.24324 150.74324 15074.32432 2.24324 299.24326 150.74324 15074.32433 2.24324 299.24324 150.7432399999999 15074.32400 2020-01-01 2020-01-02 2020-01-01 00:12:27 2020-01-02 03:40:48 2020-01-01 00:12:27.000 2020-01-02 03:40:48.000 747 99648 50197.5 5019750 747 99648 50197.5 5019750 -32423 32715 4977.66 497766 -128 127 -1.54 -154 -748 100 10738 99649 2.24624 299.24624 150.74624 15074.62462 2.24624 299.24625 150.74624 15074.62463 2.24624 299.24624 150.7462400000001 15074.62400 2020-01-01 2020-01-02 2020-01-01 00:12:28 2020-01-02 03:40:49 2020-01-01 00:12:28.000 2020-01-02 03:40:49.000 748 99649 50198.5 5019850 748 99649 50198.5 5019850 -32422 32716 4978.66 497866 -128 123 -3.1 -310 -749 100 10739 99650 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74923999999996 15074.92400 2020-01-01 2020-01-02 2020-01-01 00:12:29 2020-01-02 03:40:50 2020-01-01 00:12:29.000 2020-01-02 03:40:50.000 749 99650 50199.5 5019950 749 99650 50199.5 5019950 -32421 32717 4979.66 497966 -127 124 -2.1 -210 -75 102 10065 99975 0.22522 300.22522 150.22522 15172.74774 0.22522 300.22522 150.22522 15172.74745 0.22522 300.22522 150.22522000000004 15172.74722 2020-01-01 2020-01-02 2020-01-01 00:01:15 2020-01-02 03:46:15 2020-01-01 00:01:15.000 2020-01-02 03:46:15.000 75 99975 50025 5052525 75 99975 50025 5052525 -32494 32441 4604.009900990099 465005 -127 124 -3.99009900990099 -403 -750 100 10740 99651 2.25225 299.25225 150.75225 15075.22522 2.25225 299.25226 150.75225 15075.22552 2.25225 299.25225 150.75224999999983 15075.22500 2020-01-01 2020-01-02 2020-01-01 00:12:30 2020-01-02 03:40:51 2020-01-01 00:12:30.000 2020-01-02 03:40:51.000 750 99651 50200.5 5020050 750 99651 50200.5 5020050 -32420 32718 4980.66 498066 -126 125 -1.1 -110 -751 100 10741 99652 2.25525 299.25525 150.75525 15075.52552 2.25525 299.25525 150.75525 15075.5252 2.25525 299.25525 150.75525000000007 15075.52500 2020-01-01 2020-01-02 2020-01-01 00:12:31 2020-01-02 03:40:52 2020-01-01 00:12:31.000 2020-01-02 03:40:52.000 751 99652 50201.5 5020150 751 99652 50201.5 5020150 -32419 32719 4981.66 498166 -125 126 -0.1 -10 -752 100 10742 99653 2.25825 299.25825 150.75825 15075.82582 2.25825 299.25827 150.75825 15075.8258 2.25825 299.25825 150.75825000000003 15075.82500 2020-01-01 2020-01-02 2020-01-01 00:12:32 2020-01-02 03:40:53 2020-01-01 00:12:32.000 2020-01-02 03:40:53.000 752 99653 50202.5 5020250 752 99653 50202.5 5020250 -32418 32720 4982.66 498266 -124 127 0.9 90 +745 100 10735 99646 2.23723 299.23723 150.73723 15073.72372 2.23723 299.23724 150.73724 15073.72405 2.23723 299.23723 150.73723 15073.72300 2020-01-01 2020-01-02 2020-01-01 00:12:25 2020-01-02 03:40:46 2020-01-01 00:12:25.000 2020-01-02 03:40:46.000 745 99646 50195.5 5019550 745 99646 50195.5 5019550 -32425 32713 4975.66 497566 -125 126 -0.98 -98 +746 100 10736 99647 2.24024 299.24024 150.74024 15074.02402 2.24024 299.24023 150.74023 15074.02373 2.24024 299.24024 150.74024 15074.02400 2020-01-01 2020-01-02 2020-01-01 00:12:26 2020-01-02 03:40:47 2020-01-01 00:12:26.000 2020-01-02 03:40:47.000 746 99647 50196.5 5019650 746 99647 50196.5 5019650 -32424 32714 4976.66 497666 -124 127 0.02 2 +747 100 10737 99648 2.24324 299.24324 150.74324 15074.32432 2.24324 299.24326 150.74324 15074.32433 2.24324 299.24324 150.74324000000001 15074.32400 2020-01-01 2020-01-02 2020-01-01 00:12:27 2020-01-02 03:40:48 2020-01-01 00:12:27.000 2020-01-02 03:40:48.000 747 99648 50197.5 5019750 747 99648 50197.5 5019750 -32423 32715 4977.66 497766 -128 127 -1.54 -154 +748 100 10738 99649 2.24624 299.24624 150.74624 15074.62462 2.24624 299.24625 150.74624 15074.62463 2.24624 299.24624 150.74624 15074.62400 2020-01-01 2020-01-02 2020-01-01 00:12:28 2020-01-02 03:40:49 2020-01-01 00:12:28.000 2020-01-02 03:40:49.000 748 99649 50198.5 5019850 748 99649 50198.5 5019850 -32422 32716 4978.66 497866 -128 123 -3.1 -310 +749 100 10739 99650 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924000000001 15074.92400 2020-01-01 2020-01-02 2020-01-01 00:12:29 2020-01-02 03:40:50 2020-01-01 00:12:29.000 2020-01-02 03:40:50.000 749 99650 50199.5 5019950 749 99650 50199.5 5019950 -32421 32717 4979.66 497966 -127 124 -2.1 -210 +75 102 10065 99975 0.22522 300.22522 150.22522 15172.74774 0.22522 300.22522 150.22522 15172.74745 0.22522 300.22522 150.22521999999998 15172.74722 2020-01-01 2020-01-02 2020-01-01 00:01:15 2020-01-02 03:46:15 2020-01-01 00:01:15.000 2020-01-02 03:46:15.000 75 99975 50025 5052525 75 99975 50025 5052525 -32494 32441 4604.009900990099 465005 -127 124 -3.99009900990099 -403 +750 100 10740 99651 2.25225 299.25225 150.75225 15075.22522 2.25225 299.25226 150.75225 15075.22552 2.25225 299.25225 150.75225 15075.22500 2020-01-01 2020-01-02 2020-01-01 00:12:30 2020-01-02 03:40:51 2020-01-01 00:12:30.000 2020-01-02 03:40:51.000 750 99651 50200.5 5020050 750 99651 50200.5 5020050 -32420 32718 4980.66 498066 -126 125 -1.1 -110 +751 100 10741 99652 2.25525 299.25525 150.75525 15075.52552 2.25525 299.25525 150.75525 15075.5252 2.25525 299.25525 150.75525 15075.52500 2020-01-01 2020-01-02 2020-01-01 00:12:31 2020-01-02 03:40:52 2020-01-01 00:12:31.000 2020-01-02 03:40:52.000 751 99652 50201.5 5020150 751 99652 50201.5 5020150 -32419 32719 4981.66 498166 -125 126 -0.1 -10 +752 100 10742 99653 2.25825 299.25825 150.75825 15075.82582 2.25825 299.25827 150.75825 15075.8258 2.25825 299.25825 150.75825 15075.82500 2020-01-01 2020-01-02 2020-01-01 00:12:32 2020-01-02 03:40:53 2020-01-01 00:12:32.000 2020-01-02 03:40:53.000 752 99653 50202.5 5020250 752 99653 50202.5 5020250 -32418 32720 4982.66 498266 -124 127 0.9 90 753 100 10743 99654 2.26126 299.26126 150.76126 15076.12612 2.26126 299.26126 150.76126 15076.12609 2.26126 299.26126 150.76126 15076.12600 2020-01-01 2020-01-02 2020-01-01 00:12:33 2020-01-02 03:40:54 2020-01-01 00:12:33.000 2020-01-02 03:40:54.000 753 99654 50203.5 5020350 753 99654 50203.5 5020350 -32417 32721 4983.66 498366 -128 127 -0.66 -66 -754 100 10744 99655 2.26426 299.26426 150.76426 15076.42642 2.26426 299.26425 150.76426 15076.4264 2.26426 299.26426 150.76425999999995 15076.42600 2020-01-01 2020-01-02 2020-01-01 00:12:34 2020-01-02 03:40:55 2020-01-01 00:12:34.000 2020-01-02 03:40:55.000 754 99655 50204.5 5020450 754 99655 50204.5 5020450 -32416 32722 4984.66 498466 -128 123 -2.22 -222 +754 100 10744 99655 2.26426 299.26426 150.76426 15076.42642 2.26426 299.26425 150.76426 15076.4264 2.26426 299.26426 150.76426 15076.42600 2020-01-01 2020-01-02 2020-01-01 00:12:34 2020-01-02 03:40:55 2020-01-01 00:12:34.000 2020-01-02 03:40:55.000 754 99655 50204.5 5020450 754 99655 50204.5 5020450 -32416 32722 4984.66 498466 -128 123 -2.22 -222 755 100 10745 99656 2.26726 299.26726 150.76726 15076.72672 2.26726 299.26727 150.76727 15076.72703 2.26726 299.26726 150.76726 15076.72600 2020-01-01 2020-01-02 2020-01-01 00:12:35 2020-01-02 03:40:56 2020-01-01 00:12:35.000 2020-01-02 03:40:56.000 755 99656 50205.5 5020550 755 99656 50205.5 5020550 -32415 32723 4985.66 498566 -127 124 -1.22 -122 -756 100 10746 99657 2.27027 299.27027 150.77027 15077.02702 2.27027 299.27026 150.77026 15077.02667 2.27027 299.27027 150.7702700000001 15077.02700 2020-01-01 2020-01-02 2020-01-01 00:12:36 2020-01-02 03:40:57 2020-01-01 00:12:36.000 2020-01-02 03:40:57.000 756 99657 50206.5 5020650 756 99657 50206.5 5020650 -32414 32724 4986.66 498666 -126 125 -0.22 -22 -757 100 10747 99658 2.27327 299.27327 150.77327 15077.32732 2.27327 299.2733 150.77327 15077.32727 2.27327 299.27327 150.7732699999999 15077.32700 2020-01-01 2020-01-02 2020-01-01 00:12:37 2020-01-02 03:40:58 2020-01-01 00:12:37.000 2020-01-02 03:40:58.000 757 99658 50207.5 5020750 757 99658 50207.5 5020750 -32413 32725 4987.66 498766 -125 126 0.78 78 -758 100 10748 99659 2.27627 299.27627 150.77627 15077.62762 2.27627 299.27628 150.77627 15077.62756 2.27627 299.27627 150.77627000000007 15077.62700 2020-01-01 2020-01-02 2020-01-01 00:12:38 2020-01-02 03:40:59 2020-01-01 00:12:38.000 2020-01-02 03:40:59.000 758 99659 50208.5 5020850 758 99659 50208.5 5020850 -32412 32726 4988.66 498866 -124 127 1.78 178 -759 100 10749 99660 2.27927 299.27927 150.77927 15077.92792 2.27927 299.27927 150.77927 15077.92787 2.27927 299.27927 150.7792699999999 15077.92700 2020-01-01 2020-01-02 2020-01-01 00:12:39 2020-01-02 03:41:00 2020-01-01 00:12:39.000 2020-01-02 03:41:00.000 759 99660 50209.5 5020950 759 99660 50209.5 5020950 -32411 32727 4989.66 498966 -128 127 0.22 22 -76 102 10066 99976 0.22822 300.22822 150.22822 15173.05105 0.22822 300.22824 150.22822 15173.05109 0.22822 300.22822 150.22822000000008 15173.05022 2020-01-01 2020-01-02 2020-01-01 00:01:16 2020-01-02 03:46:16 2020-01-01 00:01:16.000 2020-01-02 03:46:16.000 76 99976 50026 5052626 76 99976 50026 5052626 -32493 32442 4605.009900990099 465106 -126 125 -2.99009900990099 -302 +756 100 10746 99657 2.27027 299.27027 150.77027 15077.02702 2.27027 299.27026 150.77026 15077.02667 2.27027 299.27027 150.77027 15077.02700 2020-01-01 2020-01-02 2020-01-01 00:12:36 2020-01-02 03:40:57 2020-01-01 00:12:36.000 2020-01-02 03:40:57.000 756 99657 50206.5 5020650 756 99657 50206.5 5020650 -32414 32724 4986.66 498666 -126 125 -0.22 -22 +757 100 10747 99658 2.27327 299.27327 150.77327 15077.32732 2.27327 299.2733 150.77327 15077.32727 2.27327 299.27327 150.77327 15077.32700 2020-01-01 2020-01-02 2020-01-01 00:12:37 2020-01-02 03:40:58 2020-01-01 00:12:37.000 2020-01-02 03:40:58.000 757 99658 50207.5 5020750 757 99658 50207.5 5020750 -32413 32725 4987.66 498766 -125 126 0.78 78 +758 100 10748 99659 2.27627 299.27627 150.77627 15077.62762 2.27627 299.27628 150.77627 15077.62756 2.27627 299.27627 150.77627 15077.62700 2020-01-01 2020-01-02 2020-01-01 00:12:38 2020-01-02 03:40:59 2020-01-01 00:12:38.000 2020-01-02 03:40:59.000 758 99659 50208.5 5020850 758 99659 50208.5 5020850 -32412 32726 4988.66 498866 -124 127 1.78 178 +759 100 10749 99660 2.27927 299.27927 150.77927 15077.92792 2.27927 299.27927 150.77927 15077.92787 2.27927 299.27927 150.77927 15077.92700 2020-01-01 2020-01-02 2020-01-01 00:12:39 2020-01-02 03:41:00 2020-01-01 00:12:39.000 2020-01-02 03:41:00.000 759 99660 50209.5 5020950 759 99660 50209.5 5020950 -32411 32727 4989.66 498966 -128 127 0.22 22 +76 102 10066 99976 0.22822 300.22822 150.22822 15173.05105 0.22822 300.22824 150.22822 15173.05109 0.22822 300.22822 150.22822 15173.05022 2020-01-01 2020-01-02 2020-01-01 00:01:16 2020-01-02 03:46:16 2020-01-01 00:01:16.000 2020-01-02 03:46:16.000 76 99976 50026 5052626 76 99976 50026 5052626 -32493 32442 4605.009900990099 465106 -126 125 -2.99009900990099 -302 760 100 10750 99661 2.28228 299.28228 150.78228 15078.22822 2.28228 299.2823 150.78228 15078.2285 2.28228 299.28228 150.78228 15078.22800 2020-01-01 2020-01-02 2020-01-01 00:12:40 2020-01-02 03:41:01 2020-01-01 00:12:40.000 2020-01-02 03:41:01.000 760 99661 50210.5 5021050 760 99661 50210.5 5021050 -32410 32728 4990.66 499066 -128 127 -1.34 -134 -761 100 10751 99662 2.28528 299.28528 150.78528 15078.52852 2.28528 299.28528 150.78528 15078.52814 2.28528 299.28528 150.78528000000003 15078.52800 2020-01-01 2020-01-02 2020-01-01 00:12:41 2020-01-02 03:41:02 2020-01-01 00:12:41.000 2020-01-02 03:41:02.000 761 99662 50211.5 5021150 761 99662 50211.5 5021150 -32409 32729 4991.66 499166 -128 124 -2.9 -290 +761 100 10751 99662 2.28528 299.28528 150.78528 15078.52852 2.28528 299.28528 150.78528 15078.52814 2.28528 299.28528 150.78528 15078.52800 2020-01-01 2020-01-02 2020-01-01 00:12:41 2020-01-02 03:41:02 2020-01-01 00:12:41.000 2020-01-02 03:41:02.000 761 99662 50211.5 5021150 761 99662 50211.5 5021150 -32409 32729 4991.66 499166 -128 124 -2.9 -290 762 100 10752 99663 2.28828 299.28828 150.78828 15078.82882 2.28828 299.2883 150.78828 15078.82889 2.28828 299.28828 150.78828 15078.82800 2020-01-01 2020-01-02 2020-01-01 00:12:42 2020-01-02 03:41:03 2020-01-01 00:12:42.000 2020-01-02 03:41:03.000 762 99663 50212.5 5021250 762 99663 50212.5 5021250 -32408 32730 4992.66 499266 -127 125 -1.9 -190 -763 100 10753 99664 2.29129 299.29129 150.79129 15079.12912 2.29129 299.2913 150.79129 15079.12904 2.29129 299.29129 150.79128999999998 15079.12900 2020-01-01 2020-01-02 2020-01-01 00:12:43 2020-01-02 03:41:04 2020-01-01 00:12:43.000 2020-01-02 03:41:04.000 763 99664 50213.5 5021350 763 99664 50213.5 5021350 -32407 32731 4993.66 499366 -126 126 -0.9 -90 -764 100 10754 99665 2.29429 299.29429 150.79429 15079.42942 2.29429 299.29428 150.79429 15079.42933 2.29429 299.29429 150.7942899999999 15079.42900 2020-01-01 2020-01-02 2020-01-01 00:12:44 2020-01-02 03:41:05 2020-01-01 00:12:44.000 2020-01-02 03:41:05.000 764 99665 50214.5 5021450 764 99665 50214.5 5021450 -32406 32732 4994.66 499466 -125 127 0.1 10 -765 100 10755 99666 2.29729 299.29729 150.79729 15079.72972 2.29729 299.2973 150.79729 15079.72996 2.29729 299.29729 150.79729000000015 15079.72900 2020-01-01 2020-01-02 2020-01-01 00:12:45 2020-01-02 03:41:06 2020-01-01 00:12:45.000 2020-01-02 03:41:06.000 765 99666 50215.5 5021550 765 99666 50215.5 5021550 -32405 32733 4995.66 499566 -128 127 -1.46 -146 -766 100 10756 99667 2.3003 299.3003 150.8003 15080.03003 2.3003 299.3003 150.80029 15080.02961 2.30030 299.30030 150.80030000000002 15080.03000 2020-01-01 2020-01-02 2020-01-01 00:12:46 2020-01-02 03:41:07 2020-01-01 00:12:46.000 2020-01-02 03:41:07.000 766 99667 50216.5 5021650 766 99667 50216.5 5021650 -32404 32734 4996.66 499666 -128 127 -3.02 -302 -767 100 10757 99668 2.3033 299.3033 150.8033 15080.33033 2.3033 299.3033 150.8033 15080.33036 2.30330 299.30330 150.80329999999987 15080.33000 2020-01-01 2020-01-02 2020-01-01 00:12:47 2020-01-02 03:41:08 2020-01-01 00:12:47.000 2020-01-02 03:41:08.000 767 99668 50217.5 5021750 767 99668 50217.5 5021750 -32403 32735 4997.66 499766 -128 123 -4.58 -458 -768 100 10758 99669 2.3063 299.3063 150.8063 15080.63063 2.3063 299.3063 150.8063 15080.6305 2.30630 299.30630 150.8063000000001 15080.63000 2020-01-01 2020-01-02 2020-01-01 00:12:48 2020-01-02 03:41:09 2020-01-01 00:12:48.000 2020-01-02 03:41:09.000 768 99669 50218.5 5021850 768 99669 50218.5 5021850 -32402 32736 4998.66 499866 -127 124 -3.58 -358 -769 100 10759 99670 2.3093 299.3093 150.8093 15080.93093 2.3093 299.3093 150.8093 15080.93084 2.30930 299.30930 150.80930000000004 15080.93000 2020-01-01 2020-01-02 2020-01-01 00:12:49 2020-01-02 03:41:10 2020-01-01 00:12:49.000 2020-01-02 03:41:10.000 769 99670 50219.5 5021950 769 99670 50219.5 5021950 -32401 32737 4999.66 499966 -126 125 -2.58 -258 -77 102 10067 99977 0.23123 300.23123 150.23123 15173.35435 0.23123 300.23123 150.23123 15173.35439 0.23123 300.23123 150.23122999999998 15173.35423 2020-01-01 2020-01-02 2020-01-01 00:01:17 2020-01-02 03:46:17 2020-01-01 00:01:17.000 2020-01-02 03:46:17.000 77 99977 50027 5052727 77 99977 50027 5052727 -32492 32443 4606.009900990099 465207 -125 126 -1.99009900990099 -201 -770 100 10760 99671 2.31231 299.31231 150.81231 15081.23123 2.31231 299.31232 150.81231 15081.23144 2.31231 299.31231 150.81230999999997 15081.23100 2020-01-01 2020-01-02 2020-01-01 00:12:50 2020-01-02 03:41:11 2020-01-01 00:12:50.000 2020-01-02 03:41:11.000 770 99671 50220.5 5022050 770 99671 50220.5 5022050 -32400 32738 5000.66 500066 -125 126 -1.58 -158 -771 100 10761 99672 2.31531 299.31531 150.81531 15081.53153 2.31531 299.3153 150.81531 15081.53173 2.31531 299.31531 150.81530999999998 15081.53100 2020-01-01 2020-01-02 2020-01-01 00:12:51 2020-01-02 03:41:12 2020-01-01 00:12:51.000 2020-01-02 03:41:12.000 771 99672 50221.5 5022150 771 99672 50221.5 5022150 -32399 32739 5001.66 500166 -124 127 -0.58 -58 -772 100 10762 99673 2.31831 299.31831 150.81831 15081.83183 2.31831 299.31833 150.81831 15081.83183 2.31831 299.31831 150.81831000000003 15081.83100 2020-01-01 2020-01-02 2020-01-01 00:12:52 2020-01-02 03:41:13 2020-01-01 00:12:52.000 2020-01-02 03:41:13.000 772 99673 50222.5 5022250 772 99673 50222.5 5022250 -32398 32740 5002.66 500266 -128 127 -2.14 -214 -773 100 10763 99674 2.32132 299.32132 150.82132 15082.13213 2.32132 299.32132 150.82131 15082.13197 2.32132 299.32132 150.8213199999999 15082.13200 2020-01-01 2020-01-02 2020-01-01 00:12:53 2020-01-02 03:41:14 2020-01-01 00:12:53.000 2020-01-02 03:41:14.000 773 99674 50223.5 5022350 773 99674 50223.5 5022350 -32397 32741 5003.66 500366 -128 123 -3.7 -370 -774 100 10764 99675 2.32432 299.32432 150.82432 15082.43243 2.32432 299.3243 150.82432 15082.43231 2.32432 299.32432 150.82431999999994 15082.43200 2020-01-01 2020-01-02 2020-01-01 00:12:54 2020-01-02 03:41:15 2020-01-01 00:12:54.000 2020-01-02 03:41:15.000 774 99675 50224.5 5022450 774 99675 50224.5 5022450 -32396 32742 5004.66 500466 -127 124 -2.7 -270 -775 100 10765 99676 2.32732 299.32732 150.82732 15082.73273 2.32732 299.32733 150.82732 15082.73291 2.32732 299.32732 150.8273200000001 15082.73200 2020-01-01 2020-01-02 2020-01-01 00:12:55 2020-01-02 03:41:16 2020-01-01 00:12:55.000 2020-01-02 03:41:16.000 775 99676 50225.5 5022550 775 99676 50225.5 5022550 -32395 32743 5005.66 500566 -126 125 -1.7 -170 +763 100 10753 99664 2.29129 299.29129 150.79129 15079.12912 2.29129 299.2913 150.79129 15079.12904 2.29129 299.29129 150.79129 15079.12900 2020-01-01 2020-01-02 2020-01-01 00:12:43 2020-01-02 03:41:04 2020-01-01 00:12:43.000 2020-01-02 03:41:04.000 763 99664 50213.5 5021350 763 99664 50213.5 5021350 -32407 32731 4993.66 499366 -126 126 -0.9 -90 +764 100 10754 99665 2.29429 299.29429 150.79429 15079.42942 2.29429 299.29428 150.79429 15079.42933 2.29429 299.29429 150.79429 15079.42900 2020-01-01 2020-01-02 2020-01-01 00:12:44 2020-01-02 03:41:05 2020-01-01 00:12:44.000 2020-01-02 03:41:05.000 764 99665 50214.5 5021450 764 99665 50214.5 5021450 -32406 32732 4994.66 499466 -125 127 0.1 10 +765 100 10755 99666 2.29729 299.29729 150.79729 15079.72972 2.29729 299.2973 150.79729 15079.72996 2.29729 299.29729 150.79729 15079.72900 2020-01-01 2020-01-02 2020-01-01 00:12:45 2020-01-02 03:41:06 2020-01-01 00:12:45.000 2020-01-02 03:41:06.000 765 99666 50215.5 5021550 765 99666 50215.5 5021550 -32405 32733 4995.66 499566 -128 127 -1.46 -146 +766 100 10756 99667 2.3003 299.3003 150.8003 15080.03003 2.3003 299.3003 150.80029 15080.02961 2.30030 299.30030 150.8003 15080.03000 2020-01-01 2020-01-02 2020-01-01 00:12:46 2020-01-02 03:41:07 2020-01-01 00:12:46.000 2020-01-02 03:41:07.000 766 99667 50216.5 5021650 766 99667 50216.5 5021650 -32404 32734 4996.66 499666 -128 127 -3.02 -302 +767 100 10757 99668 2.3033 299.3033 150.8033 15080.33033 2.3033 299.3033 150.8033 15080.33036 2.30330 299.30330 150.8033 15080.33000 2020-01-01 2020-01-02 2020-01-01 00:12:47 2020-01-02 03:41:08 2020-01-01 00:12:47.000 2020-01-02 03:41:08.000 767 99668 50217.5 5021750 767 99668 50217.5 5021750 -32403 32735 4997.66 499766 -128 123 -4.58 -458 +768 100 10758 99669 2.3063 299.3063 150.8063 15080.63063 2.3063 299.3063 150.8063 15080.6305 2.30630 299.30630 150.8063 15080.63000 2020-01-01 2020-01-02 2020-01-01 00:12:48 2020-01-02 03:41:09 2020-01-01 00:12:48.000 2020-01-02 03:41:09.000 768 99669 50218.5 5021850 768 99669 50218.5 5021850 -32402 32736 4998.66 499866 -127 124 -3.58 -358 +769 100 10759 99670 2.3093 299.3093 150.8093 15080.93093 2.3093 299.3093 150.8093 15080.93084 2.30930 299.30930 150.8093 15080.93000 2020-01-01 2020-01-02 2020-01-01 00:12:49 2020-01-02 03:41:10 2020-01-01 00:12:49.000 2020-01-02 03:41:10.000 769 99670 50219.5 5021950 769 99670 50219.5 5021950 -32401 32737 4999.66 499966 -126 125 -2.58 -258 +77 102 10067 99977 0.23123 300.23123 150.23123 15173.35435 0.23123 300.23123 150.23123 15173.35439 0.23123 300.23123 150.23123 15173.35423 2020-01-01 2020-01-02 2020-01-01 00:01:17 2020-01-02 03:46:17 2020-01-01 00:01:17.000 2020-01-02 03:46:17.000 77 99977 50027 5052727 77 99977 50027 5052727 -32492 32443 4606.009900990099 465207 -125 126 -1.99009900990099 -201 +770 100 10760 99671 2.31231 299.31231 150.81231 15081.23123 2.31231 299.31232 150.81231 15081.23144 2.31231 299.31231 150.81231 15081.23100 2020-01-01 2020-01-02 2020-01-01 00:12:50 2020-01-02 03:41:11 2020-01-01 00:12:50.000 2020-01-02 03:41:11.000 770 99671 50220.5 5022050 770 99671 50220.5 5022050 -32400 32738 5000.66 500066 -125 126 -1.58 -158 +771 100 10761 99672 2.31531 299.31531 150.81531 15081.53153 2.31531 299.3153 150.81531 15081.53173 2.31531 299.31531 150.81531 15081.53100 2020-01-01 2020-01-02 2020-01-01 00:12:51 2020-01-02 03:41:12 2020-01-01 00:12:51.000 2020-01-02 03:41:12.000 771 99672 50221.5 5022150 771 99672 50221.5 5022150 -32399 32739 5001.66 500166 -124 127 -0.58 -58 +772 100 10762 99673 2.31831 299.31831 150.81831 15081.83183 2.31831 299.31833 150.81831 15081.83183 2.31831 299.31831 150.81831 15081.83100 2020-01-01 2020-01-02 2020-01-01 00:12:52 2020-01-02 03:41:13 2020-01-01 00:12:52.000 2020-01-02 03:41:13.000 772 99673 50222.5 5022250 772 99673 50222.5 5022250 -32398 32740 5002.66 500266 -128 127 -2.14 -214 +773 100 10763 99674 2.32132 299.32132 150.82132 15082.13213 2.32132 299.32132 150.82131 15082.13197 2.32132 299.32132 150.82132 15082.13200 2020-01-01 2020-01-02 2020-01-01 00:12:53 2020-01-02 03:41:14 2020-01-01 00:12:53.000 2020-01-02 03:41:14.000 773 99674 50223.5 5022350 773 99674 50223.5 5022350 -32397 32741 5003.66 500366 -128 123 -3.7 -370 +774 100 10764 99675 2.32432 299.32432 150.82432 15082.43243 2.32432 299.3243 150.82432 15082.43231 2.32432 299.32432 150.82432 15082.43200 2020-01-01 2020-01-02 2020-01-01 00:12:54 2020-01-02 03:41:15 2020-01-01 00:12:54.000 2020-01-02 03:41:15.000 774 99675 50224.5 5022450 774 99675 50224.5 5022450 -32396 32742 5004.66 500466 -127 124 -2.7 -270 +775 100 10765 99676 2.32732 299.32732 150.82732 15082.73273 2.32732 299.32733 150.82732 15082.73291 2.32732 299.32732 150.82732 15082.73200 2020-01-01 2020-01-02 2020-01-01 00:12:55 2020-01-02 03:41:16 2020-01-01 00:12:55.000 2020-01-02 03:41:16.000 775 99676 50225.5 5022550 775 99676 50225.5 5022550 -32395 32743 5005.66 500566 -126 125 -1.7 -170 776 100 10766 99677 2.33033 299.33033 150.83033 15083.03303 2.33033 299.33032 150.83033 15083.0332 2.33033 299.33033 150.83033 15083.03300 2020-01-01 2020-01-02 2020-01-01 00:12:56 2020-01-02 03:41:17 2020-01-01 00:12:56.000 2020-01-02 03:41:17.000 776 99677 50226.5 5022650 776 99677 50226.5 5022650 -32394 32744 5006.66 500666 -125 126 -0.7 -70 777 100 10767 99678 2.33333 299.33333 150.83333 15083.33333 2.33333 299.33334 150.83333 15083.3333 2.33333 299.33333 150.83333000000002 15083.33300 2020-01-01 2020-01-02 2020-01-01 00:12:57 2020-01-02 03:41:18 2020-01-01 00:12:57.000 2020-01-02 03:41:18.000 777 99678 50227.5 5022750 777 99678 50227.5 5022750 -32393 32745 5007.66 500766 -124 127 0.3 30 -778 100 10768 99679 2.33633 299.33633 150.83633 15083.63363 2.33633 299.33633 150.83633 15083.63348 2.33633 299.33633 150.83633000000006 15083.63300 2020-01-01 2020-01-02 2020-01-01 00:12:58 2020-01-02 03:41:19 2020-01-01 00:12:58.000 2020-01-02 03:41:19.000 778 99679 50228.5 5022850 778 99679 50228.5 5022850 -32392 32746 5008.66 500866 -128 127 -1.26 -126 +778 100 10768 99679 2.33633 299.33633 150.83633 15083.63363 2.33633 299.33633 150.83633 15083.63348 2.33633 299.33633 150.83633 15083.63300 2020-01-01 2020-01-02 2020-01-01 00:12:58 2020-01-02 03:41:19 2020-01-01 00:12:58.000 2020-01-02 03:41:19.000 778 99679 50228.5 5022850 778 99679 50228.5 5022850 -32392 32746 5008.66 500866 -128 127 -1.26 -126 779 100 10769 99680 2.33933 299.33933 150.83933 15083.93393 2.33933 299.33932 150.83933 15083.93378 2.33933 299.33933 150.83933000000002 15083.93300 2020-01-01 2020-01-02 2020-01-01 00:12:59 2020-01-02 03:41:20 2020-01-01 00:12:59.000 2020-01-02 03:41:20.000 779 99680 50229.5 5022950 779 99680 50229.5 5022950 -32391 32747 5009.66 500966 -128 123 -2.82 -282 78 102 10068 99978 0.23423 300.23423 150.23423 15173.65765 0.23423 300.23422 150.23423 15173.65769 0.23423 300.23423 150.23423 15173.65723 2020-01-01 2020-01-02 2020-01-01 00:01:18 2020-01-02 03:46:18 2020-01-01 00:01:18.000 2020-01-02 03:46:18.000 78 99978 50028 5052828 78 99978 50028 5052828 -32491 32444 4607.009900990099 465308 -124 127 -0.9900990099009901 -100 -780 100 10770 99681 2.34234 299.34234 150.84234 15084.23423 2.34234 299.34235 150.84234 15084.23437 2.34234 299.34234 150.84233999999984 15084.23400 2020-01-01 2020-01-02 2020-01-01 00:13:00 2020-01-02 03:41:21 2020-01-01 00:13:00.000 2020-01-02 03:41:21.000 780 99681 50230.5 5023050 780 99681 50230.5 5023050 -32390 32748 5010.66 501066 -127 124 -1.82 -182 -781 100 10771 99682 2.34534 299.34534 150.84534 15084.53453 2.34534 299.34534 150.84534 15084.53467 2.34534 299.34534 150.84533999999988 15084.53400 2020-01-01 2020-01-02 2020-01-01 00:13:01 2020-01-02 03:41:22 2020-01-01 00:13:01.000 2020-01-02 03:41:22.000 781 99682 50231.5 5023150 781 99682 50231.5 5023150 -32389 32749 5011.66 501166 -126 125 -0.82 -82 -782 100 10772 99683 2.34834 299.34834 150.84834 15084.83483 2.34834 299.34836 150.84834 15084.83477 2.34834 299.34834 150.84834000000006 15084.83400 2020-01-01 2020-01-02 2020-01-01 00:13:02 2020-01-02 03:41:23 2020-01-01 00:13:02.000 2020-01-02 03:41:23.000 782 99683 50232.5 5023250 782 99683 50232.5 5023250 -32388 32750 5012.66 501266 -125 126 0.18 18 -783 100 10773 99684 2.35135 299.35135 150.85135 15085.13513 2.35135 299.35135 150.85134 15085.13495 2.35135 299.35135 150.85135000000022 15085.13500 2020-01-01 2020-01-02 2020-01-01 00:13:03 2020-01-02 03:41:24 2020-01-01 00:13:03.000 2020-01-02 03:41:24.000 783 99684 50233.5 5023350 783 99684 50233.5 5023350 -32387 32751 5013.66 501366 -124 127 1.18 118 -784 100 10774 99685 2.35435 299.35435 150.85435 15085.43543 2.35435 299.35434 150.85435 15085.43525 2.35435 299.35435 150.85434999999973 15085.43500 2020-01-01 2020-01-02 2020-01-01 00:13:04 2020-01-02 03:41:25 2020-01-01 00:13:04.000 2020-01-02 03:41:25.000 784 99685 50234.5 5023450 784 99685 50234.5 5023450 -32386 32752 5014.66 501466 -128 127 -0.38 -38 -785 100 10775 99686 2.35735 299.35735 150.85735 15085.73573 2.35735 299.35736 150.85736 15085.736 2.35735 299.35735 150.8573500000002 15085.73500 2020-01-01 2020-01-02 2020-01-01 00:13:05 2020-01-02 03:41:26 2020-01-01 00:13:05.000 2020-01-02 03:41:26.000 785 99686 50235.5 5023550 785 99686 50235.5 5023550 -32385 32753 5015.66 501566 -128 127 -1.94 -194 -786 100 10776 99687 2.36036 299.36036 150.86036 15086.03603 2.36036 299.36035 150.86036 15086.03614 2.36036 299.36036 150.86036000000007 15086.03600 2020-01-01 2020-01-02 2020-01-01 00:13:06 2020-01-02 03:41:27 2020-01-01 00:13:06.000 2020-01-02 03:41:27.000 786 99687 50236.5 5023650 786 99687 50236.5 5023650 -32384 32754 5016.66 501666 -128 124 -3.5 -350 -787 100 10777 99688 2.36336 299.36336 150.86336 15086.33633 2.36336 299.36337 150.86336 15086.33628 2.36336 299.36336 150.8633599999999 15086.33600 2020-01-01 2020-01-02 2020-01-01 00:13:07 2020-01-02 03:41:28 2020-01-01 00:13:07.000 2020-01-02 03:41:28.000 787 99688 50237.5 5023750 787 99688 50237.5 5023750 -32383 32755 5017.66 501766 -127 125 -2.5 -250 -788 100 10778 99689 2.36636 299.36636 150.86636 15086.63663 2.36636 299.36636 150.86636 15086.63641 2.36636 299.36636 150.86636 15086.63600 2020-01-01 2020-01-02 2020-01-01 00:13:08 2020-01-02 03:41:29 2020-01-01 00:13:08.000 2020-01-02 03:41:29.000 788 99689 50238.5 5023850 788 99689 50238.5 5023850 -32382 32756 5018.66 501866 -126 126 -1.5 -150 -789 100 10779 99690 2.36936 299.36936 150.86936 15086.93693 2.36936 299.36935 150.86936 15086.93672 2.36936 299.36936 150.86936000000009 15086.93600 2020-01-01 2020-01-02 2020-01-01 00:13:09 2020-01-02 03:41:30 2020-01-01 00:13:09.000 2020-01-02 03:41:30.000 789 99690 50239.5 5023950 789 99690 50239.5 5023950 -32381 32757 5019.66 501966 -125 127 -0.5 -50 -79 102 10069 99979 0.23723 300.23723 150.23723 15173.96096 0.23723 300.23724 150.23724 15173.96129 0.23723 300.23723 150.23723000000012 15173.96023 2020-01-01 2020-01-02 2020-01-01 00:01:19 2020-01-02 03:46:19 2020-01-01 00:01:19.000 2020-01-02 03:46:19.000 79 99979 50029 5052929 79 99979 50029 5052929 -32490 32445 4608.009900990099 465409 -128 127 -2.5247524752475248 -255 -790 100 10780 99691 2.37237 299.37237 150.87237 15087.23723 2.37237 299.37238 150.87237 15087.23747 2.37237 299.37237 150.87236999999973 15087.23700 2020-01-01 2020-01-02 2020-01-01 00:13:10 2020-01-02 03:41:31 2020-01-01 00:13:10.000 2020-01-02 03:41:31.000 790 99691 50240.5 5024050 790 99691 50240.5 5024050 -32380 32758 5020.66 502066 -128 127 -2.06 -206 -791 100 10781 99692 2.37537 299.37537 150.87537 15087.53753 2.37537 299.37537 150.87537 15087.53761 2.37537 299.37537 150.87536999999983 15087.53700 2020-01-01 2020-01-02 2020-01-01 00:13:11 2020-01-02 03:41:32 2020-01-01 00:13:11.000 2020-01-02 03:41:32.000 791 99692 50241.5 5024150 791 99692 50241.5 5024150 -32379 32759 5021.66 502166 -128 127 -3.62 -362 -792 100 10782 99693 2.37837 299.37837 150.87837 15087.83783 2.37837 299.3784 150.87837 15087.83775 2.37837 299.37837 150.87837000000025 15087.83700 2020-01-01 2020-01-02 2020-01-01 00:13:12 2020-01-02 03:41:33 2020-01-01 00:13:12.000 2020-01-02 03:41:33.000 792 99693 50242.5 5024250 792 99693 50242.5 5024250 -32378 32760 5022.66 502266 -128 123 -5.18 -518 -793 100 10783 99694 2.38138 299.38138 150.88138 15088.13813 2.38138 299.38138 150.88137 15088.13789 2.38138 299.38138 150.88138000000012 15088.13800 2020-01-01 2020-01-02 2020-01-01 00:13:13 2020-01-02 03:41:34 2020-01-01 00:13:13.000 2020-01-02 03:41:34.000 793 99694 50243.5 5024350 793 99694 50243.5 5024350 -32377 32761 5023.66 502366 -127 124 -4.18 -418 -794 100 10784 99695 2.38438 299.38438 150.88438 15088.43843 2.38438 299.3844 150.88438 15088.43864 2.38438 299.38438 150.88438000000002 15088.43800 2020-01-01 2020-01-02 2020-01-01 00:13:14 2020-01-02 03:41:35 2020-01-01 00:13:14.000 2020-01-02 03:41:35.000 794 99695 50244.5 5024450 794 99695 50244.5 5024450 -32376 32762 5024.66 502466 -126 125 -3.18 -318 -795 100 10785 99696 2.38738 299.38738 150.88738 15088.73873 2.38738 299.3874 150.88738 15088.73894 2.38738 299.38738 150.8873800000001 15088.73800 2020-01-01 2020-01-02 2020-01-01 00:13:15 2020-01-02 03:41:36 2020-01-01 00:13:15.000 2020-01-02 03:41:36.000 795 99696 50245.5 5024550 795 99696 50245.5 5024550 -32375 32763 5025.66 502566 -125 126 -2.18 -218 -796 100 10786 99697 2.39039 299.39039 150.89039 15089.03903 2.39039 299.39038 150.89039 15089.03908 2.39039 299.39039 150.89038999999997 15089.03900 2020-01-01 2020-01-02 2020-01-01 00:13:16 2020-01-02 03:41:37 2020-01-01 00:13:16.000 2020-01-02 03:41:37.000 796 99697 50246.5 5024650 796 99697 50246.5 5024650 -32374 32764 5026.66 502666 -124 127 -1.18 -118 -797 100 10787 99698 2.39339 299.39339 150.89339 15089.33933 2.39339 299.3934 150.89339 15089.33921 2.39339 299.39339 150.89338999999987 15089.33900 2020-01-01 2020-01-02 2020-01-01 00:13:17 2020-01-02 03:41:38 2020-01-01 00:13:17.000 2020-01-02 03:41:38.000 797 99698 50247.5 5024750 797 99698 50247.5 5024750 -32373 32765 5027.66 502766 -128 127 -2.74 -274 -798 100 10788 99699 2.39639 299.39639 150.89639 15089.63963 2.39639 299.3964 150.89639 15089.63936 2.39639 299.39639 150.8963899999999 15089.63900 2020-01-01 2020-01-02 2020-01-01 00:13:18 2020-01-02 03:41:39 2020-01-01 00:13:18.000 2020-01-02 03:41:39.000 798 99699 50248.5 5024850 798 99699 50248.5 5024850 -32372 32766 5028.66 502866 -128 123 -4.3 -430 -799 100 10789 99700 2.39939 299.39939 150.89939 15089.93993 2.39939 299.3994 150.8994 15089.94011 2.39939 299.39939 150.89938999999998 15089.93900 2020-01-01 2020-01-02 2020-01-01 00:13:19 2020-01-02 03:41:40 2020-01-01 00:13:19.000 2020-01-02 03:41:40.000 799 99700 50249.5 5024950 799 99700 50249.5 5024950 -32371 32767 5029.66 502966 -127 124 -3.3 -330 -8 102 1007 9998 0.02402 300.02402 150.02402 15152.42642 0.02402 300.02402 150.02402 15152.42607 0.02402 300.02402 150.02402000000015 15152.42602 2020-01-01 2020-01-02 2020-01-01 00:00:08 2020-01-02 03:45:08 2020-01-01 00:00:08.000 2020-01-02 03:45:08.000 8 99908 49958 5045758 8 99908 49958 5045758 -32561 32374 4537.009900990099 458238 -125 126 -0.019801980198019802 -2 -80 102 10070 99980 0.24024 300.24024 150.24024 15174.26426 0.24024 300.24023 150.24023 15174.26397 0.24024 300.24024 150.24023999999977 15174.26424 2020-01-01 2020-01-02 2020-01-01 00:01:20 2020-01-02 03:46:20 2020-01-01 00:01:20.000 2020-01-02 03:46:20.000 80 99980 50030 5053030 80 99980 50030 5053030 -32489 32446 4609.009900990099 465510 -128 123 -4.0594059405940595 -410 -800 100 10790 99701 2.4024 299.4024 150.9024 15090.24024 2.4024 299.4024 150.9024 15090.24041 2.40240 299.40240 150.90240000000026 15090.24000 2020-01-01 2020-01-02 2020-01-01 00:13:20 2020-01-02 03:41:41 2020-01-01 00:13:20.000 2020-01-02 03:41:41.000 800 99701 50250.5 5025050 800 99701 50250.5 5025050 -32768 32167 4375.3 437530 -126 125 -2.3 -230 -801 100 10791 99702 2.4054 299.4054 150.9054 15090.54054 2.4054 299.4054 150.9054 15090.54058 2.40540 299.40540 150.90539999999976 15090.54000 2020-01-01 2020-01-02 2020-01-01 00:13:21 2020-01-02 03:41:42 2020-01-01 00:13:21.000 2020-01-02 03:41:42.000 801 99702 50251.5 5025150 801 99702 50251.5 5025150 -32767 32168 4376.3 437630 -125 126 -1.3 -130 -802 100 10792 99703 2.4084 299.4084 150.9084 15090.84084 2.4084 299.40842 150.9084 15090.84069 2.40840 299.40840 150.90840000000014 15090.84000 2020-01-01 2020-01-02 2020-01-01 00:13:22 2020-01-02 03:41:43 2020-01-01 00:13:22.000 2020-01-02 03:41:43.000 802 99703 50252.5 5025250 802 99703 50252.5 5025250 -32766 32169 4377.3 437730 -124 127 -0.3 -30 -803 100 10793 99704 2.41141 299.41141 150.91141 15091.14114 2.41141 299.4114 150.9114 15091.14098 2.41141 299.41141 150.9114100000001 15091.14100 2020-01-01 2020-01-02 2020-01-01 00:13:23 2020-01-02 03:41:44 2020-01-01 00:13:23.000 2020-01-02 03:41:44.000 803 99704 50253.5 5025350 803 99704 50253.5 5025350 -32765 32170 4378.3 437830 -128 127 -1.86 -186 -804 100 10794 99705 2.41441 299.41441 150.91441 15091.44144 2.41441 299.41443 150.91441 15091.44158 2.41441 299.41441 150.91440999999992 15091.44100 2020-01-01 2020-01-02 2020-01-01 00:13:24 2020-01-02 03:41:45 2020-01-01 00:13:24.000 2020-01-02 03:41:45.000 804 99705 50254.5 5025450 804 99705 50254.5 5025450 -32764 32171 4379.3 437930 -128 123 -3.42 -342 -805 100 10795 99706 2.41741 299.41741 150.91741 15091.74174 2.41741 299.41742 150.91741 15091.74188 2.41741 299.41741 150.91741000000002 15091.74100 2020-01-01 2020-01-02 2020-01-01 00:13:25 2020-01-02 03:41:46 2020-01-01 00:13:25.000 2020-01-02 03:41:46.000 805 99706 50255.5 5025550 805 99706 50255.5 5025550 -32763 32172 4380.3 438030 -127 124 -2.42 -242 -806 100 10796 99707 2.42042 299.42042 150.92042 15092.04204 2.42042 299.4204 150.92042 15092.04205 2.42042 299.42042 150.92042000000026 15092.04200 2020-01-01 2020-01-02 2020-01-01 00:13:26 2020-01-02 03:41:47 2020-01-01 00:13:26.000 2020-01-02 03:41:47.000 806 99707 50256.5 5025650 806 99707 50256.5 5025650 -32762 32173 4381.3 438130 -126 125 -1.42 -142 -807 100 10797 99708 2.42342 299.42342 150.92342 15092.34234 2.42342 299.42343 150.92342 15092.34216 2.42342 299.42342 150.92341999999977 15092.34200 2020-01-01 2020-01-02 2020-01-01 00:13:27 2020-01-02 03:41:48 2020-01-01 00:13:27.000 2020-01-02 03:41:48.000 807 99708 50257.5 5025750 807 99708 50257.5 5025750 -32761 32174 4382.3 438230 -125 126 -0.42 -42 -808 100 10798 99709 2.42642 299.42642 150.92642 15092.64264 2.42642 299.42642 150.92642 15092.64245 2.42642 299.42642 150.9264199999998 15092.64200 2020-01-01 2020-01-02 2020-01-01 00:13:28 2020-01-02 03:41:49 2020-01-01 00:13:28.000 2020-01-02 03:41:49.000 808 99709 50258.5 5025850 808 99709 50258.5 5025850 -32760 32175 4383.3 438330 -124 127 0.58 58 -809 100 10799 99710 2.42942 299.42942 150.92942 15092.94294 2.42942 299.42944 150.92943 15092.94305 2.42942 299.42942 150.92942000000028 15092.94200 2020-01-01 2020-01-02 2020-01-01 00:13:29 2020-01-02 03:41:50 2020-01-01 00:13:29.000 2020-01-02 03:41:50.000 809 99710 50259.5 5025950 809 99710 50259.5 5025950 -32759 32176 4384.3 438430 -128 127 -0.98 -98 -81 102 10071 99981 0.24324 300.24324 150.24324 15174.56756 0.24324 300.24326 150.24324 15174.56758 0.24324 300.24324 150.24323999999982 15174.56724 2020-01-01 2020-01-02 2020-01-01 00:01:21 2020-01-02 03:46:21 2020-01-01 00:01:21.000 2020-01-02 03:46:21.000 81 99981 50031 5053131 81 99981 50031 5053131 -32488 32447 4610.009900990099 465611 -127 124 -3.0594059405940595 -309 -810 100 10800 99711 2.43243 299.43243 150.93243 15093.24324 2.43243 299.43243 150.93243 15093.24338 2.43243 299.43243 150.93243000000015 15093.24300 2020-01-01 2020-01-02 2020-01-01 00:13:30 2020-01-02 03:41:51 2020-01-01 00:13:30.000 2020-01-02 03:41:51.000 810 99711 50260.5 5026050 810 99711 50260.5 5026050 -32758 32177 4385.3 438530 -128 127 -2.54 -254 -811 100 10801 99712 2.43543 299.43543 150.93543 15093.54354 2.43543 299.43542 150.93543 15093.54353 2.43543 299.43543 150.93542999999966 15093.54300 2020-01-01 2020-01-02 2020-01-01 00:13:31 2020-01-02 03:41:52 2020-01-01 00:13:31.000 2020-01-02 03:41:52.000 811 99712 50261.5 5026150 811 99712 50261.5 5026150 -32757 32178 4386.3 438630 -128 124 -4.1 -410 -812 100 10802 99713 2.43843 299.43843 150.93843 15093.84384 2.43843 299.43845 150.93844 15093.84428 2.43843 299.43843 150.93843000000012 15093.84300 2020-01-01 2020-01-02 2020-01-01 00:13:32 2020-01-02 03:41:53 2020-01-01 00:13:32.000 2020-01-02 03:41:53.000 812 99713 50262.5 5026250 812 99713 50262.5 5026250 -32756 32179 4387.3 438730 -127 125 -3.1 -310 +780 100 10770 99681 2.34234 299.34234 150.84234 15084.23423 2.34234 299.34235 150.84234 15084.23437 2.34234 299.34234 150.84234 15084.23400 2020-01-01 2020-01-02 2020-01-01 00:13:00 2020-01-02 03:41:21 2020-01-01 00:13:00.000 2020-01-02 03:41:21.000 780 99681 50230.5 5023050 780 99681 50230.5 5023050 -32390 32748 5010.66 501066 -127 124 -1.82 -182 +781 100 10771 99682 2.34534 299.34534 150.84534 15084.53453 2.34534 299.34534 150.84534 15084.53467 2.34534 299.34534 150.84534 15084.53400 2020-01-01 2020-01-02 2020-01-01 00:13:01 2020-01-02 03:41:22 2020-01-01 00:13:01.000 2020-01-02 03:41:22.000 781 99682 50231.5 5023150 781 99682 50231.5 5023150 -32389 32749 5011.66 501166 -126 125 -0.82 -82 +782 100 10772 99683 2.34834 299.34834 150.84834 15084.83483 2.34834 299.34836 150.84834 15084.83477 2.34834 299.34834 150.84834 15084.83400 2020-01-01 2020-01-02 2020-01-01 00:13:02 2020-01-02 03:41:23 2020-01-01 00:13:02.000 2020-01-02 03:41:23.000 782 99683 50232.5 5023250 782 99683 50232.5 5023250 -32388 32750 5012.66 501266 -125 126 0.18 18 +783 100 10773 99684 2.35135 299.35135 150.85135 15085.13513 2.35135 299.35135 150.85134 15085.13495 2.35135 299.35135 150.85135 15085.13500 2020-01-01 2020-01-02 2020-01-01 00:13:03 2020-01-02 03:41:24 2020-01-01 00:13:03.000 2020-01-02 03:41:24.000 783 99684 50233.5 5023350 783 99684 50233.5 5023350 -32387 32751 5013.66 501366 -124 127 1.18 118 +784 100 10774 99685 2.35435 299.35435 150.85435 15085.43543 2.35435 299.35434 150.85435 15085.43525 2.35435 299.35435 150.85434999999998 15085.43500 2020-01-01 2020-01-02 2020-01-01 00:13:04 2020-01-02 03:41:25 2020-01-01 00:13:04.000 2020-01-02 03:41:25.000 784 99685 50234.5 5023450 784 99685 50234.5 5023450 -32386 32752 5014.66 501466 -128 127 -0.38 -38 +785 100 10775 99686 2.35735 299.35735 150.85735 15085.73573 2.35735 299.35736 150.85736 15085.736 2.35735 299.35735 150.85735 15085.73500 2020-01-01 2020-01-02 2020-01-01 00:13:05 2020-01-02 03:41:26 2020-01-01 00:13:05.000 2020-01-02 03:41:26.000 785 99686 50235.5 5023550 785 99686 50235.5 5023550 -32385 32753 5015.66 501566 -128 127 -1.94 -194 +786 100 10776 99687 2.36036 299.36036 150.86036 15086.03603 2.36036 299.36035 150.86036 15086.03614 2.36036 299.36036 150.86036000000001 15086.03600 2020-01-01 2020-01-02 2020-01-01 00:13:06 2020-01-02 03:41:27 2020-01-01 00:13:06.000 2020-01-02 03:41:27.000 786 99687 50236.5 5023650 786 99687 50236.5 5023650 -32384 32754 5016.66 501666 -128 124 -3.5 -350 +787 100 10777 99688 2.36336 299.36336 150.86336 15086.33633 2.36336 299.36337 150.86336 15086.33628 2.36336 299.36336 150.86336 15086.33600 2020-01-01 2020-01-02 2020-01-01 00:13:07 2020-01-02 03:41:28 2020-01-01 00:13:07.000 2020-01-02 03:41:28.000 787 99688 50237.5 5023750 787 99688 50237.5 5023750 -32383 32755 5017.66 501766 -127 125 -2.5 -250 +788 100 10778 99689 2.36636 299.36636 150.86636 15086.63663 2.36636 299.36636 150.86636 15086.63641 2.36636 299.36636 150.86636000000001 15086.63600 2020-01-01 2020-01-02 2020-01-01 00:13:08 2020-01-02 03:41:29 2020-01-01 00:13:08.000 2020-01-02 03:41:29.000 788 99689 50238.5 5023850 788 99689 50238.5 5023850 -32382 32756 5018.66 501866 -126 126 -1.5 -150 +789 100 10779 99690 2.36936 299.36936 150.86936 15086.93693 2.36936 299.36935 150.86936 15086.93672 2.36936 299.36936 150.86936 15086.93600 2020-01-01 2020-01-02 2020-01-01 00:13:09 2020-01-02 03:41:30 2020-01-01 00:13:09.000 2020-01-02 03:41:30.000 789 99690 50239.5 5023950 789 99690 50239.5 5023950 -32381 32757 5019.66 501966 -125 127 -0.5 -50 +79 102 10069 99979 0.23723 300.23723 150.23723 15173.96096 0.23723 300.23724 150.23724 15173.96129 0.23723 300.23723 150.23723 15173.96023 2020-01-01 2020-01-02 2020-01-01 00:01:19 2020-01-02 03:46:19 2020-01-01 00:01:19.000 2020-01-02 03:46:19.000 79 99979 50029 5052929 79 99979 50029 5052929 -32490 32445 4608.009900990099 465409 -128 127 -2.5247524752475248 -255 +790 100 10780 99691 2.37237 299.37237 150.87237 15087.23723 2.37237 299.37238 150.87237 15087.23747 2.37237 299.37237 150.87237 15087.23700 2020-01-01 2020-01-02 2020-01-01 00:13:10 2020-01-02 03:41:31 2020-01-01 00:13:10.000 2020-01-02 03:41:31.000 790 99691 50240.5 5024050 790 99691 50240.5 5024050 -32380 32758 5020.66 502066 -128 127 -2.06 -206 +791 100 10781 99692 2.37537 299.37537 150.87537 15087.53753 2.37537 299.37537 150.87537 15087.53761 2.37537 299.37537 150.87537 15087.53700 2020-01-01 2020-01-02 2020-01-01 00:13:11 2020-01-02 03:41:32 2020-01-01 00:13:11.000 2020-01-02 03:41:32.000 791 99692 50241.5 5024150 791 99692 50241.5 5024150 -32379 32759 5021.66 502166 -128 127 -3.62 -362 +792 100 10782 99693 2.37837 299.37837 150.87837 15087.83783 2.37837 299.3784 150.87837 15087.83775 2.37837 299.37837 150.87837 15087.83700 2020-01-01 2020-01-02 2020-01-01 00:13:12 2020-01-02 03:41:33 2020-01-01 00:13:12.000 2020-01-02 03:41:33.000 792 99693 50242.5 5024250 792 99693 50242.5 5024250 -32378 32760 5022.66 502266 -128 123 -5.18 -518 +793 100 10783 99694 2.38138 299.38138 150.88138 15088.13813 2.38138 299.38138 150.88137 15088.13789 2.38138 299.38138 150.88138 15088.13800 2020-01-01 2020-01-02 2020-01-01 00:13:13 2020-01-02 03:41:34 2020-01-01 00:13:13.000 2020-01-02 03:41:34.000 793 99694 50243.5 5024350 793 99694 50243.5 5024350 -32377 32761 5023.66 502366 -127 124 -4.18 -418 +794 100 10784 99695 2.38438 299.38438 150.88438 15088.43843 2.38438 299.3844 150.88438 15088.43864 2.38438 299.38438 150.88438 15088.43800 2020-01-01 2020-01-02 2020-01-01 00:13:14 2020-01-02 03:41:35 2020-01-01 00:13:14.000 2020-01-02 03:41:35.000 794 99695 50244.5 5024450 794 99695 50244.5 5024450 -32376 32762 5024.66 502466 -126 125 -3.18 -318 +795 100 10785 99696 2.38738 299.38738 150.88738 15088.73873 2.38738 299.3874 150.88738 15088.73894 2.38738 299.38738 150.88738 15088.73800 2020-01-01 2020-01-02 2020-01-01 00:13:15 2020-01-02 03:41:36 2020-01-01 00:13:15.000 2020-01-02 03:41:36.000 795 99696 50245.5 5024550 795 99696 50245.5 5024550 -32375 32763 5025.66 502566 -125 126 -2.18 -218 +796 100 10786 99697 2.39039 299.39039 150.89039 15089.03903 2.39039 299.39038 150.89039 15089.03908 2.39039 299.39039 150.89039 15089.03900 2020-01-01 2020-01-02 2020-01-01 00:13:16 2020-01-02 03:41:37 2020-01-01 00:13:16.000 2020-01-02 03:41:37.000 796 99697 50246.5 5024650 796 99697 50246.5 5024650 -32374 32764 5026.66 502666 -124 127 -1.18 -118 +797 100 10787 99698 2.39339 299.39339 150.89339 15089.33933 2.39339 299.3934 150.89339 15089.33921 2.39339 299.39339 150.89339 15089.33900 2020-01-01 2020-01-02 2020-01-01 00:13:17 2020-01-02 03:41:38 2020-01-01 00:13:17.000 2020-01-02 03:41:38.000 797 99698 50247.5 5024750 797 99698 50247.5 5024750 -32373 32765 5027.66 502766 -128 127 -2.74 -274 +798 100 10788 99699 2.39639 299.39639 150.89639 15089.63963 2.39639 299.3964 150.89639 15089.63936 2.39639 299.39639 150.89639 15089.63900 2020-01-01 2020-01-02 2020-01-01 00:13:18 2020-01-02 03:41:39 2020-01-01 00:13:18.000 2020-01-02 03:41:39.000 798 99699 50248.5 5024850 798 99699 50248.5 5024850 -32372 32766 5028.66 502866 -128 123 -4.3 -430 +799 100 10789 99700 2.39939 299.39939 150.89939 15089.93993 2.39939 299.3994 150.8994 15089.94011 2.39939 299.39939 150.89939 15089.93900 2020-01-01 2020-01-02 2020-01-01 00:13:19 2020-01-02 03:41:40 2020-01-01 00:13:19.000 2020-01-02 03:41:40.000 799 99700 50249.5 5024950 799 99700 50249.5 5024950 -32371 32767 5029.66 502966 -127 124 -3.3 -330 +8 102 1007 9998 0.02402 300.02402 150.02402 15152.42642 0.02402 300.02402 150.02402 15152.42607 0.02402 300.02402 150.02402 15152.42602 2020-01-01 2020-01-02 2020-01-01 00:00:08 2020-01-02 03:45:08 2020-01-01 00:00:08.000 2020-01-02 03:45:08.000 8 99908 49958 5045758 8 99908 49958 5045758 -32561 32374 4537.009900990099 458238 -125 126 -0.019801980198019802 -2 +80 102 10070 99980 0.24024 300.24024 150.24024 15174.26426 0.24024 300.24023 150.24023 15174.26397 0.24024 300.24024 150.24024 15174.26424 2020-01-01 2020-01-02 2020-01-01 00:01:20 2020-01-02 03:46:20 2020-01-01 00:01:20.000 2020-01-02 03:46:20.000 80 99980 50030 5053030 80 99980 50030 5053030 -32489 32446 4609.009900990099 465510 -128 123 -4.0594059405940595 -410 +800 100 10790 99701 2.4024 299.4024 150.9024 15090.24024 2.4024 299.4024 150.9024 15090.24041 2.40240 299.40240 150.9024 15090.24000 2020-01-01 2020-01-02 2020-01-01 00:13:20 2020-01-02 03:41:41 2020-01-01 00:13:20.000 2020-01-02 03:41:41.000 800 99701 50250.5 5025050 800 99701 50250.5 5025050 -32768 32167 4375.3 437530 -126 125 -2.3 -230 +801 100 10791 99702 2.4054 299.4054 150.9054 15090.54054 2.4054 299.4054 150.9054 15090.54058 2.40540 299.40540 150.90540000000001 15090.54000 2020-01-01 2020-01-02 2020-01-01 00:13:21 2020-01-02 03:41:42 2020-01-01 00:13:21.000 2020-01-02 03:41:42.000 801 99702 50251.5 5025150 801 99702 50251.5 5025150 -32767 32168 4376.3 437630 -125 126 -1.3 -130 +802 100 10792 99703 2.4084 299.4084 150.9084 15090.84084 2.4084 299.40842 150.9084 15090.84069 2.40840 299.40840 150.9084 15090.84000 2020-01-01 2020-01-02 2020-01-01 00:13:22 2020-01-02 03:41:43 2020-01-01 00:13:22.000 2020-01-02 03:41:43.000 802 99703 50252.5 5025250 802 99703 50252.5 5025250 -32766 32169 4377.3 437730 -124 127 -0.3 -30 +803 100 10793 99704 2.41141 299.41141 150.91141 15091.14114 2.41141 299.4114 150.9114 15091.14098 2.41141 299.41141 150.91141 15091.14100 2020-01-01 2020-01-02 2020-01-01 00:13:23 2020-01-02 03:41:44 2020-01-01 00:13:23.000 2020-01-02 03:41:44.000 803 99704 50253.5 5025350 803 99704 50253.5 5025350 -32765 32170 4378.3 437830 -128 127 -1.86 -186 +804 100 10794 99705 2.41441 299.41441 150.91441 15091.44144 2.41441 299.41443 150.91441 15091.44158 2.41441 299.41441 150.91441 15091.44100 2020-01-01 2020-01-02 2020-01-01 00:13:24 2020-01-02 03:41:45 2020-01-01 00:13:24.000 2020-01-02 03:41:45.000 804 99705 50254.5 5025450 804 99705 50254.5 5025450 -32764 32171 4379.3 437930 -128 123 -3.42 -342 +805 100 10795 99706 2.41741 299.41741 150.91741 15091.74174 2.41741 299.41742 150.91741 15091.74188 2.41741 299.41741 150.91741 15091.74100 2020-01-01 2020-01-02 2020-01-01 00:13:25 2020-01-02 03:41:46 2020-01-01 00:13:25.000 2020-01-02 03:41:46.000 805 99706 50255.5 5025550 805 99706 50255.5 5025550 -32763 32172 4380.3 438030 -127 124 -2.42 -242 +806 100 10796 99707 2.42042 299.42042 150.92042 15092.04204 2.42042 299.4204 150.92042 15092.04205 2.42042 299.42042 150.92042 15092.04200 2020-01-01 2020-01-02 2020-01-01 00:13:26 2020-01-02 03:41:47 2020-01-01 00:13:26.000 2020-01-02 03:41:47.000 806 99707 50256.5 5025650 806 99707 50256.5 5025650 -32762 32173 4381.3 438130 -126 125 -1.42 -142 +807 100 10797 99708 2.42342 299.42342 150.92342 15092.34234 2.42342 299.42343 150.92342 15092.34216 2.42342 299.42342 150.92342 15092.34200 2020-01-01 2020-01-02 2020-01-01 00:13:27 2020-01-02 03:41:48 2020-01-01 00:13:27.000 2020-01-02 03:41:48.000 807 99708 50257.5 5025750 807 99708 50257.5 5025750 -32761 32174 4382.3 438230 -125 126 -0.42 -42 +808 100 10798 99709 2.42642 299.42642 150.92642 15092.64264 2.42642 299.42642 150.92642 15092.64245 2.42642 299.42642 150.92642 15092.64200 2020-01-01 2020-01-02 2020-01-01 00:13:28 2020-01-02 03:41:49 2020-01-01 00:13:28.000 2020-01-02 03:41:49.000 808 99709 50258.5 5025850 808 99709 50258.5 5025850 -32760 32175 4383.3 438330 -124 127 0.58 58 +809 100 10799 99710 2.42942 299.42942 150.92942 15092.94294 2.42942 299.42944 150.92943 15092.94305 2.42942 299.42942 150.92942 15092.94200 2020-01-01 2020-01-02 2020-01-01 00:13:29 2020-01-02 03:41:50 2020-01-01 00:13:29.000 2020-01-02 03:41:50.000 809 99710 50259.5 5025950 809 99710 50259.5 5025950 -32759 32176 4384.3 438430 -128 127 -0.98 -98 +81 102 10071 99981 0.24324 300.24324 150.24324 15174.56756 0.24324 300.24326 150.24324 15174.56758 0.24324 300.24324 150.24324000000001 15174.56724 2020-01-01 2020-01-02 2020-01-01 00:01:21 2020-01-02 03:46:21 2020-01-01 00:01:21.000 2020-01-02 03:46:21.000 81 99981 50031 5053131 81 99981 50031 5053131 -32488 32447 4610.009900990099 465611 -127 124 -3.0594059405940595 -309 +810 100 10800 99711 2.43243 299.43243 150.93243 15093.24324 2.43243 299.43243 150.93243 15093.24338 2.43243 299.43243 150.93243 15093.24300 2020-01-01 2020-01-02 2020-01-01 00:13:30 2020-01-02 03:41:51 2020-01-01 00:13:30.000 2020-01-02 03:41:51.000 810 99711 50260.5 5026050 810 99711 50260.5 5026050 -32758 32177 4385.3 438530 -128 127 -2.54 -254 +811 100 10801 99712 2.43543 299.43543 150.93543 15093.54354 2.43543 299.43542 150.93543 15093.54353 2.43543 299.43543 150.93543 15093.54300 2020-01-01 2020-01-02 2020-01-01 00:13:31 2020-01-02 03:41:52 2020-01-01 00:13:31.000 2020-01-02 03:41:52.000 811 99712 50261.5 5026150 811 99712 50261.5 5026150 -32757 32178 4386.3 438630 -128 124 -4.1 -410 +812 100 10802 99713 2.43843 299.43843 150.93843 15093.84384 2.43843 299.43845 150.93844 15093.84428 2.43843 299.43843 150.93843 15093.84300 2020-01-01 2020-01-02 2020-01-01 00:13:32 2020-01-02 03:41:53 2020-01-01 00:13:32.000 2020-01-02 03:41:53.000 812 99713 50262.5 5026250 812 99713 50262.5 5026250 -32756 32179 4387.3 438730 -127 125 -3.1 -310 813 100 10803 99714 2.44144 299.44144 150.94144 15094.14414 2.44144 299.44144 150.94143 15094.14392 2.44144 299.44144 150.94144 15094.14400 2020-01-01 2020-01-02 2020-01-01 00:13:33 2020-01-02 03:41:54 2020-01-01 00:13:33.000 2020-01-02 03:41:54.000 813 99714 50263.5 5026350 813 99714 50263.5 5026350 -32755 32180 4388.3 438830 -126 126 -2.1 -210 -814 100 10804 99715 2.44444 299.44444 150.94444 15094.44444 2.44444 299.44446 150.94444 15094.44452 2.44444 299.44444 150.9444399999999 15094.44400 2020-01-01 2020-01-02 2020-01-01 00:13:34 2020-01-02 03:41:55 2020-01-01 00:13:34.000 2020-01-02 03:41:55.000 814 99715 50264.5 5026450 814 99715 50264.5 5026450 -32754 32181 4389.3 438930 -125 127 -1.1 -110 -815 100 10805 99716 2.44744 299.44744 150.94744 15094.74474 2.44744 299.44745 150.94744 15094.74485 2.44744 299.44744 150.94743999999994 15094.74400 2020-01-01 2020-01-02 2020-01-01 00:13:35 2020-01-02 03:41:56 2020-01-01 00:13:35.000 2020-01-02 03:41:56.000 815 99716 50265.5 5026550 815 99716 50265.5 5026550 -32753 32182 4390.3 439030 -128 127 -2.66 -266 -816 100 10806 99717 2.45045 299.45045 150.95045 15095.04504 2.45045 299.45044 150.95045 15095.045 2.45045 299.45045 150.95045000000016 15095.04500 2020-01-01 2020-01-02 2020-01-01 00:13:36 2020-01-02 03:41:57 2020-01-01 00:13:36.000 2020-01-02 03:41:57.000 816 99717 50266.5 5026650 816 99717 50266.5 5026650 -32752 32183 4391.3 439130 -128 127 -4.22 -422 -817 100 10807 99718 2.45345 299.45345 150.95345 15095.34534 2.45345 299.45346 150.95345 15095.34574 2.45345 299.45345 150.95345000000026 15095.34500 2020-01-01 2020-01-02 2020-01-01 00:13:37 2020-01-02 03:41:58 2020-01-01 00:13:37.000 2020-01-02 03:41:58.000 817 99718 50267.5 5026750 817 99718 50267.5 5026750 -32751 32184 4392.3 439230 -128 123 -5.78 -578 -818 100 10808 99719 2.45645 299.45645 150.95645 15095.64564 2.45645 299.45645 150.95645 15095.64539 2.45645 299.45645 150.9564499999998 15095.64500 2020-01-01 2020-01-02 2020-01-01 00:13:38 2020-01-02 03:41:59 2020-01-01 00:13:38.000 2020-01-02 03:41:59.000 818 99719 50268.5 5026850 818 99719 50268.5 5026850 -32750 32185 4393.3 439330 -127 124 -4.78 -478 -819 100 10809 99720 2.45945 299.45945 150.95945 15095.94594 2.45945 299.45947 150.95946 15095.94602 2.45945 299.45945 150.95945000000017 15095.94500 2020-01-01 2020-01-02 2020-01-01 00:13:39 2020-01-02 03:42:00 2020-01-01 00:13:39.000 2020-01-02 03:42:00.000 819 99720 50269.5 5026950 819 99720 50269.5 5026950 -32749 32186 4394.3 439430 -126 125 -3.78 -378 -82 102 10072 99982 0.24624 300.24624 150.24624 15174.87087 0.24624 300.24625 150.24624 15174.87088 0.24624 300.24624 150.24624000000028 15174.87024 2020-01-01 2020-01-02 2020-01-01 00:01:22 2020-01-02 03:46:22 2020-01-01 00:01:22.000 2020-01-02 03:46:22.000 82 99982 50032 5053232 82 99982 50032 5053232 -32487 32448 4611.009900990099 465712 -126 125 -2.0594059405940595 -208 -820 100 10810 99721 2.46246 299.46246 150.96246 15096.24624 2.46246 299.46246 150.96246 15096.24633 2.46246 299.46246 150.96246000000014 15096.24600 2020-01-01 2020-01-02 2020-01-01 00:13:40 2020-01-02 03:42:01 2020-01-01 00:13:40.000 2020-01-02 03:42:01.000 820 99721 50270.5 5027050 820 99721 50270.5 5027050 -32748 32187 4395.3 439530 -125 126 -2.78 -278 -821 100 10811 99722 2.46546 299.46546 150.96546 15096.54654 2.46546 299.46545 150.96546 15096.54646 2.46546 299.46546 150.96545999999995 15096.54600 2020-01-01 2020-01-02 2020-01-01 00:13:41 2020-01-02 03:42:02 2020-01-01 00:13:41.000 2020-01-02 03:42:02.000 821 99722 50271.5 5027150 821 99722 50271.5 5027150 -32747 32188 4396.3 439630 -124 127 -1.78 -178 -822 100 10812 99723 2.46846 299.46846 150.96846 15096.84684 2.46846 299.46848 150.96847 15096.84721 2.46846 299.46846 150.96846000000002 15096.84600 2020-01-01 2020-01-02 2020-01-01 00:13:42 2020-01-02 03:42:03 2020-01-01 00:13:42.000 2020-01-02 03:42:03.000 822 99723 50272.5 5027250 822 99723 50272.5 5027250 -32746 32189 4397.3 439730 -128 127 -3.34 -334 -823 100 10813 99724 2.47147 299.47147 150.97147 15097.14714 2.47147 299.47147 150.97146 15097.14686 2.47147 299.47147 150.9714700000003 15097.14700 2020-01-01 2020-01-02 2020-01-01 00:13:43 2020-01-02 03:42:04 2020-01-01 00:13:43.000 2020-01-02 03:42:04.000 823 99724 50273.5 5027350 823 99724 50273.5 5027350 -32745 32190 4398.3 439830 -128 123 -4.9 -490 -824 100 10814 99725 2.47447 299.47447 150.97447 15097.44744 2.47447 299.4745 150.97447 15097.44749 2.47447 299.47447 150.9744699999998 15097.44700 2020-01-01 2020-01-02 2020-01-01 00:13:44 2020-01-02 03:42:05 2020-01-01 00:13:44.000 2020-01-02 03:42:05.000 824 99725 50274.5 5027450 824 99725 50274.5 5027450 -32744 32191 4399.3 439930 -127 124 -3.9 -390 -825 100 10815 99726 2.47747 299.47747 150.97747 15097.74774 2.47747 299.47748 150.97747 15097.74779 2.47747 299.47747 150.97746999999984 15097.74700 2020-01-01 2020-01-02 2020-01-01 00:13:45 2020-01-02 03:42:06 2020-01-01 00:13:45.000 2020-01-02 03:42:06.000 825 99726 50275.5 5027550 825 99726 50275.5 5027550 -32743 32192 4400.3 440030 -126 125 -2.9 -290 -826 100 10816 99727 2.48048 299.48048 150.98048 15098.04804 2.48048 299.48047 150.98048 15098.04809 2.48048 299.48048 150.98048000000014 15098.04800 2020-01-01 2020-01-02 2020-01-01 00:13:46 2020-01-02 03:42:07 2020-01-01 00:13:46.000 2020-01-02 03:42:07.000 826 99727 50276.5 5027650 826 99727 50276.5 5027650 -32742 32193 4401.3 440130 -125 126 -1.9 -190 -827 100 10817 99728 2.48348 299.48348 150.98348 15098.34834 2.48348 299.4835 150.98348 15098.34869 2.48348 299.48348 150.98348000000018 15098.34800 2020-01-01 2020-01-02 2020-01-01 00:13:47 2020-01-02 03:42:08 2020-01-01 00:13:47.000 2020-01-02 03:42:08.000 827 99728 50277.5 5027750 827 99728 50277.5 5027750 -32741 32194 4402.3 440230 -124 127 -0.9 -90 -828 100 10818 99729 2.48648 299.48648 150.98648 15098.64864 2.48648 299.48648 150.98648 15098.64837 2.48648 299.48648 150.9864799999997 15098.64800 2020-01-01 2020-01-02 2020-01-01 00:13:48 2020-01-02 03:42:09 2020-01-01 00:13:48.000 2020-01-02 03:42:09.000 828 99729 50278.5 5027850 828 99729 50278.5 5027850 -32740 32195 4403.3 440330 -128 127 -2.46 -246 -829 100 10819 99730 2.48948 299.48948 150.98948 15098.94894 2.48948 299.4895 150.98948 15098.94896 2.48948 299.48948 150.98948000000016 15098.94800 2020-01-01 2020-01-02 2020-01-01 00:13:49 2020-01-02 03:42:10 2020-01-01 00:13:49.000 2020-01-02 03:42:10.000 829 99730 50279.5 5027950 829 99730 50279.5 5027950 -32739 32196 4404.3 440430 -128 123 -4.02 -402 -83 102 10073 99983 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.2492399999998 15175.17324 2020-01-01 2020-01-02 2020-01-01 00:01:23 2020-01-02 03:46:23 2020-01-01 00:01:23.000 2020-01-02 03:46:23.000 83 99983 50033 5053333 83 99983 50033 5053333 -32486 32449 4612.009900990099 465813 -125 126 -1.0594059405940595 -107 -830 100 10820 99731 2.49249 299.49249 150.99249 15099.24924 2.49249 299.4925 150.99249 15099.24926 2.49249 299.49249 150.99249000000003 15099.24900 2020-01-01 2020-01-02 2020-01-01 00:13:50 2020-01-02 03:42:11 2020-01-01 00:13:50.000 2020-01-02 03:42:11.000 830 99731 50280.5 5028050 830 99731 50280.5 5028050 -32738 32197 4405.3 440530 -127 124 -3.02 -302 -831 100 10821 99732 2.49549 299.49549 150.99549 15099.54954 2.49549 299.49548 150.99549 15099.54956 2.49549 299.49549 150.99548999999988 15099.54900 2020-01-01 2020-01-02 2020-01-01 00:13:51 2020-01-02 03:42:12 2020-01-01 00:13:51.000 2020-01-02 03:42:12.000 831 99732 50281.5 5028150 831 99732 50281.5 5028150 -32737 32198 4406.3 440630 -126 125 -2.02 -202 -832 100 10822 99733 2.49849 299.49849 150.99849 15099.84984 2.49849 299.4985 150.9985 15099.85016 2.49849 299.49849 150.99848999999998 15099.84900 2020-01-01 2020-01-02 2020-01-01 00:13:52 2020-01-02 03:42:13 2020-01-01 00:13:52.000 2020-01-02 03:42:13.000 832 99733 50282.5 5028250 832 99733 50282.5 5028250 -32736 32199 4407.3 440730 -125 126 -1.02 -102 -833 100 10823 99734 2.5015 299.5015 151.0015 15100.15015 2.5015 299.5015 151.00149 15100.14983 2.50150 299.50150 151.0015000000002 15100.15000 2020-01-01 2020-01-02 2020-01-01 00:13:53 2020-01-02 03:42:14 2020-01-01 00:13:53.000 2020-01-02 03:42:14.000 833 99734 50283.5 5028350 833 99734 50283.5 5028350 -32735 32200 4408.3 440830 -124 127 -0.02 -2 -834 100 10824 99735 2.5045 299.5045 151.0045 15100.45045 2.5045 299.50452 151.0045 15100.45043 2.50450 299.50450 151.00449999999972 15100.45000 2020-01-01 2020-01-02 2020-01-01 00:13:54 2020-01-02 03:42:15 2020-01-01 00:13:54.000 2020-01-02 03:42:15.000 834 99735 50284.5 5028450 834 99735 50284.5 5028450 -32734 32201 4409.3 440930 -128 127 -1.58 -158 -835 100 10825 99736 2.5075 299.5075 151.0075 15100.75075 2.5075 299.5075 151.0075 15100.75073 2.50750 299.50750 151.00749999999982 15100.75000 2020-01-01 2020-01-02 2020-01-01 00:13:55 2020-01-02 03:42:16 2020-01-01 00:13:55.000 2020-01-02 03:42:16.000 835 99736 50285.5 5028550 835 99736 50285.5 5028550 -32733 32202 4410.3 441030 -128 123 -3.14 -314 -836 100 10826 99737 2.51051 299.51051 151.01051 15101.05105 2.51051 299.5105 151.01051 15101.05103 2.51051 299.51051 151.01051000000004 15101.05100 2020-01-01 2020-01-02 2020-01-01 00:13:56 2020-01-02 03:42:17 2020-01-01 00:13:56.000 2020-01-02 03:42:17.000 836 99737 50286.5 5028650 836 99737 50286.5 5028650 -32732 32203 4411.3 441130 -127 124 -2.14 -214 -837 100 10827 99738 2.51351 299.51351 151.01351 15101.35135 2.51351 299.51352 151.01351 15101.35162 2.51351 299.51351 151.01351000000014 15101.35100 2020-01-01 2020-01-02 2020-01-01 00:13:57 2020-01-02 03:42:18 2020-01-01 00:13:57.000 2020-01-02 03:42:18.000 837 99738 50287.5 5028750 837 99738 50287.5 5028750 -32731 32204 4412.3 441230 -126 125 -1.14 -114 -838 100 10828 99739 2.51651 299.51651 151.01651 15101.65165 2.51651 299.5165 151.01651 15101.6513 2.51651 299.51651 151.01650999999998 15101.65100 2020-01-01 2020-01-02 2020-01-01 00:13:58 2020-01-02 03:42:19 2020-01-01 00:13:58.000 2020-01-02 03:42:19.000 838 99739 50288.5 5028850 838 99739 50288.5 5028850 -32730 32205 4413.3 441330 -125 126 -0.14 -14 -839 100 10829 99740 2.51951 299.51951 151.01951 15101.95195 2.51951 299.51953 151.01951 15101.9519 2.51951 299.51951 151.01951000000005 15101.95100 2020-01-01 2020-01-02 2020-01-01 00:13:59 2020-01-02 03:42:20 2020-01-01 00:13:59.000 2020-01-02 03:42:20.000 839 99740 50289.5 5028950 839 99740 50289.5 5028950 -32729 32206 4414.3 441430 -124 127 0.86 86 -84 102 10074 99984 0.25225 300.25225 150.25225 15175.47747 0.25225 300.25226 150.25225 15175.47778 0.25225 300.25225 150.25224999999966 15175.47725 2020-01-01 2020-01-02 2020-01-01 00:01:24 2020-01-02 03:46:24 2020-01-01 00:01:24.000 2020-01-02 03:46:24.000 84 99984 50034 5053434 84 99984 50034 5053434 -32485 32450 4613.009900990099 465914 -124 127 -0.0594059405940594 -6 -840 100 10830 99741 2.52252 299.52252 151.02252 15102.25225 2.52252 299.52252 151.02252 15102.2522 2.52252 299.52252 151.02252000000033 15102.25200 2020-01-01 2020-01-02 2020-01-01 00:14:00 2020-01-02 03:42:21 2020-01-01 00:14:00.000 2020-01-02 03:42:21.000 840 99741 50290.5 5029050 840 99741 50290.5 5029050 -32728 32207 4415.3 441530 -128 127 -0.7 -70 -841 100 10831 99742 2.52552 299.52552 151.02552 15102.55255 2.52552 299.5255 151.02552 15102.5525 2.52552 299.52552 151.02551999999983 15102.55200 2020-01-01 2020-01-02 2020-01-01 00:14:01 2020-01-02 03:42:22 2020-01-01 00:14:01.000 2020-01-02 03:42:22.000 841 99742 50291.5 5029150 841 99742 50291.5 5029150 -32727 32208 4416.3 441630 -128 127 -2.26 -226 -842 100 10832 99743 2.52852 299.52852 151.02852 15102.85285 2.52852 299.52853 151.02853 15102.85313 2.52852 299.52852 151.02851999999987 15102.85200 2020-01-01 2020-01-02 2020-01-01 00:14:02 2020-01-02 03:42:23 2020-01-01 00:14:02.000 2020-01-02 03:42:23.000 842 99743 50292.5 5029250 842 99743 50292.5 5029250 -32726 32209 4417.3 441730 -128 123 -3.82 -382 -843 100 10833 99744 2.53153 299.53153 151.03153 15103.15315 2.53153 299.53152 151.03152 15103.15278 2.53153 299.53153 151.03153000000017 15103.15300 2020-01-01 2020-01-02 2020-01-01 00:14:03 2020-01-02 03:42:24 2020-01-01 00:14:03.000 2020-01-02 03:42:24.000 843 99744 50293.5 5029350 843 99744 50293.5 5029350 -32725 32210 4418.3 441830 -127 124 -2.82 -282 -844 100 10834 99745 2.53453 299.53453 151.03453 15103.45345 2.53453 299.53455 151.03453 15103.45353 2.53453 299.53453 151.03453000000022 15103.45300 2020-01-01 2020-01-02 2020-01-01 00:14:04 2020-01-02 03:42:25 2020-01-01 00:14:04.000 2020-01-02 03:42:25.000 844 99745 50294.5 5029450 844 99745 50294.5 5029450 -32724 32211 4419.3 441930 -126 125 -1.82 -182 -845 100 10835 99746 2.53753 299.53753 151.03753 15103.75375 2.53753 299.53754 151.03753 15103.75366 2.53753 299.53753 151.03752999999972 15103.75300 2020-01-01 2020-01-02 2020-01-01 00:14:05 2020-01-02 03:42:26 2020-01-01 00:14:05.000 2020-01-02 03:42:26.000 845 99746 50295.5 5029550 845 99746 50295.5 5029550 -32723 32212 4420.3 442030 -125 126 -0.82 -82 +814 100 10804 99715 2.44444 299.44444 150.94444 15094.44444 2.44444 299.44446 150.94444 15094.44452 2.44444 299.44444 150.94444 15094.44400 2020-01-01 2020-01-02 2020-01-01 00:13:34 2020-01-02 03:41:55 2020-01-01 00:13:34.000 2020-01-02 03:41:55.000 814 99715 50264.5 5026450 814 99715 50264.5 5026450 -32754 32181 4389.3 438930 -125 127 -1.1 -110 +815 100 10805 99716 2.44744 299.44744 150.94744 15094.74474 2.44744 299.44745 150.94744 15094.74485 2.44744 299.44744 150.94744 15094.74400 2020-01-01 2020-01-02 2020-01-01 00:13:35 2020-01-02 03:41:56 2020-01-01 00:13:35.000 2020-01-02 03:41:56.000 815 99716 50265.5 5026550 815 99716 50265.5 5026550 -32753 32182 4390.3 439030 -128 127 -2.66 -266 +816 100 10806 99717 2.45045 299.45045 150.95045 15095.04504 2.45045 299.45044 150.95045 15095.045 2.45045 299.45045 150.95045 15095.04500 2020-01-01 2020-01-02 2020-01-01 00:13:36 2020-01-02 03:41:57 2020-01-01 00:13:36.000 2020-01-02 03:41:57.000 816 99717 50266.5 5026650 816 99717 50266.5 5026650 -32752 32183 4391.3 439130 -128 127 -4.22 -422 +817 100 10807 99718 2.45345 299.45345 150.95345 15095.34534 2.45345 299.45346 150.95345 15095.34574 2.45345 299.45345 150.95345 15095.34500 2020-01-01 2020-01-02 2020-01-01 00:13:37 2020-01-02 03:41:58 2020-01-01 00:13:37.000 2020-01-02 03:41:58.000 817 99718 50267.5 5026750 817 99718 50267.5 5026750 -32751 32184 4392.3 439230 -128 123 -5.78 -578 +818 100 10808 99719 2.45645 299.45645 150.95645 15095.64564 2.45645 299.45645 150.95645 15095.64539 2.45645 299.45645 150.95645000000002 15095.64500 2020-01-01 2020-01-02 2020-01-01 00:13:38 2020-01-02 03:41:59 2020-01-01 00:13:38.000 2020-01-02 03:41:59.000 818 99719 50268.5 5026850 818 99719 50268.5 5026850 -32750 32185 4393.3 439330 -127 124 -4.78 -478 +819 100 10809 99720 2.45945 299.45945 150.95945 15095.94594 2.45945 299.45947 150.95946 15095.94602 2.45945 299.45945 150.95945 15095.94500 2020-01-01 2020-01-02 2020-01-01 00:13:39 2020-01-02 03:42:00 2020-01-01 00:13:39.000 2020-01-02 03:42:00.000 819 99720 50269.5 5026950 819 99720 50269.5 5026950 -32749 32186 4394.3 439430 -126 125 -3.78 -378 +82 102 10072 99982 0.24624 300.24624 150.24624 15174.87087 0.24624 300.24625 150.24624 15174.87088 0.24624 300.24624 150.24624 15174.87024 2020-01-01 2020-01-02 2020-01-01 00:01:22 2020-01-02 03:46:22 2020-01-01 00:01:22.000 2020-01-02 03:46:22.000 82 99982 50032 5053232 82 99982 50032 5053232 -32487 32448 4611.009900990099 465712 -126 125 -2.0594059405940595 -208 +820 100 10810 99721 2.46246 299.46246 150.96246 15096.24624 2.46246 299.46246 150.96246 15096.24633 2.46246 299.46246 150.96246 15096.24600 2020-01-01 2020-01-02 2020-01-01 00:13:40 2020-01-02 03:42:01 2020-01-01 00:13:40.000 2020-01-02 03:42:01.000 820 99721 50270.5 5027050 820 99721 50270.5 5027050 -32748 32187 4395.3 439530 -125 126 -2.78 -278 +821 100 10811 99722 2.46546 299.46546 150.96546 15096.54654 2.46546 299.46545 150.96546 15096.54646 2.46546 299.46546 150.96546 15096.54600 2020-01-01 2020-01-02 2020-01-01 00:13:41 2020-01-02 03:42:02 2020-01-01 00:13:41.000 2020-01-02 03:42:02.000 821 99722 50271.5 5027150 821 99722 50271.5 5027150 -32747 32188 4396.3 439630 -124 127 -1.78 -178 +822 100 10812 99723 2.46846 299.46846 150.96846 15096.84684 2.46846 299.46848 150.96847 15096.84721 2.46846 299.46846 150.96846 15096.84600 2020-01-01 2020-01-02 2020-01-01 00:13:42 2020-01-02 03:42:03 2020-01-01 00:13:42.000 2020-01-02 03:42:03.000 822 99723 50272.5 5027250 822 99723 50272.5 5027250 -32746 32189 4397.3 439730 -128 127 -3.34 -334 +823 100 10813 99724 2.47147 299.47147 150.97147 15097.14714 2.47147 299.47147 150.97146 15097.14686 2.47147 299.47147 150.97147 15097.14700 2020-01-01 2020-01-02 2020-01-01 00:13:43 2020-01-02 03:42:04 2020-01-01 00:13:43.000 2020-01-02 03:42:04.000 823 99724 50273.5 5027350 823 99724 50273.5 5027350 -32745 32190 4398.3 439830 -128 123 -4.9 -490 +824 100 10814 99725 2.47447 299.47447 150.97447 15097.44744 2.47447 299.4745 150.97447 15097.44749 2.47447 299.47447 150.97447 15097.44700 2020-01-01 2020-01-02 2020-01-01 00:13:44 2020-01-02 03:42:05 2020-01-01 00:13:44.000 2020-01-02 03:42:05.000 824 99725 50274.5 5027450 824 99725 50274.5 5027450 -32744 32191 4399.3 439930 -127 124 -3.9 -390 +825 100 10815 99726 2.47747 299.47747 150.97747 15097.74774 2.47747 299.47748 150.97747 15097.74779 2.47747 299.47747 150.97746999999998 15097.74700 2020-01-01 2020-01-02 2020-01-01 00:13:45 2020-01-02 03:42:06 2020-01-01 00:13:45.000 2020-01-02 03:42:06.000 825 99726 50275.5 5027550 825 99726 50275.5 5027550 -32743 32192 4400.3 440030 -126 125 -2.9 -290 +826 100 10816 99727 2.48048 299.48048 150.98048 15098.04804 2.48048 299.48047 150.98048 15098.04809 2.48048 299.48048 150.98048 15098.04800 2020-01-01 2020-01-02 2020-01-01 00:13:46 2020-01-02 03:42:07 2020-01-01 00:13:46.000 2020-01-02 03:42:07.000 826 99727 50276.5 5027650 826 99727 50276.5 5027650 -32742 32193 4401.3 440130 -125 126 -1.9 -190 +827 100 10817 99728 2.48348 299.48348 150.98348 15098.34834 2.48348 299.4835 150.98348 15098.34869 2.48348 299.48348 150.98348 15098.34800 2020-01-01 2020-01-02 2020-01-01 00:13:47 2020-01-02 03:42:08 2020-01-01 00:13:47.000 2020-01-02 03:42:08.000 827 99728 50277.5 5027750 827 99728 50277.5 5027750 -32741 32194 4402.3 440230 -124 127 -0.9 -90 +828 100 10818 99729 2.48648 299.48648 150.98648 15098.64864 2.48648 299.48648 150.98648 15098.64837 2.48648 299.48648 150.98648 15098.64800 2020-01-01 2020-01-02 2020-01-01 00:13:48 2020-01-02 03:42:09 2020-01-01 00:13:48.000 2020-01-02 03:42:09.000 828 99729 50278.5 5027850 828 99729 50278.5 5027850 -32740 32195 4403.3 440330 -128 127 -2.46 -246 +829 100 10819 99730 2.48948 299.48948 150.98948 15098.94894 2.48948 299.4895 150.98948 15098.94896 2.48948 299.48948 150.98948000000001 15098.94800 2020-01-01 2020-01-02 2020-01-01 00:13:49 2020-01-02 03:42:10 2020-01-01 00:13:49.000 2020-01-02 03:42:10.000 829 99730 50279.5 5027950 829 99730 50279.5 5027950 -32739 32196 4404.3 440430 -128 123 -4.02 -402 +83 102 10073 99983 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924000000001 15175.17324 2020-01-01 2020-01-02 2020-01-01 00:01:23 2020-01-02 03:46:23 2020-01-01 00:01:23.000 2020-01-02 03:46:23.000 83 99983 50033 5053333 83 99983 50033 5053333 -32486 32449 4612.009900990099 465813 -125 126 -1.0594059405940595 -107 +830 100 10820 99731 2.49249 299.49249 150.99249 15099.24924 2.49249 299.4925 150.99249 15099.24926 2.49249 299.49249 150.99249 15099.24900 2020-01-01 2020-01-02 2020-01-01 00:13:50 2020-01-02 03:42:11 2020-01-01 00:13:50.000 2020-01-02 03:42:11.000 830 99731 50280.5 5028050 830 99731 50280.5 5028050 -32738 32197 4405.3 440530 -127 124 -3.02 -302 +831 100 10821 99732 2.49549 299.49549 150.99549 15099.54954 2.49549 299.49548 150.99549 15099.54956 2.49549 299.49549 150.99549000000002 15099.54900 2020-01-01 2020-01-02 2020-01-01 00:13:51 2020-01-02 03:42:12 2020-01-01 00:13:51.000 2020-01-02 03:42:12.000 831 99732 50281.5 5028150 831 99732 50281.5 5028150 -32737 32198 4406.3 440630 -126 125 -2.02 -202 +832 100 10822 99733 2.49849 299.49849 150.99849 15099.84984 2.49849 299.4985 150.9985 15099.85016 2.49849 299.49849 150.99849 15099.84900 2020-01-01 2020-01-02 2020-01-01 00:13:52 2020-01-02 03:42:13 2020-01-01 00:13:52.000 2020-01-02 03:42:13.000 832 99733 50282.5 5028250 832 99733 50282.5 5028250 -32736 32199 4407.3 440730 -125 126 -1.02 -102 +833 100 10823 99734 2.5015 299.5015 151.0015 15100.15015 2.5015 299.5015 151.00149 15100.14983 2.50150 299.50150 151.0015 15100.15000 2020-01-01 2020-01-02 2020-01-01 00:13:53 2020-01-02 03:42:14 2020-01-01 00:13:53.000 2020-01-02 03:42:14.000 833 99734 50283.5 5028350 833 99734 50283.5 5028350 -32735 32200 4408.3 440830 -124 127 -0.02 -2 +834 100 10824 99735 2.5045 299.5045 151.0045 15100.45045 2.5045 299.50452 151.0045 15100.45043 2.50450 299.50450 151.0045 15100.45000 2020-01-01 2020-01-02 2020-01-01 00:13:54 2020-01-02 03:42:15 2020-01-01 00:13:54.000 2020-01-02 03:42:15.000 834 99735 50284.5 5028450 834 99735 50284.5 5028450 -32734 32201 4409.3 440930 -128 127 -1.58 -158 +835 100 10825 99736 2.5075 299.5075 151.0075 15100.75075 2.5075 299.5075 151.0075 15100.75073 2.50750 299.50750 151.0075 15100.75000 2020-01-01 2020-01-02 2020-01-01 00:13:55 2020-01-02 03:42:16 2020-01-01 00:13:55.000 2020-01-02 03:42:16.000 835 99736 50285.5 5028550 835 99736 50285.5 5028550 -32733 32202 4410.3 441030 -128 123 -3.14 -314 +836 100 10826 99737 2.51051 299.51051 151.01051 15101.05105 2.51051 299.5105 151.01051 15101.05103 2.51051 299.51051 151.01050999999998 15101.05100 2020-01-01 2020-01-02 2020-01-01 00:13:56 2020-01-02 03:42:17 2020-01-01 00:13:56.000 2020-01-02 03:42:17.000 836 99737 50286.5 5028650 836 99737 50286.5 5028650 -32732 32203 4411.3 441130 -127 124 -2.14 -214 +837 100 10827 99738 2.51351 299.51351 151.01351 15101.35135 2.51351 299.51352 151.01351 15101.35162 2.51351 299.51351 151.01351 15101.35100 2020-01-01 2020-01-02 2020-01-01 00:13:57 2020-01-02 03:42:18 2020-01-01 00:13:57.000 2020-01-02 03:42:18.000 837 99738 50287.5 5028750 837 99738 50287.5 5028750 -32731 32204 4412.3 441230 -126 125 -1.14 -114 +838 100 10828 99739 2.51651 299.51651 151.01651 15101.65165 2.51651 299.5165 151.01651 15101.6513 2.51651 299.51651 151.01651 15101.65100 2020-01-01 2020-01-02 2020-01-01 00:13:58 2020-01-02 03:42:19 2020-01-01 00:13:58.000 2020-01-02 03:42:19.000 838 99739 50288.5 5028850 838 99739 50288.5 5028850 -32730 32205 4413.3 441330 -125 126 -0.14 -14 +839 100 10829 99740 2.51951 299.51951 151.01951 15101.95195 2.51951 299.51953 151.01951 15101.9519 2.51951 299.51951 151.01951 15101.95100 2020-01-01 2020-01-02 2020-01-01 00:13:59 2020-01-02 03:42:20 2020-01-01 00:13:59.000 2020-01-02 03:42:20.000 839 99740 50289.5 5028950 839 99740 50289.5 5028950 -32729 32206 4414.3 441430 -124 127 0.86 86 +84 102 10074 99984 0.25225 300.25225 150.25225 15175.47747 0.25225 300.25226 150.25225 15175.47778 0.25225 300.25225 150.25225 15175.47725 2020-01-01 2020-01-02 2020-01-01 00:01:24 2020-01-02 03:46:24 2020-01-01 00:01:24.000 2020-01-02 03:46:24.000 84 99984 50034 5053434 84 99984 50034 5053434 -32485 32450 4613.009900990099 465914 -124 127 -0.0594059405940594 -6 +840 100 10830 99741 2.52252 299.52252 151.02252 15102.25225 2.52252 299.52252 151.02252 15102.2522 2.52252 299.52252 151.02252000000001 15102.25200 2020-01-01 2020-01-02 2020-01-01 00:14:00 2020-01-02 03:42:21 2020-01-01 00:14:00.000 2020-01-02 03:42:21.000 840 99741 50290.5 5029050 840 99741 50290.5 5029050 -32728 32207 4415.3 441530 -128 127 -0.7 -70 +841 100 10831 99742 2.52552 299.52552 151.02552 15102.55255 2.52552 299.5255 151.02552 15102.5525 2.52552 299.52552 151.02552 15102.55200 2020-01-01 2020-01-02 2020-01-01 00:14:01 2020-01-02 03:42:22 2020-01-01 00:14:01.000 2020-01-02 03:42:22.000 841 99742 50291.5 5029150 841 99742 50291.5 5029150 -32727 32208 4416.3 441630 -128 127 -2.26 -226 +842 100 10832 99743 2.52852 299.52852 151.02852 15102.85285 2.52852 299.52853 151.02853 15102.85313 2.52852 299.52852 151.02852000000001 15102.85200 2020-01-01 2020-01-02 2020-01-01 00:14:02 2020-01-02 03:42:23 2020-01-01 00:14:02.000 2020-01-02 03:42:23.000 842 99743 50292.5 5029250 842 99743 50292.5 5029250 -32726 32209 4417.3 441730 -128 123 -3.82 -382 +843 100 10833 99744 2.53153 299.53153 151.03153 15103.15315 2.53153 299.53152 151.03152 15103.15278 2.53153 299.53153 151.03153 15103.15300 2020-01-01 2020-01-02 2020-01-01 00:14:03 2020-01-02 03:42:24 2020-01-01 00:14:03.000 2020-01-02 03:42:24.000 843 99744 50293.5 5029350 843 99744 50293.5 5029350 -32725 32210 4418.3 441830 -127 124 -2.82 -282 +844 100 10834 99745 2.53453 299.53453 151.03453 15103.45345 2.53453 299.53455 151.03453 15103.45353 2.53453 299.53453 151.03453 15103.45300 2020-01-01 2020-01-02 2020-01-01 00:14:04 2020-01-02 03:42:25 2020-01-01 00:14:04.000 2020-01-02 03:42:25.000 844 99745 50294.5 5029450 844 99745 50294.5 5029450 -32724 32211 4419.3 441930 -126 125 -1.82 -182 +845 100 10835 99746 2.53753 299.53753 151.03753 15103.75375 2.53753 299.53754 151.03753 15103.75366 2.53753 299.53753 151.03753 15103.75300 2020-01-01 2020-01-02 2020-01-01 00:14:05 2020-01-02 03:42:26 2020-01-01 00:14:05.000 2020-01-02 03:42:26.000 845 99746 50295.5 5029550 845 99746 50295.5 5029550 -32723 32212 4420.3 442030 -125 126 -0.82 -82 846 100 10836 99747 2.54054 299.54054 151.04054 15104.05405 2.54054 299.54053 151.04053 15104.05397 2.54054 299.54054 151.04054 15104.05400 2020-01-01 2020-01-02 2020-01-01 00:14:06 2020-01-02 03:42:27 2020-01-01 00:14:06.000 2020-01-02 03:42:27.000 846 99747 50296.5 5029650 846 99747 50296.5 5029650 -32722 32213 4421.3 442130 -124 127 0.18 18 -847 100 10837 99748 2.54354 299.54354 151.04354 15104.35435 2.54354 299.54355 151.04354 15104.3546 2.54354 299.54354 151.04354000000006 15104.35400 2020-01-01 2020-01-02 2020-01-01 00:14:07 2020-01-02 03:42:28 2020-01-01 00:14:07.000 2020-01-02 03:42:28.000 847 99748 50297.5 5029750 847 99748 50297.5 5029750 -32721 32214 4422.3 442230 -128 127 -1.38 -138 -848 100 10838 99749 2.54654 299.54654 151.04654 15104.65465 2.54654 299.54654 151.04654 15104.65425 2.54654 299.54654 151.04653999999988 15104.65400 2020-01-01 2020-01-02 2020-01-01 00:14:08 2020-01-02 03:42:29 2020-01-01 00:14:08.000 2020-01-02 03:42:29.000 848 99749 50298.5 5029850 848 99749 50298.5 5029850 -32720 32215 4423.3 442330 -128 123 -2.94 -294 -849 100 10839 99750 2.54954 299.54954 151.04954 15104.95495 2.54954 299.54956 151.04954 15104.95499 2.54954 299.54954 151.04953999999998 15104.95400 2020-01-01 2020-01-02 2020-01-01 00:14:09 2020-01-02 03:42:30 2020-01-01 00:14:09.000 2020-01-02 03:42:30.000 849 99750 50299.5 5029950 849 99750 50299.5 5029950 -32719 32216 4424.3 442430 -127 124 -1.94 -194 -85 102 10075 99985 0.25525 300.25525 150.25525 15175.78078 0.25525 300.25525 150.25525 15175.78046 0.25525 300.25525 150.25525000000013 15175.78025 2020-01-01 2020-01-02 2020-01-01 00:01:25 2020-01-02 03:46:25 2020-01-01 00:01:25.000 2020-01-02 03:46:25.000 85 99985 50035 5053535 85 99985 50035 5053535 -32484 32451 4614.009900990099 466015 -128 127 -1.5940594059405941 -161 -850 100 10840 99751 2.55255 299.55255 151.05255 15105.25525 2.55255 299.55255 151.05255 15105.25514 2.55255 299.55255 151.05255000000022 15105.25500 2020-01-01 2020-01-02 2020-01-01 00:14:10 2020-01-02 03:42:31 2020-01-01 00:14:10.000 2020-01-02 03:42:31.000 850 99751 50300.5 5030050 850 99751 50300.5 5030050 -32718 32217 4425.3 442530 -126 125 -0.94 -94 -851 100 10841 99752 2.55555 299.55555 151.05555 15105.55555 2.55555 299.55554 151.05555 15105.55547 2.55555 299.55555 151.05554999999973 15105.55500 2020-01-01 2020-01-02 2020-01-01 00:14:11 2020-01-02 03:42:32 2020-01-01 00:14:11.000 2020-01-02 03:42:32.000 851 99752 50301.5 5030150 851 99752 50301.5 5030150 -32717 32218 4426.3 442630 -125 126 0.06 6 -852 100 10842 99753 2.55855 299.55855 151.05855 15105.85585 2.55855 299.55856 151.05856 15105.85607 2.55855 299.55855 151.05854999999985 15105.85500 2020-01-01 2020-01-02 2020-01-01 00:14:12 2020-01-02 03:42:33 2020-01-01 00:14:12.000 2020-01-02 03:42:33.000 852 99753 50302.5 5030250 852 99753 50302.5 5030250 -32716 32219 4427.3 442730 -124 127 1.06 106 -853 100 10843 99754 2.56156 299.56156 151.06156 15106.15615 2.56156 299.56155 151.06155 15106.15571 2.56156 299.56156 151.06156000000007 15106.15600 2020-01-01 2020-01-02 2020-01-01 00:14:13 2020-01-02 03:42:34 2020-01-01 00:14:13.000 2020-01-02 03:42:34.000 853 99754 50303.5 5030350 853 99754 50303.5 5030350 -32715 32220 4428.3 442830 -128 127 -0.5 -50 -854 100 10844 99755 2.56456 299.56456 151.06456 15106.45645 2.56456 299.56458 151.06456 15106.45646 2.56456 299.56456 151.0645600000001 15106.45600 2020-01-01 2020-01-02 2020-01-01 00:14:14 2020-01-02 03:42:35 2020-01-01 00:14:14.000 2020-01-02 03:42:35.000 854 99755 50304.5 5030450 854 99755 50304.5 5030450 -32714 32221 4429.3 442930 -128 123 -2.06 -206 -855 100 10845 99756 2.56756 299.56756 151.06756 15106.75675 2.56756 299.56757 151.06756 15106.75661 2.56756 299.56756 151.06756000000001 15106.75600 2020-01-01 2020-01-02 2020-01-01 00:14:15 2020-01-02 03:42:36 2020-01-01 00:14:15.000 2020-01-02 03:42:36.000 855 99756 50305.5 5030550 855 99756 50305.5 5030550 -32713 32222 4430.3 443030 -127 124 -1.06 -106 -856 100 10846 99757 2.57057 299.57057 151.07057 15107.05705 2.57057 299.57056 151.07056 15107.05694 2.57057 299.57057 151.0705699999999 15107.05700 2020-01-01 2020-01-02 2020-01-01 00:14:16 2020-01-02 03:42:37 2020-01-01 00:14:16.000 2020-01-02 03:42:37.000 856 99757 50306.5 5030650 856 99757 50306.5 5030650 -32712 32223 4431.3 443130 -126 125 -0.06 -6 -857 100 10847 99758 2.57357 299.57357 151.07357 15107.35735 2.57357 299.57358 151.07357 15107.35754 2.57357 299.57357 151.07356999999996 15107.35700 2020-01-01 2020-01-02 2020-01-01 00:14:17 2020-01-02 03:42:38 2020-01-01 00:14:17.000 2020-01-02 03:42:38.000 857 99758 50307.5 5030750 857 99758 50307.5 5030750 -32711 32224 4432.3 443230 -125 126 0.94 94 -858 100 10848 99759 2.57657 299.57657 151.07657 15107.65765 2.57657 299.57657 151.07657 15107.65783 2.57657 299.57657 151.07656999999986 15107.65700 2020-01-01 2020-01-02 2020-01-01 00:14:18 2020-01-02 03:42:39 2020-01-01 00:14:18.000 2020-01-02 03:42:39.000 858 99759 50308.5 5030850 858 99759 50308.5 5030850 -32710 32225 4433.3 443330 -124 127 1.94 194 -859 100 10849 99760 2.57957 299.57957 151.07957 15107.95795 2.57957 299.5796 151.07957 15107.95794 2.57957 299.57957 151.0795699999999 15107.95700 2020-01-01 2020-01-02 2020-01-01 00:14:19 2020-01-02 03:42:40 2020-01-01 00:14:19.000 2020-01-02 03:42:40.000 859 99760 50309.5 5030950 859 99760 50309.5 5030950 -32709 32226 4434.3 443430 -128 127 0.38 38 -86 102 10076 99986 0.25825 300.25825 150.25825 15176.08408 0.25825 300.25827 150.25825 15176.08406 0.25825 300.25825 150.25825000000017 15176.08325 2020-01-01 2020-01-02 2020-01-01 00:01:26 2020-01-02 03:46:26 2020-01-01 00:01:26.000 2020-01-02 03:46:26.000 86 99986 50036 5053636 86 99986 50036 5053636 -32483 32452 4615.009900990099 466116 -128 123 -3.128712871287129 -316 -860 100 10850 99761 2.58258 299.58258 151.08258 15108.25825 2.58258 299.58258 151.08258 15108.25811 2.58258 299.58258 151.08258000000015 15108.25800 2020-01-01 2020-01-02 2020-01-01 00:14:20 2020-01-02 03:42:41 2020-01-01 00:14:20.000 2020-01-02 03:42:41.000 860 99761 50310.5 5031050 860 99761 50310.5 5031050 -32708 32227 4435.3 443530 -128 123 -1.18 -118 -861 100 10851 99762 2.58558 299.58558 151.08558 15108.55855 2.58558 299.58557 151.08558 15108.55841 2.58558 299.58558 151.08558000000025 15108.55800 2020-01-01 2020-01-02 2020-01-01 00:14:21 2020-01-02 03:42:42 2020-01-01 00:14:21.000 2020-01-02 03:42:42.000 861 99762 50311.5 5031150 861 99762 50311.5 5031150 -32707 32228 4436.3 443630 -127 124 -0.18 -18 -862 100 10852 99763 2.58858 299.58858 151.08858 15108.85885 2.58858 299.5886 151.08859 15108.85901 2.58858 299.58858 151.08857999999975 15108.85800 2020-01-01 2020-01-02 2020-01-01 00:14:22 2020-01-02 03:42:43 2020-01-01 00:14:22.000 2020-01-02 03:42:43.000 862 99763 50312.5 5031250 862 99763 50312.5 5031250 -32706 32229 4437.3 443730 -126 125 0.82 82 +847 100 10837 99748 2.54354 299.54354 151.04354 15104.35435 2.54354 299.54355 151.04354 15104.3546 2.54354 299.54354 151.04354 15104.35400 2020-01-01 2020-01-02 2020-01-01 00:14:07 2020-01-02 03:42:28 2020-01-01 00:14:07.000 2020-01-02 03:42:28.000 847 99748 50297.5 5029750 847 99748 50297.5 5029750 -32721 32214 4422.3 442230 -128 127 -1.38 -138 +848 100 10838 99749 2.54654 299.54654 151.04654 15104.65465 2.54654 299.54654 151.04654 15104.65425 2.54654 299.54654 151.04654 15104.65400 2020-01-01 2020-01-02 2020-01-01 00:14:08 2020-01-02 03:42:29 2020-01-01 00:14:08.000 2020-01-02 03:42:29.000 848 99749 50298.5 5029850 848 99749 50298.5 5029850 -32720 32215 4423.3 442330 -128 123 -2.94 -294 +849 100 10839 99750 2.54954 299.54954 151.04954 15104.95495 2.54954 299.54956 151.04954 15104.95499 2.54954 299.54954 151.04954 15104.95400 2020-01-01 2020-01-02 2020-01-01 00:14:09 2020-01-02 03:42:30 2020-01-01 00:14:09.000 2020-01-02 03:42:30.000 849 99750 50299.5 5029950 849 99750 50299.5 5029950 -32719 32216 4424.3 442430 -127 124 -1.94 -194 +85 102 10075 99985 0.25525 300.25525 150.25525 15175.78078 0.25525 300.25525 150.25525 15175.78046 0.25525 300.25525 150.25525 15175.78025 2020-01-01 2020-01-02 2020-01-01 00:01:25 2020-01-02 03:46:25 2020-01-01 00:01:25.000 2020-01-02 03:46:25.000 85 99985 50035 5053535 85 99985 50035 5053535 -32484 32451 4614.009900990099 466015 -128 127 -1.5940594059405941 -161 +850 100 10840 99751 2.55255 299.55255 151.05255 15105.25525 2.55255 299.55255 151.05255 15105.25514 2.55255 299.55255 151.05255 15105.25500 2020-01-01 2020-01-02 2020-01-01 00:14:10 2020-01-02 03:42:31 2020-01-01 00:14:10.000 2020-01-02 03:42:31.000 850 99751 50300.5 5030050 850 99751 50300.5 5030050 -32718 32217 4425.3 442530 -126 125 -0.94 -94 +851 100 10841 99752 2.55555 299.55555 151.05555 15105.55555 2.55555 299.55554 151.05555 15105.55547 2.55555 299.55555 151.05555 15105.55500 2020-01-01 2020-01-02 2020-01-01 00:14:11 2020-01-02 03:42:32 2020-01-01 00:14:11.000 2020-01-02 03:42:32.000 851 99752 50301.5 5030150 851 99752 50301.5 5030150 -32717 32218 4426.3 442630 -125 126 0.06 6 +852 100 10842 99753 2.55855 299.55855 151.05855 15105.85585 2.55855 299.55856 151.05856 15105.85607 2.55855 299.55855 151.05855 15105.85500 2020-01-01 2020-01-02 2020-01-01 00:14:12 2020-01-02 03:42:33 2020-01-01 00:14:12.000 2020-01-02 03:42:33.000 852 99753 50302.5 5030250 852 99753 50302.5 5030250 -32716 32219 4427.3 442730 -124 127 1.06 106 +853 100 10843 99754 2.56156 299.56156 151.06156 15106.15615 2.56156 299.56155 151.06155 15106.15571 2.56156 299.56156 151.06156000000001 15106.15600 2020-01-01 2020-01-02 2020-01-01 00:14:13 2020-01-02 03:42:34 2020-01-01 00:14:13.000 2020-01-02 03:42:34.000 853 99754 50303.5 5030350 853 99754 50303.5 5030350 -32715 32220 4428.3 442830 -128 127 -0.5 -50 +854 100 10844 99755 2.56456 299.56456 151.06456 15106.45645 2.56456 299.56458 151.06456 15106.45646 2.56456 299.56456 151.06456 15106.45600 2020-01-01 2020-01-02 2020-01-01 00:14:14 2020-01-02 03:42:35 2020-01-01 00:14:14.000 2020-01-02 03:42:35.000 854 99755 50304.5 5030450 854 99755 50304.5 5030450 -32714 32221 4429.3 442930 -128 123 -2.06 -206 +855 100 10845 99756 2.56756 299.56756 151.06756 15106.75675 2.56756 299.56757 151.06756 15106.75661 2.56756 299.56756 151.06756 15106.75600 2020-01-01 2020-01-02 2020-01-01 00:14:15 2020-01-02 03:42:36 2020-01-01 00:14:15.000 2020-01-02 03:42:36.000 855 99756 50305.5 5030550 855 99756 50305.5 5030550 -32713 32222 4430.3 443030 -127 124 -1.06 -106 +856 100 10846 99757 2.57057 299.57057 151.07057 15107.05705 2.57057 299.57056 151.07056 15107.05694 2.57057 299.57057 151.07057 15107.05700 2020-01-01 2020-01-02 2020-01-01 00:14:16 2020-01-02 03:42:37 2020-01-01 00:14:16.000 2020-01-02 03:42:37.000 856 99757 50306.5 5030650 856 99757 50306.5 5030650 -32712 32223 4431.3 443130 -126 125 -0.06 -6 +857 100 10847 99758 2.57357 299.57357 151.07357 15107.35735 2.57357 299.57358 151.07357 15107.35754 2.57357 299.57357 151.07357 15107.35700 2020-01-01 2020-01-02 2020-01-01 00:14:17 2020-01-02 03:42:38 2020-01-01 00:14:17.000 2020-01-02 03:42:38.000 857 99758 50307.5 5030750 857 99758 50307.5 5030750 -32711 32224 4432.3 443230 -125 126 0.94 94 +858 100 10848 99759 2.57657 299.57657 151.07657 15107.65765 2.57657 299.57657 151.07657 15107.65783 2.57657 299.57657 151.07657 15107.65700 2020-01-01 2020-01-02 2020-01-01 00:14:18 2020-01-02 03:42:39 2020-01-01 00:14:18.000 2020-01-02 03:42:39.000 858 99759 50308.5 5030850 858 99759 50308.5 5030850 -32710 32225 4433.3 443330 -124 127 1.94 194 +859 100 10849 99760 2.57957 299.57957 151.07957 15107.95795 2.57957 299.5796 151.07957 15107.95794 2.57957 299.57957 151.07957 15107.95700 2020-01-01 2020-01-02 2020-01-01 00:14:19 2020-01-02 03:42:40 2020-01-01 00:14:19.000 2020-01-02 03:42:40.000 859 99760 50309.5 5030950 859 99760 50309.5 5030950 -32709 32226 4434.3 443430 -128 127 0.38 38 +86 102 10076 99986 0.25825 300.25825 150.25825 15176.08408 0.25825 300.25827 150.25825 15176.08406 0.25825 300.25825 150.25825 15176.08325 2020-01-01 2020-01-02 2020-01-01 00:01:26 2020-01-02 03:46:26 2020-01-01 00:01:26.000 2020-01-02 03:46:26.000 86 99986 50036 5053636 86 99986 50036 5053636 -32483 32452 4615.009900990099 466116 -128 123 -3.128712871287129 -316 +860 100 10850 99761 2.58258 299.58258 151.08258 15108.25825 2.58258 299.58258 151.08258 15108.25811 2.58258 299.58258 151.08258 15108.25800 2020-01-01 2020-01-02 2020-01-01 00:14:20 2020-01-02 03:42:41 2020-01-01 00:14:20.000 2020-01-02 03:42:41.000 860 99761 50310.5 5031050 860 99761 50310.5 5031050 -32708 32227 4435.3 443530 -128 123 -1.18 -118 +861 100 10851 99762 2.58558 299.58558 151.08558 15108.55855 2.58558 299.58557 151.08558 15108.55841 2.58558 299.58558 151.08558000000002 15108.55800 2020-01-01 2020-01-02 2020-01-01 00:14:21 2020-01-02 03:42:42 2020-01-01 00:14:21.000 2020-01-02 03:42:42.000 861 99762 50311.5 5031150 861 99762 50311.5 5031150 -32707 32228 4436.3 443630 -127 124 -0.18 -18 +862 100 10852 99763 2.58858 299.58858 151.08858 15108.85885 2.58858 299.5886 151.08859 15108.85901 2.58858 299.58858 151.08858 15108.85800 2020-01-01 2020-01-02 2020-01-01 00:14:22 2020-01-02 03:42:43 2020-01-01 00:14:22.000 2020-01-02 03:42:43.000 862 99763 50312.5 5031250 862 99763 50312.5 5031250 -32706 32229 4437.3 443730 -126 125 0.82 82 863 100 10853 99764 2.59159 299.59159 151.09159 15109.15915 2.59159 299.59158 151.09159 15109.1593 2.59159 299.59159 151.09159 15109.15900 2020-01-01 2020-01-02 2020-01-01 00:14:23 2020-01-02 03:42:44 2020-01-01 00:14:23.000 2020-01-02 03:42:44.000 863 99764 50313.5 5031350 863 99764 50313.5 5031350 -32705 32230 4438.3 443830 -125 126 1.82 182 -864 100 10854 99765 2.59459 299.59459 151.09459 15109.45945 2.59459 299.5946 151.09459 15109.45941 2.59459 299.59459 151.0945900000001 15109.45900 2020-01-01 2020-01-02 2020-01-01 00:14:24 2020-01-02 03:42:45 2020-01-01 00:14:24.000 2020-01-02 03:42:45.000 864 99765 50314.5 5031450 864 99765 50314.5 5031450 -32704 32231 4439.3 443930 -124 127 2.82 282 +864 100 10854 99765 2.59459 299.59459 151.09459 15109.45945 2.59459 299.5946 151.09459 15109.45941 2.59459 299.59459 151.09459 15109.45900 2020-01-01 2020-01-02 2020-01-01 00:14:24 2020-01-02 03:42:45 2020-01-01 00:14:24.000 2020-01-02 03:42:45.000 864 99765 50314.5 5031450 864 99765 50314.5 5031450 -32704 32231 4439.3 443930 -124 127 2.82 282 865 100 10855 99766 2.59759 299.59759 151.09759 15109.75975 2.59759 299.5976 151.09759 15109.75958 2.59759 299.59759 151.09759 15109.75900 2020-01-01 2020-01-02 2020-01-01 00:14:25 2020-01-02 03:42:46 2020-01-01 00:14:25.000 2020-01-02 03:42:46.000 865 99766 50315.5 5031550 865 99766 50315.5 5031550 -32703 32232 4440.3 444030 -128 127 1.26 126 -866 100 10856 99767 2.6006 299.6006 151.1006 15110.06006 2.6006 299.6006 151.10059 15110.05988 2.60060 299.60060 151.10059999999987 15110.06000 2020-01-01 2020-01-02 2020-01-01 00:14:26 2020-01-02 03:42:47 2020-01-01 00:14:26.000 2020-01-02 03:42:47.000 866 99767 50316.5 5031650 866 99767 50316.5 5031650 -32702 32233 4441.3 444130 -128 127 -0.3 -30 -867 100 10857 99768 2.6036 299.6036 151.1036 15110.36036 2.6036 299.6036 151.1036 15110.36063 2.60360 299.60360 151.10360000000009 15110.36000 2020-01-01 2020-01-02 2020-01-01 00:14:27 2020-01-02 03:42:48 2020-01-01 00:14:27.000 2020-01-02 03:42:48.000 867 99768 50317.5 5031750 867 99768 50317.5 5031750 -32701 32234 4442.3 444230 -128 123 -1.86 -186 -868 100 10858 99769 2.6066 299.6066 151.1066 15110.66066 2.6066 299.6066 151.1066 15110.66078 2.60660 299.60660 151.10659999999993 15110.66000 2020-01-01 2020-01-02 2020-01-01 00:14:28 2020-01-02 03:42:49 2020-01-01 00:14:28.000 2020-01-02 03:42:49.000 868 99769 50318.5 5031850 868 99769 50318.5 5031850 -32700 32235 4443.3 444330 -127 124 -0.86 -86 -869 100 10859 99770 2.6096 299.6096 151.1096 15110.96096 2.6096 299.60962 151.1096 15110.96091 2.60960 299.60960 151.1095999999999 15110.96000 2020-01-01 2020-01-02 2020-01-01 00:14:29 2020-01-02 03:42:50 2020-01-01 00:14:29.000 2020-01-02 03:42:50.000 869 99770 50319.5 5031950 869 99770 50319.5 5031950 -32699 32236 4444.3 444430 -126 125 0.14 14 -87 102 10077 99987 0.26126 300.26126 150.26126 15176.38738 0.26126 300.26126 150.26126 15176.38736 0.26126 300.26126 150.26126000000002 15176.38726 2020-01-01 2020-01-02 2020-01-01 00:01:27 2020-01-02 03:46:27 2020-01-01 00:01:27.000 2020-01-02 03:46:27.000 87 99987 50037 5053737 87 99987 50037 5053737 -32482 32453 4616.009900990099 466217 -127 124 -2.128712871287129 -215 -870 100 10860 99771 2.61261 299.61261 151.11261 15111.26126 2.61261 299.6126 151.11261 15111.26105 2.61261 299.61261 151.11261000000007 15111.26100 2020-01-01 2020-01-02 2020-01-01 00:14:30 2020-01-02 03:42:51 2020-01-01 00:14:30.000 2020-01-02 03:42:51.000 870 99771 50320.5 5032050 870 99771 50320.5 5032050 -32698 32237 4445.3 444530 -125 126 1.14 114 -871 100 10861 99772 2.61561 299.61561 151.11561 15111.56156 2.61561 299.6156 151.11561 15111.56135 2.61561 299.61561 151.11561000000003 15111.56100 2020-01-01 2020-01-02 2020-01-01 00:14:31 2020-01-02 03:42:52 2020-01-01 00:14:31.000 2020-01-02 03:42:52.000 871 99772 50321.5 5032150 871 99772 50321.5 5032150 -32697 32238 4446.3 444630 -124 127 2.14 214 -872 100 10862 99773 2.61861 299.61861 151.11861 15111.86186 2.61861 299.61862 151.11862 15111.8621 2.61861 299.61861 151.11860999999982 15111.86100 2020-01-01 2020-01-02 2020-01-01 00:14:32 2020-01-02 03:42:53 2020-01-01 00:14:32.000 2020-01-02 03:42:53.000 872 99773 50322.5 5032250 872 99773 50322.5 5032250 -32696 32239 4447.3 444730 -128 127 0.58 58 -873 100 10863 99774 2.62162 299.62162 151.12162 15112.16216 2.62162 299.6216 151.12162 15112.16224 2.62162 299.62162 151.12161999999992 15112.16200 2020-01-01 2020-01-02 2020-01-01 00:14:33 2020-01-02 03:42:54 2020-01-01 00:14:33.000 2020-01-02 03:42:54.000 873 99774 50323.5 5032350 873 99774 50323.5 5032350 -32695 32240 4448.3 444830 -128 123 -0.98 -98 -874 100 10864 99775 2.62462 299.62462 151.12462 15112.46246 2.62462 299.62463 151.12462 15112.46238 2.62462 299.62462 151.12461999999996 15112.46200 2020-01-01 2020-01-02 2020-01-01 00:14:34 2020-01-02 03:42:55 2020-01-01 00:14:34.000 2020-01-02 03:42:55.000 874 99775 50324.5 5032450 874 99775 50324.5 5032450 -32694 32241 4449.3 444930 -127 124 0.02 2 -875 100 10865 99776 2.62762 299.62762 151.12762 15112.76276 2.62762 299.62762 151.12762 15112.76252 2.62762 299.62762 151.12761999999998 15112.76200 2020-01-01 2020-01-02 2020-01-01 00:14:35 2020-01-02 03:42:56 2020-01-01 00:14:35.000 2020-01-02 03:42:56.000 875 99776 50325.5 5032550 875 99776 50325.5 5032550 -32693 32242 4450.3 445030 -126 125 1.02 102 -876 100 10866 99777 2.63063 299.63063 151.13063 15113.06306 2.63063 299.63065 151.13063 15113.06327 2.63063 299.63063 151.1306299999999 15113.06300 2020-01-01 2020-01-02 2020-01-01 00:14:36 2020-01-02 03:42:57 2020-01-01 00:14:36.000 2020-01-02 03:42:57.000 876 99777 50326.5 5032650 876 99777 50326.5 5032650 -32692 32243 4451.3 445130 -125 126 2.02 202 -877 100 10867 99778 2.63363 299.63363 151.13363 15113.36336 2.63363 299.63364 151.13363 15113.36358 2.63363 299.63363 151.13363000000004 15113.36300 2020-01-01 2020-01-02 2020-01-01 00:14:37 2020-01-02 03:42:58 2020-01-01 00:14:37.000 2020-01-02 03:42:58.000 877 99778 50327.5 5032750 877 99778 50327.5 5032750 -32691 32244 4452.3 445230 -124 127 3.02 302 -878 100 10868 99779 2.63663 299.63663 151.13663 15113.66366 2.63663 299.63663 151.13663 15113.66371 2.63663 299.63663 151.13663000000008 15113.66300 2020-01-01 2020-01-02 2020-01-01 00:14:38 2020-01-02 03:42:59 2020-01-01 00:14:38.000 2020-01-02 03:42:59.000 878 99779 50328.5 5032850 878 99779 50328.5 5032850 -32690 32245 4453.3 445330 -128 127 1.46 146 -879 100 10869 99780 2.63963 299.63963 151.13963 15113.96396 2.63963 299.63965 151.13963 15113.96385 2.63963 299.63963 151.1396299999999 15113.96300 2020-01-01 2020-01-02 2020-01-01 00:14:39 2020-01-02 03:43:00 2020-01-01 00:14:39.000 2020-01-02 03:43:00.000 879 99780 50329.5 5032950 879 99780 50329.5 5032950 -32689 32246 4454.3 445430 -128 123 -0.1 -10 -88 102 10078 99988 0.26426 300.26426 150.26426 15176.69069 0.26426 300.26425 150.26426 15176.69066 0.26426 300.26426 150.26425999999995 15176.69026 2020-01-01 2020-01-02 2020-01-01 00:01:28 2020-01-02 03:46:28 2020-01-01 00:01:28.000 2020-01-02 03:46:28.000 88 99988 50038 5053838 88 99988 50038 5053838 -32481 32454 4617.009900990099 466318 -126 125 -1.1287128712871286 -114 +866 100 10856 99767 2.6006 299.6006 151.1006 15110.06006 2.6006 299.6006 151.10059 15110.05988 2.60060 299.60060 151.1006 15110.06000 2020-01-01 2020-01-02 2020-01-01 00:14:26 2020-01-02 03:42:47 2020-01-01 00:14:26.000 2020-01-02 03:42:47.000 866 99767 50316.5 5031650 866 99767 50316.5 5031650 -32702 32233 4441.3 444130 -128 127 -0.3 -30 +867 100 10857 99768 2.6036 299.6036 151.1036 15110.36036 2.6036 299.6036 151.1036 15110.36063 2.60360 299.60360 151.1036 15110.36000 2020-01-01 2020-01-02 2020-01-01 00:14:27 2020-01-02 03:42:48 2020-01-01 00:14:27.000 2020-01-02 03:42:48.000 867 99768 50317.5 5031750 867 99768 50317.5 5031750 -32701 32234 4442.3 444230 -128 123 -1.86 -186 +868 100 10858 99769 2.6066 299.6066 151.1066 15110.66066 2.6066 299.6066 151.1066 15110.66078 2.60660 299.60660 151.1066 15110.66000 2020-01-01 2020-01-02 2020-01-01 00:14:28 2020-01-02 03:42:49 2020-01-01 00:14:28.000 2020-01-02 03:42:49.000 868 99769 50318.5 5031850 868 99769 50318.5 5031850 -32700 32235 4443.3 444330 -127 124 -0.86 -86 +869 100 10859 99770 2.6096 299.6096 151.1096 15110.96096 2.6096 299.60962 151.1096 15110.96091 2.60960 299.60960 151.1096 15110.96000 2020-01-01 2020-01-02 2020-01-01 00:14:29 2020-01-02 03:42:50 2020-01-01 00:14:29.000 2020-01-02 03:42:50.000 869 99770 50319.5 5031950 869 99770 50319.5 5031950 -32699 32236 4444.3 444430 -126 125 0.14 14 +87 102 10077 99987 0.26126 300.26126 150.26126 15176.38738 0.26126 300.26126 150.26126 15176.38736 0.26126 300.26126 150.26126 15176.38726 2020-01-01 2020-01-02 2020-01-01 00:01:27 2020-01-02 03:46:27 2020-01-01 00:01:27.000 2020-01-02 03:46:27.000 87 99987 50037 5053737 87 99987 50037 5053737 -32482 32453 4616.009900990099 466217 -127 124 -2.128712871287129 -215 +870 100 10860 99771 2.61261 299.61261 151.11261 15111.26126 2.61261 299.6126 151.11261 15111.26105 2.61261 299.61261 151.11261000000002 15111.26100 2020-01-01 2020-01-02 2020-01-01 00:14:30 2020-01-02 03:42:51 2020-01-01 00:14:30.000 2020-01-02 03:42:51.000 870 99771 50320.5 5032050 870 99771 50320.5 5032050 -32698 32237 4445.3 444530 -125 126 1.14 114 +871 100 10861 99772 2.61561 299.61561 151.11561 15111.56156 2.61561 299.6156 151.11561 15111.56135 2.61561 299.61561 151.11561 15111.56100 2020-01-01 2020-01-02 2020-01-01 00:14:31 2020-01-02 03:42:52 2020-01-01 00:14:31.000 2020-01-02 03:42:52.000 871 99772 50321.5 5032150 871 99772 50321.5 5032150 -32697 32238 4446.3 444630 -124 127 2.14 214 +872 100 10862 99773 2.61861 299.61861 151.11861 15111.86186 2.61861 299.61862 151.11862 15111.8621 2.61861 299.61861 151.11861000000002 15111.86100 2020-01-01 2020-01-02 2020-01-01 00:14:32 2020-01-02 03:42:53 2020-01-01 00:14:32.000 2020-01-02 03:42:53.000 872 99773 50322.5 5032250 872 99773 50322.5 5032250 -32696 32239 4447.3 444730 -128 127 0.58 58 +873 100 10863 99774 2.62162 299.62162 151.12162 15112.16216 2.62162 299.6216 151.12162 15112.16224 2.62162 299.62162 151.12162 15112.16200 2020-01-01 2020-01-02 2020-01-01 00:14:33 2020-01-02 03:42:54 2020-01-01 00:14:33.000 2020-01-02 03:42:54.000 873 99774 50323.5 5032350 873 99774 50323.5 5032350 -32695 32240 4448.3 444830 -128 123 -0.98 -98 +874 100 10864 99775 2.62462 299.62462 151.12462 15112.46246 2.62462 299.62463 151.12462 15112.46238 2.62462 299.62462 151.12462 15112.46200 2020-01-01 2020-01-02 2020-01-01 00:14:34 2020-01-02 03:42:55 2020-01-01 00:14:34.000 2020-01-02 03:42:55.000 874 99775 50324.5 5032450 874 99775 50324.5 5032450 -32694 32241 4449.3 444930 -127 124 0.02 2 +875 100 10865 99776 2.62762 299.62762 151.12762 15112.76276 2.62762 299.62762 151.12762 15112.76252 2.62762 299.62762 151.12762 15112.76200 2020-01-01 2020-01-02 2020-01-01 00:14:35 2020-01-02 03:42:56 2020-01-01 00:14:35.000 2020-01-02 03:42:56.000 875 99776 50325.5 5032550 875 99776 50325.5 5032550 -32693 32242 4450.3 445030 -126 125 1.02 102 +876 100 10866 99777 2.63063 299.63063 151.13063 15113.06306 2.63063 299.63065 151.13063 15113.06327 2.63063 299.63063 151.13063 15113.06300 2020-01-01 2020-01-02 2020-01-01 00:14:36 2020-01-02 03:42:57 2020-01-01 00:14:36.000 2020-01-02 03:42:57.000 876 99777 50326.5 5032650 876 99777 50326.5 5032650 -32692 32243 4451.3 445130 -125 126 2.02 202 +877 100 10867 99778 2.63363 299.63363 151.13363 15113.36336 2.63363 299.63364 151.13363 15113.36358 2.63363 299.63363 151.13362999999998 15113.36300 2020-01-01 2020-01-02 2020-01-01 00:14:37 2020-01-02 03:42:58 2020-01-01 00:14:37.000 2020-01-02 03:42:58.000 877 99778 50327.5 5032750 877 99778 50327.5 5032750 -32691 32244 4452.3 445230 -124 127 3.02 302 +878 100 10868 99779 2.63663 299.63663 151.13663 15113.66366 2.63663 299.63663 151.13663 15113.66371 2.63663 299.63663 151.13663 15113.66300 2020-01-01 2020-01-02 2020-01-01 00:14:38 2020-01-02 03:42:59 2020-01-01 00:14:38.000 2020-01-02 03:42:59.000 878 99779 50328.5 5032850 878 99779 50328.5 5032850 -32690 32245 4453.3 445330 -128 127 1.46 146 +879 100 10869 99780 2.63963 299.63963 151.13963 15113.96396 2.63963 299.63965 151.13963 15113.96385 2.63963 299.63963 151.13963 15113.96300 2020-01-01 2020-01-02 2020-01-01 00:14:39 2020-01-02 03:43:00 2020-01-01 00:14:39.000 2020-01-02 03:43:00.000 879 99780 50329.5 5032950 879 99780 50329.5 5032950 -32689 32246 4454.3 445430 -128 123 -0.1 -10 +88 102 10078 99988 0.26426 300.26426 150.26426 15176.69069 0.26426 300.26425 150.26426 15176.69066 0.26426 300.26426 150.26426 15176.69026 2020-01-01 2020-01-02 2020-01-01 00:01:28 2020-01-02 03:46:28 2020-01-01 00:01:28.000 2020-01-02 03:46:28.000 88 99988 50038 5053838 88 99988 50038 5053838 -32481 32454 4617.009900990099 466318 -126 125 -1.1287128712871286 -114 880 100 10870 99781 2.64264 299.64264 151.14264 15114.26426 2.64264 299.64264 151.14263 15114.26399 2.64264 299.64264 151.14264 15114.26400 2020-01-01 2020-01-02 2020-01-01 00:14:40 2020-01-02 03:43:01 2020-01-01 00:14:40.000 2020-01-02 03:43:01.000 880 99781 50330.5 5033050 880 99781 50330.5 5033050 -32688 32247 4455.3 445530 -127 124 0.9 90 -881 100 10871 99782 2.64564 299.64564 151.14564 15114.56456 2.64564 299.64566 151.14564 15114.56474 2.64564 299.64564 151.14564000000007 15114.56400 2020-01-01 2020-01-02 2020-01-01 00:14:41 2020-01-02 03:43:02 2020-01-01 00:14:41.000 2020-01-02 03:43:02.000 881 99782 50331.5 5033150 881 99782 50331.5 5033150 -32687 32248 4456.3 445630 -126 125 1.9 190 -882 100 10872 99783 2.64864 299.64864 151.14864 15114.86486 2.64864 299.64865 151.14865 15114.86504 2.64864 299.64864 151.14863999999997 15114.86400 2020-01-01 2020-01-02 2020-01-01 00:14:42 2020-01-02 03:43:03 2020-01-01 00:14:42.000 2020-01-02 03:43:03.000 882 99783 50332.5 5033250 882 99783 50332.5 5033250 -32686 32249 4457.3 445730 -125 126 2.9 290 -883 100 10873 99784 2.65165 299.65165 151.15165 15115.16516 2.65165 299.65164 151.15165 15115.16522 2.65165 299.65165 151.1516499999999 15115.16500 2020-01-01 2020-01-02 2020-01-01 00:14:43 2020-01-02 03:43:04 2020-01-01 00:14:43.000 2020-01-02 03:43:04.000 883 99784 50333.5 5033350 883 99784 50333.5 5033350 -32685 32250 4458.3 445830 -124 127 3.9 390 -884 100 10874 99785 2.65465 299.65465 151.15465 15115.46546 2.65465 299.65466 151.15465 15115.46532 2.65465 299.65465 151.15465000000012 15115.46500 2020-01-01 2020-01-02 2020-01-01 00:14:44 2020-01-02 03:43:05 2020-01-01 00:14:44.000 2020-01-02 03:43:05.000 884 99785 50334.5 5033450 884 99785 50334.5 5033450 -32684 32251 4459.3 445930 -128 127 2.34 234 -885 100 10875 99786 2.65765 299.65765 151.15765 15115.76576 2.65765 299.65765 151.15765 15115.76562 2.65765 299.65765 151.15764999999996 15115.76500 2020-01-01 2020-01-02 2020-01-01 00:14:45 2020-01-02 03:43:06 2020-01-01 00:14:45.000 2020-01-02 03:43:06.000 885 99786 50335.5 5033550 885 99786 50335.5 5033550 -32683 32252 4460.3 446030 -128 123 0.78 78 -886 100 10876 99787 2.66066 299.66066 151.16066 15116.06606 2.66066 299.66068 151.16066 15116.06621 2.66066 299.66066 151.16065999999984 15116.06600 2020-01-01 2020-01-02 2020-01-01 00:14:46 2020-01-02 03:43:07 2020-01-01 00:14:46.000 2020-01-02 03:43:07.000 886 99787 50336.5 5033650 886 99787 50336.5 5033650 -32682 32253 4461.3 446130 -127 124 1.78 178 -887 100 10877 99788 2.66366 299.66366 151.16366 15116.36636 2.66366 299.66367 151.16366 15116.36651 2.66366 299.66366 151.16366000000008 15116.36600 2020-01-01 2020-01-02 2020-01-01 00:14:47 2020-01-02 03:43:08 2020-01-01 00:14:47.000 2020-01-02 03:43:08.000 887 99788 50337.5 5033750 887 99788 50337.5 5033750 -32681 32254 4462.3 446230 -126 125 2.78 278 -888 100 10878 99789 2.66666 299.66666 151.16666 15116.66666 2.66666 299.66666 151.16666 15116.66669 2.66666 299.66666 151.16666000000004 15116.66600 2020-01-01 2020-01-02 2020-01-01 00:14:48 2020-01-02 03:43:09 2020-01-01 00:14:48.000 2020-01-02 03:43:09.000 888 99789 50338.5 5033850 888 99789 50338.5 5033850 -32680 32255 4463.3 446330 -125 126 3.78 378 -889 100 10879 99790 2.66966 299.66966 151.16966 15116.96696 2.66966 299.66968 151.16966 15116.96679 2.66966 299.66966 151.16965999999985 15116.96600 2020-01-01 2020-01-02 2020-01-01 00:14:49 2020-01-02 03:43:10 2020-01-01 00:14:49.000 2020-01-02 03:43:10.000 889 99790 50339.5 5033950 889 99790 50339.5 5033950 -32679 32256 4464.3 446430 -124 127 4.78 478 -89 102 10079 99989 0.26726 300.26726 150.26726 15176.99399 0.26726 300.26727 150.26727 15176.9943 0.26726 300.26726 150.26726000000002 15176.99326 2020-01-01 2020-01-02 2020-01-01 00:01:29 2020-01-02 03:46:29 2020-01-01 00:01:29.000 2020-01-02 03:46:29.000 89 99989 50039 5053939 89 99989 50039 5053939 -32480 32455 4618.009900990099 466419 -125 126 -0.12871287128712872 -13 -890 100 10880 99791 2.67267 299.67267 151.17267 15117.26726 2.67267 299.67267 151.17267 15117.26708 2.67267 299.67267 151.17266999999995 15117.26700 2020-01-01 2020-01-02 2020-01-01 00:14:50 2020-01-02 03:43:11 2020-01-01 00:14:50.000 2020-01-02 03:43:11.000 890 99791 50340.5 5034050 890 99791 50340.5 5034050 -32678 32257 4465.3 446530 -128 127 3.22 322 +881 100 10871 99782 2.64564 299.64564 151.14564 15114.56456 2.64564 299.64566 151.14564 15114.56474 2.64564 299.64564 151.14564000000001 15114.56400 2020-01-01 2020-01-02 2020-01-01 00:14:41 2020-01-02 03:43:02 2020-01-01 00:14:41.000 2020-01-02 03:43:02.000 881 99782 50331.5 5033150 881 99782 50331.5 5033150 -32687 32248 4456.3 445630 -126 125 1.9 190 +882 100 10872 99783 2.64864 299.64864 151.14864 15114.86486 2.64864 299.64865 151.14865 15114.86504 2.64864 299.64864 151.14864 15114.86400 2020-01-01 2020-01-02 2020-01-01 00:14:42 2020-01-02 03:43:03 2020-01-01 00:14:42.000 2020-01-02 03:43:03.000 882 99783 50332.5 5033250 882 99783 50332.5 5033250 -32686 32249 4457.3 445730 -125 126 2.9 290 +883 100 10873 99784 2.65165 299.65165 151.15165 15115.16516 2.65165 299.65164 151.15165 15115.16522 2.65165 299.65165 151.15165000000002 15115.16500 2020-01-01 2020-01-02 2020-01-01 00:14:43 2020-01-02 03:43:04 2020-01-01 00:14:43.000 2020-01-02 03:43:04.000 883 99784 50333.5 5033350 883 99784 50333.5 5033350 -32685 32250 4458.3 445830 -124 127 3.9 390 +884 100 10874 99785 2.65465 299.65465 151.15465 15115.46546 2.65465 299.65466 151.15465 15115.46532 2.65465 299.65465 151.15465 15115.46500 2020-01-01 2020-01-02 2020-01-01 00:14:44 2020-01-02 03:43:05 2020-01-01 00:14:44.000 2020-01-02 03:43:05.000 884 99785 50334.5 5033450 884 99785 50334.5 5033450 -32684 32251 4459.3 445930 -128 127 2.34 234 +885 100 10875 99786 2.65765 299.65765 151.15765 15115.76576 2.65765 299.65765 151.15765 15115.76562 2.65765 299.65765 151.15765 15115.76500 2020-01-01 2020-01-02 2020-01-01 00:14:45 2020-01-02 03:43:06 2020-01-01 00:14:45.000 2020-01-02 03:43:06.000 885 99786 50335.5 5033550 885 99786 50335.5 5033550 -32683 32252 4460.3 446030 -128 123 0.78 78 +886 100 10876 99787 2.66066 299.66066 151.16066 15116.06606 2.66066 299.66068 151.16066 15116.06621 2.66066 299.66066 151.16066 15116.06600 2020-01-01 2020-01-02 2020-01-01 00:14:46 2020-01-02 03:43:07 2020-01-01 00:14:46.000 2020-01-02 03:43:07.000 886 99787 50336.5 5033650 886 99787 50336.5 5033650 -32682 32253 4461.3 446130 -127 124 1.78 178 +887 100 10877 99788 2.66366 299.66366 151.16366 15116.36636 2.66366 299.66367 151.16366 15116.36651 2.66366 299.66366 151.16366 15116.36600 2020-01-01 2020-01-02 2020-01-01 00:14:47 2020-01-02 03:43:08 2020-01-01 00:14:47.000 2020-01-02 03:43:08.000 887 99788 50337.5 5033750 887 99788 50337.5 5033750 -32681 32254 4462.3 446230 -126 125 2.78 278 +888 100 10878 99789 2.66666 299.66666 151.16666 15116.66666 2.66666 299.66666 151.16666 15116.66669 2.66666 299.66666 151.16665999999998 15116.66600 2020-01-01 2020-01-02 2020-01-01 00:14:48 2020-01-02 03:43:09 2020-01-01 00:14:48.000 2020-01-02 03:43:09.000 888 99789 50338.5 5033850 888 99789 50338.5 5033850 -32680 32255 4463.3 446330 -125 126 3.78 378 +889 100 10879 99790 2.66966 299.66966 151.16966 15116.96696 2.66966 299.66968 151.16966 15116.96679 2.66966 299.66966 151.16966 15116.96600 2020-01-01 2020-01-02 2020-01-01 00:14:49 2020-01-02 03:43:10 2020-01-01 00:14:49.000 2020-01-02 03:43:10.000 889 99790 50339.5 5033950 889 99790 50339.5 5033950 -32679 32256 4464.3 446430 -124 127 4.78 478 +89 102 10079 99989 0.26726 300.26726 150.26726 15176.99399 0.26726 300.26727 150.26727 15176.9943 0.26726 300.26726 150.26726 15176.99326 2020-01-01 2020-01-02 2020-01-01 00:01:29 2020-01-02 03:46:29 2020-01-01 00:01:29.000 2020-01-02 03:46:29.000 89 99989 50039 5053939 89 99989 50039 5053939 -32480 32455 4618.009900990099 466419 -125 126 -0.12871287128712872 -13 +890 100 10880 99791 2.67267 299.67267 151.17267 15117.26726 2.67267 299.67267 151.17267 15117.26708 2.67267 299.67267 151.17267 15117.26700 2020-01-01 2020-01-02 2020-01-01 00:14:50 2020-01-02 03:43:11 2020-01-01 00:14:50.000 2020-01-02 03:43:11.000 890 99791 50340.5 5034050 890 99791 50340.5 5034050 -32678 32257 4465.3 446530 -128 127 3.22 322 891 100 10881 99792 2.67567 299.67567 151.17567 15117.56756 2.67567 299.6757 151.17567 15117.56768 2.67567 299.67567 151.17567 15117.56700 2020-01-01 2020-01-02 2020-01-01 00:14:51 2020-01-02 03:43:12 2020-01-01 00:14:51.000 2020-01-02 03:43:12.000 891 99792 50341.5 5034150 891 99792 50341.5 5034150 -32677 32258 4466.3 446630 -128 127 1.66 166 892 100 10882 99793 2.67867 299.67867 151.17867 15117.86786 2.67867 299.67868 151.17868 15117.86802 2.67867 299.67867 151.17867 15117.86700 2020-01-01 2020-01-02 2020-01-01 00:14:52 2020-01-02 03:43:13 2020-01-01 00:14:52.000 2020-01-02 03:43:13.000 892 99793 50342.5 5034250 892 99793 50342.5 5034250 -32676 32259 4467.3 446730 -128 124 0.1 10 -893 100 10883 99794 2.68168 299.68168 151.18168 15118.16816 2.68168 299.68167 151.18168 15118.16816 2.68168 299.68168 151.18167999999991 15118.16800 2020-01-01 2020-01-02 2020-01-01 00:14:53 2020-01-02 03:43:14 2020-01-01 00:14:53.000 2020-01-02 03:43:14.000 893 99794 50343.5 5034350 893 99794 50343.5 5034350 -32675 32260 4468.3 446830 -127 125 1.1 110 -894 100 10884 99795 2.68468 299.68468 151.18468 15118.46846 2.68468 299.6847 151.18468 15118.46826 2.68468 299.68468 151.18468000000007 15118.46800 2020-01-01 2020-01-02 2020-01-01 00:14:54 2020-01-02 03:43:15 2020-01-01 00:14:54.000 2020-01-02 03:43:15.000 894 99795 50344.5 5034450 894 99795 50344.5 5034450 -32674 32261 4469.3 446930 -126 126 2.1 210 -895 100 10885 99796 2.68768 299.68768 151.18768 15118.76876 2.68768 299.68768 151.18768 15118.76855 2.68768 299.68768 151.18767999999992 15118.76800 2020-01-01 2020-01-02 2020-01-01 00:14:55 2020-01-02 03:43:16 2020-01-01 00:14:55.000 2020-01-02 03:43:16.000 895 99796 50345.5 5034550 895 99796 50345.5 5034550 -32673 32262 4470.3 447030 -125 127 3.1 310 +893 100 10883 99794 2.68168 299.68168 151.18168 15118.16816 2.68168 299.68167 151.18168 15118.16816 2.68168 299.68168 151.18168 15118.16800 2020-01-01 2020-01-02 2020-01-01 00:14:53 2020-01-02 03:43:14 2020-01-01 00:14:53.000 2020-01-02 03:43:14.000 893 99794 50343.5 5034350 893 99794 50343.5 5034350 -32675 32260 4468.3 446830 -127 125 1.1 110 +894 100 10884 99795 2.68468 299.68468 151.18468 15118.46846 2.68468 299.6847 151.18468 15118.46826 2.68468 299.68468 151.18468000000001 15118.46800 2020-01-01 2020-01-02 2020-01-01 00:14:54 2020-01-02 03:43:15 2020-01-01 00:14:54.000 2020-01-02 03:43:15.000 894 99795 50344.5 5034450 894 99795 50344.5 5034450 -32674 32261 4469.3 446930 -126 126 2.1 210 +895 100 10885 99796 2.68768 299.68768 151.18768 15118.76876 2.68768 299.68768 151.18768 15118.76855 2.68768 299.68768 151.18768 15118.76800 2020-01-01 2020-01-02 2020-01-01 00:14:55 2020-01-02 03:43:16 2020-01-01 00:14:55.000 2020-01-02 03:43:16.000 895 99796 50345.5 5034550 895 99796 50345.5 5034550 -32673 32262 4470.3 447030 -125 127 3.1 310 896 100 10886 99797 2.69069 299.69069 151.19069 15119.06906 2.69069 299.6907 151.19069 15119.06915 2.69069 299.69069 151.19069 15119.06900 2020-01-01 2020-01-02 2020-01-01 00:14:56 2020-01-02 03:43:17 2020-01-01 00:14:56.000 2020-01-02 03:43:17.000 896 99797 50346.5 5034650 896 99797 50346.5 5034650 -32672 32263 4471.3 447130 -128 127 1.54 154 -897 100 10887 99798 2.69369 299.69369 151.19369 15119.36936 2.69369 299.6937 151.19369 15119.36949 2.69369 299.69369 151.19369000000003 15119.36900 2020-01-01 2020-01-02 2020-01-01 00:14:57 2020-01-02 03:43:18 2020-01-01 00:14:57.000 2020-01-02 03:43:18.000 897 99798 50347.5 5034750 897 99798 50347.5 5034750 -32671 32264 4472.3 447230 -128 127 -0.02 -2 +897 100 10887 99798 2.69369 299.69369 151.19369 15119.36936 2.69369 299.6937 151.19369 15119.36949 2.69369 299.69369 151.19369 15119.36900 2020-01-01 2020-01-02 2020-01-01 00:14:57 2020-01-02 03:43:18 2020-01-01 00:14:57.000 2020-01-02 03:43:18.000 897 99798 50347.5 5034750 897 99798 50347.5 5034750 -32671 32264 4472.3 447230 -128 127 -0.02 -2 898 100 10888 99799 2.69669 299.69669 151.19669 15119.66966 2.69669 299.6967 151.19669 15119.66963 2.69669 299.69669 151.19669 15119.66900 2020-01-01 2020-01-02 2020-01-01 00:14:58 2020-01-02 03:43:19 2020-01-01 00:14:58.000 2020-01-02 03:43:19.000 898 99799 50348.5 5034850 898 99799 50348.5 5034850 -32670 32265 4473.3 447330 -128 123 -1.58 -158 899 100 10889 99800 2.69969 299.69969 151.19969 15119.96996 2.69969 299.6997 151.1997 15119.97038 2.69969 299.69969 151.19969 15119.96900 2020-01-01 2020-01-02 2020-01-01 00:14:59 2020-01-02 03:43:20 2020-01-01 00:14:59.000 2020-01-02 03:43:20.000 899 99800 50349.5 5034950 899 99800 50349.5 5034950 -32669 32266 4474.3 447430 -127 124 -0.58 -58 -9 102 1008 9999 0.02702 300.02702 150.02702 15152.72972 0.02702 300.02704 150.02702 15152.72966 0.02702 300.02702 150.02701999999985 15152.72902 2020-01-01 2020-01-02 2020-01-01 00:00:09 2020-01-02 03:45:09 2020-01-01 00:00:09.000 2020-01-02 03:45:09.000 9 99909 49959 5045859 9 99909 49959 5045859 -32560 32375 4538.009900990099 458339 -124 127 0.9801980198019802 99 -90 102 10080 99990 0.27027 300.27027 150.27027 15177.29729 0.27027 300.27026 150.27026 15177.29694 0.27027 300.27027 150.27027000000007 15177.29727 2020-01-01 2020-01-02 2020-01-01 00:01:30 2020-01-02 03:46:30 2020-01-01 00:01:30.000 2020-01-02 03:46:30.000 90 99990 50040 5054040 90 99990 50040 5054040 -32479 32456 4619.009900990099 466520 -124 127 0.8712871287128713 88 -900 100 10890 99801 2.7027 299.7027 151.2027 15120.27027 2.7027 299.7027 151.2027 15120.27003 2.70270 299.70270 151.2026999999999 15120.27000 2020-01-01 2020-01-02 2020-01-01 00:15:00 2020-01-02 03:43:21 2020-01-01 00:15:00.000 2020-01-02 03:43:21.000 900 99801 50350.5 5035050 900 99801 50350.5 5035050 -32668 32267 4475.3 447530 -126 125 0.42 42 -901 100 10891 99802 2.7057 299.7057 151.2057 15120.57057 2.7057 299.70572 151.2057 15120.57066 2.70570 299.70570 151.20570000000015 15120.57000 2020-01-01 2020-01-02 2020-01-01 00:15:01 2020-01-02 03:43:22 2020-01-01 00:15:01.000 2020-01-02 03:43:22.000 901 99802 50351.5 5035150 901 99802 50351.5 5035150 -32667 32268 4476.3 447630 -125 126 1.42 142 -902 100 10892 99803 2.7087 299.7087 151.2087 15120.87087 2.7087 299.7087 151.2087 15120.87095 2.70870 299.70870 151.20869999999996 15120.87000 2020-01-01 2020-01-02 2020-01-01 00:15:02 2020-01-02 03:43:23 2020-01-01 00:15:02.000 2020-01-02 03:43:23.000 902 99803 50352.5 5035250 902 99803 50352.5 5035250 -32666 32269 4477.3 447730 -124 127 2.42 242 -903 100 10893 99804 2.71171 299.71171 151.21171 15121.17117 2.71171 299.7117 151.21171 15121.1711 2.71171 299.71171 151.21170999999987 15121.17100 2020-01-01 2020-01-02 2020-01-01 00:15:03 2020-01-02 03:43:24 2020-01-01 00:15:03.000 2020-01-02 03:43:24.000 903 99804 50353.5 5035350 903 99804 50353.5 5035350 -32665 32270 4478.3 447830 -128 127 0.86 86 -904 100 10894 99805 2.71471 299.71471 151.21471 15121.47147 2.71471 299.71472 151.21471 15121.47185 2.71471 299.71471 151.21471000000008 15121.47100 2020-01-01 2020-01-02 2020-01-01 00:15:04 2020-01-02 03:43:25 2020-01-01 00:15:04.000 2020-01-02 03:43:25.000 904 99805 50354.5 5035450 904 99805 50354.5 5035450 -32664 32271 4479.3 447930 -128 123 -0.7 -70 -905 100 10895 99806 2.71771 299.71771 151.21771 15121.77177 2.71771 299.7177 151.21771 15121.77149 2.71771 299.71771 151.21771000000004 15121.77100 2020-01-01 2020-01-02 2020-01-01 00:15:05 2020-01-02 03:43:26 2020-01-01 00:15:05.000 2020-01-02 03:43:26.000 905 99806 50355.5 5035550 905 99806 50355.5 5035550 -32663 32272 4480.3 448030 -127 124 0.3 30 -906 100 10896 99807 2.72072 299.72072 151.22072 15122.07207 2.72072 299.72073 151.22072 15122.07212 2.72072 299.72072 151.22071999999991 15122.07200 2020-01-01 2020-01-02 2020-01-01 00:15:06 2020-01-02 03:43:27 2020-01-01 00:15:06.000 2020-01-02 03:43:27.000 906 99807 50356.5 5035650 906 99807 50356.5 5035650 -32662 32273 4481.3 448130 -126 125 1.3 130 +9 102 1008 9999 0.02702 300.02702 150.02702 15152.72972 0.02702 300.02704 150.02702 15152.72966 0.02702 300.02702 150.02702 15152.72902 2020-01-01 2020-01-02 2020-01-01 00:00:09 2020-01-02 03:45:09 2020-01-01 00:00:09.000 2020-01-02 03:45:09.000 9 99909 49959 5045859 9 99909 49959 5045859 -32560 32375 4538.009900990099 458339 -124 127 0.9801980198019802 99 +90 102 10080 99990 0.27027 300.27027 150.27027 15177.29729 0.27027 300.27026 150.27026 15177.29694 0.27027 300.27027 150.27026999999998 15177.29727 2020-01-01 2020-01-02 2020-01-01 00:01:30 2020-01-02 03:46:30 2020-01-01 00:01:30.000 2020-01-02 03:46:30.000 90 99990 50040 5054040 90 99990 50040 5054040 -32479 32456 4619.009900990099 466520 -124 127 0.8712871287128713 88 +900 100 10890 99801 2.7027 299.7027 151.2027 15120.27027 2.7027 299.7027 151.2027 15120.27003 2.70270 299.70270 151.2027 15120.27000 2020-01-01 2020-01-02 2020-01-01 00:15:00 2020-01-02 03:43:21 2020-01-01 00:15:00.000 2020-01-02 03:43:21.000 900 99801 50350.5 5035050 900 99801 50350.5 5035050 -32668 32267 4475.3 447530 -126 125 0.42 42 +901 100 10891 99802 2.7057 299.7057 151.2057 15120.57057 2.7057 299.70572 151.2057 15120.57066 2.70570 299.70570 151.2057 15120.57000 2020-01-01 2020-01-02 2020-01-01 00:15:01 2020-01-02 03:43:22 2020-01-01 00:15:01.000 2020-01-02 03:43:22.000 901 99802 50351.5 5035150 901 99802 50351.5 5035150 -32667 32268 4476.3 447630 -125 126 1.42 142 +902 100 10892 99803 2.7087 299.7087 151.2087 15120.87087 2.7087 299.7087 151.2087 15120.87095 2.70870 299.70870 151.20870000000002 15120.87000 2020-01-01 2020-01-02 2020-01-01 00:15:02 2020-01-02 03:43:23 2020-01-01 00:15:02.000 2020-01-02 03:43:23.000 902 99803 50352.5 5035250 902 99803 50352.5 5035250 -32666 32269 4477.3 447730 -124 127 2.42 242 +903 100 10893 99804 2.71171 299.71171 151.21171 15121.17117 2.71171 299.7117 151.21171 15121.1711 2.71171 299.71171 151.21171 15121.17100 2020-01-01 2020-01-02 2020-01-01 00:15:03 2020-01-02 03:43:24 2020-01-01 00:15:03.000 2020-01-02 03:43:24.000 903 99804 50353.5 5035350 903 99804 50353.5 5035350 -32665 32270 4478.3 447830 -128 127 0.86 86 +904 100 10894 99805 2.71471 299.71471 151.21471 15121.47147 2.71471 299.71472 151.21471 15121.47185 2.71471 299.71471 151.21471 15121.47100 2020-01-01 2020-01-02 2020-01-01 00:15:04 2020-01-02 03:43:25 2020-01-01 00:15:04.000 2020-01-02 03:43:25.000 904 99805 50354.5 5035450 904 99805 50354.5 5035450 -32664 32271 4479.3 447930 -128 123 -0.7 -70 +905 100 10895 99806 2.71771 299.71771 151.21771 15121.77177 2.71771 299.7177 151.21771 15121.77149 2.71771 299.71771 151.21771 15121.77100 2020-01-01 2020-01-02 2020-01-01 00:15:05 2020-01-02 03:43:26 2020-01-01 00:15:05.000 2020-01-02 03:43:26.000 905 99806 50355.5 5035550 905 99806 50355.5 5035550 -32663 32272 4480.3 448030 -127 124 0.3 30 +906 100 10896 99807 2.72072 299.72072 151.22072 15122.07207 2.72072 299.72073 151.22072 15122.07212 2.72072 299.72072 151.22072 15122.07200 2020-01-01 2020-01-02 2020-01-01 00:15:06 2020-01-02 03:43:27 2020-01-01 00:15:06.000 2020-01-02 03:43:27.000 906 99807 50356.5 5035650 906 99807 50356.5 5035650 -32662 32273 4481.3 448130 -126 125 1.3 130 907 100 10897 99808 2.72372 299.72372 151.22372 15122.37237 2.72372 299.72372 151.22372 15122.37243 2.72372 299.72372 151.22372 15122.37200 2020-01-01 2020-01-02 2020-01-01 00:15:07 2020-01-02 03:43:28 2020-01-01 00:15:07.000 2020-01-02 03:43:28.000 907 99808 50357.5 5035750 907 99808 50357.5 5035750 -32661 32274 4482.3 448230 -125 126 2.3 230 -908 100 10898 99809 2.72672 299.72672 151.22672 15122.67267 2.72672 299.7267 151.22672 15122.67272 2.72672 299.72672 151.22672000000003 15122.67200 2020-01-01 2020-01-02 2020-01-01 00:15:08 2020-01-02 03:43:29 2020-01-01 00:15:08.000 2020-01-02 03:43:29.000 908 99809 50358.5 5035850 908 99809 50358.5 5035850 -32660 32275 4483.3 448330 -124 127 3.3 330 -909 100 10899 99810 2.72972 299.72972 151.22972 15122.97297 2.72972 299.72974 151.22973 15122.97332 2.72972 299.72972 151.22971999999993 15122.97200 2020-01-01 2020-01-02 2020-01-01 00:15:09 2020-01-02 03:43:30 2020-01-01 00:15:09.000 2020-01-02 03:43:30.000 909 99810 50359.5 5035950 909 99810 50359.5 5035950 -32659 32276 4484.3 448430 -128 127 1.74 174 -91 102 10081 99991 0.27327 300.27327 150.27327 15177.6006 0.27327 300.2733 150.27327 15177.60054 0.27327 300.27327 150.2732699999999 15177.60027 2020-01-01 2020-01-02 2020-01-01 00:01:31 2020-01-02 03:46:31 2020-01-01 00:01:31.000 2020-01-02 03:46:31.000 91 99991 50041 5054141 91 99991 50041 5054141 -32478 32457 4620.009900990099 466621 -128 127 -0.6633663366336634 -67 -910 100 10900 99811 2.73273 299.73273 151.23273 15123.27327 2.73273 299.73273 151.23272 15123.27296 2.73273 299.73273 151.23272999999995 15123.27300 2020-01-01 2020-01-02 2020-01-01 00:15:10 2020-01-02 03:43:31 2020-01-01 00:15:10.000 2020-01-02 03:43:31.000 910 99811 50360.5 5036050 910 99811 50360.5 5036050 -32658 32277 4485.3 448530 -128 123 0.18 18 -911 100 10901 99812 2.73573 299.73573 151.23573 15123.57357 2.73573 299.73575 151.23573 15123.57359 2.73573 299.73573 151.2357300000001 15123.57300 2020-01-01 2020-01-02 2020-01-01 00:15:11 2020-01-02 03:43:32 2020-01-01 00:15:11.000 2020-01-02 03:43:32.000 911 99812 50361.5 5036150 911 99812 50361.5 5036150 -32657 32278 4486.3 448630 -127 124 1.18 118 -912 100 10902 99813 2.73873 299.73873 151.23873 15123.87387 2.73873 299.73874 151.23873 15123.8739 2.73873 299.73873 151.23872999999992 15123.87300 2020-01-01 2020-01-02 2020-01-01 00:15:12 2020-01-02 03:43:33 2020-01-01 00:15:12.000 2020-01-02 03:43:33.000 912 99813 50362.5 5036250 912 99813 50362.5 5036250 -32656 32279 4487.3 448730 -126 125 2.18 218 +908 100 10898 99809 2.72672 299.72672 151.22672 15122.67267 2.72672 299.7267 151.22672 15122.67272 2.72672 299.72672 151.22672 15122.67200 2020-01-01 2020-01-02 2020-01-01 00:15:08 2020-01-02 03:43:29 2020-01-01 00:15:08.000 2020-01-02 03:43:29.000 908 99809 50358.5 5035850 908 99809 50358.5 5035850 -32660 32275 4483.3 448330 -124 127 3.3 330 +909 100 10899 99810 2.72972 299.72972 151.22972 15122.97297 2.72972 299.72974 151.22973 15122.97332 2.72972 299.72972 151.22972 15122.97200 2020-01-01 2020-01-02 2020-01-01 00:15:09 2020-01-02 03:43:30 2020-01-01 00:15:09.000 2020-01-02 03:43:30.000 909 99810 50359.5 5035950 909 99810 50359.5 5035950 -32659 32276 4484.3 448430 -128 127 1.74 174 +91 102 10081 99991 0.27327 300.27327 150.27327 15177.6006 0.27327 300.2733 150.27327 15177.60054 0.27327 300.27327 150.27327 15177.60027 2020-01-01 2020-01-02 2020-01-01 00:01:31 2020-01-02 03:46:31 2020-01-01 00:01:31.000 2020-01-02 03:46:31.000 91 99991 50041 5054141 91 99991 50041 5054141 -32478 32457 4620.009900990099 466621 -128 127 -0.6633663366336634 -67 +910 100 10900 99811 2.73273 299.73273 151.23273 15123.27327 2.73273 299.73273 151.23272 15123.27296 2.73273 299.73273 151.23273 15123.27300 2020-01-01 2020-01-02 2020-01-01 00:15:10 2020-01-02 03:43:31 2020-01-01 00:15:10.000 2020-01-02 03:43:31.000 910 99811 50360.5 5036050 910 99811 50360.5 5036050 -32658 32277 4485.3 448530 -128 123 0.18 18 +911 100 10901 99812 2.73573 299.73573 151.23573 15123.57357 2.73573 299.73575 151.23573 15123.57359 2.73573 299.73573 151.23573 15123.57300 2020-01-01 2020-01-02 2020-01-01 00:15:11 2020-01-02 03:43:32 2020-01-01 00:15:11.000 2020-01-02 03:43:32.000 911 99812 50361.5 5036150 911 99812 50361.5 5036150 -32657 32278 4486.3 448630 -127 124 1.18 118 +912 100 10902 99813 2.73873 299.73873 151.23873 15123.87387 2.73873 299.73874 151.23873 15123.8739 2.73873 299.73873 151.23873 15123.87300 2020-01-01 2020-01-02 2020-01-01 00:15:12 2020-01-02 03:43:33 2020-01-01 00:15:12.000 2020-01-02 03:43:33.000 912 99813 50362.5 5036250 912 99813 50362.5 5036250 -32656 32279 4487.3 448730 -126 125 2.18 218 913 100 10903 99814 2.74174 299.74174 151.24174 15124.17417 2.74174 299.74173 151.24174 15124.17419 2.74174 299.74174 151.24174000000002 15124.17400 2020-01-01 2020-01-02 2020-01-01 00:15:13 2020-01-02 03:43:34 2020-01-01 00:15:13.000 2020-01-02 03:43:34.000 913 99814 50363.5 5036350 913 99814 50363.5 5036350 -32655 32280 4488.3 448830 -125 126 3.18 318 -914 100 10904 99815 2.74474 299.74474 151.24474 15124.47447 2.74474 299.74475 151.24474 15124.47479 2.74474 299.74474 151.24474000000006 15124.47400 2020-01-01 2020-01-02 2020-01-01 00:15:14 2020-01-02 03:43:35 2020-01-01 00:15:14.000 2020-01-02 03:43:35.000 914 99815 50364.5 5036450 914 99815 50364.5 5036450 -32654 32281 4489.3 448930 -124 127 4.18 418 -915 100 10905 99816 2.74774 299.74774 151.24774 15124.77477 2.74774 299.74774 151.24774 15124.77447 2.74774 299.74774 151.24774000000002 15124.77400 2020-01-01 2020-01-02 2020-01-01 00:15:15 2020-01-02 03:43:36 2020-01-01 00:15:15.000 2020-01-02 03:43:36.000 915 99816 50365.5 5036550 915 99816 50365.5 5036550 -32653 32282 4490.3 449030 -128 127 2.62 262 -916 100 10906 99817 2.75075 299.75075 151.25075 15125.07507 2.75075 299.75076 151.25075 15125.07507 2.75075 299.75075 151.25074999999995 15125.07500 2020-01-01 2020-01-02 2020-01-01 00:15:16 2020-01-02 03:43:37 2020-01-01 00:15:16.000 2020-01-02 03:43:37.000 916 99817 50366.5 5036650 916 99817 50366.5 5036650 -32652 32283 4491.3 449130 -128 127 1.06 106 -917 100 10907 99818 2.75375 299.75375 151.25375 15125.37537 2.75375 299.75375 151.25375 15125.37536 2.75375 299.75375 151.25374999999994 15125.37500 2020-01-01 2020-01-02 2020-01-01 00:15:17 2020-01-02 03:43:38 2020-01-01 00:15:17.000 2020-01-02 03:43:38.000 917 99818 50367.5 5036750 917 99818 50367.5 5036750 -32651 32284 4492.3 449230 -128 124 -0.5 -50 -918 100 10908 99819 2.75675 299.75675 151.25675 15125.67567 2.75675 299.75674 151.25675 15125.67566 2.75675 299.75675 151.25675000000018 15125.67500 2020-01-01 2020-01-02 2020-01-01 00:15:18 2020-01-02 03:43:39 2020-01-01 00:15:18.000 2020-01-02 03:43:39.000 918 99819 50368.5 5036850 918 99819 50368.5 5036850 -32650 32285 4493.3 449330 -127 125 0.5 50 -919 100 10909 99820 2.75975 299.75975 151.25975 15125.97597 2.75975 299.75977 151.25976 15125.97626 2.75975 299.75975 151.25974999999997 15125.97500 2020-01-01 2020-01-02 2020-01-01 00:15:19 2020-01-02 03:43:40 2020-01-01 00:15:19.000 2020-01-02 03:43:40.000 919 99820 50369.5 5036950 919 99820 50369.5 5036950 -32649 32286 4494.3 449430 -126 126 1.5 150 -92 102 10082 99992 0.27627 300.27627 150.27627 15177.9039 0.27627 300.27628 150.27627 15177.90384 0.27627 300.27627 150.27627000000007 15177.90327 2020-01-01 2020-01-02 2020-01-01 00:01:32 2020-01-02 03:46:32 2020-01-01 00:01:32.000 2020-01-02 03:46:32.000 92 99992 50042 5054242 92 99992 50042 5054242 -32477 32458 4621.009900990099 466722 -128 123 -2.198019801980198 -222 -920 100 10910 99821 2.76276 299.76276 151.26276 15126.27627 2.76276 299.76276 151.26275 15126.27594 2.76276 299.76276 151.2627599999999 15126.27600 2020-01-01 2020-01-02 2020-01-01 00:15:20 2020-01-02 03:43:41 2020-01-01 00:15:20.000 2020-01-02 03:43:41.000 920 99821 50370.5 5037050 920 99821 50370.5 5037050 -32648 32287 4495.3 449530 -125 127 2.5 250 -921 100 10911 99822 2.76576 299.76576 151.26576 15126.57657 2.76576 299.76578 151.26576 15126.57654 2.76576 299.76576 151.2657600000001 15126.57600 2020-01-01 2020-01-02 2020-01-01 00:15:21 2020-01-02 03:43:42 2020-01-01 00:15:21.000 2020-01-02 03:43:42.000 921 99822 50371.5 5037150 921 99822 50371.5 5037150 -32647 32288 4496.3 449630 -128 127 0.94 94 -922 100 10912 99823 2.76876 299.76876 151.26876 15126.87687 2.76876 299.76877 151.26876 15126.87683 2.76876 299.76876 151.26876000000007 15126.87600 2020-01-01 2020-01-02 2020-01-01 00:15:22 2020-01-02 03:43:43 2020-01-01 00:15:22.000 2020-01-02 03:43:43.000 922 99823 50372.5 5037250 922 99823 50372.5 5037250 -32646 32289 4497.3 449730 -128 127 -0.62 -62 -923 100 10913 99824 2.77177 299.77177 151.27177 15127.17717 2.77177 299.77176 151.27177 15127.17713 2.77177 299.77177 151.27176999999995 15127.17700 2020-01-01 2020-01-02 2020-01-01 00:15:23 2020-01-02 03:43:44 2020-01-01 00:15:23.000 2020-01-02 03:43:44.000 923 99824 50373.5 5037350 923 99824 50373.5 5037350 -32645 32290 4498.3 449830 -128 123 -2.18 -218 -924 100 10914 99825 2.77477 299.77477 151.27477 15127.47747 2.77477 299.77478 151.27477 15127.47776 2.77477 299.77477 151.27477 15127.47700 2020-01-01 2020-01-02 2020-01-01 00:15:24 2020-01-02 03:43:45 2020-01-01 00:15:24.000 2020-01-02 03:43:45.000 924 99825 50374.5 5037450 924 99825 50374.5 5037450 -32644 32291 4499.3 449930 -127 124 -1.18 -118 -925 100 10915 99826 2.77777 299.77777 151.27777 15127.77777 2.77777 299.77777 151.27777 15127.77741 2.77777 299.77777 151.27777000000003 15127.77700 2020-01-01 2020-01-02 2020-01-01 00:15:25 2020-01-02 03:43:46 2020-01-01 00:15:25.000 2020-01-02 03:43:46.000 925 99826 50375.5 5037550 925 99826 50375.5 5037550 -32643 32292 4500.3 450030 -126 125 -0.18 -18 -926 100 10916 99827 2.78078 299.78078 151.28078 15128.07807 2.78078 299.7808 151.28078 15128.078 2.78078 299.78078 151.28077999999994 15128.07800 2020-01-01 2020-01-02 2020-01-01 00:15:26 2020-01-02 03:43:47 2020-01-01 00:15:26.000 2020-01-02 03:43:47.000 926 99827 50376.5 5037650 926 99827 50376.5 5037650 -32642 32293 4501.3 450130 -125 126 0.82 82 -927 100 10917 99828 2.78378 299.78378 151.28378 15128.37837 2.78378 299.78378 151.28378 15128.3783 2.78378 299.78378 151.28377999999992 15128.37800 2020-01-01 2020-01-02 2020-01-01 00:15:27 2020-01-02 03:43:48 2020-01-01 00:15:27.000 2020-01-02 03:43:48.000 927 99828 50377.5 5037750 927 99828 50377.5 5037750 -32641 32294 4502.3 450230 -124 127 1.82 182 -928 100 10918 99829 2.78678 299.78678 151.28678 15128.67867 2.78678 299.78677 151.28678 15128.6786 2.78678 299.78678 151.28678000000008 15128.67800 2020-01-01 2020-01-02 2020-01-01 00:15:28 2020-01-02 03:43:49 2020-01-01 00:15:28.000 2020-01-02 03:43:49.000 928 99829 50378.5 5037850 928 99829 50378.5 5037850 -32640 32295 4503.3 450330 -128 127 0.26 26 -929 100 10919 99830 2.78978 299.78978 151.28978 15128.97897 2.78978 299.7898 151.28979 15128.97923 2.78978 299.78978 151.28977999999995 15128.97800 2020-01-01 2020-01-02 2020-01-01 00:15:29 2020-01-02 03:43:50 2020-01-01 00:15:29.000 2020-01-02 03:43:50.000 929 99830 50379.5 5037950 929 99830 50379.5 5037950 -32639 32296 4504.3 450430 -128 123 -1.3 -130 -93 102 10083 99993 0.27927 300.27927 150.27927 15178.2072 0.27927 300.27927 150.27927 15178.20715 0.27927 300.27927 150.2792699999999 15178.20627 2020-01-01 2020-01-02 2020-01-01 00:01:33 2020-01-02 03:46:33 2020-01-01 00:01:33.000 2020-01-02 03:46:33.000 93 99993 50043 5054343 93 99993 50043 5054343 -32476 32459 4622.009900990099 466823 -127 124 -1.198019801980198 -121 -930 100 10920 99831 2.79279 299.79279 151.29279 15129.27927 2.79279 299.7928 151.29278 15129.27888 2.79279 299.79279 151.29279000000002 15129.27900 2020-01-01 2020-01-02 2020-01-01 00:15:30 2020-01-02 03:43:51 2020-01-01 00:15:30.000 2020-01-02 03:43:51.000 930 99831 50380.5 5038050 930 99831 50380.5 5038050 -32638 32297 4505.3 450530 -127 124 -0.3 -30 -931 100 10921 99832 2.79579 299.79579 151.29579 15129.57957 2.79579 299.7958 151.29579 15129.57963 2.79579 299.79579 151.29579000000007 15129.57900 2020-01-01 2020-01-02 2020-01-01 00:15:31 2020-01-02 03:43:52 2020-01-01 00:15:31.000 2020-01-02 03:43:52.000 931 99832 50381.5 5038150 931 99832 50381.5 5038150 -32637 32298 4506.3 450630 -126 125 0.7 70 -932 100 10922 99833 2.79879 299.79879 151.29879 15129.87987 2.79879 299.7988 151.29879 15129.87977 2.79879 299.79879 151.29879000000003 15129.87900 2020-01-01 2020-01-02 2020-01-01 00:15:32 2020-01-02 03:43:53 2020-01-01 00:15:32.000 2020-01-02 03:43:53.000 932 99833 50382.5 5038250 932 99833 50382.5 5038250 -32636 32299 4507.3 450730 -125 126 1.7 170 -933 100 10923 99834 2.8018 299.8018 151.3018 15130.18018 2.8018 299.8018 151.3018 15130.18011 2.80180 299.80180 151.3018 15130.18000 2020-01-01 2020-01-02 2020-01-01 00:15:33 2020-01-02 03:43:54 2020-01-01 00:15:33.000 2020-01-02 03:43:54.000 933 99834 50383.5 5038350 933 99834 50383.5 5038350 -32635 32300 4508.3 450830 -124 127 2.7 270 -934 100 10924 99835 2.8048 299.8048 151.3048 15130.48048 2.8048 299.8048 151.3048 15130.48071 2.80480 299.80480 151.30479999999991 15130.48000 2020-01-01 2020-01-02 2020-01-01 00:15:34 2020-01-02 03:43:55 2020-01-01 00:15:34.000 2020-01-02 03:43:55.000 934 99835 50384.5 5038450 934 99835 50384.5 5038450 -32634 32301 4509.3 450930 -128 127 1.14 114 -935 100 10925 99836 2.8078 299.8078 151.3078 15130.78078 2.8078 299.8078 151.3078 15130.78034 2.80780 299.80780 151.3078 15130.78000 2020-01-01 2020-01-02 2020-01-01 00:15:35 2020-01-02 03:43:56 2020-01-01 00:15:35.000 2020-01-02 03:43:56.000 935 99836 50385.5 5038550 935 99836 50385.5 5038550 -32633 32302 4510.3 451030 -128 123 -0.42 -42 -936 100 10926 99837 2.81081 299.81081 151.31081 15131.08108 2.81081 299.81082 151.31081 15131.0811 2.81081 299.81081 151.31081000000006 15131.08100 2020-01-01 2020-01-02 2020-01-01 00:15:36 2020-01-02 03:43:57 2020-01-01 00:15:36.000 2020-01-02 03:43:57.000 936 99837 50386.5 5038650 936 99837 50386.5 5038650 -32632 32303 4511.3 451130 -127 124 0.58 58 -937 100 10927 99838 2.81381 299.81381 151.31381 15131.38138 2.81381 299.8138 151.31381 15131.38124 2.81381 299.81381 151.31380999999993 15131.38100 2020-01-01 2020-01-02 2020-01-01 00:15:37 2020-01-02 03:43:58 2020-01-01 00:15:37.000 2020-01-02 03:43:58.000 937 99838 50387.5 5038750 937 99838 50387.5 5038750 -32631 32304 4512.3 451230 -126 125 1.58 158 -938 100 10928 99839 2.81681 299.81681 151.31681 15131.68168 2.81681 299.8168 151.31681 15131.68157 2.81681 299.81681 151.31681000000003 15131.68100 2020-01-01 2020-01-02 2020-01-01 00:15:38 2020-01-02 03:43:59 2020-01-01 00:15:38.000 2020-01-02 03:43:59.000 938 99839 50388.5 5038850 938 99839 50388.5 5038850 -32630 32305 4513.3 451330 -125 126 2.58 258 -939 100 10929 99840 2.81981 299.81981 151.31981 15131.98198 2.81982 299.81982 151.31982 15131.98217 2.81981 299.81981 151.31981000000007 15131.98100 2020-01-01 2020-01-02 2020-01-01 00:15:39 2020-01-02 03:44:00 2020-01-01 00:15:39.000 2020-01-02 03:44:00.000 939 99840 50389.5 5038950 939 99840 50389.5 5038950 -32629 32306 4514.3 451430 -124 127 3.58 358 -94 102 10084 99994 0.28228 300.28228 150.28228 15178.51051 0.28228 300.2823 150.28228 15178.51078 0.28228 300.28228 150.28228 15178.51028 2020-01-01 2020-01-02 2020-01-01 00:01:34 2020-01-02 03:46:34 2020-01-01 00:01:34.000 2020-01-02 03:46:34.000 94 99994 50044 5054444 94 99994 50044 5054444 -32475 32460 4623.009900990099 466924 -126 125 -0.19801980198019803 -20 +914 100 10904 99815 2.74474 299.74474 151.24474 15124.47447 2.74474 299.74475 151.24474 15124.47479 2.74474 299.74474 151.24474 15124.47400 2020-01-01 2020-01-02 2020-01-01 00:15:14 2020-01-02 03:43:35 2020-01-01 00:15:14.000 2020-01-02 03:43:35.000 914 99815 50364.5 5036450 914 99815 50364.5 5036450 -32654 32281 4489.3 448930 -124 127 4.18 418 +915 100 10905 99816 2.74774 299.74774 151.24774 15124.77477 2.74774 299.74774 151.24774 15124.77447 2.74774 299.74774 151.24774 15124.77400 2020-01-01 2020-01-02 2020-01-01 00:15:15 2020-01-02 03:43:36 2020-01-01 00:15:15.000 2020-01-02 03:43:36.000 915 99816 50365.5 5036550 915 99816 50365.5 5036550 -32653 32282 4490.3 449030 -128 127 2.62 262 +916 100 10906 99817 2.75075 299.75075 151.25075 15125.07507 2.75075 299.75076 151.25075 15125.07507 2.75075 299.75075 151.25075 15125.07500 2020-01-01 2020-01-02 2020-01-01 00:15:16 2020-01-02 03:43:37 2020-01-01 00:15:16.000 2020-01-02 03:43:37.000 916 99817 50366.5 5036650 916 99817 50366.5 5036650 -32652 32283 4491.3 449130 -128 127 1.06 106 +917 100 10907 99818 2.75375 299.75375 151.25375 15125.37537 2.75375 299.75375 151.25375 15125.37536 2.75375 299.75375 151.25375 15125.37500 2020-01-01 2020-01-02 2020-01-01 00:15:17 2020-01-02 03:43:38 2020-01-01 00:15:17.000 2020-01-02 03:43:38.000 917 99818 50367.5 5036750 917 99818 50367.5 5036750 -32651 32284 4492.3 449230 -128 124 -0.5 -50 +918 100 10908 99819 2.75675 299.75675 151.25675 15125.67567 2.75675 299.75674 151.25675 15125.67566 2.75675 299.75675 151.25674999999998 15125.67500 2020-01-01 2020-01-02 2020-01-01 00:15:18 2020-01-02 03:43:39 2020-01-01 00:15:18.000 2020-01-02 03:43:39.000 918 99819 50368.5 5036850 918 99819 50368.5 5036850 -32650 32285 4493.3 449330 -127 125 0.5 50 +919 100 10909 99820 2.75975 299.75975 151.25975 15125.97597 2.75975 299.75977 151.25976 15125.97626 2.75975 299.75975 151.25975 15125.97500 2020-01-01 2020-01-02 2020-01-01 00:15:19 2020-01-02 03:43:40 2020-01-01 00:15:19.000 2020-01-02 03:43:40.000 919 99820 50369.5 5036950 919 99820 50369.5 5036950 -32649 32286 4494.3 449430 -126 126 1.5 150 +92 102 10082 99992 0.27627 300.27627 150.27627 15177.9039 0.27627 300.27628 150.27627 15177.90384 0.27627 300.27627 150.27627 15177.90327 2020-01-01 2020-01-02 2020-01-01 00:01:32 2020-01-02 03:46:32 2020-01-01 00:01:32.000 2020-01-02 03:46:32.000 92 99992 50042 5054242 92 99992 50042 5054242 -32477 32458 4621.009900990099 466722 -128 123 -2.198019801980198 -222 +920 100 10910 99821 2.76276 299.76276 151.26276 15126.27627 2.76276 299.76276 151.26275 15126.27594 2.76276 299.76276 151.26276 15126.27600 2020-01-01 2020-01-02 2020-01-01 00:15:20 2020-01-02 03:43:41 2020-01-01 00:15:20.000 2020-01-02 03:43:41.000 920 99821 50370.5 5037050 920 99821 50370.5 5037050 -32648 32287 4495.3 449530 -125 127 2.5 250 +921 100 10911 99822 2.76576 299.76576 151.26576 15126.57657 2.76576 299.76578 151.26576 15126.57654 2.76576 299.76576 151.26576 15126.57600 2020-01-01 2020-01-02 2020-01-01 00:15:21 2020-01-02 03:43:42 2020-01-01 00:15:21.000 2020-01-02 03:43:42.000 921 99822 50371.5 5037150 921 99822 50371.5 5037150 -32647 32288 4496.3 449630 -128 127 0.94 94 +922 100 10912 99823 2.76876 299.76876 151.26876 15126.87687 2.76876 299.76877 151.26876 15126.87683 2.76876 299.76876 151.26876000000001 15126.87600 2020-01-01 2020-01-02 2020-01-01 00:15:22 2020-01-02 03:43:43 2020-01-01 00:15:22.000 2020-01-02 03:43:43.000 922 99823 50372.5 5037250 922 99823 50372.5 5037250 -32646 32289 4497.3 449730 -128 127 -0.62 -62 +923 100 10913 99824 2.77177 299.77177 151.27177 15127.17717 2.77177 299.77176 151.27177 15127.17713 2.77177 299.77177 151.27177 15127.17700 2020-01-01 2020-01-02 2020-01-01 00:15:23 2020-01-02 03:43:44 2020-01-01 00:15:23.000 2020-01-02 03:43:44.000 923 99824 50373.5 5037350 923 99824 50373.5 5037350 -32645 32290 4498.3 449830 -128 123 -2.18 -218 +924 100 10914 99825 2.77477 299.77477 151.27477 15127.47747 2.77477 299.77478 151.27477 15127.47776 2.77477 299.77477 151.27477000000002 15127.47700 2020-01-01 2020-01-02 2020-01-01 00:15:24 2020-01-02 03:43:45 2020-01-01 00:15:24.000 2020-01-02 03:43:45.000 924 99825 50374.5 5037450 924 99825 50374.5 5037450 -32644 32291 4499.3 449930 -127 124 -1.18 -118 +925 100 10915 99826 2.77777 299.77777 151.27777 15127.77777 2.77777 299.77777 151.27777 15127.77741 2.77777 299.77777 151.27777 15127.77700 2020-01-01 2020-01-02 2020-01-01 00:15:25 2020-01-02 03:43:46 2020-01-01 00:15:25.000 2020-01-02 03:43:46.000 925 99826 50375.5 5037550 925 99826 50375.5 5037550 -32643 32292 4500.3 450030 -126 125 -0.18 -18 +926 100 10916 99827 2.78078 299.78078 151.28078 15128.07807 2.78078 299.7808 151.28078 15128.078 2.78078 299.78078 151.28078 15128.07800 2020-01-01 2020-01-02 2020-01-01 00:15:26 2020-01-02 03:43:47 2020-01-01 00:15:26.000 2020-01-02 03:43:47.000 926 99827 50376.5 5037650 926 99827 50376.5 5037650 -32642 32293 4501.3 450130 -125 126 0.82 82 +927 100 10917 99828 2.78378 299.78378 151.28378 15128.37837 2.78378 299.78378 151.28378 15128.3783 2.78378 299.78378 151.28378 15128.37800 2020-01-01 2020-01-02 2020-01-01 00:15:27 2020-01-02 03:43:48 2020-01-01 00:15:27.000 2020-01-02 03:43:48.000 927 99828 50377.5 5037750 927 99828 50377.5 5037750 -32641 32294 4502.3 450230 -124 127 1.82 182 +928 100 10918 99829 2.78678 299.78678 151.28678 15128.67867 2.78678 299.78677 151.28678 15128.6786 2.78678 299.78678 151.28678 15128.67800 2020-01-01 2020-01-02 2020-01-01 00:15:28 2020-01-02 03:43:49 2020-01-01 00:15:28.000 2020-01-02 03:43:49.000 928 99829 50378.5 5037850 928 99829 50378.5 5037850 -32640 32295 4503.3 450330 -128 127 0.26 26 +929 100 10919 99830 2.78978 299.78978 151.28978 15128.97897 2.78978 299.7898 151.28979 15128.97923 2.78978 299.78978 151.28977999999998 15128.97800 2020-01-01 2020-01-02 2020-01-01 00:15:29 2020-01-02 03:43:50 2020-01-01 00:15:29.000 2020-01-02 03:43:50.000 929 99830 50379.5 5037950 929 99830 50379.5 5037950 -32639 32296 4504.3 450430 -128 123 -1.3 -130 +93 102 10083 99993 0.27927 300.27927 150.27927 15178.2072 0.27927 300.27927 150.27927 15178.20715 0.27927 300.27927 150.27927 15178.20627 2020-01-01 2020-01-02 2020-01-01 00:01:33 2020-01-02 03:46:33 2020-01-01 00:01:33.000 2020-01-02 03:46:33.000 93 99993 50043 5054343 93 99993 50043 5054343 -32476 32459 4622.009900990099 466823 -127 124 -1.198019801980198 -121 +930 100 10920 99831 2.79279 299.79279 151.29279 15129.27927 2.79279 299.7928 151.29278 15129.27888 2.79279 299.79279 151.29279 15129.27900 2020-01-01 2020-01-02 2020-01-01 00:15:30 2020-01-02 03:43:51 2020-01-01 00:15:30.000 2020-01-02 03:43:51.000 930 99831 50380.5 5038050 930 99831 50380.5 5038050 -32638 32297 4505.3 450530 -127 124 -0.3 -30 +931 100 10921 99832 2.79579 299.79579 151.29579 15129.57957 2.79579 299.7958 151.29579 15129.57963 2.79579 299.79579 151.29579 15129.57900 2020-01-01 2020-01-02 2020-01-01 00:15:31 2020-01-02 03:43:52 2020-01-01 00:15:31.000 2020-01-02 03:43:52.000 931 99832 50381.5 5038150 931 99832 50381.5 5038150 -32637 32298 4506.3 450630 -126 125 0.7 70 +932 100 10922 99833 2.79879 299.79879 151.29879 15129.87987 2.79879 299.7988 151.29879 15129.87977 2.79879 299.79879 151.29879 15129.87900 2020-01-01 2020-01-02 2020-01-01 00:15:32 2020-01-02 03:43:53 2020-01-01 00:15:32.000 2020-01-02 03:43:53.000 932 99833 50382.5 5038250 932 99833 50382.5 5038250 -32636 32299 4507.3 450730 -125 126 1.7 170 +933 100 10923 99834 2.8018 299.8018 151.3018 15130.18018 2.8018 299.8018 151.3018 15130.18011 2.80180 299.80180 151.30180000000001 15130.18000 2020-01-01 2020-01-02 2020-01-01 00:15:33 2020-01-02 03:43:54 2020-01-01 00:15:33.000 2020-01-02 03:43:54.000 933 99834 50383.5 5038350 933 99834 50383.5 5038350 -32635 32300 4508.3 450830 -124 127 2.7 270 +934 100 10924 99835 2.8048 299.8048 151.3048 15130.48048 2.8048 299.8048 151.3048 15130.48071 2.80480 299.80480 151.3048 15130.48000 2020-01-01 2020-01-02 2020-01-01 00:15:34 2020-01-02 03:43:55 2020-01-01 00:15:34.000 2020-01-02 03:43:55.000 934 99835 50384.5 5038450 934 99835 50384.5 5038450 -32634 32301 4509.3 450930 -128 127 1.14 114 +935 100 10925 99836 2.8078 299.8078 151.3078 15130.78078 2.8078 299.8078 151.3078 15130.78034 2.80780 299.80780 151.30780000000001 15130.78000 2020-01-01 2020-01-02 2020-01-01 00:15:35 2020-01-02 03:43:56 2020-01-01 00:15:35.000 2020-01-02 03:43:56.000 935 99836 50385.5 5038550 935 99836 50385.5 5038550 -32633 32302 4510.3 451030 -128 123 -0.42 -42 +936 100 10926 99837 2.81081 299.81081 151.31081 15131.08108 2.81081 299.81082 151.31081 15131.0811 2.81081 299.81081 151.31081 15131.08100 2020-01-01 2020-01-02 2020-01-01 00:15:36 2020-01-02 03:43:57 2020-01-01 00:15:36.000 2020-01-02 03:43:57.000 936 99837 50386.5 5038650 936 99837 50386.5 5038650 -32632 32303 4511.3 451130 -127 124 0.58 58 +937 100 10927 99838 2.81381 299.81381 151.31381 15131.38138 2.81381 299.8138 151.31381 15131.38124 2.81381 299.81381 151.31381 15131.38100 2020-01-01 2020-01-02 2020-01-01 00:15:37 2020-01-02 03:43:58 2020-01-01 00:15:37.000 2020-01-02 03:43:58.000 937 99838 50387.5 5038750 937 99838 50387.5 5038750 -32631 32304 4512.3 451230 -126 125 1.58 158 +938 100 10928 99839 2.81681 299.81681 151.31681 15131.68168 2.81681 299.8168 151.31681 15131.68157 2.81681 299.81681 151.31681 15131.68100 2020-01-01 2020-01-02 2020-01-01 00:15:38 2020-01-02 03:43:59 2020-01-01 00:15:38.000 2020-01-02 03:43:59.000 938 99839 50388.5 5038850 938 99839 50388.5 5038850 -32630 32305 4513.3 451330 -125 126 2.58 258 +939 100 10929 99840 2.81981 299.81981 151.31981 15131.98198 2.81982 299.81982 151.31982 15131.98217 2.81981 299.81981 151.31981 15131.98100 2020-01-01 2020-01-02 2020-01-01 00:15:39 2020-01-02 03:44:00 2020-01-01 00:15:39.000 2020-01-02 03:44:00.000 939 99840 50389.5 5038950 939 99840 50389.5 5038950 -32629 32306 4514.3 451430 -124 127 3.58 358 +94 102 10084 99994 0.28228 300.28228 150.28228 15178.51051 0.28228 300.2823 150.28228 15178.51078 0.28228 300.28228 150.28228000000001 15178.51028 2020-01-01 2020-01-02 2020-01-01 00:01:34 2020-01-02 03:46:34 2020-01-01 00:01:34.000 2020-01-02 03:46:34.000 94 99994 50044 5054444 94 99994 50044 5054444 -32475 32460 4623.009900990099 466924 -126 125 -0.19801980198019803 -20 940 100 10930 99841 2.82282 299.82282 151.32282 15132.28228 2.82282 299.8228 151.32282 15132.28247 2.82282 299.82282 151.32281999999998 15132.28200 2020-01-01 2020-01-02 2020-01-01 00:15:40 2020-01-02 03:44:01 2020-01-01 00:15:40.000 2020-01-02 03:44:01.000 940 99841 50390.5 5039050 940 99841 50390.5 5039050 -32628 32307 4515.3 451530 -128 127 2.02 202 941 100 10931 99842 2.82582 299.82582 151.32582 15132.58258 2.82582 299.82584 151.32582 15132.58257 2.82582 299.82582 151.32582 15132.58200 2020-01-01 2020-01-02 2020-01-01 00:15:41 2020-01-02 03:44:02 2020-01-01 00:15:41.000 2020-01-02 03:44:02.000 941 99842 50391.5 5039150 941 99842 50391.5 5039150 -32627 32308 4516.3 451630 -128 127 0.46 46 -942 100 10932 99843 2.82882 299.82882 151.32882 15132.88288 2.82882 299.82883 151.32882 15132.88275 2.82882 299.82882 151.32882000000006 15132.88200 2020-01-01 2020-01-02 2020-01-01 00:15:42 2020-01-02 03:44:03 2020-01-01 00:15:42.000 2020-01-02 03:44:03.000 942 99843 50392.5 5039250 942 99843 50392.5 5039250 -32626 32309 4517.3 451730 -128 124 -1.1 -110 -943 100 10933 99844 2.83183 299.83183 151.33183 15133.18318 2.83183 299.83182 151.33183 15133.18304 2.83183 299.83183 151.33182999999994 15133.18300 2020-01-01 2020-01-02 2020-01-01 00:15:43 2020-01-02 03:44:04 2020-01-01 00:15:43.000 2020-01-02 03:44:04.000 943 99844 50393.5 5039350 943 99844 50393.5 5039350 -32625 32310 4518.3 451830 -127 125 -0.1 -10 -944 100 10934 99845 2.83483 299.83483 151.33483 15133.48348 2.83483 299.83484 151.33483 15133.48364 2.83483 299.83483 151.3348299999999 15133.48300 2020-01-01 2020-01-02 2020-01-01 00:15:44 2020-01-02 03:44:05 2020-01-01 00:15:44.000 2020-01-02 03:44:05.000 944 99845 50394.5 5039450 944 99845 50394.5 5039450 -32624 32311 4519.3 451930 -126 126 0.9 90 -945 100 10935 99846 2.83783 299.83783 151.33783 15133.78378 2.83783 299.83783 151.33783 15133.78393 2.83783 299.83783 151.3378300000001 15133.78300 2020-01-01 2020-01-02 2020-01-01 00:15:45 2020-01-02 03:44:06 2020-01-01 00:15:45.000 2020-01-02 03:44:06.000 945 99846 50395.5 5039550 945 99846 50395.5 5039550 -32623 32312 4520.3 452030 -125 127 1.9 190 -946 100 10936 99847 2.84084 299.84084 151.34084 15134.08408 2.84084 299.84085 151.34084 15134.08404 2.84084 299.84084 151.34084000000004 15134.08400 2020-01-01 2020-01-02 2020-01-01 00:15:46 2020-01-02 03:44:07 2020-01-01 00:15:46.000 2020-01-02 03:44:07.000 946 99847 50396.5 5039650 946 99847 50396.5 5039650 -32622 32313 4521.3 452130 -128 127 0.34 34 -947 100 10937 99848 2.84384 299.84384 151.34384 15134.38438 2.84384 299.84384 151.34384 15134.38421 2.84384 299.84384 151.34383999999983 15134.38400 2020-01-01 2020-01-02 2020-01-01 00:15:47 2020-01-02 03:44:08 2020-01-01 00:15:47.000 2020-01-02 03:44:08.000 947 99848 50397.5 5039750 947 99848 50397.5 5039750 -32621 32314 4522.3 452230 -128 127 -1.22 -122 -948 100 10938 99849 2.84684 299.84684 151.34684 15134.68468 2.84684 299.84683 151.34684 15134.68452 2.84684 299.84684 151.34684000000007 15134.68400 2020-01-01 2020-01-02 2020-01-01 00:15:48 2020-01-02 03:44:09 2020-01-01 00:15:48.000 2020-01-02 03:44:09.000 948 99849 50398.5 5039850 948 99849 50398.5 5039850 -32620 32315 4523.3 452330 -128 123 -2.78 -278 -949 100 10939 99850 2.84984 299.84984 151.34984 15134.98498 2.84985 299.84985 151.34985 15134.98527 2.84984 299.84984 151.34984000000003 15134.98400 2020-01-01 2020-01-02 2020-01-01 00:15:49 2020-01-02 03:44:10 2020-01-01 00:15:49.000 2020-01-02 03:44:10.000 949 99850 50399.5 5039950 949 99850 50399.5 5039950 -32619 32316 4524.3 452430 -127 124 -1.78 -178 -95 102 10085 99995 0.28528 300.28528 150.28528 15178.81381 0.28528 300.28528 150.28528 15178.81343 0.28528 300.28528 150.28528000000003 15178.81328 2020-01-01 2020-01-02 2020-01-01 00:01:35 2020-01-02 03:46:35 2020-01-01 00:01:35.000 2020-01-02 03:46:35.000 95 99995 50045 5054545 95 99995 50045 5054545 -32474 32461 4624.009900990099 467025 -125 126 0.801980198019802 81 -950 100 10940 99851 2.85285 299.85285 151.35285 15135.28528 2.85285 299.85284 151.35285 15135.28541 2.85285 299.85285 151.35285000000005 15135.28500 2020-01-01 2020-01-02 2020-01-01 00:15:50 2020-01-02 03:44:11 2020-01-01 00:15:50.000 2020-01-02 03:44:11.000 950 99851 50400.5 5040050 950 99851 50400.5 5040050 -32618 32317 4525.3 452530 -126 125 -0.78 -78 -951 100 10941 99852 2.85585 299.85585 151.35585 15135.58558 2.85585 299.85587 151.35585 15135.58551 2.85585 299.85585 151.35584999999995 15135.58500 2020-01-01 2020-01-02 2020-01-01 00:15:51 2020-01-02 03:44:12 2020-01-01 00:15:51.000 2020-01-02 03:44:12.000 951 99852 50401.5 5040150 951 99852 50401.5 5040150 -32617 32318 4526.3 452630 -125 126 0.22 22 -952 100 10942 99853 2.85885 299.85885 151.35885 15135.88588 2.85885 299.85886 151.35885 15135.88568 2.85885 299.85885 151.35885000000005 15135.88500 2020-01-01 2020-01-02 2020-01-01 00:15:52 2020-01-02 03:44:13 2020-01-01 00:15:52.000 2020-01-02 03:44:13.000 952 99853 50402.5 5040250 952 99853 50402.5 5040250 -32616 32319 4527.3 452730 -124 127 1.22 122 -953 100 10943 99854 2.86186 299.86186 151.36186 15136.18618 2.86186 299.86185 151.36185 15136.18598 2.86186 299.86186 151.36186000000026 15136.18600 2020-01-01 2020-01-02 2020-01-01 00:15:53 2020-01-02 03:44:14 2020-01-01 00:15:53.000 2020-01-02 03:44:14.000 953 99854 50403.5 5040350 953 99854 50403.5 5040350 -32615 32320 4528.3 452830 -128 127 -0.34 -34 -954 100 10944 99855 2.86486 299.86486 151.36486 15136.48648 2.86486 299.86487 151.36486 15136.48674 2.86486 299.86486 151.3648599999998 15136.48600 2020-01-01 2020-01-02 2020-01-01 00:15:54 2020-01-02 03:44:15 2020-01-01 00:15:54.000 2020-01-02 03:44:15.000 954 99855 50404.5 5040450 954 99855 50404.5 5040450 -32614 32321 4529.3 452930 -128 123 -1.9 -190 -955 100 10945 99856 2.86786 299.86786 151.36786 15136.78678 2.86786 299.86786 151.36786 15136.78688 2.86786 299.86786 151.36786000000018 15136.78600 2020-01-01 2020-01-02 2020-01-01 00:15:55 2020-01-02 03:44:16 2020-01-01 00:15:55.000 2020-01-02 03:44:16.000 955 99856 50405.5 5040550 955 99856 50405.5 5040550 -32613 32322 4530.3 453030 -127 124 -0.9 -90 -956 100 10946 99857 2.87087 299.87087 151.37087 15137.08708 2.87087 299.87088 151.37087 15137.08701 2.87087 299.87087 151.37087000000014 15137.08700 2020-01-01 2020-01-02 2020-01-01 00:15:56 2020-01-02 03:44:17 2020-01-01 00:15:56.000 2020-01-02 03:44:17.000 956 99857 50406.5 5040650 956 99857 50406.5 5040650 -32612 32323 4531.3 453130 -126 125 0.1 10 -957 100 10947 99858 2.87387 299.87387 151.37387 15137.38738 2.87387 299.87387 151.37387 15137.38716 2.87387 299.87387 151.37386999999993 15137.38700 2020-01-01 2020-01-02 2020-01-01 00:15:57 2020-01-02 03:44:18 2020-01-01 00:15:57.000 2020-01-02 03:44:18.000 957 99858 50407.5 5040750 957 99858 50407.5 5040750 -32611 32324 4532.3 453230 -125 126 1.1 110 -958 100 10948 99859 2.87687 299.87687 151.37687 15137.68768 2.87687 299.8769 151.37687 15137.68791 2.87687 299.87687 151.37687000000003 15137.68700 2020-01-01 2020-01-02 2020-01-01 00:15:58 2020-01-02 03:44:19 2020-01-01 00:15:58.000 2020-01-02 03:44:19.000 958 99859 50408.5 5040850 958 99859 50408.5 5040850 -32610 32325 4533.3 453330 -124 127 2.1 210 -959 100 10949 99860 2.87987 299.87987 151.37987 15137.98798 2.87988 299.87988 151.37988 15137.9882 2.87987 299.87987 151.37987000000012 15137.98700 2020-01-01 2020-01-02 2020-01-01 00:15:59 2020-01-02 03:44:20 2020-01-01 00:15:59.000 2020-01-02 03:44:20.000 959 99860 50409.5 5040950 959 99860 50409.5 5040950 -32609 32326 4534.3 453430 -128 127 0.54 54 -96 102 10086 99996 0.28828 300.28828 150.28828 15179.11711 0.28828 300.2883 150.28828 15179.11718 0.28828 300.28828 150.28828000000013 15179.11628 2020-01-01 2020-01-02 2020-01-01 00:01:36 2020-01-02 03:46:36 2020-01-01 00:01:36.000 2020-01-02 03:46:36.000 96 99996 50046 5054646 96 99996 50046 5054646 -32473 32462 4625.009900990099 467126 -124 127 1.801980198019802 182 -960 100 10950 99861 2.88288 299.88288 151.38288 15138.28828 2.88288 299.88287 151.38288 15138.28834 2.88288 299.88288 151.3828799999998 15138.28800 2020-01-01 2020-01-02 2020-01-01 00:16:00 2020-01-02 03:44:21 2020-01-01 00:16:00.000 2020-01-02 03:44:21.000 960 99861 50410.5 5041050 960 99861 50410.5 5041050 -32608 32327 4535.3 453530 -128 123 -1.02 -102 -961 100 10951 99862 2.88588 299.88588 151.38588 15138.58858 2.88588 299.8859 151.38588 15138.58848 2.88588 299.88588 151.38587999999984 15138.58800 2020-01-01 2020-01-02 2020-01-01 00:16:01 2020-01-02 03:44:22 2020-01-01 00:16:01.000 2020-01-02 03:44:22.000 961 99862 50411.5 5041150 961 99862 50411.5 5041150 -32607 32328 4536.3 453630 -127 124 -0.02 -2 -962 100 10952 99863 2.88888 299.88888 151.38888 15138.88888 2.88888 299.8889 151.38888 15138.88862 2.88888 299.88888 151.3888800000003 15138.88800 2020-01-01 2020-01-02 2020-01-01 00:16:02 2020-01-02 03:44:23 2020-01-01 00:16:02.000 2020-01-02 03:44:23.000 962 99863 50412.5 5041250 962 99863 50412.5 5041250 -32606 32329 4537.3 453730 -126 125 0.98 98 -963 100 10953 99864 2.89189 299.89189 151.39189 15139.18918 2.89189 299.8919 151.39189 15139.18937 2.89189 299.89189 151.3918900000002 15139.18900 2020-01-01 2020-01-02 2020-01-01 00:16:03 2020-01-02 03:44:24 2020-01-01 00:16:03.000 2020-01-02 03:44:24.000 963 99864 50413.5 5041350 963 99864 50413.5 5041350 -32605 32330 4538.3 453830 -125 126 1.98 198 -964 100 10954 99865 2.89489 299.89489 151.39489 15139.48948 2.89489 299.8949 151.39489 15139.48968 2.89489 299.89489 151.3948899999997 15139.48900 2020-01-01 2020-01-02 2020-01-01 00:16:04 2020-01-02 03:44:25 2020-01-01 00:16:04.000 2020-01-02 03:44:25.000 964 99865 50414.5 5041450 964 99865 50414.5 5041450 -32604 32331 4539.3 453930 -124 127 2.98 298 -965 100 10955 99866 2.89789 299.89789 151.39789 15139.78978 2.89789 299.8979 151.39789 15139.78985 2.89789 299.89789 151.39789000000016 15139.78900 2020-01-01 2020-01-02 2020-01-01 00:16:05 2020-01-02 03:44:26 2020-01-01 00:16:05.000 2020-01-02 03:44:26.000 965 99866 50415.5 5041550 965 99866 50415.5 5041550 -32603 32332 4540.3 454030 -128 127 1.42 142 -966 100 10956 99867 2.9009 299.9009 151.4009 15140.09009 2.9009 299.9009 151.40089 15140.08996 2.90090 299.90090 151.40090000000006 15140.09000 2020-01-01 2020-01-02 2020-01-01 00:16:06 2020-01-02 03:44:27 2020-01-01 00:16:06.000 2020-01-02 03:44:27.000 966 99867 50416.5 5041650 966 99867 50416.5 5041650 -32602 32333 4541.3 454130 -128 127 -0.14 -14 -967 100 10957 99868 2.9039 299.9039 151.4039 15140.39039 2.9039 299.9039 151.4039 15140.39009 2.90390 299.90390 151.40389999999988 15140.39000 2020-01-01 2020-01-02 2020-01-01 00:16:07 2020-01-02 03:44:28 2020-01-01 00:16:07.000 2020-01-02 03:44:28.000 967 99868 50417.5 5041750 967 99868 50417.5 5041750 -32601 32334 4542.3 454230 -128 124 -1.7 -170 -968 100 10958 99869 2.9069 299.9069 151.4069 15140.69069 2.9069 299.90692 151.4069 15140.69084 2.90690 299.90690 151.40689999999998 15140.69000 2020-01-01 2020-01-02 2020-01-01 00:16:08 2020-01-02 03:44:29 2020-01-01 00:16:08.000 2020-01-02 03:44:29.000 968 99869 50418.5 5041850 968 99869 50418.5 5041850 -32600 32335 4543.3 454330 -127 125 -0.7 -70 -969 100 10959 99870 2.9099 299.9099 151.4099 15140.99099 2.90991 299.9099 151.40991 15140.99114 2.90990 299.90990 151.40990000000008 15140.99000 2020-01-01 2020-01-02 2020-01-01 00:16:09 2020-01-02 03:44:30 2020-01-01 00:16:09.000 2020-01-02 03:44:30.000 969 99870 50419.5 5041950 969 99870 50419.5 5041950 -32599 32336 4544.3 454430 -126 126 0.3 30 -97 102 10087 99997 0.29129 300.29129 150.29129 15179.42042 0.29129 300.2913 150.29129 15179.42033 0.29129 300.29129 150.2912899999998 15179.42029 2020-01-01 2020-01-02 2020-01-01 00:01:37 2020-01-02 03:46:37 2020-01-01 00:01:37.000 2020-01-02 03:46:37.000 97 99997 50047 5054747 97 99997 50047 5054747 -32472 32463 4626.009900990099 467227 -128 127 0.26732673267326734 27 -970 100 10960 99871 2.91291 299.91291 151.41291 15141.29129 2.91291 299.9129 151.41291 15141.29132 2.91291 299.91291 151.41290999999973 15141.29100 2020-01-01 2020-01-02 2020-01-01 00:16:10 2020-01-02 03:44:31 2020-01-01 00:16:10.000 2020-01-02 03:44:31.000 970 99871 50420.5 5042050 970 99871 50420.5 5042050 -32598 32337 4545.3 454530 -125 127 1.3 130 -971 100 10961 99872 2.91591 299.91591 151.41591 15141.59159 2.91591 299.91592 151.41591 15141.59142 2.91591 299.91591 151.41590999999983 15141.59100 2020-01-01 2020-01-02 2020-01-01 00:16:11 2020-01-02 03:44:32 2020-01-01 00:16:11.000 2020-01-02 03:44:32.000 971 99872 50421.5 5042150 971 99872 50421.5 5042150 -32597 32338 4546.3 454630 -128 127 -0.26 -26 -972 100 10962 99873 2.91891 299.91891 151.41891 15141.89189 2.91891 299.9189 151.41891 15141.89172 2.91891 299.91891 151.4189100000002 15141.89100 2020-01-01 2020-01-02 2020-01-01 00:16:12 2020-01-02 03:44:33 2020-01-01 00:16:12.000 2020-01-02 03:44:33.000 972 99873 50422.5 5042250 972 99873 50422.5 5042250 -32596 32339 4547.3 454730 -128 127 -1.82 -182 -973 100 10963 99874 2.92192 299.92192 151.42192 15142.19219 2.92192 299.92194 151.42192 15142.19232 2.92192 299.92192 151.4219200000001 15142.19200 2020-01-01 2020-01-02 2020-01-01 00:16:13 2020-01-02 03:44:34 2020-01-01 00:16:13.000 2020-01-02 03:44:34.000 973 99874 50423.5 5042350 973 99874 50423.5 5042350 -32595 32340 4548.3 454830 -128 123 -3.38 -338 -974 100 10964 99875 2.92492 299.92492 151.42492 15142.49249 2.92492 299.92493 151.42492 15142.49265 2.92492 299.92492 151.42491999999996 15142.49200 2020-01-01 2020-01-02 2020-01-01 00:16:14 2020-01-02 03:44:35 2020-01-01 00:16:14.000 2020-01-02 03:44:35.000 974 99875 50424.5 5042450 974 99875 50424.5 5042450 -32594 32341 4549.3 454930 -127 124 -2.38 -238 -975 100 10965 99876 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792000000006 15142.79200 2020-01-01 2020-01-02 2020-01-01 00:16:15 2020-01-02 03:44:36 2020-01-01 00:16:15.000 2020-01-02 03:44:36.000 975 99876 50425.5 5042550 975 99876 50425.5 5042550 -32593 32342 4550.3 455030 -126 125 -1.38 -138 -976 100 10966 99877 2.93093 299.93093 151.43093 15143.09309 2.93093 299.93094 151.43092 15143.09289 2.93093 299.93093 151.43093000000033 15143.09300 2020-01-01 2020-01-02 2020-01-01 00:16:16 2020-01-02 03:44:37 2020-01-01 00:16:16.000 2020-01-02 03:44:37.000 976 99877 50426.5 5042650 976 99877 50426.5 5042650 -32592 32343 4551.3 455130 -125 126 -0.38 -38 -977 100 10967 99878 2.93393 299.93393 151.43393 15143.39339 2.93393 299.93393 151.43393 15143.39318 2.93393 299.93393 151.4339299999998 15143.39300 2020-01-01 2020-01-02 2020-01-01 00:16:17 2020-01-02 03:44:38 2020-01-01 00:16:17.000 2020-01-02 03:44:38.000 977 99878 50427.5 5042750 977 99878 50427.5 5042750 -32591 32344 4552.3 455230 -124 127 0.62 62 -978 100 10968 99879 2.93693 299.93693 151.43693 15143.69369 2.93693 299.93695 151.43693 15143.69378 2.93693 299.93693 151.43692999999988 15143.69300 2020-01-01 2020-01-02 2020-01-01 00:16:18 2020-01-02 03:44:39 2020-01-01 00:16:18.000 2020-01-02 03:44:39.000 978 99879 50428.5 5042850 978 99879 50428.5 5042850 -32590 32345 4553.3 455330 -128 127 -0.94 -94 -979 100 10969 99880 2.93993 299.93993 151.43993 15143.99399 2.93994 299.93994 151.43994 15143.99412 2.93993 299.93993 151.43993000000023 15143.99300 2020-01-01 2020-01-02 2020-01-01 00:16:19 2020-01-02 03:44:40 2020-01-01 00:16:19.000 2020-01-02 03:44:40.000 979 99880 50429.5 5042950 979 99880 50429.5 5042950 -32589 32346 4554.3 455430 -128 123 -2.5 -250 -98 102 10088 99998 0.29429 300.29429 150.29429 15179.72372 0.29429 300.29428 150.29429 15179.72363 0.29429 300.29429 150.29428999999985 15179.72329 2020-01-01 2020-01-02 2020-01-01 00:01:38 2020-01-02 03:46:38 2020-01-01 00:01:38.000 2020-01-02 03:46:38.000 98 99998 50048 5054848 98 99998 50048 5054848 -32471 32464 4627.009900990099 467328 -128 127 -1.2673267326732673 -128 -980 100 10970 99881 2.94294 299.94294 151.44294 15144.29429 2.94294 299.94293 151.44294 15144.29426 2.94294 299.94294 151.44294000000022 15144.29400 2020-01-01 2020-01-02 2020-01-01 00:16:20 2020-01-02 03:44:41 2020-01-01 00:16:20.000 2020-01-02 03:44:41.000 980 99881 50430.5 5043050 980 99881 50430.5 5043050 -32588 32347 4555.3 455530 -127 124 -1.5 -150 -981 100 10971 99882 2.94594 299.94594 151.44594 15144.59459 2.94594 299.94595 151.44595 15144.59501 2.94594 299.94594 151.44593999999972 15144.59400 2020-01-01 2020-01-02 2020-01-01 00:16:21 2020-01-02 03:44:42 2020-01-01 00:16:21.000 2020-01-02 03:44:42.000 981 99882 50431.5 5043150 981 99882 50431.5 5043150 -32587 32348 4556.3 455630 -126 125 -0.5 -50 -982 100 10972 99883 2.94894 299.94894 151.44894 15144.89489 2.94894 299.94894 151.44894 15144.89466 2.94894 299.94894 151.4489400000002 15144.89400 2020-01-01 2020-01-02 2020-01-01 00:16:22 2020-01-02 03:44:43 2020-01-01 00:16:22.000 2020-01-02 03:44:43.000 982 99883 50432.5 5043250 982 99883 50432.5 5043250 -32586 32349 4557.3 455730 -125 126 0.5 50 -983 100 10973 99884 2.95195 299.95195 151.45195 15145.19519 2.95195 299.95197 151.45195 15145.19525 2.95195 299.95195 151.4519500000001 15145.19500 2020-01-01 2020-01-02 2020-01-01 00:16:23 2020-01-02 03:44:44 2020-01-01 00:16:23.000 2020-01-02 03:44:44.000 983 99884 50433.5 5043350 983 99884 50433.5 5043350 -32585 32350 4558.3 455830 -124 127 1.5 150 -984 100 10974 99885 2.95495 299.95495 151.45495 15145.49549 2.95495 299.95496 151.45495 15145.49559 2.95495 299.95495 151.45494999999988 15145.49500 2020-01-01 2020-01-02 2020-01-01 00:16:24 2020-01-02 03:44:45 2020-01-01 00:16:24.000 2020-01-02 03:44:45.000 984 99885 50434.5 5043450 984 99885 50434.5 5043450 -32584 32351 4559.3 455930 -128 127 -0.06 -6 -985 100 10975 99886 2.95795 299.95795 151.45795 15145.79579 2.95795 299.95795 151.45795 15145.79573 2.95795 299.95795 151.45794999999998 15145.79500 2020-01-01 2020-01-02 2020-01-01 00:16:25 2020-01-02 03:44:46 2020-01-01 00:16:25.000 2020-01-02 03:44:46.000 985 99886 50435.5 5043550 985 99886 50435.5 5043550 -32583 32352 4560.3 456030 -128 123 -1.62 -162 -986 100 10976 99887 2.96096 299.96096 151.46096 15146.09609 2.96096 299.96097 151.46096 15146.09648 2.96096 299.96096 151.46096000000023 15146.09600 2020-01-01 2020-01-02 2020-01-01 00:16:26 2020-01-02 03:44:47 2020-01-01 00:16:26.000 2020-01-02 03:44:47.000 986 99887 50436.5 5043650 986 99887 50436.5 5043650 -32582 32353 4561.3 456130 -127 124 -0.62 -62 -987 100 10977 99888 2.96396 299.96396 151.46396 15146.39639 2.96396 299.96396 151.46396 15146.39612 2.96396 299.96396 151.46395999999973 15146.39600 2020-01-01 2020-01-02 2020-01-01 00:16:27 2020-01-02 03:44:48 2020-01-01 00:16:27.000 2020-01-02 03:44:48.000 987 99888 50437.5 5043750 987 99888 50437.5 5043750 -32581 32354 4562.3 456230 -126 125 0.38 38 -988 100 10978 99889 2.96696 299.96696 151.46696 15146.69669 2.96696 299.96698 151.46696 15146.69676 2.96696 299.96696 151.46695999999983 15146.69600 2020-01-01 2020-01-02 2020-01-01 00:16:28 2020-01-02 03:44:49 2020-01-01 00:16:28.000 2020-01-02 03:44:49.000 988 99889 50438.5 5043850 988 99889 50438.5 5043850 -32580 32355 4563.3 456330 -125 126 1.38 138 -989 100 10979 99890 2.96996 299.96996 151.46996 15146.99699 2.96997 299.96997 151.46997 15146.99706 2.96996 299.96996 151.46996000000024 15146.99600 2020-01-01 2020-01-02 2020-01-01 00:16:29 2020-01-02 03:44:50 2020-01-01 00:16:29.000 2020-01-02 03:44:50.000 989 99890 50439.5 5043950 989 99890 50439.5 5043950 -32579 32356 4564.3 456430 -124 127 2.38 238 -99 102 10089 99999 0.29729 300.29729 150.29729 15180.02702 0.29729 300.2973 150.29729 15180.02726 0.29729 300.29729 150.29729000000032 15180.02629 2020-01-01 2020-01-02 2020-01-01 00:01:39 2020-01-02 03:46:39 2020-01-01 00:01:39.000 2020-01-02 03:46:39.000 99 99999 50049 5054949 99 99999 50049 5054949 -32470 32465 4628.009900990099 467429 -128 123 -2.801980198019802 -283 -990 100 10980 99891 2.97297 299.97297 151.47297 15147.29729 2.97297 299.97296 151.47297 15147.29735 2.97297 299.97297 151.47297000000015 15147.29700 2020-01-01 2020-01-02 2020-01-01 00:16:30 2020-01-02 03:44:51 2020-01-01 00:16:30.000 2020-01-02 03:44:51.000 990 99891 50440.5 5044050 990 99891 50440.5 5044050 -32578 32357 4565.3 456530 -128 127 0.82 82 +942 100 10932 99843 2.82882 299.82882 151.32882 15132.88288 2.82882 299.82883 151.32882 15132.88275 2.82882 299.82882 151.32882 15132.88200 2020-01-01 2020-01-02 2020-01-01 00:15:42 2020-01-02 03:44:03 2020-01-01 00:15:42.000 2020-01-02 03:44:03.000 942 99843 50392.5 5039250 942 99843 50392.5 5039250 -32626 32309 4517.3 451730 -128 124 -1.1 -110 +943 100 10933 99844 2.83183 299.83183 151.33183 15133.18318 2.83183 299.83182 151.33183 15133.18304 2.83183 299.83183 151.33183 15133.18300 2020-01-01 2020-01-02 2020-01-01 00:15:43 2020-01-02 03:44:04 2020-01-01 00:15:43.000 2020-01-02 03:44:04.000 943 99844 50393.5 5039350 943 99844 50393.5 5039350 -32625 32310 4518.3 451830 -127 125 -0.1 -10 +944 100 10934 99845 2.83483 299.83483 151.33483 15133.48348 2.83483 299.83484 151.33483 15133.48364 2.83483 299.83483 151.33483 15133.48300 2020-01-01 2020-01-02 2020-01-01 00:15:44 2020-01-02 03:44:05 2020-01-01 00:15:44.000 2020-01-02 03:44:05.000 944 99845 50394.5 5039450 944 99845 50394.5 5039450 -32624 32311 4519.3 451930 -126 126 0.9 90 +945 100 10935 99846 2.83783 299.83783 151.33783 15133.78378 2.83783 299.83783 151.33783 15133.78393 2.83783 299.83783 151.33783 15133.78300 2020-01-01 2020-01-02 2020-01-01 00:15:45 2020-01-02 03:44:06 2020-01-01 00:15:45.000 2020-01-02 03:44:06.000 945 99846 50395.5 5039550 945 99846 50395.5 5039550 -32623 32312 4520.3 452030 -125 127 1.9 190 +946 100 10936 99847 2.84084 299.84084 151.34084 15134.08408 2.84084 299.84085 151.34084 15134.08404 2.84084 299.84084 151.34084000000001 15134.08400 2020-01-01 2020-01-02 2020-01-01 00:15:46 2020-01-02 03:44:07 2020-01-01 00:15:46.000 2020-01-02 03:44:07.000 946 99847 50396.5 5039650 946 99847 50396.5 5039650 -32622 32313 4521.3 452130 -128 127 0.34 34 +947 100 10937 99848 2.84384 299.84384 151.34384 15134.38438 2.84384 299.84384 151.34384 15134.38421 2.84384 299.84384 151.34384 15134.38400 2020-01-01 2020-01-02 2020-01-01 00:15:47 2020-01-02 03:44:08 2020-01-01 00:15:47.000 2020-01-02 03:44:08.000 947 99848 50397.5 5039750 947 99848 50397.5 5039750 -32621 32314 4522.3 452230 -128 127 -1.22 -122 +948 100 10938 99849 2.84684 299.84684 151.34684 15134.68468 2.84684 299.84683 151.34684 15134.68452 2.84684 299.84684 151.34684 15134.68400 2020-01-01 2020-01-02 2020-01-01 00:15:48 2020-01-02 03:44:09 2020-01-01 00:15:48.000 2020-01-02 03:44:09.000 948 99849 50398.5 5039850 948 99849 50398.5 5039850 -32620 32315 4523.3 452330 -128 123 -2.78 -278 +949 100 10939 99850 2.84984 299.84984 151.34984 15134.98498 2.84985 299.84985 151.34985 15134.98527 2.84984 299.84984 151.34984 15134.98400 2020-01-01 2020-01-02 2020-01-01 00:15:49 2020-01-02 03:44:10 2020-01-01 00:15:49.000 2020-01-02 03:44:10.000 949 99850 50399.5 5039950 949 99850 50399.5 5039950 -32619 32316 4524.3 452430 -127 124 -1.78 -178 +95 102 10085 99995 0.28528 300.28528 150.28528 15178.81381 0.28528 300.28528 150.28528 15178.81343 0.28528 300.28528 150.28528 15178.81328 2020-01-01 2020-01-02 2020-01-01 00:01:35 2020-01-02 03:46:35 2020-01-01 00:01:35.000 2020-01-02 03:46:35.000 95 99995 50045 5054545 95 99995 50045 5054545 -32474 32461 4624.009900990099 467025 -125 126 0.801980198019802 81 +950 100 10940 99851 2.85285 299.85285 151.35285 15135.28528 2.85285 299.85284 151.35285 15135.28541 2.85285 299.85285 151.35285 15135.28500 2020-01-01 2020-01-02 2020-01-01 00:15:50 2020-01-02 03:44:11 2020-01-01 00:15:50.000 2020-01-02 03:44:11.000 950 99851 50400.5 5040050 950 99851 50400.5 5040050 -32618 32317 4525.3 452530 -126 125 -0.78 -78 +951 100 10941 99852 2.85585 299.85585 151.35585 15135.58558 2.85585 299.85587 151.35585 15135.58551 2.85585 299.85585 151.35585 15135.58500 2020-01-01 2020-01-02 2020-01-01 00:15:51 2020-01-02 03:44:12 2020-01-01 00:15:51.000 2020-01-02 03:44:12.000 951 99852 50401.5 5040150 951 99852 50401.5 5040150 -32617 32318 4526.3 452630 -125 126 0.22 22 +952 100 10942 99853 2.85885 299.85885 151.35885 15135.88588 2.85885 299.85886 151.35885 15135.88568 2.85885 299.85885 151.35885 15135.88500 2020-01-01 2020-01-02 2020-01-01 00:15:52 2020-01-02 03:44:13 2020-01-01 00:15:52.000 2020-01-02 03:44:13.000 952 99853 50402.5 5040250 952 99853 50402.5 5040250 -32616 32319 4527.3 452730 -124 127 1.22 122 +953 100 10943 99854 2.86186 299.86186 151.36186 15136.18618 2.86186 299.86185 151.36185 15136.18598 2.86186 299.86186 151.36186 15136.18600 2020-01-01 2020-01-02 2020-01-01 00:15:53 2020-01-02 03:44:14 2020-01-01 00:15:53.000 2020-01-02 03:44:14.000 953 99854 50403.5 5040350 953 99854 50403.5 5040350 -32615 32320 4528.3 452830 -128 127 -0.34 -34 +954 100 10944 99855 2.86486 299.86486 151.36486 15136.48648 2.86486 299.86487 151.36486 15136.48674 2.86486 299.86486 151.36486000000002 15136.48600 2020-01-01 2020-01-02 2020-01-01 00:15:54 2020-01-02 03:44:15 2020-01-01 00:15:54.000 2020-01-02 03:44:15.000 954 99855 50404.5 5040450 954 99855 50404.5 5040450 -32614 32321 4529.3 452930 -128 123 -1.9 -190 +955 100 10945 99856 2.86786 299.86786 151.36786 15136.78678 2.86786 299.86786 151.36786 15136.78688 2.86786 299.86786 151.36786 15136.78600 2020-01-01 2020-01-02 2020-01-01 00:15:55 2020-01-02 03:44:16 2020-01-01 00:15:55.000 2020-01-02 03:44:16.000 955 99856 50405.5 5040550 955 99856 50405.5 5040550 -32613 32322 4530.3 453030 -127 124 -0.9 -90 +956 100 10946 99857 2.87087 299.87087 151.37087 15137.08708 2.87087 299.87088 151.37087 15137.08701 2.87087 299.87087 151.37087 15137.08700 2020-01-01 2020-01-02 2020-01-01 00:15:56 2020-01-02 03:44:17 2020-01-01 00:15:56.000 2020-01-02 03:44:17.000 956 99857 50406.5 5040650 956 99857 50406.5 5040650 -32612 32323 4531.3 453130 -126 125 0.1 10 +957 100 10947 99858 2.87387 299.87387 151.37387 15137.38738 2.87387 299.87387 151.37387 15137.38716 2.87387 299.87387 151.37387 15137.38700 2020-01-01 2020-01-02 2020-01-01 00:15:57 2020-01-02 03:44:18 2020-01-01 00:15:57.000 2020-01-02 03:44:18.000 957 99858 50407.5 5040750 957 99858 50407.5 5040750 -32611 32324 4532.3 453230 -125 126 1.1 110 +958 100 10948 99859 2.87687 299.87687 151.37687 15137.68768 2.87687 299.8769 151.37687 15137.68791 2.87687 299.87687 151.37687 15137.68700 2020-01-01 2020-01-02 2020-01-01 00:15:58 2020-01-02 03:44:19 2020-01-01 00:15:58.000 2020-01-02 03:44:19.000 958 99859 50408.5 5040850 958 99859 50408.5 5040850 -32610 32325 4533.3 453330 -124 127 2.1 210 +959 100 10949 99860 2.87987 299.87987 151.37987 15137.98798 2.87988 299.87988 151.37988 15137.9882 2.87987 299.87987 151.37986999999998 15137.98700 2020-01-01 2020-01-02 2020-01-01 00:15:59 2020-01-02 03:44:20 2020-01-01 00:15:59.000 2020-01-02 03:44:20.000 959 99860 50409.5 5040950 959 99860 50409.5 5040950 -32609 32326 4534.3 453430 -128 127 0.54 54 +96 102 10086 99996 0.28828 300.28828 150.28828 15179.11711 0.28828 300.2883 150.28828 15179.11718 0.28828 300.28828 150.28828000000001 15179.11628 2020-01-01 2020-01-02 2020-01-01 00:01:36 2020-01-02 03:46:36 2020-01-01 00:01:36.000 2020-01-02 03:46:36.000 96 99996 50046 5054646 96 99996 50046 5054646 -32473 32462 4625.009900990099 467126 -124 127 1.801980198019802 182 +960 100 10950 99861 2.88288 299.88288 151.38288 15138.28828 2.88288 299.88287 151.38288 15138.28834 2.88288 299.88288 151.38288 15138.28800 2020-01-01 2020-01-02 2020-01-01 00:16:00 2020-01-02 03:44:21 2020-01-01 00:16:00.000 2020-01-02 03:44:21.000 960 99861 50410.5 5041050 960 99861 50410.5 5041050 -32608 32327 4535.3 453530 -128 123 -1.02 -102 +961 100 10951 99862 2.88588 299.88588 151.38588 15138.58858 2.88588 299.8859 151.38588 15138.58848 2.88588 299.88588 151.38588 15138.58800 2020-01-01 2020-01-02 2020-01-01 00:16:01 2020-01-02 03:44:22 2020-01-01 00:16:01.000 2020-01-02 03:44:22.000 961 99862 50411.5 5041150 961 99862 50411.5 5041150 -32607 32328 4536.3 453630 -127 124 -0.02 -2 +962 100 10952 99863 2.88888 299.88888 151.38888 15138.88888 2.88888 299.8889 151.38888 15138.88862 2.88888 299.88888 151.38888 15138.88800 2020-01-01 2020-01-02 2020-01-01 00:16:02 2020-01-02 03:44:23 2020-01-01 00:16:02.000 2020-01-02 03:44:23.000 962 99863 50412.5 5041250 962 99863 50412.5 5041250 -32606 32329 4537.3 453730 -126 125 0.98 98 +963 100 10953 99864 2.89189 299.89189 151.39189 15139.18918 2.89189 299.8919 151.39189 15139.18937 2.89189 299.89189 151.39189 15139.18900 2020-01-01 2020-01-02 2020-01-01 00:16:03 2020-01-02 03:44:24 2020-01-01 00:16:03.000 2020-01-02 03:44:24.000 963 99864 50413.5 5041350 963 99864 50413.5 5041350 -32605 32330 4538.3 453830 -125 126 1.98 198 +964 100 10954 99865 2.89489 299.89489 151.39489 15139.48948 2.89489 299.8949 151.39489 15139.48968 2.89489 299.89489 151.39489 15139.48900 2020-01-01 2020-01-02 2020-01-01 00:16:04 2020-01-02 03:44:25 2020-01-01 00:16:04.000 2020-01-02 03:44:25.000 964 99865 50414.5 5041450 964 99865 50414.5 5041450 -32604 32331 4539.3 453930 -124 127 2.98 298 +965 100 10955 99866 2.89789 299.89789 151.39789 15139.78978 2.89789 299.8979 151.39789 15139.78985 2.89789 299.89789 151.39789000000002 15139.78900 2020-01-01 2020-01-02 2020-01-01 00:16:05 2020-01-02 03:44:26 2020-01-01 00:16:05.000 2020-01-02 03:44:26.000 965 99866 50415.5 5041550 965 99866 50415.5 5041550 -32603 32332 4540.3 454030 -128 127 1.42 142 +966 100 10956 99867 2.9009 299.9009 151.4009 15140.09009 2.9009 299.9009 151.40089 15140.08996 2.90090 299.90090 151.4009 15140.09000 2020-01-01 2020-01-02 2020-01-01 00:16:06 2020-01-02 03:44:27 2020-01-01 00:16:06.000 2020-01-02 03:44:27.000 966 99867 50416.5 5041650 966 99867 50416.5 5041650 -32602 32333 4541.3 454130 -128 127 -0.14 -14 +967 100 10957 99868 2.9039 299.9039 151.4039 15140.39039 2.9039 299.9039 151.4039 15140.39009 2.90390 299.90390 151.4039 15140.39000 2020-01-01 2020-01-02 2020-01-01 00:16:07 2020-01-02 03:44:28 2020-01-01 00:16:07.000 2020-01-02 03:44:28.000 967 99868 50417.5 5041750 967 99868 50417.5 5041750 -32601 32334 4542.3 454230 -128 124 -1.7 -170 +968 100 10958 99869 2.9069 299.9069 151.4069 15140.69069 2.9069 299.90692 151.4069 15140.69084 2.90690 299.90690 151.4069 15140.69000 2020-01-01 2020-01-02 2020-01-01 00:16:08 2020-01-02 03:44:29 2020-01-01 00:16:08.000 2020-01-02 03:44:29.000 968 99869 50418.5 5041850 968 99869 50418.5 5041850 -32600 32335 4543.3 454330 -127 125 -0.7 -70 +969 100 10959 99870 2.9099 299.9099 151.4099 15140.99099 2.90991 299.9099 151.40991 15140.99114 2.90990 299.90990 151.4099 15140.99000 2020-01-01 2020-01-02 2020-01-01 00:16:09 2020-01-02 03:44:30 2020-01-01 00:16:09.000 2020-01-02 03:44:30.000 969 99870 50419.5 5041950 969 99870 50419.5 5041950 -32599 32336 4544.3 454430 -126 126 0.3 30 +97 102 10087 99997 0.29129 300.29129 150.29129 15179.42042 0.29129 300.2913 150.29129 15179.42033 0.29129 300.29129 150.29129 15179.42029 2020-01-01 2020-01-02 2020-01-01 00:01:37 2020-01-02 03:46:37 2020-01-01 00:01:37.000 2020-01-02 03:46:37.000 97 99997 50047 5054747 97 99997 50047 5054747 -32472 32463 4626.009900990099 467227 -128 127 0.26732673267326734 27 +970 100 10960 99871 2.91291 299.91291 151.41291 15141.29129 2.91291 299.9129 151.41291 15141.29132 2.91291 299.91291 151.41290999999998 15141.29100 2020-01-01 2020-01-02 2020-01-01 00:16:10 2020-01-02 03:44:31 2020-01-01 00:16:10.000 2020-01-02 03:44:31.000 970 99871 50420.5 5042050 970 99871 50420.5 5042050 -32598 32337 4545.3 454530 -125 127 1.3 130 +971 100 10961 99872 2.91591 299.91591 151.41591 15141.59159 2.91591 299.91592 151.41591 15141.59142 2.91591 299.91591 151.41591 15141.59100 2020-01-01 2020-01-02 2020-01-01 00:16:11 2020-01-02 03:44:32 2020-01-01 00:16:11.000 2020-01-02 03:44:32.000 971 99872 50421.5 5042150 971 99872 50421.5 5042150 -32597 32338 4546.3 454630 -128 127 -0.26 -26 +972 100 10962 99873 2.91891 299.91891 151.41891 15141.89189 2.91891 299.9189 151.41891 15141.89172 2.91891 299.91891 151.41890999999998 15141.89100 2020-01-01 2020-01-02 2020-01-01 00:16:12 2020-01-02 03:44:33 2020-01-01 00:16:12.000 2020-01-02 03:44:33.000 972 99873 50422.5 5042250 972 99873 50422.5 5042250 -32596 32339 4547.3 454730 -128 127 -1.82 -182 +973 100 10963 99874 2.92192 299.92192 151.42192 15142.19219 2.92192 299.92194 151.42192 15142.19232 2.92192 299.92192 151.42192 15142.19200 2020-01-01 2020-01-02 2020-01-01 00:16:13 2020-01-02 03:44:34 2020-01-01 00:16:13.000 2020-01-02 03:44:34.000 973 99874 50423.5 5042350 973 99874 50423.5 5042350 -32595 32340 4548.3 454830 -128 123 -3.38 -338 +974 100 10964 99875 2.92492 299.92492 151.42492 15142.49249 2.92492 299.92493 151.42492 15142.49265 2.92492 299.92492 151.42492000000001 15142.49200 2020-01-01 2020-01-02 2020-01-01 00:16:14 2020-01-02 03:44:35 2020-01-01 00:16:14.000 2020-01-02 03:44:35.000 974 99875 50424.5 5042450 974 99875 50424.5 5042450 -32594 32341 4549.3 454930 -127 124 -2.38 -238 +975 100 10965 99876 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792 15142.79200 2020-01-01 2020-01-02 2020-01-01 00:16:15 2020-01-02 03:44:36 2020-01-01 00:16:15.000 2020-01-02 03:44:36.000 975 99876 50425.5 5042550 975 99876 50425.5 5042550 -32593 32342 4550.3 455030 -126 125 -1.38 -138 +976 100 10966 99877 2.93093 299.93093 151.43093 15143.09309 2.93093 299.93094 151.43092 15143.09289 2.93093 299.93093 151.43093000000002 15143.09300 2020-01-01 2020-01-02 2020-01-01 00:16:16 2020-01-02 03:44:37 2020-01-01 00:16:16.000 2020-01-02 03:44:37.000 976 99877 50426.5 5042650 976 99877 50426.5 5042650 -32592 32343 4551.3 455130 -125 126 -0.38 -38 +977 100 10967 99878 2.93393 299.93393 151.43393 15143.39339 2.93393 299.93393 151.43393 15143.39318 2.93393 299.93393 151.43393 15143.39300 2020-01-01 2020-01-02 2020-01-01 00:16:17 2020-01-02 03:44:38 2020-01-01 00:16:17.000 2020-01-02 03:44:38.000 977 99878 50427.5 5042750 977 99878 50427.5 5042750 -32591 32344 4552.3 455230 -124 127 0.62 62 +978 100 10968 99879 2.93693 299.93693 151.43693 15143.69369 2.93693 299.93695 151.43693 15143.69378 2.93693 299.93693 151.43693 15143.69300 2020-01-01 2020-01-02 2020-01-01 00:16:18 2020-01-02 03:44:39 2020-01-01 00:16:18.000 2020-01-02 03:44:39.000 978 99879 50428.5 5042850 978 99879 50428.5 5042850 -32590 32345 4553.3 455330 -128 127 -0.94 -94 +979 100 10969 99880 2.93993 299.93993 151.43993 15143.99399 2.93994 299.93994 151.43994 15143.99412 2.93993 299.93993 151.43993 15143.99300 2020-01-01 2020-01-02 2020-01-01 00:16:19 2020-01-02 03:44:40 2020-01-01 00:16:19.000 2020-01-02 03:44:40.000 979 99880 50429.5 5042950 979 99880 50429.5 5042950 -32589 32346 4554.3 455430 -128 123 -2.5 -250 +98 102 10088 99998 0.29429 300.29429 150.29429 15179.72372 0.29429 300.29428 150.29429 15179.72363 0.29429 300.29429 150.29429 15179.72329 2020-01-01 2020-01-02 2020-01-01 00:01:38 2020-01-02 03:46:38 2020-01-01 00:01:38.000 2020-01-02 03:46:38.000 98 99998 50048 5054848 98 99998 50048 5054848 -32471 32464 4627.009900990099 467328 -128 127 -1.2673267326732673 -128 +980 100 10970 99881 2.94294 299.94294 151.44294 15144.29429 2.94294 299.94293 151.44294 15144.29426 2.94294 299.94294 151.44294 15144.29400 2020-01-01 2020-01-02 2020-01-01 00:16:20 2020-01-02 03:44:41 2020-01-01 00:16:20.000 2020-01-02 03:44:41.000 980 99881 50430.5 5043050 980 99881 50430.5 5043050 -32588 32347 4555.3 455530 -127 124 -1.5 -150 +981 100 10971 99882 2.94594 299.94594 151.44594 15144.59459 2.94594 299.94595 151.44595 15144.59501 2.94594 299.94594 151.44593999999998 15144.59400 2020-01-01 2020-01-02 2020-01-01 00:16:21 2020-01-02 03:44:42 2020-01-01 00:16:21.000 2020-01-02 03:44:42.000 981 99882 50431.5 5043150 981 99882 50431.5 5043150 -32587 32348 4556.3 455630 -126 125 -0.5 -50 +982 100 10972 99883 2.94894 299.94894 151.44894 15144.89489 2.94894 299.94894 151.44894 15144.89466 2.94894 299.94894 151.44894 15144.89400 2020-01-01 2020-01-02 2020-01-01 00:16:22 2020-01-02 03:44:43 2020-01-01 00:16:22.000 2020-01-02 03:44:43.000 982 99883 50432.5 5043250 982 99883 50432.5 5043250 -32586 32349 4557.3 455730 -125 126 0.5 50 +983 100 10973 99884 2.95195 299.95195 151.45195 15145.19519 2.95195 299.95197 151.45195 15145.19525 2.95195 299.95195 151.45195 15145.19500 2020-01-01 2020-01-02 2020-01-01 00:16:23 2020-01-02 03:44:44 2020-01-01 00:16:23.000 2020-01-02 03:44:44.000 983 99884 50433.5 5043350 983 99884 50433.5 5043350 -32585 32350 4558.3 455830 -124 127 1.5 150 +984 100 10974 99885 2.95495 299.95495 151.45495 15145.49549 2.95495 299.95496 151.45495 15145.49559 2.95495 299.95495 151.45495 15145.49500 2020-01-01 2020-01-02 2020-01-01 00:16:24 2020-01-02 03:44:45 2020-01-01 00:16:24.000 2020-01-02 03:44:45.000 984 99885 50434.5 5043450 984 99885 50434.5 5043450 -32584 32351 4559.3 455930 -128 127 -0.06 -6 +985 100 10975 99886 2.95795 299.95795 151.45795 15145.79579 2.95795 299.95795 151.45795 15145.79573 2.95795 299.95795 151.45795 15145.79500 2020-01-01 2020-01-02 2020-01-01 00:16:25 2020-01-02 03:44:46 2020-01-01 00:16:25.000 2020-01-02 03:44:46.000 985 99886 50435.5 5043550 985 99886 50435.5 5043550 -32583 32352 4560.3 456030 -128 123 -1.62 -162 +986 100 10976 99887 2.96096 299.96096 151.46096 15146.09609 2.96096 299.96097 151.46096 15146.09648 2.96096 299.96096 151.46096 15146.09600 2020-01-01 2020-01-02 2020-01-01 00:16:26 2020-01-02 03:44:47 2020-01-01 00:16:26.000 2020-01-02 03:44:47.000 986 99887 50436.5 5043650 986 99887 50436.5 5043650 -32582 32353 4561.3 456130 -127 124 -0.62 -62 +987 100 10977 99888 2.96396 299.96396 151.46396 15146.39639 2.96396 299.96396 151.46396 15146.39612 2.96396 299.96396 151.46396000000001 15146.39600 2020-01-01 2020-01-02 2020-01-01 00:16:27 2020-01-02 03:44:48 2020-01-01 00:16:27.000 2020-01-02 03:44:48.000 987 99888 50437.5 5043750 987 99888 50437.5 5043750 -32581 32354 4562.3 456230 -126 125 0.38 38 +988 100 10978 99889 2.96696 299.96696 151.46696 15146.69669 2.96696 299.96698 151.46696 15146.69676 2.96696 299.96696 151.46696 15146.69600 2020-01-01 2020-01-02 2020-01-01 00:16:28 2020-01-02 03:44:49 2020-01-01 00:16:28.000 2020-01-02 03:44:49.000 988 99889 50438.5 5043850 988 99889 50438.5 5043850 -32580 32355 4563.3 456330 -125 126 1.38 138 +989 100 10979 99890 2.96996 299.96996 151.46996 15146.99699 2.96997 299.96997 151.46997 15146.99706 2.96996 299.96996 151.46996 15146.99600 2020-01-01 2020-01-02 2020-01-01 00:16:29 2020-01-02 03:44:50 2020-01-01 00:16:29.000 2020-01-02 03:44:50.000 989 99890 50439.5 5043950 989 99890 50439.5 5043950 -32579 32356 4564.3 456430 -124 127 2.38 238 +99 102 10089 99999 0.29729 300.29729 150.29729 15180.02702 0.29729 300.2973 150.29729 15180.02726 0.29729 300.29729 150.29729 15180.02629 2020-01-01 2020-01-02 2020-01-01 00:01:39 2020-01-02 03:46:39 2020-01-01 00:01:39.000 2020-01-02 03:46:39.000 99 99999 50049 5054949 99 99999 50049 5054949 -32470 32465 4628.009900990099 467429 -128 123 -2.801980198019802 -283 +990 100 10980 99891 2.97297 299.97297 151.47297 15147.29729 2.97297 299.97296 151.47297 15147.29735 2.97297 299.97297 151.47297 15147.29700 2020-01-01 2020-01-02 2020-01-01 00:16:30 2020-01-02 03:44:51 2020-01-01 00:16:30.000 2020-01-02 03:44:51.000 990 99891 50440.5 5044050 990 99891 50440.5 5044050 -32578 32357 4565.3 456530 -128 127 0.82 82 991 100 10981 99892 2.97597 299.97597 151.47597 15147.59759 2.97597 299.97598 151.47597 15147.59795 2.97597 299.97597 151.47597 15147.59700 2020-01-01 2020-01-02 2020-01-01 00:16:31 2020-01-02 03:44:52 2020-01-01 00:16:31.000 2020-01-02 03:44:52.000 991 99892 50441.5 5044150 991 99892 50441.5 5044150 -32577 32358 4566.3 456630 -128 127 -0.74 -74 -992 100 10982 99893 2.97897 299.97897 151.47897 15147.89789 2.97897 299.97897 151.47897 15147.89759 2.97897 299.97897 151.4789700000001 15147.89700 2020-01-01 2020-01-02 2020-01-01 00:16:32 2020-01-02 03:44:53 2020-01-01 00:16:32.000 2020-01-02 03:44:53.000 992 99893 50442.5 5044250 992 99893 50442.5 5044250 -32576 32359 4567.3 456730 -128 124 -2.3 -230 +992 100 10982 99893 2.97897 299.97897 151.47897 15147.89789 2.97897 299.97897 151.47897 15147.89759 2.97897 299.97897 151.47897 15147.89700 2020-01-01 2020-01-02 2020-01-01 00:16:32 2020-01-02 03:44:53 2020-01-01 00:16:32.000 2020-01-02 03:44:53.000 992 99893 50442.5 5044250 992 99893 50442.5 5044250 -32576 32359 4567.3 456730 -128 124 -2.3 -230 993 100 10983 99894 2.98198 299.98198 151.48198 15148.19819 2.98198 299.982 151.48198 15148.19823 2.98198 299.98198 151.48198 15148.19800 2020-01-01 2020-01-02 2020-01-01 00:16:33 2020-01-02 03:44:54 2020-01-01 00:16:33.000 2020-01-02 03:44:54.000 993 99894 50443.5 5044350 993 99894 50443.5 5044350 -32575 32360 4568.3 456830 -127 125 -1.3 -130 -994 100 10984 99895 2.98498 299.98498 151.48498 15148.49849 2.98498 299.985 151.48498 15148.49853 2.98498 299.98498 151.48497999999984 15148.49800 2020-01-01 2020-01-02 2020-01-01 00:16:34 2020-01-02 03:44:55 2020-01-01 00:16:34.000 2020-01-02 03:44:55.000 994 99895 50444.5 5044450 994 99895 50444.5 5044450 -32574 32361 4569.3 456930 -126 126 -0.3 -30 -995 100 10985 99896 2.98798 299.98798 151.48798 15148.79879 2.98798 299.98798 151.48798 15148.79882 2.98798 299.98798 151.4879799999999 15148.79800 2020-01-01 2020-01-02 2020-01-01 00:16:35 2020-01-02 03:44:56 2020-01-01 00:16:35.000 2020-01-02 03:44:56.000 995 99896 50445.5 5044550 995 99896 50445.5 5044550 -32573 32362 4570.3 457030 -125 127 0.7 70 -996 100 10986 99897 2.99099 299.99099 151.49099 15149.09909 2.99099 299.991 151.49099 15149.09942 2.99099 299.99099 151.49099000000015 15149.09900 2020-01-01 2020-01-02 2020-01-01 00:16:36 2020-01-02 03:44:57 2020-01-01 00:16:36.000 2020-01-02 03:44:57.000 996 99897 50446.5 5044650 996 99897 50446.5 5044650 -32572 32363 4571.3 457130 -128 127 -0.86 -86 -997 100 10987 99898 2.99399 299.99399 151.49399 15149.39939 2.99399 299.994 151.49399 15149.3991 2.99399 299.99399 151.49399000000025 15149.39900 2020-01-01 2020-01-02 2020-01-01 00:16:37 2020-01-02 03:44:58 2020-01-01 00:16:37.000 2020-01-02 03:44:58.000 997 99898 50447.5 5044750 997 99898 50447.5 5044750 -32571 32364 4572.3 457230 -128 127 -2.42 -242 -998 100 10988 99899 2.99699 299.99699 151.49699 15149.69969 2.99699 299.997 151.49699 15149.6997 2.99699 299.99699 151.49698999999976 15149.69900 2020-01-01 2020-01-02 2020-01-01 00:16:38 2020-01-02 03:44:59 2020-01-01 00:16:38.000 2020-01-02 03:44:59.000 998 99899 50448.5 5044850 998 99899 50448.5 5044850 -32570 32365 4573.3 457330 -128 123 -3.98 -398 +994 100 10984 99895 2.98498 299.98498 151.48498 15148.49849 2.98498 299.985 151.48498 15148.49853 2.98498 299.98498 151.48498 15148.49800 2020-01-01 2020-01-02 2020-01-01 00:16:34 2020-01-02 03:44:55 2020-01-01 00:16:34.000 2020-01-02 03:44:55.000 994 99895 50444.5 5044450 994 99895 50444.5 5044450 -32574 32361 4569.3 456930 -126 126 -0.3 -30 +995 100 10985 99896 2.98798 299.98798 151.48798 15148.79879 2.98798 299.98798 151.48798 15148.79882 2.98798 299.98798 151.48798 15148.79800 2020-01-01 2020-01-02 2020-01-01 00:16:35 2020-01-02 03:44:56 2020-01-01 00:16:35.000 2020-01-02 03:44:56.000 995 99896 50445.5 5044550 995 99896 50445.5 5044550 -32573 32362 4570.3 457030 -125 127 0.7 70 +996 100 10986 99897 2.99099 299.99099 151.49099 15149.09909 2.99099 299.991 151.49099 15149.09942 2.99099 299.99099 151.49099 15149.09900 2020-01-01 2020-01-02 2020-01-01 00:16:36 2020-01-02 03:44:57 2020-01-01 00:16:36.000 2020-01-02 03:44:57.000 996 99897 50446.5 5044650 996 99897 50446.5 5044650 -32572 32363 4571.3 457130 -128 127 -0.86 -86 +997 100 10987 99898 2.99399 299.99399 151.49399 15149.39939 2.99399 299.994 151.49399 15149.3991 2.99399 299.99399 151.49399 15149.39900 2020-01-01 2020-01-02 2020-01-01 00:16:37 2020-01-02 03:44:58 2020-01-01 00:16:37.000 2020-01-02 03:44:58.000 997 99898 50447.5 5044750 997 99898 50447.5 5044750 -32571 32364 4572.3 457230 -128 127 -2.42 -242 +998 100 10988 99899 2.99699 299.99699 151.49699 15149.69969 2.99699 299.997 151.49699 15149.6997 2.99699 299.99699 151.49699 15149.69900 2020-01-01 2020-01-02 2020-01-01 00:16:38 2020-01-02 03:44:59 2020-01-01 00:16:38.000 2020-01-02 03:44:59.000 998 99899 50448.5 5044850 998 99899 50448.5 5044850 -32570 32365 4573.3 457330 -128 123 -3.98 -398 ---- select with states ---- -1 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N -2 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N @@ -1012,1004 +1012,1004 @@ -4 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N -5 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 0 2 0 99900 0 300 150 15150 0 300 150 15150 0.00000 300.00000 150 15150.00000 2020-01-01 2020-01-02 2020-01-01 00:00:00 2020-01-02 03:45:00 2020-01-01 00:00:00.000 2020-01-02 03:45:00.000 0 99900 49950 5044950 0 99900 49950 5044950 -32569 32366 4529.009900990099 457430 -127 124 -2.9504950495049505 -298 -1 2 1 9991 0.003 300.003 150.003 15150.3033 0.003 300.003 150.003 15150.30329 0.00300 300.00300 150.00300000000007 15150.30300 2020-01-01 2020-01-02 2020-01-01 00:00:01 2020-01-02 03:45:01 2020-01-01 00:00:01.000 2020-01-02 03:45:01.000 1 99901 49951 5045051 1 99901 49951 5045051 -32568 32367 4530.009900990099 457531 -126 125 -1.9504950495049505 -197 -10 2 10 99910 0.03003 300.03003 150.03003 15153.03303 0.03003 300.03003 150.03002 15153.03296 0.03003 300.03003 150.03002999999993 15153.03303 2020-01-01 2020-01-02 2020-01-01 00:00:10 2020-01-02 03:45:10 2020-01-01 00:00:10.000 2020-01-02 03:45:10.000 10 99910 49960 5045960 10 99910 49960 5045960 -32559 32376 4539.009900990099 458440 -128 127 -0.5544554455445545 -56 -100 2 100 99001 0.3003 297.3003 148.8003 14880.03003 0.3003 297.3003 148.80029 14880.02962 0.30030 297.30030 148.8003000000001 14880.03000 2020-01-01 2020-01-02 2020-01-01 00:01:40 2020-01-02 03:30:01 2020-01-01 00:01:40.000 2020-01-02 03:30:01.000 100 99001 49550.5 4955050 100 99001 49550.5 4955050 -32469 32466 4986.02 498602 -127 124 -0.86 -86 -101 2 10091 99002 0.3033 297.3033 148.8033 14880.33033 0.3033 297.3033 148.8033 14880.33035 0.30330 297.30330 148.80329999999978 14880.33000 2020-01-01 2020-01-02 2020-01-01 00:01:41 2020-01-02 03:30:02 2020-01-01 00:01:41.000 2020-01-02 03:30:02.000 101 99002 49551.5 4955150 101 99002 49551.5 4955150 -32468 32467 4987.02 498702 -126 125 0.14 14 -102 2 10092 99003 0.3063 297.3063 148.8063 14880.63063 0.3063 297.3063 148.8063 14880.6305 0.30630 297.30630 148.80630000000014 14880.63000 2020-01-01 2020-01-02 2020-01-01 00:01:42 2020-01-02 03:30:03 2020-01-01 00:01:42.000 2020-01-02 03:30:03.000 102 99003 49552.5 4955250 102 99003 49552.5 4955250 -32467 32468 4988.02 498802 -125 126 1.14 114 -103 2 10093 99004 0.3093 297.3093 148.8093 14880.93093 0.3093 297.3093 148.8093 14880.93085 0.30930 297.30930 148.80930000000012 14880.93000 2020-01-01 2020-01-02 2020-01-01 00:01:43 2020-01-02 03:30:04 2020-01-01 00:01:43.000 2020-01-02 03:30:04.000 103 99004 49553.5 4955350 103 99004 49553.5 4955350 -32466 32469 4989.02 498902 -124 127 2.14 214 -104 2 10094 99005 0.31231 297.31231 148.81231 14881.23123 0.31231 297.31232 148.81231 14881.23144 0.31231 297.31231 148.81230999999988 14881.23100 2020-01-01 2020-01-02 2020-01-01 00:01:44 2020-01-02 03:30:05 2020-01-01 00:01:44.000 2020-01-02 03:30:05.000 104 99005 49554.5 4955450 104 99005 49554.5 4955450 -32465 32470 4990.02 499002 -128 127 0.58 58 -105 2 10095 99006 0.31531 297.31531 148.81531 14881.53153 0.31531 297.3153 148.81531 14881.53174 0.31531 297.31531 148.81530999999998 14881.53100 2020-01-01 2020-01-02 2020-01-01 00:01:45 2020-01-02 03:30:06 2020-01-01 00:01:45.000 2020-01-02 03:30:06.000 105 99006 49555.5 4955550 105 99006 49555.5 4955550 -32464 32471 4991.02 499102 -128 123 -0.98 -98 -106 2 10096 99007 0.31831 297.31831 148.81831 14881.83183 0.31831 297.31833 148.81831 14881.83182 0.31831 297.31831 148.81831000000003 14881.83100 2020-01-01 2020-01-02 2020-01-01 00:01:46 2020-01-02 03:30:07 2020-01-01 00:01:46.000 2020-01-02 03:30:07.000 106 99007 49556.5 4955650 106 99007 49556.5 4955650 -32463 32472 4992.02 499202 -127 124 0.02 2 -107 2 10097 99008 0.32132 297.32132 148.82132 14882.13213 0.32132 297.32132 148.82131 14882.13197 0.32132 297.32132 148.8213199999998 14882.13200 2020-01-01 2020-01-02 2020-01-01 00:01:47 2020-01-02 03:30:08 2020-01-01 00:01:47.000 2020-01-02 03:30:08.000 107 99008 49557.5 4955750 107 99008 49557.5 4955750 -32462 32473 4993.02 499302 -126 125 1.02 102 -108 2 10098 99009 0.32432 297.32432 148.82432 14882.43243 0.32432 297.3243 148.82432 14882.43232 0.32432 297.32432 148.82431999999986 14882.43200 2020-01-01 2020-01-02 2020-01-01 00:01:48 2020-01-02 03:30:09 2020-01-01 00:01:48.000 2020-01-02 03:30:09.000 108 99009 49558.5 4955850 108 99009 49558.5 4955850 -32461 32474 4994.02 499402 -125 126 2.02 202 -109 2 10099 99010 0.32732 297.32732 148.82732 14882.73273 0.32732 297.32733 148.82732 14882.7329 0.32732 297.32732 148.82732000000016 14882.73200 2020-01-01 2020-01-02 2020-01-01 00:01:49 2020-01-02 03:30:10 2020-01-01 00:01:49.000 2020-01-02 03:30:10.000 109 99010 49559.5 4955950 109 99010 49559.5 4955950 -32460 32475 4995.02 499502 -124 127 3.02 302 +1 2 1 9991 0.003 300.003 150.003 15150.3033 0.003 300.003 150.003 15150.30329 0.00300 300.00300 150.003 15150.30300 2020-01-01 2020-01-02 2020-01-01 00:00:01 2020-01-02 03:45:01 2020-01-01 00:00:01.000 2020-01-02 03:45:01.000 1 99901 49951 5045051 1 99901 49951 5045051 -32568 32367 4530.009900990099 457531 -126 125 -1.9504950495049505 -197 +10 2 10 99910 0.03003 300.03003 150.03003 15153.03303 0.03003 300.03003 150.03002 15153.03296 0.03003 300.03003 150.03003 15153.03303 2020-01-01 2020-01-02 2020-01-01 00:00:10 2020-01-02 03:45:10 2020-01-01 00:00:10.000 2020-01-02 03:45:10.000 10 99910 49960 5045960 10 99910 49960 5045960 -32559 32376 4539.009900990099 458440 -128 127 -0.5544554455445545 -56 +100 2 100 99001 0.3003 297.3003 148.8003 14880.03003 0.3003 297.3003 148.80029 14880.02962 0.30030 297.30030 148.8003 14880.03000 2020-01-01 2020-01-02 2020-01-01 00:01:40 2020-01-02 03:30:01 2020-01-01 00:01:40.000 2020-01-02 03:30:01.000 100 99001 49550.5 4955050 100 99001 49550.5 4955050 -32469 32466 4986.02 498602 -127 124 -0.86 -86 +101 2 10091 99002 0.3033 297.3033 148.8033 14880.33033 0.3033 297.3033 148.8033 14880.33035 0.30330 297.30330 148.8033 14880.33000 2020-01-01 2020-01-02 2020-01-01 00:01:41 2020-01-02 03:30:02 2020-01-01 00:01:41.000 2020-01-02 03:30:02.000 101 99002 49551.5 4955150 101 99002 49551.5 4955150 -32468 32467 4987.02 498702 -126 125 0.14 14 +102 2 10092 99003 0.3063 297.3063 148.8063 14880.63063 0.3063 297.3063 148.8063 14880.6305 0.30630 297.30630 148.8063 14880.63000 2020-01-01 2020-01-02 2020-01-01 00:01:42 2020-01-02 03:30:03 2020-01-01 00:01:42.000 2020-01-02 03:30:03.000 102 99003 49552.5 4955250 102 99003 49552.5 4955250 -32467 32468 4988.02 498802 -125 126 1.14 114 +103 2 10093 99004 0.3093 297.3093 148.8093 14880.93093 0.3093 297.3093 148.8093 14880.93085 0.30930 297.30930 148.8093 14880.93000 2020-01-01 2020-01-02 2020-01-01 00:01:43 2020-01-02 03:30:04 2020-01-01 00:01:43.000 2020-01-02 03:30:04.000 103 99004 49553.5 4955350 103 99004 49553.5 4955350 -32466 32469 4989.02 498902 -124 127 2.14 214 +104 2 10094 99005 0.31231 297.31231 148.81231 14881.23123 0.31231 297.31232 148.81231 14881.23144 0.31231 297.31231 148.81231 14881.23100 2020-01-01 2020-01-02 2020-01-01 00:01:44 2020-01-02 03:30:05 2020-01-01 00:01:44.000 2020-01-02 03:30:05.000 104 99005 49554.5 4955450 104 99005 49554.5 4955450 -32465 32470 4990.02 499002 -128 127 0.58 58 +105 2 10095 99006 0.31531 297.31531 148.81531 14881.53153 0.31531 297.3153 148.81531 14881.53174 0.31531 297.31531 148.81531 14881.53100 2020-01-01 2020-01-02 2020-01-01 00:01:45 2020-01-02 03:30:06 2020-01-01 00:01:45.000 2020-01-02 03:30:06.000 105 99006 49555.5 4955550 105 99006 49555.5 4955550 -32464 32471 4991.02 499102 -128 123 -0.98 -98 +106 2 10096 99007 0.31831 297.31831 148.81831 14881.83183 0.31831 297.31833 148.81831 14881.83182 0.31831 297.31831 148.81831 14881.83100 2020-01-01 2020-01-02 2020-01-01 00:01:46 2020-01-02 03:30:07 2020-01-01 00:01:46.000 2020-01-02 03:30:07.000 106 99007 49556.5 4955650 106 99007 49556.5 4955650 -32463 32472 4992.02 499202 -127 124 0.02 2 +107 2 10097 99008 0.32132 297.32132 148.82132 14882.13213 0.32132 297.32132 148.82131 14882.13197 0.32132 297.32132 148.82132 14882.13200 2020-01-01 2020-01-02 2020-01-01 00:01:47 2020-01-02 03:30:08 2020-01-01 00:01:47.000 2020-01-02 03:30:08.000 107 99008 49557.5 4955750 107 99008 49557.5 4955750 -32462 32473 4993.02 499302 -126 125 1.02 102 +108 2 10098 99009 0.32432 297.32432 148.82432 14882.43243 0.32432 297.3243 148.82432 14882.43232 0.32432 297.32432 148.82432 14882.43200 2020-01-01 2020-01-02 2020-01-01 00:01:48 2020-01-02 03:30:09 2020-01-01 00:01:48.000 2020-01-02 03:30:09.000 108 99009 49558.5 4955850 108 99009 49558.5 4955850 -32461 32474 4994.02 499402 -125 126 2.02 202 +109 2 10099 99010 0.32732 297.32732 148.82732 14882.73273 0.32732 297.32733 148.82732 14882.7329 0.32732 297.32732 148.82732 14882.73200 2020-01-01 2020-01-02 2020-01-01 00:01:49 2020-01-02 03:30:10 2020-01-01 00:01:49.000 2020-01-02 03:30:10.000 109 99010 49559.5 4955950 109 99010 49559.5 4955950 -32460 32475 4995.02 499502 -124 127 3.02 302 11 2 10001 99911 0.03303 300.03303 150.03303 15153.33633 0.03303 300.03302 150.03303 15153.33627 0.03303 300.03303 150.03303 15153.33603 2020-01-01 2020-01-02 2020-01-01 00:00:11 2020-01-02 03:45:11 2020-01-01 00:00:11.000 2020-01-02 03:45:11.000 11 99911 49961 5046061 11 99911 49961 5046061 -32558 32377 4540.009900990099 458541 -128 123 -2.089108910891089 -211 -110 2 10100 99011 0.33033 297.33033 148.83033 14883.03303 0.33033 297.33032 148.83033 14883.03321 0.33033 297.33033 148.83033000000006 14883.03300 2020-01-01 2020-01-02 2020-01-01 00:01:50 2020-01-02 03:30:11 2020-01-01 00:01:50.000 2020-01-02 03:30:11.000 110 99011 49560.5 4956050 110 99011 49560.5 4956050 -32459 32476 4996.02 499602 -128 127 1.46 146 -111 2 10101 99012 0.33333 297.33333 148.83333 14883.33333 0.33333 297.33334 148.83333 14883.33329 0.33333 297.33333 148.83332999999996 14883.33300 2020-01-01 2020-01-02 2020-01-01 00:01:51 2020-01-02 03:30:12 2020-01-01 00:01:51.000 2020-01-02 03:30:12.000 111 99012 49561.5 4956150 111 99012 49561.5 4956150 -32458 32477 4997.02 499702 -128 123 -0.1 -10 -112 2 10102 99013 0.33633 297.33633 148.83633 14883.63363 0.33633 297.33633 148.83633 14883.63348 0.33633 297.33633 148.83633000000006 14883.63300 2020-01-01 2020-01-02 2020-01-01 00:01:52 2020-01-02 03:30:13 2020-01-01 00:01:52.000 2020-01-02 03:30:13.000 112 99013 49562.5 4956250 112 99013 49562.5 4956250 -32457 32478 4998.02 499802 -127 124 0.9 90 -113 2 10103 99014 0.33933 297.33933 148.83933 14883.93393 0.33933 297.33932 148.83933 14883.9338 0.33933 297.33933 148.83933000000007 14883.93300 2020-01-01 2020-01-02 2020-01-01 00:01:53 2020-01-02 03:30:14 2020-01-01 00:01:53.000 2020-01-02 03:30:14.000 113 99014 49563.5 4956350 113 99014 49563.5 4956350 -32456 32479 4999.02 499902 -126 125 1.9 190 -114 2 10104 99015 0.34234 297.34234 148.84234 14884.23423 0.34234 297.34235 148.84234 14884.23437 0.34234 297.34234 148.84233999999992 14884.23400 2020-01-01 2020-01-02 2020-01-01 00:01:54 2020-01-02 03:30:15 2020-01-01 00:01:54.000 2020-01-02 03:30:15.000 114 99015 49564.5 4956450 114 99015 49564.5 4956450 -32455 32480 5000.02 500002 -125 126 2.9 290 -115 2 10105 99016 0.34534 297.34534 148.84534 14884.53453 0.34534 297.34534 148.84534 14884.53468 0.34534 297.34534 148.8453399999999 14884.53400 2020-01-01 2020-01-02 2020-01-01 00:01:55 2020-01-02 03:30:16 2020-01-01 00:01:55.000 2020-01-02 03:30:16.000 115 99016 49565.5 4956550 115 99016 49565.5 4956550 -32454 32481 5001.02 500102 -124 127 3.9 390 -116 2 10106 99017 0.34834 297.34834 148.84834 14884.83483 0.34834 297.34836 148.84834 14884.83476 0.34834 297.34834 148.84834000000004 14884.83400 2020-01-01 2020-01-02 2020-01-01 00:01:56 2020-01-02 03:30:17 2020-01-01 00:01:56.000 2020-01-02 03:30:17.000 116 99017 49566.5 4956650 116 99017 49566.5 4956650 -32453 32482 5002.02 500202 -128 127 2.34 234 -117 2 10107 99018 0.35135 297.35135 148.85135 14885.13513 0.35135 297.35135 148.85134 14885.13495 0.35135 297.35135 148.85135000000014 14885.13500 2020-01-01 2020-01-02 2020-01-01 00:01:57 2020-01-02 03:30:18 2020-01-01 00:01:57.000 2020-01-02 03:30:18.000 117 99018 49567.5 4956750 117 99018 49567.5 4956750 -32452 32483 5003.02 500302 -128 123 0.78 78 -118 2 10108 99019 0.35435 297.35435 148.85435 14885.43543 0.35435 297.35434 148.85435 14885.43526 0.35435 297.35435 148.8543499999998 14885.43500 2020-01-01 2020-01-02 2020-01-01 00:01:58 2020-01-02 03:30:19 2020-01-01 00:01:58.000 2020-01-02 03:30:19.000 118 99019 49568.5 4956850 118 99019 49568.5 4956850 -32451 32484 5004.02 500402 -127 124 1.78 178 -119 2 10109 99020 0.35735 297.35735 148.85735 14885.73573 0.35735 297.35736 148.85736 14885.736 0.35735 297.35735 148.85735000000017 14885.73500 2020-01-01 2020-01-02 2020-01-01 00:01:59 2020-01-02 03:30:20 2020-01-01 00:01:59.000 2020-01-02 03:30:20.000 119 99020 49569.5 4956950 119 99020 49569.5 4956950 -32450 32485 5005.02 500502 -126 125 2.78 278 -12 2 10002 99912 0.03603 300.03603 150.03603 15153.63963 0.03603 300.03604 150.03603 15153.6399 0.03603 300.03603 150.03602999999995 15153.63903 2020-01-01 2020-01-02 2020-01-01 00:00:12 2020-01-02 03:45:12 2020-01-01 00:00:12.000 2020-01-02 03:45:12.000 12 99912 49962 5046162 12 99912 49962 5046162 -32557 32378 4541.009900990099 458642 -127 124 -1.0891089108910892 -110 -120 2 10110 99021 0.36036 297.36036 148.86036 14886.03603 0.36036 297.36035 148.86036 14886.03615 0.36036 297.36036 148.86036000000007 14886.03600 2020-01-01 2020-01-02 2020-01-01 00:02:00 2020-01-02 03:30:21 2020-01-01 00:02:00.000 2020-01-02 03:30:21.000 120 99021 49570.5 4957050 120 99021 49570.5 4957050 -32449 32486 5006.02 500602 -125 126 3.78 378 -121 2 10111 99022 0.36336 297.36336 148.86336 14886.33633 0.36336 297.36337 148.86336 14886.33627 0.36336 297.36336 148.86335999999991 14886.33600 2020-01-01 2020-01-02 2020-01-01 00:02:01 2020-01-02 03:30:22 2020-01-01 00:02:01.000 2020-01-02 03:30:22.000 121 99022 49571.5 4957150 121 99022 49571.5 4957150 -32448 32487 5007.02 500702 -124 127 4.78 478 -122 2 10112 99023 0.36636 297.36636 148.86636 14886.63663 0.36636 297.36636 148.86636 14886.63642 0.36636 297.36636 148.86636 14886.63600 2020-01-01 2020-01-02 2020-01-01 00:02:02 2020-01-02 03:30:23 2020-01-01 00:02:02.000 2020-01-02 03:30:23.000 122 99023 49572.5 4957250 122 99023 49572.5 4957250 -32447 32488 5008.02 500802 -128 127 3.22 322 -123 2 10113 99024 0.36936 297.36936 148.86936 14886.93693 0.36936 297.36935 148.86936 14886.93673 0.36936 297.36936 148.86936000000003 14886.93600 2020-01-01 2020-01-02 2020-01-01 00:02:03 2020-01-02 03:30:24 2020-01-01 00:02:03.000 2020-01-02 03:30:24.000 123 99024 49573.5 4957350 123 99024 49573.5 4957350 -32446 32489 5009.02 500902 -128 127 1.66 166 -124 2 10114 99025 0.37237 297.37237 148.87237 14887.23723 0.37237 297.37238 148.87237 14887.23746 0.37237 297.37237 148.8723699999998 14887.23700 2020-01-01 2020-01-02 2020-01-01 00:02:04 2020-01-02 03:30:25 2020-01-01 00:02:04.000 2020-01-02 03:30:25.000 124 99025 49574.5 4957450 124 99025 49574.5 4957450 -32445 32490 5010.02 501002 -128 124 0.1 10 -125 2 10115 99026 0.37537 297.37537 148.87537 14887.53753 0.37537 297.37537 148.87537 14887.53762 0.37537 297.37537 148.87536999999986 14887.53700 2020-01-01 2020-01-02 2020-01-01 00:02:05 2020-01-02 03:30:26 2020-01-01 00:02:05.000 2020-01-02 03:30:26.000 125 99026 49575.5 4957550 125 99026 49575.5 4957550 -32444 32491 5011.02 501102 -127 125 1.1 110 -126 2 10116 99027 0.37837 297.37837 148.87837 14887.83783 0.37837 297.3784 148.87837 14887.83774 0.37837 297.37837 148.87837000000016 14887.83700 2020-01-01 2020-01-02 2020-01-01 00:02:06 2020-01-02 03:30:27 2020-01-01 00:02:06.000 2020-01-02 03:30:27.000 126 99027 49576.5 4957650 126 99027 49576.5 4957650 -32443 32492 5012.02 501202 -126 126 2.1 210 -127 2 10117 99028 0.38138 297.38138 148.88138 14888.13813 0.38138 297.38138 148.88137 14888.13789 0.38138 297.38138 148.8813800000001 14888.13800 2020-01-01 2020-01-02 2020-01-01 00:02:07 2020-01-02 03:30:28 2020-01-01 00:02:07.000 2020-01-02 03:30:28.000 127 99028 49577.5 4957750 127 99028 49577.5 4957750 -32442 32493 5013.02 501302 -125 127 3.1 310 +110 2 10100 99011 0.33033 297.33033 148.83033 14883.03303 0.33033 297.33032 148.83033 14883.03321 0.33033 297.33033 148.83033 14883.03300 2020-01-01 2020-01-02 2020-01-01 00:01:50 2020-01-02 03:30:11 2020-01-01 00:01:50.000 2020-01-02 03:30:11.000 110 99011 49560.5 4956050 110 99011 49560.5 4956050 -32459 32476 4996.02 499602 -128 127 1.46 146 +111 2 10101 99012 0.33333 297.33333 148.83333 14883.33333 0.33333 297.33334 148.83333 14883.33329 0.33333 297.33333 148.83333000000002 14883.33300 2020-01-01 2020-01-02 2020-01-01 00:01:51 2020-01-02 03:30:12 2020-01-01 00:01:51.000 2020-01-02 03:30:12.000 111 99012 49561.5 4956150 111 99012 49561.5 4956150 -32458 32477 4997.02 499702 -128 123 -0.1 -10 +112 2 10102 99013 0.33633 297.33633 148.83633 14883.63363 0.33633 297.33633 148.83633 14883.63348 0.33633 297.33633 148.83633 14883.63300 2020-01-01 2020-01-02 2020-01-01 00:01:52 2020-01-02 03:30:13 2020-01-01 00:01:52.000 2020-01-02 03:30:13.000 112 99013 49562.5 4956250 112 99013 49562.5 4956250 -32457 32478 4998.02 499802 -127 124 0.9 90 +113 2 10103 99014 0.33933 297.33933 148.83933 14883.93393 0.33933 297.33932 148.83933 14883.9338 0.33933 297.33933 148.83933000000002 14883.93300 2020-01-01 2020-01-02 2020-01-01 00:01:53 2020-01-02 03:30:14 2020-01-01 00:01:53.000 2020-01-02 03:30:14.000 113 99014 49563.5 4956350 113 99014 49563.5 4956350 -32456 32479 4999.02 499902 -126 125 1.9 190 +114 2 10104 99015 0.34234 297.34234 148.84234 14884.23423 0.34234 297.34235 148.84234 14884.23437 0.34234 297.34234 148.84234 14884.23400 2020-01-01 2020-01-02 2020-01-01 00:01:54 2020-01-02 03:30:15 2020-01-01 00:01:54.000 2020-01-02 03:30:15.000 114 99015 49564.5 4956450 114 99015 49564.5 4956450 -32455 32480 5000.02 500002 -125 126 2.9 290 +115 2 10105 99016 0.34534 297.34534 148.84534 14884.53453 0.34534 297.34534 148.84534 14884.53468 0.34534 297.34534 148.84534 14884.53400 2020-01-01 2020-01-02 2020-01-01 00:01:55 2020-01-02 03:30:16 2020-01-01 00:01:55.000 2020-01-02 03:30:16.000 115 99016 49565.5 4956550 115 99016 49565.5 4956550 -32454 32481 5001.02 500102 -124 127 3.9 390 +116 2 10106 99017 0.34834 297.34834 148.84834 14884.83483 0.34834 297.34836 148.84834 14884.83476 0.34834 297.34834 148.84834 14884.83400 2020-01-01 2020-01-02 2020-01-01 00:01:56 2020-01-02 03:30:17 2020-01-01 00:01:56.000 2020-01-02 03:30:17.000 116 99017 49566.5 4956650 116 99017 49566.5 4956650 -32453 32482 5002.02 500202 -128 127 2.34 234 +117 2 10107 99018 0.35135 297.35135 148.85135 14885.13513 0.35135 297.35135 148.85134 14885.13495 0.35135 297.35135 148.85135 14885.13500 2020-01-01 2020-01-02 2020-01-01 00:01:57 2020-01-02 03:30:18 2020-01-01 00:01:57.000 2020-01-02 03:30:18.000 117 99018 49567.5 4956750 117 99018 49567.5 4956750 -32452 32483 5003.02 500302 -128 123 0.78 78 +118 2 10108 99019 0.35435 297.35435 148.85435 14885.43543 0.35435 297.35434 148.85435 14885.43526 0.35435 297.35435 148.85434999999998 14885.43500 2020-01-01 2020-01-02 2020-01-01 00:01:58 2020-01-02 03:30:19 2020-01-01 00:01:58.000 2020-01-02 03:30:19.000 118 99019 49568.5 4956850 118 99019 49568.5 4956850 -32451 32484 5004.02 500402 -127 124 1.78 178 +119 2 10109 99020 0.35735 297.35735 148.85735 14885.73573 0.35735 297.35736 148.85736 14885.736 0.35735 297.35735 148.85735 14885.73500 2020-01-01 2020-01-02 2020-01-01 00:01:59 2020-01-02 03:30:20 2020-01-01 00:01:59.000 2020-01-02 03:30:20.000 119 99020 49569.5 4956950 119 99020 49569.5 4956950 -32450 32485 5005.02 500502 -126 125 2.78 278 +12 2 10002 99912 0.03603 300.03603 150.03603 15153.63963 0.03603 300.03604 150.03603 15153.6399 0.03603 300.03603 150.03603 15153.63903 2020-01-01 2020-01-02 2020-01-01 00:00:12 2020-01-02 03:45:12 2020-01-01 00:00:12.000 2020-01-02 03:45:12.000 12 99912 49962 5046162 12 99912 49962 5046162 -32557 32378 4541.009900990099 458642 -127 124 -1.0891089108910892 -110 +120 2 10110 99021 0.36036 297.36036 148.86036 14886.03603 0.36036 297.36035 148.86036 14886.03615 0.36036 297.36036 148.86036000000001 14886.03600 2020-01-01 2020-01-02 2020-01-01 00:02:00 2020-01-02 03:30:21 2020-01-01 00:02:00.000 2020-01-02 03:30:21.000 120 99021 49570.5 4957050 120 99021 49570.5 4957050 -32449 32486 5006.02 500602 -125 126 3.78 378 +121 2 10111 99022 0.36336 297.36336 148.86336 14886.33633 0.36336 297.36337 148.86336 14886.33627 0.36336 297.36336 148.86336 14886.33600 2020-01-01 2020-01-02 2020-01-01 00:02:01 2020-01-02 03:30:22 2020-01-01 00:02:01.000 2020-01-02 03:30:22.000 121 99022 49571.5 4957150 121 99022 49571.5 4957150 -32448 32487 5007.02 500702 -124 127 4.78 478 +122 2 10112 99023 0.36636 297.36636 148.86636 14886.63663 0.36636 297.36636 148.86636 14886.63642 0.36636 297.36636 148.86636000000001 14886.63600 2020-01-01 2020-01-02 2020-01-01 00:02:02 2020-01-02 03:30:23 2020-01-01 00:02:02.000 2020-01-02 03:30:23.000 122 99023 49572.5 4957250 122 99023 49572.5 4957250 -32447 32488 5008.02 500802 -128 127 3.22 322 +123 2 10113 99024 0.36936 297.36936 148.86936 14886.93693 0.36936 297.36935 148.86936 14886.93673 0.36936 297.36936 148.86936 14886.93600 2020-01-01 2020-01-02 2020-01-01 00:02:03 2020-01-02 03:30:24 2020-01-01 00:02:03.000 2020-01-02 03:30:24.000 123 99024 49573.5 4957350 123 99024 49573.5 4957350 -32446 32489 5009.02 500902 -128 127 1.66 166 +124 2 10114 99025 0.37237 297.37237 148.87237 14887.23723 0.37237 297.37238 148.87237 14887.23746 0.37237 297.37237 148.87237 14887.23700 2020-01-01 2020-01-02 2020-01-01 00:02:04 2020-01-02 03:30:25 2020-01-01 00:02:04.000 2020-01-02 03:30:25.000 124 99025 49574.5 4957450 124 99025 49574.5 4957450 -32445 32490 5010.02 501002 -128 124 0.1 10 +125 2 10115 99026 0.37537 297.37537 148.87537 14887.53753 0.37537 297.37537 148.87537 14887.53762 0.37537 297.37537 148.87537 14887.53700 2020-01-01 2020-01-02 2020-01-01 00:02:05 2020-01-02 03:30:26 2020-01-01 00:02:05.000 2020-01-02 03:30:26.000 125 99026 49575.5 4957550 125 99026 49575.5 4957550 -32444 32491 5011.02 501102 -127 125 1.1 110 +126 2 10116 99027 0.37837 297.37837 148.87837 14887.83783 0.37837 297.3784 148.87837 14887.83774 0.37837 297.37837 148.87837 14887.83700 2020-01-01 2020-01-02 2020-01-01 00:02:06 2020-01-02 03:30:27 2020-01-01 00:02:06.000 2020-01-02 03:30:27.000 126 99027 49576.5 4957650 126 99027 49576.5 4957650 -32443 32492 5012.02 501202 -126 126 2.1 210 +127 2 10117 99028 0.38138 297.38138 148.88138 14888.13813 0.38138 297.38138 148.88137 14888.13789 0.38138 297.38138 148.88138 14888.13800 2020-01-01 2020-01-02 2020-01-01 00:02:07 2020-01-02 03:30:28 2020-01-01 00:02:07.000 2020-01-02 03:30:28.000 127 99028 49577.5 4957750 127 99028 49577.5 4957750 -32442 32493 5013.02 501302 -125 127 3.1 310 128 2 10118 99029 0.38438 297.38438 148.88438 14888.43843 0.38438 297.3844 148.88438 14888.43862 0.38438 297.38438 148.88438 14888.43800 2020-01-01 2020-01-02 2020-01-01 00:02:08 2020-01-02 03:30:29 2020-01-01 00:02:08.000 2020-01-02 03:30:29.000 128 99029 49578.5 4957850 128 99029 49578.5 4957850 -32441 32494 5014.02 501402 -128 127 1.54 154 -129 2 10119 99030 0.38738 297.38738 148.88738 14888.73873 0.38738 297.3874 148.88738 14888.73894 0.38738 297.38738 148.8873800000001 14888.73800 2020-01-01 2020-01-02 2020-01-01 00:02:09 2020-01-02 03:30:30 2020-01-01 00:02:09.000 2020-01-02 03:30:30.000 129 99030 49579.5 4957950 129 99030 49579.5 4957950 -32440 32495 5015.02 501502 -128 127 -0.02 -2 -13 2 10003 99913 0.03903 300.03903 150.03903 15153.94294 0.03903 300.03903 150.03903 15153.94255 0.03903 300.03903 150.03902999999994 15153.94203 2020-01-01 2020-01-02 2020-01-01 00:00:13 2020-01-02 03:45:13 2020-01-01 00:00:13.000 2020-01-02 03:45:13.000 13 99913 49963 5046263 13 99913 49963 5046263 -32556 32379 4542.009900990099 458743 -126 125 -0.0891089108910891 -9 -130 2 10120 99031 0.39039 297.39039 148.89039 14889.03903 0.39039 297.39038 148.89039 14889.03909 0.39039 297.39039 148.89038999999994 14889.03900 2020-01-01 2020-01-02 2020-01-01 00:02:10 2020-01-02 03:30:31 2020-01-01 00:02:10.000 2020-01-02 03:30:31.000 130 99031 49580.5 4958050 130 99031 49580.5 4958050 -32439 32496 5016.02 501602 -128 123 -1.58 -158 -131 2 10121 99032 0.39339 297.39339 148.89339 14889.33933 0.39339 297.3934 148.89339 14889.33921 0.39339 297.39339 148.89338999999995 14889.33900 2020-01-01 2020-01-02 2020-01-01 00:02:11 2020-01-02 03:30:32 2020-01-01 00:02:11.000 2020-01-02 03:30:32.000 131 99032 49581.5 4958150 131 99032 49581.5 4958150 -32438 32497 5017.02 501702 -127 124 -0.58 -58 -132 2 10122 99033 0.39639 297.39639 148.89639 14889.63963 0.39639 297.3964 148.89639 14889.63936 0.39639 297.39639 148.8963899999999 14889.63900 2020-01-01 2020-01-02 2020-01-01 00:02:12 2020-01-02 03:30:33 2020-01-01 00:02:12.000 2020-01-02 03:30:33.000 132 99033 49582.5 4958250 132 99033 49582.5 4958250 -32437 32498 5018.02 501802 -126 125 0.42 42 -133 2 10123 99034 0.39939 297.39939 148.89939 14889.93993 0.39939 297.3994 148.8994 14889.94009 0.39939 297.39939 148.89938999999998 14889.93900 2020-01-01 2020-01-02 2020-01-01 00:02:13 2020-01-02 03:30:34 2020-01-01 00:02:13.000 2020-01-02 03:30:34.000 133 99034 49583.5 4958350 133 99034 49583.5 4958350 -32436 32499 5019.02 501902 -125 126 1.42 142 -134 2 10124 99035 0.4024 297.4024 148.9024 14890.24024 0.4024 297.4024 148.9024 14890.24041 0.40240 297.40240 148.90240000000017 14890.24000 2020-01-01 2020-01-02 2020-01-01 00:02:14 2020-01-02 03:30:35 2020-01-01 00:02:14.000 2020-01-02 03:30:35.000 134 99035 49584.5 4958450 134 99035 49584.5 4958450 -32435 32500 5020.02 502002 -124 127 2.42 242 -135 2 10125 99036 0.4054 297.4054 148.9054 14890.54054 0.4054 297.4054 148.9054 14890.54059 0.40540 297.40540 148.90539999999984 14890.54000 2020-01-01 2020-01-02 2020-01-01 00:02:15 2020-01-02 03:30:36 2020-01-01 00:02:15.000 2020-01-02 03:30:36.000 135 99036 49585.5 4958550 135 99036 49585.5 4958550 -32434 32501 5021.02 502102 -128 127 0.86 86 -136 2 10126 99037 0.4084 297.4084 148.9084 14890.84084 0.4084 297.40842 148.9084 14890.84068 0.40840 297.40840 148.9084000000001 14890.84000 2020-01-01 2020-01-02 2020-01-01 00:02:16 2020-01-02 03:30:37 2020-01-01 00:02:16.000 2020-01-02 03:30:37.000 136 99037 49586.5 4958650 136 99037 49586.5 4958650 -32433 32502 5022.02 502202 -128 123 -0.7 -70 -137 2 10127 99038 0.41141 297.41141 148.91141 14891.14114 0.41141 297.4114 148.9114 14891.14099 0.41141 297.41141 148.91141000000007 14891.14100 2020-01-01 2020-01-02 2020-01-01 00:02:17 2020-01-02 03:30:38 2020-01-01 00:02:17.000 2020-01-02 03:30:38.000 137 99038 49587.5 4958750 137 99038 49587.5 4958750 -32432 32503 5023.02 502302 -127 124 0.3 30 -138 2 10128 99039 0.41441 297.41441 148.91441 14891.44144 0.41441 297.41443 148.91441 14891.44157 0.41441 297.41441 148.91440999999995 14891.44100 2020-01-01 2020-01-02 2020-01-01 00:02:18 2020-01-02 03:30:39 2020-01-01 00:02:18.000 2020-01-02 03:30:39.000 138 99039 49588.5 4958850 138 99039 49588.5 4958850 -32431 32504 5024.02 502402 -126 125 1.3 130 +129 2 10119 99030 0.38738 297.38738 148.88738 14888.73873 0.38738 297.3874 148.88738 14888.73894 0.38738 297.38738 148.88738 14888.73800 2020-01-01 2020-01-02 2020-01-01 00:02:09 2020-01-02 03:30:30 2020-01-01 00:02:09.000 2020-01-02 03:30:30.000 129 99030 49579.5 4957950 129 99030 49579.5 4957950 -32440 32495 5015.02 501502 -128 127 -0.02 -2 +13 2 10003 99913 0.03903 300.03903 150.03903 15153.94294 0.03903 300.03903 150.03903 15153.94255 0.03903 300.03903 150.03903 15153.94203 2020-01-01 2020-01-02 2020-01-01 00:00:13 2020-01-02 03:45:13 2020-01-01 00:00:13.000 2020-01-02 03:45:13.000 13 99913 49963 5046263 13 99913 49963 5046263 -32556 32379 4542.009900990099 458743 -126 125 -0.0891089108910891 -9 +130 2 10120 99031 0.39039 297.39039 148.89039 14889.03903 0.39039 297.39038 148.89039 14889.03909 0.39039 297.39039 148.89039 14889.03900 2020-01-01 2020-01-02 2020-01-01 00:02:10 2020-01-02 03:30:31 2020-01-01 00:02:10.000 2020-01-02 03:30:31.000 130 99031 49580.5 4958050 130 99031 49580.5 4958050 -32439 32496 5016.02 501602 -128 123 -1.58 -158 +131 2 10121 99032 0.39339 297.39339 148.89339 14889.33933 0.39339 297.3934 148.89339 14889.33921 0.39339 297.39339 148.89339 14889.33900 2020-01-01 2020-01-02 2020-01-01 00:02:11 2020-01-02 03:30:32 2020-01-01 00:02:11.000 2020-01-02 03:30:32.000 131 99032 49581.5 4958150 131 99032 49581.5 4958150 -32438 32497 5017.02 501702 -127 124 -0.58 -58 +132 2 10122 99033 0.39639 297.39639 148.89639 14889.63963 0.39639 297.3964 148.89639 14889.63936 0.39639 297.39639 148.89639 14889.63900 2020-01-01 2020-01-02 2020-01-01 00:02:12 2020-01-02 03:30:33 2020-01-01 00:02:12.000 2020-01-02 03:30:33.000 132 99033 49582.5 4958250 132 99033 49582.5 4958250 -32437 32498 5018.02 501802 -126 125 0.42 42 +133 2 10123 99034 0.39939 297.39939 148.89939 14889.93993 0.39939 297.3994 148.8994 14889.94009 0.39939 297.39939 148.89939 14889.93900 2020-01-01 2020-01-02 2020-01-01 00:02:13 2020-01-02 03:30:34 2020-01-01 00:02:13.000 2020-01-02 03:30:34.000 133 99034 49583.5 4958350 133 99034 49583.5 4958350 -32436 32499 5019.02 501902 -125 126 1.42 142 +134 2 10124 99035 0.4024 297.4024 148.9024 14890.24024 0.4024 297.4024 148.9024 14890.24041 0.40240 297.40240 148.9024 14890.24000 2020-01-01 2020-01-02 2020-01-01 00:02:14 2020-01-02 03:30:35 2020-01-01 00:02:14.000 2020-01-02 03:30:35.000 134 99035 49584.5 4958450 134 99035 49584.5 4958450 -32435 32500 5020.02 502002 -124 127 2.42 242 +135 2 10125 99036 0.4054 297.4054 148.9054 14890.54054 0.4054 297.4054 148.9054 14890.54059 0.40540 297.40540 148.90540000000001 14890.54000 2020-01-01 2020-01-02 2020-01-01 00:02:15 2020-01-02 03:30:36 2020-01-01 00:02:15.000 2020-01-02 03:30:36.000 135 99036 49585.5 4958550 135 99036 49585.5 4958550 -32434 32501 5021.02 502102 -128 127 0.86 86 +136 2 10126 99037 0.4084 297.4084 148.9084 14890.84084 0.4084 297.40842 148.9084 14890.84068 0.40840 297.40840 148.9084 14890.84000 2020-01-01 2020-01-02 2020-01-01 00:02:16 2020-01-02 03:30:37 2020-01-01 00:02:16.000 2020-01-02 03:30:37.000 136 99037 49586.5 4958650 136 99037 49586.5 4958650 -32433 32502 5022.02 502202 -128 123 -0.7 -70 +137 2 10127 99038 0.41141 297.41141 148.91141 14891.14114 0.41141 297.4114 148.9114 14891.14099 0.41141 297.41141 148.91141 14891.14100 2020-01-01 2020-01-02 2020-01-01 00:02:17 2020-01-02 03:30:38 2020-01-01 00:02:17.000 2020-01-02 03:30:38.000 137 99038 49587.5 4958750 137 99038 49587.5 4958750 -32432 32503 5023.02 502302 -127 124 0.3 30 +138 2 10128 99039 0.41441 297.41441 148.91441 14891.44144 0.41441 297.41443 148.91441 14891.44157 0.41441 297.41441 148.91441 14891.44100 2020-01-01 2020-01-02 2020-01-01 00:02:18 2020-01-02 03:30:39 2020-01-01 00:02:18.000 2020-01-02 03:30:39.000 138 99039 49588.5 4958850 138 99039 49588.5 4958850 -32431 32504 5024.02 502402 -126 125 1.3 130 139 2 10129 99040 0.41741 297.41741 148.91741 14891.74174 0.41741 297.41742 148.91741 14891.74188 0.41741 297.41741 148.91741 14891.74100 2020-01-01 2020-01-02 2020-01-01 00:02:19 2020-01-02 03:30:40 2020-01-01 00:02:19.000 2020-01-02 03:30:40.000 139 99040 49589.5 4958950 139 99040 49589.5 4958950 -32430 32505 5025.02 502502 -125 126 2.3 230 -14 2 10004 99914 0.04204 300.04204 150.04204 15154.24624 0.04204 300.04205 150.04204 15154.2463 0.04204 300.04204 150.0420400000001 15154.24604 2020-01-01 2020-01-02 2020-01-01 00:00:14 2020-01-02 03:45:14 2020-01-01 00:00:14.000 2020-01-02 03:45:14.000 14 99914 49964 5046364 14 99914 49964 5046364 -32555 32380 4543.009900990099 458844 -125 126 0.9108910891089109 92 -140 2 10130 99041 0.42042 297.42042 148.92042 14892.04204 0.42042 297.4204 148.92042 14892.04206 0.42042 297.42042 148.9204200000002 14892.04200 2020-01-01 2020-01-02 2020-01-01 00:02:20 2020-01-02 03:30:41 2020-01-01 00:02:20.000 2020-01-02 03:30:41.000 140 99041 49590.5 4959050 140 99041 49590.5 4959050 -32429 32506 5026.02 502602 -124 127 3.3 330 -141 2 10131 99042 0.42342 297.42342 148.92342 14892.34234 0.42342 297.42343 148.92342 14892.34215 0.42342 297.42342 148.92341999999988 14892.34200 2020-01-01 2020-01-02 2020-01-01 00:02:21 2020-01-02 03:30:42 2020-01-01 00:02:21.000 2020-01-02 03:30:42.000 141 99042 49591.5 4959150 141 99042 49591.5 4959150 -32428 32507 5027.02 502702 -128 127 1.74 174 -142 2 10132 99043 0.42642 297.42642 148.92642 14892.64264 0.42642 297.42642 148.92642 14892.64246 0.42642 297.42642 148.92641999999984 14892.64200 2020-01-01 2020-01-02 2020-01-01 00:02:22 2020-01-02 03:30:43 2020-01-01 00:02:22.000 2020-01-02 03:30:43.000 142 99043 49592.5 4959250 142 99043 49592.5 4959250 -32427 32508 5028.02 502802 -128 123 0.18 18 -143 2 10133 99044 0.42942 297.42942 148.92942 14892.94294 0.42942 297.42944 148.92943 14892.94304 0.42942 297.42942 148.92942000000016 14892.94200 2020-01-01 2020-01-02 2020-01-01 00:02:23 2020-01-02 03:30:44 2020-01-01 00:02:23.000 2020-01-02 03:30:44.000 143 99044 49593.5 4959350 143 99044 49593.5 4959350 -32426 32509 5029.02 502902 -127 124 1.18 118 -144 2 10134 99045 0.43243 297.43243 148.93243 14893.24324 0.43243 297.43243 148.93243 14893.24338 0.43243 297.43243 148.93243000000012 14893.24300 2020-01-01 2020-01-02 2020-01-01 00:02:24 2020-01-02 03:30:45 2020-01-01 00:02:24.000 2020-01-02 03:30:45.000 144 99045 49594.5 4959450 144 99045 49594.5 4959450 -32425 32510 5030.02 503002 -126 125 2.18 218 -145 2 10135 99046 0.43543 297.43543 148.93543 14893.54354 0.43543 297.43542 148.93543 14893.54354 0.43543 297.43543 148.93542999999983 14893.54300 2020-01-01 2020-01-02 2020-01-01 00:02:25 2020-01-02 03:30:46 2020-01-01 00:02:25.000 2020-01-02 03:30:46.000 145 99046 49595.5 4959550 145 99046 49595.5 4959550 -32424 32511 5031.02 503102 -125 126 3.18 318 -146 2 10136 99047 0.43843 297.43843 148.93843 14893.84384 0.43843 297.43845 148.93844 14893.84427 0.43843 297.43843 148.9384300000001 14893.84300 2020-01-01 2020-01-02 2020-01-01 00:02:26 2020-01-02 03:30:47 2020-01-01 00:02:26.000 2020-01-02 03:30:47.000 146 99047 49596.5 4959650 146 99047 49596.5 4959650 -32423 32512 5032.02 503202 -124 127 4.18 418 +14 2 10004 99914 0.04204 300.04204 150.04204 15154.24624 0.04204 300.04205 150.04204 15154.2463 0.04204 300.04204 150.04204 15154.24604 2020-01-01 2020-01-02 2020-01-01 00:00:14 2020-01-02 03:45:14 2020-01-01 00:00:14.000 2020-01-02 03:45:14.000 14 99914 49964 5046364 14 99914 49964 5046364 -32555 32380 4543.009900990099 458844 -125 126 0.9108910891089109 92 +140 2 10130 99041 0.42042 297.42042 148.92042 14892.04204 0.42042 297.4204 148.92042 14892.04206 0.42042 297.42042 148.92042 14892.04200 2020-01-01 2020-01-02 2020-01-01 00:02:20 2020-01-02 03:30:41 2020-01-01 00:02:20.000 2020-01-02 03:30:41.000 140 99041 49590.5 4959050 140 99041 49590.5 4959050 -32429 32506 5026.02 502602 -124 127 3.3 330 +141 2 10131 99042 0.42342 297.42342 148.92342 14892.34234 0.42342 297.42343 148.92342 14892.34215 0.42342 297.42342 148.92342 14892.34200 2020-01-01 2020-01-02 2020-01-01 00:02:21 2020-01-02 03:30:42 2020-01-01 00:02:21.000 2020-01-02 03:30:42.000 141 99042 49591.5 4959150 141 99042 49591.5 4959150 -32428 32507 5027.02 502702 -128 127 1.74 174 +142 2 10132 99043 0.42642 297.42642 148.92642 14892.64264 0.42642 297.42642 148.92642 14892.64246 0.42642 297.42642 148.92642 14892.64200 2020-01-01 2020-01-02 2020-01-01 00:02:22 2020-01-02 03:30:43 2020-01-01 00:02:22.000 2020-01-02 03:30:43.000 142 99043 49592.5 4959250 142 99043 49592.5 4959250 -32427 32508 5028.02 502802 -128 123 0.18 18 +143 2 10133 99044 0.42942 297.42942 148.92942 14892.94294 0.42942 297.42944 148.92943 14892.94304 0.42942 297.42942 148.92942 14892.94200 2020-01-01 2020-01-02 2020-01-01 00:02:23 2020-01-02 03:30:44 2020-01-01 00:02:23.000 2020-01-02 03:30:44.000 143 99044 49593.5 4959350 143 99044 49593.5 4959350 -32426 32509 5029.02 502902 -127 124 1.18 118 +144 2 10134 99045 0.43243 297.43243 148.93243 14893.24324 0.43243 297.43243 148.93243 14893.24338 0.43243 297.43243 148.93243 14893.24300 2020-01-01 2020-01-02 2020-01-01 00:02:24 2020-01-02 03:30:45 2020-01-01 00:02:24.000 2020-01-02 03:30:45.000 144 99045 49594.5 4959450 144 99045 49594.5 4959450 -32425 32510 5030.02 503002 -126 125 2.18 218 +145 2 10135 99046 0.43543 297.43543 148.93543 14893.54354 0.43543 297.43542 148.93543 14893.54354 0.43543 297.43543 148.93543 14893.54300 2020-01-01 2020-01-02 2020-01-01 00:02:25 2020-01-02 03:30:46 2020-01-01 00:02:25.000 2020-01-02 03:30:46.000 145 99046 49595.5 4959550 145 99046 49595.5 4959550 -32424 32511 5031.02 503102 -125 126 3.18 318 +146 2 10136 99047 0.43843 297.43843 148.93843 14893.84384 0.43843 297.43845 148.93844 14893.84427 0.43843 297.43843 148.93843 14893.84300 2020-01-01 2020-01-02 2020-01-01 00:02:26 2020-01-02 03:30:47 2020-01-01 00:02:26.000 2020-01-02 03:30:47.000 146 99047 49596.5 4959650 146 99047 49596.5 4959650 -32423 32512 5032.02 503202 -124 127 4.18 418 147 2 10137 99048 0.44144 297.44144 148.94144 14894.14414 0.44144 297.44144 148.94143 14894.14392 0.44144 297.44144 148.94144 14894.14400 2020-01-01 2020-01-02 2020-01-01 00:02:27 2020-01-02 03:30:48 2020-01-01 00:02:27.000 2020-01-02 03:30:48.000 147 99048 49597.5 4959750 147 99048 49597.5 4959750 -32422 32513 5033.02 503302 -128 127 2.62 262 -148 2 10138 99049 0.44444 297.44444 148.94444 14894.44444 0.44444 297.44446 148.94444 14894.4445 0.44444 297.44444 148.94443999999996 14894.44400 2020-01-01 2020-01-02 2020-01-01 00:02:28 2020-01-02 03:30:49 2020-01-01 00:02:28.000 2020-01-02 03:30:49.000 148 99049 49598.5 4959850 148 99049 49598.5 4959850 -32421 32514 5034.02 503402 -128 127 1.06 106 -149 2 10139 99050 0.44744 297.44744 148.94744 14894.74474 0.44744 297.44745 148.94744 14894.74485 0.44744 297.44744 148.94743999999994 14894.74400 2020-01-01 2020-01-02 2020-01-01 00:02:29 2020-01-02 03:30:50 2020-01-01 00:02:29.000 2020-01-02 03:30:50.000 149 99050 49599.5 4959950 149 99050 49599.5 4959950 -32420 32515 5035.02 503502 -128 124 -0.5 -50 -15 2 10005 99915 0.04504 300.04504 150.04504 15154.54954 0.04504 300.04504 150.04504 15154.54945 0.04504 300.04504 150.04504000000017 15154.54904 2020-01-01 2020-01-02 2020-01-01 00:00:15 2020-01-02 03:45:15 2020-01-01 00:00:15.000 2020-01-02 03:45:15.000 15 99915 49965 5046465 15 99915 49965 5046465 -32554 32381 4544.009900990099 458945 -124 127 1.9108910891089108 193 -150 2 10140 99051 0.45045 297.45045 148.95045 14895.04504 0.45045 297.45044 148.95045 14895.04501 0.45045 297.45045 148.95045000000013 14895.04500 2020-01-01 2020-01-02 2020-01-01 00:02:30 2020-01-02 03:30:51 2020-01-01 00:02:30.000 2020-01-02 03:30:51.000 150 99051 49600.5 4960050 150 99051 49600.5 4960050 -32419 32516 5036.02 503602 -127 125 0.5 50 -151 2 10141 99052 0.45345 297.45345 148.95345 14895.34534 0.45345 297.45346 148.95345 14895.34574 0.45345 297.45345 148.9534500000002 14895.34500 2020-01-01 2020-01-02 2020-01-01 00:02:31 2020-01-02 03:30:52 2020-01-01 00:02:31.000 2020-01-02 03:30:52.000 151 99052 49601.5 4960150 151 99052 49601.5 4960150 -32418 32517 5037.02 503702 -126 126 1.5 150 -152 2 10142 99053 0.45645 297.45645 148.95645 14895.64564 0.45645 297.45645 148.95645 14895.6454 0.45645 297.45645 148.95644999999985 14895.64500 2020-01-01 2020-01-02 2020-01-01 00:02:32 2020-01-02 03:30:53 2020-01-01 00:02:32.000 2020-01-02 03:30:53.000 152 99053 49602.5 4960250 152 99053 49602.5 4960250 -32417 32518 5038.02 503802 -125 127 2.5 250 -153 2 10143 99054 0.45945 297.45945 148.95945 14895.94594 0.45945 297.45947 148.95946 14895.94601 0.45945 297.45945 148.95945000000015 14895.94500 2020-01-01 2020-01-02 2020-01-01 00:02:33 2020-01-02 03:30:54 2020-01-01 00:02:33.000 2020-01-02 03:30:54.000 153 99054 49603.5 4960350 153 99054 49603.5 4960350 -32416 32519 5039.02 503902 -128 127 0.94 94 -154 2 10144 99055 0.46246 297.46246 148.96246 14896.24624 0.46246 297.46246 148.96246 14896.24633 0.46246 297.46246 148.9624600000001 14896.24600 2020-01-01 2020-01-02 2020-01-01 00:02:34 2020-01-02 03:30:55 2020-01-01 00:02:34.000 2020-01-02 03:30:55.000 154 99055 49604.5 4960450 154 99055 49604.5 4960450 -32415 32520 5040.02 504002 -128 127 -0.62 -62 -155 2 10145 99056 0.46546 297.46546 148.96546 14896.54654 0.46546 297.46545 148.96546 14896.54647 0.46546 297.46546 148.96545999999995 14896.54600 2020-01-01 2020-01-02 2020-01-01 00:02:35 2020-01-02 03:30:56 2020-01-01 00:02:35.000 2020-01-02 03:30:56.000 155 99056 49605.5 4960550 155 99056 49605.5 4960550 -32414 32521 5041.02 504102 -128 123 -2.18 -218 -156 2 10146 99057 0.46846 297.46846 148.96846 14896.84684 0.46846 297.46848 148.96847 14896.84721 0.46846 297.46846 148.96846000000002 14896.84600 2020-01-01 2020-01-02 2020-01-01 00:02:36 2020-01-02 03:30:57 2020-01-01 00:02:36.000 2020-01-02 03:30:57.000 156 99057 49606.5 4960650 156 99057 49606.5 4960650 -32413 32522 5042.02 504202 -127 124 -1.18 -118 -157 2 10147 99058 0.47147 297.47147 148.97147 14897.14714 0.47147 297.47147 148.97146 14897.14687 0.47147 297.47147 148.97147000000024 14897.14700 2020-01-01 2020-01-02 2020-01-01 00:02:37 2020-01-02 03:30:58 2020-01-01 00:02:37.000 2020-01-02 03:30:58.000 157 99058 49607.5 4960750 157 99058 49607.5 4960750 -32412 32523 5043.02 504302 -126 125 -0.18 -18 -158 2 10148 99059 0.47447 297.47447 148.97447 14897.44744 0.47447 297.4745 148.97447 14897.44748 0.47447 297.47447 148.97446999999985 14897.44700 2020-01-01 2020-01-02 2020-01-01 00:02:38 2020-01-02 03:30:59 2020-01-01 00:02:38.000 2020-01-02 03:30:59.000 158 99059 49608.5 4960850 158 99059 49608.5 4960850 -32411 32524 5044.02 504402 -125 126 0.82 82 -159 2 10149 99060 0.47747 297.47747 148.97747 14897.74774 0.47747 297.47748 148.97747 14897.74779 0.47747 297.47747 148.97746999999987 14897.74700 2020-01-01 2020-01-02 2020-01-01 00:02:39 2020-01-02 03:31:00 2020-01-01 00:02:39.000 2020-01-02 03:31:00.000 159 99060 49609.5 4960950 159 99060 49609.5 4960950 -32410 32525 5045.02 504502 -124 127 1.82 182 -16 2 10006 99916 0.04804 300.04804 150.04804 15154.85285 0.04804 300.04803 150.04804 15154.85279 0.04804 300.04804 150.04803999999982 15154.85204 2020-01-01 2020-01-02 2020-01-01 00:00:16 2020-01-02 03:45:16 2020-01-01 00:00:16.000 2020-01-02 03:45:16.000 16 99916 49966 5046566 16 99916 49966 5046566 -32553 32382 4545.009900990099 459046 -128 127 0.37623762376237624 38 -160 2 10150 99061 0.48048 297.48048 148.98048 14898.04804 0.48048 297.48047 148.98048 14898.0481 0.48048 297.48048 148.9804800000001 14898.04800 2020-01-01 2020-01-02 2020-01-01 00:02:40 2020-01-02 03:31:01 2020-01-01 00:02:40.000 2020-01-02 03:31:01.000 160 99061 49610.5 4961050 160 99061 49610.5 4961050 -32409 32526 5046.02 504602 -128 127 0.26 26 -161 2 10151 99062 0.48348 297.48348 148.98348 14898.34834 0.48348 297.4835 148.98348 14898.34868 0.48348 297.48348 148.98348000000016 14898.34800 2020-01-01 2020-01-02 2020-01-01 00:02:41 2020-01-02 03:31:02 2020-01-01 00:02:41.000 2020-01-02 03:31:02.000 161 99062 49611.5 4961150 161 99062 49611.5 4961150 -32408 32527 5047.02 504702 -128 123 -1.3 -130 -162 2 10152 99063 0.48648 297.48648 148.98648 14898.64864 0.48648 297.48648 148.98648 14898.64837 0.48648 297.48648 148.98647999999974 14898.64800 2020-01-01 2020-01-02 2020-01-01 00:02:42 2020-01-02 03:31:03 2020-01-01 00:02:42.000 2020-01-02 03:31:03.000 162 99063 49612.5 4961250 162 99063 49612.5 4961250 -32407 32528 5048.02 504802 -127 124 -0.3 -30 -163 2 10153 99064 0.48948 297.48948 148.98948 14898.94894 0.48948 297.4895 148.98948 14898.94895 0.48948 297.48948 148.98948000000013 14898.94800 2020-01-01 2020-01-02 2020-01-01 00:02:43 2020-01-02 03:31:04 2020-01-01 00:02:43.000 2020-01-02 03:31:04.000 163 99064 49613.5 4961350 163 99064 49613.5 4961350 -32406 32529 5049.02 504902 -126 125 0.7 70 -164 2 10154 99065 0.49249 297.49249 148.99249 14899.24924 0.49249 297.4925 148.99249 14899.24926 0.49249 297.49249 148.99249000000006 14899.24900 2020-01-01 2020-01-02 2020-01-01 00:02:44 2020-01-02 03:31:05 2020-01-01 00:02:44.000 2020-01-02 03:31:05.000 164 99065 49614.5 4961450 164 99065 49614.5 4961450 -32405 32530 5050.02 505002 -125 126 1.7 170 -165 2 10155 99066 0.49549 297.49549 148.99549 14899.54954 0.49549 297.49548 148.99549 14899.54957 0.49549 297.49549 148.99548999999988 14899.54900 2020-01-01 2020-01-02 2020-01-01 00:02:45 2020-01-02 03:31:06 2020-01-01 00:02:45.000 2020-01-02 03:31:06.000 165 99066 49615.5 4961550 165 99066 49615.5 4961550 -32404 32531 5051.02 505102 -124 127 2.7 270 -166 2 10156 99067 0.49849 297.49849 148.99849 14899.84984 0.49849 297.4985 148.9985 14899.85015 0.49849 297.49849 148.99848999999998 14899.84900 2020-01-01 2020-01-02 2020-01-01 00:02:46 2020-01-02 03:31:07 2020-01-01 00:02:46.000 2020-01-02 03:31:07.000 166 99067 49616.5 4961650 166 99067 49616.5 4961650 -32403 32532 5052.02 505202 -128 127 1.14 114 -167 2 10157 99068 0.5015 297.5015 149.0015 14900.15015 0.5015 297.5015 149.00149 14900.14984 0.50150 297.50150 149.00150000000016 14900.15000 2020-01-01 2020-01-02 2020-01-01 00:02:47 2020-01-02 03:31:08 2020-01-01 00:02:47.000 2020-01-02 03:31:08.000 167 99068 49617.5 4961750 167 99068 49617.5 4961750 -32402 32533 5053.02 505302 -128 123 -0.42 -42 -168 2 10158 99069 0.5045 297.5045 149.0045 14900.45045 0.5045 297.50452 149.0045 14900.45042 0.50450 297.50450 149.00449999999978 14900.45000 2020-01-01 2020-01-02 2020-01-01 00:02:48 2020-01-02 03:31:09 2020-01-01 00:02:48.000 2020-01-02 03:31:09.000 168 99069 49618.5 4961850 168 99069 49618.5 4961850 -32401 32534 5054.02 505402 -127 124 0.58 58 -169 2 10159 99070 0.5075 297.5075 149.0075 14900.75075 0.5075 297.5075 149.0075 14900.75073 0.50750 297.50750 149.00749999999985 14900.75000 2020-01-01 2020-01-02 2020-01-01 00:02:49 2020-01-02 03:31:10 2020-01-01 00:02:49.000 2020-01-02 03:31:10.000 169 99070 49619.5 4961950 169 99070 49619.5 4961950 -32400 32535 5055.02 505502 -126 125 1.58 158 +148 2 10138 99049 0.44444 297.44444 148.94444 14894.44444 0.44444 297.44446 148.94444 14894.4445 0.44444 297.44444 148.94444 14894.44400 2020-01-01 2020-01-02 2020-01-01 00:02:28 2020-01-02 03:30:49 2020-01-01 00:02:28.000 2020-01-02 03:30:49.000 148 99049 49598.5 4959850 148 99049 49598.5 4959850 -32421 32514 5034.02 503402 -128 127 1.06 106 +149 2 10139 99050 0.44744 297.44744 148.94744 14894.74474 0.44744 297.44745 148.94744 14894.74485 0.44744 297.44744 148.94744 14894.74400 2020-01-01 2020-01-02 2020-01-01 00:02:29 2020-01-02 03:30:50 2020-01-01 00:02:29.000 2020-01-02 03:30:50.000 149 99050 49599.5 4959950 149 99050 49599.5 4959950 -32420 32515 5035.02 503502 -128 124 -0.5 -50 +15 2 10005 99915 0.04504 300.04504 150.04504 15154.54954 0.04504 300.04504 150.04504 15154.54945 0.04504 300.04504 150.04504 15154.54904 2020-01-01 2020-01-02 2020-01-01 00:00:15 2020-01-02 03:45:15 2020-01-01 00:00:15.000 2020-01-02 03:45:15.000 15 99915 49965 5046465 15 99915 49965 5046465 -32554 32381 4544.009900990099 458945 -124 127 1.9108910891089108 193 +150 2 10140 99051 0.45045 297.45045 148.95045 14895.04504 0.45045 297.45044 148.95045 14895.04501 0.45045 297.45045 148.95045 14895.04500 2020-01-01 2020-01-02 2020-01-01 00:02:30 2020-01-02 03:30:51 2020-01-01 00:02:30.000 2020-01-02 03:30:51.000 150 99051 49600.5 4960050 150 99051 49600.5 4960050 -32419 32516 5036.02 503602 -127 125 0.5 50 +151 2 10141 99052 0.45345 297.45345 148.95345 14895.34534 0.45345 297.45346 148.95345 14895.34574 0.45345 297.45345 148.95345 14895.34500 2020-01-01 2020-01-02 2020-01-01 00:02:31 2020-01-02 03:30:52 2020-01-01 00:02:31.000 2020-01-02 03:30:52.000 151 99052 49601.5 4960150 151 99052 49601.5 4960150 -32418 32517 5037.02 503702 -126 126 1.5 150 +152 2 10142 99053 0.45645 297.45645 148.95645 14895.64564 0.45645 297.45645 148.95645 14895.6454 0.45645 297.45645 148.95645000000002 14895.64500 2020-01-01 2020-01-02 2020-01-01 00:02:32 2020-01-02 03:30:53 2020-01-01 00:02:32.000 2020-01-02 03:30:53.000 152 99053 49602.5 4960250 152 99053 49602.5 4960250 -32417 32518 5038.02 503802 -125 127 2.5 250 +153 2 10143 99054 0.45945 297.45945 148.95945 14895.94594 0.45945 297.45947 148.95946 14895.94601 0.45945 297.45945 148.95945 14895.94500 2020-01-01 2020-01-02 2020-01-01 00:02:33 2020-01-02 03:30:54 2020-01-01 00:02:33.000 2020-01-02 03:30:54.000 153 99054 49603.5 4960350 153 99054 49603.5 4960350 -32416 32519 5039.02 503902 -128 127 0.94 94 +154 2 10144 99055 0.46246 297.46246 148.96246 14896.24624 0.46246 297.46246 148.96246 14896.24633 0.46246 297.46246 148.96246 14896.24600 2020-01-01 2020-01-02 2020-01-01 00:02:34 2020-01-02 03:30:55 2020-01-01 00:02:34.000 2020-01-02 03:30:55.000 154 99055 49604.5 4960450 154 99055 49604.5 4960450 -32415 32520 5040.02 504002 -128 127 -0.62 -62 +155 2 10145 99056 0.46546 297.46546 148.96546 14896.54654 0.46546 297.46545 148.96546 14896.54647 0.46546 297.46546 148.96546 14896.54600 2020-01-01 2020-01-02 2020-01-01 00:02:35 2020-01-02 03:30:56 2020-01-01 00:02:35.000 2020-01-02 03:30:56.000 155 99056 49605.5 4960550 155 99056 49605.5 4960550 -32414 32521 5041.02 504102 -128 123 -2.18 -218 +156 2 10146 99057 0.46846 297.46846 148.96846 14896.84684 0.46846 297.46848 148.96847 14896.84721 0.46846 297.46846 148.96846 14896.84600 2020-01-01 2020-01-02 2020-01-01 00:02:36 2020-01-02 03:30:57 2020-01-01 00:02:36.000 2020-01-02 03:30:57.000 156 99057 49606.5 4960650 156 99057 49606.5 4960650 -32413 32522 5042.02 504202 -127 124 -1.18 -118 +157 2 10147 99058 0.47147 297.47147 148.97147 14897.14714 0.47147 297.47147 148.97146 14897.14687 0.47147 297.47147 148.97147 14897.14700 2020-01-01 2020-01-02 2020-01-01 00:02:37 2020-01-02 03:30:58 2020-01-01 00:02:37.000 2020-01-02 03:30:58.000 157 99058 49607.5 4960750 157 99058 49607.5 4960750 -32412 32523 5043.02 504302 -126 125 -0.18 -18 +158 2 10148 99059 0.47447 297.47447 148.97447 14897.44744 0.47447 297.4745 148.97447 14897.44748 0.47447 297.47447 148.97447 14897.44700 2020-01-01 2020-01-02 2020-01-01 00:02:38 2020-01-02 03:30:59 2020-01-01 00:02:38.000 2020-01-02 03:30:59.000 158 99059 49608.5 4960850 158 99059 49608.5 4960850 -32411 32524 5044.02 504402 -125 126 0.82 82 +159 2 10149 99060 0.47747 297.47747 148.97747 14897.74774 0.47747 297.47748 148.97747 14897.74779 0.47747 297.47747 148.97746999999998 14897.74700 2020-01-01 2020-01-02 2020-01-01 00:02:39 2020-01-02 03:31:00 2020-01-01 00:02:39.000 2020-01-02 03:31:00.000 159 99060 49609.5 4960950 159 99060 49609.5 4960950 -32410 32525 5045.02 504502 -124 127 1.82 182 +16 2 10006 99916 0.04804 300.04804 150.04804 15154.85285 0.04804 300.04803 150.04804 15154.85279 0.04804 300.04804 150.04804 15154.85204 2020-01-01 2020-01-02 2020-01-01 00:00:16 2020-01-02 03:45:16 2020-01-01 00:00:16.000 2020-01-02 03:45:16.000 16 99916 49966 5046566 16 99916 49966 5046566 -32553 32382 4545.009900990099 459046 -128 127 0.37623762376237624 38 +160 2 10150 99061 0.48048 297.48048 148.98048 14898.04804 0.48048 297.48047 148.98048 14898.0481 0.48048 297.48048 148.98048 14898.04800 2020-01-01 2020-01-02 2020-01-01 00:02:40 2020-01-02 03:31:01 2020-01-01 00:02:40.000 2020-01-02 03:31:01.000 160 99061 49610.5 4961050 160 99061 49610.5 4961050 -32409 32526 5046.02 504602 -128 127 0.26 26 +161 2 10151 99062 0.48348 297.48348 148.98348 14898.34834 0.48348 297.4835 148.98348 14898.34868 0.48348 297.48348 148.98348 14898.34800 2020-01-01 2020-01-02 2020-01-01 00:02:41 2020-01-02 03:31:02 2020-01-01 00:02:41.000 2020-01-02 03:31:02.000 161 99062 49611.5 4961150 161 99062 49611.5 4961150 -32408 32527 5047.02 504702 -128 123 -1.3 -130 +162 2 10152 99063 0.48648 297.48648 148.98648 14898.64864 0.48648 297.48648 148.98648 14898.64837 0.48648 297.48648 148.98648 14898.64800 2020-01-01 2020-01-02 2020-01-01 00:02:42 2020-01-02 03:31:03 2020-01-01 00:02:42.000 2020-01-02 03:31:03.000 162 99063 49612.5 4961250 162 99063 49612.5 4961250 -32407 32528 5048.02 504802 -127 124 -0.3 -30 +163 2 10153 99064 0.48948 297.48948 148.98948 14898.94894 0.48948 297.4895 148.98948 14898.94895 0.48948 297.48948 148.98948000000001 14898.94800 2020-01-01 2020-01-02 2020-01-01 00:02:43 2020-01-02 03:31:04 2020-01-01 00:02:43.000 2020-01-02 03:31:04.000 163 99064 49613.5 4961350 163 99064 49613.5 4961350 -32406 32529 5049.02 504902 -126 125 0.7 70 +164 2 10154 99065 0.49249 297.49249 148.99249 14899.24924 0.49249 297.4925 148.99249 14899.24926 0.49249 297.49249 148.99249 14899.24900 2020-01-01 2020-01-02 2020-01-01 00:02:44 2020-01-02 03:31:05 2020-01-01 00:02:44.000 2020-01-02 03:31:05.000 164 99065 49614.5 4961450 164 99065 49614.5 4961450 -32405 32530 5050.02 505002 -125 126 1.7 170 +165 2 10155 99066 0.49549 297.49549 148.99549 14899.54954 0.49549 297.49548 148.99549 14899.54957 0.49549 297.49549 148.99549000000002 14899.54900 2020-01-01 2020-01-02 2020-01-01 00:02:45 2020-01-02 03:31:06 2020-01-01 00:02:45.000 2020-01-02 03:31:06.000 165 99066 49615.5 4961550 165 99066 49615.5 4961550 -32404 32531 5051.02 505102 -124 127 2.7 270 +166 2 10156 99067 0.49849 297.49849 148.99849 14899.84984 0.49849 297.4985 148.9985 14899.85015 0.49849 297.49849 148.99849 14899.84900 2020-01-01 2020-01-02 2020-01-01 00:02:46 2020-01-02 03:31:07 2020-01-01 00:02:46.000 2020-01-02 03:31:07.000 166 99067 49616.5 4961650 166 99067 49616.5 4961650 -32403 32532 5052.02 505202 -128 127 1.14 114 +167 2 10157 99068 0.5015 297.5015 149.0015 14900.15015 0.5015 297.5015 149.00149 14900.14984 0.50150 297.50150 149.0015 14900.15000 2020-01-01 2020-01-02 2020-01-01 00:02:47 2020-01-02 03:31:08 2020-01-01 00:02:47.000 2020-01-02 03:31:08.000 167 99068 49617.5 4961750 167 99068 49617.5 4961750 -32402 32533 5053.02 505302 -128 123 -0.42 -42 +168 2 10158 99069 0.5045 297.5045 149.0045 14900.45045 0.5045 297.50452 149.0045 14900.45042 0.50450 297.50450 149.0045 14900.45000 2020-01-01 2020-01-02 2020-01-01 00:02:48 2020-01-02 03:31:09 2020-01-01 00:02:48.000 2020-01-02 03:31:09.000 168 99069 49618.5 4961850 168 99069 49618.5 4961850 -32401 32534 5054.02 505402 -127 124 0.58 58 +169 2 10159 99070 0.5075 297.5075 149.0075 14900.75075 0.5075 297.5075 149.0075 14900.75073 0.50750 297.50750 149.0075 14900.75000 2020-01-01 2020-01-02 2020-01-01 00:02:49 2020-01-02 03:31:10 2020-01-01 00:02:49.000 2020-01-02 03:31:10.000 169 99070 49619.5 4961950 169 99070 49619.5 4961950 -32400 32535 5055.02 505502 -126 125 1.58 158 17 2 10007 99917 0.05105 300.05105 150.05105 15155.15615 0.05105 300.05106 150.05105 15155.15638 0.05105 300.05105 150.05105 15155.15605 2020-01-01 2020-01-02 2020-01-01 00:00:17 2020-01-02 03:45:17 2020-01-01 00:00:17.000 2020-01-02 03:45:17.000 17 99917 49967 5046667 17 99917 49967 5046667 -32552 32383 4546.009900990099 459147 -128 127 -1.1584158415841583 -117 -170 2 10160 99071 0.51051 297.51051 149.01051 14901.05105 0.51051 297.5105 149.01051 14901.05104 0.51051 297.51051 149.01051000000004 14901.05100 2020-01-01 2020-01-02 2020-01-01 00:02:50 2020-01-02 03:31:11 2020-01-01 00:02:50.000 2020-01-02 03:31:11.000 170 99071 49620.5 4962050 170 99071 49620.5 4962050 -32399 32536 5056.02 505602 -125 126 2.58 258 -171 2 10161 99072 0.51351 297.51351 149.01351 14901.35135 0.51351 297.51352 149.01351 14901.35162 0.51351 297.51351 149.01351000000005 14901.35100 2020-01-01 2020-01-02 2020-01-01 00:02:51 2020-01-02 03:31:12 2020-01-01 00:02:51.000 2020-01-02 03:31:12.000 171 99072 49621.5 4962150 171 99072 49621.5 4962150 -32398 32537 5057.02 505702 -124 127 3.58 358 -172 2 10162 99073 0.51651 297.51651 149.01651 14901.65165 0.51651 297.5165 149.01651 14901.65131 0.51651 297.51651 149.01650999999998 14901.65100 2020-01-01 2020-01-02 2020-01-01 00:02:52 2020-01-02 03:31:13 2020-01-01 00:02:52.000 2020-01-02 03:31:13.000 172 99073 49622.5 4962250 172 99073 49622.5 4962250 -32397 32538 5058.02 505802 -128 127 2.02 202 -173 2 10163 99074 0.51951 297.51951 149.01951 14901.95195 0.51951 297.51953 149.01951 14901.95189 0.51951 297.51951 149.01951000000005 14901.95100 2020-01-01 2020-01-02 2020-01-01 00:02:53 2020-01-02 03:31:14 2020-01-01 00:02:53.000 2020-01-02 03:31:14.000 173 99074 49623.5 4962350 173 99074 49623.5 4962350 -32396 32539 5059.02 505902 -128 127 0.46 46 -174 2 10164 99075 0.52252 297.52252 149.02252 14902.25225 0.52252 297.52252 149.02252 14902.2522 0.52252 297.52252 149.02252000000024 14902.25200 2020-01-01 2020-01-02 2020-01-01 00:02:54 2020-01-02 03:31:15 2020-01-01 00:02:54.000 2020-01-02 03:31:15.000 174 99075 49624.5 4962450 174 99075 49624.5 4962450 -32395 32540 5060.02 506002 -128 124 -1.1 -110 -175 2 10165 99076 0.52552 297.52552 149.02552 14902.55255 0.52552 297.5255 149.02552 14902.55251 0.52552 297.52552 149.02551999999986 14902.55200 2020-01-01 2020-01-02 2020-01-01 00:02:55 2020-01-02 03:31:16 2020-01-01 00:02:55.000 2020-01-02 03:31:16.000 175 99076 49625.5 4962550 175 99076 49625.5 4962550 -32394 32541 5061.02 506102 -127 125 -0.1 -10 -176 2 10166 99077 0.52852 297.52852 149.02852 14902.85285 0.52852 297.52853 149.02853 14902.85312 0.52852 297.52852 149.0285199999999 14902.85200 2020-01-01 2020-01-02 2020-01-01 00:02:56 2020-01-02 03:31:17 2020-01-01 00:02:56.000 2020-01-02 03:31:17.000 176 99077 49626.5 4962650 176 99077 49626.5 4962650 -32393 32542 5062.02 506202 -126 126 0.9 90 -177 2 10167 99078 0.53153 297.53153 149.03153 14903.15315 0.53153 297.53152 149.03152 14903.15278 0.53153 297.53153 149.03153000000015 14903.15300 2020-01-01 2020-01-02 2020-01-01 00:02:57 2020-01-02 03:31:18 2020-01-01 00:02:57.000 2020-01-02 03:31:18.000 177 99078 49627.5 4962750 177 99078 49627.5 4962750 -32392 32543 5063.02 506302 -125 127 1.9 190 -178 2 10168 99079 0.53453 297.53453 149.03453 14903.45345 0.53453 297.53455 149.03453 14903.45352 0.53453 297.53453 149.03453000000016 14903.45300 2020-01-01 2020-01-02 2020-01-01 00:02:58 2020-01-02 03:31:19 2020-01-01 00:02:58.000 2020-01-02 03:31:19.000 178 99079 49628.5 4962850 178 99079 49628.5 4962850 -32391 32544 5064.02 506402 -128 127 0.34 34 -179 2 10169 99080 0.53753 297.53753 149.03753 14903.75375 0.53753 297.53754 149.03753 14903.75366 0.53753 297.53753 149.0375299999998 14903.75300 2020-01-01 2020-01-02 2020-01-01 00:02:59 2020-01-02 03:31:20 2020-01-01 00:02:59.000 2020-01-02 03:31:20.000 179 99080 49629.5 4962950 179 99080 49629.5 4962950 -32390 32545 5065.02 506502 -128 127 -1.22 -122 -18 2 10008 99918 0.05405 300.05405 150.05405 15155.45945 0.05405 300.05405 150.05404 15155.45903 0.05405 300.05405 150.05405000000007 15155.45905 2020-01-01 2020-01-02 2020-01-01 00:00:18 2020-01-02 03:45:18 2020-01-01 00:00:18.000 2020-01-02 03:45:18.000 18 99918 49968 5046768 18 99918 49968 5046768 -32551 32384 4547.009900990099 459248 -128 124 -2.6930693069306932 -272 +170 2 10160 99071 0.51051 297.51051 149.01051 14901.05105 0.51051 297.5105 149.01051 14901.05104 0.51051 297.51051 149.01050999999998 14901.05100 2020-01-01 2020-01-02 2020-01-01 00:02:50 2020-01-02 03:31:11 2020-01-01 00:02:50.000 2020-01-02 03:31:11.000 170 99071 49620.5 4962050 170 99071 49620.5 4962050 -32399 32536 5056.02 505602 -125 126 2.58 258 +171 2 10161 99072 0.51351 297.51351 149.01351 14901.35135 0.51351 297.51352 149.01351 14901.35162 0.51351 297.51351 149.01351 14901.35100 2020-01-01 2020-01-02 2020-01-01 00:02:51 2020-01-02 03:31:12 2020-01-01 00:02:51.000 2020-01-02 03:31:12.000 171 99072 49621.5 4962150 171 99072 49621.5 4962150 -32398 32537 5057.02 505702 -124 127 3.58 358 +172 2 10162 99073 0.51651 297.51651 149.01651 14901.65165 0.51651 297.5165 149.01651 14901.65131 0.51651 297.51651 149.01651 14901.65100 2020-01-01 2020-01-02 2020-01-01 00:02:52 2020-01-02 03:31:13 2020-01-01 00:02:52.000 2020-01-02 03:31:13.000 172 99073 49622.5 4962250 172 99073 49622.5 4962250 -32397 32538 5058.02 505802 -128 127 2.02 202 +173 2 10163 99074 0.51951 297.51951 149.01951 14901.95195 0.51951 297.51953 149.01951 14901.95189 0.51951 297.51951 149.01951 14901.95100 2020-01-01 2020-01-02 2020-01-01 00:02:53 2020-01-02 03:31:14 2020-01-01 00:02:53.000 2020-01-02 03:31:14.000 173 99074 49623.5 4962350 173 99074 49623.5 4962350 -32396 32539 5059.02 505902 -128 127 0.46 46 +174 2 10164 99075 0.52252 297.52252 149.02252 14902.25225 0.52252 297.52252 149.02252 14902.2522 0.52252 297.52252 149.02252000000001 14902.25200 2020-01-01 2020-01-02 2020-01-01 00:02:54 2020-01-02 03:31:15 2020-01-01 00:02:54.000 2020-01-02 03:31:15.000 174 99075 49624.5 4962450 174 99075 49624.5 4962450 -32395 32540 5060.02 506002 -128 124 -1.1 -110 +175 2 10165 99076 0.52552 297.52552 149.02552 14902.55255 0.52552 297.5255 149.02552 14902.55251 0.52552 297.52552 149.02552 14902.55200 2020-01-01 2020-01-02 2020-01-01 00:02:55 2020-01-02 03:31:16 2020-01-01 00:02:55.000 2020-01-02 03:31:16.000 175 99076 49625.5 4962550 175 99076 49625.5 4962550 -32394 32541 5061.02 506102 -127 125 -0.1 -10 +176 2 10166 99077 0.52852 297.52852 149.02852 14902.85285 0.52852 297.52853 149.02853 14902.85312 0.52852 297.52852 149.02852000000001 14902.85200 2020-01-01 2020-01-02 2020-01-01 00:02:56 2020-01-02 03:31:17 2020-01-01 00:02:56.000 2020-01-02 03:31:17.000 176 99077 49626.5 4962650 176 99077 49626.5 4962650 -32393 32542 5062.02 506202 -126 126 0.9 90 +177 2 10167 99078 0.53153 297.53153 149.03153 14903.15315 0.53153 297.53152 149.03152 14903.15278 0.53153 297.53153 149.03153 14903.15300 2020-01-01 2020-01-02 2020-01-01 00:02:57 2020-01-02 03:31:18 2020-01-01 00:02:57.000 2020-01-02 03:31:18.000 177 99078 49627.5 4962750 177 99078 49627.5 4962750 -32392 32543 5063.02 506302 -125 127 1.9 190 +178 2 10168 99079 0.53453 297.53453 149.03453 14903.45345 0.53453 297.53455 149.03453 14903.45352 0.53453 297.53453 149.03453 14903.45300 2020-01-01 2020-01-02 2020-01-01 00:02:58 2020-01-02 03:31:19 2020-01-01 00:02:58.000 2020-01-02 03:31:19.000 178 99079 49628.5 4962850 178 99079 49628.5 4962850 -32391 32544 5064.02 506402 -128 127 0.34 34 +179 2 10169 99080 0.53753 297.53753 149.03753 14903.75375 0.53753 297.53754 149.03753 14903.75366 0.53753 297.53753 149.03753 14903.75300 2020-01-01 2020-01-02 2020-01-01 00:02:59 2020-01-02 03:31:20 2020-01-01 00:02:59.000 2020-01-02 03:31:20.000 179 99080 49629.5 4962950 179 99080 49629.5 4962950 -32390 32545 5065.02 506502 -128 127 -1.22 -122 +18 2 10008 99918 0.05405 300.05405 150.05405 15155.45945 0.05405 300.05405 150.05404 15155.45903 0.05405 300.05405 150.05405 15155.45905 2020-01-01 2020-01-02 2020-01-01 00:00:18 2020-01-02 03:45:18 2020-01-01 00:00:18.000 2020-01-02 03:45:18.000 18 99918 49968 5046768 18 99918 49968 5046768 -32551 32384 4547.009900990099 459248 -128 124 -2.6930693069306932 -272 180 2 10170 99081 0.54054 297.54054 149.04054 14904.05405 0.54054 297.54053 149.04053 14904.05398 0.54054 297.54054 149.04054 14904.05400 2020-01-01 2020-01-02 2020-01-01 00:03:00 2020-01-02 03:31:21 2020-01-01 00:03:00.000 2020-01-02 03:31:21.000 180 99081 49630.5 4963050 180 99081 49630.5 4963050 -32389 32546 5066.02 506602 -128 123 -2.78 -278 -181 2 10171 99082 0.54354 297.54354 149.04354 14904.35435 0.54354 297.54355 149.04354 14904.35459 0.54354 297.54354 149.04354000000006 14904.35400 2020-01-01 2020-01-02 2020-01-01 00:03:01 2020-01-02 03:31:22 2020-01-01 00:03:01.000 2020-01-02 03:31:22.000 181 99082 49631.5 4963150 181 99082 49631.5 4963150 -32388 32547 5067.02 506702 -127 124 -1.78 -178 -182 2 10172 99083 0.54654 297.54654 149.04654 14904.65465 0.54654 297.54654 149.04654 14904.65425 0.54654 297.54654 149.0465399999999 14904.65400 2020-01-01 2020-01-02 2020-01-01 00:03:02 2020-01-02 03:31:23 2020-01-01 00:03:02.000 2020-01-02 03:31:23.000 182 99083 49632.5 4963250 182 99083 49632.5 4963250 -32387 32548 5068.02 506802 -126 125 -0.78 -78 -183 2 10173 99084 0.54954 297.54954 149.04954 14904.95495 0.54954 297.54956 149.04954 14904.95498 0.54954 297.54954 149.04953999999998 14904.95400 2020-01-01 2020-01-02 2020-01-01 00:03:03 2020-01-02 03:31:24 2020-01-01 00:03:03.000 2020-01-02 03:31:24.000 183 99084 49633.5 4963350 183 99084 49633.5 4963350 -32386 32549 5069.02 506902 -125 126 0.22 22 -184 2 10174 99085 0.55255 297.55255 149.05255 14905.25525 0.55255 297.55255 149.05255 14905.25514 0.55255 297.55255 149.05255000000017 14905.25500 2020-01-01 2020-01-02 2020-01-01 00:03:04 2020-01-02 03:31:25 2020-01-01 00:03:04.000 2020-01-02 03:31:25.000 184 99085 49634.5 4963450 184 99085 49634.5 4963450 -32385 32550 5070.02 507002 -124 127 1.22 122 -185 2 10175 99086 0.55555 297.55555 149.05555 14905.55555 0.55555 297.55554 149.05555 14905.55549 0.55555 297.55555 149.05554999999978 14905.55500 2020-01-01 2020-01-02 2020-01-01 00:03:05 2020-01-02 03:31:26 2020-01-01 00:03:05.000 2020-01-02 03:31:26.000 185 99086 49635.5 4963550 185 99086 49635.5 4963550 -32384 32551 5071.02 507102 -128 127 -0.34 -34 -186 2 10176 99087 0.55855 297.55855 149.05855 14905.85585 0.55855 297.55856 149.05856 14905.85607 0.55855 297.55855 149.05854999999988 14905.85500 2020-01-01 2020-01-02 2020-01-01 00:03:06 2020-01-02 03:31:27 2020-01-01 00:03:06.000 2020-01-02 03:31:27.000 186 99087 49636.5 4963650 186 99087 49636.5 4963650 -32383 32552 5072.02 507202 -128 123 -1.9 -190 -187 2 10177 99088 0.56156 297.56156 149.06156 14906.15615 0.56156 297.56155 149.06155 14906.15572 0.56156 297.56156 149.06156000000007 14906.15600 2020-01-01 2020-01-02 2020-01-01 00:03:07 2020-01-02 03:31:28 2020-01-01 00:03:07.000 2020-01-02 03:31:28.000 187 99088 49637.5 4963750 187 99088 49637.5 4963750 -32382 32553 5073.02 507302 -127 124 -0.9 -90 -188 2 10178 99089 0.56456 297.56456 149.06456 14906.45645 0.56456 297.56458 149.06456 14906.45645 0.56456 297.56456 149.06456000000009 14906.45600 2020-01-01 2020-01-02 2020-01-01 00:03:08 2020-01-02 03:31:29 2020-01-01 00:03:08.000 2020-01-02 03:31:29.000 188 99089 49638.5 4963850 188 99089 49638.5 4963850 -32381 32554 5074.02 507402 -126 125 0.1 10 +181 2 10171 99082 0.54354 297.54354 149.04354 14904.35435 0.54354 297.54355 149.04354 14904.35459 0.54354 297.54354 149.04354 14904.35400 2020-01-01 2020-01-02 2020-01-01 00:03:01 2020-01-02 03:31:22 2020-01-01 00:03:01.000 2020-01-02 03:31:22.000 181 99082 49631.5 4963150 181 99082 49631.5 4963150 -32388 32547 5067.02 506702 -127 124 -1.78 -178 +182 2 10172 99083 0.54654 297.54654 149.04654 14904.65465 0.54654 297.54654 149.04654 14904.65425 0.54654 297.54654 149.04654 14904.65400 2020-01-01 2020-01-02 2020-01-01 00:03:02 2020-01-02 03:31:23 2020-01-01 00:03:02.000 2020-01-02 03:31:23.000 182 99083 49632.5 4963250 182 99083 49632.5 4963250 -32387 32548 5068.02 506802 -126 125 -0.78 -78 +183 2 10173 99084 0.54954 297.54954 149.04954 14904.95495 0.54954 297.54956 149.04954 14904.95498 0.54954 297.54954 149.04954 14904.95400 2020-01-01 2020-01-02 2020-01-01 00:03:03 2020-01-02 03:31:24 2020-01-01 00:03:03.000 2020-01-02 03:31:24.000 183 99084 49633.5 4963350 183 99084 49633.5 4963350 -32386 32549 5069.02 506902 -125 126 0.22 22 +184 2 10174 99085 0.55255 297.55255 149.05255 14905.25525 0.55255 297.55255 149.05255 14905.25514 0.55255 297.55255 149.05255 14905.25500 2020-01-01 2020-01-02 2020-01-01 00:03:04 2020-01-02 03:31:25 2020-01-01 00:03:04.000 2020-01-02 03:31:25.000 184 99085 49634.5 4963450 184 99085 49634.5 4963450 -32385 32550 5070.02 507002 -124 127 1.22 122 +185 2 10175 99086 0.55555 297.55555 149.05555 14905.55555 0.55555 297.55554 149.05555 14905.55549 0.55555 297.55555 149.05555 14905.55500 2020-01-01 2020-01-02 2020-01-01 00:03:05 2020-01-02 03:31:26 2020-01-01 00:03:05.000 2020-01-02 03:31:26.000 185 99086 49635.5 4963550 185 99086 49635.5 4963550 -32384 32551 5071.02 507102 -128 127 -0.34 -34 +186 2 10176 99087 0.55855 297.55855 149.05855 14905.85585 0.55855 297.55856 149.05856 14905.85607 0.55855 297.55855 149.05855 14905.85500 2020-01-01 2020-01-02 2020-01-01 00:03:06 2020-01-02 03:31:27 2020-01-01 00:03:06.000 2020-01-02 03:31:27.000 186 99087 49636.5 4963650 186 99087 49636.5 4963650 -32383 32552 5072.02 507202 -128 123 -1.9 -190 +187 2 10177 99088 0.56156 297.56156 149.06156 14906.15615 0.56156 297.56155 149.06155 14906.15572 0.56156 297.56156 149.06156000000001 14906.15600 2020-01-01 2020-01-02 2020-01-01 00:03:07 2020-01-02 03:31:28 2020-01-01 00:03:07.000 2020-01-02 03:31:28.000 187 99088 49637.5 4963750 187 99088 49637.5 4963750 -32382 32553 5073.02 507302 -127 124 -0.9 -90 +188 2 10178 99089 0.56456 297.56456 149.06456 14906.45645 0.56456 297.56458 149.06456 14906.45645 0.56456 297.56456 149.06456 14906.45600 2020-01-01 2020-01-02 2020-01-01 00:03:08 2020-01-02 03:31:29 2020-01-01 00:03:08.000 2020-01-02 03:31:29.000 188 99089 49638.5 4963850 188 99089 49638.5 4963850 -32381 32554 5074.02 507402 -126 125 0.1 10 189 2 10179 99090 0.56756 297.56756 149.06756 14906.75675 0.56756 297.56757 149.06756 14906.75661 0.56756 297.56756 149.06756 14906.75600 2020-01-01 2020-01-02 2020-01-01 00:03:09 2020-01-02 03:31:30 2020-01-01 00:03:09.000 2020-01-02 03:31:30.000 189 99090 49639.5 4963950 189 99090 49639.5 4963950 -32380 32555 5075.02 507502 -125 126 1.1 110 19 2 10009 99919 0.05705 300.05705 150.05705 15155.76276 0.05705 300.05707 150.05705 15155.76279 0.05705 300.05705 150.05705 15155.76205 2020-01-01 2020-01-02 2020-01-01 00:00:19 2020-01-02 03:45:19 2020-01-01 00:00:19.000 2020-01-02 03:45:19.000 19 99919 49969 5046869 19 99919 49969 5046869 -32550 32385 4548.009900990099 459349 -127 125 -1.693069306930693 -171 -190 2 10180 99091 0.57057 297.57057 149.07057 14907.05705 0.57057 297.57056 149.07056 14907.05695 0.57057 297.57057 149.0705699999999 14907.05700 2020-01-01 2020-01-02 2020-01-01 00:03:10 2020-01-02 03:31:31 2020-01-01 00:03:10.000 2020-01-02 03:31:31.000 190 99091 49640.5 4964050 190 99091 49640.5 4964050 -32379 32556 5076.02 507602 -124 127 2.1 210 +190 2 10180 99091 0.57057 297.57057 149.07057 14907.05705 0.57057 297.57056 149.07056 14907.05695 0.57057 297.57057 149.07057 14907.05700 2020-01-01 2020-01-02 2020-01-01 00:03:10 2020-01-02 03:31:31 2020-01-01 00:03:10.000 2020-01-02 03:31:31.000 190 99091 49640.5 4964050 190 99091 49640.5 4964050 -32379 32556 5076.02 507602 -124 127 2.1 210 191 2 10181 99092 0.57357 297.57357 149.07357 14907.35735 0.57357 297.57358 149.07357 14907.35753 0.57357 297.57357 149.07357 14907.35700 2020-01-01 2020-01-02 2020-01-01 00:03:11 2020-01-02 03:31:32 2020-01-01 00:03:11.000 2020-01-02 03:31:32.000 191 99092 49641.5 4964150 191 99092 49641.5 4964150 -32378 32557 5077.02 507702 -128 127 0.54 54 -192 2 10182 99093 0.57657 297.57657 149.07657 14907.65765 0.57657 297.57657 149.07657 14907.65784 0.57657 297.57657 149.07656999999992 14907.65700 2020-01-01 2020-01-02 2020-01-01 00:03:12 2020-01-02 03:31:33 2020-01-01 00:03:12.000 2020-01-02 03:31:33.000 192 99093 49642.5 4964250 192 99093 49642.5 4964250 -32377 32558 5078.02 507802 -128 123 -1.02 -102 -193 2 10183 99094 0.57957 297.57957 149.07957 14907.95795 0.57957 297.5796 149.07957 14907.95793 0.57957 297.57957 149.0795699999999 14907.95700 2020-01-01 2020-01-02 2020-01-01 00:03:13 2020-01-02 03:31:34 2020-01-01 00:03:13.000 2020-01-02 03:31:34.000 193 99094 49643.5 4964350 193 99094 49643.5 4964350 -32376 32559 5079.02 507902 -127 124 -0.02 -2 -194 2 10184 99095 0.58258 297.58258 149.08258 14908.25825 0.58258 297.58258 149.08258 14908.25811 0.58258 297.58258 149.08258000000015 14908.25800 2020-01-01 2020-01-02 2020-01-01 00:03:14 2020-01-02 03:31:35 2020-01-01 00:03:14.000 2020-01-02 03:31:35.000 194 99095 49644.5 4964450 194 99095 49644.5 4964450 -32375 32560 5080.02 508002 -126 125 0.98 98 -195 2 10185 99096 0.58558 297.58558 149.08558 14908.55855 0.58558 297.58557 149.08558 14908.55842 0.58558 297.58558 149.0855800000002 14908.55800 2020-01-01 2020-01-02 2020-01-01 00:03:15 2020-01-02 03:31:36 2020-01-01 00:03:15.000 2020-01-02 03:31:36.000 195 99096 49645.5 4964550 195 99096 49645.5 4964550 -32374 32561 5081.02 508102 -125 126 1.98 198 -196 2 10186 99097 0.58858 297.58858 149.08858 14908.85885 0.58858 297.5886 149.08859 14908.859 0.58858 297.58858 149.08857999999984 14908.85800 2020-01-01 2020-01-02 2020-01-01 00:03:16 2020-01-02 03:31:37 2020-01-01 00:03:16.000 2020-01-02 03:31:37.000 196 99097 49646.5 4964650 196 99097 49646.5 4964650 -32373 32562 5082.02 508202 -124 127 2.98 298 +192 2 10182 99093 0.57657 297.57657 149.07657 14907.65765 0.57657 297.57657 149.07657 14907.65784 0.57657 297.57657 149.07657 14907.65700 2020-01-01 2020-01-02 2020-01-01 00:03:12 2020-01-02 03:31:33 2020-01-01 00:03:12.000 2020-01-02 03:31:33.000 192 99093 49642.5 4964250 192 99093 49642.5 4964250 -32377 32558 5078.02 507802 -128 123 -1.02 -102 +193 2 10183 99094 0.57957 297.57957 149.07957 14907.95795 0.57957 297.5796 149.07957 14907.95793 0.57957 297.57957 149.07957 14907.95700 2020-01-01 2020-01-02 2020-01-01 00:03:13 2020-01-02 03:31:34 2020-01-01 00:03:13.000 2020-01-02 03:31:34.000 193 99094 49643.5 4964350 193 99094 49643.5 4964350 -32376 32559 5079.02 507902 -127 124 -0.02 -2 +194 2 10184 99095 0.58258 297.58258 149.08258 14908.25825 0.58258 297.58258 149.08258 14908.25811 0.58258 297.58258 149.08258 14908.25800 2020-01-01 2020-01-02 2020-01-01 00:03:14 2020-01-02 03:31:35 2020-01-01 00:03:14.000 2020-01-02 03:31:35.000 194 99095 49644.5 4964450 194 99095 49644.5 4964450 -32375 32560 5080.02 508002 -126 125 0.98 98 +195 2 10185 99096 0.58558 297.58558 149.08558 14908.55855 0.58558 297.58557 149.08558 14908.55842 0.58558 297.58558 149.08558000000002 14908.55800 2020-01-01 2020-01-02 2020-01-01 00:03:15 2020-01-02 03:31:36 2020-01-01 00:03:15.000 2020-01-02 03:31:36.000 195 99096 49645.5 4964550 195 99096 49645.5 4964550 -32374 32561 5081.02 508102 -125 126 1.98 198 +196 2 10186 99097 0.58858 297.58858 149.08858 14908.85885 0.58858 297.5886 149.08859 14908.859 0.58858 297.58858 149.08858 14908.85800 2020-01-01 2020-01-02 2020-01-01 00:03:16 2020-01-02 03:31:37 2020-01-01 00:03:16.000 2020-01-02 03:31:37.000 196 99097 49646.5 4964650 196 99097 49646.5 4964650 -32373 32562 5082.02 508202 -124 127 2.98 298 197 2 10187 99098 0.59159 297.59159 149.09159 14909.15915 0.59159 297.59158 149.09159 14909.15931 0.59159 297.59159 149.09159 14909.15900 2020-01-01 2020-01-02 2020-01-01 00:03:17 2020-01-02 03:31:38 2020-01-01 00:03:17.000 2020-01-02 03:31:38.000 197 99098 49647.5 4964750 197 99098 49647.5 4964750 -32372 32563 5083.02 508302 -128 127 1.42 142 -198 2 10188 99099 0.59459 297.59459 149.09459 14909.45945 0.59459 297.5946 149.09459 14909.4594 0.59459 297.59459 149.09459000000007 14909.45900 2020-01-01 2020-01-02 2020-01-01 00:03:18 2020-01-02 03:31:39 2020-01-01 00:03:18.000 2020-01-02 03:31:39.000 198 99099 49648.5 4964850 198 99099 49648.5 4964850 -32371 32564 5084.02 508402 -128 127 -0.14 -14 -199 2 10189 99100 0.59759 297.59759 149.09759 14909.75975 0.59759 297.5976 149.09759 14909.75958 0.59759 297.59759 149.09758999999994 14909.75900 2020-01-01 2020-01-02 2020-01-01 00:03:19 2020-01-02 03:31:40 2020-01-01 00:03:19.000 2020-01-02 03:31:40.000 199 99100 49649.5 4964950 199 99100 49649.5 4964950 -32370 32565 5085.02 508502 -128 124 -1.7 -170 -2 2 1001 9992 0.006 300.006 150.006 15150.6066 0.006 300.006 150.006 15150.6069 0.00600 300.00600 150.00599999999991 15150.60600 2020-01-01 2020-01-02 2020-01-01 00:00:02 2020-01-02 03:45:02 2020-01-01 00:00:02.000 2020-01-02 03:45:02.000 2 99902 49952 5045152 2 99902 49952 5045152 -32567 32368 4531.009900990099 457632 -125 126 -0.9504950495049505 -96 -20 2 10010 99920 0.06006 300.06006 150.06006 15156.06606 0.06006 300.06006 150.06005 15156.06593 0.06006 300.06006 150.06005999999988 15156.06606 2020-01-01 2020-01-02 2020-01-01 00:00:20 2020-01-02 03:45:20 2020-01-01 00:00:20.000 2020-01-02 03:45:20.000 20 99920 49970 5046970 20 99920 49970 5046970 -32549 32386 4549.009900990099 459450 -126 126 -0.693069306930693 -70 -200 2 10190 99101 0.6006 297.6006 149.1006 14910.06006 0.6006 297.6006 149.10059 14910.0599 0.60060 297.60060 149.10059999999987 14910.06000 2020-01-01 2020-01-02 2020-01-01 00:03:20 2020-01-02 03:31:41 2020-01-01 00:03:20.000 2020-01-02 03:31:41.000 200 99101 49650.5 4965050 200 99101 49650.5 4965050 -32369 32566 5086.02 508602 -127 125 -0.7 -70 -201 2 10191 99102 0.6036 297.6036 149.1036 14910.36036 0.6036 297.6036 149.1036 14910.36063 0.60360 297.60360 149.10360000000017 14910.36000 2020-01-01 2020-01-02 2020-01-01 00:03:21 2020-01-02 03:31:42 2020-01-01 00:03:21.000 2020-01-02 03:31:42.000 201 99102 49651.5 4965150 201 99102 49651.5 4965150 -32368 32567 5087.02 508702 -126 126 0.3 30 -202 2 10192 99103 0.6066 297.6066 149.1066 14910.66066 0.6066 297.6066 149.1066 14910.66078 0.60660 297.60660 149.10659999999982 14910.66000 2020-01-01 2020-01-02 2020-01-01 00:03:22 2020-01-02 03:31:43 2020-01-01 00:03:22.000 2020-01-02 03:31:43.000 202 99103 49652.5 4965250 202 99103 49652.5 4965250 -32367 32568 5088.02 508802 -125 127 1.3 130 -203 2 10193 99104 0.6096 297.6096 149.1096 14910.96096 0.6096 297.60962 149.1096 14910.9609 0.60960 297.60960 149.10959999999983 14910.96000 2020-01-01 2020-01-02 2020-01-01 00:03:23 2020-01-02 03:31:44 2020-01-01 00:03:23.000 2020-01-02 03:31:44.000 203 99104 49653.5 4965350 203 99104 49653.5 4965350 -32366 32569 5089.02 508902 -128 127 -0.26 -26 -204 2 10194 99105 0.61261 297.61261 149.11261 14911.26126 0.61261 297.6126 149.11261 14911.26105 0.61261 297.61261 149.1126100000001 14911.26100 2020-01-01 2020-01-02 2020-01-01 00:03:24 2020-01-02 03:31:45 2020-01-01 00:03:24.000 2020-01-02 03:31:45.000 204 99105 49654.5 4965450 204 99105 49654.5 4965450 -32365 32570 5090.02 509002 -128 127 -1.82 -182 -205 2 10195 99106 0.61561 297.61561 149.11561 14911.56156 0.61561 297.6156 149.11561 14911.56137 0.61561 297.61561 149.11561000000012 14911.56100 2020-01-01 2020-01-02 2020-01-01 00:03:25 2020-01-02 03:31:46 2020-01-01 00:03:25.000 2020-01-02 03:31:46.000 205 99106 49655.5 4965550 205 99106 49655.5 4965550 -32364 32571 5091.02 509102 -128 123 -3.38 -338 -206 2 10196 99107 0.61861 297.61861 149.11861 14911.86186 0.61861 297.61862 149.11862 14911.8621 0.61861 297.61861 149.1186099999999 14911.86100 2020-01-01 2020-01-02 2020-01-01 00:03:26 2020-01-02 03:31:47 2020-01-01 00:03:26.000 2020-01-02 03:31:47.000 206 99107 49656.5 4965650 206 99107 49656.5 4965650 -32363 32572 5092.02 509202 -127 124 -2.38 -238 -207 2 10197 99108 0.62162 297.62162 149.12162 14912.16216 0.62162 297.6216 149.12162 14912.16225 0.62162 297.62162 149.12161999999992 14912.16200 2020-01-01 2020-01-02 2020-01-01 00:03:27 2020-01-02 03:31:48 2020-01-01 00:03:27.000 2020-01-02 03:31:48.000 207 99108 49657.5 4965750 207 99108 49657.5 4965750 -32362 32573 5093.02 509302 -126 125 -1.38 -138 +198 2 10188 99099 0.59459 297.59459 149.09459 14909.45945 0.59459 297.5946 149.09459 14909.4594 0.59459 297.59459 149.09459 14909.45900 2020-01-01 2020-01-02 2020-01-01 00:03:18 2020-01-02 03:31:39 2020-01-01 00:03:18.000 2020-01-02 03:31:39.000 198 99099 49648.5 4964850 198 99099 49648.5 4964850 -32371 32564 5084.02 508402 -128 127 -0.14 -14 +199 2 10189 99100 0.59759 297.59759 149.09759 14909.75975 0.59759 297.5976 149.09759 14909.75958 0.59759 297.59759 149.09759 14909.75900 2020-01-01 2020-01-02 2020-01-01 00:03:19 2020-01-02 03:31:40 2020-01-01 00:03:19.000 2020-01-02 03:31:40.000 199 99100 49649.5 4964950 199 99100 49649.5 4964950 -32370 32565 5085.02 508502 -128 124 -1.7 -170 +2 2 1001 9992 0.006 300.006 150.006 15150.6066 0.006 300.006 150.006 15150.6069 0.00600 300.00600 150.006 15150.60600 2020-01-01 2020-01-02 2020-01-01 00:00:02 2020-01-02 03:45:02 2020-01-01 00:00:02.000 2020-01-02 03:45:02.000 2 99902 49952 5045152 2 99902 49952 5045152 -32567 32368 4531.009900990099 457632 -125 126 -0.9504950495049505 -96 +20 2 10010 99920 0.06006 300.06006 150.06006 15156.06606 0.06006 300.06006 150.06005 15156.06593 0.06006 300.06006 150.06006 15156.06606 2020-01-01 2020-01-02 2020-01-01 00:00:20 2020-01-02 03:45:20 2020-01-01 00:00:20.000 2020-01-02 03:45:20.000 20 99920 49970 5046970 20 99920 49970 5046970 -32549 32386 4549.009900990099 459450 -126 126 -0.693069306930693 -70 +200 2 10190 99101 0.6006 297.6006 149.1006 14910.06006 0.6006 297.6006 149.10059 14910.0599 0.60060 297.60060 149.1006 14910.06000 2020-01-01 2020-01-02 2020-01-01 00:03:20 2020-01-02 03:31:41 2020-01-01 00:03:20.000 2020-01-02 03:31:41.000 200 99101 49650.5 4965050 200 99101 49650.5 4965050 -32369 32566 5086.02 508602 -127 125 -0.7 -70 +201 2 10191 99102 0.6036 297.6036 149.1036 14910.36036 0.6036 297.6036 149.1036 14910.36063 0.60360 297.60360 149.1036 14910.36000 2020-01-01 2020-01-02 2020-01-01 00:03:21 2020-01-02 03:31:42 2020-01-01 00:03:21.000 2020-01-02 03:31:42.000 201 99102 49651.5 4965150 201 99102 49651.5 4965150 -32368 32567 5087.02 508702 -126 126 0.3 30 +202 2 10192 99103 0.6066 297.6066 149.1066 14910.66066 0.6066 297.6066 149.1066 14910.66078 0.60660 297.60660 149.1066 14910.66000 2020-01-01 2020-01-02 2020-01-01 00:03:22 2020-01-02 03:31:43 2020-01-01 00:03:22.000 2020-01-02 03:31:43.000 202 99103 49652.5 4965250 202 99103 49652.5 4965250 -32367 32568 5088.02 508802 -125 127 1.3 130 +203 2 10193 99104 0.6096 297.6096 149.1096 14910.96096 0.6096 297.60962 149.1096 14910.9609 0.60960 297.60960 149.1096 14910.96000 2020-01-01 2020-01-02 2020-01-01 00:03:23 2020-01-02 03:31:44 2020-01-01 00:03:23.000 2020-01-02 03:31:44.000 203 99104 49653.5 4965350 203 99104 49653.5 4965350 -32366 32569 5089.02 508902 -128 127 -0.26 -26 +204 2 10194 99105 0.61261 297.61261 149.11261 14911.26126 0.61261 297.6126 149.11261 14911.26105 0.61261 297.61261 149.11261000000002 14911.26100 2020-01-01 2020-01-02 2020-01-01 00:03:24 2020-01-02 03:31:45 2020-01-01 00:03:24.000 2020-01-02 03:31:45.000 204 99105 49654.5 4965450 204 99105 49654.5 4965450 -32365 32570 5090.02 509002 -128 127 -1.82 -182 +205 2 10195 99106 0.61561 297.61561 149.11561 14911.56156 0.61561 297.6156 149.11561 14911.56137 0.61561 297.61561 149.11561 14911.56100 2020-01-01 2020-01-02 2020-01-01 00:03:25 2020-01-02 03:31:46 2020-01-01 00:03:25.000 2020-01-02 03:31:46.000 205 99106 49655.5 4965550 205 99106 49655.5 4965550 -32364 32571 5091.02 509102 -128 123 -3.38 -338 +206 2 10196 99107 0.61861 297.61861 149.11861 14911.86186 0.61861 297.61862 149.11862 14911.8621 0.61861 297.61861 149.11861000000002 14911.86100 2020-01-01 2020-01-02 2020-01-01 00:03:26 2020-01-02 03:31:47 2020-01-01 00:03:26.000 2020-01-02 03:31:47.000 206 99107 49656.5 4965650 206 99107 49656.5 4965650 -32363 32572 5092.02 509202 -127 124 -2.38 -238 +207 2 10197 99108 0.62162 297.62162 149.12162 14912.16216 0.62162 297.6216 149.12162 14912.16225 0.62162 297.62162 149.12162 14912.16200 2020-01-01 2020-01-02 2020-01-01 00:03:27 2020-01-02 03:31:48 2020-01-01 00:03:27.000 2020-01-02 03:31:48.000 207 99108 49657.5 4965750 207 99108 49657.5 4965750 -32362 32573 5093.02 509302 -126 125 -1.38 -138 208 2 10198 99109 0.62462 297.62462 149.12462 14912.46246 0.62462 297.62463 149.12462 14912.46237 0.62462 297.62462 149.12462 14912.46200 2020-01-01 2020-01-02 2020-01-01 00:03:28 2020-01-02 03:31:49 2020-01-01 00:03:28.000 2020-01-02 03:31:49.000 208 99109 49658.5 4965850 208 99109 49658.5 4965850 -32361 32574 5094.02 509402 -125 126 -0.38 -38 -209 2 10199 99110 0.62762 297.62762 149.12762 14912.76276 0.62762 297.62762 149.12762 14912.76253 0.62762 297.62762 149.12761999999992 14912.76200 2020-01-01 2020-01-02 2020-01-01 00:03:29 2020-01-02 03:31:50 2020-01-01 00:03:29.000 2020-01-02 03:31:50.000 209 99110 49659.5 4965950 209 99110 49659.5 4965950 -32360 32575 5095.02 509502 -124 127 0.62 62 -21 2 10011 99921 0.06306 300.06306 150.06306 15156.36936 0.06306 300.06305 150.06306 15156.36927 0.06306 300.06306 150.0630600000002 15156.36906 2020-01-01 2020-01-02 2020-01-01 00:00:21 2020-01-02 03:45:21 2020-01-01 00:00:21.000 2020-01-02 03:45:21.000 21 99921 49971 5047071 21 99921 49971 5047071 -32548 32387 4550.009900990099 459551 -125 127 0.3069306930693069 31 -210 2 10200 99111 0.63063 297.63063 149.13063 14913.06306 0.63063 297.63065 149.13063 14913.06326 0.63063 297.63063 149.13062999999983 14913.06300 2020-01-01 2020-01-02 2020-01-01 00:03:30 2020-01-02 03:31:51 2020-01-01 00:03:30.000 2020-01-02 03:31:51.000 210 99111 49660.5 4966050 210 99111 49660.5 4966050 -32359 32576 5096.02 509602 -128 127 -0.94 -94 -211 2 10201 99112 0.63363 297.63363 149.13363 14913.36336 0.63363 297.63364 149.13363 14913.36357 0.63363 297.63363 149.13363000000012 14913.36300 2020-01-01 2020-01-02 2020-01-01 00:03:31 2020-01-02 03:31:52 2020-01-01 00:03:31.000 2020-01-02 03:31:52.000 211 99112 49661.5 4966150 211 99112 49661.5 4966150 -32358 32577 5097.02 509702 -128 123 -2.5 -250 -212 2 10202 99113 0.63663 297.63663 149.13663 14913.66366 0.63663 297.63663 149.13663 14913.66372 0.63663 297.63663 149.1366300000002 14913.66300 2020-01-01 2020-01-02 2020-01-01 00:03:32 2020-01-02 03:31:53 2020-01-01 00:03:32.000 2020-01-02 03:31:53.000 212 99113 49662.5 4966250 212 99113 49662.5 4966250 -32357 32578 5098.02 509802 -127 124 -1.5 -150 -213 2 10203 99114 0.63963 297.63963 149.13963 14913.96396 0.63963 297.63965 149.13963 14913.96384 0.63963 297.63963 149.13962999999984 14913.96300 2020-01-01 2020-01-02 2020-01-01 00:03:33 2020-01-02 03:31:54 2020-01-01 00:03:33.000 2020-01-02 03:31:54.000 213 99114 49663.5 4966350 213 99114 49663.5 4966350 -32356 32579 5099.02 509902 -126 125 -0.5 -50 +209 2 10199 99110 0.62762 297.62762 149.12762 14912.76276 0.62762 297.62762 149.12762 14912.76253 0.62762 297.62762 149.12762 14912.76200 2020-01-01 2020-01-02 2020-01-01 00:03:29 2020-01-02 03:31:50 2020-01-01 00:03:29.000 2020-01-02 03:31:50.000 209 99110 49659.5 4965950 209 99110 49659.5 4965950 -32360 32575 5095.02 509502 -124 127 0.62 62 +21 2 10011 99921 0.06306 300.06306 150.06306 15156.36936 0.06306 300.06305 150.06306 15156.36927 0.06306 300.06306 150.06306 15156.36906 2020-01-01 2020-01-02 2020-01-01 00:00:21 2020-01-02 03:45:21 2020-01-01 00:00:21.000 2020-01-02 03:45:21.000 21 99921 49971 5047071 21 99921 49971 5047071 -32548 32387 4550.009900990099 459551 -125 127 0.3069306930693069 31 +210 2 10200 99111 0.63063 297.63063 149.13063 14913.06306 0.63063 297.63065 149.13063 14913.06326 0.63063 297.63063 149.13063 14913.06300 2020-01-01 2020-01-02 2020-01-01 00:03:30 2020-01-02 03:31:51 2020-01-01 00:03:30.000 2020-01-02 03:31:51.000 210 99111 49660.5 4966050 210 99111 49660.5 4966050 -32359 32576 5096.02 509602 -128 127 -0.94 -94 +211 2 10201 99112 0.63363 297.63363 149.13363 14913.36336 0.63363 297.63364 149.13363 14913.36357 0.63363 297.63363 149.13362999999998 14913.36300 2020-01-01 2020-01-02 2020-01-01 00:03:31 2020-01-02 03:31:52 2020-01-01 00:03:31.000 2020-01-02 03:31:52.000 211 99112 49661.5 4966150 211 99112 49661.5 4966150 -32358 32577 5097.02 509702 -128 123 -2.5 -250 +212 2 10202 99113 0.63663 297.63663 149.13663 14913.66366 0.63663 297.63663 149.13663 14913.66372 0.63663 297.63663 149.13663 14913.66300 2020-01-01 2020-01-02 2020-01-01 00:03:32 2020-01-02 03:31:53 2020-01-01 00:03:32.000 2020-01-02 03:31:53.000 212 99113 49662.5 4966250 212 99113 49662.5 4966250 -32357 32578 5098.02 509802 -127 124 -1.5 -150 +213 2 10203 99114 0.63963 297.63963 149.13963 14913.96396 0.63963 297.63965 149.13963 14913.96384 0.63963 297.63963 149.13963 14913.96300 2020-01-01 2020-01-02 2020-01-01 00:03:33 2020-01-02 03:31:54 2020-01-01 00:03:33.000 2020-01-02 03:31:54.000 213 99114 49663.5 4966350 213 99114 49663.5 4966350 -32356 32579 5099.02 509902 -126 125 -0.5 -50 214 2 10204 99115 0.64264 297.64264 149.14264 14914.26426 0.64264 297.64264 149.14263 14914.26399 0.64264 297.64264 149.14264 14914.26400 2020-01-01 2020-01-02 2020-01-01 00:03:34 2020-01-02 03:31:55 2020-01-01 00:03:34.000 2020-01-02 03:31:55.000 214 99115 49664.5 4966450 214 99115 49664.5 4966450 -32355 32580 5100.02 510002 -125 126 0.5 50 -215 2 10205 99116 0.64564 297.64564 149.14564 14914.56456 0.64564 297.64566 149.14564 14914.56473 0.64564 297.64564 149.1456400000001 14914.56400 2020-01-01 2020-01-02 2020-01-01 00:03:35 2020-01-02 03:31:56 2020-01-01 00:03:35.000 2020-01-02 03:31:56.000 215 99116 49665.5 4966550 215 99116 49665.5 4966550 -32354 32581 5101.02 510102 -124 127 1.5 150 -216 2 10206 99117 0.64864 297.64864 149.14864 14914.86486 0.64864 297.64865 149.14865 14914.86504 0.64864 297.64864 149.14863999999994 14914.86400 2020-01-01 2020-01-02 2020-01-01 00:03:36 2020-01-02 03:31:57 2020-01-01 00:03:36.000 2020-01-02 03:31:57.000 216 99117 49666.5 4966650 216 99117 49666.5 4966650 -32353 32582 5102.02 510202 -128 127 -0.06 -6 -217 2 10207 99118 0.65165 297.65165 149.15165 14915.16516 0.65165 297.65164 149.15165 14915.16523 0.65165 297.65165 149.15164999999988 14915.16500 2020-01-01 2020-01-02 2020-01-01 00:03:37 2020-01-02 03:31:58 2020-01-01 00:03:37.000 2020-01-02 03:31:58.000 217 99118 49667.5 4966750 217 99118 49667.5 4966750 -32352 32583 5103.02 510302 -128 123 -1.62 -162 -218 2 10208 99119 0.65465 297.65465 149.15465 14915.46546 0.65465 297.65466 149.15465 14915.46531 0.65465 297.65465 149.1546500000002 14915.46500 2020-01-01 2020-01-02 2020-01-01 00:03:38 2020-01-02 03:31:59 2020-01-01 00:03:38.000 2020-01-02 03:31:59.000 218 99119 49668.5 4966850 218 99119 49668.5 4966850 -32351 32584 5104.02 510402 -127 124 -0.62 -62 -219 2 10209 99120 0.65765 297.65765 149.15765 14915.76576 0.65765 297.65765 149.15765 14915.76562 0.65765 297.65765 149.15764999999985 14915.76500 2020-01-01 2020-01-02 2020-01-01 00:03:39 2020-01-02 03:32:00 2020-01-01 00:03:39.000 2020-01-02 03:32:00.000 219 99120 49669.5 4966950 219 99120 49669.5 4966950 -32350 32585 5105.02 510502 -126 125 0.38 38 -22 2 10012 99922 0.06606 300.06606 150.06606 15156.67267 0.06606 300.06607 150.06606 15156.67287 0.06606 300.06606 150.0660599999999 15156.67206 2020-01-01 2020-01-02 2020-01-01 00:00:22 2020-01-02 03:45:22 2020-01-01 00:00:22.000 2020-01-02 03:45:22.000 22 99922 49972 5047172 22 99922 49972 5047172 -32547 32388 4551.009900990099 459652 -128 127 -1.2277227722772277 -124 -220 2 10210 99121 0.66066 297.66066 149.16066 14916.06606 0.66066 297.66068 149.16066 14916.06619 0.66066 297.66066 149.16065999999975 14916.06600 2020-01-01 2020-01-02 2020-01-01 00:03:40 2020-01-02 03:32:01 2020-01-01 00:03:40.000 2020-01-02 03:32:01.000 220 99121 49670.5 4967050 220 99121 49670.5 4967050 -32349 32586 5106.02 510602 -125 126 1.38 138 -221 2 10211 99122 0.66366 297.66366 149.16366 14916.36636 0.66366 297.66367 149.16366 14916.36651 0.66366 297.66366 149.1636600000001 14916.36600 2020-01-01 2020-01-02 2020-01-01 00:03:41 2020-01-02 03:32:02 2020-01-01 00:03:41.000 2020-01-02 03:32:02.000 221 99122 49671.5 4967150 221 99122 49671.5 4967150 -32348 32587 5107.02 510702 -124 127 2.38 238 -222 2 10212 99123 0.66666 297.66666 149.16666 14916.66666 0.66666 297.66666 149.16666 14916.6667 0.66666 297.66666 149.1666600000001 14916.66600 2020-01-01 2020-01-02 2020-01-01 00:03:42 2020-01-02 03:32:03 2020-01-01 00:03:42.000 2020-01-02 03:32:03.000 222 99123 49672.5 4967250 222 99123 49672.5 4967250 -32347 32588 5108.02 510802 -128 127 0.82 82 -223 2 10213 99124 0.66966 297.66966 149.16966 14916.96696 0.66966 297.66968 149.16966 14916.96678 0.66966 297.66966 149.16965999999974 14916.96600 2020-01-01 2020-01-02 2020-01-01 00:03:43 2020-01-02 03:32:04 2020-01-01 00:03:43.000 2020-01-02 03:32:04.000 223 99124 49673.5 4967350 223 99124 49673.5 4967350 -32346 32589 5109.02 510902 -128 127 -0.74 -74 -224 2 10214 99125 0.67267 297.67267 149.17267 14917.26726 0.67267 297.67267 149.17267 14917.26709 0.67267 297.67267 149.17266999999995 14917.26700 2020-01-01 2020-01-02 2020-01-01 00:03:44 2020-01-02 03:32:05 2020-01-01 00:03:44.000 2020-01-02 03:32:05.000 224 99125 49674.5 4967450 224 99125 49674.5 4967450 -32345 32590 5110.02 511002 -128 124 -2.3 -230 -225 2 10215 99126 0.67567 297.67567 149.17567 14917.56756 0.67567 297.6757 149.17567 14917.56767 0.67567 297.67567 149.17567000000003 14917.56700 2020-01-01 2020-01-02 2020-01-01 00:03:45 2020-01-02 03:32:06 2020-01-01 00:03:45.000 2020-01-02 03:32:06.000 225 99126 49675.5 4967550 225 99126 49675.5 4967550 -32344 32591 5111.02 511102 -127 125 -1.3 -130 -226 2 10216 99127 0.67867 297.67867 149.17867 14917.86786 0.67867 297.67868 149.17868 14917.86802 0.67867 297.67867 149.17866999999995 14917.86700 2020-01-01 2020-01-02 2020-01-01 00:03:46 2020-01-02 03:32:07 2020-01-01 00:03:46.000 2020-01-02 03:32:07.000 226 99127 49676.5 4967650 226 99127 49676.5 4967650 -32343 32592 5112.02 511202 -126 126 -0.3 -30 -227 2 10217 99128 0.68168 297.68168 149.18168 14918.16816 0.68168 297.68167 149.18168 14918.16817 0.68168 297.68168 149.18167999999986 14918.16800 2020-01-01 2020-01-02 2020-01-01 00:03:47 2020-01-02 03:32:08 2020-01-01 00:03:47.000 2020-01-02 03:32:08.000 227 99128 49677.5 4967750 227 99128 49677.5 4967750 -32342 32593 5113.02 511302 -125 127 0.7 70 -228 2 10218 99129 0.68468 297.68468 149.18468 14918.46846 0.68468 297.6847 149.18468 14918.46825 0.68468 297.68468 149.18468000000013 14918.46800 2020-01-01 2020-01-02 2020-01-01 00:03:48 2020-01-02 03:32:09 2020-01-01 00:03:48.000 2020-01-02 03:32:09.000 228 99129 49678.5 4967850 228 99129 49678.5 4967850 -32341 32594 5114.02 511402 -128 127 -0.86 -86 -229 2 10219 99130 0.68768 297.68768 149.18768 14918.76876 0.68768 297.68768 149.18768 14918.76855 0.68768 297.68768 149.18767999999983 14918.76800 2020-01-01 2020-01-02 2020-01-01 00:03:49 2020-01-02 03:32:10 2020-01-01 00:03:49.000 2020-01-02 03:32:10.000 229 99130 49679.5 4967950 229 99130 49679.5 4967950 -32340 32595 5115.02 511502 -128 127 -2.42 -242 -23 2 10013 99923 0.06906 300.06906 150.06906 15156.97597 0.06906 300.06906 150.06907 15156.97617 0.06906 300.06906 150.06905999999987 15156.97506 2020-01-01 2020-01-02 2020-01-01 00:00:23 2020-01-02 03:45:23 2020-01-01 00:00:23.000 2020-01-02 03:45:23.000 23 99923 49973 5047273 23 99923 49973 5047273 -32546 32389 4552.009900990099 459753 -128 127 -2.762376237623762 -279 -230 2 10220 99131 0.69069 297.69069 149.19069 14919.06906 0.69069 297.6907 149.19069 14919.06914 0.69069 297.69069 149.19068999999996 14919.06900 2020-01-01 2020-01-02 2020-01-01 00:03:50 2020-01-02 03:32:11 2020-01-01 00:03:50.000 2020-01-02 03:32:11.000 230 99131 49680.5 4968050 230 99131 49680.5 4968050 -32339 32596 5116.02 511602 -128 123 -3.98 -398 -231 2 10221 99132 0.69369 297.69369 149.19369 14919.36936 0.69369 297.6937 149.19369 14919.36949 0.69369 297.69369 149.19369000000003 14919.36900 2020-01-01 2020-01-02 2020-01-01 00:03:51 2020-01-02 03:32:12 2020-01-01 00:03:51.000 2020-01-02 03:32:12.000 231 99132 49681.5 4968150 231 99132 49681.5 4968150 -32338 32597 5117.02 511702 -127 124 -2.98 -298 -232 2 10222 99133 0.69669 297.69669 149.19669 14919.66966 0.69669 297.6967 149.19669 14919.66964 0.69669 297.69669 149.1966900000001 14919.66900 2020-01-01 2020-01-02 2020-01-01 00:03:52 2020-01-02 03:32:13 2020-01-01 00:03:52.000 2020-01-02 03:32:13.000 232 99133 49682.5 4968250 232 99133 49682.5 4968250 -32337 32598 5118.02 511802 -126 125 -1.98 -198 -233 2 10223 99134 0.69969 297.69969 149.19969 14919.96996 0.69969 297.6997 149.1997 14919.97037 0.69969 297.69969 149.19968999999998 14919.96900 2020-01-01 2020-01-02 2020-01-01 00:03:53 2020-01-02 03:32:14 2020-01-01 00:03:53.000 2020-01-02 03:32:14.000 233 99134 49683.5 4968350 233 99134 49683.5 4968350 -32336 32599 5119.02 511902 -125 126 -0.98 -98 -234 2 10224 99135 0.7027 297.7027 149.2027 14920.27027 0.7027 297.7027 149.2027 14920.27003 0.70270 297.70270 149.20269999999988 14920.27000 2020-01-01 2020-01-02 2020-01-01 00:03:54 2020-01-02 03:32:15 2020-01-01 00:03:54.000 2020-01-02 03:32:15.000 234 99135 49684.5 4968450 234 99135 49684.5 4968450 -32335 32600 5120.02 512002 -124 127 0.02 2 -235 2 10225 99136 0.7057 297.7057 149.2057 14920.57057 0.7057 297.70572 149.2057 14920.57065 0.70570 297.70570 149.20570000000023 14920.57000 2020-01-01 2020-01-02 2020-01-01 00:03:55 2020-01-02 03:32:16 2020-01-01 00:03:55.000 2020-01-02 03:32:16.000 235 99136 49685.5 4968550 235 99136 49685.5 4968550 -32334 32601 5121.02 512102 -128 127 -1.54 -154 -236 2 10226 99137 0.7087 297.7087 149.2087 14920.87087 0.7087 297.7087 149.2087 14920.87095 0.70870 297.70870 149.2086999999999 14920.87000 2020-01-01 2020-01-02 2020-01-01 00:03:56 2020-01-02 03:32:17 2020-01-01 00:03:56.000 2020-01-02 03:32:17.000 236 99137 49686.5 4968650 236 99137 49686.5 4968650 -32333 32602 5122.02 512202 -128 123 -3.1 -310 -237 2 10227 99138 0.71171 297.71171 149.21171 14921.17117 0.71171 297.7117 149.21171 14921.17111 0.71171 297.71171 149.21170999999978 14921.17100 2020-01-01 2020-01-02 2020-01-01 00:03:57 2020-01-02 03:32:18 2020-01-01 00:03:57.000 2020-01-02 03:32:18.000 237 99138 49687.5 4968750 237 99138 49687.5 4968750 -32332 32603 5123.02 512302 -127 124 -2.1 -210 -238 2 10228 99139 0.71471 297.71471 149.21471 14921.47147 0.71471 297.71472 149.21471 14921.47184 0.71471 297.71471 149.21471000000014 14921.47100 2020-01-01 2020-01-02 2020-01-01 00:03:58 2020-01-02 03:32:19 2020-01-01 00:03:58.000 2020-01-02 03:32:19.000 238 99139 49688.5 4968850 238 99139 49688.5 4968850 -32331 32604 5124.02 512402 -126 125 -1.1 -110 -239 2 10229 99140 0.71771 297.71771 149.21771 14921.77177 0.71771 297.7177 149.21771 14921.7715 0.71771 297.71771 149.2177100000001 14921.77100 2020-01-01 2020-01-02 2020-01-01 00:03:59 2020-01-02 03:32:20 2020-01-01 00:03:59.000 2020-01-02 03:32:20.000 239 99140 49689.5 4968950 239 99140 49689.5 4968950 -32330 32605 5125.02 512502 -125 126 -0.1 -10 -24 2 10014 99924 0.07207 300.07207 150.07207 15157.27927 0.07207 300.07208 150.07207 15157.27928 0.07207 300.07207 150.0720700000001 15157.27907 2020-01-01 2020-01-02 2020-01-01 00:00:24 2020-01-02 03:45:24 2020-01-01 00:00:24.000 2020-01-02 03:45:24.000 24 99924 49974 5047374 24 99924 49974 5047374 -32545 32390 4553.009900990099 459854 -128 123 -4.297029702970297 -434 -240 2 10230 99141 0.72072 297.72072 149.22072 14922.07207 0.72072 297.72073 149.22072 14922.07211 0.72072 297.72072 149.22071999999991 14922.07200 2020-01-01 2020-01-02 2020-01-01 00:04:00 2020-01-02 03:32:21 2020-01-01 00:04:00.000 2020-01-02 03:32:21.000 240 99141 49690.5 4969050 240 99141 49690.5 4969050 -32329 32606 5126.02 512602 -124 127 0.9 90 +215 2 10205 99116 0.64564 297.64564 149.14564 14914.56456 0.64564 297.64566 149.14564 14914.56473 0.64564 297.64564 149.14564000000001 14914.56400 2020-01-01 2020-01-02 2020-01-01 00:03:35 2020-01-02 03:31:56 2020-01-01 00:03:35.000 2020-01-02 03:31:56.000 215 99116 49665.5 4966550 215 99116 49665.5 4966550 -32354 32581 5101.02 510102 -124 127 1.5 150 +216 2 10206 99117 0.64864 297.64864 149.14864 14914.86486 0.64864 297.64865 149.14865 14914.86504 0.64864 297.64864 149.14864 14914.86400 2020-01-01 2020-01-02 2020-01-01 00:03:36 2020-01-02 03:31:57 2020-01-01 00:03:36.000 2020-01-02 03:31:57.000 216 99117 49666.5 4966650 216 99117 49666.5 4966650 -32353 32582 5102.02 510202 -128 127 -0.06 -6 +217 2 10207 99118 0.65165 297.65165 149.15165 14915.16516 0.65165 297.65164 149.15165 14915.16523 0.65165 297.65165 149.15165000000002 14915.16500 2020-01-01 2020-01-02 2020-01-01 00:03:37 2020-01-02 03:31:58 2020-01-01 00:03:37.000 2020-01-02 03:31:58.000 217 99118 49667.5 4966750 217 99118 49667.5 4966750 -32352 32583 5103.02 510302 -128 123 -1.62 -162 +218 2 10208 99119 0.65465 297.65465 149.15465 14915.46546 0.65465 297.65466 149.15465 14915.46531 0.65465 297.65465 149.15465 14915.46500 2020-01-01 2020-01-02 2020-01-01 00:03:38 2020-01-02 03:31:59 2020-01-01 00:03:38.000 2020-01-02 03:31:59.000 218 99119 49668.5 4966850 218 99119 49668.5 4966850 -32351 32584 5104.02 510402 -127 124 -0.62 -62 +219 2 10209 99120 0.65765 297.65765 149.15765 14915.76576 0.65765 297.65765 149.15765 14915.76562 0.65765 297.65765 149.15765 14915.76500 2020-01-01 2020-01-02 2020-01-01 00:03:39 2020-01-02 03:32:00 2020-01-01 00:03:39.000 2020-01-02 03:32:00.000 219 99120 49669.5 4966950 219 99120 49669.5 4966950 -32350 32585 5105.02 510502 -126 125 0.38 38 +22 2 10012 99922 0.06606 300.06606 150.06606 15156.67267 0.06606 300.06607 150.06606 15156.67287 0.06606 300.06606 150.06606000000002 15156.67206 2020-01-01 2020-01-02 2020-01-01 00:00:22 2020-01-02 03:45:22 2020-01-01 00:00:22.000 2020-01-02 03:45:22.000 22 99922 49972 5047172 22 99922 49972 5047172 -32547 32388 4551.009900990099 459652 -128 127 -1.2277227722772277 -124 +220 2 10210 99121 0.66066 297.66066 149.16066 14916.06606 0.66066 297.66068 149.16066 14916.06619 0.66066 297.66066 149.16066 14916.06600 2020-01-01 2020-01-02 2020-01-01 00:03:40 2020-01-02 03:32:01 2020-01-01 00:03:40.000 2020-01-02 03:32:01.000 220 99121 49670.5 4967050 220 99121 49670.5 4967050 -32349 32586 5106.02 510602 -125 126 1.38 138 +221 2 10211 99122 0.66366 297.66366 149.16366 14916.36636 0.66366 297.66367 149.16366 14916.36651 0.66366 297.66366 149.16366 14916.36600 2020-01-01 2020-01-02 2020-01-01 00:03:41 2020-01-02 03:32:02 2020-01-01 00:03:41.000 2020-01-02 03:32:02.000 221 99122 49671.5 4967150 221 99122 49671.5 4967150 -32348 32587 5107.02 510702 -124 127 2.38 238 +222 2 10212 99123 0.66666 297.66666 149.16666 14916.66666 0.66666 297.66666 149.16666 14916.6667 0.66666 297.66666 149.16665999999998 14916.66600 2020-01-01 2020-01-02 2020-01-01 00:03:42 2020-01-02 03:32:03 2020-01-01 00:03:42.000 2020-01-02 03:32:03.000 222 99123 49672.5 4967250 222 99123 49672.5 4967250 -32347 32588 5108.02 510802 -128 127 0.82 82 +223 2 10213 99124 0.66966 297.66966 149.16966 14916.96696 0.66966 297.66968 149.16966 14916.96678 0.66966 297.66966 149.16966 14916.96600 2020-01-01 2020-01-02 2020-01-01 00:03:43 2020-01-02 03:32:04 2020-01-01 00:03:43.000 2020-01-02 03:32:04.000 223 99124 49673.5 4967350 223 99124 49673.5 4967350 -32346 32589 5109.02 510902 -128 127 -0.74 -74 +224 2 10214 99125 0.67267 297.67267 149.17267 14917.26726 0.67267 297.67267 149.17267 14917.26709 0.67267 297.67267 149.17267 14917.26700 2020-01-01 2020-01-02 2020-01-01 00:03:44 2020-01-02 03:32:05 2020-01-01 00:03:44.000 2020-01-02 03:32:05.000 224 99125 49674.5 4967450 224 99125 49674.5 4967450 -32345 32590 5110.02 511002 -128 124 -2.3 -230 +225 2 10215 99126 0.67567 297.67567 149.17567 14917.56756 0.67567 297.6757 149.17567 14917.56767 0.67567 297.67567 149.17567 14917.56700 2020-01-01 2020-01-02 2020-01-01 00:03:45 2020-01-02 03:32:06 2020-01-01 00:03:45.000 2020-01-02 03:32:06.000 225 99126 49675.5 4967550 225 99126 49675.5 4967550 -32344 32591 5111.02 511102 -127 125 -1.3 -130 +226 2 10216 99127 0.67867 297.67867 149.17867 14917.86786 0.67867 297.67868 149.17868 14917.86802 0.67867 297.67867 149.17867 14917.86700 2020-01-01 2020-01-02 2020-01-01 00:03:46 2020-01-02 03:32:07 2020-01-01 00:03:46.000 2020-01-02 03:32:07.000 226 99127 49676.5 4967650 226 99127 49676.5 4967650 -32343 32592 5112.02 511202 -126 126 -0.3 -30 +227 2 10217 99128 0.68168 297.68168 149.18168 14918.16816 0.68168 297.68167 149.18168 14918.16817 0.68168 297.68168 149.18168 14918.16800 2020-01-01 2020-01-02 2020-01-01 00:03:47 2020-01-02 03:32:08 2020-01-01 00:03:47.000 2020-01-02 03:32:08.000 227 99128 49677.5 4967750 227 99128 49677.5 4967750 -32342 32593 5113.02 511302 -125 127 0.7 70 +228 2 10218 99129 0.68468 297.68468 149.18468 14918.46846 0.68468 297.6847 149.18468 14918.46825 0.68468 297.68468 149.18468000000001 14918.46800 2020-01-01 2020-01-02 2020-01-01 00:03:48 2020-01-02 03:32:09 2020-01-01 00:03:48.000 2020-01-02 03:32:09.000 228 99129 49678.5 4967850 228 99129 49678.5 4967850 -32341 32594 5114.02 511402 -128 127 -0.86 -86 +229 2 10219 99130 0.68768 297.68768 149.18768 14918.76876 0.68768 297.68768 149.18768 14918.76855 0.68768 297.68768 149.18768 14918.76800 2020-01-01 2020-01-02 2020-01-01 00:03:49 2020-01-02 03:32:10 2020-01-01 00:03:49.000 2020-01-02 03:32:10.000 229 99130 49679.5 4967950 229 99130 49679.5 4967950 -32340 32595 5115.02 511502 -128 127 -2.42 -242 +23 2 10013 99923 0.06906 300.06906 150.06906 15156.97597 0.06906 300.06906 150.06907 15156.97617 0.06906 300.06906 150.06906 15156.97506 2020-01-01 2020-01-02 2020-01-01 00:00:23 2020-01-02 03:45:23 2020-01-01 00:00:23.000 2020-01-02 03:45:23.000 23 99923 49973 5047273 23 99923 49973 5047273 -32546 32389 4552.009900990099 459753 -128 127 -2.762376237623762 -279 +230 2 10220 99131 0.69069 297.69069 149.19069 14919.06906 0.69069 297.6907 149.19069 14919.06914 0.69069 297.69069 149.19069 14919.06900 2020-01-01 2020-01-02 2020-01-01 00:03:50 2020-01-02 03:32:11 2020-01-01 00:03:50.000 2020-01-02 03:32:11.000 230 99131 49680.5 4968050 230 99131 49680.5 4968050 -32339 32596 5116.02 511602 -128 123 -3.98 -398 +231 2 10221 99132 0.69369 297.69369 149.19369 14919.36936 0.69369 297.6937 149.19369 14919.36949 0.69369 297.69369 149.19369 14919.36900 2020-01-01 2020-01-02 2020-01-01 00:03:51 2020-01-02 03:32:12 2020-01-01 00:03:51.000 2020-01-02 03:32:12.000 231 99132 49681.5 4968150 231 99132 49681.5 4968150 -32338 32597 5117.02 511702 -127 124 -2.98 -298 +232 2 10222 99133 0.69669 297.69669 149.19669 14919.66966 0.69669 297.6967 149.19669 14919.66964 0.69669 297.69669 149.19669 14919.66900 2020-01-01 2020-01-02 2020-01-01 00:03:52 2020-01-02 03:32:13 2020-01-01 00:03:52.000 2020-01-02 03:32:13.000 232 99133 49682.5 4968250 232 99133 49682.5 4968250 -32337 32598 5118.02 511802 -126 125 -1.98 -198 +233 2 10223 99134 0.69969 297.69969 149.19969 14919.96996 0.69969 297.6997 149.1997 14919.97037 0.69969 297.69969 149.19969 14919.96900 2020-01-01 2020-01-02 2020-01-01 00:03:53 2020-01-02 03:32:14 2020-01-01 00:03:53.000 2020-01-02 03:32:14.000 233 99134 49683.5 4968350 233 99134 49683.5 4968350 -32336 32599 5119.02 511902 -125 126 -0.98 -98 +234 2 10224 99135 0.7027 297.7027 149.2027 14920.27027 0.7027 297.7027 149.2027 14920.27003 0.70270 297.70270 149.2027 14920.27000 2020-01-01 2020-01-02 2020-01-01 00:03:54 2020-01-02 03:32:15 2020-01-01 00:03:54.000 2020-01-02 03:32:15.000 234 99135 49684.5 4968450 234 99135 49684.5 4968450 -32335 32600 5120.02 512002 -124 127 0.02 2 +235 2 10225 99136 0.7057 297.7057 149.2057 14920.57057 0.7057 297.70572 149.2057 14920.57065 0.70570 297.70570 149.2057 14920.57000 2020-01-01 2020-01-02 2020-01-01 00:03:55 2020-01-02 03:32:16 2020-01-01 00:03:55.000 2020-01-02 03:32:16.000 235 99136 49685.5 4968550 235 99136 49685.5 4968550 -32334 32601 5121.02 512102 -128 127 -1.54 -154 +236 2 10226 99137 0.7087 297.7087 149.2087 14920.87087 0.7087 297.7087 149.2087 14920.87095 0.70870 297.70870 149.20870000000002 14920.87000 2020-01-01 2020-01-02 2020-01-01 00:03:56 2020-01-02 03:32:17 2020-01-01 00:03:56.000 2020-01-02 03:32:17.000 236 99137 49686.5 4968650 236 99137 49686.5 4968650 -32333 32602 5122.02 512202 -128 123 -3.1 -310 +237 2 10227 99138 0.71171 297.71171 149.21171 14921.17117 0.71171 297.7117 149.21171 14921.17111 0.71171 297.71171 149.21171 14921.17100 2020-01-01 2020-01-02 2020-01-01 00:03:57 2020-01-02 03:32:18 2020-01-01 00:03:57.000 2020-01-02 03:32:18.000 237 99138 49687.5 4968750 237 99138 49687.5 4968750 -32332 32603 5123.02 512302 -127 124 -2.1 -210 +238 2 10228 99139 0.71471 297.71471 149.21471 14921.47147 0.71471 297.71472 149.21471 14921.47184 0.71471 297.71471 149.21471 14921.47100 2020-01-01 2020-01-02 2020-01-01 00:03:58 2020-01-02 03:32:19 2020-01-01 00:03:58.000 2020-01-02 03:32:19.000 238 99139 49688.5 4968850 238 99139 49688.5 4968850 -32331 32604 5124.02 512402 -126 125 -1.1 -110 +239 2 10229 99140 0.71771 297.71771 149.21771 14921.77177 0.71771 297.7177 149.21771 14921.7715 0.71771 297.71771 149.21771 14921.77100 2020-01-01 2020-01-02 2020-01-01 00:03:59 2020-01-02 03:32:20 2020-01-01 00:03:59.000 2020-01-02 03:32:20.000 239 99140 49689.5 4968950 239 99140 49689.5 4968950 -32330 32605 5125.02 512502 -125 126 -0.1 -10 +24 2 10014 99924 0.07207 300.07207 150.07207 15157.27927 0.07207 300.07208 150.07207 15157.27928 0.07207 300.07207 150.07207 15157.27907 2020-01-01 2020-01-02 2020-01-01 00:00:24 2020-01-02 03:45:24 2020-01-01 00:00:24.000 2020-01-02 03:45:24.000 24 99924 49974 5047374 24 99924 49974 5047374 -32545 32390 4553.009900990099 459854 -128 123 -4.297029702970297 -434 +240 2 10230 99141 0.72072 297.72072 149.22072 14922.07207 0.72072 297.72073 149.22072 14922.07211 0.72072 297.72072 149.22072 14922.07200 2020-01-01 2020-01-02 2020-01-01 00:04:00 2020-01-02 03:32:21 2020-01-01 00:04:00.000 2020-01-02 03:32:21.000 240 99141 49690.5 4969050 240 99141 49690.5 4969050 -32329 32606 5126.02 512602 -124 127 0.9 90 241 2 10231 99142 0.72372 297.72372 149.22372 14922.37237 0.72372 297.72372 149.22372 14922.37243 0.72372 297.72372 149.22372 14922.37200 2020-01-01 2020-01-02 2020-01-01 00:04:01 2020-01-02 03:32:22 2020-01-01 00:04:01.000 2020-01-02 03:32:22.000 241 99142 49691.5 4969150 241 99142 49691.5 4969150 -32328 32607 5127.02 512702 -128 127 -0.66 -66 -242 2 10232 99143 0.72672 297.72672 149.22672 14922.67267 0.72672 297.7267 149.22672 14922.67273 0.72672 297.72672 149.22672000000006 14922.67200 2020-01-01 2020-01-02 2020-01-01 00:04:02 2020-01-02 03:32:23 2020-01-01 00:04:02.000 2020-01-02 03:32:23.000 242 99143 49692.5 4969250 242 99143 49692.5 4969250 -32327 32608 5128.02 512802 -128 123 -2.22 -222 -243 2 10233 99144 0.72972 297.72972 149.22972 14922.97297 0.72972 297.72974 149.22973 14922.97332 0.72972 297.72972 149.2297199999999 14922.97200 2020-01-01 2020-01-02 2020-01-01 00:04:03 2020-01-02 03:32:24 2020-01-01 00:04:03.000 2020-01-02 03:32:24.000 243 99144 49693.5 4969350 243 99144 49693.5 4969350 -32326 32609 5129.02 512902 -127 124 -1.22 -122 -244 2 10234 99145 0.73273 297.73273 149.23273 14923.27327 0.73273 297.73273 149.23272 14923.27297 0.73273 297.73273 149.23272999999986 14923.27300 2020-01-01 2020-01-02 2020-01-01 00:04:04 2020-01-02 03:32:25 2020-01-01 00:04:04.000 2020-01-02 03:32:25.000 244 99145 49694.5 4969450 244 99145 49694.5 4969450 -32325 32610 5130.02 513002 -126 125 -0.22 -22 -245 2 10235 99146 0.73573 297.73573 149.23573 14923.57357 0.73573 297.73575 149.23573 14923.57358 0.73573 297.73573 149.23573000000016 14923.57300 2020-01-01 2020-01-02 2020-01-01 00:04:05 2020-01-02 03:32:26 2020-01-01 00:04:05.000 2020-01-02 03:32:26.000 245 99146 49695.5 4969550 245 99146 49695.5 4969550 -32324 32611 5131.02 513102 -125 126 0.78 78 -246 2 10236 99147 0.73873 297.73873 149.23873 14923.87387 0.73873 297.73874 149.23873 14923.8739 0.73873 297.73873 149.2387299999998 14923.87300 2020-01-01 2020-01-02 2020-01-01 00:04:06 2020-01-02 03:32:27 2020-01-01 00:04:06.000 2020-01-02 03:32:27.000 246 99147 49696.5 4969650 246 99147 49696.5 4969650 -32323 32612 5132.02 513202 -124 127 1.78 178 -247 2 10237 99148 0.74174 297.74174 149.24174 14924.17417 0.74174 297.74173 149.24174 14924.1742 0.74174 297.74174 149.24174 14924.17400 2020-01-01 2020-01-02 2020-01-01 00:04:07 2020-01-02 03:32:28 2020-01-01 00:04:07.000 2020-01-02 03:32:28.000 247 99148 49697.5 4969750 247 99148 49697.5 4969750 -32322 32613 5133.02 513302 -128 127 0.22 22 -248 2 10238 99149 0.74474 297.74474 149.24474 14924.47447 0.74474 297.74475 149.24474 14924.47478 0.74474 297.74474 149.24474000000006 14924.47400 2020-01-01 2020-01-02 2020-01-01 00:04:08 2020-01-02 03:32:29 2020-01-01 00:04:08.000 2020-01-02 03:32:29.000 248 99149 49698.5 4969850 248 99149 49698.5 4969850 -32321 32614 5134.02 513402 -128 127 -1.34 -134 -249 2 10239 99150 0.74774 297.74774 149.24774 14924.77477 0.74774 297.74774 149.24774 14924.77447 0.74774 297.74774 149.24774000000005 14924.77400 2020-01-01 2020-01-02 2020-01-01 00:04:09 2020-01-02 03:32:30 2020-01-01 00:04:09.000 2020-01-02 03:32:30.000 249 99150 49699.5 4969950 249 99150 49699.5 4969950 -32320 32615 5135.02 513502 -128 124 -2.9 -290 -25 2 10015 99925 0.07507 300.07507 150.07507 15157.58258 0.07507 300.07507 150.07507 15157.58241 0.07507 300.07507 150.07507000000012 15157.58207 2020-01-01 2020-01-02 2020-01-01 00:00:25 2020-01-02 03:45:25 2020-01-01 00:00:25.000 2020-01-02 03:45:25.000 25 99925 49975 5047475 25 99925 49975 5047475 -32544 32391 4554.009900990099 459955 -127 124 -3.297029702970297 -333 -250 2 10240 99151 0.75075 297.75075 149.25075 14925.07507 0.75075 297.75076 149.25075 14925.07506 0.75075 297.75075 149.25074999999987 14925.07500 2020-01-01 2020-01-02 2020-01-01 00:04:10 2020-01-02 03:32:31 2020-01-01 00:04:10.000 2020-01-02 03:32:31.000 250 99151 49700.5 4970050 250 99151 49700.5 4970050 -32319 32616 5136.02 513602 -127 125 -1.9 -190 -251 2 10241 99152 0.75375 297.75375 149.25375 14925.37537 0.75375 297.75375 149.25375 14925.37536 0.75375 297.75375 149.2537499999999 14925.37500 2020-01-01 2020-01-02 2020-01-01 00:04:11 2020-01-02 03:32:32 2020-01-01 00:04:11.000 2020-01-02 03:32:32.000 251 99152 49701.5 4970150 251 99152 49701.5 4970150 -32318 32617 5137.02 513702 -126 126 -0.9 -90 -252 2 10242 99153 0.75675 297.75675 149.25675 14925.67567 0.75675 297.75674 149.25675 14925.67567 0.75675 297.75675 149.2567500000001 14925.67500 2020-01-01 2020-01-02 2020-01-01 00:04:12 2020-01-02 03:32:33 2020-01-01 00:04:12.000 2020-01-02 03:32:33.000 252 99153 49702.5 4970250 252 99153 49702.5 4970250 -32317 32618 5138.02 513802 -125 127 0.1 10 -253 2 10243 99154 0.75975 297.75975 149.25975 14925.97597 0.75975 297.75977 149.25976 14925.97625 0.75975 297.75975 149.25974999999988 14925.97500 2020-01-01 2020-01-02 2020-01-01 00:04:13 2020-01-02 03:32:34 2020-01-01 00:04:13.000 2020-01-02 03:32:34.000 253 99154 49703.5 4970350 253 99154 49703.5 4970350 -32316 32619 5139.02 513902 -128 127 -1.46 -146 -254 2 10244 99155 0.76276 297.76276 149.26276 14926.27627 0.76276 297.76276 149.26275 14926.27594 0.76276 297.76276 149.2627599999998 14926.27600 2020-01-01 2020-01-02 2020-01-01 00:04:14 2020-01-02 03:32:35 2020-01-01 00:04:14.000 2020-01-02 03:32:35.000 254 99155 49704.5 4970450 254 99155 49704.5 4970450 -32315 32620 5140.02 514002 -128 127 -3.02 -302 -255 2 10245 99156 0.76576 297.76576 149.26576 14926.57657 0.76576 297.76578 149.26576 14926.57652 0.76576 297.76576 149.26576000000017 14926.57600 2020-01-01 2020-01-02 2020-01-01 00:04:15 2020-01-02 03:32:36 2020-01-01 00:04:15.000 2020-01-02 03:32:36.000 255 99156 49705.5 4970550 255 99156 49705.5 4970550 -32314 32621 5141.02 514102 -128 123 -4.58 -458 -256 2 10246 99157 0.76876 297.76876 149.26876 14926.87687 0.76876 297.76877 149.26876 14926.87683 0.76876 297.76876 149.26876000000016 14926.87600 2020-01-01 2020-01-02 2020-01-01 00:04:16 2020-01-02 03:32:37 2020-01-01 00:04:16.000 2020-01-02 03:32:37.000 256 99157 49706.5 4970650 256 99157 49706.5 4970650 -32313 32622 5142.02 514202 -127 124 -3.58 -358 -257 2 10247 99158 0.77177 297.77177 149.27177 14927.17717 0.77177 297.77176 149.27177 14927.17714 0.77177 297.77177 149.27176999999992 14927.17700 2020-01-01 2020-01-02 2020-01-01 00:04:17 2020-01-02 03:32:38 2020-01-01 00:04:17.000 2020-01-02 03:32:38.000 257 99158 49707.5 4970750 257 99158 49707.5 4970750 -32312 32623 5143.02 514302 -126 125 -2.58 -258 -258 2 10248 99159 0.77477 297.77477 149.27477 14927.47747 0.77477 297.77478 149.27477 14927.47776 0.77477 297.77477 149.27477 14927.47700 2020-01-01 2020-01-02 2020-01-01 00:04:18 2020-01-02 03:32:39 2020-01-01 00:04:18.000 2020-01-02 03:32:39.000 258 99159 49708.5 4970850 258 99159 49708.5 4970850 -32311 32624 5144.02 514402 -125 126 -1.58 -158 -259 2 10249 99160 0.77777 297.77777 149.27777 14927.77777 0.77777 297.77777 149.27777 14927.77742 0.77777 297.77777 149.27777000000006 14927.77700 2020-01-01 2020-01-02 2020-01-01 00:04:19 2020-01-02 03:32:40 2020-01-01 00:04:19.000 2020-01-02 03:32:40.000 259 99160 49709.5 4970950 259 99160 49709.5 4970950 -32310 32625 5145.02 514502 -124 127 -0.58 -58 -26 2 10016 99926 0.07807 300.07807 150.07807 15157.88588 0.07807 300.07806 150.07807 15157.88575 0.07807 300.07807 150.0780699999998 15157.88507 2020-01-01 2020-01-02 2020-01-01 00:00:26 2020-01-02 03:45:26 2020-01-01 00:00:26.000 2020-01-02 03:45:26.000 26 99926 49976 5047576 26 99926 49976 5047576 -32543 32392 4555.009900990099 460056 -126 125 -2.297029702970297 -232 -260 2 10250 99161 0.78078 297.78078 149.28078 14928.07807 0.78078 297.7808 149.28077 14928.07799 0.78078 297.78078 149.28077999999982 14928.07800 2020-01-01 2020-01-02 2020-01-01 00:04:20 2020-01-02 03:32:41 2020-01-01 00:04:20.000 2020-01-02 03:32:41.000 260 99161 49710.5 4971050 260 99161 49710.5 4971050 -32309 32626 5146.02 514602 -128 127 -2.14 -214 -261 2 10251 99162 0.78378 297.78378 149.28378 14928.37837 0.78378 297.78378 149.28378 14928.3783 0.78378 297.78378 149.28377999999987 14928.37800 2020-01-01 2020-01-02 2020-01-01 00:04:21 2020-01-02 03:32:42 2020-01-01 00:04:21.000 2020-01-02 03:32:42.000 261 99162 49711.5 4971150 261 99162 49711.5 4971150 -32308 32627 5147.02 514702 -128 123 -3.7 -370 -262 2 10252 99163 0.78678 297.78678 149.28678 14928.67867 0.78678 297.78677 149.28678 14928.67861 0.78678 297.78678 149.2867800000002 14928.67800 2020-01-01 2020-01-02 2020-01-01 00:04:22 2020-01-02 03:32:43 2020-01-01 00:04:22.000 2020-01-02 03:32:43.000 262 99163 49712.5 4971250 262 99163 49712.5 4971250 -32307 32628 5148.02 514802 -127 124 -2.7 -270 -263 2 10253 99164 0.78978 297.78978 149.28978 14928.97897 0.78978 297.7898 149.28979 14928.97923 0.78978 297.78978 149.2897799999999 14928.97800 2020-01-01 2020-01-02 2020-01-01 00:04:23 2020-01-02 03:32:44 2020-01-01 00:04:23.000 2020-01-02 03:32:44.000 263 99164 49713.5 4971350 263 99164 49713.5 4971350 -32306 32629 5149.02 514902 -126 125 -1.7 -170 +242 2 10232 99143 0.72672 297.72672 149.22672 14922.67267 0.72672 297.7267 149.22672 14922.67273 0.72672 297.72672 149.22672 14922.67200 2020-01-01 2020-01-02 2020-01-01 00:04:02 2020-01-02 03:32:23 2020-01-01 00:04:02.000 2020-01-02 03:32:23.000 242 99143 49692.5 4969250 242 99143 49692.5 4969250 -32327 32608 5128.02 512802 -128 123 -2.22 -222 +243 2 10233 99144 0.72972 297.72972 149.22972 14922.97297 0.72972 297.72974 149.22973 14922.97332 0.72972 297.72972 149.22972 14922.97200 2020-01-01 2020-01-02 2020-01-01 00:04:03 2020-01-02 03:32:24 2020-01-01 00:04:03.000 2020-01-02 03:32:24.000 243 99144 49693.5 4969350 243 99144 49693.5 4969350 -32326 32609 5129.02 512902 -127 124 -1.22 -122 +244 2 10234 99145 0.73273 297.73273 149.23273 14923.27327 0.73273 297.73273 149.23272 14923.27297 0.73273 297.73273 149.23273 14923.27300 2020-01-01 2020-01-02 2020-01-01 00:04:04 2020-01-02 03:32:25 2020-01-01 00:04:04.000 2020-01-02 03:32:25.000 244 99145 49694.5 4969450 244 99145 49694.5 4969450 -32325 32610 5130.02 513002 -126 125 -0.22 -22 +245 2 10235 99146 0.73573 297.73573 149.23573 14923.57357 0.73573 297.73575 149.23573 14923.57358 0.73573 297.73573 149.23573 14923.57300 2020-01-01 2020-01-02 2020-01-01 00:04:05 2020-01-02 03:32:26 2020-01-01 00:04:05.000 2020-01-02 03:32:26.000 245 99146 49695.5 4969550 245 99146 49695.5 4969550 -32324 32611 5131.02 513102 -125 126 0.78 78 +246 2 10236 99147 0.73873 297.73873 149.23873 14923.87387 0.73873 297.73874 149.23873 14923.8739 0.73873 297.73873 149.23873 14923.87300 2020-01-01 2020-01-02 2020-01-01 00:04:06 2020-01-02 03:32:27 2020-01-01 00:04:06.000 2020-01-02 03:32:27.000 246 99147 49696.5 4969650 246 99147 49696.5 4969650 -32323 32612 5132.02 513202 -124 127 1.78 178 +247 2 10237 99148 0.74174 297.74174 149.24174 14924.17417 0.74174 297.74173 149.24174 14924.1742 0.74174 297.74174 149.24174000000002 14924.17400 2020-01-01 2020-01-02 2020-01-01 00:04:07 2020-01-02 03:32:28 2020-01-01 00:04:07.000 2020-01-02 03:32:28.000 247 99148 49697.5 4969750 247 99148 49697.5 4969750 -32322 32613 5133.02 513302 -128 127 0.22 22 +248 2 10238 99149 0.74474 297.74474 149.24474 14924.47447 0.74474 297.74475 149.24474 14924.47478 0.74474 297.74474 149.24474 14924.47400 2020-01-01 2020-01-02 2020-01-01 00:04:08 2020-01-02 03:32:29 2020-01-01 00:04:08.000 2020-01-02 03:32:29.000 248 99149 49698.5 4969850 248 99149 49698.5 4969850 -32321 32614 5134.02 513402 -128 127 -1.34 -134 +249 2 10239 99150 0.74774 297.74774 149.24774 14924.77477 0.74774 297.74774 149.24774 14924.77447 0.74774 297.74774 149.24774 14924.77400 2020-01-01 2020-01-02 2020-01-01 00:04:09 2020-01-02 03:32:30 2020-01-01 00:04:09.000 2020-01-02 03:32:30.000 249 99150 49699.5 4969950 249 99150 49699.5 4969950 -32320 32615 5135.02 513502 -128 124 -2.9 -290 +25 2 10015 99925 0.07507 300.07507 150.07507 15157.58258 0.07507 300.07507 150.07507 15157.58241 0.07507 300.07507 150.07507 15157.58207 2020-01-01 2020-01-02 2020-01-01 00:00:25 2020-01-02 03:45:25 2020-01-01 00:00:25.000 2020-01-02 03:45:25.000 25 99925 49975 5047475 25 99925 49975 5047475 -32544 32391 4554.009900990099 459955 -127 124 -3.297029702970297 -333 +250 2 10240 99151 0.75075 297.75075 149.25075 14925.07507 0.75075 297.75076 149.25075 14925.07506 0.75075 297.75075 149.25075 14925.07500 2020-01-01 2020-01-02 2020-01-01 00:04:10 2020-01-02 03:32:31 2020-01-01 00:04:10.000 2020-01-02 03:32:31.000 250 99151 49700.5 4970050 250 99151 49700.5 4970050 -32319 32616 5136.02 513602 -127 125 -1.9 -190 +251 2 10241 99152 0.75375 297.75375 149.25375 14925.37537 0.75375 297.75375 149.25375 14925.37536 0.75375 297.75375 149.25375 14925.37500 2020-01-01 2020-01-02 2020-01-01 00:04:11 2020-01-02 03:32:32 2020-01-01 00:04:11.000 2020-01-02 03:32:32.000 251 99152 49701.5 4970150 251 99152 49701.5 4970150 -32318 32617 5137.02 513702 -126 126 -0.9 -90 +252 2 10242 99153 0.75675 297.75675 149.25675 14925.67567 0.75675 297.75674 149.25675 14925.67567 0.75675 297.75675 149.25674999999998 14925.67500 2020-01-01 2020-01-02 2020-01-01 00:04:12 2020-01-02 03:32:33 2020-01-01 00:04:12.000 2020-01-02 03:32:33.000 252 99153 49702.5 4970250 252 99153 49702.5 4970250 -32317 32618 5138.02 513802 -125 127 0.1 10 +253 2 10243 99154 0.75975 297.75975 149.25975 14925.97597 0.75975 297.75977 149.25976 14925.97625 0.75975 297.75975 149.25975 14925.97500 2020-01-01 2020-01-02 2020-01-01 00:04:13 2020-01-02 03:32:34 2020-01-01 00:04:13.000 2020-01-02 03:32:34.000 253 99154 49703.5 4970350 253 99154 49703.5 4970350 -32316 32619 5139.02 513902 -128 127 -1.46 -146 +254 2 10244 99155 0.76276 297.76276 149.26276 14926.27627 0.76276 297.76276 149.26275 14926.27594 0.76276 297.76276 149.26276 14926.27600 2020-01-01 2020-01-02 2020-01-01 00:04:14 2020-01-02 03:32:35 2020-01-01 00:04:14.000 2020-01-02 03:32:35.000 254 99155 49704.5 4970450 254 99155 49704.5 4970450 -32315 32620 5140.02 514002 -128 127 -3.02 -302 +255 2 10245 99156 0.76576 297.76576 149.26576 14926.57657 0.76576 297.76578 149.26576 14926.57652 0.76576 297.76576 149.26576 14926.57600 2020-01-01 2020-01-02 2020-01-01 00:04:15 2020-01-02 03:32:36 2020-01-01 00:04:15.000 2020-01-02 03:32:36.000 255 99156 49705.5 4970550 255 99156 49705.5 4970550 -32314 32621 5141.02 514102 -128 123 -4.58 -458 +256 2 10246 99157 0.76876 297.76876 149.26876 14926.87687 0.76876 297.76877 149.26876 14926.87683 0.76876 297.76876 149.26876000000001 14926.87600 2020-01-01 2020-01-02 2020-01-01 00:04:16 2020-01-02 03:32:37 2020-01-01 00:04:16.000 2020-01-02 03:32:37.000 256 99157 49706.5 4970650 256 99157 49706.5 4970650 -32313 32622 5142.02 514202 -127 124 -3.58 -358 +257 2 10247 99158 0.77177 297.77177 149.27177 14927.17717 0.77177 297.77176 149.27177 14927.17714 0.77177 297.77177 149.27177 14927.17700 2020-01-01 2020-01-02 2020-01-01 00:04:17 2020-01-02 03:32:38 2020-01-01 00:04:17.000 2020-01-02 03:32:38.000 257 99158 49707.5 4970750 257 99158 49707.5 4970750 -32312 32623 5143.02 514302 -126 125 -2.58 -258 +258 2 10248 99159 0.77477 297.77477 149.27477 14927.47747 0.77477 297.77478 149.27477 14927.47776 0.77477 297.77477 149.27477000000002 14927.47700 2020-01-01 2020-01-02 2020-01-01 00:04:18 2020-01-02 03:32:39 2020-01-01 00:04:18.000 2020-01-02 03:32:39.000 258 99159 49708.5 4970850 258 99159 49708.5 4970850 -32311 32624 5144.02 514402 -125 126 -1.58 -158 +259 2 10249 99160 0.77777 297.77777 149.27777 14927.77777 0.77777 297.77777 149.27777 14927.77742 0.77777 297.77777 149.27777 14927.77700 2020-01-01 2020-01-02 2020-01-01 00:04:19 2020-01-02 03:32:40 2020-01-01 00:04:19.000 2020-01-02 03:32:40.000 259 99160 49709.5 4970950 259 99160 49709.5 4970950 -32310 32625 5145.02 514502 -124 127 -0.58 -58 +26 2 10016 99926 0.07807 300.07807 150.07807 15157.88588 0.07807 300.07806 150.07807 15157.88575 0.07807 300.07807 150.07807 15157.88507 2020-01-01 2020-01-02 2020-01-01 00:00:26 2020-01-02 03:45:26 2020-01-01 00:00:26.000 2020-01-02 03:45:26.000 26 99926 49976 5047576 26 99926 49976 5047576 -32543 32392 4555.009900990099 460056 -126 125 -2.297029702970297 -232 +260 2 10250 99161 0.78078 297.78078 149.28078 14928.07807 0.78078 297.7808 149.28077 14928.07799 0.78078 297.78078 149.28078 14928.07800 2020-01-01 2020-01-02 2020-01-01 00:04:20 2020-01-02 03:32:41 2020-01-01 00:04:20.000 2020-01-02 03:32:41.000 260 99161 49710.5 4971050 260 99161 49710.5 4971050 -32309 32626 5146.02 514602 -128 127 -2.14 -214 +261 2 10251 99162 0.78378 297.78378 149.28378 14928.37837 0.78378 297.78378 149.28378 14928.3783 0.78378 297.78378 149.28378 14928.37800 2020-01-01 2020-01-02 2020-01-01 00:04:21 2020-01-02 03:32:42 2020-01-01 00:04:21.000 2020-01-02 03:32:42.000 261 99162 49711.5 4971150 261 99162 49711.5 4971150 -32308 32627 5147.02 514702 -128 123 -3.7 -370 +262 2 10252 99163 0.78678 297.78678 149.28678 14928.67867 0.78678 297.78677 149.28678 14928.67861 0.78678 297.78678 149.28678 14928.67800 2020-01-01 2020-01-02 2020-01-01 00:04:22 2020-01-02 03:32:43 2020-01-01 00:04:22.000 2020-01-02 03:32:43.000 262 99163 49712.5 4971250 262 99163 49712.5 4971250 -32307 32628 5148.02 514802 -127 124 -2.7 -270 +263 2 10253 99164 0.78978 297.78978 149.28978 14928.97897 0.78978 297.7898 149.28979 14928.97923 0.78978 297.78978 149.28977999999998 14928.97800 2020-01-01 2020-01-02 2020-01-01 00:04:23 2020-01-02 03:32:44 2020-01-01 00:04:23.000 2020-01-02 03:32:44.000 263 99164 49713.5 4971350 263 99164 49713.5 4971350 -32306 32629 5149.02 514902 -126 125 -1.7 -170 264 2 10254 99165 0.79279 297.79279 149.29279 14929.27927 0.79279 297.7928 149.29278 14929.27888 0.79279 297.79279 149.29279 14929.27900 2020-01-01 2020-01-02 2020-01-01 00:04:24 2020-01-02 03:32:45 2020-01-01 00:04:24.000 2020-01-02 03:32:45.000 264 99165 49714.5 4971450 264 99165 49714.5 4971450 -32305 32630 5150.02 515002 -125 126 -0.7 -70 -265 2 10255 99166 0.79579 297.79579 149.29579 14929.57957 0.79579 297.7958 149.29579 14929.57962 0.79579 297.79579 149.2957900000001 14929.57900 2020-01-01 2020-01-02 2020-01-01 00:04:25 2020-01-02 03:32:46 2020-01-01 00:04:25.000 2020-01-02 03:32:46.000 265 99166 49715.5 4971550 265 99166 49715.5 4971550 -32304 32631 5151.02 515102 -124 127 0.3 30 -266 2 10256 99167 0.79879 297.79879 149.29879 14929.87987 0.79879 297.7988 149.29879 14929.87977 0.79879 297.79879 149.2987900000001 14929.87900 2020-01-01 2020-01-02 2020-01-01 00:04:26 2020-01-02 03:32:47 2020-01-01 00:04:26.000 2020-01-02 03:32:47.000 266 99167 49716.5 4971650 266 99167 49716.5 4971650 -32303 32632 5152.02 515202 -128 127 -1.26 -126 -267 2 10257 99168 0.8018 297.8018 149.3018 14930.18018 0.8018 297.8018 149.3018 14930.18012 0.80180 297.80180 149.3017999999999 14930.18000 2020-01-01 2020-01-02 2020-01-01 00:04:27 2020-01-02 03:32:48 2020-01-01 00:04:27.000 2020-01-02 03:32:48.000 267 99168 49717.5 4971750 267 99168 49717.5 4971750 -32302 32633 5153.02 515302 -128 123 -2.82 -282 -268 2 10258 99169 0.8048 297.8048 149.3048 14930.48048 0.8048 297.8048 149.3048 14930.4807 0.80480 297.80480 149.30479999999991 14930.48000 2020-01-01 2020-01-02 2020-01-01 00:04:28 2020-01-02 03:32:49 2020-01-01 00:04:28.000 2020-01-02 03:32:49.000 268 99169 49718.5 4971850 268 99169 49718.5 4971850 -32301 32634 5154.02 515402 -127 124 -1.82 -182 -269 2 10259 99170 0.8078 297.8078 149.3078 14930.78078 0.8078 297.8078 149.3078 14930.78035 0.80780 297.80780 149.3078 14930.78000 2020-01-01 2020-01-02 2020-01-01 00:04:29 2020-01-02 03:32:50 2020-01-01 00:04:29.000 2020-01-02 03:32:50.000 269 99170 49719.5 4971950 269 99170 49719.5 4971950 -32300 32635 5155.02 515502 -126 125 -0.82 -82 -27 2 10017 99927 0.08108 300.08108 150.08108 15158.18918 0.08108 300.0811 150.08108 15158.18936 0.08108 300.08108 150.08107999999996 15158.18908 2020-01-01 2020-01-02 2020-01-01 00:00:27 2020-01-02 03:45:27 2020-01-01 00:00:27.000 2020-01-02 03:45:27.000 27 99927 49977 5047677 27 99927 49977 5047677 -32542 32393 4556.009900990099 460157 -125 126 -1.297029702970297 -131 -270 2 10260 99171 0.81081 297.81081 149.31081 14931.08108 0.81081 297.81082 149.31081 14931.08109 0.81081 297.81081 149.31081000000017 14931.08100 2020-01-01 2020-01-02 2020-01-01 00:04:30 2020-01-02 03:32:51 2020-01-01 00:04:30.000 2020-01-02 03:32:51.000 270 99171 49720.5 4972050 270 99171 49720.5 4972050 -32299 32636 5156.02 515602 -125 126 0.18 18 -271 2 10261 99172 0.81381 297.81381 149.31381 14931.38138 0.81381 297.8138 149.31381 14931.38124 0.81381 297.81381 149.31380999999985 14931.38100 2020-01-01 2020-01-02 2020-01-01 00:04:31 2020-01-02 03:32:52 2020-01-01 00:04:31.000 2020-01-02 03:32:52.000 271 99172 49721.5 4972150 271 99172 49721.5 4972150 -32298 32637 5157.02 515702 -124 127 1.18 118 -272 2 10262 99173 0.81681 297.81681 149.31681 14931.68168 0.81681 297.8168 149.31681 14931.68159 0.81681 297.81681 149.31681000000012 14931.68100 2020-01-01 2020-01-02 2020-01-01 00:04:32 2020-01-02 03:32:53 2020-01-01 00:04:32.000 2020-01-02 03:32:53.000 272 99173 49722.5 4972250 272 99173 49722.5 4972250 -32297 32638 5158.02 515802 -128 127 -0.38 -38 -273 2 10263 99174 0.81981 297.81981 149.31981 14931.98198 0.81981 297.81982 149.31982 14931.98217 0.81981 297.81981 149.3198100000002 14931.98100 2020-01-01 2020-01-02 2020-01-01 00:04:33 2020-01-02 03:32:54 2020-01-01 00:04:33.000 2020-01-02 03:32:54.000 273 99174 49723.5 4972350 273 99174 49723.5 4972350 -32296 32639 5159.02 515902 -128 127 -1.94 -194 -274 2 10264 99175 0.82282 297.82282 149.32282 14932.28228 0.82282 297.8228 149.32282 14932.28247 0.82282 297.82282 149.3228199999999 14932.28200 2020-01-01 2020-01-02 2020-01-01 00:04:34 2020-01-02 03:32:55 2020-01-01 00:04:34.000 2020-01-02 03:32:55.000 274 99175 49724.5 4972450 274 99175 49724.5 4972450 -32295 32640 5160.02 516002 -128 124 -3.5 -350 +265 2 10255 99166 0.79579 297.79579 149.29579 14929.57957 0.79579 297.7958 149.29579 14929.57962 0.79579 297.79579 149.29579 14929.57900 2020-01-01 2020-01-02 2020-01-01 00:04:25 2020-01-02 03:32:46 2020-01-01 00:04:25.000 2020-01-02 03:32:46.000 265 99166 49715.5 4971550 265 99166 49715.5 4971550 -32304 32631 5151.02 515102 -124 127 0.3 30 +266 2 10256 99167 0.79879 297.79879 149.29879 14929.87987 0.79879 297.7988 149.29879 14929.87977 0.79879 297.79879 149.29879 14929.87900 2020-01-01 2020-01-02 2020-01-01 00:04:26 2020-01-02 03:32:47 2020-01-01 00:04:26.000 2020-01-02 03:32:47.000 266 99167 49716.5 4971650 266 99167 49716.5 4971650 -32303 32632 5152.02 515202 -128 127 -1.26 -126 +267 2 10257 99168 0.8018 297.8018 149.3018 14930.18018 0.8018 297.8018 149.3018 14930.18012 0.80180 297.80180 149.30180000000001 14930.18000 2020-01-01 2020-01-02 2020-01-01 00:04:27 2020-01-02 03:32:48 2020-01-01 00:04:27.000 2020-01-02 03:32:48.000 267 99168 49717.5 4971750 267 99168 49717.5 4971750 -32302 32633 5153.02 515302 -128 123 -2.82 -282 +268 2 10258 99169 0.8048 297.8048 149.3048 14930.48048 0.8048 297.8048 149.3048 14930.4807 0.80480 297.80480 149.3048 14930.48000 2020-01-01 2020-01-02 2020-01-01 00:04:28 2020-01-02 03:32:49 2020-01-01 00:04:28.000 2020-01-02 03:32:49.000 268 99169 49718.5 4971850 268 99169 49718.5 4971850 -32301 32634 5154.02 515402 -127 124 -1.82 -182 +269 2 10259 99170 0.8078 297.8078 149.3078 14930.78078 0.8078 297.8078 149.3078 14930.78035 0.80780 297.80780 149.30780000000001 14930.78000 2020-01-01 2020-01-02 2020-01-01 00:04:29 2020-01-02 03:32:50 2020-01-01 00:04:29.000 2020-01-02 03:32:50.000 269 99170 49719.5 4971950 269 99170 49719.5 4971950 -32300 32635 5155.02 515502 -126 125 -0.82 -82 +27 2 10017 99927 0.08108 300.08108 150.08108 15158.18918 0.08108 300.0811 150.08108 15158.18936 0.08108 300.08108 150.08108000000001 15158.18908 2020-01-01 2020-01-02 2020-01-01 00:00:27 2020-01-02 03:45:27 2020-01-01 00:00:27.000 2020-01-02 03:45:27.000 27 99927 49977 5047677 27 99927 49977 5047677 -32542 32393 4556.009900990099 460157 -125 126 -1.297029702970297 -131 +270 2 10260 99171 0.81081 297.81081 149.31081 14931.08108 0.81081 297.81082 149.31081 14931.08109 0.81081 297.81081 149.31081 14931.08100 2020-01-01 2020-01-02 2020-01-01 00:04:30 2020-01-02 03:32:51 2020-01-01 00:04:30.000 2020-01-02 03:32:51.000 270 99171 49720.5 4972050 270 99171 49720.5 4972050 -32299 32636 5156.02 515602 -125 126 0.18 18 +271 2 10261 99172 0.81381 297.81381 149.31381 14931.38138 0.81381 297.8138 149.31381 14931.38124 0.81381 297.81381 149.31381 14931.38100 2020-01-01 2020-01-02 2020-01-01 00:04:31 2020-01-02 03:32:52 2020-01-01 00:04:31.000 2020-01-02 03:32:52.000 271 99172 49721.5 4972150 271 99172 49721.5 4972150 -32298 32637 5157.02 515702 -124 127 1.18 118 +272 2 10262 99173 0.81681 297.81681 149.31681 14931.68168 0.81681 297.8168 149.31681 14931.68159 0.81681 297.81681 149.31681 14931.68100 2020-01-01 2020-01-02 2020-01-01 00:04:32 2020-01-02 03:32:53 2020-01-01 00:04:32.000 2020-01-02 03:32:53.000 272 99173 49722.5 4972250 272 99173 49722.5 4972250 -32297 32638 5158.02 515802 -128 127 -0.38 -38 +273 2 10263 99174 0.81981 297.81981 149.31981 14931.98198 0.81981 297.81982 149.31982 14931.98217 0.81981 297.81981 149.31981 14931.98100 2020-01-01 2020-01-02 2020-01-01 00:04:33 2020-01-02 03:32:54 2020-01-01 00:04:33.000 2020-01-02 03:32:54.000 273 99174 49723.5 4972350 273 99174 49723.5 4972350 -32296 32639 5159.02 515902 -128 127 -1.94 -194 +274 2 10264 99175 0.82282 297.82282 149.32282 14932.28228 0.82282 297.8228 149.32282 14932.28247 0.82282 297.82282 149.32281999999998 14932.28200 2020-01-01 2020-01-02 2020-01-01 00:04:34 2020-01-02 03:32:55 2020-01-01 00:04:34.000 2020-01-02 03:32:55.000 274 99175 49724.5 4972450 274 99175 49724.5 4972450 -32295 32640 5160.02 516002 -128 124 -3.5 -350 275 2 10265 99176 0.82582 297.82582 149.32582 14932.58258 0.82582 297.82584 149.32582 14932.58256 0.82582 297.82582 149.32582 14932.58200 2020-01-01 2020-01-02 2020-01-01 00:04:35 2020-01-02 03:32:56 2020-01-01 00:04:35.000 2020-01-02 03:32:56.000 275 99176 49725.5 4972550 275 99176 49725.5 4972550 -32294 32641 5161.02 516102 -127 125 -2.5 -250 -276 2 10266 99177 0.82882 297.82882 149.32882 14932.88288 0.82882 297.82883 149.32882 14932.88275 0.82882 297.82882 149.3288200000001 14932.88200 2020-01-01 2020-01-02 2020-01-01 00:04:36 2020-01-02 03:32:57 2020-01-01 00:04:36.000 2020-01-02 03:32:57.000 276 99177 49726.5 4972650 276 99177 49726.5 4972650 -32293 32642 5162.02 516202 -126 126 -1.5 -150 -277 2 10267 99178 0.83183 297.83183 149.33183 14933.18318 0.83183 297.83182 149.33183 14933.18305 0.83183 297.83183 149.33182999999985 14933.18300 2020-01-01 2020-01-02 2020-01-01 00:04:37 2020-01-02 03:32:58 2020-01-01 00:04:37.000 2020-01-02 03:32:58.000 277 99178 49727.5 4972750 277 99178 49727.5 4972750 -32292 32643 5163.02 516302 -125 127 -0.5 -50 -278 2 10268 99179 0.83483 297.83483 149.33483 14933.48348 0.83483 297.83484 149.33483 14933.48364 0.83483 297.83483 149.33482999999984 14933.48300 2020-01-01 2020-01-02 2020-01-01 00:04:38 2020-01-02 03:32:59 2020-01-01 00:04:38.000 2020-01-02 03:32:59.000 278 99179 49728.5 4972850 278 99179 49728.5 4972850 -32291 32644 5164.02 516402 -128 127 -2.06 -206 -279 2 10269 99180 0.83783 297.83783 149.33783 14933.78378 0.83783 297.83783 149.33783 14933.78394 0.83783 297.83783 149.33783000000022 14933.78300 2020-01-01 2020-01-02 2020-01-01 00:04:39 2020-01-02 03:33:00 2020-01-01 00:04:39.000 2020-01-02 03:33:00.000 279 99180 49729.5 4972950 279 99180 49729.5 4972950 -32290 32645 5165.02 516502 -128 127 -3.62 -362 -28 2 10018 99928 0.08408 300.08408 150.08408 15158.49249 0.08408 300.08408 150.08408 15158.49265 0.08408 300.08408 150.08408000000006 15158.49208 2020-01-01 2020-01-02 2020-01-01 00:00:28 2020-01-02 03:45:28 2020-01-01 00:00:28.000 2020-01-02 03:45:28.000 28 99928 49978 5047778 28 99928 49978 5047778 -32541 32394 4557.009900990099 460258 -124 127 -0.297029702970297 -30 -280 2 10270 99181 0.84084 297.84084 149.34084 14934.08408 0.84084 297.84085 149.34084 14934.08403 0.84084 297.84084 149.34084000000007 14934.08400 2020-01-01 2020-01-02 2020-01-01 00:04:40 2020-01-02 03:33:01 2020-01-01 00:04:40.000 2020-01-02 03:33:01.000 280 99181 49730.5 4973050 280 99181 49730.5 4973050 -32289 32646 5166.02 516602 -128 123 -5.18 -518 -281 2 10271 99182 0.84384 297.84384 149.34384 14934.38438 0.84384 297.84384 149.34384 14934.38421 0.84384 297.84384 149.34383999999974 14934.38400 2020-01-01 2020-01-02 2020-01-01 00:04:41 2020-01-02 03:33:02 2020-01-01 00:04:41.000 2020-01-02 03:33:02.000 281 99182 49731.5 4973150 281 99182 49731.5 4973150 -32288 32647 5167.02 516702 -127 124 -4.18 -418 -282 2 10272 99183 0.84684 297.84684 149.34684 14934.68468 0.84684 297.84683 149.34684 14934.68453 0.84684 297.84684 149.3468400000001 14934.68400 2020-01-01 2020-01-02 2020-01-01 00:04:42 2020-01-02 03:33:03 2020-01-01 00:04:42.000 2020-01-02 03:33:03.000 282 99183 49732.5 4973250 282 99183 49732.5 4973250 -32287 32648 5168.02 516802 -126 125 -3.18 -318 -283 2 10273 99184 0.84984 297.84984 149.34984 14934.98498 0.84984 297.84985 149.34985 14934.98526 0.84984 297.84984 149.34984000000014 14934.98400 2020-01-01 2020-01-02 2020-01-01 00:04:43 2020-01-02 03:33:04 2020-01-01 00:04:43.000 2020-01-02 03:33:04.000 283 99184 49733.5 4973350 283 99184 49733.5 4973350 -32286 32649 5169.02 516902 -125 126 -2.18 -218 -284 2 10274 99185 0.85285 297.85285 149.35285 14935.28528 0.85285 297.85284 149.35285 14935.28542 0.85285 297.85285 149.35284999999993 14935.28500 2020-01-01 2020-01-02 2020-01-01 00:04:44 2020-01-02 03:33:05 2020-01-01 00:04:44.000 2020-01-02 03:33:05.000 284 99185 49734.5 4973450 284 99185 49734.5 4973450 -32285 32650 5170.02 517002 -124 127 -1.18 -118 -285 2 10275 99186 0.85585 297.85585 149.35585 14935.58558 0.85585 297.85587 149.35585 14935.5855 0.85585 297.85585 149.35584999999995 14935.58500 2020-01-01 2020-01-02 2020-01-01 00:04:45 2020-01-02 03:33:06 2020-01-01 00:04:45.000 2020-01-02 03:33:06.000 285 99186 49735.5 4973550 285 99186 49735.5 4973550 -32284 32651 5171.02 517102 -128 127 -2.74 -274 -286 2 10276 99187 0.85885 297.85885 149.35885 14935.88588 0.85885 297.85886 149.35885 14935.88568 0.85885 297.85885 149.35885000000005 14935.88500 2020-01-01 2020-01-02 2020-01-01 00:04:46 2020-01-02 03:33:07 2020-01-01 00:04:46.000 2020-01-02 03:33:07.000 286 99187 49736.5 4973650 286 99187 49736.5 4973650 -32283 32652 5172.02 517202 -128 123 -4.3 -430 -287 2 10277 99188 0.86186 297.86186 149.36186 14936.18618 0.86186 297.86185 149.36186 14936.186 0.86186 297.86186 149.3618600000002 14936.18600 2020-01-01 2020-01-02 2020-01-01 00:04:47 2020-01-02 03:33:08 2020-01-01 00:04:47.000 2020-01-02 03:33:08.000 287 99188 49737.5 4973750 287 99188 49737.5 4973750 -32282 32653 5173.02 517302 -127 124 -3.3 -330 -288 2 10278 99189 0.86486 297.86486 149.36486 14936.48648 0.86486 297.86487 149.36486 14936.48673 0.86486 297.86486 149.36485999999985 14936.48600 2020-01-01 2020-01-02 2020-01-01 00:04:48 2020-01-02 03:33:09 2020-01-01 00:04:48.000 2020-01-02 03:33:09.000 288 99189 49738.5 4973850 288 99189 49738.5 4973850 -32281 32654 5174.02 517402 -126 125 -2.3 -230 -289 2 10279 99190 0.86786 297.86786 149.36786 14936.78678 0.86786 297.86786 149.36786 14936.78688 0.86786 297.86786 149.36786000000015 14936.78600 2020-01-01 2020-01-02 2020-01-01 00:04:49 2020-01-02 03:33:10 2020-01-01 00:04:49.000 2020-01-02 03:33:10.000 289 99190 49739.5 4973950 289 99190 49739.5 4973950 -32280 32655 5175.02 517502 -125 126 -1.3 -130 -29 2 10019 99929 0.08708 300.08708 150.08708 15158.79579 0.08708 300.0871 150.08708 15158.79576 0.08708 300.08708 150.0870799999999 15158.79508 2020-01-01 2020-01-02 2020-01-01 00:00:29 2020-01-02 03:45:29 2020-01-01 00:00:29.000 2020-01-02 03:45:29.000 29 99929 49979 5047879 29 99929 49979 5047879 -32540 32395 4558.009900990099 460359 -128 127 -1.8316831683168318 -185 -290 2 10280 99191 0.87087 297.87087 149.37087 14937.08708 0.87087 297.87088 149.37087 14937.087 0.87087 297.87087 149.37087000000008 14937.08700 2020-01-01 2020-01-02 2020-01-01 00:04:50 2020-01-02 03:33:11 2020-01-01 00:04:50.000 2020-01-02 03:33:11.000 290 99191 49740.5 4974050 290 99191 49740.5 4974050 -32279 32656 5176.02 517602 -124 127 -0.3 -30 -291 2 10281 99192 0.87387 297.87387 149.37387 14937.38738 0.87387 297.87387 149.37387 14937.38716 0.87387 297.87387 149.37386999999993 14937.38700 2020-01-01 2020-01-02 2020-01-01 00:04:51 2020-01-02 03:33:12 2020-01-01 00:04:51.000 2020-01-02 03:33:12.000 291 99192 49741.5 4974150 291 99192 49741.5 4974150 -32278 32657 5177.02 517702 -128 127 -1.86 -186 -292 2 10282 99193 0.87687 297.87687 149.37687 14937.68768 0.87687 297.8769 149.37687 14937.68789 0.87687 297.87687 149.37687000000003 14937.68700 2020-01-01 2020-01-02 2020-01-01 00:04:52 2020-01-02 03:33:13 2020-01-01 00:04:52.000 2020-01-02 03:33:13.000 292 99193 49742.5 4974250 292 99193 49742.5 4974250 -32277 32658 5178.02 517802 -128 123 -3.42 -342 -293 2 10283 99194 0.87987 297.87987 149.37987 14937.98798 0.87987 297.87988 149.37988 14937.9882 0.87987 297.87987 149.3798700000001 14937.98700 2020-01-01 2020-01-02 2020-01-01 00:04:53 2020-01-02 03:33:14 2020-01-01 00:04:53.000 2020-01-02 03:33:14.000 293 99194 49743.5 4974350 293 99194 49743.5 4974350 -32276 32659 5179.02 517902 -127 124 -2.42 -242 -294 2 10284 99195 0.88288 297.88288 149.38288 14938.28828 0.88288 297.88287 149.38288 14938.28835 0.88288 297.88288 149.38287999999986 14938.28800 2020-01-01 2020-01-02 2020-01-01 00:04:54 2020-01-02 03:33:15 2020-01-01 00:04:54.000 2020-01-02 03:33:15.000 294 99195 49744.5 4974450 294 99195 49744.5 4974450 -32275 32660 5180.02 518002 -126 125 -1.42 -142 -295 2 10285 99196 0.88588 297.88588 149.38588 14938.58858 0.88588 297.8859 149.38588 14938.58847 0.88588 297.88588 149.38587999999987 14938.58800 2020-01-01 2020-01-02 2020-01-01 00:04:55 2020-01-02 03:33:16 2020-01-01 00:04:55.000 2020-01-02 03:33:16.000 295 99196 49745.5 4974550 295 99196 49745.5 4974550 -32274 32661 5181.02 518102 -125 126 -0.42 -42 -296 2 10286 99197 0.88888 297.88888 149.38888 14938.88888 0.88888 297.8889 149.38888 14938.88863 0.88888 297.88888 149.38888000000023 14938.88800 2020-01-01 2020-01-02 2020-01-01 00:04:56 2020-01-02 03:33:17 2020-01-01 00:04:56.000 2020-01-02 03:33:17.000 296 99197 49746.5 4974650 296 99197 49746.5 4974650 -32273 32662 5182.02 518202 -124 127 0.58 58 -297 2 10287 99198 0.89189 297.89189 149.39189 14939.18918 0.89189 297.8919 149.39189 14939.18936 0.89189 297.89189 149.3918900000001 14939.18900 2020-01-01 2020-01-02 2020-01-01 00:04:57 2020-01-02 03:33:18 2020-01-01 00:04:57.000 2020-01-02 03:33:18.000 297 99198 49747.5 4974750 297 99198 49747.5 4974750 -32272 32663 5183.02 518302 -128 127 -0.98 -98 -298 2 10288 99199 0.89489 297.89489 149.39489 14939.48948 0.89489 297.8949 149.39489 14939.48967 0.89489 297.89489 149.39488999999975 14939.48900 2020-01-01 2020-01-02 2020-01-01 00:04:58 2020-01-02 03:33:19 2020-01-01 00:04:58.000 2020-01-02 03:33:19.000 298 99199 49748.5 4974850 298 99199 49748.5 4974850 -32271 32664 5184.02 518402 -128 127 -2.54 -254 -299 2 10289 99200 0.89789 297.89789 149.39789 14939.78978 0.89789 297.8979 149.39789 14939.78986 0.89789 297.89789 149.39789000000013 14939.78900 2020-01-01 2020-01-02 2020-01-01 00:04:59 2020-01-02 03:33:20 2020-01-01 00:04:59.000 2020-01-02 03:33:20.000 299 99200 49749.5 4974950 299 99200 49749.5 4974950 -32270 32665 5185.02 518502 -128 124 -4.1 -410 +276 2 10266 99177 0.82882 297.82882 149.32882 14932.88288 0.82882 297.82883 149.32882 14932.88275 0.82882 297.82882 149.32882 14932.88200 2020-01-01 2020-01-02 2020-01-01 00:04:36 2020-01-02 03:32:57 2020-01-01 00:04:36.000 2020-01-02 03:32:57.000 276 99177 49726.5 4972650 276 99177 49726.5 4972650 -32293 32642 5162.02 516202 -126 126 -1.5 -150 +277 2 10267 99178 0.83183 297.83183 149.33183 14933.18318 0.83183 297.83182 149.33183 14933.18305 0.83183 297.83183 149.33183 14933.18300 2020-01-01 2020-01-02 2020-01-01 00:04:37 2020-01-02 03:32:58 2020-01-01 00:04:37.000 2020-01-02 03:32:58.000 277 99178 49727.5 4972750 277 99178 49727.5 4972750 -32292 32643 5163.02 516302 -125 127 -0.5 -50 +278 2 10268 99179 0.83483 297.83483 149.33483 14933.48348 0.83483 297.83484 149.33483 14933.48364 0.83483 297.83483 149.33483 14933.48300 2020-01-01 2020-01-02 2020-01-01 00:04:38 2020-01-02 03:32:59 2020-01-01 00:04:38.000 2020-01-02 03:32:59.000 278 99179 49728.5 4972850 278 99179 49728.5 4972850 -32291 32644 5164.02 516402 -128 127 -2.06 -206 +279 2 10269 99180 0.83783 297.83783 149.33783 14933.78378 0.83783 297.83783 149.33783 14933.78394 0.83783 297.83783 149.33783 14933.78300 2020-01-01 2020-01-02 2020-01-01 00:04:39 2020-01-02 03:33:00 2020-01-01 00:04:39.000 2020-01-02 03:33:00.000 279 99180 49729.5 4972950 279 99180 49729.5 4972950 -32290 32645 5165.02 516502 -128 127 -3.62 -362 +28 2 10018 99928 0.08408 300.08408 150.08408 15158.49249 0.08408 300.08408 150.08408 15158.49265 0.08408 300.08408 150.08408 15158.49208 2020-01-01 2020-01-02 2020-01-01 00:00:28 2020-01-02 03:45:28 2020-01-01 00:00:28.000 2020-01-02 03:45:28.000 28 99928 49978 5047778 28 99928 49978 5047778 -32541 32394 4557.009900990099 460258 -124 127 -0.297029702970297 -30 +280 2 10270 99181 0.84084 297.84084 149.34084 14934.08408 0.84084 297.84085 149.34084 14934.08403 0.84084 297.84084 149.34084000000001 14934.08400 2020-01-01 2020-01-02 2020-01-01 00:04:40 2020-01-02 03:33:01 2020-01-01 00:04:40.000 2020-01-02 03:33:01.000 280 99181 49730.5 4973050 280 99181 49730.5 4973050 -32289 32646 5166.02 516602 -128 123 -5.18 -518 +281 2 10271 99182 0.84384 297.84384 149.34384 14934.38438 0.84384 297.84384 149.34384 14934.38421 0.84384 297.84384 149.34384 14934.38400 2020-01-01 2020-01-02 2020-01-01 00:04:41 2020-01-02 03:33:02 2020-01-01 00:04:41.000 2020-01-02 03:33:02.000 281 99182 49731.5 4973150 281 99182 49731.5 4973150 -32288 32647 5167.02 516702 -127 124 -4.18 -418 +282 2 10272 99183 0.84684 297.84684 149.34684 14934.68468 0.84684 297.84683 149.34684 14934.68453 0.84684 297.84684 149.34684 14934.68400 2020-01-01 2020-01-02 2020-01-01 00:04:42 2020-01-02 03:33:03 2020-01-01 00:04:42.000 2020-01-02 03:33:03.000 282 99183 49732.5 4973250 282 99183 49732.5 4973250 -32287 32648 5168.02 516802 -126 125 -3.18 -318 +283 2 10273 99184 0.84984 297.84984 149.34984 14934.98498 0.84984 297.84985 149.34985 14934.98526 0.84984 297.84984 149.34984 14934.98400 2020-01-01 2020-01-02 2020-01-01 00:04:43 2020-01-02 03:33:04 2020-01-01 00:04:43.000 2020-01-02 03:33:04.000 283 99184 49733.5 4973350 283 99184 49733.5 4973350 -32286 32649 5169.02 516902 -125 126 -2.18 -218 +284 2 10274 99185 0.85285 297.85285 149.35285 14935.28528 0.85285 297.85284 149.35285 14935.28542 0.85285 297.85285 149.35285 14935.28500 2020-01-01 2020-01-02 2020-01-01 00:04:44 2020-01-02 03:33:05 2020-01-01 00:04:44.000 2020-01-02 03:33:05.000 284 99185 49734.5 4973450 284 99185 49734.5 4973450 -32285 32650 5170.02 517002 -124 127 -1.18 -118 +285 2 10275 99186 0.85585 297.85585 149.35585 14935.58558 0.85585 297.85587 149.35585 14935.5855 0.85585 297.85585 149.35585 14935.58500 2020-01-01 2020-01-02 2020-01-01 00:04:45 2020-01-02 03:33:06 2020-01-01 00:04:45.000 2020-01-02 03:33:06.000 285 99186 49735.5 4973550 285 99186 49735.5 4973550 -32284 32651 5171.02 517102 -128 127 -2.74 -274 +286 2 10276 99187 0.85885 297.85885 149.35885 14935.88588 0.85885 297.85886 149.35885 14935.88568 0.85885 297.85885 149.35885 14935.88500 2020-01-01 2020-01-02 2020-01-01 00:04:46 2020-01-02 03:33:07 2020-01-01 00:04:46.000 2020-01-02 03:33:07.000 286 99187 49736.5 4973650 286 99187 49736.5 4973650 -32283 32652 5172.02 517202 -128 123 -4.3 -430 +287 2 10277 99188 0.86186 297.86186 149.36186 14936.18618 0.86186 297.86185 149.36186 14936.186 0.86186 297.86186 149.36186 14936.18600 2020-01-01 2020-01-02 2020-01-01 00:04:47 2020-01-02 03:33:08 2020-01-01 00:04:47.000 2020-01-02 03:33:08.000 287 99188 49737.5 4973750 287 99188 49737.5 4973750 -32282 32653 5173.02 517302 -127 124 -3.3 -330 +288 2 10278 99189 0.86486 297.86486 149.36486 14936.48648 0.86486 297.86487 149.36486 14936.48673 0.86486 297.86486 149.36486000000002 14936.48600 2020-01-01 2020-01-02 2020-01-01 00:04:48 2020-01-02 03:33:09 2020-01-01 00:04:48.000 2020-01-02 03:33:09.000 288 99189 49738.5 4973850 288 99189 49738.5 4973850 -32281 32654 5174.02 517402 -126 125 -2.3 -230 +289 2 10279 99190 0.86786 297.86786 149.36786 14936.78678 0.86786 297.86786 149.36786 14936.78688 0.86786 297.86786 149.36786 14936.78600 2020-01-01 2020-01-02 2020-01-01 00:04:49 2020-01-02 03:33:10 2020-01-01 00:04:49.000 2020-01-02 03:33:10.000 289 99190 49739.5 4973950 289 99190 49739.5 4973950 -32280 32655 5175.02 517502 -125 126 -1.3 -130 +29 2 10019 99929 0.08708 300.08708 150.08708 15158.79579 0.08708 300.0871 150.08708 15158.79576 0.08708 300.08708 150.08708 15158.79508 2020-01-01 2020-01-02 2020-01-01 00:00:29 2020-01-02 03:45:29 2020-01-01 00:00:29.000 2020-01-02 03:45:29.000 29 99929 49979 5047879 29 99929 49979 5047879 -32540 32395 4558.009900990099 460359 -128 127 -1.8316831683168318 -185 +290 2 10280 99191 0.87087 297.87087 149.37087 14937.08708 0.87087 297.87088 149.37087 14937.087 0.87087 297.87087 149.37087 14937.08700 2020-01-01 2020-01-02 2020-01-01 00:04:50 2020-01-02 03:33:11 2020-01-01 00:04:50.000 2020-01-02 03:33:11.000 290 99191 49740.5 4974050 290 99191 49740.5 4974050 -32279 32656 5176.02 517602 -124 127 -0.3 -30 +291 2 10281 99192 0.87387 297.87387 149.37387 14937.38738 0.87387 297.87387 149.37387 14937.38716 0.87387 297.87387 149.37387 14937.38700 2020-01-01 2020-01-02 2020-01-01 00:04:51 2020-01-02 03:33:12 2020-01-01 00:04:51.000 2020-01-02 03:33:12.000 291 99192 49741.5 4974150 291 99192 49741.5 4974150 -32278 32657 5177.02 517702 -128 127 -1.86 -186 +292 2 10282 99193 0.87687 297.87687 149.37687 14937.68768 0.87687 297.8769 149.37687 14937.68789 0.87687 297.87687 149.37687 14937.68700 2020-01-01 2020-01-02 2020-01-01 00:04:52 2020-01-02 03:33:13 2020-01-01 00:04:52.000 2020-01-02 03:33:13.000 292 99193 49742.5 4974250 292 99193 49742.5 4974250 -32277 32658 5178.02 517802 -128 123 -3.42 -342 +293 2 10283 99194 0.87987 297.87987 149.37987 14937.98798 0.87987 297.87988 149.37988 14937.9882 0.87987 297.87987 149.37986999999998 14937.98700 2020-01-01 2020-01-02 2020-01-01 00:04:53 2020-01-02 03:33:14 2020-01-01 00:04:53.000 2020-01-02 03:33:14.000 293 99194 49743.5 4974350 293 99194 49743.5 4974350 -32276 32659 5179.02 517902 -127 124 -2.42 -242 +294 2 10284 99195 0.88288 297.88288 149.38288 14938.28828 0.88288 297.88287 149.38288 14938.28835 0.88288 297.88288 149.38288 14938.28800 2020-01-01 2020-01-02 2020-01-01 00:04:54 2020-01-02 03:33:15 2020-01-01 00:04:54.000 2020-01-02 03:33:15.000 294 99195 49744.5 4974450 294 99195 49744.5 4974450 -32275 32660 5180.02 518002 -126 125 -1.42 -142 +295 2 10285 99196 0.88588 297.88588 149.38588 14938.58858 0.88588 297.8859 149.38588 14938.58847 0.88588 297.88588 149.38588 14938.58800 2020-01-01 2020-01-02 2020-01-01 00:04:55 2020-01-02 03:33:16 2020-01-01 00:04:55.000 2020-01-02 03:33:16.000 295 99196 49745.5 4974550 295 99196 49745.5 4974550 -32274 32661 5181.02 518102 -125 126 -0.42 -42 +296 2 10286 99197 0.88888 297.88888 149.38888 14938.88888 0.88888 297.8889 149.38888 14938.88863 0.88888 297.88888 149.38888 14938.88800 2020-01-01 2020-01-02 2020-01-01 00:04:56 2020-01-02 03:33:17 2020-01-01 00:04:56.000 2020-01-02 03:33:17.000 296 99197 49746.5 4974650 296 99197 49746.5 4974650 -32273 32662 5182.02 518202 -124 127 0.58 58 +297 2 10287 99198 0.89189 297.89189 149.39189 14939.18918 0.89189 297.8919 149.39189 14939.18936 0.89189 297.89189 149.39189 14939.18900 2020-01-01 2020-01-02 2020-01-01 00:04:57 2020-01-02 03:33:18 2020-01-01 00:04:57.000 2020-01-02 03:33:18.000 297 99198 49747.5 4974750 297 99198 49747.5 4974750 -32272 32663 5183.02 518302 -128 127 -0.98 -98 +298 2 10288 99199 0.89489 297.89489 149.39489 14939.48948 0.89489 297.8949 149.39489 14939.48967 0.89489 297.89489 149.39489 14939.48900 2020-01-01 2020-01-02 2020-01-01 00:04:58 2020-01-02 03:33:19 2020-01-01 00:04:58.000 2020-01-02 03:33:19.000 298 99199 49748.5 4974850 298 99199 49748.5 4974850 -32271 32664 5184.02 518402 -128 127 -2.54 -254 +299 2 10289 99200 0.89789 297.89789 149.39789 14939.78978 0.89789 297.8979 149.39789 14939.78986 0.89789 297.89789 149.39789000000002 14939.78900 2020-01-01 2020-01-02 2020-01-01 00:04:59 2020-01-02 03:33:20 2020-01-01 00:04:59.000 2020-01-02 03:33:20.000 299 99200 49749.5 4974950 299 99200 49749.5 4974950 -32270 32665 5185.02 518502 -128 124 -4.1 -410 3 2 1002 9993 0.009 300.009 150.009 15150.9099 0.009 300.009 150.009 15150.90958 0.00900 300.00900 150.009 15150.90900 2020-01-01 2020-01-02 2020-01-01 00:00:03 2020-01-02 03:45:03 2020-01-01 00:00:03.000 2020-01-02 03:45:03.000 3 99903 49953 5045253 3 99903 49953 5045253 -32566 32369 4532.009900990099 457733 -124 127 0.04950495049504951 5 -30 2 10020 99930 0.09009 300.09009 150.09009 15159.09909 0.09009 300.0901 150.09008 15159.09894 0.09009 300.09009 150.09008999999986 15159.09909 2020-01-01 2020-01-02 2020-01-01 00:00:30 2020-01-02 03:45:30 2020-01-01 00:00:30.000 2020-01-02 03:45:30.000 30 99930 49980 5047980 30 99930 49980 5047980 -32539 32396 4559.009900990099 460460 -128 123 -3.366336633663366 -340 -300 2 10290 99201 0.9009 297.9009 149.4009 14940.09009 0.9009 297.9009 149.40089 14940.08995 0.90090 297.90090 149.40090000000004 14940.09000 2020-01-01 2020-01-02 2020-01-01 00:05:00 2020-01-02 03:33:21 2020-01-01 00:05:00.000 2020-01-02 03:33:21.000 300 99201 49750.5 4975050 300 99201 49750.5 4975050 -32269 32666 5186.02 518602 -127 125 -3.1 -310 -301 2 10291 99202 0.9039 297.9039 149.4039 14940.39039 0.9039 297.9039 149.4039 14940.39009 0.90390 297.90390 149.4038999999999 14940.39000 2020-01-01 2020-01-02 2020-01-01 00:05:01 2020-01-02 03:33:22 2020-01-01 00:05:01.000 2020-01-02 03:33:22.000 301 99202 49751.5 4975150 301 99202 49751.5 4975150 -32268 32667 5187.02 518702 -126 126 -2.1 -210 -302 2 10292 99203 0.9069 297.9069 149.4069 14940.69069 0.9069 297.90692 149.4069 14940.69083 0.90690 297.90690 149.40689999999998 14940.69000 2020-01-01 2020-01-02 2020-01-01 00:05:02 2020-01-02 03:33:23 2020-01-01 00:05:02.000 2020-01-02 03:33:23.000 302 99203 49752.5 4975250 302 99203 49752.5 4975250 -32267 32668 5188.02 518802 -125 127 -1.1 -110 -303 2 10293 99204 0.9099 297.9099 149.4099 14940.99099 0.9099 297.9099 149.40991 14940.99114 0.90990 297.90990 149.40990000000005 14940.99000 2020-01-01 2020-01-02 2020-01-01 00:05:03 2020-01-02 03:33:24 2020-01-01 00:05:03.000 2020-01-02 03:33:24.000 303 99204 49753.5 4975350 303 99204 49753.5 4975350 -32266 32669 5189.02 518902 -128 127 -2.66 -266 -304 2 10294 99205 0.91291 297.91291 149.41291 14941.29129 0.91291 297.9129 149.41291 14941.29133 0.91291 297.91291 149.41290999999978 14941.29100 2020-01-01 2020-01-02 2020-01-01 00:05:04 2020-01-02 03:33:25 2020-01-01 00:05:04.000 2020-01-02 03:33:25.000 304 99205 49754.5 4975450 304 99205 49754.5 4975450 -32265 32670 5190.02 519002 -128 127 -4.22 -422 -305 2 10295 99206 0.91591 297.91591 149.41591 14941.59159 0.91591 297.91592 149.41591 14941.59141 0.91591 297.91591 149.41590999999985 14941.59100 2020-01-01 2020-01-02 2020-01-01 00:05:05 2020-01-02 03:33:26 2020-01-01 00:05:05.000 2020-01-02 03:33:26.000 305 99206 49755.5 4975550 305 99206 49755.5 4975550 -32264 32671 5191.02 519102 -128 123 -5.78 -578 -306 2 10296 99207 0.91891 297.91891 149.41891 14941.89189 0.91891 297.9189 149.41891 14941.89172 0.91891 297.91891 149.41891000000015 14941.89100 2020-01-01 2020-01-02 2020-01-01 00:05:06 2020-01-02 03:33:27 2020-01-01 00:05:06.000 2020-01-02 03:33:27.000 306 99207 49756.5 4975650 306 99207 49756.5 4975650 -32263 32672 5192.02 519202 -127 124 -4.78 -478 -307 2 10297 99208 0.92192 297.92192 149.42192 14942.19219 0.92192 297.92194 149.42192 14942.1923 0.92192 297.92192 149.42192000000003 14942.19200 2020-01-01 2020-01-02 2020-01-01 00:05:07 2020-01-02 03:33:28 2020-01-01 00:05:07.000 2020-01-02 03:33:28.000 307 99208 49757.5 4975750 307 99208 49757.5 4975750 -32262 32673 5193.02 519302 -126 125 -3.78 -378 -308 2 10298 99209 0.92492 297.92492 149.42492 14942.49249 0.92492 297.92493 149.42492 14942.49265 0.92492 297.92492 149.42492 14942.49200 2020-01-01 2020-01-02 2020-01-01 00:05:08 2020-01-02 03:33:29 2020-01-01 00:05:08.000 2020-01-02 03:33:29.000 308 99209 49758.5 4975850 308 99209 49758.5 4975850 -32261 32674 5194.02 519402 -125 126 -2.78 -278 -309 2 10299 99210 0.92792 297.92792 149.42792 14942.79279 0.92792 297.92792 149.42792 14942.7928 0.92792 297.92792 149.42792000000006 14942.79200 2020-01-01 2020-01-02 2020-01-01 00:05:09 2020-01-02 03:33:30 2020-01-01 00:05:09.000 2020-01-02 03:33:30.000 309 99210 49759.5 4975950 309 99210 49759.5 4975950 -32260 32675 5195.02 519502 -124 127 -1.78 -178 -31 2 10021 99931 0.09309 300.09309 150.09309 15159.4024 0.09309 300.09308 150.09309 15159.40224 0.09309 300.09309 150.09309000000013 15159.40209 2020-01-01 2020-01-02 2020-01-01 00:00:31 2020-01-02 03:45:31 2020-01-01 00:00:31.000 2020-01-02 03:45:31.000 31 99931 49981 5048081 31 99931 49981 5048081 -32538 32397 4560.009900990099 460561 -127 124 -2.366336633663366 -239 -310 2 10300 99211 0.93093 297.93093 149.43093 14943.09309 0.93093 297.93094 149.43092 14943.09288 0.93093 297.93093 149.43093000000025 14943.09300 2020-01-01 2020-01-02 2020-01-01 00:05:10 2020-01-02 03:33:31 2020-01-01 00:05:10.000 2020-01-02 03:33:31.000 310 99211 49760.5 4976050 310 99211 49760.5 4976050 -32259 32676 5196.02 519602 -128 127 -3.34 -334 -311 2 10301 99212 0.93393 297.93393 149.43393 14943.39339 0.93393 297.93393 149.43393 14943.39319 0.93393 297.93393 149.43392999999992 14943.39300 2020-01-01 2020-01-02 2020-01-01 00:05:11 2020-01-02 03:33:32 2020-01-01 00:05:11.000 2020-01-02 03:33:32.000 311 99212 49761.5 4976150 311 99212 49761.5 4976150 -32258 32677 5197.02 519702 -128 123 -4.9 -490 -312 2 10302 99213 0.93693 297.93693 149.43693 14943.69369 0.93693 297.93695 149.43693 14943.69377 0.93693 297.93693 149.4369299999999 14943.69300 2020-01-01 2020-01-02 2020-01-01 00:05:12 2020-01-02 03:33:33 2020-01-01 00:05:12.000 2020-01-02 03:33:33.000 312 99213 49762.5 4976250 312 99213 49762.5 4976250 -32257 32678 5198.02 519802 -127 124 -3.9 -390 -313 2 10303 99214 0.93993 297.93993 149.43993 14943.99399 0.93993 297.93994 149.43994 14943.99412 0.93993 297.93993 149.43993000000012 14943.99300 2020-01-01 2020-01-02 2020-01-01 00:05:13 2020-01-02 03:33:34 2020-01-01 00:05:13.000 2020-01-02 03:33:34.000 313 99214 49763.5 4976350 313 99214 49763.5 4976350 -32256 32679 5199.02 519902 -126 125 -2.9 -290 -314 2 10304 99215 0.94294 297.94294 149.44294 14944.29429 0.94294 297.94293 149.44294 14944.29427 0.94294 297.94294 149.44294000000014 14944.29400 2020-01-01 2020-01-02 2020-01-01 00:05:14 2020-01-02 03:33:35 2020-01-01 00:05:14.000 2020-01-02 03:33:35.000 314 99215 49764.5 4976450 314 99215 49764.5 4976450 -32255 32680 5200.02 520002 -125 126 -1.9 -190 -315 2 10305 99216 0.94594 297.94594 149.44594 14944.59459 0.94594 297.94595 149.44595 14944.595 0.94594 297.94594 149.4459399999998 14944.59400 2020-01-01 2020-01-02 2020-01-01 00:05:15 2020-01-02 03:33:36 2020-01-01 00:05:15.000 2020-01-02 03:33:36.000 315 99216 49765.5 4976550 315 99216 49765.5 4976550 -32254 32681 5201.02 520102 -124 127 -0.9 -90 -316 2 10306 99217 0.94894 297.94894 149.44894 14944.89489 0.94894 297.94894 149.44894 14944.89466 0.94894 297.94894 149.44894000000016 14944.89400 2020-01-01 2020-01-02 2020-01-01 00:05:16 2020-01-02 03:33:37 2020-01-01 00:05:16.000 2020-01-02 03:33:37.000 316 99217 49766.5 4976650 316 99217 49766.5 4976650 -32253 32682 5202.02 520202 -128 127 -2.46 -246 -317 2 10307 99218 0.95195 297.95195 149.45195 14945.19519 0.95195 297.95197 149.45195 14945.19524 0.95195 297.95195 149.45195000000004 14945.19500 2020-01-01 2020-01-02 2020-01-01 00:05:17 2020-01-02 03:33:38 2020-01-01 00:05:17.000 2020-01-02 03:33:38.000 317 99218 49767.5 4976750 317 99218 49767.5 4976750 -32252 32683 5203.02 520302 -128 123 -4.02 -402 -318 2 10308 99219 0.95495 297.95495 149.45495 14945.49549 0.95495 297.95496 149.45495 14945.49558 0.95495 297.95495 149.4549499999999 14945.49500 2020-01-01 2020-01-02 2020-01-01 00:05:18 2020-01-02 03:33:39 2020-01-01 00:05:18.000 2020-01-02 03:33:39.000 318 99219 49768.5 4976850 318 99219 49768.5 4976850 -32251 32684 5204.02 520402 -127 124 -3.02 -302 +30 2 10020 99930 0.09009 300.09009 150.09009 15159.09909 0.09009 300.0901 150.09008 15159.09894 0.09009 300.09009 150.09009 15159.09909 2020-01-01 2020-01-02 2020-01-01 00:00:30 2020-01-02 03:45:30 2020-01-01 00:00:30.000 2020-01-02 03:45:30.000 30 99930 49980 5047980 30 99930 49980 5047980 -32539 32396 4559.009900990099 460460 -128 123 -3.366336633663366 -340 +300 2 10290 99201 0.9009 297.9009 149.4009 14940.09009 0.9009 297.9009 149.40089 14940.08995 0.90090 297.90090 149.4009 14940.09000 2020-01-01 2020-01-02 2020-01-01 00:05:00 2020-01-02 03:33:21 2020-01-01 00:05:00.000 2020-01-02 03:33:21.000 300 99201 49750.5 4975050 300 99201 49750.5 4975050 -32269 32666 5186.02 518602 -127 125 -3.1 -310 +301 2 10291 99202 0.9039 297.9039 149.4039 14940.39039 0.9039 297.9039 149.4039 14940.39009 0.90390 297.90390 149.4039 14940.39000 2020-01-01 2020-01-02 2020-01-01 00:05:01 2020-01-02 03:33:22 2020-01-01 00:05:01.000 2020-01-02 03:33:22.000 301 99202 49751.5 4975150 301 99202 49751.5 4975150 -32268 32667 5187.02 518702 -126 126 -2.1 -210 +302 2 10292 99203 0.9069 297.9069 149.4069 14940.69069 0.9069 297.90692 149.4069 14940.69083 0.90690 297.90690 149.4069 14940.69000 2020-01-01 2020-01-02 2020-01-01 00:05:02 2020-01-02 03:33:23 2020-01-01 00:05:02.000 2020-01-02 03:33:23.000 302 99203 49752.5 4975250 302 99203 49752.5 4975250 -32267 32668 5188.02 518802 -125 127 -1.1 -110 +303 2 10293 99204 0.9099 297.9099 149.4099 14940.99099 0.9099 297.9099 149.40991 14940.99114 0.90990 297.90990 149.4099 14940.99000 2020-01-01 2020-01-02 2020-01-01 00:05:03 2020-01-02 03:33:24 2020-01-01 00:05:03.000 2020-01-02 03:33:24.000 303 99204 49753.5 4975350 303 99204 49753.5 4975350 -32266 32669 5189.02 518902 -128 127 -2.66 -266 +304 2 10294 99205 0.91291 297.91291 149.41291 14941.29129 0.91291 297.9129 149.41291 14941.29133 0.91291 297.91291 149.41290999999998 14941.29100 2020-01-01 2020-01-02 2020-01-01 00:05:04 2020-01-02 03:33:25 2020-01-01 00:05:04.000 2020-01-02 03:33:25.000 304 99205 49754.5 4975450 304 99205 49754.5 4975450 -32265 32670 5190.02 519002 -128 127 -4.22 -422 +305 2 10295 99206 0.91591 297.91591 149.41591 14941.59159 0.91591 297.91592 149.41591 14941.59141 0.91591 297.91591 149.41591 14941.59100 2020-01-01 2020-01-02 2020-01-01 00:05:05 2020-01-02 03:33:26 2020-01-01 00:05:05.000 2020-01-02 03:33:26.000 305 99206 49755.5 4975550 305 99206 49755.5 4975550 -32264 32671 5191.02 519102 -128 123 -5.78 -578 +306 2 10296 99207 0.91891 297.91891 149.41891 14941.89189 0.91891 297.9189 149.41891 14941.89172 0.91891 297.91891 149.41890999999998 14941.89100 2020-01-01 2020-01-02 2020-01-01 00:05:06 2020-01-02 03:33:27 2020-01-01 00:05:06.000 2020-01-02 03:33:27.000 306 99207 49756.5 4975650 306 99207 49756.5 4975650 -32263 32672 5192.02 519202 -127 124 -4.78 -478 +307 2 10297 99208 0.92192 297.92192 149.42192 14942.19219 0.92192 297.92194 149.42192 14942.1923 0.92192 297.92192 149.42192 14942.19200 2020-01-01 2020-01-02 2020-01-01 00:05:07 2020-01-02 03:33:28 2020-01-01 00:05:07.000 2020-01-02 03:33:28.000 307 99208 49757.5 4975750 307 99208 49757.5 4975750 -32262 32673 5193.02 519302 -126 125 -3.78 -378 +308 2 10298 99209 0.92492 297.92492 149.42492 14942.49249 0.92492 297.92493 149.42492 14942.49265 0.92492 297.92492 149.42492000000001 14942.49200 2020-01-01 2020-01-02 2020-01-01 00:05:08 2020-01-02 03:33:29 2020-01-01 00:05:08.000 2020-01-02 03:33:29.000 308 99209 49758.5 4975850 308 99209 49758.5 4975850 -32261 32674 5194.02 519402 -125 126 -2.78 -278 +309 2 10299 99210 0.92792 297.92792 149.42792 14942.79279 0.92792 297.92792 149.42792 14942.7928 0.92792 297.92792 149.42792 14942.79200 2020-01-01 2020-01-02 2020-01-01 00:05:09 2020-01-02 03:33:30 2020-01-01 00:05:09.000 2020-01-02 03:33:30.000 309 99210 49759.5 4975950 309 99210 49759.5 4975950 -32260 32675 5195.02 519502 -124 127 -1.78 -178 +31 2 10021 99931 0.09309 300.09309 150.09309 15159.4024 0.09309 300.09308 150.09309 15159.40224 0.09309 300.09309 150.09309 15159.40209 2020-01-01 2020-01-02 2020-01-01 00:00:31 2020-01-02 03:45:31 2020-01-01 00:00:31.000 2020-01-02 03:45:31.000 31 99931 49981 5048081 31 99931 49981 5048081 -32538 32397 4560.009900990099 460561 -127 124 -2.366336633663366 -239 +310 2 10300 99211 0.93093 297.93093 149.43093 14943.09309 0.93093 297.93094 149.43092 14943.09288 0.93093 297.93093 149.43093000000002 14943.09300 2020-01-01 2020-01-02 2020-01-01 00:05:10 2020-01-02 03:33:31 2020-01-01 00:05:10.000 2020-01-02 03:33:31.000 310 99211 49760.5 4976050 310 99211 49760.5 4976050 -32259 32676 5196.02 519602 -128 127 -3.34 -334 +311 2 10301 99212 0.93393 297.93393 149.43393 14943.39339 0.93393 297.93393 149.43393 14943.39319 0.93393 297.93393 149.43393 14943.39300 2020-01-01 2020-01-02 2020-01-01 00:05:11 2020-01-02 03:33:32 2020-01-01 00:05:11.000 2020-01-02 03:33:32.000 311 99212 49761.5 4976150 311 99212 49761.5 4976150 -32258 32677 5197.02 519702 -128 123 -4.9 -490 +312 2 10302 99213 0.93693 297.93693 149.43693 14943.69369 0.93693 297.93695 149.43693 14943.69377 0.93693 297.93693 149.43693 14943.69300 2020-01-01 2020-01-02 2020-01-01 00:05:12 2020-01-02 03:33:33 2020-01-01 00:05:12.000 2020-01-02 03:33:33.000 312 99213 49762.5 4976250 312 99213 49762.5 4976250 -32257 32678 5198.02 519802 -127 124 -3.9 -390 +313 2 10303 99214 0.93993 297.93993 149.43993 14943.99399 0.93993 297.93994 149.43994 14943.99412 0.93993 297.93993 149.43993 14943.99300 2020-01-01 2020-01-02 2020-01-01 00:05:13 2020-01-02 03:33:34 2020-01-01 00:05:13.000 2020-01-02 03:33:34.000 313 99214 49763.5 4976350 313 99214 49763.5 4976350 -32256 32679 5199.02 519902 -126 125 -2.9 -290 +314 2 10304 99215 0.94294 297.94294 149.44294 14944.29429 0.94294 297.94293 149.44294 14944.29427 0.94294 297.94294 149.44294 14944.29400 2020-01-01 2020-01-02 2020-01-01 00:05:14 2020-01-02 03:33:35 2020-01-01 00:05:14.000 2020-01-02 03:33:35.000 314 99215 49764.5 4976450 314 99215 49764.5 4976450 -32255 32680 5200.02 520002 -125 126 -1.9 -190 +315 2 10305 99216 0.94594 297.94594 149.44594 14944.59459 0.94594 297.94595 149.44595 14944.595 0.94594 297.94594 149.44593999999998 14944.59400 2020-01-01 2020-01-02 2020-01-01 00:05:15 2020-01-02 03:33:36 2020-01-01 00:05:15.000 2020-01-02 03:33:36.000 315 99216 49765.5 4976550 315 99216 49765.5 4976550 -32254 32681 5201.02 520102 -124 127 -0.9 -90 +316 2 10306 99217 0.94894 297.94894 149.44894 14944.89489 0.94894 297.94894 149.44894 14944.89466 0.94894 297.94894 149.44894 14944.89400 2020-01-01 2020-01-02 2020-01-01 00:05:16 2020-01-02 03:33:37 2020-01-01 00:05:16.000 2020-01-02 03:33:37.000 316 99217 49766.5 4976650 316 99217 49766.5 4976650 -32253 32682 5202.02 520202 -128 127 -2.46 -246 +317 2 10307 99218 0.95195 297.95195 149.45195 14945.19519 0.95195 297.95197 149.45195 14945.19524 0.95195 297.95195 149.45195 14945.19500 2020-01-01 2020-01-02 2020-01-01 00:05:17 2020-01-02 03:33:38 2020-01-01 00:05:17.000 2020-01-02 03:33:38.000 317 99218 49767.5 4976750 317 99218 49767.5 4976750 -32252 32683 5203.02 520302 -128 123 -4.02 -402 +318 2 10308 99219 0.95495 297.95495 149.45495 14945.49549 0.95495 297.95496 149.45495 14945.49558 0.95495 297.95495 149.45495 14945.49500 2020-01-01 2020-01-02 2020-01-01 00:05:18 2020-01-02 03:33:39 2020-01-01 00:05:18.000 2020-01-02 03:33:39.000 318 99219 49768.5 4976850 318 99219 49768.5 4976850 -32251 32684 5204.02 520402 -127 124 -3.02 -302 319 2 10309 99220 0.95795 297.95795 149.45795 14945.79579 0.95795 297.95795 149.45795 14945.79574 0.95795 297.95795 149.45795 14945.79500 2020-01-01 2020-01-02 2020-01-01 00:05:19 2020-01-02 03:33:40 2020-01-01 00:05:19.000 2020-01-02 03:33:40.000 319 99220 49769.5 4976950 319 99220 49769.5 4976950 -32250 32685 5205.02 520502 -126 125 -2.02 -202 -32 2 10022 99932 0.09609 300.09609 150.09609 15159.7057 0.09609 300.0961 150.09609 15159.706 0.09609 300.09609 150.0960899999998 15159.70509 2020-01-01 2020-01-02 2020-01-01 00:00:32 2020-01-02 03:45:32 2020-01-01 00:00:32.000 2020-01-02 03:45:32.000 32 99932 49982 5048182 32 99932 49982 5048182 -32537 32398 4561.009900990099 460662 -126 125 -1.3663366336633664 -138 -320 2 10310 99221 0.96096 297.96096 149.46096 14946.09609 0.96096 297.96097 149.46096 14946.09647 0.96096 297.96096 149.46096000000014 14946.09600 2020-01-01 2020-01-02 2020-01-01 00:05:20 2020-01-02 03:33:41 2020-01-01 00:05:20.000 2020-01-02 03:33:41.000 320 99221 49770.5 4977050 320 99221 49770.5 4977050 -32249 32686 5206.02 520602 -125 126 -1.02 -102 -321 2 10311 99222 0.96396 297.96396 149.46396 14946.39639 0.96396 297.96396 149.46396 14946.39613 0.96396 297.96396 149.4639599999998 14946.39600 2020-01-01 2020-01-02 2020-01-01 00:05:21 2020-01-02 03:33:42 2020-01-01 00:05:21.000 2020-01-02 03:33:42.000 321 99222 49771.5 4977150 321 99222 49771.5 4977150 -32248 32687 5207.02 520702 -124 127 -0.02 -2 -322 2 10312 99223 0.96696 297.96696 149.46696 14946.69669 0.96696 297.96698 149.46696 14946.69674 0.96696 297.96696 149.4669599999999 14946.69600 2020-01-01 2020-01-02 2020-01-01 00:05:22 2020-01-02 03:33:43 2020-01-01 00:05:22.000 2020-01-02 03:33:43.000 322 99223 49772.5 4977250 322 99223 49772.5 4977250 -32247 32688 5208.02 520802 -128 127 -1.58 -158 -323 2 10313 99224 0.96996 297.96996 149.46996 14946.99699 0.96997 297.96997 149.46997 14946.99706 0.96996 297.96996 149.46996000000013 14946.99600 2020-01-01 2020-01-02 2020-01-01 00:05:23 2020-01-02 03:33:44 2020-01-01 00:05:23.000 2020-01-02 03:33:44.000 323 99224 49773.5 4977350 323 99224 49773.5 4977350 -32246 32689 5209.02 520902 -128 123 -3.14 -314 -324 2 10314 99225 0.97297 297.97297 149.47297 14947.29729 0.97297 297.97296 149.47297 14947.29737 0.97297 297.97297 149.4729700000001 14947.29700 2020-01-01 2020-01-02 2020-01-01 00:05:24 2020-01-02 03:33:45 2020-01-01 00:05:24.000 2020-01-02 03:33:45.000 324 99225 49774.5 4977450 324 99225 49774.5 4977450 -32245 32690 5210.02 521002 -127 124 -2.14 -214 +32 2 10022 99932 0.09609 300.09609 150.09609 15159.7057 0.09609 300.0961 150.09609 15159.706 0.09609 300.09609 150.09609 15159.70509 2020-01-01 2020-01-02 2020-01-01 00:00:32 2020-01-02 03:45:32 2020-01-01 00:00:32.000 2020-01-02 03:45:32.000 32 99932 49982 5048182 32 99932 49982 5048182 -32537 32398 4561.009900990099 460662 -126 125 -1.3663366336633664 -138 +320 2 10310 99221 0.96096 297.96096 149.46096 14946.09609 0.96096 297.96097 149.46096 14946.09647 0.96096 297.96096 149.46096 14946.09600 2020-01-01 2020-01-02 2020-01-01 00:05:20 2020-01-02 03:33:41 2020-01-01 00:05:20.000 2020-01-02 03:33:41.000 320 99221 49770.5 4977050 320 99221 49770.5 4977050 -32249 32686 5206.02 520602 -125 126 -1.02 -102 +321 2 10311 99222 0.96396 297.96396 149.46396 14946.39639 0.96396 297.96396 149.46396 14946.39613 0.96396 297.96396 149.46396000000001 14946.39600 2020-01-01 2020-01-02 2020-01-01 00:05:21 2020-01-02 03:33:42 2020-01-01 00:05:21.000 2020-01-02 03:33:42.000 321 99222 49771.5 4977150 321 99222 49771.5 4977150 -32248 32687 5207.02 520702 -124 127 -0.02 -2 +322 2 10312 99223 0.96696 297.96696 149.46696 14946.69669 0.96696 297.96698 149.46696 14946.69674 0.96696 297.96696 149.46696 14946.69600 2020-01-01 2020-01-02 2020-01-01 00:05:22 2020-01-02 03:33:43 2020-01-01 00:05:22.000 2020-01-02 03:33:43.000 322 99223 49772.5 4977250 322 99223 49772.5 4977250 -32247 32688 5208.02 520802 -128 127 -1.58 -158 +323 2 10313 99224 0.96996 297.96996 149.46996 14946.99699 0.96997 297.96997 149.46997 14946.99706 0.96996 297.96996 149.46996 14946.99600 2020-01-01 2020-01-02 2020-01-01 00:05:23 2020-01-02 03:33:44 2020-01-01 00:05:23.000 2020-01-02 03:33:44.000 323 99224 49773.5 4977350 323 99224 49773.5 4977350 -32246 32689 5209.02 520902 -128 123 -3.14 -314 +324 2 10314 99225 0.97297 297.97297 149.47297 14947.29729 0.97297 297.97296 149.47297 14947.29737 0.97297 297.97297 149.47297 14947.29700 2020-01-01 2020-01-02 2020-01-01 00:05:24 2020-01-02 03:33:45 2020-01-01 00:05:24.000 2020-01-02 03:33:45.000 324 99225 49774.5 4977450 324 99225 49774.5 4977450 -32245 32690 5210.02 521002 -127 124 -2.14 -214 325 2 10315 99226 0.97597 297.97597 149.47597 14947.59759 0.97597 297.97598 149.47597 14947.59794 0.97597 297.97597 149.47597 14947.59700 2020-01-01 2020-01-02 2020-01-01 00:05:25 2020-01-02 03:33:46 2020-01-01 00:05:25.000 2020-01-02 03:33:46.000 325 99226 49775.5 4977550 325 99226 49775.5 4977550 -32244 32691 5211.02 521102 -126 125 -1.14 -114 -326 2 10316 99227 0.97897 297.97897 149.47897 14947.89789 0.97897 297.97897 149.47897 14947.8976 0.97897 297.97897 149.4789700000001 14947.89700 2020-01-01 2020-01-02 2020-01-01 00:05:26 2020-01-02 03:33:47 2020-01-01 00:05:26.000 2020-01-02 03:33:47.000 326 99227 49776.5 4977650 326 99227 49776.5 4977650 -32243 32692 5212.02 521202 -125 126 -0.14 -14 +326 2 10316 99227 0.97897 297.97897 149.47897 14947.89789 0.97897 297.97897 149.47897 14947.8976 0.97897 297.97897 149.47897 14947.89700 2020-01-01 2020-01-02 2020-01-01 00:05:26 2020-01-02 03:33:47 2020-01-01 00:05:26.000 2020-01-02 03:33:47.000 326 99227 49776.5 4977650 326 99227 49776.5 4977650 -32243 32692 5212.02 521202 -125 126 -0.14 -14 327 2 10317 99228 0.98198 297.98198 149.48198 14948.19819 0.98198 297.982 149.48198 14948.19821 0.98198 297.98198 149.48198 14948.19800 2020-01-01 2020-01-02 2020-01-01 00:05:27 2020-01-02 03:33:48 2020-01-01 00:05:27.000 2020-01-02 03:33:48.000 327 99228 49777.5 4977750 327 99228 49777.5 4977750 -32242 32693 5213.02 521302 -124 127 0.86 86 -328 2 10318 99229 0.98498 297.98498 149.48498 14948.49849 0.98498 297.985 149.48498 14948.49853 0.98498 297.98498 149.4849799999999 14948.49800 2020-01-01 2020-01-02 2020-01-01 00:05:28 2020-01-02 03:33:49 2020-01-01 00:05:28.000 2020-01-02 03:33:49.000 328 99229 49778.5 4977850 328 99229 49778.5 4977850 -32241 32694 5214.02 521402 -128 127 -0.7 -70 -329 2 10319 99230 0.98798 297.98798 149.48798 14948.79879 0.98798 297.98798 149.48798 14948.79883 0.98798 297.98798 149.4879799999999 14948.79800 2020-01-01 2020-01-02 2020-01-01 00:05:29 2020-01-02 03:33:50 2020-01-01 00:05:29.000 2020-01-02 03:33:50.000 329 99230 49779.5 4977950 329 99230 49779.5 4977950 -32240 32695 5215.02 521502 -128 127 -2.26 -226 -33 2 10023 99933 0.09909 300.09909 150.09909 15160.009 0.09909 300.0991 150.0991 15160.00913 0.09909 300.09909 150.09908999999988 15160.00809 2020-01-01 2020-01-02 2020-01-01 00:00:33 2020-01-02 03:45:33 2020-01-01 00:00:33.000 2020-01-02 03:45:33.000 33 99933 49983 5048283 33 99933 49983 5048283 -32536 32399 4562.009900990099 460763 -125 126 -0.36633663366336633 -37 -330 2 10320 99231 0.99099 297.99099 149.49099 14949.09909 0.99099 297.991 149.49099 14949.09941 0.99099 297.99099 149.49099000000012 14949.09900 2020-01-01 2020-01-02 2020-01-01 00:05:30 2020-01-02 03:33:51 2020-01-01 00:05:30.000 2020-01-02 03:33:51.000 330 99231 49780.5 4978050 330 99231 49780.5 4978050 -32239 32696 5216.02 521602 -128 123 -3.82 -382 -331 2 10321 99232 0.99399 297.99399 149.49399 14949.39939 0.99399 297.994 149.49399 14949.39911 0.99399 297.99399 149.49399000000017 14949.39900 2020-01-01 2020-01-02 2020-01-01 00:05:31 2020-01-02 03:33:52 2020-01-01 00:05:31.000 2020-01-02 03:33:52.000 331 99232 49781.5 4978150 331 99232 49781.5 4978150 -32238 32697 5217.02 521702 -127 124 -2.82 -282 -332 2 10322 99233 0.99699 297.99699 149.49699 14949.69969 0.99699 297.997 149.49699 14949.69969 0.99699 297.99699 149.4969899999998 14949.69900 2020-01-01 2020-01-02 2020-01-01 00:05:32 2020-01-02 03:33:53 2020-01-01 00:05:32.000 2020-01-02 03:33:53.000 332 99233 49782.5 4978250 332 99233 49782.5 4978250 -32237 32698 5218.02 521802 -126 125 -1.82 -182 +328 2 10318 99229 0.98498 297.98498 149.48498 14948.49849 0.98498 297.985 149.48498 14948.49853 0.98498 297.98498 149.48498 14948.49800 2020-01-01 2020-01-02 2020-01-01 00:05:28 2020-01-02 03:33:49 2020-01-01 00:05:28.000 2020-01-02 03:33:49.000 328 99229 49778.5 4977850 328 99229 49778.5 4977850 -32241 32694 5214.02 521402 -128 127 -0.7 -70 +329 2 10319 99230 0.98798 297.98798 149.48798 14948.79879 0.98798 297.98798 149.48798 14948.79883 0.98798 297.98798 149.48798 14948.79800 2020-01-01 2020-01-02 2020-01-01 00:05:29 2020-01-02 03:33:50 2020-01-01 00:05:29.000 2020-01-02 03:33:50.000 329 99230 49779.5 4977950 329 99230 49779.5 4977950 -32240 32695 5215.02 521502 -128 127 -2.26 -226 +33 2 10023 99933 0.09909 300.09909 150.09909 15160.009 0.09909 300.0991 150.0991 15160.00913 0.09909 300.09909 150.09909 15160.00809 2020-01-01 2020-01-02 2020-01-01 00:00:33 2020-01-02 03:45:33 2020-01-01 00:00:33.000 2020-01-02 03:45:33.000 33 99933 49983 5048283 33 99933 49983 5048283 -32536 32399 4562.009900990099 460763 -125 126 -0.36633663366336633 -37 +330 2 10320 99231 0.99099 297.99099 149.49099 14949.09909 0.99099 297.991 149.49099 14949.09941 0.99099 297.99099 149.49099 14949.09900 2020-01-01 2020-01-02 2020-01-01 00:05:30 2020-01-02 03:33:51 2020-01-01 00:05:30.000 2020-01-02 03:33:51.000 330 99231 49780.5 4978050 330 99231 49780.5 4978050 -32239 32696 5216.02 521602 -128 123 -3.82 -382 +331 2 10321 99232 0.99399 297.99399 149.49399 14949.39939 0.99399 297.994 149.49399 14949.39911 0.99399 297.99399 149.49399 14949.39900 2020-01-01 2020-01-02 2020-01-01 00:05:31 2020-01-02 03:33:52 2020-01-01 00:05:31.000 2020-01-02 03:33:52.000 331 99232 49781.5 4978150 331 99232 49781.5 4978150 -32238 32697 5217.02 521702 -127 124 -2.82 -282 +332 2 10322 99233 0.99699 297.99699 149.49699 14949.69969 0.99699 297.997 149.49699 14949.69969 0.99699 297.99699 149.49699 14949.69900 2020-01-01 2020-01-02 2020-01-01 00:05:32 2020-01-02 03:33:53 2020-01-01 00:05:32.000 2020-01-02 03:33:53.000 332 99233 49782.5 4978250 332 99233 49782.5 4978250 -32237 32698 5218.02 521802 -126 125 -1.82 -182 333 2 10323 99234 1 298 149.5 14950 1 298 149.5 14950 1.00000 298.00000 149.5 14950.00000 2020-01-01 2020-01-02 2020-01-01 00:05:33 2020-01-02 03:33:54 2020-01-01 00:05:33.000 2020-01-02 03:33:54.000 333 99234 49783.5 4978350 333 99234 49783.5 4978350 -32236 32699 5219.02 521902 -125 126 -0.82 -82 -334 2 10324 99235 1.003 298.003 149.503 14950.3003 1.003 298.003 149.503 14950.30029 1.00300 298.00300 149.50300000000007 14950.30000 2020-01-01 2020-01-02 2020-01-01 00:05:34 2020-01-02 03:33:55 2020-01-01 00:05:34.000 2020-01-02 03:33:55.000 334 99235 49784.5 4978450 334 99235 49784.5 4978450 -32235 32700 5220.02 522002 -124 127 0.18 18 -335 2 10325 99236 1.006 298.006 149.506 14950.6006 1.006 298.006 149.506 14950.60088 1.00600 298.00600 149.50599999999991 14950.60000 2020-01-01 2020-01-02 2020-01-01 00:05:35 2020-01-02 03:33:56 2020-01-01 00:05:35.000 2020-01-02 03:33:56.000 335 99236 49785.5 4978550 335 99236 49785.5 4978550 -32234 32701 5221.02 522102 -128 127 -1.38 -138 -336 2 10326 99237 1.009 298.009 149.509 14950.9009 1.009 298.009 149.509 14950.90057 1.00900 298.00900 149.50900000000001 14950.90000 2020-01-01 2020-01-02 2020-01-01 00:05:36 2020-01-02 03:33:57 2020-01-01 00:05:36.000 2020-01-02 03:33:57.000 336 99237 49786.5 4978650 336 99237 49786.5 4978650 -32233 32702 5222.02 522202 -128 123 -2.94 -294 -337 2 10327 99238 1.01201 298.01201 149.51201 14951.2012 1.01201 298.01202 149.51201 14951.20117 1.01201 298.01201 149.51201000000017 14951.20100 2020-01-01 2020-01-02 2020-01-01 00:05:37 2020-01-02 03:33:58 2020-01-01 00:05:37.000 2020-01-02 03:33:58.000 337 99238 49787.5 4978750 337 99238 49787.5 4978750 -32232 32703 5223.02 522302 -127 124 -1.94 -194 -338 2 10328 99239 1.01501 298.01501 149.51501 14951.5015 1.01501 298.015 149.51501 14951.50146 1.01501 298.01501 149.51500999999982 14951.50100 2020-01-01 2020-01-02 2020-01-01 00:05:38 2020-01-02 03:33:59 2020-01-01 00:05:38.000 2020-01-02 03:33:59.000 338 99239 49788.5 4978850 338 99239 49788.5 4978850 -32231 32704 5224.02 522402 -126 125 -0.94 -94 -339 2 10329 99240 1.01801 298.01801 149.51801 14951.8018 1.01801 298.018 149.51801 14951.80177 1.01801 298.01801 149.51800999999983 14951.80100 2020-01-01 2020-01-02 2020-01-01 00:05:39 2020-01-02 03:34:00 2020-01-01 00:05:39.000 2020-01-02 03:34:00.000 339 99240 49789.5 4978950 339 99240 49789.5 4978950 -32230 32705 5225.02 522502 -125 126 0.06 6 -34 2 10024 99934 0.1021 300.1021 150.1021 15160.31231 0.1021 300.1021 150.1021 15160.31224 0.10210 300.10210 150.10210000000004 15160.31210 2020-01-01 2020-01-02 2020-01-01 00:00:34 2020-01-02 03:45:34 2020-01-01 00:00:34.000 2020-01-02 03:45:34.000 34 99934 49984 5048384 34 99934 49984 5048384 -32535 32400 4563.009900990099 460864 -124 127 0.6336633663366337 64 -340 2 10330 99241 1.02102 298.02102 149.52102 14952.1021 1.02102 298.02103 149.52102 14952.10239 1.02102 298.02102 149.5210200000001 14952.10200 2020-01-01 2020-01-02 2020-01-01 00:05:40 2020-01-02 03:34:01 2020-01-01 00:05:40.000 2020-01-02 03:34:01.000 340 99241 49790.5 4979050 340 99241 49790.5 4979050 -32229 32706 5226.02 522602 -124 127 1.06 106 -341 2 10331 99242 1.02402 298.02402 149.52402 14952.4024 1.02402 298.02402 149.52402 14952.40205 1.02402 298.02402 149.52402000000006 14952.40200 2020-01-01 2020-01-02 2020-01-01 00:05:41 2020-01-02 03:34:02 2020-01-01 00:05:41.000 2020-01-02 03:34:02.000 341 99242 49791.5 4979150 341 99242 49791.5 4979150 -32228 32707 5227.02 522702 -128 127 -0.5 -50 -342 2 10332 99243 1.02702 298.02702 149.52702 14952.7027 1.02702 298.02704 149.52702 14952.70264 1.02702 298.02702 149.52701999999982 14952.70200 2020-01-01 2020-01-02 2020-01-01 00:05:42 2020-01-02 03:34:03 2020-01-01 00:05:42.000 2020-01-02 03:34:03.000 342 99243 49792.5 4979250 342 99243 49792.5 4979250 -32227 32708 5228.02 522802 -128 123 -2.06 -206 -343 2 10333 99244 1.03003 298.03003 149.53003 14953.003 1.03003 298.03003 149.53002 14953.00293 1.03003 298.03003 149.53002999999993 14953.00300 2020-01-01 2020-01-02 2020-01-01 00:05:43 2020-01-02 03:34:04 2020-01-01 00:05:43.000 2020-01-02 03:34:04.000 343 99244 49793.5 4979350 343 99244 49793.5 4979350 -32226 32709 5229.02 522902 -127 124 -1.06 -106 -344 2 10334 99245 1.03303 298.03303 149.53303 14953.3033 1.03303 298.03302 149.53303 14953.30323 1.03303 298.03303 149.53303000000002 14953.30300 2020-01-01 2020-01-02 2020-01-01 00:05:44 2020-01-02 03:34:05 2020-01-01 00:05:44.000 2020-01-02 03:34:05.000 344 99245 49794.5 4979450 344 99245 49794.5 4979450 -32225 32710 5230.02 523002 -126 125 -0.06 -6 -345 2 10335 99246 1.03603 298.03603 149.53603 14953.6036 1.03603 298.03604 149.53603 14953.60386 1.03603 298.03603 149.53602999999993 14953.60300 2020-01-01 2020-01-02 2020-01-01 00:05:45 2020-01-02 03:34:06 2020-01-01 00:05:45.000 2020-01-02 03:34:06.000 345 99246 49795.5 4979550 345 99246 49795.5 4979550 -32224 32711 5231.02 523102 -125 126 0.94 94 -346 2 10336 99247 1.03903 298.03903 149.53903 14953.9039 1.03903 298.03903 149.53903 14953.90352 1.03903 298.03903 149.53902999999994 14953.90300 2020-01-01 2020-01-02 2020-01-01 00:05:46 2020-01-02 03:34:07 2020-01-01 00:05:46.000 2020-01-02 03:34:07.000 346 99247 49796.5 4979650 346 99247 49796.5 4979650 -32223 32712 5232.02 523202 -124 127 1.94 194 -347 2 10337 99248 1.04204 298.04204 149.54204 14954.2042 1.04204 298.04205 149.54204 14954.20427 1.04204 298.04204 149.5420400000001 14954.20400 2020-01-01 2020-01-02 2020-01-01 00:05:47 2020-01-02 03:34:08 2020-01-01 00:05:47.000 2020-01-02 03:34:08.000 347 99248 49797.5 4979750 347 99248 49797.5 4979750 -32222 32713 5233.02 523302 -128 127 0.38 38 -348 2 10338 99249 1.04504 298.04504 149.54504 14954.5045 1.04504 298.04504 149.54504 14954.50441 1.04504 298.04504 149.5450400000002 14954.50400 2020-01-01 2020-01-02 2020-01-01 00:05:48 2020-01-02 03:34:09 2020-01-01 00:05:48.000 2020-01-02 03:34:09.000 348 99249 49798.5 4979850 348 99249 49798.5 4979850 -32221 32714 5234.02 523402 -128 123 -1.18 -118 -349 2 10339 99250 1.04804 298.04804 149.54804 14954.8048 1.04804 298.04803 149.54804 14954.80474 1.04804 298.04804 149.54803999999987 14954.80400 2020-01-01 2020-01-02 2020-01-01 00:05:49 2020-01-02 03:34:10 2020-01-01 00:05:49.000 2020-01-02 03:34:10.000 349 99250 49799.5 4979950 349 99250 49799.5 4979950 -32220 32715 5235.02 523502 -127 124 -0.18 -18 -35 2 10025 99935 0.1051 300.1051 150.1051 15160.61561 0.1051 300.1051 150.1051 15160.61542 0.10510 300.10510 150.10510000000008 15160.61510 2020-01-01 2020-01-02 2020-01-01 00:00:35 2020-01-02 03:45:35 2020-01-01 00:00:35.000 2020-01-02 03:45:35.000 35 99935 49985 5048485 35 99935 49985 5048485 -32534 32401 4564.009900990099 460965 -128 127 -0.900990099009901 -91 +334 2 10324 99235 1.003 298.003 149.503 14950.3003 1.003 298.003 149.503 14950.30029 1.00300 298.00300 149.503 14950.30000 2020-01-01 2020-01-02 2020-01-01 00:05:34 2020-01-02 03:33:55 2020-01-01 00:05:34.000 2020-01-02 03:33:55.000 334 99235 49784.5 4978450 334 99235 49784.5 4978450 -32235 32700 5220.02 522002 -124 127 0.18 18 +335 2 10325 99236 1.006 298.006 149.506 14950.6006 1.006 298.006 149.506 14950.60088 1.00600 298.00600 149.506 14950.60000 2020-01-01 2020-01-02 2020-01-01 00:05:35 2020-01-02 03:33:56 2020-01-01 00:05:35.000 2020-01-02 03:33:56.000 335 99236 49785.5 4978550 335 99236 49785.5 4978550 -32234 32701 5221.02 522102 -128 127 -1.38 -138 +336 2 10326 99237 1.009 298.009 149.509 14950.9009 1.009 298.009 149.509 14950.90057 1.00900 298.00900 149.509 14950.90000 2020-01-01 2020-01-02 2020-01-01 00:05:36 2020-01-02 03:33:57 2020-01-01 00:05:36.000 2020-01-02 03:33:57.000 336 99237 49786.5 4978650 336 99237 49786.5 4978650 -32233 32702 5222.02 522202 -128 123 -2.94 -294 +337 2 10327 99238 1.01201 298.01201 149.51201 14951.2012 1.01201 298.01202 149.51201 14951.20117 1.01201 298.01201 149.51201 14951.20100 2020-01-01 2020-01-02 2020-01-01 00:05:37 2020-01-02 03:33:58 2020-01-01 00:05:37.000 2020-01-02 03:33:58.000 337 99238 49787.5 4978750 337 99238 49787.5 4978750 -32232 32703 5223.02 522302 -127 124 -1.94 -194 +338 2 10328 99239 1.01501 298.01501 149.51501 14951.5015 1.01501 298.015 149.51501 14951.50146 1.01501 298.01501 149.51501 14951.50100 2020-01-01 2020-01-02 2020-01-01 00:05:38 2020-01-02 03:33:59 2020-01-01 00:05:38.000 2020-01-02 03:33:59.000 338 99239 49788.5 4978850 338 99239 49788.5 4978850 -32231 32704 5224.02 522402 -126 125 -0.94 -94 +339 2 10329 99240 1.01801 298.01801 149.51801 14951.8018 1.01801 298.018 149.51801 14951.80177 1.01801 298.01801 149.51801 14951.80100 2020-01-01 2020-01-02 2020-01-01 00:05:39 2020-01-02 03:34:00 2020-01-01 00:05:39.000 2020-01-02 03:34:00.000 339 99240 49789.5 4978950 339 99240 49789.5 4978950 -32230 32705 5225.02 522502 -125 126 0.06 6 +34 2 10024 99934 0.1021 300.1021 150.1021 15160.31231 0.1021 300.1021 150.1021 15160.31224 0.10210 300.10210 150.10209999999998 15160.31210 2020-01-01 2020-01-02 2020-01-01 00:00:34 2020-01-02 03:45:34 2020-01-01 00:00:34.000 2020-01-02 03:45:34.000 34 99934 49984 5048384 34 99934 49984 5048384 -32535 32400 4563.009900990099 460864 -124 127 0.6336633663366337 64 +340 2 10330 99241 1.02102 298.02102 149.52102 14952.1021 1.02102 298.02103 149.52102 14952.10239 1.02102 298.02102 149.52102000000002 14952.10200 2020-01-01 2020-01-02 2020-01-01 00:05:40 2020-01-02 03:34:01 2020-01-01 00:05:40.000 2020-01-02 03:34:01.000 340 99241 49790.5 4979050 340 99241 49790.5 4979050 -32229 32706 5226.02 522602 -124 127 1.06 106 +341 2 10331 99242 1.02402 298.02402 149.52402 14952.4024 1.02402 298.02402 149.52402 14952.40205 1.02402 298.02402 149.52402 14952.40200 2020-01-01 2020-01-02 2020-01-01 00:05:41 2020-01-02 03:34:02 2020-01-01 00:05:41.000 2020-01-02 03:34:02.000 341 99242 49791.5 4979150 341 99242 49791.5 4979150 -32228 32707 5227.02 522702 -128 127 -0.5 -50 +342 2 10332 99243 1.02702 298.02702 149.52702 14952.7027 1.02702 298.02704 149.52702 14952.70264 1.02702 298.02702 149.52702 14952.70200 2020-01-01 2020-01-02 2020-01-01 00:05:42 2020-01-02 03:34:03 2020-01-01 00:05:42.000 2020-01-02 03:34:03.000 342 99243 49792.5 4979250 342 99243 49792.5 4979250 -32227 32708 5228.02 522802 -128 123 -2.06 -206 +343 2 10333 99244 1.03003 298.03003 149.53003 14953.003 1.03003 298.03003 149.53002 14953.00293 1.03003 298.03003 149.53003 14953.00300 2020-01-01 2020-01-02 2020-01-01 00:05:43 2020-01-02 03:34:04 2020-01-01 00:05:43.000 2020-01-02 03:34:04.000 343 99244 49793.5 4979350 343 99244 49793.5 4979350 -32226 32709 5229.02 522902 -127 124 -1.06 -106 +344 2 10334 99245 1.03303 298.03303 149.53303 14953.3033 1.03303 298.03302 149.53303 14953.30323 1.03303 298.03303 149.53303 14953.30300 2020-01-01 2020-01-02 2020-01-01 00:05:44 2020-01-02 03:34:05 2020-01-01 00:05:44.000 2020-01-02 03:34:05.000 344 99245 49794.5 4979450 344 99245 49794.5 4979450 -32225 32710 5230.02 523002 -126 125 -0.06 -6 +345 2 10335 99246 1.03603 298.03603 149.53603 14953.6036 1.03603 298.03604 149.53603 14953.60386 1.03603 298.03603 149.53602999999998 14953.60300 2020-01-01 2020-01-02 2020-01-01 00:05:45 2020-01-02 03:34:06 2020-01-01 00:05:45.000 2020-01-02 03:34:06.000 345 99246 49795.5 4979550 345 99246 49795.5 4979550 -32224 32711 5231.02 523102 -125 126 0.94 94 +346 2 10336 99247 1.03903 298.03903 149.53903 14953.9039 1.03903 298.03903 149.53903 14953.90352 1.03903 298.03903 149.53903 14953.90300 2020-01-01 2020-01-02 2020-01-01 00:05:46 2020-01-02 03:34:07 2020-01-01 00:05:46.000 2020-01-02 03:34:07.000 346 99247 49796.5 4979650 346 99247 49796.5 4979650 -32223 32712 5232.02 523202 -124 127 1.94 194 +347 2 10337 99248 1.04204 298.04204 149.54204 14954.2042 1.04204 298.04205 149.54204 14954.20427 1.04204 298.04204 149.54204 14954.20400 2020-01-01 2020-01-02 2020-01-01 00:05:47 2020-01-02 03:34:08 2020-01-01 00:05:47.000 2020-01-02 03:34:08.000 347 99248 49797.5 4979750 347 99248 49797.5 4979750 -32222 32713 5233.02 523302 -128 127 0.38 38 +348 2 10338 99249 1.04504 298.04504 149.54504 14954.5045 1.04504 298.04504 149.54504 14954.50441 1.04504 298.04504 149.54504 14954.50400 2020-01-01 2020-01-02 2020-01-01 00:05:48 2020-01-02 03:34:09 2020-01-01 00:05:48.000 2020-01-02 03:34:09.000 348 99249 49798.5 4979850 348 99249 49798.5 4979850 -32221 32714 5234.02 523402 -128 123 -1.18 -118 +349 2 10339 99250 1.04804 298.04804 149.54804 14954.8048 1.04804 298.04803 149.54804 14954.80474 1.04804 298.04804 149.54804000000001 14954.80400 2020-01-01 2020-01-02 2020-01-01 00:05:49 2020-01-02 03:34:10 2020-01-01 00:05:49.000 2020-01-02 03:34:10.000 349 99250 49799.5 4979950 349 99250 49799.5 4979950 -32220 32715 5235.02 523502 -127 124 -0.18 -18 +35 2 10025 99935 0.1051 300.1051 150.1051 15160.61561 0.1051 300.1051 150.1051 15160.61542 0.10510 300.10510 150.10510000000002 15160.61510 2020-01-01 2020-01-02 2020-01-01 00:00:35 2020-01-02 03:45:35 2020-01-01 00:00:35.000 2020-01-02 03:45:35.000 35 99935 49985 5048485 35 99935 49985 5048485 -32534 32401 4564.009900990099 460965 -128 127 -0.900990099009901 -91 350 2 10340 99251 1.05105 298.05105 149.55105 14955.1051 1.05105 298.05106 149.55105 14955.10532 1.05105 298.05105 149.55105 14955.10500 2020-01-01 2020-01-02 2020-01-01 00:05:50 2020-01-02 03:34:11 2020-01-01 00:05:50.000 2020-01-02 03:34:11.000 350 99251 49800.5 4980050 350 99251 49800.5 4980050 -32219 32716 5236.02 523602 -126 125 0.82 82 -351 2 10341 99252 1.05405 298.05405 149.55405 14955.4054 1.05405 298.05405 149.55404 14955.40499 1.05405 298.05405 149.55405000000007 14955.40500 2020-01-01 2020-01-02 2020-01-01 00:05:51 2020-01-02 03:34:12 2020-01-01 00:05:51.000 2020-01-02 03:34:12.000 351 99252 49801.5 4980150 351 99252 49801.5 4980150 -32218 32717 5237.02 523702 -125 126 1.82 182 -352 2 10342 99253 1.05705 298.05705 149.55705 14955.7057 1.05705 298.05707 149.55705 14955.70574 1.05705 298.05705 149.55704999999998 14955.70500 2020-01-01 2020-01-02 2020-01-01 00:05:52 2020-01-02 03:34:13 2020-01-01 00:05:52.000 2020-01-02 03:34:13.000 352 99253 49802.5 4980250 352 99253 49802.5 4980250 -32217 32718 5238.02 523802 -124 127 2.82 282 -353 2 10343 99254 1.06006 298.06006 149.56006 14956.006 1.06006 298.06006 149.56005 14956.00587 1.06006 298.06006 149.56005999999985 14956.00600 2020-01-01 2020-01-02 2020-01-01 00:05:53 2020-01-02 03:34:14 2020-01-01 00:05:53.000 2020-01-02 03:34:14.000 353 99254 49803.5 4980350 353 99254 49803.5 4980350 -32216 32719 5239.02 523902 -128 127 1.26 126 -354 2 10344 99255 1.06306 298.06306 149.56306 14956.3063 1.06306 298.06305 149.56306 14956.3062 1.06306 298.06306 149.5630600000002 14956.30600 2020-01-01 2020-01-02 2020-01-01 00:05:54 2020-01-02 03:34:15 2020-01-01 00:05:54.000 2020-01-02 03:34:15.000 354 99255 49804.5 4980450 354 99255 49804.5 4980450 -32215 32720 5240.02 524002 -128 127 -0.3 -30 -355 2 10345 99256 1.06606 298.06606 149.56606 14956.6066 1.06606 298.06607 149.56606 14956.6068 1.06606 298.06606 149.56605999999988 14956.60600 2020-01-01 2020-01-02 2020-01-01 00:05:55 2020-01-02 03:34:16 2020-01-01 00:05:55.000 2020-01-02 03:34:16.000 355 99256 49805.5 4980550 355 99256 49805.5 4980550 -32214 32721 5241.02 524102 -128 123 -1.86 -186 -356 2 10346 99257 1.06906 298.06906 149.56906 14956.9069 1.06906 298.06906 149.56907 14956.90709 1.06906 298.06906 149.56905999999987 14956.90600 2020-01-01 2020-01-02 2020-01-01 00:05:56 2020-01-02 03:34:17 2020-01-01 00:05:56.000 2020-01-02 03:34:17.000 356 99257 49806.5 4980650 356 99257 49806.5 4980650 -32213 32722 5242.02 524202 -127 124 -0.86 -86 -357 2 10347 99258 1.07207 298.07207 149.57207 14957.2072 1.07207 298.07208 149.57207 14957.20721 1.07207 298.07207 149.5720700000001 14957.20700 2020-01-01 2020-01-02 2020-01-01 00:05:57 2020-01-02 03:34:18 2020-01-01 00:05:57.000 2020-01-02 03:34:18.000 357 99258 49807.5 4980750 357 99258 49807.5 4980750 -32212 32723 5243.02 524302 -126 125 0.14 14 -358 2 10348 99259 1.07507 298.07507 149.57507 14957.5075 1.07507 298.07507 149.57507 14957.50734 1.07507 298.07507 149.57507000000012 14957.50700 2020-01-01 2020-01-02 2020-01-01 00:05:58 2020-01-02 03:34:19 2020-01-01 00:05:58.000 2020-01-02 03:34:19.000 358 99259 49808.5 4980850 358 99259 49808.5 4980850 -32211 32724 5244.02 524402 -125 126 1.14 114 -359 2 10349 99260 1.07807 298.07807 149.57807 14957.8078 1.07807 298.07806 149.57807 14957.80767 1.07807 298.07807 149.57806999999977 14957.80700 2020-01-01 2020-01-02 2020-01-01 00:05:59 2020-01-02 03:34:20 2020-01-01 00:05:59.000 2020-01-02 03:34:20.000 359 99260 49809.5 4980950 359 99260 49809.5 4980950 -32210 32725 5245.02 524502 -124 127 2.14 214 -36 2 10026 99936 0.1081 300.1081 150.1081 15160.91891 0.1081 300.1081 150.1081 15160.91873 0.10810 300.10810 150.10809999999995 15160.91810 2020-01-01 2020-01-02 2020-01-01 00:00:36 2020-01-02 03:45:36 2020-01-01 00:00:36.000 2020-01-02 03:45:36.000 36 99936 49986 5048586 36 99936 49986 5048586 -32533 32402 4565.009900990099 461066 -128 123 -2.4356435643564356 -246 -360 2 10350 99261 1.08108 298.08108 149.58108 14958.1081 1.08108 298.0811 149.58108 14958.10827 1.08108 298.08108 149.58107999999996 14958.10800 2020-01-01 2020-01-02 2020-01-01 00:06:00 2020-01-02 03:34:21 2020-01-01 00:06:00.000 2020-01-02 03:34:21.000 360 99261 49810.5 4981050 360 99261 49810.5 4981050 -32209 32726 5246.02 524602 -128 127 0.58 58 -361 2 10351 99262 1.08408 298.08408 149.58408 14958.4084 1.08408 298.08408 149.58408 14958.40856 1.08408 298.08408 149.58408000000006 14958.40800 2020-01-01 2020-01-02 2020-01-01 00:06:01 2020-01-02 03:34:22 2020-01-01 00:06:01.000 2020-01-02 03:34:22.000 361 99262 49811.5 4981150 361 99262 49811.5 4981150 -32208 32727 5247.02 524702 -128 123 -0.98 -98 -362 2 10352 99263 1.08708 298.08708 149.58708 14958.7087 1.08708 298.0871 149.58708 14958.70868 1.08708 298.08708 149.5870799999999 14958.70800 2020-01-01 2020-01-02 2020-01-01 00:06:02 2020-01-02 03:34:23 2020-01-01 00:06:02.000 2020-01-02 03:34:23.000 362 99263 49812.5 4981250 362 99263 49812.5 4981250 -32207 32728 5248.02 524802 -127 124 0.02 2 -363 2 10353 99264 1.09009 298.09009 149.59009 14959.009 1.09009 298.0901 149.59008 14959.00884 1.09009 298.09009 149.59008999999983 14959.00900 2020-01-01 2020-01-02 2020-01-01 00:06:03 2020-01-02 03:34:24 2020-01-01 00:06:03.000 2020-01-02 03:34:24.000 363 99264 49813.5 4981350 363 99264 49813.5 4981350 -32206 32729 5249.02 524902 -126 125 1.02 102 -364 2 10354 99265 1.09309 298.09309 149.59309 14959.3093 1.09309 298.09308 149.59309 14959.30915 1.09309 298.09309 149.59309000000013 14959.30900 2020-01-01 2020-01-02 2020-01-01 00:06:04 2020-01-02 03:34:25 2020-01-01 00:06:04.000 2020-01-02 03:34:25.000 364 99265 49814.5 4981450 364 99265 49814.5 4981450 -32205 32730 5250.02 525002 -125 126 2.02 202 -365 2 10355 99266 1.09609 298.09609 149.59609 14959.6096 1.09609 298.0961 149.59609 14959.6099 1.09609 298.09609 149.59608999999978 14959.60900 2020-01-01 2020-01-02 2020-01-01 00:06:05 2020-01-02 03:34:26 2020-01-01 00:06:05.000 2020-01-02 03:34:26.000 365 99266 49815.5 4981550 365 99266 49815.5 4981550 -32204 32731 5251.02 525102 -124 127 3.02 302 -366 2 10356 99267 1.09909 298.09909 149.59909 14959.9099 1.09909 298.0991 149.5991 14959.91003 1.09909 298.09909 149.59908999999985 14959.90900 2020-01-01 2020-01-02 2020-01-01 00:06:06 2020-01-02 03:34:27 2020-01-01 00:06:06.000 2020-01-02 03:34:27.000 366 99267 49816.5 4981650 366 99267 49816.5 4981650 -32203 32732 5252.02 525202 -128 127 1.46 146 -367 2 10357 99268 1.1021 298.1021 149.6021 14960.21021 1.1021 298.1021 149.6021 14960.21015 1.10210 298.10210 149.60210000000004 14960.21000 2020-01-01 2020-01-02 2020-01-01 00:06:07 2020-01-02 03:34:28 2020-01-01 00:06:07.000 2020-01-02 03:34:28.000 367 99268 49817.5 4981750 367 99268 49817.5 4981750 -32202 32733 5253.02 525302 -128 123 -0.1 -10 -368 2 10358 99269 1.1051 298.1051 149.6051 14960.51051 1.1051 298.1051 149.6051 14960.51031 1.10510 298.10510 149.6051000000001 14960.51000 2020-01-01 2020-01-02 2020-01-01 00:06:08 2020-01-02 03:34:29 2020-01-01 00:06:08.000 2020-01-02 03:34:29.000 368 99269 49818.5 4981850 368 99269 49818.5 4981850 -32201 32734 5254.02 525402 -127 124 0.9 90 -369 2 10359 99270 1.1081 298.1081 149.6081 14960.81081 1.1081 298.1081 149.6081 14960.81062 1.10810 298.10810 149.60809999999995 14960.81000 2020-01-01 2020-01-02 2020-01-01 00:06:09 2020-01-02 03:34:30 2020-01-01 00:06:09.000 2020-01-02 03:34:30.000 369 99270 49819.5 4981950 369 99270 49819.5 4981950 -32200 32735 5255.02 525502 -126 125 1.9 190 -37 2 10027 99937 0.11111 300.11111 150.11111 15161.22222 0.11111 300.1111 150.11111 15161.22248 0.11111 300.11111 150.11110999999988 15161.22211 2020-01-01 2020-01-02 2020-01-01 00:00:37 2020-01-02 03:45:37 2020-01-01 00:00:37.000 2020-01-02 03:45:37.000 37 99937 49987 5048687 37 99937 49987 5048687 -32532 32403 4566.009900990099 461167 -127 124 -1.4356435643564356 -145 -370 2 10360 99271 1.11111 298.11111 149.61111 14961.11111 1.11111 298.1111 149.61111 14961.11137 1.11111 298.11111 149.6111099999999 14961.11100 2020-01-01 2020-01-02 2020-01-01 00:06:10 2020-01-02 03:34:31 2020-01-01 00:06:10.000 2020-01-02 03:34:31.000 370 99271 49820.5 4982050 370 99271 49820.5 4982050 -32199 32736 5256.02 525602 -125 126 2.9 290 -371 2 10361 99272 1.11411 298.11411 149.61411 14961.41141 1.11411 298.1141 149.61411 14961.4115 1.11411 298.11411 149.61411000000024 14961.41100 2020-01-01 2020-01-02 2020-01-01 00:06:11 2020-01-02 03:34:32 2020-01-01 00:06:11.000 2020-01-02 03:34:32.000 371 99272 49821.5 4982150 371 99272 49821.5 4982150 -32198 32737 5257.02 525702 -124 127 3.9 390 -372 2 10362 99273 1.11711 298.11711 149.61711 14961.71171 1.11711 298.11713 149.61711 14961.71165 1.11711 298.11711 149.61710999999985 14961.71100 2020-01-01 2020-01-02 2020-01-01 00:06:12 2020-01-02 03:34:33 2020-01-01 00:06:12.000 2020-01-02 03:34:33.000 372 99273 49822.5 4982250 372 99273 49822.5 4982250 -32197 32738 5258.02 525802 -128 127 2.34 234 -373 2 10363 99274 1.12012 298.12012 149.62012 14962.01201 1.12012 298.12012 149.62011 14962.01179 1.12012 298.12012 149.6201199999998 14962.01200 2020-01-01 2020-01-02 2020-01-01 00:06:13 2020-01-02 03:34:34 2020-01-01 00:06:13.000 2020-01-02 03:34:34.000 373 99274 49823.5 4982350 373 99274 49823.5 4982350 -32196 32739 5259.02 525902 -128 123 0.78 78 -374 2 10364 99275 1.12312 298.12312 149.62312 14962.31231 1.12312 298.1231 149.62312 14962.31208 1.12312 298.12312 149.62312000000014 14962.31200 2020-01-01 2020-01-02 2020-01-01 00:06:14 2020-01-02 03:34:35 2020-01-01 00:06:14.000 2020-01-02 03:34:35.000 374 99275 49824.5 4982450 374 99275 49824.5 4982450 -32195 32740 5260.02 526002 -127 124 1.78 178 -375 2 10365 99276 1.12612 298.12612 149.62612 14962.61261 1.12612 298.12613 149.62612 14962.61283 1.12612 298.12612 149.62612000000016 14962.61200 2020-01-01 2020-01-02 2020-01-01 00:06:15 2020-01-02 03:34:36 2020-01-01 00:06:15.000 2020-01-02 03:34:36.000 375 99276 49825.5 4982550 375 99276 49825.5 4982550 -32194 32741 5261.02 526102 -126 125 2.78 278 -376 2 10366 99277 1.12912 298.12912 149.62912 14962.91291 1.12912 298.12912 149.62912 14962.91297 1.12912 298.12912 149.6291199999998 14962.91200 2020-01-01 2020-01-02 2020-01-01 00:06:16 2020-01-02 03:34:37 2020-01-01 00:06:16.000 2020-01-02 03:34:37.000 376 99277 49826.5 4982650 376 99277 49826.5 4982650 -32193 32742 5262.02 526202 -125 126 3.78 378 +351 2 10341 99252 1.05405 298.05405 149.55405 14955.4054 1.05405 298.05405 149.55404 14955.40499 1.05405 298.05405 149.55405000000002 14955.40500 2020-01-01 2020-01-02 2020-01-01 00:05:51 2020-01-02 03:34:12 2020-01-01 00:05:51.000 2020-01-02 03:34:12.000 351 99252 49801.5 4980150 351 99252 49801.5 4980150 -32218 32717 5237.02 523702 -125 126 1.82 182 +352 2 10342 99253 1.05705 298.05705 149.55705 14955.7057 1.05705 298.05707 149.55705 14955.70574 1.05705 298.05705 149.55705 14955.70500 2020-01-01 2020-01-02 2020-01-01 00:05:52 2020-01-02 03:34:13 2020-01-01 00:05:52.000 2020-01-02 03:34:13.000 352 99253 49802.5 4980250 352 99253 49802.5 4980250 -32217 32718 5238.02 523802 -124 127 2.82 282 +353 2 10343 99254 1.06006 298.06006 149.56006 14956.006 1.06006 298.06006 149.56005 14956.00587 1.06006 298.06006 149.56006 14956.00600 2020-01-01 2020-01-02 2020-01-01 00:05:53 2020-01-02 03:34:14 2020-01-01 00:05:53.000 2020-01-02 03:34:14.000 353 99254 49803.5 4980350 353 99254 49803.5 4980350 -32216 32719 5239.02 523902 -128 127 1.26 126 +354 2 10344 99255 1.06306 298.06306 149.56306 14956.3063 1.06306 298.06305 149.56306 14956.3062 1.06306 298.06306 149.56306 14956.30600 2020-01-01 2020-01-02 2020-01-01 00:05:54 2020-01-02 03:34:15 2020-01-01 00:05:54.000 2020-01-02 03:34:15.000 354 99255 49804.5 4980450 354 99255 49804.5 4980450 -32215 32720 5240.02 524002 -128 127 -0.3 -30 +355 2 10345 99256 1.06606 298.06606 149.56606 14956.6066 1.06606 298.06607 149.56606 14956.6068 1.06606 298.06606 149.56606 14956.60600 2020-01-01 2020-01-02 2020-01-01 00:05:55 2020-01-02 03:34:16 2020-01-01 00:05:55.000 2020-01-02 03:34:16.000 355 99256 49805.5 4980550 355 99256 49805.5 4980550 -32214 32721 5241.02 524102 -128 123 -1.86 -186 +356 2 10346 99257 1.06906 298.06906 149.56906 14956.9069 1.06906 298.06906 149.56907 14956.90709 1.06906 298.06906 149.56906 14956.90600 2020-01-01 2020-01-02 2020-01-01 00:05:56 2020-01-02 03:34:17 2020-01-01 00:05:56.000 2020-01-02 03:34:17.000 356 99257 49806.5 4980650 356 99257 49806.5 4980650 -32213 32722 5242.02 524202 -127 124 -0.86 -86 +357 2 10347 99258 1.07207 298.07207 149.57207 14957.2072 1.07207 298.07208 149.57207 14957.20721 1.07207 298.07207 149.57207 14957.20700 2020-01-01 2020-01-02 2020-01-01 00:05:57 2020-01-02 03:34:18 2020-01-01 00:05:57.000 2020-01-02 03:34:18.000 357 99258 49807.5 4980750 357 99258 49807.5 4980750 -32212 32723 5243.02 524302 -126 125 0.14 14 +358 2 10348 99259 1.07507 298.07507 149.57507 14957.5075 1.07507 298.07507 149.57507 14957.50734 1.07507 298.07507 149.57506999999998 14957.50700 2020-01-01 2020-01-02 2020-01-01 00:05:58 2020-01-02 03:34:19 2020-01-01 00:05:58.000 2020-01-02 03:34:19.000 358 99259 49808.5 4980850 358 99259 49808.5 4980850 -32211 32724 5244.02 524402 -125 126 1.14 114 +359 2 10349 99260 1.07807 298.07807 149.57807 14957.8078 1.07807 298.07806 149.57807 14957.80767 1.07807 298.07807 149.57807 14957.80700 2020-01-01 2020-01-02 2020-01-01 00:05:59 2020-01-02 03:34:20 2020-01-01 00:05:59.000 2020-01-02 03:34:20.000 359 99260 49809.5 4980950 359 99260 49809.5 4980950 -32210 32725 5245.02 524502 -124 127 2.14 214 +36 2 10026 99936 0.1081 300.1081 150.1081 15160.91891 0.1081 300.1081 150.1081 15160.91873 0.10810 300.10810 150.1081 15160.91810 2020-01-01 2020-01-02 2020-01-01 00:00:36 2020-01-02 03:45:36 2020-01-01 00:00:36.000 2020-01-02 03:45:36.000 36 99936 49986 5048586 36 99936 49986 5048586 -32533 32402 4565.009900990099 461066 -128 123 -2.4356435643564356 -246 +360 2 10350 99261 1.08108 298.08108 149.58108 14958.1081 1.08108 298.0811 149.58108 14958.10827 1.08108 298.08108 149.58108000000001 14958.10800 2020-01-01 2020-01-02 2020-01-01 00:06:00 2020-01-02 03:34:21 2020-01-01 00:06:00.000 2020-01-02 03:34:21.000 360 99261 49810.5 4981050 360 99261 49810.5 4981050 -32209 32726 5246.02 524602 -128 127 0.58 58 +361 2 10351 99262 1.08408 298.08408 149.58408 14958.4084 1.08408 298.08408 149.58408 14958.40856 1.08408 298.08408 149.58408 14958.40800 2020-01-01 2020-01-02 2020-01-01 00:06:01 2020-01-02 03:34:22 2020-01-01 00:06:01.000 2020-01-02 03:34:22.000 361 99262 49811.5 4981150 361 99262 49811.5 4981150 -32208 32727 5247.02 524702 -128 123 -0.98 -98 +362 2 10352 99263 1.08708 298.08708 149.58708 14958.7087 1.08708 298.0871 149.58708 14958.70868 1.08708 298.08708 149.58708000000001 14958.70800 2020-01-01 2020-01-02 2020-01-01 00:06:02 2020-01-02 03:34:23 2020-01-01 00:06:02.000 2020-01-02 03:34:23.000 362 99263 49812.5 4981250 362 99263 49812.5 4981250 -32207 32728 5248.02 524802 -127 124 0.02 2 +363 2 10353 99264 1.09009 298.09009 149.59009 14959.009 1.09009 298.0901 149.59008 14959.00884 1.09009 298.09009 149.59009 14959.00900 2020-01-01 2020-01-02 2020-01-01 00:06:03 2020-01-02 03:34:24 2020-01-01 00:06:03.000 2020-01-02 03:34:24.000 363 99264 49813.5 4981350 363 99264 49813.5 4981350 -32206 32729 5249.02 524902 -126 125 1.02 102 +364 2 10354 99265 1.09309 298.09309 149.59309 14959.3093 1.09309 298.09308 149.59309 14959.30915 1.09309 298.09309 149.59309 14959.30900 2020-01-01 2020-01-02 2020-01-01 00:06:04 2020-01-02 03:34:25 2020-01-01 00:06:04.000 2020-01-02 03:34:25.000 364 99265 49814.5 4981450 364 99265 49814.5 4981450 -32205 32730 5250.02 525002 -125 126 2.02 202 +365 2 10355 99266 1.09609 298.09609 149.59609 14959.6096 1.09609 298.0961 149.59609 14959.6099 1.09609 298.09609 149.59609 14959.60900 2020-01-01 2020-01-02 2020-01-01 00:06:05 2020-01-02 03:34:26 2020-01-01 00:06:05.000 2020-01-02 03:34:26.000 365 99266 49815.5 4981550 365 99266 49815.5 4981550 -32204 32731 5251.02 525102 -124 127 3.02 302 +366 2 10356 99267 1.09909 298.09909 149.59909 14959.9099 1.09909 298.0991 149.5991 14959.91003 1.09909 298.09909 149.59909 14959.90900 2020-01-01 2020-01-02 2020-01-01 00:06:06 2020-01-02 03:34:27 2020-01-01 00:06:06.000 2020-01-02 03:34:27.000 366 99267 49816.5 4981650 366 99267 49816.5 4981650 -32203 32732 5252.02 525202 -128 127 1.46 146 +367 2 10357 99268 1.1021 298.1021 149.6021 14960.21021 1.1021 298.1021 149.6021 14960.21015 1.10210 298.10210 149.60209999999998 14960.21000 2020-01-01 2020-01-02 2020-01-01 00:06:07 2020-01-02 03:34:28 2020-01-01 00:06:07.000 2020-01-02 03:34:28.000 367 99268 49817.5 4981750 367 99268 49817.5 4981750 -32202 32733 5253.02 525302 -128 123 -0.1 -10 +368 2 10358 99269 1.1051 298.1051 149.6051 14960.51051 1.1051 298.1051 149.6051 14960.51031 1.10510 298.10510 149.6051 14960.51000 2020-01-01 2020-01-02 2020-01-01 00:06:08 2020-01-02 03:34:29 2020-01-01 00:06:08.000 2020-01-02 03:34:29.000 368 99269 49818.5 4981850 368 99269 49818.5 4981850 -32201 32734 5254.02 525402 -127 124 0.9 90 +369 2 10359 99270 1.1081 298.1081 149.6081 14960.81081 1.1081 298.1081 149.6081 14960.81062 1.10810 298.10810 149.6081 14960.81000 2020-01-01 2020-01-02 2020-01-01 00:06:09 2020-01-02 03:34:30 2020-01-01 00:06:09.000 2020-01-02 03:34:30.000 369 99270 49819.5 4981950 369 99270 49819.5 4981950 -32200 32735 5255.02 525502 -126 125 1.9 190 +37 2 10027 99937 0.11111 300.11111 150.11111 15161.22222 0.11111 300.1111 150.11111 15161.22248 0.11111 300.11111 150.11111 15161.22211 2020-01-01 2020-01-02 2020-01-01 00:00:37 2020-01-02 03:45:37 2020-01-01 00:00:37.000 2020-01-02 03:45:37.000 37 99937 49987 5048687 37 99937 49987 5048687 -32532 32403 4566.009900990099 461167 -127 124 -1.4356435643564356 -145 +370 2 10360 99271 1.11111 298.11111 149.61111 14961.11111 1.11111 298.1111 149.61111 14961.11137 1.11111 298.11111 149.61111 14961.11100 2020-01-01 2020-01-02 2020-01-01 00:06:10 2020-01-02 03:34:31 2020-01-01 00:06:10.000 2020-01-02 03:34:31.000 370 99271 49820.5 4982050 370 99271 49820.5 4982050 -32199 32736 5256.02 525602 -125 126 2.9 290 +371 2 10361 99272 1.11411 298.11411 149.61411 14961.41141 1.11411 298.1141 149.61411 14961.4115 1.11411 298.11411 149.61411 14961.41100 2020-01-01 2020-01-02 2020-01-01 00:06:11 2020-01-02 03:34:32 2020-01-01 00:06:11.000 2020-01-02 03:34:32.000 371 99272 49821.5 4982150 371 99272 49821.5 4982150 -32198 32737 5257.02 525702 -124 127 3.9 390 +372 2 10362 99273 1.11711 298.11711 149.61711 14961.71171 1.11711 298.11713 149.61711 14961.71165 1.11711 298.11711 149.61711 14961.71100 2020-01-01 2020-01-02 2020-01-01 00:06:12 2020-01-02 03:34:33 2020-01-01 00:06:12.000 2020-01-02 03:34:33.000 372 99273 49822.5 4982250 372 99273 49822.5 4982250 -32197 32738 5258.02 525802 -128 127 2.34 234 +373 2 10363 99274 1.12012 298.12012 149.62012 14962.01201 1.12012 298.12012 149.62011 14962.01179 1.12012 298.12012 149.62012000000001 14962.01200 2020-01-01 2020-01-02 2020-01-01 00:06:13 2020-01-02 03:34:34 2020-01-01 00:06:13.000 2020-01-02 03:34:34.000 373 99274 49823.5 4982350 373 99274 49823.5 4982350 -32196 32739 5259.02 525902 -128 123 0.78 78 +374 2 10364 99275 1.12312 298.12312 149.62312 14962.31231 1.12312 298.1231 149.62312 14962.31208 1.12312 298.12312 149.62312 14962.31200 2020-01-01 2020-01-02 2020-01-01 00:06:14 2020-01-02 03:34:35 2020-01-01 00:06:14.000 2020-01-02 03:34:35.000 374 99275 49824.5 4982450 374 99275 49824.5 4982450 -32195 32740 5260.02 526002 -127 124 1.78 178 +375 2 10365 99276 1.12612 298.12612 149.62612 14962.61261 1.12612 298.12613 149.62612 14962.61283 1.12612 298.12612 149.62612 14962.61200 2020-01-01 2020-01-02 2020-01-01 00:06:15 2020-01-02 03:34:36 2020-01-01 00:06:15.000 2020-01-02 03:34:36.000 375 99276 49825.5 4982550 375 99276 49825.5 4982550 -32194 32741 5261.02 526102 -126 125 2.78 278 +376 2 10366 99277 1.12912 298.12912 149.62912 14962.91291 1.12912 298.12912 149.62912 14962.91297 1.12912 298.12912 149.62912 14962.91200 2020-01-01 2020-01-02 2020-01-01 00:06:16 2020-01-02 03:34:37 2020-01-01 00:06:16.000 2020-01-02 03:34:37.000 376 99277 49826.5 4982650 376 99277 49826.5 4982650 -32193 32742 5262.02 526202 -125 126 3.78 378 377 2 10367 99278 1.13213 298.13213 149.63213 14963.21321 1.13213 298.13214 149.63213 14963.21312 1.13213 298.13213 149.63213 14963.21300 2020-01-01 2020-01-02 2020-01-01 00:06:17 2020-01-02 03:34:38 2020-01-01 00:06:17.000 2020-01-02 03:34:38.000 377 99278 49827.5 4982750 377 99278 49827.5 4982750 -32192 32743 5263.02 526302 -124 127 4.78 478 -378 2 10368 99279 1.13513 298.13513 149.63513 14963.51351 1.13513 298.13513 149.63513 14963.51326 1.13513 298.13513 149.6351300000001 14963.51300 2020-01-01 2020-01-02 2020-01-01 00:06:18 2020-01-02 03:34:39 2020-01-01 00:06:18.000 2020-01-02 03:34:39.000 378 99279 49828.5 4982850 378 99279 49828.5 4982850 -32191 32744 5264.02 526402 -128 127 3.22 322 -379 2 10369 99280 1.13813 298.13813 149.63813 14963.81381 1.13813 298.13815 149.63814 14963.81401 1.13813 298.13813 149.63812999999993 14963.81300 2020-01-01 2020-01-02 2020-01-01 00:06:19 2020-01-02 03:34:40 2020-01-01 00:06:19.000 2020-01-02 03:34:40.000 379 99280 49829.5 4982950 379 99280 49829.5 4982950 -32190 32745 5265.02 526502 -128 127 1.66 166 -38 2 10028 99938 0.11411 300.11411 150.11411 15161.52552 0.11411 300.1141 150.11411 15161.52562 0.11411 300.11411 150.11411000000024 15161.52511 2020-01-01 2020-01-02 2020-01-01 00:00:38 2020-01-02 03:45:38 2020-01-01 00:00:38.000 2020-01-02 03:45:38.000 38 99938 49988 5048788 38 99938 49988 5048788 -32531 32404 4567.009900990099 461268 -126 125 -0.43564356435643564 -44 -380 2 10370 99281 1.14114 298.14114 149.64114 14964.11411 1.14114 298.14114 149.64114 14964.11431 1.14114 298.14114 149.64113999999987 14964.11400 2020-01-01 2020-01-02 2020-01-01 00:06:20 2020-01-02 03:34:41 2020-01-01 00:06:20.000 2020-01-02 03:34:41.000 380 99281 49830.5 4983050 380 99281 49830.5 4983050 -32189 32746 5266.02 526602 -128 124 0.1 10 -381 2 10371 99282 1.14414 298.14414 149.64414 14964.41441 1.14414 298.14413 149.64414 14964.41448 1.14414 298.14414 149.64414000000016 14964.41400 2020-01-01 2020-01-02 2020-01-01 00:06:21 2020-01-02 03:34:42 2020-01-01 00:06:21.000 2020-01-02 03:34:42.000 381 99282 49831.5 4983150 381 99282 49831.5 4983150 -32188 32747 5267.02 526702 -127 125 1.1 110 -382 2 10372 99283 1.14714 298.14714 149.64714 14964.71471 1.14714 298.14716 149.64714 14964.71459 1.14714 298.14714 149.64713999999978 14964.71400 2020-01-01 2020-01-02 2020-01-01 00:06:22 2020-01-02 03:34:43 2020-01-01 00:06:22.000 2020-01-02 03:34:43.000 382 99283 49832.5 4983250 382 99283 49832.5 4983250 -32187 32748 5268.02 526802 -126 126 2.1 210 +378 2 10368 99279 1.13513 298.13513 149.63513 14963.51351 1.13513 298.13513 149.63513 14963.51326 1.13513 298.13513 149.63513 14963.51300 2020-01-01 2020-01-02 2020-01-01 00:06:18 2020-01-02 03:34:39 2020-01-01 00:06:18.000 2020-01-02 03:34:39.000 378 99279 49828.5 4982850 378 99279 49828.5 4982850 -32191 32744 5264.02 526402 -128 127 3.22 322 +379 2 10369 99280 1.13813 298.13813 149.63813 14963.81381 1.13813 298.13815 149.63814 14963.81401 1.13813 298.13813 149.63813 14963.81300 2020-01-01 2020-01-02 2020-01-01 00:06:19 2020-01-02 03:34:40 2020-01-01 00:06:19.000 2020-01-02 03:34:40.000 379 99280 49829.5 4982950 379 99280 49829.5 4982950 -32190 32745 5265.02 526502 -128 127 1.66 166 +38 2 10028 99938 0.11411 300.11411 150.11411 15161.52552 0.11411 300.1141 150.11411 15161.52562 0.11411 300.11411 150.11411 15161.52511 2020-01-01 2020-01-02 2020-01-01 00:00:38 2020-01-02 03:45:38 2020-01-01 00:00:38.000 2020-01-02 03:45:38.000 38 99938 49988 5048788 38 99938 49988 5048788 -32531 32404 4567.009900990099 461268 -126 125 -0.43564356435643564 -44 +380 2 10370 99281 1.14114 298.14114 149.64114 14964.11411 1.14114 298.14114 149.64114 14964.11431 1.14114 298.14114 149.64114 14964.11400 2020-01-01 2020-01-02 2020-01-01 00:06:20 2020-01-02 03:34:41 2020-01-01 00:06:20.000 2020-01-02 03:34:41.000 380 99281 49830.5 4983050 380 99281 49830.5 4983050 -32189 32746 5266.02 526602 -128 124 0.1 10 +381 2 10371 99282 1.14414 298.14414 149.64414 14964.41441 1.14414 298.14413 149.64414 14964.41448 1.14414 298.14414 149.64414 14964.41400 2020-01-01 2020-01-02 2020-01-01 00:06:21 2020-01-02 03:34:42 2020-01-01 00:06:21.000 2020-01-02 03:34:42.000 381 99282 49831.5 4983150 381 99282 49831.5 4983150 -32188 32747 5267.02 526702 -127 125 1.1 110 +382 2 10372 99283 1.14714 298.14714 149.64714 14964.71471 1.14714 298.14716 149.64714 14964.71459 1.14714 298.14714 149.64714 14964.71400 2020-01-01 2020-01-02 2020-01-01 00:06:22 2020-01-02 03:34:43 2020-01-01 00:06:22.000 2020-01-02 03:34:43.000 382 99283 49832.5 4983250 382 99283 49832.5 4983250 -32187 32748 5268.02 526802 -126 126 2.1 210 383 2 10373 99284 1.15015 298.15015 149.65015 14965.01501 1.15015 298.15015 149.65014 14965.01472 1.15015 298.15015 149.65015 14965.01500 2020-01-01 2020-01-02 2020-01-01 00:06:23 2020-01-02 03:34:44 2020-01-01 00:06:23.000 2020-01-02 03:34:44.000 383 99284 49833.5 4983350 383 99284 49833.5 4983350 -32186 32749 5269.02 526902 -125 127 3.1 310 -384 2 10374 99285 1.15315 298.15315 149.65315 14965.31531 1.15315 298.15317 149.65315 14965.31547 1.15315 298.15315 149.65315000000007 14965.31500 2020-01-01 2020-01-02 2020-01-01 00:06:24 2020-01-02 03:34:45 2020-01-01 00:06:24.000 2020-01-02 03:34:45.000 384 99285 49834.5 4983450 384 99285 49834.5 4983450 -32185 32750 5270.02 527002 -128 127 1.54 154 -385 2 10375 99286 1.15615 298.15615 149.65615 14965.61561 1.15615 298.15616 149.65615 14965.61578 1.15615 298.15615 149.65615000000008 14965.61500 2020-01-01 2020-01-02 2020-01-01 00:06:25 2020-01-02 03:34:46 2020-01-01 00:06:25.000 2020-01-02 03:34:46.000 385 99286 49835.5 4983550 385 99286 49835.5 4983550 -32184 32751 5271.02 527102 -128 127 -0.02 -2 -386 2 10376 99287 1.15915 298.15915 149.65915 14965.91591 1.15915 298.15915 149.65915 14965.91594 1.15915 298.15915 149.65914999999998 14965.91500 2020-01-01 2020-01-02 2020-01-01 00:06:26 2020-01-02 03:34:47 2020-01-01 00:06:26.000 2020-01-02 03:34:47.000 386 99287 49836.5 4983650 386 99287 49836.5 4983650 -32183 32752 5272.02 527202 -128 123 -1.58 -158 -387 2 10377 99288 1.16216 298.16216 149.66216 14966.21621 1.16216 298.16217 149.66216 14966.21606 1.16216 298.16216 149.6621599999999 14966.21600 2020-01-01 2020-01-02 2020-01-01 00:06:27 2020-01-02 03:34:48 2020-01-01 00:06:27.000 2020-01-02 03:34:48.000 387 99288 49837.5 4983750 387 99288 49837.5 4983750 -32182 32753 5273.02 527302 -127 124 -0.58 -58 +384 2 10374 99285 1.15315 298.15315 149.65315 14965.31531 1.15315 298.15317 149.65315 14965.31547 1.15315 298.15315 149.65315 14965.31500 2020-01-01 2020-01-02 2020-01-01 00:06:24 2020-01-02 03:34:45 2020-01-01 00:06:24.000 2020-01-02 03:34:45.000 384 99285 49834.5 4983450 384 99285 49834.5 4983450 -32185 32750 5270.02 527002 -128 127 1.54 154 +385 2 10375 99286 1.15615 298.15615 149.65615 14965.61561 1.15615 298.15616 149.65615 14965.61578 1.15615 298.15615 149.65615 14965.61500 2020-01-01 2020-01-02 2020-01-01 00:06:25 2020-01-02 03:34:46 2020-01-01 00:06:25.000 2020-01-02 03:34:46.000 385 99286 49835.5 4983550 385 99286 49835.5 4983550 -32184 32751 5271.02 527102 -128 127 -0.02 -2 +386 2 10376 99287 1.15915 298.15915 149.65915 14965.91591 1.15915 298.15915 149.65915 14965.91594 1.15915 298.15915 149.65915 14965.91500 2020-01-01 2020-01-02 2020-01-01 00:06:26 2020-01-02 03:34:47 2020-01-01 00:06:26.000 2020-01-02 03:34:47.000 386 99287 49836.5 4983650 386 99287 49836.5 4983650 -32183 32752 5272.02 527202 -128 123 -1.58 -158 +387 2 10377 99288 1.16216 298.16216 149.66216 14966.21621 1.16216 298.16217 149.66216 14966.21606 1.16216 298.16216 149.66216 14966.21600 2020-01-01 2020-01-02 2020-01-01 00:06:27 2020-01-02 03:34:48 2020-01-01 00:06:27.000 2020-01-02 03:34:48.000 387 99288 49837.5 4983750 387 99288 49837.5 4983750 -32182 32753 5273.02 527302 -127 124 -0.58 -58 388 2 10378 99289 1.16516 298.16516 149.66516 14966.51651 1.16516 298.16516 149.66516 14966.51636 1.16516 298.16516 149.66516 14966.51600 2020-01-01 2020-01-02 2020-01-01 00:06:28 2020-01-02 03:34:49 2020-01-01 00:06:28.000 2020-01-02 03:34:49.000 388 99289 49838.5 4983850 388 99289 49838.5 4983850 -32181 32754 5274.02 527402 -126 125 0.42 42 -389 2 10379 99290 1.16816 298.16816 149.66816 14966.81681 1.16816 298.16818 149.66816 14966.81695 1.16816 298.16816 149.6681599999999 14966.81600 2020-01-01 2020-01-02 2020-01-01 00:06:29 2020-01-02 03:34:50 2020-01-01 00:06:29.000 2020-01-02 03:34:50.000 389 99290 49839.5 4983950 389 99290 49839.5 4983950 -32180 32755 5275.02 527502 -125 126 1.42 142 -39 2 10029 99939 0.11711 300.11711 150.11711 15161.82882 0.11711 300.11713 150.11711 15161.82876 0.11711 300.11711 150.1171099999999 15161.82811 2020-01-01 2020-01-02 2020-01-01 00:00:39 2020-01-02 03:45:39 2020-01-01 00:00:39.000 2020-01-02 03:45:39.000 39 99939 49989 5048889 39 99939 49989 5048889 -32530 32405 4568.009900990099 461369 -125 126 0.5643564356435643 57 -390 2 10380 99291 1.17117 298.17117 149.67117 14967.11711 1.17117 298.17117 149.67117 14967.11725 1.17117 298.17117 149.67116999999982 14967.11700 2020-01-01 2020-01-02 2020-01-01 00:06:30 2020-01-02 03:34:51 2020-01-01 00:06:30.000 2020-01-02 03:34:51.000 390 99291 49840.5 4984050 390 99291 49840.5 4984050 -32179 32756 5276.02 527602 -124 127 2.42 242 -391 2 10381 99292 1.17417 298.17417 149.67417 14967.41741 1.17417 298.17416 149.67417 14967.41741 1.17417 298.17417 149.67417000000015 14967.41700 2020-01-01 2020-01-02 2020-01-01 00:06:31 2020-01-02 03:34:52 2020-01-01 00:06:31.000 2020-01-02 03:34:52.000 391 99292 49841.5 4984150 391 99292 49841.5 4984150 -32178 32757 5277.02 527702 -128 127 0.86 86 -392 2 10382 99293 1.17717 298.17717 149.67717 14967.71771 1.17717 298.1772 149.67717 14967.71753 1.17717 298.17717 149.67717000000016 14967.71700 2020-01-01 2020-01-02 2020-01-01 00:06:32 2020-01-02 03:34:53 2020-01-01 00:06:32.000 2020-01-02 03:34:53.000 392 99293 49842.5 4984250 392 99293 49842.5 4984250 -32177 32758 5278.02 527802 -128 123 -0.7 -70 -393 2 10383 99294 1.18018 298.18018 149.68018 14968.01801 1.18018 298.18018 149.68017 14968.01782 1.18018 298.18018 149.68017999999995 14968.01800 2020-01-01 2020-01-02 2020-01-01 00:06:33 2020-01-02 03:34:54 2020-01-01 00:06:33.000 2020-01-02 03:34:54.000 393 99294 49843.5 4984350 393 99294 49843.5 4984350 -32176 32759 5279.02 527902 -127 124 0.3 30 +389 2 10379 99290 1.16816 298.16816 149.66816 14966.81681 1.16816 298.16818 149.66816 14966.81695 1.16816 298.16816 149.66816 14966.81600 2020-01-01 2020-01-02 2020-01-01 00:06:29 2020-01-02 03:34:50 2020-01-01 00:06:29.000 2020-01-02 03:34:50.000 389 99290 49839.5 4983950 389 99290 49839.5 4983950 -32180 32755 5275.02 527502 -125 126 1.42 142 +39 2 10029 99939 0.11711 300.11711 150.11711 15161.82882 0.11711 300.11713 150.11711 15161.82876 0.11711 300.11711 150.11711 15161.82811 2020-01-01 2020-01-02 2020-01-01 00:00:39 2020-01-02 03:45:39 2020-01-01 00:00:39.000 2020-01-02 03:45:39.000 39 99939 49989 5048889 39 99939 49989 5048889 -32530 32405 4568.009900990099 461369 -125 126 0.5643564356435643 57 +390 2 10380 99291 1.17117 298.17117 149.67117 14967.11711 1.17117 298.17117 149.67117 14967.11725 1.17117 298.17117 149.67117 14967.11700 2020-01-01 2020-01-02 2020-01-01 00:06:30 2020-01-02 03:34:51 2020-01-01 00:06:30.000 2020-01-02 03:34:51.000 390 99291 49840.5 4984050 390 99291 49840.5 4984050 -32179 32756 5276.02 527602 -124 127 2.42 242 +391 2 10381 99292 1.17417 298.17417 149.67417 14967.41741 1.17417 298.17416 149.67417 14967.41741 1.17417 298.17417 149.67417 14967.41700 2020-01-01 2020-01-02 2020-01-01 00:06:31 2020-01-02 03:34:52 2020-01-01 00:06:31.000 2020-01-02 03:34:52.000 391 99292 49841.5 4984150 391 99292 49841.5 4984150 -32178 32757 5277.02 527702 -128 127 0.86 86 +392 2 10382 99293 1.17717 298.17717 149.67717 14967.71771 1.17717 298.1772 149.67717 14967.71753 1.17717 298.17717 149.67717000000002 14967.71700 2020-01-01 2020-01-02 2020-01-01 00:06:32 2020-01-02 03:34:53 2020-01-01 00:06:32.000 2020-01-02 03:34:53.000 392 99293 49842.5 4984250 392 99293 49842.5 4984250 -32177 32758 5278.02 527802 -128 123 -0.7 -70 +393 2 10383 99294 1.18018 298.18018 149.68018 14968.01801 1.18018 298.18018 149.68017 14968.01782 1.18018 298.18018 149.68018 14968.01800 2020-01-01 2020-01-02 2020-01-01 00:06:33 2020-01-02 03:34:54 2020-01-01 00:06:33.000 2020-01-02 03:34:54.000 393 99294 49843.5 4984350 393 99294 49843.5 4984350 -32176 32759 5279.02 527902 -127 124 0.3 30 394 2 10384 99295 1.18318 298.18318 149.68318 14968.31831 1.18318 298.1832 149.68318 14968.31842 1.18318 298.18318 149.68318 14968.31800 2020-01-01 2020-01-02 2020-01-01 00:06:34 2020-01-02 03:34:55 2020-01-01 00:06:34.000 2020-01-02 03:34:55.000 394 99295 49844.5 4984450 394 99295 49844.5 4984450 -32175 32760 5280.02 528002 -126 125 1.3 130 -395 2 10385 99296 1.18618 298.18618 149.68618 14968.61861 1.18618 298.1862 149.68618 14968.61875 1.18618 298.18618 149.68618000000006 14968.61800 2020-01-01 2020-01-02 2020-01-01 00:06:35 2020-01-02 03:34:56 2020-01-01 00:06:35.000 2020-01-02 03:34:56.000 395 99296 49845.5 4984550 395 99296 49845.5 4984550 -32174 32761 5281.02 528102 -125 126 2.3 230 -396 2 10386 99297 1.18918 298.18918 149.68918 14968.91891 1.18918 298.18918 149.68918 14968.91889 1.18918 298.18918 149.68917999999994 14968.91800 2020-01-01 2020-01-02 2020-01-01 00:06:36 2020-01-02 03:34:57 2020-01-01 00:06:36.000 2020-01-02 03:34:57.000 396 99297 49846.5 4984650 396 99297 49846.5 4984650 -32173 32762 5282.02 528202 -124 127 3.3 330 -397 2 10387 99298 1.19219 298.19219 149.69219 14969.21921 1.19219 298.1922 149.69219 14969.21964 1.19219 298.19219 149.6921899999999 14969.21900 2020-01-01 2020-01-02 2020-01-01 00:06:37 2020-01-02 03:34:58 2020-01-01 00:06:37.000 2020-01-02 03:34:58.000 397 99298 49847.5 4984750 397 99298 49847.5 4984750 -32172 32763 5283.02 528302 -128 127 1.74 174 -398 2 10388 99299 1.19519 298.19519 149.69519 14969.51951 1.19519 298.1952 149.69519 14969.51929 1.19519 298.19519 149.6951900000002 14969.51900 2020-01-01 2020-01-02 2020-01-01 00:06:38 2020-01-02 03:34:59 2020-01-01 00:06:38.000 2020-01-02 03:34:59.000 398 99299 49848.5 4984850 398 99299 49848.5 4984850 -32171 32764 5284.02 528402 -128 123 0.18 18 -399 2 10389 99300 1.19819 298.19819 149.69819 14969.81981 1.19819 298.1982 149.69819 14969.81989 1.19819 298.19819 149.69818999999987 14969.81900 2020-01-01 2020-01-02 2020-01-01 00:06:39 2020-01-02 03:35:00 2020-01-01 00:06:39.000 2020-01-02 03:35:00.000 399 99300 49849.5 4984950 399 99300 49849.5 4984950 -32170 32765 5285.02 528502 -127 124 1.18 118 -4 2 1003 9994 0.01201 300.01201 150.01201 15151.21321 0.01201 300.01202 150.01201 15151.21318 0.01201 300.01201 150.01201000000017 15151.21301 2020-01-01 2020-01-02 2020-01-01 00:00:04 2020-01-02 03:45:04 2020-01-01 00:00:04.000 2020-01-02 03:45:04.000 4 99904 49954 5045354 4 99904 49954 5045354 -32565 32370 4533.009900990099 457834 -128 127 -1.4851485148514851 -150 -40 2 10030 99940 0.12012 300.12012 150.12012 15162.13213 0.12012 300.12012 150.12011 15162.13191 0.12012 300.12012 150.1201199999998 15162.13212 2020-01-01 2020-01-02 2020-01-01 00:00:40 2020-01-02 03:45:40 2020-01-01 00:00:40.000 2020-01-02 03:45:40.000 40 99940 49990 5048990 40 99940 49990 5048990 -32529 32406 4569.009900990099 461470 -124 127 1.5643564356435644 158 -400 2 10390 99301 1.2012 298.2012 149.7012 14970.12012 1.2012 298.2012 149.7012 14970.12022 1.20120 298.20120 149.70120000000003 14970.12000 2020-01-01 2020-01-02 2020-01-01 00:06:40 2020-01-02 03:35:01 2020-01-01 00:06:40.000 2020-01-02 03:35:01.000 400 99301 49850.5 4985050 400 99301 49850.5 4985050 -32169 32766 5286.02 528602 -126 125 2.18 218 -401 2 10391 99302 1.2042 298.2042 149.7042 14970.42042 1.2042 298.2042 149.7042 14970.42035 1.20420 298.20420 149.7042000000001 14970.42000 2020-01-01 2020-01-02 2020-01-01 00:06:41 2020-01-02 03:35:02 2020-01-01 00:06:41.000 2020-01-02 03:35:02.000 401 99302 49851.5 4985150 401 99302 49851.5 4985150 -32168 32767 5287.02 528702 -125 126 3.18 318 -402 2 10392 99303 1.2072 298.2072 149.7072 14970.72072 1.2072 298.2072 149.70721 14970.72111 1.20720 298.20720 149.70720000000006 14970.72000 2020-01-01 2020-01-02 2020-01-01 00:06:42 2020-01-02 03:35:03 2020-01-01 00:06:42.000 2020-01-02 03:35:03.000 402 99303 49852.5 4985250 402 99303 49852.5 4985250 -32768 32370 4632.66 463266 -124 127 4.18 418 -403 2 10393 99304 1.21021 298.21021 149.71021 14971.02102 1.21021 298.2102 149.7102 14971.02077 1.21021 298.21021 149.71020999999993 14971.02100 2020-01-01 2020-01-02 2020-01-01 00:06:43 2020-01-02 03:35:04 2020-01-01 00:06:43.000 2020-01-02 03:35:04.000 403 99304 49853.5 4985350 403 99304 49853.5 4985350 -32767 32371 4633.66 463366 -128 127 2.62 262 -404 2 10394 99305 1.21321 298.21321 149.71321 14971.32132 1.21321 298.21323 149.71321 14971.32139 1.21321 298.21321 149.71320999999992 14971.32100 2020-01-01 2020-01-02 2020-01-01 00:06:44 2020-01-02 03:35:05 2020-01-01 00:06:44.000 2020-01-02 03:35:05.000 404 99305 49854.5 4985450 404 99305 49854.5 4985450 -32766 32372 4634.66 463466 -128 127 1.06 106 +395 2 10385 99296 1.18618 298.18618 149.68618 14968.61861 1.18618 298.1862 149.68618 14968.61875 1.18618 298.18618 149.68618 14968.61800 2020-01-01 2020-01-02 2020-01-01 00:06:35 2020-01-02 03:34:56 2020-01-01 00:06:35.000 2020-01-02 03:34:56.000 395 99296 49845.5 4984550 395 99296 49845.5 4984550 -32174 32761 5281.02 528102 -125 126 2.3 230 +396 2 10386 99297 1.18918 298.18918 149.68918 14968.91891 1.18918 298.18918 149.68918 14968.91889 1.18918 298.18918 149.68918 14968.91800 2020-01-01 2020-01-02 2020-01-01 00:06:36 2020-01-02 03:34:57 2020-01-01 00:06:36.000 2020-01-02 03:34:57.000 396 99297 49846.5 4984650 396 99297 49846.5 4984650 -32173 32762 5282.02 528202 -124 127 3.3 330 +397 2 10387 99298 1.19219 298.19219 149.69219 14969.21921 1.19219 298.1922 149.69219 14969.21964 1.19219 298.19219 149.69218999999998 14969.21900 2020-01-01 2020-01-02 2020-01-01 00:06:37 2020-01-02 03:34:58 2020-01-01 00:06:37.000 2020-01-02 03:34:58.000 397 99298 49847.5 4984750 397 99298 49847.5 4984750 -32172 32763 5283.02 528302 -128 127 1.74 174 +398 2 10388 99299 1.19519 298.19519 149.69519 14969.51951 1.19519 298.1952 149.69519 14969.51929 1.19519 298.19519 149.69519 14969.51900 2020-01-01 2020-01-02 2020-01-01 00:06:38 2020-01-02 03:34:59 2020-01-01 00:06:38.000 2020-01-02 03:34:59.000 398 99299 49848.5 4984850 398 99299 49848.5 4984850 -32171 32764 5284.02 528402 -128 123 0.18 18 +399 2 10389 99300 1.19819 298.19819 149.69819 14969.81981 1.19819 298.1982 149.69819 14969.81989 1.19819 298.19819 149.69818999999998 14969.81900 2020-01-01 2020-01-02 2020-01-01 00:06:39 2020-01-02 03:35:00 2020-01-01 00:06:39.000 2020-01-02 03:35:00.000 399 99300 49849.5 4984950 399 99300 49849.5 4984950 -32170 32765 5285.02 528502 -127 124 1.18 118 +4 2 1003 9994 0.01201 300.01201 150.01201 15151.21321 0.01201 300.01202 150.01201 15151.21318 0.01201 300.01201 150.01201 15151.21301 2020-01-01 2020-01-02 2020-01-01 00:00:04 2020-01-02 03:45:04 2020-01-01 00:00:04.000 2020-01-02 03:45:04.000 4 99904 49954 5045354 4 99904 49954 5045354 -32565 32370 4533.009900990099 457834 -128 127 -1.4851485148514851 -150 +40 2 10030 99940 0.12012 300.12012 150.12012 15162.13213 0.12012 300.12012 150.12011 15162.13191 0.12012 300.12012 150.12012000000001 15162.13212 2020-01-01 2020-01-02 2020-01-01 00:00:40 2020-01-02 03:45:40 2020-01-01 00:00:40.000 2020-01-02 03:45:40.000 40 99940 49990 5048990 40 99940 49990 5048990 -32529 32406 4569.009900990099 461470 -124 127 1.5643564356435644 158 +400 2 10390 99301 1.2012 298.2012 149.7012 14970.12012 1.2012 298.2012 149.7012 14970.12022 1.20120 298.20120 149.7012 14970.12000 2020-01-01 2020-01-02 2020-01-01 00:06:40 2020-01-02 03:35:01 2020-01-01 00:06:40.000 2020-01-02 03:35:01.000 400 99301 49850.5 4985050 400 99301 49850.5 4985050 -32169 32766 5286.02 528602 -126 125 2.18 218 +401 2 10391 99302 1.2042 298.2042 149.7042 14970.42042 1.2042 298.2042 149.7042 14970.42035 1.20420 298.20420 149.70420000000001 14970.42000 2020-01-01 2020-01-02 2020-01-01 00:06:41 2020-01-02 03:35:02 2020-01-01 00:06:41.000 2020-01-02 03:35:02.000 401 99302 49851.5 4985150 401 99302 49851.5 4985150 -32168 32767 5287.02 528702 -125 126 3.18 318 +402 2 10392 99303 1.2072 298.2072 149.7072 14970.72072 1.2072 298.2072 149.70721 14970.72111 1.20720 298.20720 149.7072 14970.72000 2020-01-01 2020-01-02 2020-01-01 00:06:42 2020-01-02 03:35:03 2020-01-01 00:06:42.000 2020-01-02 03:35:03.000 402 99303 49852.5 4985250 402 99303 49852.5 4985250 -32768 32370 4632.66 463266 -124 127 4.18 418 +403 2 10393 99304 1.21021 298.21021 149.71021 14971.02102 1.21021 298.2102 149.7102 14971.02077 1.21021 298.21021 149.71021000000002 14971.02100 2020-01-01 2020-01-02 2020-01-01 00:06:43 2020-01-02 03:35:04 2020-01-01 00:06:43.000 2020-01-02 03:35:04.000 403 99304 49853.5 4985350 403 99304 49853.5 4985350 -32767 32371 4633.66 463366 -128 127 2.62 262 +404 2 10394 99305 1.21321 298.21321 149.71321 14971.32132 1.21321 298.21323 149.71321 14971.32139 1.21321 298.21321 149.71321 14971.32100 2020-01-01 2020-01-02 2020-01-01 00:06:44 2020-01-02 03:35:05 2020-01-01 00:06:44.000 2020-01-02 03:35:05.000 404 99305 49854.5 4985450 404 99305 49854.5 4985450 -32766 32372 4634.66 463466 -128 127 1.06 106 405 2 10395 99306 1.21621 298.21621 149.71621 14971.62162 1.21621 298.21622 149.71621 14971.62169 1.21621 298.21621 149.71621 14971.62100 2020-01-01 2020-01-02 2020-01-01 00:06:45 2020-01-02 03:35:06 2020-01-01 00:06:45.000 2020-01-02 03:35:06.000 405 99306 49855.5 4985550 405 99306 49855.5 4985550 -32765 32373 4635.66 463566 -128 124 -0.5 -50 -406 2 10396 99307 1.21921 298.21921 149.71921 14971.92192 1.21921 298.2192 149.71921 14971.92199 1.21921 298.21921 149.71920999999992 14971.92100 2020-01-01 2020-01-02 2020-01-01 00:06:46 2020-01-02 03:35:07 2020-01-01 00:06:46.000 2020-01-02 03:35:07.000 406 99307 49856.5 4985650 406 99307 49856.5 4985650 -32764 32374 4636.66 463666 -127 125 0.5 50 -407 2 10397 99308 1.22222 298.22222 149.72222 14972.22222 1.22222 298.22223 149.72222 14972.22257 1.22222 298.22222 149.72221999999982 14972.22200 2020-01-01 2020-01-02 2020-01-01 00:06:47 2020-01-02 03:35:08 2020-01-01 00:06:47.000 2020-01-02 03:35:08.000 407 99308 49857.5 4985750 407 99308 49857.5 4985750 -32763 32375 4637.66 463766 -126 126 1.5 150 -408 2 10398 99309 1.22522 298.22522 149.72522 14972.52252 1.22522 298.22522 149.72522 14972.52224 1.22522 298.22522 149.7252200000001 14972.52200 2020-01-01 2020-01-02 2020-01-01 00:06:48 2020-01-02 03:35:09 2020-01-01 00:06:48.000 2020-01-02 03:35:09.000 408 99309 49858.5 4985850 408 99309 49858.5 4985850 -32762 32376 4638.66 463866 -125 127 2.5 250 -409 2 10399 99310 1.22822 298.22822 149.72822 14972.82282 1.22822 298.22824 149.72822 14972.82286 1.22822 298.22822 149.7282200000002 14972.82200 2020-01-01 2020-01-02 2020-01-01 00:06:49 2020-01-02 03:35:10 2020-01-01 00:06:49.000 2020-01-02 03:35:10.000 409 99310 49859.5 4985950 409 99310 49859.5 4985950 -32761 32377 4639.66 463966 -128 127 0.94 94 -41 2 10031 99941 0.12312 300.12312 150.12312 15162.43543 0.12312 300.1231 150.12312 15162.43521 0.12312 300.12312 150.12312000000014 15162.43512 2020-01-01 2020-01-02 2020-01-01 00:00:41 2020-01-02 03:45:41 2020-01-01 00:00:41.000 2020-01-02 03:45:41.000 41 99941 49991 5049091 41 99941 49991 5049091 -32528 32407 4570.009900990099 461571 -128 127 0.0297029702970297 3 -410 2 10400 99311 1.23123 298.23123 149.73123 14973.12312 1.23123 298.23123 149.73123 14973.12316 1.23123 298.23123 149.73122999999995 14973.12300 2020-01-01 2020-01-02 2020-01-01 00:06:50 2020-01-02 03:35:11 2020-01-01 00:06:50.000 2020-01-02 03:35:11.000 410 99311 49860.5 4986050 410 99311 49860.5 4986050 -32760 32378 4640.66 464066 -128 127 -0.62 -62 +406 2 10396 99307 1.21921 298.21921 149.71921 14971.92192 1.21921 298.2192 149.71921 14971.92199 1.21921 298.21921 149.71921 14971.92100 2020-01-01 2020-01-02 2020-01-01 00:06:46 2020-01-02 03:35:07 2020-01-01 00:06:46.000 2020-01-02 03:35:07.000 406 99307 49856.5 4985650 406 99307 49856.5 4985650 -32764 32374 4636.66 463666 -127 125 0.5 50 +407 2 10397 99308 1.22222 298.22222 149.72222 14972.22222 1.22222 298.22223 149.72222 14972.22257 1.22222 298.22222 149.72222 14972.22200 2020-01-01 2020-01-02 2020-01-01 00:06:47 2020-01-02 03:35:08 2020-01-01 00:06:47.000 2020-01-02 03:35:08.000 407 99308 49857.5 4985750 407 99308 49857.5 4985750 -32763 32375 4637.66 463766 -126 126 1.5 150 +408 2 10398 99309 1.22522 298.22522 149.72522 14972.52252 1.22522 298.22522 149.72522 14972.52224 1.22522 298.22522 149.72522 14972.52200 2020-01-01 2020-01-02 2020-01-01 00:06:48 2020-01-02 03:35:09 2020-01-01 00:06:48.000 2020-01-02 03:35:09.000 408 99309 49858.5 4985850 408 99309 49858.5 4985850 -32762 32376 4638.66 463866 -125 127 2.5 250 +409 2 10399 99310 1.22822 298.22822 149.72822 14972.82282 1.22822 298.22824 149.72822 14972.82286 1.22822 298.22822 149.72822 14972.82200 2020-01-01 2020-01-02 2020-01-01 00:06:49 2020-01-02 03:35:10 2020-01-01 00:06:49.000 2020-01-02 03:35:10.000 409 99310 49859.5 4985950 409 99310 49859.5 4985950 -32761 32377 4639.66 463966 -128 127 0.94 94 +41 2 10031 99941 0.12312 300.12312 150.12312 15162.43543 0.12312 300.1231 150.12312 15162.43521 0.12312 300.12312 150.12312 15162.43512 2020-01-01 2020-01-02 2020-01-01 00:00:41 2020-01-02 03:45:41 2020-01-01 00:00:41.000 2020-01-02 03:45:41.000 41 99941 49991 5049091 41 99941 49991 5049091 -32528 32407 4570.009900990099 461571 -128 127 0.0297029702970297 3 +410 2 10400 99311 1.23123 298.23123 149.73123 14973.12312 1.23123 298.23123 149.73123 14973.12316 1.23123 298.23123 149.73122999999998 14973.12300 2020-01-01 2020-01-02 2020-01-01 00:06:50 2020-01-02 03:35:11 2020-01-01 00:06:50.000 2020-01-02 03:35:11.000 410 99311 49860.5 4986050 410 99311 49860.5 4986050 -32760 32378 4640.66 464066 -128 127 -0.62 -62 411 2 10401 99312 1.23423 298.23423 149.73423 14973.42342 1.23423 298.23422 149.73423 14973.42345 1.23423 298.23423 149.73423 14973.42300 2020-01-01 2020-01-02 2020-01-01 00:06:51 2020-01-02 03:35:12 2020-01-01 00:06:51.000 2020-01-02 03:35:12.000 411 99312 49861.5 4986150 411 99312 49861.5 4986150 -32759 32379 4641.66 464166 -128 123 -2.18 -218 -412 2 10402 99313 1.23723 298.23723 149.73723 14973.72372 1.23723 298.23724 149.73724 14973.72405 1.23723 298.23723 149.73723000000007 14973.72300 2020-01-01 2020-01-02 2020-01-01 00:06:52 2020-01-02 03:35:13 2020-01-01 00:06:52.000 2020-01-02 03:35:13.000 412 99313 49862.5 4986250 412 99313 49862.5 4986250 -32758 32380 4642.66 464266 -127 124 -1.18 -118 -413 2 10403 99314 1.24024 298.24024 149.74024 14974.02402 1.24024 298.24023 149.74023 14974.02374 1.24024 298.24024 149.74023999999983 14974.02400 2020-01-01 2020-01-02 2020-01-01 00:06:53 2020-01-02 03:35:14 2020-01-01 00:06:53.000 2020-01-02 03:35:14.000 413 99314 49863.5 4986350 413 99314 49863.5 4986350 -32757 32381 4643.66 464366 -126 125 -0.18 -18 -414 2 10404 99315 1.24324 298.24324 149.74324 14974.32432 1.24324 298.24326 149.74324 14974.32433 1.24324 298.24324 149.74323999999984 14974.32400 2020-01-01 2020-01-02 2020-01-01 00:06:54 2020-01-02 03:35:15 2020-01-01 00:06:54.000 2020-01-02 03:35:15.000 414 99315 49864.5 4986450 414 99315 49864.5 4986450 -32756 32382 4644.66 464466 -125 126 0.82 82 -415 2 10405 99316 1.24624 298.24624 149.74624 14974.62462 1.24624 298.24625 149.74624 14974.62463 1.24624 298.24624 149.74624000000023 14974.62400 2020-01-01 2020-01-02 2020-01-01 00:06:55 2020-01-02 03:35:16 2020-01-01 00:06:55.000 2020-01-02 03:35:16.000 415 99316 49865.5 4986550 415 99316 49865.5 4986550 -32755 32383 4645.66 464566 -124 127 1.82 182 -416 2 10406 99317 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74923999999987 14974.92400 2020-01-01 2020-01-02 2020-01-01 00:06:56 2020-01-02 03:35:17 2020-01-01 00:06:56.000 2020-01-02 03:35:17.000 416 99317 49866.5 4986650 416 99317 49866.5 4986650 -32754 32384 4646.66 464666 -128 127 0.26 26 -417 2 10407 99318 1.25225 298.25225 149.75225 14975.22522 1.25225 298.25226 149.75225 14975.22552 1.25225 298.25225 149.75224999999975 14975.22500 2020-01-01 2020-01-02 2020-01-01 00:06:57 2020-01-02 03:35:18 2020-01-01 00:06:57.000 2020-01-02 03:35:18.000 417 99318 49867.5 4986750 417 99318 49867.5 4986750 -32753 32385 4647.66 464766 -128 123 -1.3 -130 -418 2 10408 99319 1.25525 298.25525 149.75525 14975.52552 1.25525 298.25525 149.75525 14975.52521 1.25525 298.25525 149.7552500000001 14975.52500 2020-01-01 2020-01-02 2020-01-01 00:06:58 2020-01-02 03:35:19 2020-01-01 00:06:58.000 2020-01-02 03:35:19.000 418 99319 49868.5 4986850 418 99319 49868.5 4986850 -32752 32386 4648.66 464866 -127 124 -0.3 -30 -419 2 10409 99320 1.25825 298.25825 149.75825 14975.82582 1.25825 298.25827 149.75825 14975.8258 1.25825 298.25825 149.75825000000012 14975.82500 2020-01-01 2020-01-02 2020-01-01 00:06:59 2020-01-02 03:35:20 2020-01-01 00:06:59.000 2020-01-02 03:35:20.000 419 99320 49869.5 4986950 419 99320 49869.5 4986950 -32751 32387 4649.66 464966 -126 125 0.7 70 -42 2 10032 99942 0.12612 300.12612 150.12612 15162.73873 0.12612 300.12613 150.12612 15162.73896 0.12612 300.12612 150.12612000000007 15162.73812 2020-01-01 2020-01-02 2020-01-01 00:00:42 2020-01-02 03:45:42 2020-01-01 00:00:42.000 2020-01-02 03:45:42.000 42 99942 49992 5049192 42 99942 49992 5049192 -32527 32408 4571.009900990099 461672 -128 127 -1.504950495049505 -152 -420 2 10410 99321 1.26126 298.26126 149.76126 14976.12612 1.26126 298.26126 149.76126 14976.12609 1.26126 298.26126 149.76125999999994 14976.12600 2020-01-01 2020-01-02 2020-01-01 00:07:00 2020-01-02 03:35:21 2020-01-01 00:07:00.000 2020-01-02 03:35:21.000 420 99321 49870.5 4987050 420 99321 49870.5 4987050 -32750 32388 4650.66 465066 -125 126 1.7 170 -421 2 10411 99322 1.26426 298.26426 149.76426 14976.42642 1.26426 298.26425 149.76426 14976.4264 1.26426 298.26426 149.76425999999995 14976.42600 2020-01-01 2020-01-02 2020-01-01 00:07:01 2020-01-02 03:35:22 2020-01-01 00:07:01.000 2020-01-02 03:35:22.000 421 99322 49871.5 4987150 421 99322 49871.5 4987150 -32749 32389 4651.66 465166 -124 127 2.7 270 +412 2 10402 99313 1.23723 298.23723 149.73723 14973.72372 1.23723 298.23724 149.73724 14973.72405 1.23723 298.23723 149.73723 14973.72300 2020-01-01 2020-01-02 2020-01-01 00:06:52 2020-01-02 03:35:13 2020-01-01 00:06:52.000 2020-01-02 03:35:13.000 412 99313 49862.5 4986250 412 99313 49862.5 4986250 -32758 32380 4642.66 464266 -127 124 -1.18 -118 +413 2 10403 99314 1.24024 298.24024 149.74024 14974.02402 1.24024 298.24023 149.74023 14974.02374 1.24024 298.24024 149.74024 14974.02400 2020-01-01 2020-01-02 2020-01-01 00:06:53 2020-01-02 03:35:14 2020-01-01 00:06:53.000 2020-01-02 03:35:14.000 413 99314 49863.5 4986350 413 99314 49863.5 4986350 -32757 32381 4643.66 464366 -126 125 -0.18 -18 +414 2 10404 99315 1.24324 298.24324 149.74324 14974.32432 1.24324 298.24326 149.74324 14974.32433 1.24324 298.24324 149.74324000000001 14974.32400 2020-01-01 2020-01-02 2020-01-01 00:06:54 2020-01-02 03:35:15 2020-01-01 00:06:54.000 2020-01-02 03:35:15.000 414 99315 49864.5 4986450 414 99315 49864.5 4986450 -32756 32382 4644.66 464466 -125 126 0.82 82 +415 2 10405 99316 1.24624 298.24624 149.74624 14974.62462 1.24624 298.24625 149.74624 14974.62463 1.24624 298.24624 149.74624 14974.62400 2020-01-01 2020-01-02 2020-01-01 00:06:55 2020-01-02 03:35:16 2020-01-01 00:06:55.000 2020-01-02 03:35:16.000 415 99316 49865.5 4986550 415 99316 49865.5 4986550 -32755 32383 4645.66 464566 -124 127 1.82 182 +416 2 10406 99317 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924 14974.92492 1.24924 298.24924 149.74924000000001 14974.92400 2020-01-01 2020-01-02 2020-01-01 00:06:56 2020-01-02 03:35:17 2020-01-01 00:06:56.000 2020-01-02 03:35:17.000 416 99317 49866.5 4986650 416 99317 49866.5 4986650 -32754 32384 4646.66 464666 -128 127 0.26 26 +417 2 10407 99318 1.25225 298.25225 149.75225 14975.22522 1.25225 298.25226 149.75225 14975.22552 1.25225 298.25225 149.75225 14975.22500 2020-01-01 2020-01-02 2020-01-01 00:06:57 2020-01-02 03:35:18 2020-01-01 00:06:57.000 2020-01-02 03:35:18.000 417 99318 49867.5 4986750 417 99318 49867.5 4986750 -32753 32385 4647.66 464766 -128 123 -1.3 -130 +418 2 10408 99319 1.25525 298.25525 149.75525 14975.52552 1.25525 298.25525 149.75525 14975.52521 1.25525 298.25525 149.75525 14975.52500 2020-01-01 2020-01-02 2020-01-01 00:06:58 2020-01-02 03:35:19 2020-01-01 00:06:58.000 2020-01-02 03:35:19.000 418 99319 49868.5 4986850 418 99319 49868.5 4986850 -32752 32386 4648.66 464866 -127 124 -0.3 -30 +419 2 10409 99320 1.25825 298.25825 149.75825 14975.82582 1.25825 298.25827 149.75825 14975.8258 1.25825 298.25825 149.75825 14975.82500 2020-01-01 2020-01-02 2020-01-01 00:06:59 2020-01-02 03:35:20 2020-01-01 00:06:59.000 2020-01-02 03:35:20.000 419 99320 49869.5 4986950 419 99320 49869.5 4986950 -32751 32387 4649.66 464966 -126 125 0.7 70 +42 2 10032 99942 0.12612 300.12612 150.12612 15162.73873 0.12612 300.12613 150.12612 15162.73896 0.12612 300.12612 150.12612 15162.73812 2020-01-01 2020-01-02 2020-01-01 00:00:42 2020-01-02 03:45:42 2020-01-01 00:00:42.000 2020-01-02 03:45:42.000 42 99942 49992 5049192 42 99942 49992 5049192 -32527 32408 4571.009900990099 461672 -128 127 -1.504950495049505 -152 +420 2 10410 99321 1.26126 298.26126 149.76126 14976.12612 1.26126 298.26126 149.76126 14976.12609 1.26126 298.26126 149.76126 14976.12600 2020-01-01 2020-01-02 2020-01-01 00:07:00 2020-01-02 03:35:21 2020-01-01 00:07:00.000 2020-01-02 03:35:21.000 420 99321 49870.5 4987050 420 99321 49870.5 4987050 -32750 32388 4650.66 465066 -125 126 1.7 170 +421 2 10411 99322 1.26426 298.26426 149.76426 14976.42642 1.26426 298.26425 149.76426 14976.4264 1.26426 298.26426 149.76426 14976.42600 2020-01-01 2020-01-02 2020-01-01 00:07:01 2020-01-02 03:35:22 2020-01-01 00:07:01.000 2020-01-02 03:35:22.000 421 99322 49871.5 4987150 421 99322 49871.5 4987150 -32749 32389 4651.66 465166 -124 127 2.7 270 422 2 10412 99323 1.26726 298.26726 149.76726 14976.72672 1.26726 298.26727 149.76727 14976.72702 1.26726 298.26726 149.76726 14976.72600 2020-01-01 2020-01-02 2020-01-01 00:07:02 2020-01-02 03:35:23 2020-01-01 00:07:02.000 2020-01-02 03:35:23.000 422 99323 49872.5 4987250 422 99323 49872.5 4987250 -32748 32390 4652.66 465266 -128 127 1.14 114 -423 2 10413 99324 1.27027 298.27027 149.77027 14977.02702 1.27027 298.27026 149.77026 14977.02667 1.27027 298.27027 149.7702700000002 14977.02700 2020-01-01 2020-01-02 2020-01-01 00:07:03 2020-01-02 03:35:24 2020-01-01 00:07:03.000 2020-01-02 03:35:24.000 423 99324 49873.5 4987350 423 99324 49873.5 4987350 -32747 32391 4653.66 465366 -128 123 -0.42 -42 -424 2 10414 99325 1.27327 298.27327 149.77327 14977.32732 1.27327 298.2733 149.77327 14977.32727 1.27327 298.27327 149.77326999999985 14977.32700 2020-01-01 2020-01-02 2020-01-01 00:07:04 2020-01-02 03:35:25 2020-01-01 00:07:04.000 2020-01-02 03:35:25.000 424 99325 49874.5 4987450 424 99325 49874.5 4987450 -32746 32392 4654.66 465466 -127 124 0.58 58 -425 2 10415 99326 1.27627 298.27627 149.77627 14977.62762 1.27627 298.27628 149.77627 14977.62756 1.27627 298.27627 149.77627000000012 14977.62700 2020-01-01 2020-01-02 2020-01-01 00:07:05 2020-01-02 03:35:26 2020-01-01 00:07:05.000 2020-01-02 03:35:26.000 425 99326 49875.5 4987550 425 99326 49875.5 4987550 -32745 32393 4655.66 465566 -126 125 1.58 158 -426 2 10416 99327 1.27927 298.27927 149.77927 14977.92792 1.27927 298.27927 149.77927 14977.92787 1.27927 298.27927 149.7792699999999 14977.92700 2020-01-01 2020-01-02 2020-01-01 00:07:06 2020-01-02 03:35:27 2020-01-01 00:07:06.000 2020-01-02 03:35:27.000 426 99327 49876.5 4987650 426 99327 49876.5 4987650 -32744 32394 4656.66 465666 -125 126 2.58 258 -427 2 10417 99328 1.28228 298.28228 149.78228 14978.22822 1.28228 298.2823 149.78228 14978.22849 1.28228 298.28228 149.78227999999996 14978.22800 2020-01-01 2020-01-02 2020-01-01 00:07:07 2020-01-02 03:35:28 2020-01-01 00:07:07.000 2020-01-02 03:35:28.000 427 99328 49877.5 4987750 427 99328 49877.5 4987750 -32743 32395 4657.66 465766 -124 127 3.58 358 -428 2 10418 99329 1.28528 298.28528 149.78528 14978.52852 1.28528 298.28528 149.78528 14978.52815 1.28528 298.28528 149.78528000000003 14978.52800 2020-01-01 2020-01-02 2020-01-01 00:07:08 2020-01-02 03:35:29 2020-01-01 00:07:08.000 2020-01-02 03:35:29.000 428 99329 49878.5 4987850 428 99329 49878.5 4987850 -32742 32396 4658.66 465866 -128 127 2.02 202 -429 2 10419 99330 1.28828 298.28828 149.78828 14978.82882 1.28828 298.2883 149.78828 14978.8289 1.28828 298.28828 149.7882800000001 14978.82800 2020-01-01 2020-01-02 2020-01-01 00:07:09 2020-01-02 03:35:30 2020-01-01 00:07:09.000 2020-01-02 03:35:30.000 429 99330 49879.5 4987950 429 99330 49879.5 4987950 -32741 32397 4659.66 465966 -128 127 0.46 46 -43 2 10033 99943 0.12912 300.12912 150.12912 15163.04204 0.12912 300.12912 150.12912 15163.04211 0.12912 300.12912 150.1291199999998 15163.04112 2020-01-01 2020-01-02 2020-01-01 00:00:43 2020-01-02 03:45:43 2020-01-01 00:00:43.000 2020-01-02 03:45:43.000 43 99943 49993 5049293 43 99943 49993 5049293 -32526 32409 4572.009900990099 461773 -128 124 -3.0396039603960396 -307 -430 2 10420 99331 1.29129 298.29129 149.79129 14979.12912 1.29129 298.2913 149.79129 14979.12904 1.29129 298.29129 149.7912899999999 14979.12900 2020-01-01 2020-01-02 2020-01-01 00:07:10 2020-01-02 03:35:31 2020-01-01 00:07:10.000 2020-01-02 03:35:31.000 430 99331 49880.5 4988050 430 99331 49880.5 4988050 -32740 32398 4660.66 466066 -128 124 -1.1 -110 -431 2 10421 99332 1.29429 298.29429 149.79429 14979.42942 1.29429 298.29428 149.79429 14979.42933 1.29429 298.29429 149.79428999999988 14979.42900 2020-01-01 2020-01-02 2020-01-01 00:07:11 2020-01-02 03:35:32 2020-01-01 00:07:11.000 2020-01-02 03:35:32.000 431 99332 49881.5 4988150 431 99332 49881.5 4988150 -32739 32399 4661.66 466166 -127 125 -0.1 -10 -432 2 10422 99333 1.29729 298.29729 149.79729 14979.72972 1.29729 298.2973 149.79729 14979.72996 1.29729 298.29729 149.79729000000023 14979.72900 2020-01-01 2020-01-02 2020-01-01 00:07:12 2020-01-02 03:35:33 2020-01-01 00:07:12.000 2020-01-02 03:35:33.000 432 99333 49882.5 4988250 432 99333 49882.5 4988250 -32738 32400 4662.66 466266 -126 126 0.9 90 -433 2 10423 99334 1.3003 298.3003 149.8003 14980.03003 1.3003 298.3003 149.80029 14980.02962 1.30030 298.30030 149.8003000000001 14980.03000 2020-01-01 2020-01-02 2020-01-01 00:07:13 2020-01-02 03:35:34 2020-01-01 00:07:13.000 2020-01-02 03:35:34.000 433 99334 49883.5 4988350 433 99334 49883.5 4988350 -32737 32401 4663.66 466366 -125 127 1.9 190 -434 2 10424 99335 1.3033 298.3033 149.8033 14980.33033 1.3033 298.3033 149.8033 14980.33037 1.30330 298.30330 149.80329999999978 14980.33000 2020-01-01 2020-01-02 2020-01-01 00:07:14 2020-01-02 03:35:35 2020-01-01 00:07:14.000 2020-01-02 03:35:35.000 434 99335 49884.5 4988450 434 99335 49884.5 4988450 -32736 32402 4664.66 466466 -128 127 0.34 34 -435 2 10425 99336 1.3063 298.3063 149.8063 14980.63063 1.3063 298.3063 149.8063 14980.63051 1.30630 298.30630 149.80630000000014 14980.63000 2020-01-01 2020-01-02 2020-01-01 00:07:15 2020-01-02 03:35:36 2020-01-01 00:07:15.000 2020-01-02 03:35:36.000 435 99336 49885.5 4988550 435 99336 49885.5 4988550 -32735 32403 4665.66 466566 -128 127 -1.22 -122 -436 2 10426 99337 1.3093 298.3093 149.8093 14980.93093 1.3093 298.3093 149.8093 14980.93084 1.30930 298.30930 149.80930000000018 14980.93000 2020-01-01 2020-01-02 2020-01-01 00:07:16 2020-01-02 03:35:37 2020-01-01 00:07:16.000 2020-01-02 03:35:37.000 436 99337 49886.5 4988650 436 99337 49886.5 4988650 -32734 32404 4666.66 466666 -128 123 -2.78 -278 -437 2 10427 99338 1.31231 298.31231 149.81231 14981.23123 1.31231 298.31232 149.81231 14981.23143 1.31231 298.31231 149.8123099999999 14981.23100 2020-01-01 2020-01-02 2020-01-01 00:07:17 2020-01-02 03:35:38 2020-01-01 00:07:17.000 2020-01-02 03:35:38.000 437 99338 49887.5 4988750 437 99338 49887.5 4988750 -32733 32405 4667.66 466766 -127 124 -1.78 -178 -438 2 10428 99339 1.31531 298.31531 149.81531 14981.53153 1.31531 298.3153 149.81531 14981.53173 1.31531 298.31531 149.81530999999998 14981.53100 2020-01-01 2020-01-02 2020-01-01 00:07:18 2020-01-02 03:35:39 2020-01-01 00:07:18.000 2020-01-02 03:35:39.000 438 99339 49888.5 4988850 438 99339 49888.5 4988850 -32732 32406 4668.66 466866 -126 125 -0.78 -78 -439 2 10429 99340 1.31831 298.31831 149.81831 14981.83183 1.31831 298.31833 149.81831 14981.83184 1.31831 298.31831 149.81831000000005 14981.83100 2020-01-01 2020-01-02 2020-01-01 00:07:19 2020-01-02 03:35:40 2020-01-01 00:07:19.000 2020-01-02 03:35:40.000 439 99340 49889.5 4988950 439 99340 49889.5 4988950 -32731 32407 4669.66 466966 -125 126 0.22 22 +423 2 10413 99324 1.27027 298.27027 149.77027 14977.02702 1.27027 298.27026 149.77026 14977.02667 1.27027 298.27027 149.77027 14977.02700 2020-01-01 2020-01-02 2020-01-01 00:07:03 2020-01-02 03:35:24 2020-01-01 00:07:03.000 2020-01-02 03:35:24.000 423 99324 49873.5 4987350 423 99324 49873.5 4987350 -32747 32391 4653.66 465366 -128 123 -0.42 -42 +424 2 10414 99325 1.27327 298.27327 149.77327 14977.32732 1.27327 298.2733 149.77327 14977.32727 1.27327 298.27327 149.77327 14977.32700 2020-01-01 2020-01-02 2020-01-01 00:07:04 2020-01-02 03:35:25 2020-01-01 00:07:04.000 2020-01-02 03:35:25.000 424 99325 49874.5 4987450 424 99325 49874.5 4987450 -32746 32392 4654.66 465466 -127 124 0.58 58 +425 2 10415 99326 1.27627 298.27627 149.77627 14977.62762 1.27627 298.27628 149.77627 14977.62756 1.27627 298.27627 149.77627 14977.62700 2020-01-01 2020-01-02 2020-01-01 00:07:05 2020-01-02 03:35:26 2020-01-01 00:07:05.000 2020-01-02 03:35:26.000 425 99326 49875.5 4987550 425 99326 49875.5 4987550 -32745 32393 4655.66 465566 -126 125 1.58 158 +426 2 10416 99327 1.27927 298.27927 149.77927 14977.92792 1.27927 298.27927 149.77927 14977.92787 1.27927 298.27927 149.77927 14977.92700 2020-01-01 2020-01-02 2020-01-01 00:07:06 2020-01-02 03:35:27 2020-01-01 00:07:06.000 2020-01-02 03:35:27.000 426 99327 49876.5 4987650 426 99327 49876.5 4987650 -32744 32394 4656.66 465666 -125 126 2.58 258 +427 2 10417 99328 1.28228 298.28228 149.78228 14978.22822 1.28228 298.2823 149.78228 14978.22849 1.28228 298.28228 149.78228 14978.22800 2020-01-01 2020-01-02 2020-01-01 00:07:07 2020-01-02 03:35:28 2020-01-01 00:07:07.000 2020-01-02 03:35:28.000 427 99328 49877.5 4987750 427 99328 49877.5 4987750 -32743 32395 4657.66 465766 -124 127 3.58 358 +428 2 10418 99329 1.28528 298.28528 149.78528 14978.52852 1.28528 298.28528 149.78528 14978.52815 1.28528 298.28528 149.78528 14978.52800 2020-01-01 2020-01-02 2020-01-01 00:07:08 2020-01-02 03:35:29 2020-01-01 00:07:08.000 2020-01-02 03:35:29.000 428 99329 49878.5 4987850 428 99329 49878.5 4987850 -32742 32396 4658.66 465866 -128 127 2.02 202 +429 2 10419 99330 1.28828 298.28828 149.78828 14978.82882 1.28828 298.2883 149.78828 14978.8289 1.28828 298.28828 149.78828 14978.82800 2020-01-01 2020-01-02 2020-01-01 00:07:09 2020-01-02 03:35:30 2020-01-01 00:07:09.000 2020-01-02 03:35:30.000 429 99330 49879.5 4987950 429 99330 49879.5 4987950 -32741 32397 4659.66 465966 -128 127 0.46 46 +43 2 10033 99943 0.12912 300.12912 150.12912 15163.04204 0.12912 300.12912 150.12912 15163.04211 0.12912 300.12912 150.12912 15163.04112 2020-01-01 2020-01-02 2020-01-01 00:00:43 2020-01-02 03:45:43 2020-01-01 00:00:43.000 2020-01-02 03:45:43.000 43 99943 49993 5049293 43 99943 49993 5049293 -32526 32409 4572.009900990099 461773 -128 124 -3.0396039603960396 -307 +430 2 10420 99331 1.29129 298.29129 149.79129 14979.12912 1.29129 298.2913 149.79129 14979.12904 1.29129 298.29129 149.79129 14979.12900 2020-01-01 2020-01-02 2020-01-01 00:07:10 2020-01-02 03:35:31 2020-01-01 00:07:10.000 2020-01-02 03:35:31.000 430 99331 49880.5 4988050 430 99331 49880.5 4988050 -32740 32398 4660.66 466066 -128 124 -1.1 -110 +431 2 10421 99332 1.29429 298.29429 149.79429 14979.42942 1.29429 298.29428 149.79429 14979.42933 1.29429 298.29429 149.79429 14979.42900 2020-01-01 2020-01-02 2020-01-01 00:07:11 2020-01-02 03:35:32 2020-01-01 00:07:11.000 2020-01-02 03:35:32.000 431 99332 49881.5 4988150 431 99332 49881.5 4988150 -32739 32399 4661.66 466166 -127 125 -0.1 -10 +432 2 10422 99333 1.29729 298.29729 149.79729 14979.72972 1.29729 298.2973 149.79729 14979.72996 1.29729 298.29729 149.79729 14979.72900 2020-01-01 2020-01-02 2020-01-01 00:07:12 2020-01-02 03:35:33 2020-01-01 00:07:12.000 2020-01-02 03:35:33.000 432 99333 49882.5 4988250 432 99333 49882.5 4988250 -32738 32400 4662.66 466266 -126 126 0.9 90 +433 2 10423 99334 1.3003 298.3003 149.8003 14980.03003 1.3003 298.3003 149.80029 14980.02962 1.30030 298.30030 149.8003 14980.03000 2020-01-01 2020-01-02 2020-01-01 00:07:13 2020-01-02 03:35:34 2020-01-01 00:07:13.000 2020-01-02 03:35:34.000 433 99334 49883.5 4988350 433 99334 49883.5 4988350 -32737 32401 4663.66 466366 -125 127 1.9 190 +434 2 10424 99335 1.3033 298.3033 149.8033 14980.33033 1.3033 298.3033 149.8033 14980.33037 1.30330 298.30330 149.8033 14980.33000 2020-01-01 2020-01-02 2020-01-01 00:07:14 2020-01-02 03:35:35 2020-01-01 00:07:14.000 2020-01-02 03:35:35.000 434 99335 49884.5 4988450 434 99335 49884.5 4988450 -32736 32402 4664.66 466466 -128 127 0.34 34 +435 2 10425 99336 1.3063 298.3063 149.8063 14980.63063 1.3063 298.3063 149.8063 14980.63051 1.30630 298.30630 149.8063 14980.63000 2020-01-01 2020-01-02 2020-01-01 00:07:15 2020-01-02 03:35:36 2020-01-01 00:07:15.000 2020-01-02 03:35:36.000 435 99336 49885.5 4988550 435 99336 49885.5 4988550 -32735 32403 4665.66 466566 -128 127 -1.22 -122 +436 2 10426 99337 1.3093 298.3093 149.8093 14980.93093 1.3093 298.3093 149.8093 14980.93084 1.30930 298.30930 149.8093 14980.93000 2020-01-01 2020-01-02 2020-01-01 00:07:16 2020-01-02 03:35:37 2020-01-01 00:07:16.000 2020-01-02 03:35:37.000 436 99337 49886.5 4988650 436 99337 49886.5 4988650 -32734 32404 4666.66 466666 -128 123 -2.78 -278 +437 2 10427 99338 1.31231 298.31231 149.81231 14981.23123 1.31231 298.31232 149.81231 14981.23143 1.31231 298.31231 149.81231 14981.23100 2020-01-01 2020-01-02 2020-01-01 00:07:17 2020-01-02 03:35:38 2020-01-01 00:07:17.000 2020-01-02 03:35:38.000 437 99338 49887.5 4988750 437 99338 49887.5 4988750 -32733 32405 4667.66 466766 -127 124 -1.78 -178 +438 2 10428 99339 1.31531 298.31531 149.81531 14981.53153 1.31531 298.3153 149.81531 14981.53173 1.31531 298.31531 149.81531 14981.53100 2020-01-01 2020-01-02 2020-01-01 00:07:18 2020-01-02 03:35:39 2020-01-01 00:07:18.000 2020-01-02 03:35:39.000 438 99339 49888.5 4988850 438 99339 49888.5 4988850 -32732 32406 4668.66 466866 -126 125 -0.78 -78 +439 2 10429 99340 1.31831 298.31831 149.81831 14981.83183 1.31831 298.31833 149.81831 14981.83184 1.31831 298.31831 149.81831 14981.83100 2020-01-01 2020-01-02 2020-01-01 00:07:19 2020-01-02 03:35:40 2020-01-01 00:07:19.000 2020-01-02 03:35:40.000 439 99340 49889.5 4988950 439 99340 49889.5 4988950 -32731 32407 4669.66 466966 -125 126 0.22 22 44 2 10034 99944 0.13213 300.13213 150.13213 15163.34534 0.13213 300.13214 150.13213 15163.34525 0.13213 300.13213 150.13213 15163.34513 2020-01-01 2020-01-02 2020-01-01 00:00:44 2020-01-02 03:45:44 2020-01-01 00:00:44.000 2020-01-02 03:45:44.000 44 99944 49994 5049394 44 99944 49994 5049394 -32525 32410 4573.009900990099 461874 -127 125 -2.0396039603960396 -206 -440 2 10430 99341 1.32132 298.32132 149.82132 14982.13213 1.32132 298.32132 149.82131 14982.13197 1.32132 298.32132 149.82131999999984 14982.13200 2020-01-01 2020-01-02 2020-01-01 00:07:20 2020-01-02 03:35:41 2020-01-01 00:07:20.000 2020-01-02 03:35:41.000 440 99341 49890.5 4989050 440 99341 49890.5 4989050 -32730 32408 4670.66 467066 -124 127 1.22 122 -441 2 10431 99342 1.32432 298.32432 149.82432 14982.43243 1.32432 298.3243 149.82432 14982.4323 1.32432 298.32432 149.8243199999999 14982.43200 2020-01-01 2020-01-02 2020-01-01 00:07:21 2020-01-02 03:35:42 2020-01-01 00:07:21.000 2020-01-02 03:35:42.000 441 99342 49891.5 4989150 441 99342 49891.5 4989150 -32729 32409 4671.66 467166 -128 127 -0.34 -34 -442 2 10432 99343 1.32732 298.32732 149.82732 14982.73273 1.32732 298.32733 149.82732 14982.7329 1.32732 298.32732 149.82732000000016 14982.73200 2020-01-01 2020-01-02 2020-01-01 00:07:22 2020-01-02 03:35:43 2020-01-01 00:07:22.000 2020-01-02 03:35:43.000 442 99343 49892.5 4989250 442 99343 49892.5 4989250 -32728 32410 4672.66 467266 -128 123 -1.9 -190 -443 2 10433 99344 1.33033 298.33033 149.83033 14983.03303 1.33033 298.33032 149.83033 14983.03319 1.33033 298.33033 149.83033000000006 14983.03300 2020-01-01 2020-01-02 2020-01-01 00:07:23 2020-01-02 03:35:44 2020-01-01 00:07:23.000 2020-01-02 03:35:44.000 443 99344 49893.5 4989350 443 99344 49893.5 4989350 -32727 32411 4673.66 467366 -127 124 -0.9 -90 -444 2 10434 99345 1.33333 298.33333 149.83333 14983.33333 1.33333 298.33334 149.83333 14983.33331 1.33333 298.33333 149.83333 14983.33300 2020-01-01 2020-01-02 2020-01-01 00:07:24 2020-01-02 03:35:45 2020-01-01 00:07:24.000 2020-01-02 03:35:45.000 444 99345 49894.5 4989450 444 99345 49894.5 4989450 -32726 32412 4674.66 467466 -126 125 0.1 10 -445 2 10435 99346 1.33633 298.33633 149.83633 14983.63363 1.33633 298.33633 149.83633 14983.63348 1.33633 298.33633 149.83633000000006 14983.63300 2020-01-01 2020-01-02 2020-01-01 00:07:25 2020-01-02 03:35:46 2020-01-01 00:07:25.000 2020-01-02 03:35:46.000 445 99346 49895.5 4989550 445 99346 49895.5 4989550 -32725 32413 4675.66 467566 -125 126 1.1 110 -446 2 10436 99347 1.33933 298.33933 149.83933 14983.93393 1.33933 298.33932 149.83933 14983.93378 1.33933 298.33933 149.83933000000007 14983.93300 2020-01-01 2020-01-02 2020-01-01 00:07:26 2020-01-02 03:35:47 2020-01-01 00:07:26.000 2020-01-02 03:35:47.000 446 99347 49896.5 4989650 446 99347 49896.5 4989650 -32724 32414 4676.66 467666 -124 127 2.1 210 -447 2 10437 99348 1.34234 298.34234 149.84234 14984.23423 1.34234 298.34235 149.84234 14984.23437 1.34234 298.34234 149.84233999999992 14984.23400 2020-01-01 2020-01-02 2020-01-01 00:07:27 2020-01-02 03:35:48 2020-01-01 00:07:27.000 2020-01-02 03:35:48.000 447 99348 49897.5 4989750 447 99348 49897.5 4989750 -32723 32415 4677.66 467766 -128 127 0.54 54 -448 2 10438 99349 1.34534 298.34534 149.84534 14984.53453 1.34534 298.34534 149.84534 14984.53466 1.34534 298.34534 149.8453399999999 14984.53400 2020-01-01 2020-01-02 2020-01-01 00:07:28 2020-01-02 03:35:49 2020-01-01 00:07:28.000 2020-01-02 03:35:49.000 448 99349 49898.5 4989850 448 99349 49898.5 4989850 -32722 32416 4678.66 467866 -128 123 -1.02 -102 -449 2 10439 99350 1.34834 298.34834 149.84834 14984.83483 1.34834 298.34836 149.84834 14984.83478 1.34834 298.34834 149.8483400000001 14984.83400 2020-01-01 2020-01-02 2020-01-01 00:07:29 2020-01-02 03:35:50 2020-01-01 00:07:29.000 2020-01-02 03:35:50.000 449 99350 49899.5 4989950 449 99350 49899.5 4989950 -32721 32417 4679.66 467966 -127 124 -0.02 -2 -45 2 10035 99945 0.13513 300.13513 150.13513 15163.64864 0.13513 300.13513 150.13513 15163.64839 0.13513 300.13513 150.13513000000006 15163.64813 2020-01-01 2020-01-02 2020-01-01 00:00:45 2020-01-02 03:45:45 2020-01-01 00:00:45.000 2020-01-02 03:45:45.000 45 99945 49995 5049495 45 99945 49995 5049495 -32524 32411 4574.009900990099 461975 -126 126 -1.0396039603960396 -105 -450 2 10440 99351 1.35135 298.35135 149.85135 14985.13513 1.35135 298.35135 149.85134 14985.13495 1.35135 298.35135 149.85135000000014 14985.13500 2020-01-01 2020-01-02 2020-01-01 00:07:30 2020-01-02 03:35:51 2020-01-01 00:07:30.000 2020-01-02 03:35:51.000 450 99351 49900.5 4990050 450 99351 49900.5 4990050 -32720 32418 4680.66 468066 -126 125 0.98 98 -451 2 10441 99352 1.35435 298.35435 149.85435 14985.43543 1.35435 298.35434 149.85435 14985.43525 1.35435 298.35435 149.85434999999978 14985.43500 2020-01-01 2020-01-02 2020-01-01 00:07:31 2020-01-02 03:35:52 2020-01-01 00:07:31.000 2020-01-02 03:35:52.000 451 99352 49901.5 4990150 451 99352 49901.5 4990150 -32719 32419 4681.66 468166 -125 126 1.98 198 -452 2 10442 99353 1.35735 298.35735 149.85735 14985.73573 1.35735 298.35736 149.85736 14985.736 1.35735 298.35735 149.85735000000017 14985.73500 2020-01-01 2020-01-02 2020-01-01 00:07:32 2020-01-02 03:35:53 2020-01-01 00:07:32.000 2020-01-02 03:35:53.000 452 99353 49902.5 4990250 452 99353 49902.5 4990250 -32718 32420 4682.66 468266 -124 127 2.98 298 -453 2 10443 99354 1.36036 298.36036 149.86036 14986.03603 1.36036 298.36035 149.86036 14986.03614 1.36036 298.36036 149.86036000000007 14986.03600 2020-01-01 2020-01-02 2020-01-01 00:07:33 2020-01-02 03:35:54 2020-01-01 00:07:33.000 2020-01-02 03:35:54.000 453 99354 49903.5 4990350 453 99354 49903.5 4990350 -32717 32421 4683.66 468366 -128 127 1.42 142 -454 2 10444 99355 1.36336 298.36336 149.86336 14986.33633 1.36336 298.36337 149.86336 14986.33629 1.36336 298.36336 149.8633599999999 14986.33600 2020-01-01 2020-01-02 2020-01-01 00:07:34 2020-01-02 03:35:55 2020-01-01 00:07:34.000 2020-01-02 03:35:55.000 454 99355 49904.5 4990450 454 99355 49904.5 4990450 -32716 32422 4684.66 468466 -128 127 -0.14 -14 -455 2 10445 99356 1.36636 298.36636 149.86636 14986.63663 1.36636 298.36636 149.86636 14986.63641 1.36636 298.36636 149.86636 14986.63600 2020-01-01 2020-01-02 2020-01-01 00:07:35 2020-01-02 03:35:56 2020-01-01 00:07:35.000 2020-01-02 03:35:56.000 455 99356 49905.5 4990550 455 99356 49905.5 4990550 -32715 32423 4685.66 468566 -128 124 -1.7 -170 -456 2 10446 99357 1.36936 298.36936 149.86936 14986.93693 1.36936 298.36935 149.86936 14986.93672 1.36936 298.36936 149.86936000000009 14986.93600 2020-01-01 2020-01-02 2020-01-01 00:07:36 2020-01-02 03:35:57 2020-01-01 00:07:36.000 2020-01-02 03:35:57.000 456 99357 49906.5 4990650 456 99357 49906.5 4990650 -32714 32424 4686.66 468666 -127 125 -0.7 -70 -457 2 10447 99358 1.37237 298.37237 149.87237 14987.23723 1.37237 298.37238 149.87237 14987.23747 1.37237 298.37237 149.87236999999976 14987.23700 2020-01-01 2020-01-02 2020-01-01 00:07:37 2020-01-02 03:35:58 2020-01-01 00:07:37.000 2020-01-02 03:35:58.000 457 99358 49907.5 4990750 457 99358 49907.5 4990750 -32713 32425 4687.66 468766 -126 126 0.3 30 -458 2 10448 99359 1.37537 298.37537 149.87537 14987.53753 1.37537 298.37537 149.87537 14987.5376 1.37537 298.37537 149.87536999999986 14987.53700 2020-01-01 2020-01-02 2020-01-01 00:07:38 2020-01-02 03:35:59 2020-01-01 00:07:38.000 2020-01-02 03:35:59.000 458 99359 49908.5 4990850 458 99359 49908.5 4990850 -32712 32426 4688.66 468866 -125 127 1.3 130 -459 2 10449 99360 1.37837 298.37837 149.87837 14987.83783 1.37837 298.3784 149.87837 14987.83775 1.37837 298.37837 149.87837000000016 14987.83700 2020-01-01 2020-01-02 2020-01-01 00:07:39 2020-01-02 03:36:00 2020-01-01 00:07:39.000 2020-01-02 03:36:00.000 459 99360 49909.5 4990950 459 99360 49909.5 4990950 -32711 32427 4689.66 468966 -128 127 -0.26 -26 -46 2 10036 99946 0.13813 300.13813 150.13813 15163.95195 0.13813 300.13815 150.13814 15163.95214 0.13813 300.13813 150.13812999999993 15163.95113 2020-01-01 2020-01-02 2020-01-01 00:00:46 2020-01-02 03:45:46 2020-01-01 00:00:46.000 2020-01-02 03:45:46.000 46 99946 49996 5049596 46 99946 49996 5049596 -32523 32412 4575.009900990099 462076 -125 127 -0.039603960396039604 -4 -460 2 10450 99361 1.38138 298.38138 149.88138 14988.13813 1.38138 298.38138 149.88137 14988.13789 1.38138 298.38138 149.8813800000001 14988.13800 2020-01-01 2020-01-02 2020-01-01 00:07:40 2020-01-02 03:36:01 2020-01-01 00:07:40.000 2020-01-02 03:36:01.000 460 99361 49910.5 4991050 460 99361 49910.5 4991050 -32710 32428 4690.66 469066 -128 127 -1.82 -182 -461 2 10451 99362 1.38438 298.38438 149.88438 14988.43843 1.38438 298.3844 149.88438 14988.43864 1.38438 298.38438 149.88438000000002 14988.43800 2020-01-01 2020-01-02 2020-01-01 00:07:41 2020-01-02 03:36:02 2020-01-01 00:07:41.000 2020-01-02 03:36:02.000 461 99362 49911.5 4991150 461 99362 49911.5 4991150 -32709 32429 4691.66 469166 -128 123 -3.38 -338 -462 2 10452 99363 1.38738 298.38738 149.88738 14988.73873 1.38738 298.3874 149.88738 14988.73894 1.38738 298.38738 149.8873800000001 14988.73800 2020-01-01 2020-01-02 2020-01-01 00:07:42 2020-01-02 03:36:03 2020-01-01 00:07:42.000 2020-01-02 03:36:03.000 462 99363 49912.5 4991250 462 99363 49912.5 4991250 -32708 32430 4692.66 469266 -127 124 -2.38 -238 +440 2 10430 99341 1.32132 298.32132 149.82132 14982.13213 1.32132 298.32132 149.82131 14982.13197 1.32132 298.32132 149.82132 14982.13200 2020-01-01 2020-01-02 2020-01-01 00:07:20 2020-01-02 03:35:41 2020-01-01 00:07:20.000 2020-01-02 03:35:41.000 440 99341 49890.5 4989050 440 99341 49890.5 4989050 -32730 32408 4670.66 467066 -124 127 1.22 122 +441 2 10431 99342 1.32432 298.32432 149.82432 14982.43243 1.32432 298.3243 149.82432 14982.4323 1.32432 298.32432 149.82432 14982.43200 2020-01-01 2020-01-02 2020-01-01 00:07:21 2020-01-02 03:35:42 2020-01-01 00:07:21.000 2020-01-02 03:35:42.000 441 99342 49891.5 4989150 441 99342 49891.5 4989150 -32729 32409 4671.66 467166 -128 127 -0.34 -34 +442 2 10432 99343 1.32732 298.32732 149.82732 14982.73273 1.32732 298.32733 149.82732 14982.7329 1.32732 298.32732 149.82732 14982.73200 2020-01-01 2020-01-02 2020-01-01 00:07:22 2020-01-02 03:35:43 2020-01-01 00:07:22.000 2020-01-02 03:35:43.000 442 99343 49892.5 4989250 442 99343 49892.5 4989250 -32728 32410 4672.66 467266 -128 123 -1.9 -190 +443 2 10433 99344 1.33033 298.33033 149.83033 14983.03303 1.33033 298.33032 149.83033 14983.03319 1.33033 298.33033 149.83033 14983.03300 2020-01-01 2020-01-02 2020-01-01 00:07:23 2020-01-02 03:35:44 2020-01-01 00:07:23.000 2020-01-02 03:35:44.000 443 99344 49893.5 4989350 443 99344 49893.5 4989350 -32727 32411 4673.66 467366 -127 124 -0.9 -90 +444 2 10434 99345 1.33333 298.33333 149.83333 14983.33333 1.33333 298.33334 149.83333 14983.33331 1.33333 298.33333 149.83333000000002 14983.33300 2020-01-01 2020-01-02 2020-01-01 00:07:24 2020-01-02 03:35:45 2020-01-01 00:07:24.000 2020-01-02 03:35:45.000 444 99345 49894.5 4989450 444 99345 49894.5 4989450 -32726 32412 4674.66 467466 -126 125 0.1 10 +445 2 10435 99346 1.33633 298.33633 149.83633 14983.63363 1.33633 298.33633 149.83633 14983.63348 1.33633 298.33633 149.83633 14983.63300 2020-01-01 2020-01-02 2020-01-01 00:07:25 2020-01-02 03:35:46 2020-01-01 00:07:25.000 2020-01-02 03:35:46.000 445 99346 49895.5 4989550 445 99346 49895.5 4989550 -32725 32413 4675.66 467566 -125 126 1.1 110 +446 2 10436 99347 1.33933 298.33933 149.83933 14983.93393 1.33933 298.33932 149.83933 14983.93378 1.33933 298.33933 149.83933000000002 14983.93300 2020-01-01 2020-01-02 2020-01-01 00:07:26 2020-01-02 03:35:47 2020-01-01 00:07:26.000 2020-01-02 03:35:47.000 446 99347 49896.5 4989650 446 99347 49896.5 4989650 -32724 32414 4676.66 467666 -124 127 2.1 210 +447 2 10437 99348 1.34234 298.34234 149.84234 14984.23423 1.34234 298.34235 149.84234 14984.23437 1.34234 298.34234 149.84234 14984.23400 2020-01-01 2020-01-02 2020-01-01 00:07:27 2020-01-02 03:35:48 2020-01-01 00:07:27.000 2020-01-02 03:35:48.000 447 99348 49897.5 4989750 447 99348 49897.5 4989750 -32723 32415 4677.66 467766 -128 127 0.54 54 +448 2 10438 99349 1.34534 298.34534 149.84534 14984.53453 1.34534 298.34534 149.84534 14984.53466 1.34534 298.34534 149.84534 14984.53400 2020-01-01 2020-01-02 2020-01-01 00:07:28 2020-01-02 03:35:49 2020-01-01 00:07:28.000 2020-01-02 03:35:49.000 448 99349 49898.5 4989850 448 99349 49898.5 4989850 -32722 32416 4678.66 467866 -128 123 -1.02 -102 +449 2 10439 99350 1.34834 298.34834 149.84834 14984.83483 1.34834 298.34836 149.84834 14984.83478 1.34834 298.34834 149.84834 14984.83400 2020-01-01 2020-01-02 2020-01-01 00:07:29 2020-01-02 03:35:50 2020-01-01 00:07:29.000 2020-01-02 03:35:50.000 449 99350 49899.5 4989950 449 99350 49899.5 4989950 -32721 32417 4679.66 467966 -127 124 -0.02 -2 +45 2 10035 99945 0.13513 300.13513 150.13513 15163.64864 0.13513 300.13513 150.13513 15163.64839 0.13513 300.13513 150.13513 15163.64813 2020-01-01 2020-01-02 2020-01-01 00:00:45 2020-01-02 03:45:45 2020-01-01 00:00:45.000 2020-01-02 03:45:45.000 45 99945 49995 5049495 45 99945 49995 5049495 -32524 32411 4574.009900990099 461975 -126 126 -1.0396039603960396 -105 +450 2 10440 99351 1.35135 298.35135 149.85135 14985.13513 1.35135 298.35135 149.85134 14985.13495 1.35135 298.35135 149.85135 14985.13500 2020-01-01 2020-01-02 2020-01-01 00:07:30 2020-01-02 03:35:51 2020-01-01 00:07:30.000 2020-01-02 03:35:51.000 450 99351 49900.5 4990050 450 99351 49900.5 4990050 -32720 32418 4680.66 468066 -126 125 0.98 98 +451 2 10441 99352 1.35435 298.35435 149.85435 14985.43543 1.35435 298.35434 149.85435 14985.43525 1.35435 298.35435 149.85434999999998 14985.43500 2020-01-01 2020-01-02 2020-01-01 00:07:31 2020-01-02 03:35:52 2020-01-01 00:07:31.000 2020-01-02 03:35:52.000 451 99352 49901.5 4990150 451 99352 49901.5 4990150 -32719 32419 4681.66 468166 -125 126 1.98 198 +452 2 10442 99353 1.35735 298.35735 149.85735 14985.73573 1.35735 298.35736 149.85736 14985.736 1.35735 298.35735 149.85735 14985.73500 2020-01-01 2020-01-02 2020-01-01 00:07:32 2020-01-02 03:35:53 2020-01-01 00:07:32.000 2020-01-02 03:35:53.000 452 99353 49902.5 4990250 452 99353 49902.5 4990250 -32718 32420 4682.66 468266 -124 127 2.98 298 +453 2 10443 99354 1.36036 298.36036 149.86036 14986.03603 1.36036 298.36035 149.86036 14986.03614 1.36036 298.36036 149.86036000000001 14986.03600 2020-01-01 2020-01-02 2020-01-01 00:07:33 2020-01-02 03:35:54 2020-01-01 00:07:33.000 2020-01-02 03:35:54.000 453 99354 49903.5 4990350 453 99354 49903.5 4990350 -32717 32421 4683.66 468366 -128 127 1.42 142 +454 2 10444 99355 1.36336 298.36336 149.86336 14986.33633 1.36336 298.36337 149.86336 14986.33629 1.36336 298.36336 149.86336 14986.33600 2020-01-01 2020-01-02 2020-01-01 00:07:34 2020-01-02 03:35:55 2020-01-01 00:07:34.000 2020-01-02 03:35:55.000 454 99355 49904.5 4990450 454 99355 49904.5 4990450 -32716 32422 4684.66 468466 -128 127 -0.14 -14 +455 2 10445 99356 1.36636 298.36636 149.86636 14986.63663 1.36636 298.36636 149.86636 14986.63641 1.36636 298.36636 149.86636000000001 14986.63600 2020-01-01 2020-01-02 2020-01-01 00:07:35 2020-01-02 03:35:56 2020-01-01 00:07:35.000 2020-01-02 03:35:56.000 455 99356 49905.5 4990550 455 99356 49905.5 4990550 -32715 32423 4685.66 468566 -128 124 -1.7 -170 +456 2 10446 99357 1.36936 298.36936 149.86936 14986.93693 1.36936 298.36935 149.86936 14986.93672 1.36936 298.36936 149.86936 14986.93600 2020-01-01 2020-01-02 2020-01-01 00:07:36 2020-01-02 03:35:57 2020-01-01 00:07:36.000 2020-01-02 03:35:57.000 456 99357 49906.5 4990650 456 99357 49906.5 4990650 -32714 32424 4686.66 468666 -127 125 -0.7 -70 +457 2 10447 99358 1.37237 298.37237 149.87237 14987.23723 1.37237 298.37238 149.87237 14987.23747 1.37237 298.37237 149.87237 14987.23700 2020-01-01 2020-01-02 2020-01-01 00:07:37 2020-01-02 03:35:58 2020-01-01 00:07:37.000 2020-01-02 03:35:58.000 457 99358 49907.5 4990750 457 99358 49907.5 4990750 -32713 32425 4687.66 468766 -126 126 0.3 30 +458 2 10448 99359 1.37537 298.37537 149.87537 14987.53753 1.37537 298.37537 149.87537 14987.5376 1.37537 298.37537 149.87537 14987.53700 2020-01-01 2020-01-02 2020-01-01 00:07:38 2020-01-02 03:35:59 2020-01-01 00:07:38.000 2020-01-02 03:35:59.000 458 99359 49908.5 4990850 458 99359 49908.5 4990850 -32712 32426 4688.66 468866 -125 127 1.3 130 +459 2 10449 99360 1.37837 298.37837 149.87837 14987.83783 1.37837 298.3784 149.87837 14987.83775 1.37837 298.37837 149.87837 14987.83700 2020-01-01 2020-01-02 2020-01-01 00:07:39 2020-01-02 03:36:00 2020-01-01 00:07:39.000 2020-01-02 03:36:00.000 459 99360 49909.5 4990950 459 99360 49909.5 4990950 -32711 32427 4689.66 468966 -128 127 -0.26 -26 +46 2 10036 99946 0.13813 300.13813 150.13813 15163.95195 0.13813 300.13815 150.13814 15163.95214 0.13813 300.13813 150.13813 15163.95113 2020-01-01 2020-01-02 2020-01-01 00:00:46 2020-01-02 03:45:46 2020-01-01 00:00:46.000 2020-01-02 03:45:46.000 46 99946 49996 5049596 46 99946 49996 5049596 -32523 32412 4575.009900990099 462076 -125 127 -0.039603960396039604 -4 +460 2 10450 99361 1.38138 298.38138 149.88138 14988.13813 1.38138 298.38138 149.88137 14988.13789 1.38138 298.38138 149.88138 14988.13800 2020-01-01 2020-01-02 2020-01-01 00:07:40 2020-01-02 03:36:01 2020-01-01 00:07:40.000 2020-01-02 03:36:01.000 460 99361 49910.5 4991050 460 99361 49910.5 4991050 -32710 32428 4690.66 469066 -128 127 -1.82 -182 +461 2 10451 99362 1.38438 298.38438 149.88438 14988.43843 1.38438 298.3844 149.88438 14988.43864 1.38438 298.38438 149.88438 14988.43800 2020-01-01 2020-01-02 2020-01-01 00:07:41 2020-01-02 03:36:02 2020-01-01 00:07:41.000 2020-01-02 03:36:02.000 461 99362 49911.5 4991150 461 99362 49911.5 4991150 -32709 32429 4691.66 469166 -128 123 -3.38 -338 +462 2 10452 99363 1.38738 298.38738 149.88738 14988.73873 1.38738 298.3874 149.88738 14988.73894 1.38738 298.38738 149.88738 14988.73800 2020-01-01 2020-01-02 2020-01-01 00:07:42 2020-01-02 03:36:03 2020-01-01 00:07:42.000 2020-01-02 03:36:03.000 462 99363 49912.5 4991250 462 99363 49912.5 4991250 -32708 32430 4692.66 469266 -127 124 -2.38 -238 463 2 10453 99364 1.39039 298.39039 149.89039 14989.03903 1.39039 298.39038 149.89039 14989.03907 1.39039 298.39039 149.89039 14989.03900 2020-01-01 2020-01-02 2020-01-01 00:07:43 2020-01-02 03:36:04 2020-01-01 00:07:43.000 2020-01-02 03:36:04.000 463 99364 49913.5 4991350 463 99364 49913.5 4991350 -32707 32431 4693.66 469366 -126 125 -1.38 -138 -464 2 10454 99365 1.39339 298.39339 149.89339 14989.33933 1.39339 298.3934 149.89339 14989.33922 1.39339 298.39339 149.8933899999999 14989.33900 2020-01-01 2020-01-02 2020-01-01 00:07:44 2020-01-02 03:36:05 2020-01-01 00:07:44.000 2020-01-02 03:36:05.000 464 99365 49914.5 4991450 464 99365 49914.5 4991450 -32706 32432 4694.66 469466 -125 126 -0.38 -38 -465 2 10455 99366 1.39639 298.39639 149.89639 14989.63963 1.39639 298.3964 149.89639 14989.63936 1.39639 298.39639 149.8963899999999 14989.63900 2020-01-01 2020-01-02 2020-01-01 00:07:45 2020-01-02 03:36:06 2020-01-01 00:07:45.000 2020-01-02 03:36:06.000 465 99366 49915.5 4991550 465 99366 49915.5 4991550 -32705 32433 4695.66 469566 -124 127 0.62 62 +464 2 10454 99365 1.39339 298.39339 149.89339 14989.33933 1.39339 298.3934 149.89339 14989.33922 1.39339 298.39339 149.89339 14989.33900 2020-01-01 2020-01-02 2020-01-01 00:07:44 2020-01-02 03:36:05 2020-01-01 00:07:44.000 2020-01-02 03:36:05.000 464 99365 49914.5 4991450 464 99365 49914.5 4991450 -32706 32432 4694.66 469466 -125 126 -0.38 -38 +465 2 10455 99366 1.39639 298.39639 149.89639 14989.63963 1.39639 298.3964 149.89639 14989.63936 1.39639 298.39639 149.89639 14989.63900 2020-01-01 2020-01-02 2020-01-01 00:07:45 2020-01-02 03:36:06 2020-01-01 00:07:45.000 2020-01-02 03:36:06.000 465 99366 49915.5 4991550 465 99366 49915.5 4991550 -32705 32433 4695.66 469566 -124 127 0.62 62 466 2 10456 99367 1.39939 298.39939 149.89939 14989.93993 1.39939 298.3994 149.8994 14989.94011 1.39939 298.39939 149.89939 14989.93900 2020-01-01 2020-01-02 2020-01-01 00:07:46 2020-01-02 03:36:07 2020-01-01 00:07:46.000 2020-01-02 03:36:07.000 466 99367 49916.5 4991650 466 99367 49916.5 4991650 -32704 32434 4696.66 469666 -128 127 -0.94 -94 -467 2 10457 99368 1.4024 298.4024 149.9024 14990.24024 1.4024 298.4024 149.9024 14990.24041 1.40240 298.40240 149.90240000000017 14990.24000 2020-01-01 2020-01-02 2020-01-01 00:07:47 2020-01-02 03:36:08 2020-01-01 00:07:47.000 2020-01-02 03:36:08.000 467 99368 49917.5 4991750 467 99368 49917.5 4991750 -32703 32435 4697.66 469766 -128 123 -2.5 -250 -468 2 10458 99369 1.4054 298.4054 149.9054 14990.54054 1.4054 298.4054 149.9054 14990.54058 1.40540 298.40540 149.90539999999984 14990.54000 2020-01-01 2020-01-02 2020-01-01 00:07:48 2020-01-02 03:36:09 2020-01-01 00:07:48.000 2020-01-02 03:36:09.000 468 99369 49918.5 4991850 468 99369 49918.5 4991850 -32702 32436 4698.66 469866 -127 124 -1.5 -150 -469 2 10459 99370 1.4084 298.4084 149.9084 14990.84084 1.4084 298.40842 149.9084 14990.8407 1.40840 298.40840 149.90840000000009 14990.84000 2020-01-01 2020-01-02 2020-01-01 00:07:49 2020-01-02 03:36:10 2020-01-01 00:07:49.000 2020-01-02 03:36:10.000 469 99370 49919.5 4991950 469 99370 49919.5 4991950 -32701 32437 4699.66 469966 -126 125 -0.5 -50 -47 2 10037 99947 0.14114 300.14114 150.14114 15164.25525 0.14114 300.14114 150.14114 15164.25545 0.14114 300.14114 150.1411399999999 15164.25514 2020-01-01 2020-01-02 2020-01-01 00:00:47 2020-01-02 03:45:47 2020-01-01 00:00:47.000 2020-01-02 03:45:47.000 47 99947 49997 5049697 47 99947 49997 5049697 -32522 32413 4576.009900990099 462177 -128 127 -1.5742574257425743 -159 -470 2 10460 99371 1.41141 298.41141 149.91141 14991.14114 1.41141 298.4114 149.9114 14991.14099 1.41141 298.41141 149.91141000000007 14991.14100 2020-01-01 2020-01-02 2020-01-01 00:07:50 2020-01-02 03:36:11 2020-01-01 00:07:50.000 2020-01-02 03:36:11.000 470 99371 49920.5 4992050 470 99371 49920.5 4992050 -32700 32438 4700.66 470066 -125 126 0.5 50 -471 2 10461 99372 1.41441 298.41441 149.91441 14991.44144 1.41441 298.41443 149.91441 14991.44159 1.41441 298.41441 149.91440999999992 14991.44100 2020-01-01 2020-01-02 2020-01-01 00:07:51 2020-01-02 03:36:12 2020-01-01 00:07:51.000 2020-01-02 03:36:12.000 471 99372 49921.5 4992150 471 99372 49921.5 4992150 -32699 32439 4701.66 470166 -124 127 1.5 150 +467 2 10457 99368 1.4024 298.4024 149.9024 14990.24024 1.4024 298.4024 149.9024 14990.24041 1.40240 298.40240 149.9024 14990.24000 2020-01-01 2020-01-02 2020-01-01 00:07:47 2020-01-02 03:36:08 2020-01-01 00:07:47.000 2020-01-02 03:36:08.000 467 99368 49917.5 4991750 467 99368 49917.5 4991750 -32703 32435 4697.66 469766 -128 123 -2.5 -250 +468 2 10458 99369 1.4054 298.4054 149.9054 14990.54054 1.4054 298.4054 149.9054 14990.54058 1.40540 298.40540 149.90540000000001 14990.54000 2020-01-01 2020-01-02 2020-01-01 00:07:48 2020-01-02 03:36:09 2020-01-01 00:07:48.000 2020-01-02 03:36:09.000 468 99369 49918.5 4991850 468 99369 49918.5 4991850 -32702 32436 4698.66 469866 -127 124 -1.5 -150 +469 2 10459 99370 1.4084 298.4084 149.9084 14990.84084 1.4084 298.40842 149.9084 14990.8407 1.40840 298.40840 149.9084 14990.84000 2020-01-01 2020-01-02 2020-01-01 00:07:49 2020-01-02 03:36:10 2020-01-01 00:07:49.000 2020-01-02 03:36:10.000 469 99370 49919.5 4991950 469 99370 49919.5 4991950 -32701 32437 4699.66 469966 -126 125 -0.5 -50 +47 2 10037 99947 0.14114 300.14114 150.14114 15164.25525 0.14114 300.14114 150.14114 15164.25545 0.14114 300.14114 150.14114 15164.25514 2020-01-01 2020-01-02 2020-01-01 00:00:47 2020-01-02 03:45:47 2020-01-01 00:00:47.000 2020-01-02 03:45:47.000 47 99947 49997 5049697 47 99947 49997 5049697 -32522 32413 4576.009900990099 462177 -128 127 -1.5742574257425743 -159 +470 2 10460 99371 1.41141 298.41141 149.91141 14991.14114 1.41141 298.4114 149.9114 14991.14099 1.41141 298.41141 149.91141 14991.14100 2020-01-01 2020-01-02 2020-01-01 00:07:50 2020-01-02 03:36:11 2020-01-01 00:07:50.000 2020-01-02 03:36:11.000 470 99371 49920.5 4992050 470 99371 49920.5 4992050 -32700 32438 4700.66 470066 -125 126 0.5 50 +471 2 10461 99372 1.41441 298.41441 149.91441 14991.44144 1.41441 298.41443 149.91441 14991.44159 1.41441 298.41441 149.91441 14991.44100 2020-01-01 2020-01-02 2020-01-01 00:07:51 2020-01-02 03:36:12 2020-01-01 00:07:51.000 2020-01-02 03:36:12.000 471 99372 49921.5 4992150 471 99372 49921.5 4992150 -32699 32439 4701.66 470166 -124 127 1.5 150 472 2 10462 99373 1.41741 298.41741 149.91741 14991.74174 1.41741 298.41742 149.91741 14991.74188 1.41741 298.41741 149.91741 14991.74100 2020-01-01 2020-01-02 2020-01-01 00:07:52 2020-01-02 03:36:13 2020-01-01 00:07:52.000 2020-01-02 03:36:13.000 472 99373 49922.5 4992250 472 99373 49922.5 4992250 -32698 32440 4702.66 470266 -128 127 -0.06 -6 -473 2 10463 99374 1.42042 298.42042 149.92042 14992.04204 1.42042 298.4204 149.92042 14992.04204 1.42042 298.42042 149.92042000000018 14992.04200 2020-01-01 2020-01-02 2020-01-01 00:07:53 2020-01-02 03:36:14 2020-01-01 00:07:53.000 2020-01-02 03:36:14.000 473 99374 49923.5 4992350 473 99374 49923.5 4992350 -32697 32441 4703.66 470366 -128 123 -1.62 -162 -474 2 10464 99375 1.42342 298.42342 149.92342 14992.34234 1.42342 298.42343 149.92342 14992.34216 1.42342 298.42342 149.92341999999982 14992.34200 2020-01-01 2020-01-02 2020-01-01 00:07:54 2020-01-02 03:36:15 2020-01-01 00:07:54.000 2020-01-02 03:36:15.000 474 99375 49924.5 4992450 474 99375 49924.5 4992450 -32696 32442 4704.66 470466 -127 124 -0.62 -62 -475 2 10465 99376 1.42642 298.42642 149.92642 14992.64264 1.42642 298.42642 149.92642 14992.64246 1.42642 298.42642 149.92641999999987 14992.64200 2020-01-01 2020-01-02 2020-01-01 00:07:55 2020-01-02 03:36:16 2020-01-01 00:07:55.000 2020-01-02 03:36:16.000 475 99376 49925.5 4992550 475 99376 49925.5 4992550 -32695 32443 4705.66 470566 -126 125 0.38 38 -476 2 10466 99377 1.42942 298.42942 149.92942 14992.94294 1.42942 298.42944 149.92943 14992.94305 1.42942 298.42942 149.9294200000002 14992.94200 2020-01-01 2020-01-02 2020-01-01 00:07:56 2020-01-02 03:36:17 2020-01-01 00:07:56.000 2020-01-02 03:36:17.000 476 99377 49926.5 4992650 476 99377 49926.5 4992650 -32694 32444 4706.66 470666 -125 126 1.38 138 -477 2 10467 99378 1.43243 298.43243 149.93243 14993.24324 1.43243 298.43243 149.93243 14993.24338 1.43243 298.43243 149.9324300000001 14993.24300 2020-01-01 2020-01-02 2020-01-01 00:07:57 2020-01-02 03:36:18 2020-01-01 00:07:57.000 2020-01-02 03:36:18.000 477 99378 49927.5 4992750 477 99378 49927.5 4992750 -32693 32445 4707.66 470766 -124 127 2.38 238 -478 2 10468 99379 1.43543 298.43543 149.93543 14993.54354 1.43543 298.43542 149.93543 14993.54352 1.43543 298.43543 149.9354299999998 14993.54300 2020-01-01 2020-01-02 2020-01-01 00:07:58 2020-01-02 03:36:19 2020-01-01 00:07:58.000 2020-01-02 03:36:19.000 478 99379 49928.5 4992850 478 99379 49928.5 4992850 -32692 32446 4708.66 470866 -128 127 0.82 82 -479 2 10469 99380 1.43843 298.43843 149.93843 14993.84384 1.43843 298.43845 149.93844 14993.84427 1.43843 298.43843 149.9384300000001 14993.84300 2020-01-01 2020-01-02 2020-01-01 00:07:59 2020-01-02 03:36:20 2020-01-01 00:07:59.000 2020-01-02 03:36:20.000 479 99380 49929.5 4992950 479 99380 49929.5 4992950 -32691 32447 4709.66 470966 -128 127 -0.74 -74 -48 2 10038 99948 0.14414 300.14414 150.14414 15164.55855 0.14414 300.14413 150.14414 15164.55863 0.14414 300.14414 150.14414000000016 15164.55814 2020-01-01 2020-01-02 2020-01-01 00:00:48 2020-01-02 03:45:48 2020-01-01 00:00:48.000 2020-01-02 03:45:48.000 48 99948 49998 5049798 48 99948 49998 5049798 -32521 32414 4577.009900990099 462278 -128 127 -3.108910891089109 -314 +473 2 10463 99374 1.42042 298.42042 149.92042 14992.04204 1.42042 298.4204 149.92042 14992.04204 1.42042 298.42042 149.92042 14992.04200 2020-01-01 2020-01-02 2020-01-01 00:07:53 2020-01-02 03:36:14 2020-01-01 00:07:53.000 2020-01-02 03:36:14.000 473 99374 49923.5 4992350 473 99374 49923.5 4992350 -32697 32441 4703.66 470366 -128 123 -1.62 -162 +474 2 10464 99375 1.42342 298.42342 149.92342 14992.34234 1.42342 298.42343 149.92342 14992.34216 1.42342 298.42342 149.92342 14992.34200 2020-01-01 2020-01-02 2020-01-01 00:07:54 2020-01-02 03:36:15 2020-01-01 00:07:54.000 2020-01-02 03:36:15.000 474 99375 49924.5 4992450 474 99375 49924.5 4992450 -32696 32442 4704.66 470466 -127 124 -0.62 -62 +475 2 10465 99376 1.42642 298.42642 149.92642 14992.64264 1.42642 298.42642 149.92642 14992.64246 1.42642 298.42642 149.92642 14992.64200 2020-01-01 2020-01-02 2020-01-01 00:07:55 2020-01-02 03:36:16 2020-01-01 00:07:55.000 2020-01-02 03:36:16.000 475 99376 49925.5 4992550 475 99376 49925.5 4992550 -32695 32443 4705.66 470566 -126 125 0.38 38 +476 2 10466 99377 1.42942 298.42942 149.92942 14992.94294 1.42942 298.42944 149.92943 14992.94305 1.42942 298.42942 149.92942 14992.94200 2020-01-01 2020-01-02 2020-01-01 00:07:56 2020-01-02 03:36:17 2020-01-01 00:07:56.000 2020-01-02 03:36:17.000 476 99377 49926.5 4992650 476 99377 49926.5 4992650 -32694 32444 4706.66 470666 -125 126 1.38 138 +477 2 10467 99378 1.43243 298.43243 149.93243 14993.24324 1.43243 298.43243 149.93243 14993.24338 1.43243 298.43243 149.93243 14993.24300 2020-01-01 2020-01-02 2020-01-01 00:07:57 2020-01-02 03:36:18 2020-01-01 00:07:57.000 2020-01-02 03:36:18.000 477 99378 49927.5 4992750 477 99378 49927.5 4992750 -32693 32445 4707.66 470766 -124 127 2.38 238 +478 2 10468 99379 1.43543 298.43543 149.93543 14993.54354 1.43543 298.43542 149.93543 14993.54352 1.43543 298.43543 149.93543 14993.54300 2020-01-01 2020-01-02 2020-01-01 00:07:58 2020-01-02 03:36:19 2020-01-01 00:07:58.000 2020-01-02 03:36:19.000 478 99379 49928.5 4992850 478 99379 49928.5 4992850 -32692 32446 4708.66 470866 -128 127 0.82 82 +479 2 10469 99380 1.43843 298.43843 149.93843 14993.84384 1.43843 298.43845 149.93844 14993.84427 1.43843 298.43843 149.93843 14993.84300 2020-01-01 2020-01-02 2020-01-01 00:07:59 2020-01-02 03:36:20 2020-01-01 00:07:59.000 2020-01-02 03:36:20.000 479 99380 49929.5 4992950 479 99380 49929.5 4992950 -32691 32447 4709.66 470966 -128 127 -0.74 -74 +48 2 10038 99948 0.14414 300.14414 150.14414 15164.55855 0.14414 300.14413 150.14414 15164.55863 0.14414 300.14414 150.14414 15164.55814 2020-01-01 2020-01-02 2020-01-01 00:00:48 2020-01-02 03:45:48 2020-01-01 00:00:48.000 2020-01-02 03:45:48.000 48 99948 49998 5049798 48 99948 49998 5049798 -32521 32414 4577.009900990099 462278 -128 127 -3.108910891089109 -314 480 2 10470 99381 1.44144 298.44144 149.94144 14994.14414 1.44144 298.44144 149.94143 14994.14392 1.44144 298.44144 149.94144 14994.14400 2020-01-01 2020-01-02 2020-01-01 00:08:00 2020-01-02 03:36:21 2020-01-01 00:08:00.000 2020-01-02 03:36:21.000 480 99381 49930.5 4993050 480 99381 49930.5 4993050 -32690 32448 4710.66 471066 -128 124 -2.3 -230 -481 2 10471 99382 1.44444 298.44444 149.94444 14994.44444 1.44444 298.44446 149.94444 14994.44452 1.44444 298.44444 149.94443999999993 14994.44400 2020-01-01 2020-01-02 2020-01-01 00:08:01 2020-01-02 03:36:22 2020-01-01 00:08:01.000 2020-01-02 03:36:22.000 481 99382 49931.5 4993150 481 99382 49931.5 4993150 -32689 32449 4711.66 471166 -127 125 -1.3 -130 -482 2 10472 99383 1.44744 298.44744 149.94744 14994.74474 1.44744 298.44745 149.94744 14994.74485 1.44744 298.44744 149.94743999999994 14994.74400 2020-01-01 2020-01-02 2020-01-01 00:08:02 2020-01-02 03:36:23 2020-01-01 00:08:02.000 2020-01-02 03:36:23.000 482 99383 49932.5 4993250 482 99383 49932.5 4993250 -32688 32450 4712.66 471266 -126 126 -0.3 -30 -483 2 10473 99384 1.45045 298.45045 149.95045 14995.04504 1.45045 298.45044 149.95044 14995.04499 1.45045 298.45045 149.95045000000013 14995.04500 2020-01-01 2020-01-02 2020-01-01 00:08:03 2020-01-02 03:36:24 2020-01-01 00:08:03.000 2020-01-02 03:36:24.000 483 99384 49933.5 4993350 483 99384 49933.5 4993350 -32687 32451 4713.66 471366 -125 127 0.7 70 -484 2 10474 99385 1.45345 298.45345 149.95345 14995.34534 1.45345 298.45346 149.95345 14995.34574 1.45345 298.45345 149.9534500000002 14995.34500 2020-01-01 2020-01-02 2020-01-01 00:08:04 2020-01-02 03:36:25 2020-01-01 00:08:04.000 2020-01-02 03:36:25.000 484 99385 49934.5 4993450 484 99385 49934.5 4993450 -32686 32452 4714.66 471466 -128 127 -0.86 -86 -485 2 10475 99386 1.45645 298.45645 149.95645 14995.64564 1.45645 298.45645 149.95645 14995.6454 1.45645 298.45645 149.95644999999985 14995.64500 2020-01-01 2020-01-02 2020-01-01 00:08:05 2020-01-02 03:36:26 2020-01-01 00:08:05.000 2020-01-02 03:36:26.000 485 99386 49935.5 4993550 485 99386 49935.5 4993550 -32685 32453 4715.66 471566 -128 127 -2.42 -242 -486 2 10476 99387 1.45945 298.45945 149.95945 14995.94594 1.45945 298.45947 149.95946 14995.94602 1.45945 298.45945 149.95945000000012 14995.94500 2020-01-01 2020-01-02 2020-01-01 00:08:06 2020-01-02 03:36:27 2020-01-01 00:08:06.000 2020-01-02 03:36:27.000 486 99387 49936.5 4993650 486 99387 49936.5 4993650 -32684 32454 4716.66 471666 -128 123 -3.98 -398 -487 2 10477 99388 1.46246 298.46246 149.96246 14996.24624 1.46246 298.46246 149.96246 14996.24633 1.46246 298.46246 149.96246000000014 14996.24600 2020-01-01 2020-01-02 2020-01-01 00:08:07 2020-01-02 03:36:28 2020-01-01 00:08:07.000 2020-01-02 03:36:28.000 487 99388 49937.5 4993750 487 99388 49937.5 4993750 -32683 32455 4717.66 471766 -127 124 -2.98 -298 -488 2 10478 99389 1.46546 298.46546 149.96546 14996.54654 1.46546 298.46545 149.96546 14996.54645 1.46546 298.46546 149.96545999999998 14996.54600 2020-01-01 2020-01-02 2020-01-01 00:08:08 2020-01-02 03:36:29 2020-01-01 00:08:08.000 2020-01-02 03:36:29.000 488 99389 49938.5 4993850 488 99389 49938.5 4993850 -32682 32456 4718.66 471866 -126 125 -1.98 -198 -489 2 10479 99390 1.46846 298.46846 149.96846 14996.84684 1.46846 298.46848 149.96847 14996.84721 1.46846 298.46846 149.96846000000002 14996.84600 2020-01-01 2020-01-02 2020-01-01 00:08:09 2020-01-02 03:36:30 2020-01-01 00:08:09.000 2020-01-02 03:36:30.000 489 99390 49939.5 4993950 489 99390 49939.5 4993950 -32681 32457 4719.66 471966 -125 126 -0.98 -98 -49 2 10039 99949 0.14714 300.14714 150.14714 15164.86186 0.14714 300.14716 150.14714 15164.86173 0.14714 300.14714 150.14713999999984 15164.86114 2020-01-01 2020-01-02 2020-01-01 00:00:49 2020-01-02 03:45:49 2020-01-01 00:00:49.000 2020-01-02 03:45:49.000 49 99949 49999 5049899 49 99949 49999 5049899 -32520 32415 4578.009900990099 462379 -128 123 -4.643564356435643 -469 -490 2 10480 99391 1.47147 298.47147 149.97147 14997.14714 1.47147 298.47147 149.97146 14997.14687 1.47147 298.47147 149.9714700000002 14997.14700 2020-01-01 2020-01-02 2020-01-01 00:08:10 2020-01-02 03:36:31 2020-01-01 00:08:10.000 2020-01-02 03:36:31.000 490 99391 49940.5 4994050 490 99391 49940.5 4994050 -32680 32458 4720.66 472066 -124 127 0.02 2 -491 2 10481 99392 1.47447 298.47447 149.97447 14997.44744 1.47447 298.4745 149.97447 14997.44749 1.47447 298.47447 149.97446999999985 14997.44700 2020-01-01 2020-01-02 2020-01-01 00:08:11 2020-01-02 03:36:32 2020-01-01 00:08:11.000 2020-01-02 03:36:32.000 491 99392 49941.5 4994150 491 99392 49941.5 4994150 -32679 32459 4721.66 472166 -128 127 -1.54 -154 -492 2 10482 99393 1.47747 298.47747 149.97747 14997.74774 1.47747 298.47748 149.97747 14997.74779 1.47747 298.47747 149.97746999999987 14997.74700 2020-01-01 2020-01-02 2020-01-01 00:08:12 2020-01-02 03:36:33 2020-01-01 00:08:12.000 2020-01-02 03:36:33.000 492 99393 49942.5 4994250 492 99393 49942.5 4994250 -32678 32460 4722.66 472266 -128 123 -3.1 -310 -493 2 10483 99394 1.48048 298.48048 149.98048 14998.04804 1.48048 298.48047 149.98048 14998.04809 1.48048 298.48048 149.9804800000001 14998.04800 2020-01-01 2020-01-02 2020-01-01 00:08:13 2020-01-02 03:36:34 2020-01-01 00:08:13.000 2020-01-02 03:36:34.000 493 99394 49943.5 4994350 493 99394 49943.5 4994350 -32677 32461 4723.66 472366 -127 124 -2.1 -210 -494 2 10484 99395 1.48348 298.48348 149.98348 14998.34834 1.48348 298.4835 149.98348 14998.34868 1.48348 298.48348 149.9834800000001 14998.34800 2020-01-01 2020-01-02 2020-01-01 00:08:14 2020-01-02 03:36:35 2020-01-01 00:08:14.000 2020-01-02 03:36:35.000 494 99395 49944.5 4994450 494 99395 49944.5 4994450 -32676 32462 4724.66 472466 -126 125 -1.1 -110 -495 2 10485 99396 1.48648 298.48648 149.98648 14998.64864 1.48648 298.48648 149.98648 14998.64837 1.48648 298.48648 149.98647999999977 14998.64800 2020-01-01 2020-01-02 2020-01-01 00:08:15 2020-01-02 03:36:36 2020-01-01 00:08:15.000 2020-01-02 03:36:36.000 495 99396 49945.5 4994550 495 99396 49945.5 4994550 -32675 32463 4725.66 472566 -125 126 -0.1 -10 -496 2 10486 99397 1.48948 298.48948 149.98948 14998.94894 1.48948 298.4895 149.98948 14998.94896 1.48948 298.48948 149.98948000000013 14998.94800 2020-01-01 2020-01-02 2020-01-01 00:08:16 2020-01-02 03:36:37 2020-01-01 00:08:16.000 2020-01-02 03:36:37.000 496 99397 49946.5 4994650 496 99397 49946.5 4994650 -32674 32464 4726.66 472666 -124 127 0.9 90 -497 2 10487 99398 1.49249 298.49249 149.99249 14999.24924 1.49249 298.4925 149.99249 14999.24926 1.49249 298.49249 149.99249000000003 14999.24900 2020-01-01 2020-01-02 2020-01-01 00:08:17 2020-01-02 03:36:38 2020-01-01 00:08:17.000 2020-01-02 03:36:38.000 497 99398 49947.5 4994750 497 99398 49947.5 4994750 -32673 32465 4727.66 472766 -128 127 -0.66 -66 -498 2 10488 99399 1.49549 298.49549 149.99549 14999.54954 1.49549 298.49548 149.99549 14999.54956 1.49549 298.49549 149.9954899999999 14999.54900 2020-01-01 2020-01-02 2020-01-01 00:08:18 2020-01-02 03:36:39 2020-01-01 00:08:18.000 2020-01-02 03:36:39.000 498 99399 49948.5 4994850 498 99399 49948.5 4994850 -32672 32466 4728.66 472866 -128 123 -2.22 -222 -499 2 10489 99400 1.49849 298.49849 149.99849 14999.84984 1.49849 298.4985 149.9985 14999.85015 1.49849 298.49849 149.99848999999998 14999.84900 2020-01-01 2020-01-02 2020-01-01 00:08:19 2020-01-02 03:36:40 2020-01-01 00:08:19.000 2020-01-02 03:36:40.000 499 99400 49949.5 4994950 499 99400 49949.5 4994950 -32671 32467 4729.66 472966 -127 124 -1.22 -122 -5 2 1004 9995 0.01501 300.01501 150.01501 15151.51651 0.01501 300.015 150.01501 15151.51648 0.01501 300.01501 150.01500999999988 15151.51601 2020-01-01 2020-01-02 2020-01-01 00:00:05 2020-01-02 03:45:05 2020-01-01 00:00:05.000 2020-01-02 03:45:05.000 5 99905 49955 5045455 5 99905 49955 5045455 -32564 32371 4534.009900990099 457935 -128 123 -3.01980198019802 -305 +481 2 10471 99382 1.44444 298.44444 149.94444 14994.44444 1.44444 298.44446 149.94444 14994.44452 1.44444 298.44444 149.94444 14994.44400 2020-01-01 2020-01-02 2020-01-01 00:08:01 2020-01-02 03:36:22 2020-01-01 00:08:01.000 2020-01-02 03:36:22.000 481 99382 49931.5 4993150 481 99382 49931.5 4993150 -32689 32449 4711.66 471166 -127 125 -1.3 -130 +482 2 10472 99383 1.44744 298.44744 149.94744 14994.74474 1.44744 298.44745 149.94744 14994.74485 1.44744 298.44744 149.94744 14994.74400 2020-01-01 2020-01-02 2020-01-01 00:08:02 2020-01-02 03:36:23 2020-01-01 00:08:02.000 2020-01-02 03:36:23.000 482 99383 49932.5 4993250 482 99383 49932.5 4993250 -32688 32450 4712.66 471266 -126 126 -0.3 -30 +483 2 10473 99384 1.45045 298.45045 149.95045 14995.04504 1.45045 298.45044 149.95044 14995.04499 1.45045 298.45045 149.95045 14995.04500 2020-01-01 2020-01-02 2020-01-01 00:08:03 2020-01-02 03:36:24 2020-01-01 00:08:03.000 2020-01-02 03:36:24.000 483 99384 49933.5 4993350 483 99384 49933.5 4993350 -32687 32451 4713.66 471366 -125 127 0.7 70 +484 2 10474 99385 1.45345 298.45345 149.95345 14995.34534 1.45345 298.45346 149.95345 14995.34574 1.45345 298.45345 149.95345 14995.34500 2020-01-01 2020-01-02 2020-01-01 00:08:04 2020-01-02 03:36:25 2020-01-01 00:08:04.000 2020-01-02 03:36:25.000 484 99385 49934.5 4993450 484 99385 49934.5 4993450 -32686 32452 4714.66 471466 -128 127 -0.86 -86 +485 2 10475 99386 1.45645 298.45645 149.95645 14995.64564 1.45645 298.45645 149.95645 14995.6454 1.45645 298.45645 149.95645000000002 14995.64500 2020-01-01 2020-01-02 2020-01-01 00:08:05 2020-01-02 03:36:26 2020-01-01 00:08:05.000 2020-01-02 03:36:26.000 485 99386 49935.5 4993550 485 99386 49935.5 4993550 -32685 32453 4715.66 471566 -128 127 -2.42 -242 +486 2 10476 99387 1.45945 298.45945 149.95945 14995.94594 1.45945 298.45947 149.95946 14995.94602 1.45945 298.45945 149.95945 14995.94500 2020-01-01 2020-01-02 2020-01-01 00:08:06 2020-01-02 03:36:27 2020-01-01 00:08:06.000 2020-01-02 03:36:27.000 486 99387 49936.5 4993650 486 99387 49936.5 4993650 -32684 32454 4716.66 471666 -128 123 -3.98 -398 +487 2 10477 99388 1.46246 298.46246 149.96246 14996.24624 1.46246 298.46246 149.96246 14996.24633 1.46246 298.46246 149.96246 14996.24600 2020-01-01 2020-01-02 2020-01-01 00:08:07 2020-01-02 03:36:28 2020-01-01 00:08:07.000 2020-01-02 03:36:28.000 487 99388 49937.5 4993750 487 99388 49937.5 4993750 -32683 32455 4717.66 471766 -127 124 -2.98 -298 +488 2 10478 99389 1.46546 298.46546 149.96546 14996.54654 1.46546 298.46545 149.96546 14996.54645 1.46546 298.46546 149.96546 14996.54600 2020-01-01 2020-01-02 2020-01-01 00:08:08 2020-01-02 03:36:29 2020-01-01 00:08:08.000 2020-01-02 03:36:29.000 488 99389 49938.5 4993850 488 99389 49938.5 4993850 -32682 32456 4718.66 471866 -126 125 -1.98 -198 +489 2 10479 99390 1.46846 298.46846 149.96846 14996.84684 1.46846 298.46848 149.96847 14996.84721 1.46846 298.46846 149.96846 14996.84600 2020-01-01 2020-01-02 2020-01-01 00:08:09 2020-01-02 03:36:30 2020-01-01 00:08:09.000 2020-01-02 03:36:30.000 489 99390 49939.5 4993950 489 99390 49939.5 4993950 -32681 32457 4719.66 471966 -125 126 -0.98 -98 +49 2 10039 99949 0.14714 300.14714 150.14714 15164.86186 0.14714 300.14716 150.14714 15164.86173 0.14714 300.14714 150.14714 15164.86114 2020-01-01 2020-01-02 2020-01-01 00:00:49 2020-01-02 03:45:49 2020-01-01 00:00:49.000 2020-01-02 03:45:49.000 49 99949 49999 5049899 49 99949 49999 5049899 -32520 32415 4578.009900990099 462379 -128 123 -4.643564356435643 -469 +490 2 10480 99391 1.47147 298.47147 149.97147 14997.14714 1.47147 298.47147 149.97146 14997.14687 1.47147 298.47147 149.97147 14997.14700 2020-01-01 2020-01-02 2020-01-01 00:08:10 2020-01-02 03:36:31 2020-01-01 00:08:10.000 2020-01-02 03:36:31.000 490 99391 49940.5 4994050 490 99391 49940.5 4994050 -32680 32458 4720.66 472066 -124 127 0.02 2 +491 2 10481 99392 1.47447 298.47447 149.97447 14997.44744 1.47447 298.4745 149.97447 14997.44749 1.47447 298.47447 149.97447 14997.44700 2020-01-01 2020-01-02 2020-01-01 00:08:11 2020-01-02 03:36:32 2020-01-01 00:08:11.000 2020-01-02 03:36:32.000 491 99392 49941.5 4994150 491 99392 49941.5 4994150 -32679 32459 4721.66 472166 -128 127 -1.54 -154 +492 2 10482 99393 1.47747 298.47747 149.97747 14997.74774 1.47747 298.47748 149.97747 14997.74779 1.47747 298.47747 149.97746999999998 14997.74700 2020-01-01 2020-01-02 2020-01-01 00:08:12 2020-01-02 03:36:33 2020-01-01 00:08:12.000 2020-01-02 03:36:33.000 492 99393 49942.5 4994250 492 99393 49942.5 4994250 -32678 32460 4722.66 472266 -128 123 -3.1 -310 +493 2 10483 99394 1.48048 298.48048 149.98048 14998.04804 1.48048 298.48047 149.98048 14998.04809 1.48048 298.48048 149.98048 14998.04800 2020-01-01 2020-01-02 2020-01-01 00:08:13 2020-01-02 03:36:34 2020-01-01 00:08:13.000 2020-01-02 03:36:34.000 493 99394 49943.5 4994350 493 99394 49943.5 4994350 -32677 32461 4723.66 472366 -127 124 -2.1 -210 +494 2 10484 99395 1.48348 298.48348 149.98348 14998.34834 1.48348 298.4835 149.98348 14998.34868 1.48348 298.48348 149.98348 14998.34800 2020-01-01 2020-01-02 2020-01-01 00:08:14 2020-01-02 03:36:35 2020-01-01 00:08:14.000 2020-01-02 03:36:35.000 494 99395 49944.5 4994450 494 99395 49944.5 4994450 -32676 32462 4724.66 472466 -126 125 -1.1 -110 +495 2 10485 99396 1.48648 298.48648 149.98648 14998.64864 1.48648 298.48648 149.98648 14998.64837 1.48648 298.48648 149.98648 14998.64800 2020-01-01 2020-01-02 2020-01-01 00:08:15 2020-01-02 03:36:36 2020-01-01 00:08:15.000 2020-01-02 03:36:36.000 495 99396 49945.5 4994550 495 99396 49945.5 4994550 -32675 32463 4725.66 472566 -125 126 -0.1 -10 +496 2 10486 99397 1.48948 298.48948 149.98948 14998.94894 1.48948 298.4895 149.98948 14998.94896 1.48948 298.48948 149.98948000000001 14998.94800 2020-01-01 2020-01-02 2020-01-01 00:08:16 2020-01-02 03:36:37 2020-01-01 00:08:16.000 2020-01-02 03:36:37.000 496 99397 49946.5 4994650 496 99397 49946.5 4994650 -32674 32464 4726.66 472666 -124 127 0.9 90 +497 2 10487 99398 1.49249 298.49249 149.99249 14999.24924 1.49249 298.4925 149.99249 14999.24926 1.49249 298.49249 149.99249 14999.24900 2020-01-01 2020-01-02 2020-01-01 00:08:17 2020-01-02 03:36:38 2020-01-01 00:08:17.000 2020-01-02 03:36:38.000 497 99398 49947.5 4994750 497 99398 49947.5 4994750 -32673 32465 4727.66 472766 -128 127 -0.66 -66 +498 2 10488 99399 1.49549 298.49549 149.99549 14999.54954 1.49549 298.49548 149.99549 14999.54956 1.49549 298.49549 149.99549000000002 14999.54900 2020-01-01 2020-01-02 2020-01-01 00:08:18 2020-01-02 03:36:39 2020-01-01 00:08:18.000 2020-01-02 03:36:39.000 498 99399 49948.5 4994850 498 99399 49948.5 4994850 -32672 32466 4728.66 472866 -128 123 -2.22 -222 +499 2 10489 99400 1.49849 298.49849 149.99849 14999.84984 1.49849 298.4985 149.9985 14999.85015 1.49849 298.49849 149.99849 14999.84900 2020-01-01 2020-01-02 2020-01-01 00:08:19 2020-01-02 03:36:40 2020-01-01 00:08:19.000 2020-01-02 03:36:40.000 499 99400 49949.5 4994950 499 99400 49949.5 4994950 -32671 32467 4729.66 472966 -127 124 -1.22 -122 +5 2 1004 9995 0.01501 300.01501 150.01501 15151.51651 0.01501 300.015 150.01501 15151.51648 0.01501 300.01501 150.01501 15151.51601 2020-01-01 2020-01-02 2020-01-01 00:00:05 2020-01-02 03:45:05 2020-01-01 00:00:05.000 2020-01-02 03:45:05.000 5 99905 49955 5045455 5 99905 49955 5045455 -32564 32371 4534.009900990099 457935 -128 123 -3.01980198019802 -305 50 2 10040 99950 0.15015 300.15015 150.15015 15165.16516 0.15015 300.15015 150.15014 15165.16487 0.15015 300.15015 150.15015 15165.16515 2020-01-01 2020-01-02 2020-01-01 00:00:50 2020-01-02 03:45:50 2020-01-01 00:00:50.000 2020-01-02 03:45:50.000 50 99950 50000 5050000 50 99950 50000 5050000 -32519 32416 4579.009900990099 462480 -127 124 -3.6435643564356437 -368 -500 2 10490 99401 1.5015 298.5015 150.0015 15000.15015 1.5015 298.5015 150.00149 15000.14984 1.50150 298.50150 150.00150000000014 15000.15000 2020-01-01 2020-01-02 2020-01-01 00:08:20 2020-01-02 03:36:41 2020-01-01 00:08:20.000 2020-01-02 03:36:41.000 500 99401 49950.5 4995050 500 99401 49950.5 4995050 -32670 32468 4730.66 473066 -126 125 -0.22 -22 -501 2 10491 99402 1.5045 298.5045 150.0045 15000.45045 1.5045 298.50452 150.0045 15000.45043 1.50450 298.50450 150.0044999999998 15000.45000 2020-01-01 2020-01-02 2020-01-01 00:08:21 2020-01-02 03:36:42 2020-01-01 00:08:21.000 2020-01-02 03:36:42.000 501 99402 49951.5 4995150 501 99402 49951.5 4995150 -32669 32469 4731.66 473166 -125 126 0.78 78 -502 2 10492 99403 1.5075 298.5075 150.0075 15000.75075 1.5075 298.5075 150.0075 15000.75073 1.50750 298.50750 150.00749999999988 15000.75000 2020-01-01 2020-01-02 2020-01-01 00:08:22 2020-01-02 03:36:43 2020-01-01 00:08:22.000 2020-01-02 03:36:43.000 502 99403 49952.5 4995250 502 99403 49952.5 4995250 -32668 32470 4732.66 473266 -124 127 1.78 178 -503 2 10493 99404 1.51051 298.51051 150.01051 15001.05105 1.51051 298.5105 150.01051 15001.05103 1.51051 298.51051 150.01051000000004 15001.05100 2020-01-01 2020-01-02 2020-01-01 00:08:23 2020-01-02 03:36:44 2020-01-01 00:08:23.000 2020-01-02 03:36:44.000 503 99404 49953.5 4995350 503 99404 49953.5 4995350 -32667 32471 4733.66 473366 -128 127 0.22 22 -504 2 10494 99405 1.51351 298.51351 150.01351 15001.35135 1.51351 298.51352 150.01351 15001.35162 1.51351 298.51351 150.01351000000005 15001.35100 2020-01-01 2020-01-02 2020-01-01 00:08:24 2020-01-02 03:36:45 2020-01-01 00:08:24.000 2020-01-02 03:36:45.000 504 99405 49954.5 4995450 504 99405 49954.5 4995450 -32666 32472 4734.66 473466 -128 127 -1.34 -134 -505 2 10495 99406 1.51651 298.51651 150.01651 15001.65165 1.51651 298.5165 150.01651 15001.65131 1.51651 298.51651 150.01650999999998 15001.65100 2020-01-01 2020-01-02 2020-01-01 00:08:25 2020-01-02 03:36:46 2020-01-01 00:08:25.000 2020-01-02 03:36:46.000 505 99406 49955.5 4995550 505 99406 49955.5 4995550 -32665 32473 4735.66 473566 -128 124 -2.9 -290 -506 2 10496 99407 1.51951 298.51951 150.01951 15001.95195 1.51951 298.51953 150.01951 15001.9519 1.51951 298.51951 150.01951000000005 15001.95100 2020-01-01 2020-01-02 2020-01-01 00:08:26 2020-01-02 03:36:47 2020-01-01 00:08:26.000 2020-01-02 03:36:47.000 506 99407 49956.5 4995650 506 99407 49956.5 4995650 -32664 32474 4736.66 473666 -127 125 -1.9 -190 -507 2 10497 99408 1.52252 298.52252 150.02252 15002.25225 1.52252 298.52252 150.02252 15002.2522 1.52252 298.52252 150.0225200000002 15002.25200 2020-01-01 2020-01-02 2020-01-01 00:08:27 2020-01-02 03:36:48 2020-01-01 00:08:27.000 2020-01-02 03:36:48.000 507 99408 49957.5 4995750 507 99408 49957.5 4995750 -32663 32475 4737.66 473766 -126 126 -0.9 -90 -508 2 10498 99409 1.52552 298.52552 150.02552 15002.55255 1.52552 298.5255 150.02552 15002.5525 1.52552 298.52552 150.02551999999991 15002.55200 2020-01-01 2020-01-02 2020-01-01 00:08:28 2020-01-02 03:36:49 2020-01-01 00:08:28.000 2020-01-02 03:36:49.000 508 99409 49958.5 4995850 508 99409 49958.5 4995850 -32662 32476 4738.66 473866 -125 127 0.1 10 -509 2 10499 99410 1.52852 298.52852 150.02852 15002.85285 1.52852 298.52853 150.02853 15002.85312 1.52852 298.52852 150.0285199999999 15002.85200 2020-01-01 2020-01-02 2020-01-01 00:08:29 2020-01-02 03:36:50 2020-01-01 00:08:29.000 2020-01-02 03:36:50.000 509 99410 49959.5 4995950 509 99410 49959.5 4995950 -32661 32477 4739.66 473966 -128 127 -1.46 -146 -51 2 10041 99951 0.15315 300.15315 150.15315 15165.46846 0.15315 300.15317 150.15315 15165.46863 0.15315 300.15315 150.15315000000007 15165.46815 2020-01-01 2020-01-02 2020-01-01 00:00:51 2020-01-02 03:45:51 2020-01-01 00:00:51.000 2020-01-02 03:45:51.000 51 99951 50001 5050101 51 99951 50001 5050101 -32518 32417 4580.009900990099 462581 -126 125 -2.6435643564356437 -267 -510 2 10500 99411 1.53153 298.53153 150.03153 15003.15315 1.53153 298.53152 150.03152 15003.15278 1.53153 298.53153 150.03153000000012 15003.15300 2020-01-01 2020-01-02 2020-01-01 00:08:30 2020-01-02 03:36:51 2020-01-01 00:08:30.000 2020-01-02 03:36:51.000 510 99411 49960.5 4996050 510 99411 49960.5 4996050 -32660 32478 4740.66 474066 -128 127 -3.02 -302 -511 2 10501 99412 1.53453 298.53453 150.03453 15003.45345 1.53453 298.53455 150.03453 15003.45354 1.53453 298.53453 150.03453000000013 15003.45300 2020-01-01 2020-01-02 2020-01-01 00:08:31 2020-01-02 03:36:52 2020-01-01 00:08:31.000 2020-01-02 03:36:52.000 511 99412 49961.5 4996150 511 99412 49961.5 4996150 -32659 32479 4741.66 474166 -128 123 -4.58 -458 -512 2 10502 99413 1.53753 298.53753 150.03753 15003.75375 1.53753 298.53754 150.03753 15003.75366 1.53753 298.53753 150.0375299999998 15003.75300 2020-01-01 2020-01-02 2020-01-01 00:08:32 2020-01-02 03:36:53 2020-01-01 00:08:32.000 2020-01-02 03:36:53.000 512 99413 49962.5 4996250 512 99413 49962.5 4996250 -32658 32480 4742.66 474266 -127 124 -3.58 -358 +500 2 10490 99401 1.5015 298.5015 150.0015 15000.15015 1.5015 298.5015 150.00149 15000.14984 1.50150 298.50150 150.0015 15000.15000 2020-01-01 2020-01-02 2020-01-01 00:08:20 2020-01-02 03:36:41 2020-01-01 00:08:20.000 2020-01-02 03:36:41.000 500 99401 49950.5 4995050 500 99401 49950.5 4995050 -32670 32468 4730.66 473066 -126 125 -0.22 -22 +501 2 10491 99402 1.5045 298.5045 150.0045 15000.45045 1.5045 298.50452 150.0045 15000.45043 1.50450 298.50450 150.0045 15000.45000 2020-01-01 2020-01-02 2020-01-01 00:08:21 2020-01-02 03:36:42 2020-01-01 00:08:21.000 2020-01-02 03:36:42.000 501 99402 49951.5 4995150 501 99402 49951.5 4995150 -32669 32469 4731.66 473166 -125 126 0.78 78 +502 2 10492 99403 1.5075 298.5075 150.0075 15000.75075 1.5075 298.5075 150.0075 15000.75073 1.50750 298.50750 150.0075 15000.75000 2020-01-01 2020-01-02 2020-01-01 00:08:22 2020-01-02 03:36:43 2020-01-01 00:08:22.000 2020-01-02 03:36:43.000 502 99403 49952.5 4995250 502 99403 49952.5 4995250 -32668 32470 4732.66 473266 -124 127 1.78 178 +503 2 10493 99404 1.51051 298.51051 150.01051 15001.05105 1.51051 298.5105 150.01051 15001.05103 1.51051 298.51051 150.01050999999998 15001.05100 2020-01-01 2020-01-02 2020-01-01 00:08:23 2020-01-02 03:36:44 2020-01-01 00:08:23.000 2020-01-02 03:36:44.000 503 99404 49953.5 4995350 503 99404 49953.5 4995350 -32667 32471 4733.66 473366 -128 127 0.22 22 +504 2 10494 99405 1.51351 298.51351 150.01351 15001.35135 1.51351 298.51352 150.01351 15001.35162 1.51351 298.51351 150.01351 15001.35100 2020-01-01 2020-01-02 2020-01-01 00:08:24 2020-01-02 03:36:45 2020-01-01 00:08:24.000 2020-01-02 03:36:45.000 504 99405 49954.5 4995450 504 99405 49954.5 4995450 -32666 32472 4734.66 473466 -128 127 -1.34 -134 +505 2 10495 99406 1.51651 298.51651 150.01651 15001.65165 1.51651 298.5165 150.01651 15001.65131 1.51651 298.51651 150.01651 15001.65100 2020-01-01 2020-01-02 2020-01-01 00:08:25 2020-01-02 03:36:46 2020-01-01 00:08:25.000 2020-01-02 03:36:46.000 505 99406 49955.5 4995550 505 99406 49955.5 4995550 -32665 32473 4735.66 473566 -128 124 -2.9 -290 +506 2 10496 99407 1.51951 298.51951 150.01951 15001.95195 1.51951 298.51953 150.01951 15001.9519 1.51951 298.51951 150.01951 15001.95100 2020-01-01 2020-01-02 2020-01-01 00:08:26 2020-01-02 03:36:47 2020-01-01 00:08:26.000 2020-01-02 03:36:47.000 506 99407 49956.5 4995650 506 99407 49956.5 4995650 -32664 32474 4736.66 473666 -127 125 -1.9 -190 +507 2 10497 99408 1.52252 298.52252 150.02252 15002.25225 1.52252 298.52252 150.02252 15002.2522 1.52252 298.52252 150.02252000000001 15002.25200 2020-01-01 2020-01-02 2020-01-01 00:08:27 2020-01-02 03:36:48 2020-01-01 00:08:27.000 2020-01-02 03:36:48.000 507 99408 49957.5 4995750 507 99408 49957.5 4995750 -32663 32475 4737.66 473766 -126 126 -0.9 -90 +508 2 10498 99409 1.52552 298.52552 150.02552 15002.55255 1.52552 298.5255 150.02552 15002.5525 1.52552 298.52552 150.02552 15002.55200 2020-01-01 2020-01-02 2020-01-01 00:08:28 2020-01-02 03:36:49 2020-01-01 00:08:28.000 2020-01-02 03:36:49.000 508 99409 49958.5 4995850 508 99409 49958.5 4995850 -32662 32476 4738.66 473866 -125 127 0.1 10 +509 2 10499 99410 1.52852 298.52852 150.02852 15002.85285 1.52852 298.52853 150.02853 15002.85312 1.52852 298.52852 150.02852000000001 15002.85200 2020-01-01 2020-01-02 2020-01-01 00:08:29 2020-01-02 03:36:50 2020-01-01 00:08:29.000 2020-01-02 03:36:50.000 509 99410 49959.5 4995950 509 99410 49959.5 4995950 -32661 32477 4739.66 473966 -128 127 -1.46 -146 +51 2 10041 99951 0.15315 300.15315 150.15315 15165.46846 0.15315 300.15317 150.15315 15165.46863 0.15315 300.15315 150.15315 15165.46815 2020-01-01 2020-01-02 2020-01-01 00:00:51 2020-01-02 03:45:51 2020-01-01 00:00:51.000 2020-01-02 03:45:51.000 51 99951 50001 5050101 51 99951 50001 5050101 -32518 32417 4580.009900990099 462581 -126 125 -2.6435643564356437 -267 +510 2 10500 99411 1.53153 298.53153 150.03153 15003.15315 1.53153 298.53152 150.03152 15003.15278 1.53153 298.53153 150.03153 15003.15300 2020-01-01 2020-01-02 2020-01-01 00:08:30 2020-01-02 03:36:51 2020-01-01 00:08:30.000 2020-01-02 03:36:51.000 510 99411 49960.5 4996050 510 99411 49960.5 4996050 -32660 32478 4740.66 474066 -128 127 -3.02 -302 +511 2 10501 99412 1.53453 298.53453 150.03453 15003.45345 1.53453 298.53455 150.03453 15003.45354 1.53453 298.53453 150.03453 15003.45300 2020-01-01 2020-01-02 2020-01-01 00:08:31 2020-01-02 03:36:52 2020-01-01 00:08:31.000 2020-01-02 03:36:52.000 511 99412 49961.5 4996150 511 99412 49961.5 4996150 -32659 32479 4741.66 474166 -128 123 -4.58 -458 +512 2 10502 99413 1.53753 298.53753 150.03753 15003.75375 1.53753 298.53754 150.03753 15003.75366 1.53753 298.53753 150.03753 15003.75300 2020-01-01 2020-01-02 2020-01-01 00:08:32 2020-01-02 03:36:53 2020-01-01 00:08:32.000 2020-01-02 03:36:53.000 512 99413 49962.5 4996250 512 99413 49962.5 4996250 -32658 32480 4742.66 474266 -127 124 -3.58 -358 513 2 10503 99414 1.54054 298.54054 150.04054 15004.05405 1.54054 298.54053 150.04053 15004.05397 1.54054 298.54054 150.04054 15004.05400 2020-01-01 2020-01-02 2020-01-01 00:08:33 2020-01-02 03:36:54 2020-01-01 00:08:33.000 2020-01-02 03:36:54.000 513 99414 49963.5 4996350 513 99414 49963.5 4996350 -32657 32481 4743.66 474366 -126 125 -2.58 -258 -514 2 10504 99415 1.54354 298.54354 150.04354 15004.35435 1.54354 298.54355 150.04354 15004.35459 1.54354 298.54354 150.04354000000004 15004.35400 2020-01-01 2020-01-02 2020-01-01 00:08:34 2020-01-02 03:36:55 2020-01-01 00:08:34.000 2020-01-02 03:36:55.000 514 99415 49964.5 4996450 514 99415 49964.5 4996450 -32656 32482 4744.66 474466 -125 126 -1.58 -158 -515 2 10505 99416 1.54654 298.54654 150.04654 15004.65465 1.54654 298.54654 150.04654 15004.65425 1.54654 298.54654 150.0465399999999 15004.65400 2020-01-01 2020-01-02 2020-01-01 00:08:35 2020-01-02 03:36:56 2020-01-01 00:08:35.000 2020-01-02 03:36:56.000 515 99416 49965.5 4996550 515 99416 49965.5 4996550 -32655 32483 4745.66 474566 -124 127 -0.58 -58 +514 2 10504 99415 1.54354 298.54354 150.04354 15004.35435 1.54354 298.54355 150.04354 15004.35459 1.54354 298.54354 150.04354 15004.35400 2020-01-01 2020-01-02 2020-01-01 00:08:34 2020-01-02 03:36:55 2020-01-01 00:08:34.000 2020-01-02 03:36:55.000 514 99415 49964.5 4996450 514 99415 49964.5 4996450 -32656 32482 4744.66 474466 -125 126 -1.58 -158 +515 2 10505 99416 1.54654 298.54654 150.04654 15004.65465 1.54654 298.54654 150.04654 15004.65425 1.54654 298.54654 150.04654 15004.65400 2020-01-01 2020-01-02 2020-01-01 00:08:35 2020-01-02 03:36:56 2020-01-01 00:08:35.000 2020-01-02 03:36:56.000 515 99416 49965.5 4996550 515 99416 49965.5 4996550 -32655 32483 4745.66 474566 -124 127 -0.58 -58 516 2 10506 99417 1.54954 298.54954 150.04954 15004.95495 1.54954 298.54956 150.04955 15004.955 1.54954 298.54954 150.04954 15004.95400 2020-01-01 2020-01-02 2020-01-01 00:08:36 2020-01-02 03:36:57 2020-01-01 00:08:36.000 2020-01-02 03:36:57.000 516 99417 49966.5 4996650 516 99417 49966.5 4996650 -32654 32484 4746.66 474666 -128 127 -2.14 -214 -517 2 10507 99418 1.55255 298.55255 150.05255 15005.25525 1.55255 298.55255 150.05255 15005.25514 1.55255 298.55255 150.05255000000017 15005.25500 2020-01-01 2020-01-02 2020-01-01 00:08:37 2020-01-02 03:36:58 2020-01-01 00:08:37.000 2020-01-02 03:36:58.000 517 99418 49967.5 4996750 517 99418 49967.5 4996750 -32653 32485 4747.66 474766 -128 123 -3.7 -370 -518 2 10508 99419 1.55555 298.55555 150.05555 15005.55555 1.55555 298.55554 150.05555 15005.55547 1.55555 298.55555 150.0555499999998 15005.55500 2020-01-01 2020-01-02 2020-01-01 00:08:38 2020-01-02 03:36:59 2020-01-01 00:08:38.000 2020-01-02 03:36:59.000 518 99419 49968.5 4996850 518 99419 49968.5 4996850 -32652 32486 4748.66 474866 -127 124 -2.7 -270 -519 2 10509 99420 1.55855 298.55855 150.05855 15005.85585 1.55855 298.55856 150.05856 15005.85607 1.55855 298.55855 150.0585499999999 15005.85500 2020-01-01 2020-01-02 2020-01-01 00:08:39 2020-01-02 03:37:00 2020-01-01 00:08:39.000 2020-01-02 03:37:00.000 519 99420 49969.5 4996950 519 99420 49969.5 4996950 -32651 32487 4749.66 474966 -126 125 -1.7 -170 -52 2 10042 99952 0.15615 300.15615 150.15615 15165.77177 0.15615 300.15616 150.15615 15165.77193 0.15615 300.15615 150.15615000000003 15165.77115 2020-01-01 2020-01-02 2020-01-01 00:00:52 2020-01-02 03:45:52 2020-01-01 00:00:52.000 2020-01-02 03:45:52.000 52 99952 50002 5050202 52 99952 50002 5050202 -32517 32418 4581.009900990099 462682 -125 126 -1.6435643564356435 -166 -520 2 10510 99421 1.56156 298.56156 150.06156 15006.15615 1.56156 298.56155 150.06155 15006.15572 1.56156 298.56156 150.06156000000007 15006.15600 2020-01-01 2020-01-02 2020-01-01 00:08:40 2020-01-02 03:37:01 2020-01-01 00:08:40.000 2020-01-02 03:37:01.000 520 99421 49970.5 4997050 520 99421 49970.5 4997050 -32650 32488 4750.66 475066 -125 126 -0.7 -70 -521 2 10511 99422 1.56456 298.56456 150.06456 15006.45645 1.56456 298.56458 150.06456 15006.45647 1.56456 298.56456 150.06456000000009 15006.45600 2020-01-01 2020-01-02 2020-01-01 00:08:41 2020-01-02 03:37:02 2020-01-01 00:08:41.000 2020-01-02 03:37:02.000 521 99422 49971.5 4997150 521 99422 49971.5 4997150 -32649 32489 4751.66 475166 -124 127 0.3 30 -522 2 10512 99423 1.56756 298.56756 150.06756 15006.75675 1.56756 298.56757 150.06756 15006.75661 1.56756 298.56756 150.06756000000004 15006.75600 2020-01-01 2020-01-02 2020-01-01 00:08:42 2020-01-02 03:37:03 2020-01-01 00:08:42.000 2020-01-02 03:37:03.000 522 99423 49972.5 4997250 522 99423 49972.5 4997250 -32648 32490 4752.66 475266 -128 127 -1.26 -126 -523 2 10513 99424 1.57057 298.57057 150.07057 15007.05705 1.57057 298.57056 150.07056 15007.05694 1.57057 298.57057 150.0705699999999 15007.05700 2020-01-01 2020-01-02 2020-01-01 00:08:43 2020-01-02 03:37:04 2020-01-01 00:08:43.000 2020-01-02 03:37:04.000 523 99424 49973.5 4997350 523 99424 49973.5 4997350 -32647 32491 4753.66 475366 -128 123 -2.82 -282 -524 2 10514 99425 1.57357 298.57357 150.07357 15007.35735 1.57357 298.57358 150.07357 15007.35753 1.57357 298.57357 150.07356999999996 15007.35700 2020-01-01 2020-01-02 2020-01-01 00:08:44 2020-01-02 03:37:05 2020-01-01 00:08:44.000 2020-01-02 03:37:05.000 524 99425 49974.5 4997450 524 99425 49974.5 4997450 -32646 32492 4754.66 475466 -127 124 -1.82 -182 -525 2 10515 99426 1.57657 298.57657 150.07657 15007.65765 1.57657 298.57657 150.07657 15007.65783 1.57657 298.57657 150.07656999999992 15007.65700 2020-01-01 2020-01-02 2020-01-01 00:08:45 2020-01-02 03:37:06 2020-01-01 00:08:45.000 2020-01-02 03:37:06.000 525 99426 49975.5 4997550 525 99426 49975.5 4997550 -32645 32493 4755.66 475566 -126 125 -0.82 -82 -526 2 10516 99427 1.57957 298.57957 150.07957 15007.95795 1.57957 298.5796 150.07957 15007.95795 1.57957 298.57957 150.0795699999999 15007.95700 2020-01-01 2020-01-02 2020-01-01 00:08:46 2020-01-02 03:37:07 2020-01-01 00:08:46.000 2020-01-02 03:37:07.000 526 99427 49976.5 4997650 526 99427 49976.5 4997650 -32644 32494 4756.66 475666 -125 126 0.18 18 -527 2 10517 99428 1.58258 298.58258 150.08258 15008.25825 1.58258 298.58258 150.08258 15008.25811 1.58258 298.58258 150.08258000000012 15008.25800 2020-01-01 2020-01-02 2020-01-01 00:08:47 2020-01-02 03:37:08 2020-01-01 00:08:47.000 2020-01-02 03:37:08.000 527 99428 49977.5 4997750 527 99428 49977.5 4997750 -32643 32495 4757.66 475766 -124 127 1.18 118 -528 2 10518 99429 1.58558 298.58558 150.08558 15008.55855 1.58558 298.58557 150.08558 15008.5584 1.58558 298.58558 150.08558000000014 15008.55800 2020-01-01 2020-01-02 2020-01-01 00:08:48 2020-01-02 03:37:09 2020-01-01 00:08:48.000 2020-01-02 03:37:09.000 528 99429 49978.5 4997850 528 99429 49978.5 4997850 -32642 32496 4758.66 475866 -128 127 -0.38 -38 -529 2 10519 99430 1.58858 298.58858 150.08858 15008.85885 1.58858 298.5886 150.08859 15008.859 1.58858 298.58858 150.08857999999984 15008.85800 2020-01-01 2020-01-02 2020-01-01 00:08:49 2020-01-02 03:37:10 2020-01-01 00:08:49.000 2020-01-02 03:37:10.000 529 99430 49979.5 4997950 529 99430 49979.5 4997950 -32641 32497 4759.66 475966 -128 127 -1.94 -194 -53 2 10043 99953 0.15915 300.15915 150.15915 15166.07507 0.15915 300.15915 150.15915 15166.07511 0.15915 300.15915 150.15914999999998 15166.07415 2020-01-01 2020-01-02 2020-01-01 00:00:53 2020-01-02 03:45:53 2020-01-01 00:00:53.000 2020-01-02 03:45:53.000 53 99953 50003 5050303 53 99953 50003 5050303 -32516 32419 4582.009900990099 462783 -124 127 -0.6435643564356436 -65 +517 2 10507 99418 1.55255 298.55255 150.05255 15005.25525 1.55255 298.55255 150.05255 15005.25514 1.55255 298.55255 150.05255 15005.25500 2020-01-01 2020-01-02 2020-01-01 00:08:37 2020-01-02 03:36:58 2020-01-01 00:08:37.000 2020-01-02 03:36:58.000 517 99418 49967.5 4996750 517 99418 49967.5 4996750 -32653 32485 4747.66 474766 -128 123 -3.7 -370 +518 2 10508 99419 1.55555 298.55555 150.05555 15005.55555 1.55555 298.55554 150.05555 15005.55547 1.55555 298.55555 150.05555 15005.55500 2020-01-01 2020-01-02 2020-01-01 00:08:38 2020-01-02 03:36:59 2020-01-01 00:08:38.000 2020-01-02 03:36:59.000 518 99419 49968.5 4996850 518 99419 49968.5 4996850 -32652 32486 4748.66 474866 -127 124 -2.7 -270 +519 2 10509 99420 1.55855 298.55855 150.05855 15005.85585 1.55855 298.55856 150.05856 15005.85607 1.55855 298.55855 150.05855 15005.85500 2020-01-01 2020-01-02 2020-01-01 00:08:39 2020-01-02 03:37:00 2020-01-01 00:08:39.000 2020-01-02 03:37:00.000 519 99420 49969.5 4996950 519 99420 49969.5 4996950 -32651 32487 4749.66 474966 -126 125 -1.7 -170 +52 2 10042 99952 0.15615 300.15615 150.15615 15165.77177 0.15615 300.15616 150.15615 15165.77193 0.15615 300.15615 150.15615 15165.77115 2020-01-01 2020-01-02 2020-01-01 00:00:52 2020-01-02 03:45:52 2020-01-01 00:00:52.000 2020-01-02 03:45:52.000 52 99952 50002 5050202 52 99952 50002 5050202 -32517 32418 4581.009900990099 462682 -125 126 -1.6435643564356435 -166 +520 2 10510 99421 1.56156 298.56156 150.06156 15006.15615 1.56156 298.56155 150.06155 15006.15572 1.56156 298.56156 150.06156000000001 15006.15600 2020-01-01 2020-01-02 2020-01-01 00:08:40 2020-01-02 03:37:01 2020-01-01 00:08:40.000 2020-01-02 03:37:01.000 520 99421 49970.5 4997050 520 99421 49970.5 4997050 -32650 32488 4750.66 475066 -125 126 -0.7 -70 +521 2 10511 99422 1.56456 298.56456 150.06456 15006.45645 1.56456 298.56458 150.06456 15006.45647 1.56456 298.56456 150.06456 15006.45600 2020-01-01 2020-01-02 2020-01-01 00:08:41 2020-01-02 03:37:02 2020-01-01 00:08:41.000 2020-01-02 03:37:02.000 521 99422 49971.5 4997150 521 99422 49971.5 4997150 -32649 32489 4751.66 475166 -124 127 0.3 30 +522 2 10512 99423 1.56756 298.56756 150.06756 15006.75675 1.56756 298.56757 150.06756 15006.75661 1.56756 298.56756 150.06756 15006.75600 2020-01-01 2020-01-02 2020-01-01 00:08:42 2020-01-02 03:37:03 2020-01-01 00:08:42.000 2020-01-02 03:37:03.000 522 99423 49972.5 4997250 522 99423 49972.5 4997250 -32648 32490 4752.66 475266 -128 127 -1.26 -126 +523 2 10513 99424 1.57057 298.57057 150.07057 15007.05705 1.57057 298.57056 150.07056 15007.05694 1.57057 298.57057 150.07057 15007.05700 2020-01-01 2020-01-02 2020-01-01 00:08:43 2020-01-02 03:37:04 2020-01-01 00:08:43.000 2020-01-02 03:37:04.000 523 99424 49973.5 4997350 523 99424 49973.5 4997350 -32647 32491 4753.66 475366 -128 123 -2.82 -282 +524 2 10514 99425 1.57357 298.57357 150.07357 15007.35735 1.57357 298.57358 150.07357 15007.35753 1.57357 298.57357 150.07357 15007.35700 2020-01-01 2020-01-02 2020-01-01 00:08:44 2020-01-02 03:37:05 2020-01-01 00:08:44.000 2020-01-02 03:37:05.000 524 99425 49974.5 4997450 524 99425 49974.5 4997450 -32646 32492 4754.66 475466 -127 124 -1.82 -182 +525 2 10515 99426 1.57657 298.57657 150.07657 15007.65765 1.57657 298.57657 150.07657 15007.65783 1.57657 298.57657 150.07657 15007.65700 2020-01-01 2020-01-02 2020-01-01 00:08:45 2020-01-02 03:37:06 2020-01-01 00:08:45.000 2020-01-02 03:37:06.000 525 99426 49975.5 4997550 525 99426 49975.5 4997550 -32645 32493 4755.66 475566 -126 125 -0.82 -82 +526 2 10516 99427 1.57957 298.57957 150.07957 15007.95795 1.57957 298.5796 150.07957 15007.95795 1.57957 298.57957 150.07957 15007.95700 2020-01-01 2020-01-02 2020-01-01 00:08:46 2020-01-02 03:37:07 2020-01-01 00:08:46.000 2020-01-02 03:37:07.000 526 99427 49976.5 4997650 526 99427 49976.5 4997650 -32644 32494 4756.66 475666 -125 126 0.18 18 +527 2 10517 99428 1.58258 298.58258 150.08258 15008.25825 1.58258 298.58258 150.08258 15008.25811 1.58258 298.58258 150.08258 15008.25800 2020-01-01 2020-01-02 2020-01-01 00:08:47 2020-01-02 03:37:08 2020-01-01 00:08:47.000 2020-01-02 03:37:08.000 527 99428 49977.5 4997750 527 99428 49977.5 4997750 -32643 32495 4757.66 475766 -124 127 1.18 118 +528 2 10518 99429 1.58558 298.58558 150.08558 15008.55855 1.58558 298.58557 150.08558 15008.5584 1.58558 298.58558 150.08558000000002 15008.55800 2020-01-01 2020-01-02 2020-01-01 00:08:48 2020-01-02 03:37:09 2020-01-01 00:08:48.000 2020-01-02 03:37:09.000 528 99429 49978.5 4997850 528 99429 49978.5 4997850 -32642 32496 4758.66 475866 -128 127 -0.38 -38 +529 2 10519 99430 1.58858 298.58858 150.08858 15008.85885 1.58858 298.5886 150.08859 15008.859 1.58858 298.58858 150.08858 15008.85800 2020-01-01 2020-01-02 2020-01-01 00:08:49 2020-01-02 03:37:10 2020-01-01 00:08:49.000 2020-01-02 03:37:10.000 529 99430 49979.5 4997950 529 99430 49979.5 4997950 -32641 32497 4759.66 475966 -128 127 -1.94 -194 +53 2 10043 99953 0.15915 300.15915 150.15915 15166.07507 0.15915 300.15915 150.15915 15166.07511 0.15915 300.15915 150.15915 15166.07415 2020-01-01 2020-01-02 2020-01-01 00:00:53 2020-01-02 03:45:53 2020-01-01 00:00:53.000 2020-01-02 03:45:53.000 53 99953 50003 5050303 53 99953 50003 5050303 -32516 32419 4582.009900990099 462783 -124 127 -0.6435643564356436 -65 530 2 10520 99431 1.59159 298.59159 150.09159 15009.15915 1.59159 298.59158 150.09159 15009.15929 1.59159 298.59159 150.09159 15009.15900 2020-01-01 2020-01-02 2020-01-01 00:08:50 2020-01-02 03:37:11 2020-01-01 00:08:50.000 2020-01-02 03:37:11.000 530 99431 49980.5 4998050 530 99431 49980.5 4998050 -32640 32498 4760.66 476066 -128 124 -3.5 -350 -531 2 10521 99432 1.59459 298.59459 150.09459 15009.45945 1.59459 298.5946 150.09459 15009.45941 1.59459 298.59459 150.09459000000007 15009.45900 2020-01-01 2020-01-02 2020-01-01 00:08:51 2020-01-02 03:37:12 2020-01-01 00:08:51.000 2020-01-02 03:37:12.000 531 99432 49981.5 4998150 531 99432 49981.5 4998150 -32639 32499 4761.66 476166 -127 125 -2.5 -250 -532 2 10522 99433 1.59759 298.59759 150.09759 15009.75975 1.59759 298.5976 150.09759 15009.75958 1.59759 298.59759 150.09758999999994 15009.75900 2020-01-01 2020-01-02 2020-01-01 00:08:52 2020-01-02 03:37:13 2020-01-01 00:08:52.000 2020-01-02 03:37:13.000 532 99433 49982.5 4998250 532 99433 49982.5 4998250 -32638 32500 4762.66 476266 -126 126 -1.5 -150 -533 2 10523 99434 1.6006 298.6006 150.1006 15010.06006 1.6006 298.6006 150.10059 15010.05988 1.60060 298.60060 150.10059999999984 15010.06000 2020-01-01 2020-01-02 2020-01-01 00:08:53 2020-01-02 03:37:14 2020-01-01 00:08:53.000 2020-01-02 03:37:14.000 533 99434 49983.5 4998350 533 99434 49983.5 4998350 -32637 32501 4763.66 476366 -125 127 -0.5 -50 -534 2 10524 99435 1.6036 298.6036 150.1036 15010.36036 1.6036 298.6036 150.1036 15010.36063 1.60360 298.60360 150.10360000000017 15010.36000 2020-01-01 2020-01-02 2020-01-01 00:08:54 2020-01-02 03:37:15 2020-01-01 00:08:54.000 2020-01-02 03:37:15.000 534 99435 49984.5 4998450 534 99435 49984.5 4998450 -32636 32502 4764.66 476466 -128 127 -2.06 -206 -535 2 10525 99436 1.6066 298.6066 150.1066 15010.66066 1.6066 298.6066 150.1066 15010.66077 1.60660 298.60660 150.10659999999987 15010.66000 2020-01-01 2020-01-02 2020-01-01 00:08:55 2020-01-02 03:37:16 2020-01-01 00:08:55.000 2020-01-02 03:37:16.000 535 99436 49985.5 4998550 535 99436 49985.5 4998550 -32635 32503 4765.66 476566 -128 127 -3.62 -362 -536 2 10526 99437 1.6096 298.6096 150.1096 15010.96096 1.6096 298.60962 150.1096 15010.96092 1.60960 298.60960 150.10959999999986 15010.96000 2020-01-01 2020-01-02 2020-01-01 00:08:56 2020-01-02 03:37:17 2020-01-01 00:08:56.000 2020-01-02 03:37:17.000 536 99437 49986.5 4998650 536 99437 49986.5 4998650 -32634 32504 4766.66 476666 -128 123 -5.18 -518 -537 2 10527 99438 1.61261 298.61261 150.11261 15011.26126 1.61261 298.6126 150.11261 15011.26105 1.61261 298.61261 150.1126100000001 15011.26100 2020-01-01 2020-01-02 2020-01-01 00:08:57 2020-01-02 03:37:18 2020-01-01 00:08:57.000 2020-01-02 03:37:18.000 537 99438 49987.5 4998750 537 99438 49987.5 4998750 -32633 32505 4767.66 476766 -127 124 -4.18 -418 -538 2 10528 99439 1.61561 298.61561 150.11561 15011.56156 1.61561 298.6156 150.11561 15011.56135 1.61561 298.61561 150.11561000000012 15011.56100 2020-01-01 2020-01-02 2020-01-01 00:08:58 2020-01-02 03:37:19 2020-01-01 00:08:58.000 2020-01-02 03:37:19.000 538 99439 49988.5 4998850 538 99439 49988.5 4998850 -32632 32506 4768.66 476866 -126 125 -3.18 -318 -539 2 10529 99440 1.61861 298.61861 150.11861 15011.86186 1.61861 298.61862 150.11862 15011.8621 1.61861 298.61861 150.1186099999999 15011.86100 2020-01-01 2020-01-02 2020-01-01 00:08:59 2020-01-02 03:37:20 2020-01-01 00:08:59.000 2020-01-02 03:37:20.000 539 99440 49989.5 4998950 539 99440 49989.5 4998950 -32631 32507 4769.66 476966 -125 126 -2.18 -218 -54 2 10044 99954 0.16216 300.16216 150.16216 15166.37837 0.16216 300.16217 150.16216 15166.37822 0.16216 300.16216 150.1621599999999 15166.37816 2020-01-01 2020-01-02 2020-01-01 00:00:54 2020-01-02 03:45:54 2020-01-01 00:00:54.000 2020-01-02 03:45:54.000 54 99954 50004 5050404 54 99954 50004 5050404 -32515 32420 4583.009900990099 462884 -128 127 -2.1782178217821784 -220 -540 2 10530 99441 1.62162 298.62162 150.12162 15012.16216 1.62162 298.6216 150.12162 15012.16224 1.62162 298.62162 150.12161999999992 15012.16200 2020-01-01 2020-01-02 2020-01-01 00:09:00 2020-01-02 03:37:21 2020-01-01 00:09:00.000 2020-01-02 03:37:21.000 540 99441 49990.5 4999050 540 99441 49990.5 4999050 -32630 32508 4770.66 477066 -124 127 -1.18 -118 +531 2 10521 99432 1.59459 298.59459 150.09459 15009.45945 1.59459 298.5946 150.09459 15009.45941 1.59459 298.59459 150.09459 15009.45900 2020-01-01 2020-01-02 2020-01-01 00:08:51 2020-01-02 03:37:12 2020-01-01 00:08:51.000 2020-01-02 03:37:12.000 531 99432 49981.5 4998150 531 99432 49981.5 4998150 -32639 32499 4761.66 476166 -127 125 -2.5 -250 +532 2 10522 99433 1.59759 298.59759 150.09759 15009.75975 1.59759 298.5976 150.09759 15009.75958 1.59759 298.59759 150.09759 15009.75900 2020-01-01 2020-01-02 2020-01-01 00:08:52 2020-01-02 03:37:13 2020-01-01 00:08:52.000 2020-01-02 03:37:13.000 532 99433 49982.5 4998250 532 99433 49982.5 4998250 -32638 32500 4762.66 476266 -126 126 -1.5 -150 +533 2 10523 99434 1.6006 298.6006 150.1006 15010.06006 1.6006 298.6006 150.10059 15010.05988 1.60060 298.60060 150.1006 15010.06000 2020-01-01 2020-01-02 2020-01-01 00:08:53 2020-01-02 03:37:14 2020-01-01 00:08:53.000 2020-01-02 03:37:14.000 533 99434 49983.5 4998350 533 99434 49983.5 4998350 -32637 32501 4763.66 476366 -125 127 -0.5 -50 +534 2 10524 99435 1.6036 298.6036 150.1036 15010.36036 1.6036 298.6036 150.1036 15010.36063 1.60360 298.60360 150.1036 15010.36000 2020-01-01 2020-01-02 2020-01-01 00:08:54 2020-01-02 03:37:15 2020-01-01 00:08:54.000 2020-01-02 03:37:15.000 534 99435 49984.5 4998450 534 99435 49984.5 4998450 -32636 32502 4764.66 476466 -128 127 -2.06 -206 +535 2 10525 99436 1.6066 298.6066 150.1066 15010.66066 1.6066 298.6066 150.1066 15010.66077 1.60660 298.60660 150.1066 15010.66000 2020-01-01 2020-01-02 2020-01-01 00:08:55 2020-01-02 03:37:16 2020-01-01 00:08:55.000 2020-01-02 03:37:16.000 535 99436 49985.5 4998550 535 99436 49985.5 4998550 -32635 32503 4765.66 476566 -128 127 -3.62 -362 +536 2 10526 99437 1.6096 298.6096 150.1096 15010.96096 1.6096 298.60962 150.1096 15010.96092 1.60960 298.60960 150.1096 15010.96000 2020-01-01 2020-01-02 2020-01-01 00:08:56 2020-01-02 03:37:17 2020-01-01 00:08:56.000 2020-01-02 03:37:17.000 536 99437 49986.5 4998650 536 99437 49986.5 4998650 -32634 32504 4766.66 476666 -128 123 -5.18 -518 +537 2 10527 99438 1.61261 298.61261 150.11261 15011.26126 1.61261 298.6126 150.11261 15011.26105 1.61261 298.61261 150.11261000000002 15011.26100 2020-01-01 2020-01-02 2020-01-01 00:08:57 2020-01-02 03:37:18 2020-01-01 00:08:57.000 2020-01-02 03:37:18.000 537 99438 49987.5 4998750 537 99438 49987.5 4998750 -32633 32505 4767.66 476766 -127 124 -4.18 -418 +538 2 10528 99439 1.61561 298.61561 150.11561 15011.56156 1.61561 298.6156 150.11561 15011.56135 1.61561 298.61561 150.11561 15011.56100 2020-01-01 2020-01-02 2020-01-01 00:08:58 2020-01-02 03:37:19 2020-01-01 00:08:58.000 2020-01-02 03:37:19.000 538 99439 49988.5 4998850 538 99439 49988.5 4998850 -32632 32506 4768.66 476866 -126 125 -3.18 -318 +539 2 10529 99440 1.61861 298.61861 150.11861 15011.86186 1.61861 298.61862 150.11862 15011.8621 1.61861 298.61861 150.11861000000002 15011.86100 2020-01-01 2020-01-02 2020-01-01 00:08:59 2020-01-02 03:37:20 2020-01-01 00:08:59.000 2020-01-02 03:37:20.000 539 99440 49989.5 4998950 539 99440 49989.5 4998950 -32631 32507 4769.66 476966 -125 126 -2.18 -218 +54 2 10044 99954 0.16216 300.16216 150.16216 15166.37837 0.16216 300.16217 150.16216 15166.37822 0.16216 300.16216 150.16216 15166.37816 2020-01-01 2020-01-02 2020-01-01 00:00:54 2020-01-02 03:45:54 2020-01-01 00:00:54.000 2020-01-02 03:45:54.000 54 99954 50004 5050404 54 99954 50004 5050404 -32515 32420 4583.009900990099 462884 -128 127 -2.1782178217821784 -220 +540 2 10530 99441 1.62162 298.62162 150.12162 15012.16216 1.62162 298.6216 150.12162 15012.16224 1.62162 298.62162 150.12162 15012.16200 2020-01-01 2020-01-02 2020-01-01 00:09:00 2020-01-02 03:37:21 2020-01-01 00:09:00.000 2020-01-02 03:37:21.000 540 99441 49990.5 4999050 540 99441 49990.5 4999050 -32630 32508 4770.66 477066 -124 127 -1.18 -118 541 2 10531 99442 1.62462 298.62462 150.12462 15012.46246 1.62462 298.62463 150.12462 15012.46239 1.62462 298.62462 150.12462 15012.46200 2020-01-01 2020-01-02 2020-01-01 00:09:01 2020-01-02 03:37:22 2020-01-01 00:09:01.000 2020-01-02 03:37:22.000 541 99442 49991.5 4999150 541 99442 49991.5 4999150 -32629 32509 4771.66 477166 -128 127 -2.74 -274 -542 2 10532 99443 1.62762 298.62762 150.12762 15012.76276 1.62762 298.62762 150.12762 15012.76252 1.62762 298.62762 150.12761999999998 15012.76200 2020-01-01 2020-01-02 2020-01-01 00:09:02 2020-01-02 03:37:23 2020-01-01 00:09:02.000 2020-01-02 03:37:23.000 542 99443 49992.5 4999250 542 99443 49992.5 4999250 -32628 32510 4772.66 477266 -128 123 -4.3 -430 -543 2 10533 99444 1.63063 298.63063 150.13063 15013.06306 1.63063 298.63065 150.13063 15013.06327 1.63063 298.63063 150.13062999999985 15013.06300 2020-01-01 2020-01-02 2020-01-01 00:09:03 2020-01-02 03:37:24 2020-01-01 00:09:03.000 2020-01-02 03:37:24.000 543 99444 49993.5 4999350 543 99444 49993.5 4999350 -32627 32511 4773.66 477366 -127 124 -3.3 -330 -544 2 10534 99445 1.63363 298.63363 150.13363 15013.36336 1.63363 298.63364 150.13363 15013.36358 1.63363 298.63363 150.13363000000012 15013.36300 2020-01-01 2020-01-02 2020-01-01 00:09:04 2020-01-02 03:37:25 2020-01-01 00:09:04.000 2020-01-02 03:37:25.000 544 99445 49994.5 4999450 544 99445 49994.5 4999450 -32626 32512 4774.66 477466 -126 125 -2.3 -230 -545 2 10535 99446 1.63663 298.63663 150.13663 15013.66366 1.63663 298.63663 150.13663 15013.6637 1.63663 298.63663 150.1366300000002 15013.66300 2020-01-01 2020-01-02 2020-01-01 00:09:05 2020-01-02 03:37:26 2020-01-01 00:09:05.000 2020-01-02 03:37:26.000 545 99446 49995.5 4999550 545 99446 49995.5 4999550 -32625 32513 4775.66 477566 -125 126 -1.3 -130 -546 2 10536 99447 1.63963 298.63963 150.13963 15013.96396 1.63963 298.63965 150.13963 15013.96385 1.63963 298.63963 150.13962999999984 15013.96300 2020-01-01 2020-01-02 2020-01-01 00:09:06 2020-01-02 03:37:27 2020-01-01 00:09:06.000 2020-01-02 03:37:27.000 546 99447 49996.5 4999650 546 99447 49996.5 4999650 -32624 32514 4776.66 477666 -124 127 -0.3 -30 +542 2 10532 99443 1.62762 298.62762 150.12762 15012.76276 1.62762 298.62762 150.12762 15012.76252 1.62762 298.62762 150.12762 15012.76200 2020-01-01 2020-01-02 2020-01-01 00:09:02 2020-01-02 03:37:23 2020-01-01 00:09:02.000 2020-01-02 03:37:23.000 542 99443 49992.5 4999250 542 99443 49992.5 4999250 -32628 32510 4772.66 477266 -128 123 -4.3 -430 +543 2 10533 99444 1.63063 298.63063 150.13063 15013.06306 1.63063 298.63065 150.13063 15013.06327 1.63063 298.63063 150.13063 15013.06300 2020-01-01 2020-01-02 2020-01-01 00:09:03 2020-01-02 03:37:24 2020-01-01 00:09:03.000 2020-01-02 03:37:24.000 543 99444 49993.5 4999350 543 99444 49993.5 4999350 -32627 32511 4773.66 477366 -127 124 -3.3 -330 +544 2 10534 99445 1.63363 298.63363 150.13363 15013.36336 1.63363 298.63364 150.13363 15013.36358 1.63363 298.63363 150.13362999999998 15013.36300 2020-01-01 2020-01-02 2020-01-01 00:09:04 2020-01-02 03:37:25 2020-01-01 00:09:04.000 2020-01-02 03:37:25.000 544 99445 49994.5 4999450 544 99445 49994.5 4999450 -32626 32512 4774.66 477466 -126 125 -2.3 -230 +545 2 10535 99446 1.63663 298.63663 150.13663 15013.66366 1.63663 298.63663 150.13663 15013.6637 1.63663 298.63663 150.13663 15013.66300 2020-01-01 2020-01-02 2020-01-01 00:09:05 2020-01-02 03:37:26 2020-01-01 00:09:05.000 2020-01-02 03:37:26.000 545 99446 49995.5 4999550 545 99446 49995.5 4999550 -32625 32513 4775.66 477566 -125 126 -1.3 -130 +546 2 10536 99447 1.63963 298.63963 150.13963 15013.96396 1.63963 298.63965 150.13963 15013.96385 1.63963 298.63963 150.13963 15013.96300 2020-01-01 2020-01-02 2020-01-01 00:09:06 2020-01-02 03:37:27 2020-01-01 00:09:06.000 2020-01-02 03:37:27.000 546 99447 49996.5 4999650 546 99447 49996.5 4999650 -32624 32514 4776.66 477666 -124 127 -0.3 -30 547 2 10537 99448 1.64264 298.64264 150.14264 15014.26426 1.64264 298.64264 150.14263 15014.26399 1.64264 298.64264 150.14264 15014.26400 2020-01-01 2020-01-02 2020-01-01 00:09:07 2020-01-02 03:37:28 2020-01-01 00:09:07.000 2020-01-02 03:37:28.000 547 99448 49997.5 4999750 547 99448 49997.5 4999750 -32623 32515 4777.66 477766 -128 127 -1.86 -186 -548 2 10538 99449 1.64564 298.64564 150.14564 15014.56456 1.64564 298.64566 150.14564 15014.56474 1.64564 298.64564 150.1456400000001 15014.56400 2020-01-01 2020-01-02 2020-01-01 00:09:08 2020-01-02 03:37:29 2020-01-01 00:09:08.000 2020-01-02 03:37:29.000 548 99449 49998.5 4999850 548 99449 49998.5 4999850 -32622 32516 4778.66 477866 -128 123 -3.42 -342 -549 2 10539 99450 1.64864 298.64864 150.14864 15014.86486 1.64864 298.64865 150.14865 15014.86504 1.64864 298.64864 150.14863999999997 15014.86400 2020-01-01 2020-01-02 2020-01-01 00:09:09 2020-01-02 03:37:30 2020-01-01 00:09:09.000 2020-01-02 03:37:30.000 549 99450 49999.5 4999950 549 99450 49999.5 4999950 -32621 32517 4779.66 477966 -127 124 -2.42 -242 -55 2 10045 99955 0.16516 300.16516 150.16516 15166.68168 0.16516 300.16516 150.16516 15166.68151 0.16516 300.16516 150.16515999999996 15166.68116 2020-01-01 2020-01-02 2020-01-01 00:00:55 2020-01-02 03:45:55 2020-01-01 00:00:55.000 2020-01-02 03:45:55.000 55 99955 50005 5050505 55 99955 50005 5050505 -32514 32421 4584.009900990099 462985 -128 123 -3.712871287128713 -375 -550 2 10540 99451 1.65165 298.65165 150.15165 15015.16516 1.65165 298.65164 150.15165 15015.16521 1.65165 298.65165 150.15164999999985 15015.16500 2020-01-01 2020-01-02 2020-01-01 00:09:10 2020-01-02 03:37:31 2020-01-01 00:09:10.000 2020-01-02 03:37:31.000 550 99451 50000.5 5000050 550 99451 50000.5 5000050 -32620 32518 4780.66 478066 -126 125 -1.42 -142 -551 2 10541 99452 1.65465 298.65465 150.15465 15015.46546 1.65465 298.65466 150.15465 15015.46533 1.65465 298.65465 150.1546500000002 15015.46500 2020-01-01 2020-01-02 2020-01-01 00:09:11 2020-01-02 03:37:32 2020-01-01 00:09:11.000 2020-01-02 03:37:32.000 551 99452 50001.5 5000150 551 99452 50001.5 5000150 -32619 32519 4781.66 478166 -125 126 -0.42 -42 -552 2 10542 99453 1.65765 298.65765 150.15765 15015.76576 1.65765 298.65765 150.15765 15015.76562 1.65765 298.65765 150.15764999999985 15015.76500 2020-01-01 2020-01-02 2020-01-01 00:09:12 2020-01-02 03:37:33 2020-01-01 00:09:12.000 2020-01-02 03:37:33.000 552 99453 50002.5 5000250 552 99453 50002.5 5000250 -32618 32520 4782.66 478266 -124 127 0.58 58 -553 2 10543 99454 1.66066 298.66066 150.16066 15016.06606 1.66066 298.66068 150.16066 15016.06621 1.66066 298.66066 150.16065999999978 15016.06600 2020-01-01 2020-01-02 2020-01-01 00:09:13 2020-01-02 03:37:34 2020-01-01 00:09:13.000 2020-01-02 03:37:34.000 553 99454 50003.5 5000350 553 99454 50003.5 5000350 -32617 32521 4783.66 478366 -128 127 -0.98 -98 -554 2 10544 99455 1.66366 298.66366 150.16366 15016.36636 1.66366 298.66367 150.16366 15016.36651 1.66366 298.66366 150.1636600000001 15016.36600 2020-01-01 2020-01-02 2020-01-01 00:09:14 2020-01-02 03:37:35 2020-01-01 00:09:14.000 2020-01-02 03:37:35.000 554 99455 50004.5 5000450 554 99455 50004.5 5000450 -32616 32522 4784.66 478466 -128 127 -2.54 -254 -555 2 10545 99456 1.66666 298.66666 150.16666 15016.66666 1.66666 298.66666 150.16666 15016.66668 1.66666 298.66666 150.16666000000012 15016.66600 2020-01-01 2020-01-02 2020-01-01 00:09:15 2020-01-02 03:37:36 2020-01-01 00:09:15.000 2020-01-02 03:37:36.000 555 99456 50005.5 5000550 555 99456 50005.5 5000550 -32615 32523 4785.66 478566 -128 124 -4.1 -410 -556 2 10546 99457 1.66966 298.66966 150.16966 15016.96696 1.66966 298.66968 150.16966 15016.9668 1.66966 298.66966 150.1696599999998 15016.96600 2020-01-01 2020-01-02 2020-01-01 00:09:16 2020-01-02 03:37:37 2020-01-01 00:09:16.000 2020-01-02 03:37:37.000 556 99457 50006.5 5000650 556 99457 50006.5 5000650 -32614 32524 4786.66 478666 -127 125 -3.1 -310 -557 2 10547 99458 1.67267 298.67267 150.17267 15017.26726 1.67267 298.67267 150.17267 15017.26709 1.67267 298.67267 150.17266999999995 15017.26700 2020-01-01 2020-01-02 2020-01-01 00:09:17 2020-01-02 03:37:38 2020-01-01 00:09:17.000 2020-01-02 03:37:38.000 557 99458 50007.5 5000750 557 99458 50007.5 5000750 -32613 32525 4787.66 478766 -126 126 -2.1 -210 +548 2 10538 99449 1.64564 298.64564 150.14564 15014.56456 1.64564 298.64566 150.14564 15014.56474 1.64564 298.64564 150.14564000000001 15014.56400 2020-01-01 2020-01-02 2020-01-01 00:09:08 2020-01-02 03:37:29 2020-01-01 00:09:08.000 2020-01-02 03:37:29.000 548 99449 49998.5 4999850 548 99449 49998.5 4999850 -32622 32516 4778.66 477866 -128 123 -3.42 -342 +549 2 10539 99450 1.64864 298.64864 150.14864 15014.86486 1.64864 298.64865 150.14865 15014.86504 1.64864 298.64864 150.14864 15014.86400 2020-01-01 2020-01-02 2020-01-01 00:09:09 2020-01-02 03:37:30 2020-01-01 00:09:09.000 2020-01-02 03:37:30.000 549 99450 49999.5 4999950 549 99450 49999.5 4999950 -32621 32517 4779.66 477966 -127 124 -2.42 -242 +55 2 10045 99955 0.16516 300.16516 150.16516 15166.68168 0.16516 300.16516 150.16516 15166.68151 0.16516 300.16516 150.16516000000001 15166.68116 2020-01-01 2020-01-02 2020-01-01 00:00:55 2020-01-02 03:45:55 2020-01-01 00:00:55.000 2020-01-02 03:45:55.000 55 99955 50005 5050505 55 99955 50005 5050505 -32514 32421 4584.009900990099 462985 -128 123 -3.712871287128713 -375 +550 2 10540 99451 1.65165 298.65165 150.15165 15015.16516 1.65165 298.65164 150.15165 15015.16521 1.65165 298.65165 150.15165000000002 15015.16500 2020-01-01 2020-01-02 2020-01-01 00:09:10 2020-01-02 03:37:31 2020-01-01 00:09:10.000 2020-01-02 03:37:31.000 550 99451 50000.5 5000050 550 99451 50000.5 5000050 -32620 32518 4780.66 478066 -126 125 -1.42 -142 +551 2 10541 99452 1.65465 298.65465 150.15465 15015.46546 1.65465 298.65466 150.15465 15015.46533 1.65465 298.65465 150.15465 15015.46500 2020-01-01 2020-01-02 2020-01-01 00:09:11 2020-01-02 03:37:32 2020-01-01 00:09:11.000 2020-01-02 03:37:32.000 551 99452 50001.5 5000150 551 99452 50001.5 5000150 -32619 32519 4781.66 478166 -125 126 -0.42 -42 +552 2 10542 99453 1.65765 298.65765 150.15765 15015.76576 1.65765 298.65765 150.15765 15015.76562 1.65765 298.65765 150.15765 15015.76500 2020-01-01 2020-01-02 2020-01-01 00:09:12 2020-01-02 03:37:33 2020-01-01 00:09:12.000 2020-01-02 03:37:33.000 552 99453 50002.5 5000250 552 99453 50002.5 5000250 -32618 32520 4782.66 478266 -124 127 0.58 58 +553 2 10543 99454 1.66066 298.66066 150.16066 15016.06606 1.66066 298.66068 150.16066 15016.06621 1.66066 298.66066 150.16066 15016.06600 2020-01-01 2020-01-02 2020-01-01 00:09:13 2020-01-02 03:37:34 2020-01-01 00:09:13.000 2020-01-02 03:37:34.000 553 99454 50003.5 5000350 553 99454 50003.5 5000350 -32617 32521 4783.66 478366 -128 127 -0.98 -98 +554 2 10544 99455 1.66366 298.66366 150.16366 15016.36636 1.66366 298.66367 150.16366 15016.36651 1.66366 298.66366 150.16366 15016.36600 2020-01-01 2020-01-02 2020-01-01 00:09:14 2020-01-02 03:37:35 2020-01-01 00:09:14.000 2020-01-02 03:37:35.000 554 99455 50004.5 5000450 554 99455 50004.5 5000450 -32616 32522 4784.66 478466 -128 127 -2.54 -254 +555 2 10545 99456 1.66666 298.66666 150.16666 15016.66666 1.66666 298.66666 150.16666 15016.66668 1.66666 298.66666 150.16665999999998 15016.66600 2020-01-01 2020-01-02 2020-01-01 00:09:15 2020-01-02 03:37:36 2020-01-01 00:09:15.000 2020-01-02 03:37:36.000 555 99456 50005.5 5000550 555 99456 50005.5 5000550 -32615 32523 4785.66 478566 -128 124 -4.1 -410 +556 2 10546 99457 1.66966 298.66966 150.16966 15016.96696 1.66966 298.66968 150.16966 15016.9668 1.66966 298.66966 150.16966 15016.96600 2020-01-01 2020-01-02 2020-01-01 00:09:16 2020-01-02 03:37:37 2020-01-01 00:09:16.000 2020-01-02 03:37:37.000 556 99457 50006.5 5000650 556 99457 50006.5 5000650 -32614 32524 4786.66 478666 -127 125 -3.1 -310 +557 2 10547 99458 1.67267 298.67267 150.17267 15017.26726 1.67267 298.67267 150.17267 15017.26709 1.67267 298.67267 150.17267 15017.26700 2020-01-01 2020-01-02 2020-01-01 00:09:17 2020-01-02 03:37:38 2020-01-01 00:09:17.000 2020-01-02 03:37:38.000 557 99458 50007.5 5000750 557 99458 50007.5 5000750 -32613 32525 4787.66 478766 -126 126 -2.1 -210 558 2 10548 99459 1.67567 298.67567 150.17567 15017.56756 1.67567 298.6757 150.17567 15017.56769 1.67567 298.67567 150.17567 15017.56700 2020-01-01 2020-01-02 2020-01-01 00:09:18 2020-01-02 03:37:39 2020-01-01 00:09:18.000 2020-01-02 03:37:39.000 558 99459 50008.5 5000850 558 99459 50008.5 5000850 -32612 32526 4788.66 478866 -125 127 -1.1 -110 -559 2 10549 99460 1.67867 298.67867 150.17867 15017.86786 1.67867 298.67868 150.17868 15017.86802 1.67867 298.67867 150.17866999999995 15017.86700 2020-01-01 2020-01-02 2020-01-01 00:09:19 2020-01-02 03:37:40 2020-01-01 00:09:19.000 2020-01-02 03:37:40.000 559 99460 50009.5 5000950 559 99460 50009.5 5000950 -32611 32527 4789.66 478966 -128 127 -2.66 -266 -56 2 10046 99956 0.16816 300.16816 150.16816 15166.98498 0.16816 300.16818 150.16816 15166.98512 0.16816 300.16816 150.1681599999999 15166.98416 2020-01-01 2020-01-02 2020-01-01 00:00:56 2020-01-02 03:45:56 2020-01-01 00:00:56.000 2020-01-02 03:45:56.000 56 99956 50006 5050606 56 99956 50006 5050606 -32513 32422 4585.009900990099 463086 -127 124 -2.712871287128713 -274 -560 2 10550 99461 1.68168 298.68168 150.18168 15018.16816 1.68168 298.68167 150.18168 15018.16815 1.68168 298.68168 150.18167999999986 15018.16800 2020-01-01 2020-01-02 2020-01-01 00:09:20 2020-01-02 03:37:41 2020-01-01 00:09:20.000 2020-01-02 03:37:41.000 560 99461 50010.5 5001050 560 99461 50010.5 5001050 -32610 32528 4790.66 479066 -128 127 -4.22 -422 -561 2 10551 99462 1.68468 298.68468 150.18468 15018.46846 1.68468 298.6847 150.18468 15018.46826 1.68468 298.68468 150.18468000000013 15018.46800 2020-01-01 2020-01-02 2020-01-01 00:09:21 2020-01-02 03:37:42 2020-01-01 00:09:21.000 2020-01-02 03:37:42.000 561 99462 50011.5 5001150 561 99462 50011.5 5001150 -32609 32529 4791.66 479166 -128 123 -5.78 -578 -562 2 10552 99463 1.68768 298.68768 150.18768 15018.76876 1.68768 298.68768 150.18768 15018.76856 1.68768 298.68768 150.1876799999998 15018.76800 2020-01-01 2020-01-02 2020-01-01 00:09:22 2020-01-02 03:37:43 2020-01-01 00:09:22.000 2020-01-02 03:37:43.000 562 99463 50012.5 5001250 562 99463 50012.5 5001250 -32608 32530 4792.66 479266 -127 124 -4.78 -478 +559 2 10549 99460 1.67867 298.67867 150.17867 15017.86786 1.67867 298.67868 150.17868 15017.86802 1.67867 298.67867 150.17867 15017.86700 2020-01-01 2020-01-02 2020-01-01 00:09:19 2020-01-02 03:37:40 2020-01-01 00:09:19.000 2020-01-02 03:37:40.000 559 99460 50009.5 5000950 559 99460 50009.5 5000950 -32611 32527 4789.66 478966 -128 127 -2.66 -266 +56 2 10046 99956 0.16816 300.16816 150.16816 15166.98498 0.16816 300.16818 150.16816 15166.98512 0.16816 300.16816 150.16816 15166.98416 2020-01-01 2020-01-02 2020-01-01 00:00:56 2020-01-02 03:45:56 2020-01-01 00:00:56.000 2020-01-02 03:45:56.000 56 99956 50006 5050606 56 99956 50006 5050606 -32513 32422 4585.009900990099 463086 -127 124 -2.712871287128713 -274 +560 2 10550 99461 1.68168 298.68168 150.18168 15018.16816 1.68168 298.68167 150.18168 15018.16815 1.68168 298.68168 150.18168 15018.16800 2020-01-01 2020-01-02 2020-01-01 00:09:20 2020-01-02 03:37:41 2020-01-01 00:09:20.000 2020-01-02 03:37:41.000 560 99461 50010.5 5001050 560 99461 50010.5 5001050 -32610 32528 4790.66 479066 -128 127 -4.22 -422 +561 2 10551 99462 1.68468 298.68468 150.18468 15018.46846 1.68468 298.6847 150.18468 15018.46826 1.68468 298.68468 150.18468000000001 15018.46800 2020-01-01 2020-01-02 2020-01-01 00:09:21 2020-01-02 03:37:42 2020-01-01 00:09:21.000 2020-01-02 03:37:42.000 561 99462 50011.5 5001150 561 99462 50011.5 5001150 -32609 32529 4791.66 479166 -128 123 -5.78 -578 +562 2 10552 99463 1.68768 298.68768 150.18768 15018.76876 1.68768 298.68768 150.18768 15018.76856 1.68768 298.68768 150.18768 15018.76800 2020-01-01 2020-01-02 2020-01-01 00:09:22 2020-01-02 03:37:43 2020-01-01 00:09:22.000 2020-01-02 03:37:43.000 562 99463 50012.5 5001250 562 99463 50012.5 5001250 -32608 32530 4792.66 479266 -127 124 -4.78 -478 563 2 10553 99464 1.69069 298.69069 150.19069 15019.06906 1.69069 298.6907 150.19069 15019.06915 1.69069 298.69069 150.19069 15019.06900 2020-01-01 2020-01-02 2020-01-01 00:09:23 2020-01-02 03:37:44 2020-01-01 00:09:23.000 2020-01-02 03:37:44.000 563 99464 50013.5 5001350 563 99464 50013.5 5001350 -32607 32531 4793.66 479366 -126 125 -3.78 -378 -564 2 10554 99465 1.69369 298.69369 150.19369 15019.36936 1.69369 298.6937 150.19369 15019.36948 1.69369 298.69369 150.19369000000003 15019.36900 2020-01-01 2020-01-02 2020-01-01 00:09:24 2020-01-02 03:37:45 2020-01-01 00:09:24.000 2020-01-02 03:37:45.000 564 99465 50014.5 5001450 564 99465 50014.5 5001450 -32606 32532 4794.66 479466 -125 126 -2.78 -278 -565 2 10555 99466 1.69669 298.69669 150.19669 15019.66966 1.69669 298.6967 150.19669 15019.66962 1.69669 298.69669 150.1966900000001 15019.66900 2020-01-01 2020-01-02 2020-01-01 00:09:25 2020-01-02 03:37:46 2020-01-01 00:09:25.000 2020-01-02 03:37:46.000 565 99466 50015.5 5001550 565 99466 50015.5 5001550 -32605 32533 4795.66 479566 -124 127 -1.78 -178 +564 2 10554 99465 1.69369 298.69369 150.19369 15019.36936 1.69369 298.6937 150.19369 15019.36948 1.69369 298.69369 150.19369 15019.36900 2020-01-01 2020-01-02 2020-01-01 00:09:24 2020-01-02 03:37:45 2020-01-01 00:09:24.000 2020-01-02 03:37:45.000 564 99465 50014.5 5001450 564 99465 50014.5 5001450 -32606 32532 4794.66 479466 -125 126 -2.78 -278 +565 2 10555 99466 1.69669 298.69669 150.19669 15019.66966 1.69669 298.6967 150.19669 15019.66962 1.69669 298.69669 150.19669 15019.66900 2020-01-01 2020-01-02 2020-01-01 00:09:25 2020-01-02 03:37:46 2020-01-01 00:09:25.000 2020-01-02 03:37:46.000 565 99466 50015.5 5001550 565 99466 50015.5 5001550 -32605 32533 4795.66 479566 -124 127 -1.78 -178 566 2 10556 99467 1.69969 298.69969 150.19969 15019.96996 1.69969 298.6997 150.1997 15019.97037 1.69969 298.69969 150.19969 15019.96900 2020-01-01 2020-01-02 2020-01-01 00:09:26 2020-01-02 03:37:47 2020-01-01 00:09:26.000 2020-01-02 03:37:47.000 566 99467 50016.5 5001650 566 99467 50016.5 5001650 -32604 32534 4796.66 479666 -128 127 -3.34 -334 -567 2 10557 99468 1.7027 298.7027 150.2027 15020.27027 1.7027 298.7027 150.2027 15020.27003 1.70270 298.70270 150.20269999999988 15020.27000 2020-01-01 2020-01-02 2020-01-01 00:09:27 2020-01-02 03:37:48 2020-01-01 00:09:27.000 2020-01-02 03:37:48.000 567 99468 50017.5 5001750 567 99468 50017.5 5001750 -32603 32535 4797.66 479766 -128 123 -4.9 -490 -568 2 10558 99469 1.7057 298.7057 150.2057 15020.57057 1.7057 298.70572 150.2057 15020.57066 1.70570 298.70570 150.2057000000002 15020.57000 2020-01-01 2020-01-02 2020-01-01 00:09:28 2020-01-02 03:37:49 2020-01-01 00:09:28.000 2020-01-02 03:37:49.000 568 99469 50018.5 5001850 568 99469 50018.5 5001850 -32602 32536 4798.66 479866 -127 124 -3.9 -390 -569 2 10559 99470 1.7087 298.7087 150.2087 15020.87087 1.7087 298.7087 150.2087 15020.87095 1.70870 298.70870 150.20869999999988 15020.87000 2020-01-01 2020-01-02 2020-01-01 00:09:29 2020-01-02 03:37:50 2020-01-01 00:09:29.000 2020-01-02 03:37:50.000 569 99470 50019.5 5001950 569 99470 50019.5 5001950 -32601 32537 4799.66 479966 -126 125 -2.9 -290 -57 2 10047 99957 0.17117 300.17117 150.17117 15167.28828 0.17117 300.17117 150.17117 15167.28841 0.17117 300.17117 150.1711699999998 15167.28817 2020-01-01 2020-01-02 2020-01-01 00:00:57 2020-01-02 03:45:57 2020-01-01 00:00:57.000 2020-01-02 03:45:57.000 57 99957 50007 5050707 57 99957 50007 5050707 -32512 32423 4586.009900990099 463187 -126 125 -1.7128712871287128 -173 -570 2 10560 99471 1.71171 298.71171 150.21171 15021.17117 1.71171 298.7117 150.21171 15021.17109 1.71171 298.71171 150.2117099999998 15021.17100 2020-01-01 2020-01-02 2020-01-01 00:09:30 2020-01-02 03:37:51 2020-01-01 00:09:30.000 2020-01-02 03:37:51.000 570 99471 50020.5 5002050 570 99471 50020.5 5002050 -32600 32538 4800.66 480066 -125 126 -1.9 -190 -571 2 10561 99472 1.71471 298.71471 150.21471 15021.47147 1.71471 298.71472 150.21471 15021.47184 1.71471 298.71471 150.21471000000014 15021.47100 2020-01-01 2020-01-02 2020-01-01 00:09:31 2020-01-02 03:37:52 2020-01-01 00:09:31.000 2020-01-02 03:37:52.000 571 99472 50021.5 5002150 571 99472 50021.5 5002150 -32599 32539 4801.66 480166 -124 127 -0.9 -90 -572 2 10562 99473 1.71771 298.71771 150.21771 15021.77177 1.71771 298.7177 150.21771 15021.7715 1.71771 298.71771 150.21771000000012 15021.77100 2020-01-01 2020-01-02 2020-01-01 00:09:32 2020-01-02 03:37:53 2020-01-01 00:09:32.000 2020-01-02 03:37:53.000 572 99473 50022.5 5002250 572 99473 50022.5 5002250 -32598 32540 4802.66 480266 -128 127 -2.46 -246 -573 2 10563 99474 1.72072 298.72072 150.22072 15022.07207 1.72072 298.72073 150.22072 15022.07212 1.72072 298.72072 150.2207199999999 15022.07200 2020-01-01 2020-01-02 2020-01-01 00:09:33 2020-01-02 03:37:54 2020-01-01 00:09:33.000 2020-01-02 03:37:54.000 573 99474 50023.5 5002350 573 99474 50023.5 5002350 -32597 32541 4803.66 480366 -128 123 -4.02 -402 +567 2 10557 99468 1.7027 298.7027 150.2027 15020.27027 1.7027 298.7027 150.2027 15020.27003 1.70270 298.70270 150.2027 15020.27000 2020-01-01 2020-01-02 2020-01-01 00:09:27 2020-01-02 03:37:48 2020-01-01 00:09:27.000 2020-01-02 03:37:48.000 567 99468 50017.5 5001750 567 99468 50017.5 5001750 -32603 32535 4797.66 479766 -128 123 -4.9 -490 +568 2 10558 99469 1.7057 298.7057 150.2057 15020.57057 1.7057 298.70572 150.2057 15020.57066 1.70570 298.70570 150.2057 15020.57000 2020-01-01 2020-01-02 2020-01-01 00:09:28 2020-01-02 03:37:49 2020-01-01 00:09:28.000 2020-01-02 03:37:49.000 568 99469 50018.5 5001850 568 99469 50018.5 5001850 -32602 32536 4798.66 479866 -127 124 -3.9 -390 +569 2 10559 99470 1.7087 298.7087 150.2087 15020.87087 1.7087 298.7087 150.2087 15020.87095 1.70870 298.70870 150.20870000000002 15020.87000 2020-01-01 2020-01-02 2020-01-01 00:09:29 2020-01-02 03:37:50 2020-01-01 00:09:29.000 2020-01-02 03:37:50.000 569 99470 50019.5 5001950 569 99470 50019.5 5001950 -32601 32537 4799.66 479966 -126 125 -2.9 -290 +57 2 10047 99957 0.17117 300.17117 150.17117 15167.28828 0.17117 300.17117 150.17117 15167.28841 0.17117 300.17117 150.17117 15167.28817 2020-01-01 2020-01-02 2020-01-01 00:00:57 2020-01-02 03:45:57 2020-01-01 00:00:57.000 2020-01-02 03:45:57.000 57 99957 50007 5050707 57 99957 50007 5050707 -32512 32423 4586.009900990099 463187 -126 125 -1.7128712871287128 -173 +570 2 10560 99471 1.71171 298.71171 150.21171 15021.17117 1.71171 298.7117 150.21171 15021.17109 1.71171 298.71171 150.21171 15021.17100 2020-01-01 2020-01-02 2020-01-01 00:09:30 2020-01-02 03:37:51 2020-01-01 00:09:30.000 2020-01-02 03:37:51.000 570 99471 50020.5 5002050 570 99471 50020.5 5002050 -32600 32538 4800.66 480066 -125 126 -1.9 -190 +571 2 10561 99472 1.71471 298.71471 150.21471 15021.47147 1.71471 298.71472 150.21471 15021.47184 1.71471 298.71471 150.21471 15021.47100 2020-01-01 2020-01-02 2020-01-01 00:09:31 2020-01-02 03:37:52 2020-01-01 00:09:31.000 2020-01-02 03:37:52.000 571 99472 50021.5 5002150 571 99472 50021.5 5002150 -32599 32539 4801.66 480166 -124 127 -0.9 -90 +572 2 10562 99473 1.71771 298.71771 150.21771 15021.77177 1.71771 298.7177 150.21771 15021.7715 1.71771 298.71771 150.21771 15021.77100 2020-01-01 2020-01-02 2020-01-01 00:09:32 2020-01-02 03:37:53 2020-01-01 00:09:32.000 2020-01-02 03:37:53.000 572 99473 50022.5 5002250 572 99473 50022.5 5002250 -32598 32540 4802.66 480266 -128 127 -2.46 -246 +573 2 10563 99474 1.72072 298.72072 150.22072 15022.07207 1.72072 298.72073 150.22072 15022.07212 1.72072 298.72072 150.22072 15022.07200 2020-01-01 2020-01-02 2020-01-01 00:09:33 2020-01-02 03:37:54 2020-01-01 00:09:33.000 2020-01-02 03:37:54.000 573 99474 50023.5 5002350 573 99474 50023.5 5002350 -32597 32541 4803.66 480366 -128 123 -4.02 -402 574 2 10564 99475 1.72372 298.72372 150.22372 15022.37237 1.72372 298.72372 150.22372 15022.37243 1.72372 298.72372 150.22372 15022.37200 2020-01-01 2020-01-02 2020-01-01 00:09:34 2020-01-02 03:37:55 2020-01-01 00:09:34.000 2020-01-02 03:37:55.000 574 99475 50024.5 5002450 574 99475 50024.5 5002450 -32596 32542 4804.66 480466 -127 124 -3.02 -302 -575 2 10565 99476 1.72672 298.72672 150.22672 15022.67267 1.72672 298.7267 150.22672 15022.67272 1.72672 298.72672 150.22672000000006 15022.67200 2020-01-01 2020-01-02 2020-01-01 00:09:35 2020-01-02 03:37:56 2020-01-01 00:09:35.000 2020-01-02 03:37:56.000 575 99476 50025.5 5002550 575 99476 50025.5 5002550 -32595 32543 4805.66 480566 -126 125 -2.02 -202 -576 2 10566 99477 1.72972 298.72972 150.22972 15022.97297 1.72972 298.72974 150.22973 15022.97332 1.72972 298.72972 150.22971999999993 15022.97200 2020-01-01 2020-01-02 2020-01-01 00:09:36 2020-01-02 03:37:57 2020-01-01 00:09:36.000 2020-01-02 03:37:57.000 576 99477 50026.5 5002650 576 99477 50026.5 5002650 -32594 32544 4806.66 480666 -125 126 -1.02 -102 -577 2 10567 99478 1.73273 298.73273 150.23273 15023.27327 1.73273 298.73273 150.23272 15023.27297 1.73273 298.73273 150.2327299999999 15023.27300 2020-01-01 2020-01-02 2020-01-01 00:09:37 2020-01-02 03:37:58 2020-01-01 00:09:37.000 2020-01-02 03:37:58.000 577 99478 50027.5 5002750 577 99478 50027.5 5002750 -32593 32545 4807.66 480766 -124 127 -0.02 -2 -578 2 10568 99479 1.73573 298.73573 150.23573 15023.57357 1.73573 298.73575 150.23573 15023.57359 1.73573 298.73573 150.23573000000016 15023.57300 2020-01-01 2020-01-02 2020-01-01 00:09:38 2020-01-02 03:37:59 2020-01-01 00:09:38.000 2020-01-02 03:37:59.000 578 99479 50028.5 5002850 578 99479 50028.5 5002850 -32592 32546 4808.66 480866 -128 127 -1.58 -158 -579 2 10569 99480 1.73873 298.73873 150.23873 15023.87387 1.73873 298.73874 150.23873 15023.8739 1.73873 298.73873 150.23872999999983 15023.87300 2020-01-01 2020-01-02 2020-01-01 00:09:39 2020-01-02 03:38:00 2020-01-01 00:09:39.000 2020-01-02 03:38:00.000 579 99480 50029.5 5002950 579 99480 50029.5 5002950 -32591 32547 4809.66 480966 -128 123 -3.14 -314 -58 2 10048 99958 0.17417 300.17417 150.17417 15167.59159 0.17417 300.17416 150.17417 15167.59159 0.17417 300.17417 150.17417000000012 15167.59117 2020-01-01 2020-01-02 2020-01-01 00:00:58 2020-01-02 03:45:58 2020-01-01 00:00:58.000 2020-01-02 03:45:58.000 58 99958 50008 5050808 58 99958 50008 5050808 -32511 32424 4587.009900990099 463288 -125 126 -0.7128712871287128 -72 +575 2 10565 99476 1.72672 298.72672 150.22672 15022.67267 1.72672 298.7267 150.22672 15022.67272 1.72672 298.72672 150.22672 15022.67200 2020-01-01 2020-01-02 2020-01-01 00:09:35 2020-01-02 03:37:56 2020-01-01 00:09:35.000 2020-01-02 03:37:56.000 575 99476 50025.5 5002550 575 99476 50025.5 5002550 -32595 32543 4805.66 480566 -126 125 -2.02 -202 +576 2 10566 99477 1.72972 298.72972 150.22972 15022.97297 1.72972 298.72974 150.22973 15022.97332 1.72972 298.72972 150.22972 15022.97200 2020-01-01 2020-01-02 2020-01-01 00:09:36 2020-01-02 03:37:57 2020-01-01 00:09:36.000 2020-01-02 03:37:57.000 576 99477 50026.5 5002650 576 99477 50026.5 5002650 -32594 32544 4806.66 480666 -125 126 -1.02 -102 +577 2 10567 99478 1.73273 298.73273 150.23273 15023.27327 1.73273 298.73273 150.23272 15023.27297 1.73273 298.73273 150.23273 15023.27300 2020-01-01 2020-01-02 2020-01-01 00:09:37 2020-01-02 03:37:58 2020-01-01 00:09:37.000 2020-01-02 03:37:58.000 577 99478 50027.5 5002750 577 99478 50027.5 5002750 -32593 32545 4807.66 480766 -124 127 -0.02 -2 +578 2 10568 99479 1.73573 298.73573 150.23573 15023.57357 1.73573 298.73575 150.23573 15023.57359 1.73573 298.73573 150.23573 15023.57300 2020-01-01 2020-01-02 2020-01-01 00:09:38 2020-01-02 03:37:59 2020-01-01 00:09:38.000 2020-01-02 03:37:59.000 578 99479 50028.5 5002850 578 99479 50028.5 5002850 -32592 32546 4808.66 480866 -128 127 -1.58 -158 +579 2 10569 99480 1.73873 298.73873 150.23873 15023.87387 1.73873 298.73874 150.23873 15023.8739 1.73873 298.73873 150.23873 15023.87300 2020-01-01 2020-01-02 2020-01-01 00:09:39 2020-01-02 03:38:00 2020-01-01 00:09:39.000 2020-01-02 03:38:00.000 579 99480 50029.5 5002950 579 99480 50029.5 5002950 -32591 32547 4809.66 480966 -128 123 -3.14 -314 +58 2 10048 99958 0.17417 300.17417 150.17417 15167.59159 0.17417 300.17416 150.17417 15167.59159 0.17417 300.17417 150.17417 15167.59117 2020-01-01 2020-01-02 2020-01-01 00:00:58 2020-01-02 03:45:58 2020-01-01 00:00:58.000 2020-01-02 03:45:58.000 58 99958 50008 5050808 58 99958 50008 5050808 -32511 32424 4587.009900990099 463288 -125 126 -0.7128712871287128 -72 580 2 10570 99481 1.74174 298.74174 150.24174 15024.17417 1.74174 298.74173 150.24174 15024.17419 1.74174 298.74174 150.24174000000002 15024.17400 2020-01-01 2020-01-02 2020-01-01 00:09:40 2020-01-02 03:38:01 2020-01-01 00:09:40.000 2020-01-02 03:38:01.000 580 99481 50030.5 5003050 580 99481 50030.5 5003050 -32590 32548 4810.66 481066 -127 124 -2.14 -214 -581 2 10571 99482 1.74474 298.74474 150.24474 15024.47447 1.74474 298.74475 150.24474 15024.47478 1.74474 298.74474 150.24474000000006 15024.47400 2020-01-01 2020-01-02 2020-01-01 00:09:41 2020-01-02 03:38:02 2020-01-01 00:09:41.000 2020-01-02 03:38:02.000 581 99482 50031.5 5003150 581 99482 50031.5 5003150 -32589 32549 4811.66 481166 -126 125 -1.14 -114 -582 2 10572 99483 1.74774 298.74774 150.24774 15024.77477 1.74774 298.74774 150.24774 15024.77447 1.74774 298.74774 150.24774000000005 15024.77400 2020-01-01 2020-01-02 2020-01-01 00:09:42 2020-01-02 03:38:03 2020-01-01 00:09:42.000 2020-01-02 03:38:03.000 582 99483 50032.5 5003250 582 99483 50032.5 5003250 -32588 32550 4812.66 481266 -125 126 -0.14 -14 -583 2 10573 99484 1.75075 298.75075 150.25075 15025.07507 1.75075 298.75076 150.25075 15025.07507 1.75075 298.75075 150.25074999999993 15025.07500 2020-01-01 2020-01-02 2020-01-01 00:09:43 2020-01-02 03:38:04 2020-01-01 00:09:43.000 2020-01-02 03:38:04.000 583 99484 50033.5 5003350 583 99484 50033.5 5003350 -32587 32551 4813.66 481366 -124 127 0.86 86 -584 2 10574 99485 1.75375 298.75375 150.25375 15025.37537 1.75375 298.75375 150.25375 15025.37536 1.75375 298.75375 150.2537499999999 15025.37500 2020-01-01 2020-01-02 2020-01-01 00:09:44 2020-01-02 03:38:05 2020-01-01 00:09:44.000 2020-01-02 03:38:05.000 584 99485 50034.5 5003450 584 99485 50034.5 5003450 -32586 32552 4814.66 481466 -128 127 -0.7 -70 -585 2 10575 99486 1.75675 298.75675 150.25675 15025.67567 1.75675 298.75674 150.25675 15025.67566 1.75675 298.75675 150.2567500000001 15025.67500 2020-01-01 2020-01-02 2020-01-01 00:09:45 2020-01-02 03:38:06 2020-01-01 00:09:45.000 2020-01-02 03:38:06.000 585 99486 50035.5 5003550 585 99486 50035.5 5003550 -32585 32553 4815.66 481566 -128 127 -2.26 -226 -586 2 10576 99487 1.75975 298.75975 150.25975 15025.97597 1.75975 298.75977 150.25976 15025.97625 1.75975 298.75975 150.25974999999988 15025.97500 2020-01-01 2020-01-02 2020-01-01 00:09:46 2020-01-02 03:38:07 2020-01-01 00:09:46.000 2020-01-02 03:38:07.000 586 99487 50036.5 5003650 586 99487 50036.5 5003650 -32584 32554 4816.66 481666 -128 123 -3.82 -382 -587 2 10577 99488 1.76276 298.76276 150.26276 15026.27627 1.76276 298.76276 150.26275 15026.27594 1.76276 298.76276 150.26275999999982 15026.27600 2020-01-01 2020-01-02 2020-01-01 00:09:47 2020-01-02 03:38:08 2020-01-01 00:09:47.000 2020-01-02 03:38:08.000 587 99488 50037.5 5003750 587 99488 50037.5 5003750 -32583 32555 4817.66 481766 -127 124 -2.82 -282 -588 2 10578 99489 1.76576 298.76576 150.26576 15026.57657 1.76576 298.76578 150.26576 15026.57654 1.76576 298.76576 150.26576000000014 15026.57600 2020-01-01 2020-01-02 2020-01-01 00:09:48 2020-01-02 03:38:09 2020-01-01 00:09:48.000 2020-01-02 03:38:09.000 588 99489 50038.5 5003850 588 99489 50038.5 5003850 -32582 32556 4818.66 481866 -126 125 -1.82 -182 -589 2 10579 99490 1.76876 298.76876 150.26876 15026.87687 1.76876 298.76877 150.26876 15026.87683 1.76876 298.76876 150.26876000000013 15026.87600 2020-01-01 2020-01-02 2020-01-01 00:09:49 2020-01-02 03:38:10 2020-01-01 00:09:49.000 2020-01-02 03:38:10.000 589 99490 50039.5 5003950 589 99490 50039.5 5003950 -32581 32557 4819.66 481966 -125 126 -0.82 -82 -59 2 10049 99959 0.17717 300.17717 150.17717 15167.89489 0.17717 300.1772 150.17717 15167.8947 0.17717 300.17717 150.17717000000013 15167.89417 2020-01-01 2020-01-02 2020-01-01 00:00:59 2020-01-02 03:45:59 2020-01-01 00:00:59.000 2020-01-02 03:45:59.000 59 99959 50009 5050909 59 99959 50009 5050909 -32510 32425 4588.009900990099 463389 -124 127 0.2871287128712871 29 -590 2 10580 99491 1.77177 298.77177 150.27177 15027.17717 1.77177 298.77176 150.27177 15027.17713 1.77177 298.77177 150.27176999999992 15027.17700 2020-01-01 2020-01-02 2020-01-01 00:09:50 2020-01-02 03:38:11 2020-01-01 00:09:50.000 2020-01-02 03:38:11.000 590 99491 50040.5 5004050 590 99491 50040.5 5004050 -32580 32558 4820.66 482066 -124 127 0.18 18 -591 2 10581 99492 1.77477 298.77477 150.27477 15027.47747 1.77477 298.77478 150.27477 15027.47775 1.77477 298.77477 150.27477 15027.47700 2020-01-01 2020-01-02 2020-01-01 00:09:51 2020-01-02 03:38:12 2020-01-01 00:09:51.000 2020-01-02 03:38:12.000 591 99492 50041.5 5004150 591 99492 50041.5 5004150 -32579 32559 4821.66 482166 -128 127 -1.38 -138 -592 2 10582 99493 1.77777 298.77777 150.27777 15027.77777 1.77777 298.77777 150.27777 15027.77742 1.77777 298.77777 150.27777000000006 15027.77700 2020-01-01 2020-01-02 2020-01-01 00:09:52 2020-01-02 03:38:13 2020-01-01 00:09:52.000 2020-01-02 03:38:13.000 592 99493 50042.5 5004250 592 99493 50042.5 5004250 -32578 32560 4822.66 482266 -128 123 -2.94 -294 -593 2 10583 99494 1.78078 298.78078 150.28078 15028.07807 1.78078 298.7808 150.28078 15028.078 1.78078 298.78078 150.28077999999985 15028.07800 2020-01-01 2020-01-02 2020-01-01 00:09:53 2020-01-02 03:38:14 2020-01-01 00:09:53.000 2020-01-02 03:38:14.000 593 99494 50043.5 5004350 593 99494 50043.5 5004350 -32577 32561 4823.66 482366 -127 124 -1.94 -194 -594 2 10584 99495 1.78378 298.78378 150.28378 15028.37837 1.78378 298.78378 150.28378 15028.3783 1.78378 298.78378 150.2837799999999 15028.37800 2020-01-01 2020-01-02 2020-01-01 00:09:54 2020-01-02 03:38:15 2020-01-01 00:09:54.000 2020-01-02 03:38:15.000 594 99495 50044.5 5004450 594 99495 50044.5 5004450 -32576 32562 4824.66 482466 -126 125 -0.94 -94 -595 2 10585 99496 1.78678 298.78678 150.28678 15028.67867 1.78678 298.78677 150.28678 15028.6786 1.78678 298.78678 150.28678000000016 15028.67800 2020-01-01 2020-01-02 2020-01-01 00:09:55 2020-01-02 03:38:16 2020-01-01 00:09:55.000 2020-01-02 03:38:16.000 595 99496 50045.5 5004550 595 99496 50045.5 5004550 -32575 32563 4825.66 482566 -125 126 0.06 6 -596 2 10586 99497 1.78978 298.78978 150.28978 15028.97897 1.78978 298.7898 150.28979 15028.97922 1.78978 298.78978 150.28977999999984 15028.97800 2020-01-01 2020-01-02 2020-01-01 00:09:56 2020-01-02 03:38:17 2020-01-01 00:09:56.000 2020-01-02 03:38:17.000 596 99497 50046.5 5004650 596 99497 50046.5 5004650 -32574 32564 4826.66 482666 -124 127 1.06 106 -597 2 10587 99498 1.79279 298.79279 150.29279 15029.27927 1.79279 298.7928 150.29278 15029.27888 1.79279 298.79279 150.29279000000005 15029.27900 2020-01-01 2020-01-02 2020-01-01 00:09:57 2020-01-02 03:38:18 2020-01-01 00:09:57.000 2020-01-02 03:38:18.000 597 99498 50047.5 5004750 597 99498 50047.5 5004750 -32573 32565 4827.66 482766 -128 127 -0.5 -50 -598 2 10588 99499 1.79579 298.79579 150.29579 15029.57957 1.79579 298.7958 150.29579 15029.57964 1.79579 298.79579 150.2957900000001 15029.57900 2020-01-01 2020-01-02 2020-01-01 00:09:58 2020-01-02 03:38:19 2020-01-01 00:09:58.000 2020-01-02 03:38:19.000 598 99499 50048.5 5004850 598 99499 50048.5 5004850 -32572 32566 4828.66 482866 -128 123 -2.06 -206 -599 2 10589 99500 1.79879 298.79879 150.29879 15029.87987 1.79879 298.7988 150.29879 15029.87977 1.79879 298.79879 150.2987900000001 15029.87900 2020-01-01 2020-01-02 2020-01-01 00:09:59 2020-01-02 03:38:20 2020-01-01 00:09:59.000 2020-01-02 03:38:20.000 599 99500 50049.5 5004950 599 99500 50049.5 5004950 -32571 32567 4829.66 482966 -127 124 -1.06 -106 -6 2 1005 9996 0.01801 300.01801 150.01801 15151.81981 0.01801 300.018 150.01801 15151.81978 0.01801 300.01801 150.01800999999986 15151.81901 2020-01-01 2020-01-02 2020-01-01 00:00:06 2020-01-02 03:45:06 2020-01-01 00:00:06.000 2020-01-02 03:45:06.000 6 99906 49956 5045556 6 99906 49956 5045556 -32563 32372 4535.009900990099 458036 -127 124 -2.01980198019802 -204 -60 2 10050 99960 0.18018 300.18018 150.18018 15168.19819 0.18018 300.18018 150.18017 15168.198 0.18018 300.18018 150.1801799999999 15168.19818 2020-01-01 2020-01-02 2020-01-01 00:01:00 2020-01-02 03:46:00 2020-01-01 00:01:00.000 2020-01-02 03:46:00.000 60 99960 50010 5051010 60 99960 50010 5051010 -32509 32426 4589.009900990099 463490 -128 127 -1.2475247524752475 -126 -600 2 10590 99501 1.8018 298.8018 150.3018 15030.18018 1.8018 298.8018 150.3018 15030.1801 1.80180 298.80180 150.3017999999999 15030.18000 2020-01-01 2020-01-02 2020-01-01 00:10:00 2020-01-02 03:38:21 2020-01-01 00:10:00.000 2020-01-02 03:38:21.000 600 99501 50050.5 5005050 600 99501 50050.5 5005050 -32570 32568 4830.66 483066 -126 125 -0.06 -6 -601 2 10591 99502 1.8048 298.8048 150.3048 15030.48048 1.8048 298.8048 150.3048 15030.4807 1.80480 298.80480 150.30479999999991 15030.48000 2020-01-01 2020-01-02 2020-01-01 00:10:01 2020-01-02 03:38:22 2020-01-01 00:10:01.000 2020-01-02 03:38:22.000 601 99502 50051.5 5005150 601 99502 50051.5 5005150 -32569 32569 4831.66 483166 -125 126 0.94 94 -602 2 10592 99503 1.8078 298.8078 150.3078 15030.78078 1.8078 298.8078 150.3078 15030.78035 1.80780 298.80780 150.3078 15030.78000 2020-01-01 2020-01-02 2020-01-01 00:10:02 2020-01-02 03:38:23 2020-01-01 00:10:02.000 2020-01-02 03:38:23.000 602 99503 50052.5 5005250 602 99503 50052.5 5005250 -32568 32570 4832.66 483266 -124 127 1.94 194 -603 2 10593 99504 1.81081 298.81081 150.31081 15031.08108 1.81081 298.81082 150.31081 15031.0811 1.81081 298.81081 150.31081000000017 15031.08100 2020-01-01 2020-01-02 2020-01-01 00:10:03 2020-01-02 03:38:24 2020-01-01 00:10:03.000 2020-01-02 03:38:24.000 603 99504 50053.5 5005350 603 99504 50053.5 5005350 -32567 32571 4833.66 483366 -128 127 0.38 38 -604 2 10594 99505 1.81381 298.81381 150.31381 15031.38138 1.81381 298.8138 150.31381 15031.38124 1.81381 298.81381 150.31380999999985 15031.38100 2020-01-01 2020-01-02 2020-01-01 00:10:04 2020-01-02 03:38:25 2020-01-01 00:10:04.000 2020-01-02 03:38:25.000 604 99505 50054.5 5005450 604 99505 50054.5 5005450 -32566 32572 4834.66 483466 -128 123 -1.18 -118 -605 2 10595 99506 1.81681 298.81681 150.31681 15031.68168 1.81681 298.8168 150.31681 15031.68157 1.81681 298.81681 150.3168100000001 15031.68100 2020-01-01 2020-01-02 2020-01-01 00:10:05 2020-01-02 03:38:26 2020-01-01 00:10:05.000 2020-01-02 03:38:26.000 605 99506 50055.5 5005550 605 99506 50055.5 5005550 -32565 32573 4835.66 483566 -127 124 -0.18 -18 -606 2 10596 99507 1.81981 298.81981 150.31981 15031.98198 1.81981 298.81982 150.31982 15031.98217 1.81981 298.81981 150.3198100000002 15031.98100 2020-01-01 2020-01-02 2020-01-01 00:10:06 2020-01-02 03:38:27 2020-01-01 00:10:06.000 2020-01-02 03:38:27.000 606 99507 50056.5 5005650 606 99507 50056.5 5005650 -32564 32574 4836.66 483666 -126 125 0.82 82 -607 2 10597 99508 1.82282 298.82282 150.32282 15032.28228 1.82282 298.8228 150.32282 15032.28246 1.82282 298.82282 150.32281999999995 15032.28200 2020-01-01 2020-01-02 2020-01-01 00:10:07 2020-01-02 03:38:28 2020-01-01 00:10:07.000 2020-01-02 03:38:28.000 607 99508 50057.5 5005750 607 99508 50057.5 5005750 -32563 32575 4837.66 483766 -125 126 1.82 182 -608 2 10598 99509 1.82582 298.82582 150.32582 15032.58258 1.82582 298.82584 150.32582 15032.58258 1.82582 298.82582 150.32582000000002 15032.58200 2020-01-01 2020-01-02 2020-01-01 00:10:08 2020-01-02 03:38:29 2020-01-01 00:10:08.000 2020-01-02 03:38:29.000 608 99509 50058.5 5005850 608 99509 50058.5 5005850 -32562 32576 4838.66 483866 -124 127 2.82 282 -609 2 10599 99510 1.82882 298.82882 150.32882 15032.88288 1.82882 298.82883 150.32882 15032.88274 1.82882 298.82882 150.3288200000001 15032.88200 2020-01-01 2020-01-02 2020-01-01 00:10:09 2020-01-02 03:38:30 2020-01-01 00:10:09.000 2020-01-02 03:38:30.000 609 99510 50059.5 5005950 609 99510 50059.5 5005950 -32561 32577 4839.66 483966 -128 127 1.26 126 +581 2 10571 99482 1.74474 298.74474 150.24474 15024.47447 1.74474 298.74475 150.24474 15024.47478 1.74474 298.74474 150.24474 15024.47400 2020-01-01 2020-01-02 2020-01-01 00:09:41 2020-01-02 03:38:02 2020-01-01 00:09:41.000 2020-01-02 03:38:02.000 581 99482 50031.5 5003150 581 99482 50031.5 5003150 -32589 32549 4811.66 481166 -126 125 -1.14 -114 +582 2 10572 99483 1.74774 298.74774 150.24774 15024.77477 1.74774 298.74774 150.24774 15024.77447 1.74774 298.74774 150.24774 15024.77400 2020-01-01 2020-01-02 2020-01-01 00:09:42 2020-01-02 03:38:03 2020-01-01 00:09:42.000 2020-01-02 03:38:03.000 582 99483 50032.5 5003250 582 99483 50032.5 5003250 -32588 32550 4812.66 481266 -125 126 -0.14 -14 +583 2 10573 99484 1.75075 298.75075 150.25075 15025.07507 1.75075 298.75076 150.25075 15025.07507 1.75075 298.75075 150.25075 15025.07500 2020-01-01 2020-01-02 2020-01-01 00:09:43 2020-01-02 03:38:04 2020-01-01 00:09:43.000 2020-01-02 03:38:04.000 583 99484 50033.5 5003350 583 99484 50033.5 5003350 -32587 32551 4813.66 481366 -124 127 0.86 86 +584 2 10574 99485 1.75375 298.75375 150.25375 15025.37537 1.75375 298.75375 150.25375 15025.37536 1.75375 298.75375 150.25375 15025.37500 2020-01-01 2020-01-02 2020-01-01 00:09:44 2020-01-02 03:38:05 2020-01-01 00:09:44.000 2020-01-02 03:38:05.000 584 99485 50034.5 5003450 584 99485 50034.5 5003450 -32586 32552 4814.66 481466 -128 127 -0.7 -70 +585 2 10575 99486 1.75675 298.75675 150.25675 15025.67567 1.75675 298.75674 150.25675 15025.67566 1.75675 298.75675 150.25674999999998 15025.67500 2020-01-01 2020-01-02 2020-01-01 00:09:45 2020-01-02 03:38:06 2020-01-01 00:09:45.000 2020-01-02 03:38:06.000 585 99486 50035.5 5003550 585 99486 50035.5 5003550 -32585 32553 4815.66 481566 -128 127 -2.26 -226 +586 2 10576 99487 1.75975 298.75975 150.25975 15025.97597 1.75975 298.75977 150.25976 15025.97625 1.75975 298.75975 150.25975 15025.97500 2020-01-01 2020-01-02 2020-01-01 00:09:46 2020-01-02 03:38:07 2020-01-01 00:09:46.000 2020-01-02 03:38:07.000 586 99487 50036.5 5003650 586 99487 50036.5 5003650 -32584 32554 4816.66 481666 -128 123 -3.82 -382 +587 2 10577 99488 1.76276 298.76276 150.26276 15026.27627 1.76276 298.76276 150.26275 15026.27594 1.76276 298.76276 150.26276 15026.27600 2020-01-01 2020-01-02 2020-01-01 00:09:47 2020-01-02 03:38:08 2020-01-01 00:09:47.000 2020-01-02 03:38:08.000 587 99488 50037.5 5003750 587 99488 50037.5 5003750 -32583 32555 4817.66 481766 -127 124 -2.82 -282 +588 2 10578 99489 1.76576 298.76576 150.26576 15026.57657 1.76576 298.76578 150.26576 15026.57654 1.76576 298.76576 150.26576 15026.57600 2020-01-01 2020-01-02 2020-01-01 00:09:48 2020-01-02 03:38:09 2020-01-01 00:09:48.000 2020-01-02 03:38:09.000 588 99489 50038.5 5003850 588 99489 50038.5 5003850 -32582 32556 4818.66 481866 -126 125 -1.82 -182 +589 2 10579 99490 1.76876 298.76876 150.26876 15026.87687 1.76876 298.76877 150.26876 15026.87683 1.76876 298.76876 150.26876000000001 15026.87600 2020-01-01 2020-01-02 2020-01-01 00:09:49 2020-01-02 03:38:10 2020-01-01 00:09:49.000 2020-01-02 03:38:10.000 589 99490 50039.5 5003950 589 99490 50039.5 5003950 -32581 32557 4819.66 481966 -125 126 -0.82 -82 +59 2 10049 99959 0.17717 300.17717 150.17717 15167.89489 0.17717 300.1772 150.17717 15167.8947 0.17717 300.17717 150.17717 15167.89417 2020-01-01 2020-01-02 2020-01-01 00:00:59 2020-01-02 03:45:59 2020-01-01 00:00:59.000 2020-01-02 03:45:59.000 59 99959 50009 5050909 59 99959 50009 5050909 -32510 32425 4588.009900990099 463389 -124 127 0.2871287128712871 29 +590 2 10580 99491 1.77177 298.77177 150.27177 15027.17717 1.77177 298.77176 150.27177 15027.17713 1.77177 298.77177 150.27177 15027.17700 2020-01-01 2020-01-02 2020-01-01 00:09:50 2020-01-02 03:38:11 2020-01-01 00:09:50.000 2020-01-02 03:38:11.000 590 99491 50040.5 5004050 590 99491 50040.5 5004050 -32580 32558 4820.66 482066 -124 127 0.18 18 +591 2 10581 99492 1.77477 298.77477 150.27477 15027.47747 1.77477 298.77478 150.27477 15027.47775 1.77477 298.77477 150.27477000000002 15027.47700 2020-01-01 2020-01-02 2020-01-01 00:09:51 2020-01-02 03:38:12 2020-01-01 00:09:51.000 2020-01-02 03:38:12.000 591 99492 50041.5 5004150 591 99492 50041.5 5004150 -32579 32559 4821.66 482166 -128 127 -1.38 -138 +592 2 10582 99493 1.77777 298.77777 150.27777 15027.77777 1.77777 298.77777 150.27777 15027.77742 1.77777 298.77777 150.27777 15027.77700 2020-01-01 2020-01-02 2020-01-01 00:09:52 2020-01-02 03:38:13 2020-01-01 00:09:52.000 2020-01-02 03:38:13.000 592 99493 50042.5 5004250 592 99493 50042.5 5004250 -32578 32560 4822.66 482266 -128 123 -2.94 -294 +593 2 10583 99494 1.78078 298.78078 150.28078 15028.07807 1.78078 298.7808 150.28078 15028.078 1.78078 298.78078 150.28078 15028.07800 2020-01-01 2020-01-02 2020-01-01 00:09:53 2020-01-02 03:38:14 2020-01-01 00:09:53.000 2020-01-02 03:38:14.000 593 99494 50043.5 5004350 593 99494 50043.5 5004350 -32577 32561 4823.66 482366 -127 124 -1.94 -194 +594 2 10584 99495 1.78378 298.78378 150.28378 15028.37837 1.78378 298.78378 150.28378 15028.3783 1.78378 298.78378 150.28378 15028.37800 2020-01-01 2020-01-02 2020-01-01 00:09:54 2020-01-02 03:38:15 2020-01-01 00:09:54.000 2020-01-02 03:38:15.000 594 99495 50044.5 5004450 594 99495 50044.5 5004450 -32576 32562 4824.66 482466 -126 125 -0.94 -94 +595 2 10585 99496 1.78678 298.78678 150.28678 15028.67867 1.78678 298.78677 150.28678 15028.6786 1.78678 298.78678 150.28678 15028.67800 2020-01-01 2020-01-02 2020-01-01 00:09:55 2020-01-02 03:38:16 2020-01-01 00:09:55.000 2020-01-02 03:38:16.000 595 99496 50045.5 5004550 595 99496 50045.5 5004550 -32575 32563 4825.66 482566 -125 126 0.06 6 +596 2 10586 99497 1.78978 298.78978 150.28978 15028.97897 1.78978 298.7898 150.28979 15028.97922 1.78978 298.78978 150.28977999999998 15028.97800 2020-01-01 2020-01-02 2020-01-01 00:09:56 2020-01-02 03:38:17 2020-01-01 00:09:56.000 2020-01-02 03:38:17.000 596 99497 50046.5 5004650 596 99497 50046.5 5004650 -32574 32564 4826.66 482666 -124 127 1.06 106 +597 2 10587 99498 1.79279 298.79279 150.29279 15029.27927 1.79279 298.7928 150.29278 15029.27888 1.79279 298.79279 150.29279 15029.27900 2020-01-01 2020-01-02 2020-01-01 00:09:57 2020-01-02 03:38:18 2020-01-01 00:09:57.000 2020-01-02 03:38:18.000 597 99498 50047.5 5004750 597 99498 50047.5 5004750 -32573 32565 4827.66 482766 -128 127 -0.5 -50 +598 2 10588 99499 1.79579 298.79579 150.29579 15029.57957 1.79579 298.7958 150.29579 15029.57964 1.79579 298.79579 150.29579 15029.57900 2020-01-01 2020-01-02 2020-01-01 00:09:58 2020-01-02 03:38:19 2020-01-01 00:09:58.000 2020-01-02 03:38:19.000 598 99499 50048.5 5004850 598 99499 50048.5 5004850 -32572 32566 4828.66 482866 -128 123 -2.06 -206 +599 2 10589 99500 1.79879 298.79879 150.29879 15029.87987 1.79879 298.7988 150.29879 15029.87977 1.79879 298.79879 150.29879 15029.87900 2020-01-01 2020-01-02 2020-01-01 00:09:59 2020-01-02 03:38:20 2020-01-01 00:09:59.000 2020-01-02 03:38:20.000 599 99500 50049.5 5004950 599 99500 50049.5 5004950 -32571 32567 4829.66 482966 -127 124 -1.06 -106 +6 2 1005 9996 0.01801 300.01801 150.01801 15151.81981 0.01801 300.018 150.01801 15151.81978 0.01801 300.01801 150.01801 15151.81901 2020-01-01 2020-01-02 2020-01-01 00:00:06 2020-01-02 03:45:06 2020-01-01 00:00:06.000 2020-01-02 03:45:06.000 6 99906 49956 5045556 6 99906 49956 5045556 -32563 32372 4535.009900990099 458036 -127 124 -2.01980198019802 -204 +60 2 10050 99960 0.18018 300.18018 150.18018 15168.19819 0.18018 300.18018 150.18017 15168.198 0.18018 300.18018 150.18018 15168.19818 2020-01-01 2020-01-02 2020-01-01 00:01:00 2020-01-02 03:46:00 2020-01-01 00:01:00.000 2020-01-02 03:46:00.000 60 99960 50010 5051010 60 99960 50010 5051010 -32509 32426 4589.009900990099 463490 -128 127 -1.2475247524752475 -126 +600 2 10590 99501 1.8018 298.8018 150.3018 15030.18018 1.8018 298.8018 150.3018 15030.1801 1.80180 298.80180 150.30180000000001 15030.18000 2020-01-01 2020-01-02 2020-01-01 00:10:00 2020-01-02 03:38:21 2020-01-01 00:10:00.000 2020-01-02 03:38:21.000 600 99501 50050.5 5005050 600 99501 50050.5 5005050 -32570 32568 4830.66 483066 -126 125 -0.06 -6 +601 2 10591 99502 1.8048 298.8048 150.3048 15030.48048 1.8048 298.8048 150.3048 15030.4807 1.80480 298.80480 150.3048 15030.48000 2020-01-01 2020-01-02 2020-01-01 00:10:01 2020-01-02 03:38:22 2020-01-01 00:10:01.000 2020-01-02 03:38:22.000 601 99502 50051.5 5005150 601 99502 50051.5 5005150 -32569 32569 4831.66 483166 -125 126 0.94 94 +602 2 10592 99503 1.8078 298.8078 150.3078 15030.78078 1.8078 298.8078 150.3078 15030.78035 1.80780 298.80780 150.30780000000001 15030.78000 2020-01-01 2020-01-02 2020-01-01 00:10:02 2020-01-02 03:38:23 2020-01-01 00:10:02.000 2020-01-02 03:38:23.000 602 99503 50052.5 5005250 602 99503 50052.5 5005250 -32568 32570 4832.66 483266 -124 127 1.94 194 +603 2 10593 99504 1.81081 298.81081 150.31081 15031.08108 1.81081 298.81082 150.31081 15031.0811 1.81081 298.81081 150.31081 15031.08100 2020-01-01 2020-01-02 2020-01-01 00:10:03 2020-01-02 03:38:24 2020-01-01 00:10:03.000 2020-01-02 03:38:24.000 603 99504 50053.5 5005350 603 99504 50053.5 5005350 -32567 32571 4833.66 483366 -128 127 0.38 38 +604 2 10594 99505 1.81381 298.81381 150.31381 15031.38138 1.81381 298.8138 150.31381 15031.38124 1.81381 298.81381 150.31381 15031.38100 2020-01-01 2020-01-02 2020-01-01 00:10:04 2020-01-02 03:38:25 2020-01-01 00:10:04.000 2020-01-02 03:38:25.000 604 99505 50054.5 5005450 604 99505 50054.5 5005450 -32566 32572 4834.66 483466 -128 123 -1.18 -118 +605 2 10595 99506 1.81681 298.81681 150.31681 15031.68168 1.81681 298.8168 150.31681 15031.68157 1.81681 298.81681 150.31681 15031.68100 2020-01-01 2020-01-02 2020-01-01 00:10:05 2020-01-02 03:38:26 2020-01-01 00:10:05.000 2020-01-02 03:38:26.000 605 99506 50055.5 5005550 605 99506 50055.5 5005550 -32565 32573 4835.66 483566 -127 124 -0.18 -18 +606 2 10596 99507 1.81981 298.81981 150.31981 15031.98198 1.81981 298.81982 150.31982 15031.98217 1.81981 298.81981 150.31981 15031.98100 2020-01-01 2020-01-02 2020-01-01 00:10:06 2020-01-02 03:38:27 2020-01-01 00:10:06.000 2020-01-02 03:38:27.000 606 99507 50056.5 5005650 606 99507 50056.5 5005650 -32564 32574 4836.66 483666 -126 125 0.82 82 +607 2 10597 99508 1.82282 298.82282 150.32282 15032.28228 1.82282 298.8228 150.32282 15032.28246 1.82282 298.82282 150.32281999999998 15032.28200 2020-01-01 2020-01-02 2020-01-01 00:10:07 2020-01-02 03:38:28 2020-01-01 00:10:07.000 2020-01-02 03:38:28.000 607 99508 50057.5 5005750 607 99508 50057.5 5005750 -32563 32575 4837.66 483766 -125 126 1.82 182 +608 2 10598 99509 1.82582 298.82582 150.32582 15032.58258 1.82582 298.82584 150.32582 15032.58258 1.82582 298.82582 150.32582 15032.58200 2020-01-01 2020-01-02 2020-01-01 00:10:08 2020-01-02 03:38:29 2020-01-01 00:10:08.000 2020-01-02 03:38:29.000 608 99509 50058.5 5005850 608 99509 50058.5 5005850 -32562 32576 4838.66 483866 -124 127 2.82 282 +609 2 10599 99510 1.82882 298.82882 150.32882 15032.88288 1.82882 298.82883 150.32882 15032.88274 1.82882 298.82882 150.32882 15032.88200 2020-01-01 2020-01-02 2020-01-01 00:10:09 2020-01-02 03:38:30 2020-01-01 00:10:09.000 2020-01-02 03:38:30.000 609 99510 50059.5 5005950 609 99510 50059.5 5005950 -32561 32577 4839.66 483966 -128 127 1.26 126 61 2 10051 99961 0.18318 300.18318 150.18318 15168.5015 0.18318 300.1832 150.18318 15168.5016 0.18318 300.18318 150.18318 15168.50118 2020-01-01 2020-01-02 2020-01-01 00:01:01 2020-01-02 03:46:01 2020-01-01 00:01:01.000 2020-01-02 03:46:01.000 61 99961 50011 5051111 61 99961 50011 5051111 -32508 32427 4590.009900990099 463591 -128 123 -2.782178217821782 -281 -610 2 10600 99511 1.83183 298.83183 150.33183 15033.18318 1.83183 298.83182 150.33183 15033.18304 1.83183 298.83183 150.33182999999985 15033.18300 2020-01-01 2020-01-02 2020-01-01 00:10:10 2020-01-02 03:38:31 2020-01-01 00:10:10.000 2020-01-02 03:38:31.000 610 99511 50060.5 5006050 610 99511 50060.5 5006050 -32560 32578 4840.66 484066 -128 127 -0.3 -30 -611 2 10601 99512 1.83483 298.83483 150.33483 15033.48348 1.83483 298.83484 150.33483 15033.48363 1.83483 298.83483 150.33482999999984 15033.48300 2020-01-01 2020-01-02 2020-01-01 00:10:11 2020-01-02 03:38:32 2020-01-01 00:10:11.000 2020-01-02 03:38:32.000 611 99512 50061.5 5006150 611 99512 50061.5 5006150 -32559 32579 4841.66 484166 -128 123 -1.86 -186 -612 2 10602 99513 1.83783 298.83783 150.33783 15033.78378 1.83783 298.83783 150.33783 15033.78393 1.83783 298.83783 150.33783000000022 15033.78300 2020-01-01 2020-01-02 2020-01-01 00:10:12 2020-01-02 03:38:33 2020-01-01 00:10:12.000 2020-01-02 03:38:33.000 612 99513 50062.5 5006250 612 99513 50062.5 5006250 -32558 32580 4842.66 484266 -127 124 -0.86 -86 -613 2 10603 99514 1.84084 298.84084 150.34084 15034.08408 1.84084 298.84085 150.34084 15034.08405 1.84084 298.84084 150.34084000000007 15034.08400 2020-01-01 2020-01-02 2020-01-01 00:10:13 2020-01-02 03:38:34 2020-01-01 00:10:13.000 2020-01-02 03:38:34.000 613 99514 50063.5 5006350 613 99514 50063.5 5006350 -32557 32581 4843.66 484366 -126 125 0.14 14 -614 2 10604 99515 1.84384 298.84384 150.34384 15034.38438 1.84384 298.84384 150.34384 15034.38421 1.84384 298.84384 150.34383999999977 15034.38400 2020-01-01 2020-01-02 2020-01-01 00:10:14 2020-01-02 03:38:35 2020-01-01 00:10:14.000 2020-01-02 03:38:35.000 614 99515 50064.5 5006450 614 99515 50064.5 5006450 -32556 32582 4844.66 484466 -125 126 1.14 114 -615 2 10605 99516 1.84684 298.84684 150.34684 15034.68468 1.84684 298.84683 150.34684 15034.68452 1.84684 298.84684 150.34684000000007 15034.68400 2020-01-01 2020-01-02 2020-01-01 00:10:15 2020-01-02 03:38:36 2020-01-01 00:10:15.000 2020-01-02 03:38:36.000 615 99516 50065.5 5006550 615 99516 50065.5 5006550 -32555 32583 4845.66 484566 -124 127 2.14 214 -616 2 10606 99517 1.84984 298.84984 150.34984 15034.98498 1.84984 298.84985 150.34985 15034.98527 1.84984 298.84984 150.34984000000014 15034.98400 2020-01-01 2020-01-02 2020-01-01 00:10:16 2020-01-02 03:38:37 2020-01-01 00:10:16.000 2020-01-02 03:38:37.000 616 99517 50066.5 5006650 616 99517 50066.5 5006650 -32554 32584 4846.66 484666 -128 127 0.58 58 -617 2 10607 99518 1.85285 298.85285 150.35285 15035.28528 1.85285 298.85284 150.35285 15035.2854 1.85285 298.85285 150.35284999999993 15035.28500 2020-01-01 2020-01-02 2020-01-01 00:10:17 2020-01-02 03:38:38 2020-01-01 00:10:17.000 2020-01-02 03:38:38.000 617 99518 50067.5 5006750 617 99518 50067.5 5006750 -32553 32585 4847.66 484766 -128 123 -0.98 -98 -618 2 10608 99519 1.85585 298.85585 150.35585 15035.58558 1.85585 298.85587 150.35585 15035.58551 1.85585 298.85585 150.35584999999995 15035.58500 2020-01-01 2020-01-02 2020-01-01 00:10:18 2020-01-02 03:38:39 2020-01-01 00:10:18.000 2020-01-02 03:38:39.000 618 99519 50068.5 5006850 618 99519 50068.5 5006850 -32552 32586 4848.66 484866 -127 124 0.02 2 -619 2 10609 99520 1.85885 298.85885 150.35885 15035.88588 1.85885 298.85886 150.35885 15035.88568 1.85885 298.85885 150.35885000000002 15035.88500 2020-01-01 2020-01-02 2020-01-01 00:10:19 2020-01-02 03:38:40 2020-01-01 00:10:19.000 2020-01-02 03:38:40.000 619 99520 50069.5 5006950 619 99520 50069.5 5006950 -32551 32587 4849.66 484966 -126 125 1.02 102 -62 2 10052 99962 0.18618 300.18618 150.18618 15168.8048 0.18618 300.1862 150.18618 15168.80494 0.18618 300.18618 150.18618000000006 15168.80418 2020-01-01 2020-01-02 2020-01-01 00:01:02 2020-01-02 03:46:02 2020-01-01 00:01:02.000 2020-01-02 03:46:02.000 62 99962 50012 5051212 62 99962 50012 5051212 -32507 32428 4591.009900990099 463692 -127 124 -1.7821782178217822 -180 -620 2 10610 99521 1.86186 298.86186 150.36186 15036.18618 1.86186 298.86185 150.36185 15036.18598 1.86186 298.86186 150.36186000000018 15036.18600 2020-01-01 2020-01-02 2020-01-01 00:10:20 2020-01-02 03:38:41 2020-01-01 00:10:20.000 2020-01-02 03:38:41.000 620 99521 50070.5 5007050 620 99521 50070.5 5007050 -32550 32588 4850.66 485066 -125 126 2.02 202 -621 2 10611 99522 1.86486 298.86486 150.36486 15036.48648 1.86486 298.86487 150.36486 15036.48673 1.86486 298.86486 150.36485999999985 15036.48600 2020-01-01 2020-01-02 2020-01-01 00:10:21 2020-01-02 03:38:42 2020-01-01 00:10:21.000 2020-01-02 03:38:42.000 621 99522 50071.5 5007150 621 99522 50071.5 5007150 -32549 32589 4851.66 485166 -124 127 3.02 302 -622 2 10612 99523 1.86786 298.86786 150.36786 15036.78678 1.86786 298.86786 150.36786 15036.78687 1.86786 298.86786 150.36786000000012 15036.78600 2020-01-01 2020-01-02 2020-01-01 00:10:22 2020-01-02 03:38:43 2020-01-01 00:10:22.000 2020-01-02 03:38:43.000 622 99523 50072.5 5007250 622 99523 50072.5 5007250 -32548 32590 4852.66 485266 -128 127 1.46 146 -623 2 10613 99524 1.87087 298.87087 150.37087 15037.08708 1.87087 298.87088 150.37087 15037.08702 1.87087 298.87087 150.3708700000001 15037.08700 2020-01-01 2020-01-02 2020-01-01 00:10:23 2020-01-02 03:38:44 2020-01-01 00:10:23.000 2020-01-02 03:38:44.000 623 99524 50073.5 5007350 623 99524 50073.5 5007350 -32547 32591 4853.66 485366 -128 123 -0.1 -10 -624 2 10614 99525 1.87387 298.87387 150.37387 15037.38738 1.87387 298.87387 150.37387 15037.38716 1.87387 298.87387 150.37386999999998 15037.38700 2020-01-01 2020-01-02 2020-01-01 00:10:24 2020-01-02 03:38:45 2020-01-01 00:10:24.000 2020-01-02 03:38:45.000 624 99525 50074.5 5007450 624 99525 50074.5 5007450 -32546 32592 4854.66 485466 -127 124 0.9 90 -625 2 10615 99526 1.87687 298.87687 150.37687 15037.68768 1.87687 298.8769 150.37687 15037.68791 1.87687 298.87687 150.37687000000003 15037.68700 2020-01-01 2020-01-02 2020-01-01 00:10:25 2020-01-02 03:38:46 2020-01-01 00:10:25.000 2020-01-02 03:38:46.000 625 99526 50075.5 5007550 625 99526 50075.5 5007550 -32545 32593 4855.66 485566 -126 125 1.9 190 -626 2 10616 99527 1.87987 298.87987 150.37987 15037.98798 1.87987 298.87988 150.37988 15037.9882 1.87987 298.87987 150.3798700000001 15037.98700 2020-01-01 2020-01-02 2020-01-01 00:10:26 2020-01-02 03:38:47 2020-01-01 00:10:26.000 2020-01-02 03:38:47.000 626 99527 50076.5 5007650 626 99527 50076.5 5007650 -32544 32594 4856.66 485666 -125 126 2.9 290 -627 2 10617 99528 1.88288 298.88288 150.38288 15038.28828 1.88288 298.88287 150.38288 15038.28834 1.88288 298.88288 150.38287999999991 15038.28800 2020-01-01 2020-01-02 2020-01-01 00:10:27 2020-01-02 03:38:48 2020-01-01 00:10:27.000 2020-01-02 03:38:48.000 627 99528 50077.5 5007750 627 99528 50077.5 5007750 -32543 32595 4857.66 485766 -124 127 3.9 390 -628 2 10618 99529 1.88588 298.88588 150.38588 15038.58858 1.88588 298.8859 150.38588 15038.58849 1.88588 298.88588 150.38587999999987 15038.58800 2020-01-01 2020-01-02 2020-01-01 00:10:28 2020-01-02 03:38:49 2020-01-01 00:10:28.000 2020-01-02 03:38:49.000 628 99529 50078.5 5007850 628 99529 50078.5 5007850 -32542 32596 4858.66 485866 -128 127 2.34 234 -629 2 10619 99530 1.88888 298.88888 150.38888 15038.88888 1.88888 298.8889 150.38888 15038.88862 1.88888 298.88888 150.3888800000002 15038.88800 2020-01-01 2020-01-02 2020-01-01 00:10:29 2020-01-02 03:38:50 2020-01-01 00:10:29.000 2020-01-02 03:38:50.000 629 99530 50079.5 5007950 629 99530 50079.5 5007950 -32541 32597 4859.66 485966 -128 123 0.78 78 -63 2 10053 99963 0.18918 300.18918 150.18918 15169.1081 0.18918 300.18918 150.18918 15169.10808 0.18918 300.18918 150.1891799999999 15169.10718 2020-01-01 2020-01-02 2020-01-01 00:01:03 2020-01-02 03:46:03 2020-01-01 00:01:03.000 2020-01-02 03:46:03.000 63 99963 50013 5051313 63 99963 50013 5051313 -32506 32429 4592.009900990099 463793 -126 125 -0.7821782178217822 -79 -630 2 10620 99531 1.89189 298.89189 150.39189 15039.18918 1.89189 298.8919 150.39189 15039.18937 1.89189 298.89189 150.3918900000001 15039.18900 2020-01-01 2020-01-02 2020-01-01 00:10:30 2020-01-02 03:38:51 2020-01-01 00:10:30.000 2020-01-02 03:38:51.000 630 99531 50080.5 5008050 630 99531 50080.5 5008050 -32540 32598 4860.66 486066 -127 124 1.78 178 -631 2 10621 99532 1.89489 298.89489 150.39489 15039.48948 1.89489 298.8949 150.39489 15039.48968 1.89489 298.89489 150.39488999999978 15039.48900 2020-01-01 2020-01-02 2020-01-01 00:10:31 2020-01-02 03:38:52 2020-01-01 00:10:31.000 2020-01-02 03:38:52.000 631 99532 50081.5 5008150 631 99532 50081.5 5008150 -32539 32599 4861.66 486166 -126 125 2.78 278 -632 2 10622 99533 1.89789 298.89789 150.39789 15039.78978 1.89789 298.8979 150.39789 15039.78984 1.89789 298.89789 150.39789000000013 15039.78900 2020-01-01 2020-01-02 2020-01-01 00:10:32 2020-01-02 03:38:53 2020-01-01 00:10:32.000 2020-01-02 03:38:53.000 632 99533 50082.5 5008250 632 99533 50082.5 5008250 -32538 32600 4862.66 486266 -125 126 3.78 378 -633 2 10623 99534 1.9009 298.9009 150.4009 15040.09009 1.9009 298.9009 150.40089 15040.08996 1.90090 298.90090 150.40090000000004 15040.09000 2020-01-01 2020-01-02 2020-01-01 00:10:33 2020-01-02 03:38:54 2020-01-01 00:10:33.000 2020-01-02 03:38:54.000 633 99534 50083.5 5008350 633 99534 50083.5 5008350 -32537 32601 4863.66 486366 -124 127 4.78 478 -634 2 10624 99535 1.9039 298.9039 150.4039 15040.39039 1.9039 298.9039 150.4039 15040.39009 1.90390 298.90390 150.4038999999999 15040.39000 2020-01-01 2020-01-02 2020-01-01 00:10:34 2020-01-02 03:38:55 2020-01-01 00:10:34.000 2020-01-02 03:38:55.000 634 99535 50084.5 5008450 634 99535 50084.5 5008450 -32536 32602 4864.66 486466 -128 127 3.22 322 -635 2 10625 99536 1.9069 298.9069 150.4069 15040.69069 1.9069 298.90692 150.4069 15040.69084 1.90690 298.90690 150.40689999999998 15040.69000 2020-01-01 2020-01-02 2020-01-01 00:10:35 2020-01-02 03:38:56 2020-01-01 00:10:35.000 2020-01-02 03:38:56.000 635 99536 50085.5 5008550 635 99536 50085.5 5008550 -32535 32603 4865.66 486566 -128 127 1.66 166 -636 2 10626 99537 1.9099 298.9099 150.4099 15040.99099 1.90991 298.9099 150.40991 15040.99115 1.90990 298.90990 150.40990000000005 15040.99000 2020-01-01 2020-01-02 2020-01-01 00:10:36 2020-01-02 03:38:57 2020-01-01 00:10:36.000 2020-01-02 03:38:57.000 636 99537 50086.5 5008650 636 99537 50086.5 5008650 -32534 32604 4866.66 486666 -128 124 0.1 10 -637 2 10627 99538 1.91291 298.91291 150.41291 15041.29129 1.91291 298.9129 150.41291 15041.29131 1.91291 298.91291 150.4129099999998 15041.29100 2020-01-01 2020-01-02 2020-01-01 00:10:37 2020-01-02 03:38:58 2020-01-01 00:10:37.000 2020-01-02 03:38:58.000 637 99538 50087.5 5008750 637 99538 50087.5 5008750 -32533 32605 4867.66 486766 -127 125 1.1 110 -638 2 10628 99539 1.91591 298.91591 150.41591 15041.59159 1.91591 298.91592 150.41591 15041.59143 1.91591 298.91591 150.41590999999988 15041.59100 2020-01-01 2020-01-02 2020-01-01 00:10:38 2020-01-02 03:38:59 2020-01-01 00:10:38.000 2020-01-02 03:38:59.000 638 99539 50088.5 5008850 638 99539 50088.5 5008850 -32532 32606 4868.66 486866 -126 126 2.1 210 -639 2 10629 99540 1.91891 298.91891 150.41891 15041.89189 1.91891 298.9189 150.41891 15041.89172 1.91891 298.91891 150.41891000000015 15041.89100 2020-01-01 2020-01-02 2020-01-01 00:10:39 2020-01-02 03:39:00 2020-01-01 00:10:39.000 2020-01-02 03:39:00.000 639 99540 50089.5 5008950 639 99540 50089.5 5008950 -32531 32607 4869.66 486966 -125 127 3.1 310 -64 2 10054 99964 0.19219 300.19219 150.19219 15169.41141 0.19219 300.1922 150.19219 15169.41184 0.19219 300.19219 150.1921899999999 15169.41119 2020-01-01 2020-01-02 2020-01-01 00:01:04 2020-01-02 03:46:04 2020-01-01 00:01:04.000 2020-01-02 03:46:04.000 64 99964 50014 5051414 64 99964 50014 5051414 -32505 32430 4593.009900990099 463894 -125 126 0.21782178217821782 22 -640 2 10630 99541 1.92192 298.92192 150.42192 15042.19219 1.92192 298.92194 150.42192 15042.19232 1.92192 298.92192 150.42192000000006 15042.19200 2020-01-01 2020-01-02 2020-01-01 00:10:40 2020-01-02 03:39:01 2020-01-01 00:10:40.000 2020-01-02 03:39:01.000 640 99541 50090.5 5009050 640 99541 50090.5 5009050 -32530 32608 4870.66 487066 -128 127 1.54 154 -641 2 10631 99542 1.92492 298.92492 150.42492 15042.49249 1.92492 298.92493 150.42492 15042.49265 1.92492 298.92492 150.42491999999996 15042.49200 2020-01-01 2020-01-02 2020-01-01 00:10:41 2020-01-02 03:39:02 2020-01-01 00:10:41.000 2020-01-02 03:39:02.000 641 99542 50091.5 5009150 641 99542 50091.5 5009150 -32529 32609 4871.66 487166 -128 127 -0.02 -2 -642 2 10632 99543 1.92792 298.92792 150.42792 15042.79279 1.92792 298.92792 150.42792 15042.79278 1.92792 298.92792 150.42792000000006 15042.79200 2020-01-01 2020-01-02 2020-01-01 00:10:42 2020-01-02 03:39:03 2020-01-01 00:10:42.000 2020-01-02 03:39:03.000 642 99543 50092.5 5009250 642 99543 50092.5 5009250 -32528 32610 4872.66 487266 -128 123 -1.58 -158 -643 2 10633 99544 1.93093 298.93093 150.43093 15043.09309 1.93093 298.93094 150.43092 15043.0929 1.93093 298.93093 150.43093000000022 15043.09300 2020-01-01 2020-01-02 2020-01-01 00:10:43 2020-01-02 03:39:04 2020-01-01 00:10:43.000 2020-01-02 03:39:04.000 643 99544 50093.5 5009350 643 99544 50093.5 5009350 -32527 32611 4873.66 487366 -127 124 -0.58 -58 -644 2 10634 99545 1.93393 298.93393 150.43393 15043.39339 1.93393 298.93393 150.43393 15043.39319 1.93393 298.93393 150.4339299999999 15043.39300 2020-01-01 2020-01-02 2020-01-01 00:10:44 2020-01-02 03:39:05 2020-01-01 00:10:44.000 2020-01-02 03:39:05.000 644 99545 50094.5 5009450 644 99545 50094.5 5009450 -32526 32612 4874.66 487466 -126 125 0.42 42 -645 2 10635 99546 1.93693 298.93693 150.43693 15043.69369 1.93693 298.93695 150.43693 15043.69379 1.93693 298.93693 150.4369299999999 15043.69300 2020-01-01 2020-01-02 2020-01-01 00:10:45 2020-01-02 03:39:06 2020-01-01 00:10:45.000 2020-01-02 03:39:06.000 645 99546 50095.5 5009550 645 99546 50095.5 5009550 -32525 32613 4875.66 487566 -125 126 1.42 142 -646 2 10636 99547 1.93993 298.93993 150.43993 15043.99399 1.93994 298.93994 150.43994 15043.99412 1.93993 298.93993 150.4399300000002 15043.99300 2020-01-01 2020-01-02 2020-01-01 00:10:46 2020-01-02 03:39:07 2020-01-01 00:10:46.000 2020-01-02 03:39:07.000 646 99547 50096.5 5009650 646 99547 50096.5 5009650 -32524 32614 4876.66 487666 -124 127 2.42 242 -647 2 10637 99548 1.94294 298.94294 150.44294 15044.29429 1.94294 298.94293 150.44294 15044.29425 1.94294 298.94294 150.44294000000014 15044.29400 2020-01-01 2020-01-02 2020-01-01 00:10:47 2020-01-02 03:39:08 2020-01-01 00:10:47.000 2020-01-02 03:39:08.000 647 99548 50097.5 5009750 647 99548 50097.5 5009750 -32523 32615 4877.66 487766 -128 127 0.86 86 -648 2 10638 99549 1.94594 298.94594 150.44594 15044.59459 1.94594 298.94595 150.44595 15044.595 1.94594 298.94594 150.44593999999978 15044.59400 2020-01-01 2020-01-02 2020-01-01 00:10:48 2020-01-02 03:39:09 2020-01-01 00:10:48.000 2020-01-02 03:39:09.000 648 99549 50098.5 5009850 648 99549 50098.5 5009850 -32522 32616 4878.66 487866 -128 123 -0.7 -70 -649 2 10639 99550 1.94894 298.94894 150.44894 15044.89489 1.94894 298.94894 150.44894 15044.89467 1.94894 298.94894 150.44894000000016 15044.89400 2020-01-01 2020-01-02 2020-01-01 00:10:49 2020-01-02 03:39:10 2020-01-01 00:10:49.000 2020-01-02 03:39:10.000 649 99550 50099.5 5009950 649 99550 50099.5 5009950 -32521 32617 4879.66 487966 -127 124 0.3 30 -65 2 10055 99965 0.19519 300.19519 150.19519 15169.71471 0.19519 300.1952 150.19519 15169.71448 0.19519 300.19519 150.1951900000002 15169.71419 2020-01-01 2020-01-02 2020-01-01 00:01:05 2020-01-02 03:46:05 2020-01-01 00:01:05.000 2020-01-02 03:46:05.000 65 99965 50015 5051515 65 99965 50015 5051515 -32504 32431 4594.009900990099 463995 -124 127 1.2178217821782178 123 -650 2 10640 99551 1.95195 298.95195 150.45195 15045.19519 1.95195 298.95197 150.45195 15045.19525 1.95195 298.95195 150.4519500000001 15045.19500 2020-01-01 2020-01-02 2020-01-01 00:10:50 2020-01-02 03:39:11 2020-01-01 00:10:50.000 2020-01-02 03:39:11.000 650 99551 50100.5 5010050 650 99551 50100.5 5010050 -32520 32618 4880.66 488066 -126 125 1.3 130 -651 2 10641 99552 1.95495 298.95495 150.45495 15045.49549 1.95495 298.95496 150.45495 15045.49558 1.95495 298.95495 150.4549499999999 15045.49500 2020-01-01 2020-01-02 2020-01-01 00:10:51 2020-01-02 03:39:12 2020-01-01 00:10:51.000 2020-01-02 03:39:12.000 651 99552 50101.5 5010150 651 99552 50101.5 5010150 -32519 32619 4881.66 488166 -125 126 2.3 230 +610 2 10600 99511 1.83183 298.83183 150.33183 15033.18318 1.83183 298.83182 150.33183 15033.18304 1.83183 298.83183 150.33183 15033.18300 2020-01-01 2020-01-02 2020-01-01 00:10:10 2020-01-02 03:38:31 2020-01-01 00:10:10.000 2020-01-02 03:38:31.000 610 99511 50060.5 5006050 610 99511 50060.5 5006050 -32560 32578 4840.66 484066 -128 127 -0.3 -30 +611 2 10601 99512 1.83483 298.83483 150.33483 15033.48348 1.83483 298.83484 150.33483 15033.48363 1.83483 298.83483 150.33483 15033.48300 2020-01-01 2020-01-02 2020-01-01 00:10:11 2020-01-02 03:38:32 2020-01-01 00:10:11.000 2020-01-02 03:38:32.000 611 99512 50061.5 5006150 611 99512 50061.5 5006150 -32559 32579 4841.66 484166 -128 123 -1.86 -186 +612 2 10602 99513 1.83783 298.83783 150.33783 15033.78378 1.83783 298.83783 150.33783 15033.78393 1.83783 298.83783 150.33783 15033.78300 2020-01-01 2020-01-02 2020-01-01 00:10:12 2020-01-02 03:38:33 2020-01-01 00:10:12.000 2020-01-02 03:38:33.000 612 99513 50062.5 5006250 612 99513 50062.5 5006250 -32558 32580 4842.66 484266 -127 124 -0.86 -86 +613 2 10603 99514 1.84084 298.84084 150.34084 15034.08408 1.84084 298.84085 150.34084 15034.08405 1.84084 298.84084 150.34084000000001 15034.08400 2020-01-01 2020-01-02 2020-01-01 00:10:13 2020-01-02 03:38:34 2020-01-01 00:10:13.000 2020-01-02 03:38:34.000 613 99514 50063.5 5006350 613 99514 50063.5 5006350 -32557 32581 4843.66 484366 -126 125 0.14 14 +614 2 10604 99515 1.84384 298.84384 150.34384 15034.38438 1.84384 298.84384 150.34384 15034.38421 1.84384 298.84384 150.34384 15034.38400 2020-01-01 2020-01-02 2020-01-01 00:10:14 2020-01-02 03:38:35 2020-01-01 00:10:14.000 2020-01-02 03:38:35.000 614 99515 50064.5 5006450 614 99515 50064.5 5006450 -32556 32582 4844.66 484466 -125 126 1.14 114 +615 2 10605 99516 1.84684 298.84684 150.34684 15034.68468 1.84684 298.84683 150.34684 15034.68452 1.84684 298.84684 150.34684 15034.68400 2020-01-01 2020-01-02 2020-01-01 00:10:15 2020-01-02 03:38:36 2020-01-01 00:10:15.000 2020-01-02 03:38:36.000 615 99516 50065.5 5006550 615 99516 50065.5 5006550 -32555 32583 4845.66 484566 -124 127 2.14 214 +616 2 10606 99517 1.84984 298.84984 150.34984 15034.98498 1.84984 298.84985 150.34985 15034.98527 1.84984 298.84984 150.34984 15034.98400 2020-01-01 2020-01-02 2020-01-01 00:10:16 2020-01-02 03:38:37 2020-01-01 00:10:16.000 2020-01-02 03:38:37.000 616 99517 50066.5 5006650 616 99517 50066.5 5006650 -32554 32584 4846.66 484666 -128 127 0.58 58 +617 2 10607 99518 1.85285 298.85285 150.35285 15035.28528 1.85285 298.85284 150.35285 15035.2854 1.85285 298.85285 150.35285 15035.28500 2020-01-01 2020-01-02 2020-01-01 00:10:17 2020-01-02 03:38:38 2020-01-01 00:10:17.000 2020-01-02 03:38:38.000 617 99518 50067.5 5006750 617 99518 50067.5 5006750 -32553 32585 4847.66 484766 -128 123 -0.98 -98 +618 2 10608 99519 1.85585 298.85585 150.35585 15035.58558 1.85585 298.85587 150.35585 15035.58551 1.85585 298.85585 150.35585 15035.58500 2020-01-01 2020-01-02 2020-01-01 00:10:18 2020-01-02 03:38:39 2020-01-01 00:10:18.000 2020-01-02 03:38:39.000 618 99519 50068.5 5006850 618 99519 50068.5 5006850 -32552 32586 4848.66 484866 -127 124 0.02 2 +619 2 10609 99520 1.85885 298.85885 150.35885 15035.88588 1.85885 298.85886 150.35885 15035.88568 1.85885 298.85885 150.35885 15035.88500 2020-01-01 2020-01-02 2020-01-01 00:10:19 2020-01-02 03:38:40 2020-01-01 00:10:19.000 2020-01-02 03:38:40.000 619 99520 50069.5 5006950 619 99520 50069.5 5006950 -32551 32587 4849.66 484966 -126 125 1.02 102 +62 2 10052 99962 0.18618 300.18618 150.18618 15168.8048 0.18618 300.1862 150.18618 15168.80494 0.18618 300.18618 150.18617999999998 15168.80418 2020-01-01 2020-01-02 2020-01-01 00:01:02 2020-01-02 03:46:02 2020-01-01 00:01:02.000 2020-01-02 03:46:02.000 62 99962 50012 5051212 62 99962 50012 5051212 -32507 32428 4591.009900990099 463692 -127 124 -1.7821782178217822 -180 +620 2 10610 99521 1.86186 298.86186 150.36186 15036.18618 1.86186 298.86185 150.36185 15036.18598 1.86186 298.86186 150.36186 15036.18600 2020-01-01 2020-01-02 2020-01-01 00:10:20 2020-01-02 03:38:41 2020-01-01 00:10:20.000 2020-01-02 03:38:41.000 620 99521 50070.5 5007050 620 99521 50070.5 5007050 -32550 32588 4850.66 485066 -125 126 2.02 202 +621 2 10611 99522 1.86486 298.86486 150.36486 15036.48648 1.86486 298.86487 150.36486 15036.48673 1.86486 298.86486 150.36486000000002 15036.48600 2020-01-01 2020-01-02 2020-01-01 00:10:21 2020-01-02 03:38:42 2020-01-01 00:10:21.000 2020-01-02 03:38:42.000 621 99522 50071.5 5007150 621 99522 50071.5 5007150 -32549 32589 4851.66 485166 -124 127 3.02 302 +622 2 10612 99523 1.86786 298.86786 150.36786 15036.78678 1.86786 298.86786 150.36786 15036.78687 1.86786 298.86786 150.36786 15036.78600 2020-01-01 2020-01-02 2020-01-01 00:10:22 2020-01-02 03:38:43 2020-01-01 00:10:22.000 2020-01-02 03:38:43.000 622 99523 50072.5 5007250 622 99523 50072.5 5007250 -32548 32590 4852.66 485266 -128 127 1.46 146 +623 2 10613 99524 1.87087 298.87087 150.37087 15037.08708 1.87087 298.87088 150.37087 15037.08702 1.87087 298.87087 150.37087 15037.08700 2020-01-01 2020-01-02 2020-01-01 00:10:23 2020-01-02 03:38:44 2020-01-01 00:10:23.000 2020-01-02 03:38:44.000 623 99524 50073.5 5007350 623 99524 50073.5 5007350 -32547 32591 4853.66 485366 -128 123 -0.1 -10 +624 2 10614 99525 1.87387 298.87387 150.37387 15037.38738 1.87387 298.87387 150.37387 15037.38716 1.87387 298.87387 150.37387 15037.38700 2020-01-01 2020-01-02 2020-01-01 00:10:24 2020-01-02 03:38:45 2020-01-01 00:10:24.000 2020-01-02 03:38:45.000 624 99525 50074.5 5007450 624 99525 50074.5 5007450 -32546 32592 4854.66 485466 -127 124 0.9 90 +625 2 10615 99526 1.87687 298.87687 150.37687 15037.68768 1.87687 298.8769 150.37687 15037.68791 1.87687 298.87687 150.37687 15037.68700 2020-01-01 2020-01-02 2020-01-01 00:10:25 2020-01-02 03:38:46 2020-01-01 00:10:25.000 2020-01-02 03:38:46.000 625 99526 50075.5 5007550 625 99526 50075.5 5007550 -32545 32593 4855.66 485566 -126 125 1.9 190 +626 2 10616 99527 1.87987 298.87987 150.37987 15037.98798 1.87987 298.87988 150.37988 15037.9882 1.87987 298.87987 150.37986999999998 15037.98700 2020-01-01 2020-01-02 2020-01-01 00:10:26 2020-01-02 03:38:47 2020-01-01 00:10:26.000 2020-01-02 03:38:47.000 626 99527 50076.5 5007650 626 99527 50076.5 5007650 -32544 32594 4856.66 485666 -125 126 2.9 290 +627 2 10617 99528 1.88288 298.88288 150.38288 15038.28828 1.88288 298.88287 150.38288 15038.28834 1.88288 298.88288 150.38288 15038.28800 2020-01-01 2020-01-02 2020-01-01 00:10:27 2020-01-02 03:38:48 2020-01-01 00:10:27.000 2020-01-02 03:38:48.000 627 99528 50077.5 5007750 627 99528 50077.5 5007750 -32543 32595 4857.66 485766 -124 127 3.9 390 +628 2 10618 99529 1.88588 298.88588 150.38588 15038.58858 1.88588 298.8859 150.38588 15038.58849 1.88588 298.88588 150.38588 15038.58800 2020-01-01 2020-01-02 2020-01-01 00:10:28 2020-01-02 03:38:49 2020-01-01 00:10:28.000 2020-01-02 03:38:49.000 628 99529 50078.5 5007850 628 99529 50078.5 5007850 -32542 32596 4858.66 485866 -128 127 2.34 234 +629 2 10619 99530 1.88888 298.88888 150.38888 15038.88888 1.88888 298.8889 150.38888 15038.88862 1.88888 298.88888 150.38888 15038.88800 2020-01-01 2020-01-02 2020-01-01 00:10:29 2020-01-02 03:38:50 2020-01-01 00:10:29.000 2020-01-02 03:38:50.000 629 99530 50079.5 5007950 629 99530 50079.5 5007950 -32541 32597 4859.66 485966 -128 123 0.78 78 +63 2 10053 99963 0.18918 300.18918 150.18918 15169.1081 0.18918 300.18918 150.18918 15169.10808 0.18918 300.18918 150.18918000000002 15169.10718 2020-01-01 2020-01-02 2020-01-01 00:01:03 2020-01-02 03:46:03 2020-01-01 00:01:03.000 2020-01-02 03:46:03.000 63 99963 50013 5051313 63 99963 50013 5051313 -32506 32429 4592.009900990099 463793 -126 125 -0.7821782178217822 -79 +630 2 10620 99531 1.89189 298.89189 150.39189 15039.18918 1.89189 298.8919 150.39189 15039.18937 1.89189 298.89189 150.39189 15039.18900 2020-01-01 2020-01-02 2020-01-01 00:10:30 2020-01-02 03:38:51 2020-01-01 00:10:30.000 2020-01-02 03:38:51.000 630 99531 50080.5 5008050 630 99531 50080.5 5008050 -32540 32598 4860.66 486066 -127 124 1.78 178 +631 2 10621 99532 1.89489 298.89489 150.39489 15039.48948 1.89489 298.8949 150.39489 15039.48968 1.89489 298.89489 150.39489 15039.48900 2020-01-01 2020-01-02 2020-01-01 00:10:31 2020-01-02 03:38:52 2020-01-01 00:10:31.000 2020-01-02 03:38:52.000 631 99532 50081.5 5008150 631 99532 50081.5 5008150 -32539 32599 4861.66 486166 -126 125 2.78 278 +632 2 10622 99533 1.89789 298.89789 150.39789 15039.78978 1.89789 298.8979 150.39789 15039.78984 1.89789 298.89789 150.39789000000002 15039.78900 2020-01-01 2020-01-02 2020-01-01 00:10:32 2020-01-02 03:38:53 2020-01-01 00:10:32.000 2020-01-02 03:38:53.000 632 99533 50082.5 5008250 632 99533 50082.5 5008250 -32538 32600 4862.66 486266 -125 126 3.78 378 +633 2 10623 99534 1.9009 298.9009 150.4009 15040.09009 1.9009 298.9009 150.40089 15040.08996 1.90090 298.90090 150.4009 15040.09000 2020-01-01 2020-01-02 2020-01-01 00:10:33 2020-01-02 03:38:54 2020-01-01 00:10:33.000 2020-01-02 03:38:54.000 633 99534 50083.5 5008350 633 99534 50083.5 5008350 -32537 32601 4863.66 486366 -124 127 4.78 478 +634 2 10624 99535 1.9039 298.9039 150.4039 15040.39039 1.9039 298.9039 150.4039 15040.39009 1.90390 298.90390 150.4039 15040.39000 2020-01-01 2020-01-02 2020-01-01 00:10:34 2020-01-02 03:38:55 2020-01-01 00:10:34.000 2020-01-02 03:38:55.000 634 99535 50084.5 5008450 634 99535 50084.5 5008450 -32536 32602 4864.66 486466 -128 127 3.22 322 +635 2 10625 99536 1.9069 298.9069 150.4069 15040.69069 1.9069 298.90692 150.4069 15040.69084 1.90690 298.90690 150.4069 15040.69000 2020-01-01 2020-01-02 2020-01-01 00:10:35 2020-01-02 03:38:56 2020-01-01 00:10:35.000 2020-01-02 03:38:56.000 635 99536 50085.5 5008550 635 99536 50085.5 5008550 -32535 32603 4865.66 486566 -128 127 1.66 166 +636 2 10626 99537 1.9099 298.9099 150.4099 15040.99099 1.90991 298.9099 150.40991 15040.99115 1.90990 298.90990 150.4099 15040.99000 2020-01-01 2020-01-02 2020-01-01 00:10:36 2020-01-02 03:38:57 2020-01-01 00:10:36.000 2020-01-02 03:38:57.000 636 99537 50086.5 5008650 636 99537 50086.5 5008650 -32534 32604 4866.66 486666 -128 124 0.1 10 +637 2 10627 99538 1.91291 298.91291 150.41291 15041.29129 1.91291 298.9129 150.41291 15041.29131 1.91291 298.91291 150.41290999999998 15041.29100 2020-01-01 2020-01-02 2020-01-01 00:10:37 2020-01-02 03:38:58 2020-01-01 00:10:37.000 2020-01-02 03:38:58.000 637 99538 50087.5 5008750 637 99538 50087.5 5008750 -32533 32605 4867.66 486766 -127 125 1.1 110 +638 2 10628 99539 1.91591 298.91591 150.41591 15041.59159 1.91591 298.91592 150.41591 15041.59143 1.91591 298.91591 150.41591 15041.59100 2020-01-01 2020-01-02 2020-01-01 00:10:38 2020-01-02 03:38:59 2020-01-01 00:10:38.000 2020-01-02 03:38:59.000 638 99539 50088.5 5008850 638 99539 50088.5 5008850 -32532 32606 4868.66 486866 -126 126 2.1 210 +639 2 10629 99540 1.91891 298.91891 150.41891 15041.89189 1.91891 298.9189 150.41891 15041.89172 1.91891 298.91891 150.41890999999998 15041.89100 2020-01-01 2020-01-02 2020-01-01 00:10:39 2020-01-02 03:39:00 2020-01-01 00:10:39.000 2020-01-02 03:39:00.000 639 99540 50089.5 5008950 639 99540 50089.5 5008950 -32531 32607 4869.66 486966 -125 127 3.1 310 +64 2 10054 99964 0.19219 300.19219 150.19219 15169.41141 0.19219 300.1922 150.19219 15169.41184 0.19219 300.19219 150.19219 15169.41119 2020-01-01 2020-01-02 2020-01-01 00:01:04 2020-01-02 03:46:04 2020-01-01 00:01:04.000 2020-01-02 03:46:04.000 64 99964 50014 5051414 64 99964 50014 5051414 -32505 32430 4593.009900990099 463894 -125 126 0.21782178217821782 22 +640 2 10630 99541 1.92192 298.92192 150.42192 15042.19219 1.92192 298.92194 150.42192 15042.19232 1.92192 298.92192 150.42192 15042.19200 2020-01-01 2020-01-02 2020-01-01 00:10:40 2020-01-02 03:39:01 2020-01-01 00:10:40.000 2020-01-02 03:39:01.000 640 99541 50090.5 5009050 640 99541 50090.5 5009050 -32530 32608 4870.66 487066 -128 127 1.54 154 +641 2 10631 99542 1.92492 298.92492 150.42492 15042.49249 1.92492 298.92493 150.42492 15042.49265 1.92492 298.92492 150.42492000000001 15042.49200 2020-01-01 2020-01-02 2020-01-01 00:10:41 2020-01-02 03:39:02 2020-01-01 00:10:41.000 2020-01-02 03:39:02.000 641 99542 50091.5 5009150 641 99542 50091.5 5009150 -32529 32609 4871.66 487166 -128 127 -0.02 -2 +642 2 10632 99543 1.92792 298.92792 150.42792 15042.79279 1.92792 298.92792 150.42792 15042.79278 1.92792 298.92792 150.42792 15042.79200 2020-01-01 2020-01-02 2020-01-01 00:10:42 2020-01-02 03:39:03 2020-01-01 00:10:42.000 2020-01-02 03:39:03.000 642 99543 50092.5 5009250 642 99543 50092.5 5009250 -32528 32610 4872.66 487266 -128 123 -1.58 -158 +643 2 10633 99544 1.93093 298.93093 150.43093 15043.09309 1.93093 298.93094 150.43092 15043.0929 1.93093 298.93093 150.43093000000002 15043.09300 2020-01-01 2020-01-02 2020-01-01 00:10:43 2020-01-02 03:39:04 2020-01-01 00:10:43.000 2020-01-02 03:39:04.000 643 99544 50093.5 5009350 643 99544 50093.5 5009350 -32527 32611 4873.66 487366 -127 124 -0.58 -58 +644 2 10634 99545 1.93393 298.93393 150.43393 15043.39339 1.93393 298.93393 150.43393 15043.39319 1.93393 298.93393 150.43393 15043.39300 2020-01-01 2020-01-02 2020-01-01 00:10:44 2020-01-02 03:39:05 2020-01-01 00:10:44.000 2020-01-02 03:39:05.000 644 99545 50094.5 5009450 644 99545 50094.5 5009450 -32526 32612 4874.66 487466 -126 125 0.42 42 +645 2 10635 99546 1.93693 298.93693 150.43693 15043.69369 1.93693 298.93695 150.43693 15043.69379 1.93693 298.93693 150.43693 15043.69300 2020-01-01 2020-01-02 2020-01-01 00:10:45 2020-01-02 03:39:06 2020-01-01 00:10:45.000 2020-01-02 03:39:06.000 645 99546 50095.5 5009550 645 99546 50095.5 5009550 -32525 32613 4875.66 487566 -125 126 1.42 142 +646 2 10636 99547 1.93993 298.93993 150.43993 15043.99399 1.93994 298.93994 150.43994 15043.99412 1.93993 298.93993 150.43993 15043.99300 2020-01-01 2020-01-02 2020-01-01 00:10:46 2020-01-02 03:39:07 2020-01-01 00:10:46.000 2020-01-02 03:39:07.000 646 99547 50096.5 5009650 646 99547 50096.5 5009650 -32524 32614 4876.66 487666 -124 127 2.42 242 +647 2 10637 99548 1.94294 298.94294 150.44294 15044.29429 1.94294 298.94293 150.44294 15044.29425 1.94294 298.94294 150.44294 15044.29400 2020-01-01 2020-01-02 2020-01-01 00:10:47 2020-01-02 03:39:08 2020-01-01 00:10:47.000 2020-01-02 03:39:08.000 647 99548 50097.5 5009750 647 99548 50097.5 5009750 -32523 32615 4877.66 487766 -128 127 0.86 86 +648 2 10638 99549 1.94594 298.94594 150.44594 15044.59459 1.94594 298.94595 150.44595 15044.595 1.94594 298.94594 150.44593999999998 15044.59400 2020-01-01 2020-01-02 2020-01-01 00:10:48 2020-01-02 03:39:09 2020-01-01 00:10:48.000 2020-01-02 03:39:09.000 648 99549 50098.5 5009850 648 99549 50098.5 5009850 -32522 32616 4878.66 487866 -128 123 -0.7 -70 +649 2 10639 99550 1.94894 298.94894 150.44894 15044.89489 1.94894 298.94894 150.44894 15044.89467 1.94894 298.94894 150.44894 15044.89400 2020-01-01 2020-01-02 2020-01-01 00:10:49 2020-01-02 03:39:10 2020-01-01 00:10:49.000 2020-01-02 03:39:10.000 649 99550 50099.5 5009950 649 99550 50099.5 5009950 -32521 32617 4879.66 487966 -127 124 0.3 30 +65 2 10055 99965 0.19519 300.19519 150.19519 15169.71471 0.19519 300.1952 150.19519 15169.71448 0.19519 300.19519 150.19519 15169.71419 2020-01-01 2020-01-02 2020-01-01 00:01:05 2020-01-02 03:46:05 2020-01-01 00:01:05.000 2020-01-02 03:46:05.000 65 99965 50015 5051515 65 99965 50015 5051515 -32504 32431 4594.009900990099 463995 -124 127 1.2178217821782178 123 +650 2 10640 99551 1.95195 298.95195 150.45195 15045.19519 1.95195 298.95197 150.45195 15045.19525 1.95195 298.95195 150.45195 15045.19500 2020-01-01 2020-01-02 2020-01-01 00:10:50 2020-01-02 03:39:11 2020-01-01 00:10:50.000 2020-01-02 03:39:11.000 650 99551 50100.5 5010050 650 99551 50100.5 5010050 -32520 32618 4880.66 488066 -126 125 1.3 130 +651 2 10641 99552 1.95495 298.95495 150.45495 15045.49549 1.95495 298.95496 150.45495 15045.49558 1.95495 298.95495 150.45495 15045.49500 2020-01-01 2020-01-02 2020-01-01 00:10:51 2020-01-02 03:39:12 2020-01-01 00:10:51.000 2020-01-02 03:39:12.000 651 99552 50101.5 5010150 651 99552 50101.5 5010150 -32519 32619 4881.66 488166 -125 126 2.3 230 652 2 10642 99553 1.95795 298.95795 150.45795 15045.79579 1.95795 298.95795 150.45795 15045.79572 1.95795 298.95795 150.45795 15045.79500 2020-01-01 2020-01-02 2020-01-01 00:10:52 2020-01-02 03:39:13 2020-01-01 00:10:52.000 2020-01-02 03:39:13.000 652 99553 50102.5 5010250 652 99553 50102.5 5010250 -32518 32620 4882.66 488266 -124 127 3.3 330 -653 2 10643 99554 1.96096 298.96096 150.46096 15046.09609 1.96096 298.96097 150.46096 15046.09647 1.96096 298.96096 150.46096000000014 15046.09600 2020-01-01 2020-01-02 2020-01-01 00:10:53 2020-01-02 03:39:14 2020-01-01 00:10:53.000 2020-01-02 03:39:14.000 653 99554 50103.5 5010350 653 99554 50103.5 5010350 -32517 32621 4883.66 488366 -128 127 1.74 174 -654 2 10644 99555 1.96396 298.96396 150.46396 15046.39639 1.96396 298.96396 150.46396 15046.39613 1.96396 298.96396 150.46395999999982 15046.39600 2020-01-01 2020-01-02 2020-01-01 00:10:54 2020-01-02 03:39:15 2020-01-01 00:10:54.000 2020-01-02 03:39:15.000 654 99555 50104.5 5010450 654 99555 50104.5 5010450 -32516 32622 4884.66 488466 -128 123 0.18 18 -655 2 10645 99556 1.96696 298.96696 150.46696 15046.69669 1.96696 298.96698 150.46696 15046.69676 1.96696 298.96696 150.46695999999986 15046.69600 2020-01-01 2020-01-02 2020-01-01 00:10:55 2020-01-02 03:39:16 2020-01-01 00:10:55.000 2020-01-02 03:39:16.000 655 99556 50105.5 5010550 655 99556 50105.5 5010550 -32515 32623 4885.66 488566 -127 124 1.18 118 -656 2 10646 99557 1.96996 298.96996 150.46996 15046.99699 1.96997 298.96997 150.46997 15046.99706 1.96996 298.96996 150.46996000000016 15046.99600 2020-01-01 2020-01-02 2020-01-01 00:10:56 2020-01-02 03:39:17 2020-01-01 00:10:56.000 2020-01-02 03:39:17.000 656 99557 50106.5 5010650 656 99557 50106.5 5010650 -32514 32624 4886.66 488666 -126 125 2.18 218 -657 2 10647 99558 1.97297 298.97297 150.47297 15047.29729 1.97297 298.97296 150.47297 15047.29735 1.97297 298.97297 150.47297000000006 15047.29700 2020-01-01 2020-01-02 2020-01-01 00:10:57 2020-01-02 03:39:18 2020-01-01 00:10:57.000 2020-01-02 03:39:18.000 657 99558 50107.5 5010750 657 99558 50107.5 5010750 -32513 32625 4887.66 488766 -125 126 3.18 318 +653 2 10643 99554 1.96096 298.96096 150.46096 15046.09609 1.96096 298.96097 150.46096 15046.09647 1.96096 298.96096 150.46096 15046.09600 2020-01-01 2020-01-02 2020-01-01 00:10:53 2020-01-02 03:39:14 2020-01-01 00:10:53.000 2020-01-02 03:39:14.000 653 99554 50103.5 5010350 653 99554 50103.5 5010350 -32517 32621 4883.66 488366 -128 127 1.74 174 +654 2 10644 99555 1.96396 298.96396 150.46396 15046.39639 1.96396 298.96396 150.46396 15046.39613 1.96396 298.96396 150.46396000000001 15046.39600 2020-01-01 2020-01-02 2020-01-01 00:10:54 2020-01-02 03:39:15 2020-01-01 00:10:54.000 2020-01-02 03:39:15.000 654 99555 50104.5 5010450 654 99555 50104.5 5010450 -32516 32622 4884.66 488466 -128 123 0.18 18 +655 2 10645 99556 1.96696 298.96696 150.46696 15046.69669 1.96696 298.96698 150.46696 15046.69676 1.96696 298.96696 150.46696 15046.69600 2020-01-01 2020-01-02 2020-01-01 00:10:55 2020-01-02 03:39:16 2020-01-01 00:10:55.000 2020-01-02 03:39:16.000 655 99556 50105.5 5010550 655 99556 50105.5 5010550 -32515 32623 4885.66 488566 -127 124 1.18 118 +656 2 10646 99557 1.96996 298.96996 150.46996 15046.99699 1.96997 298.96997 150.46997 15046.99706 1.96996 298.96996 150.46996 15046.99600 2020-01-01 2020-01-02 2020-01-01 00:10:56 2020-01-02 03:39:17 2020-01-01 00:10:56.000 2020-01-02 03:39:17.000 656 99557 50106.5 5010650 656 99557 50106.5 5010650 -32514 32624 4886.66 488666 -126 125 2.18 218 +657 2 10647 99558 1.97297 298.97297 150.47297 15047.29729 1.97297 298.97296 150.47297 15047.29735 1.97297 298.97297 150.47297 15047.29700 2020-01-01 2020-01-02 2020-01-01 00:10:57 2020-01-02 03:39:18 2020-01-01 00:10:57.000 2020-01-02 03:39:18.000 657 99558 50107.5 5010750 657 99558 50107.5 5010750 -32513 32625 4887.66 488766 -125 126 3.18 318 658 2 10648 99559 1.97597 298.97597 150.47597 15047.59759 1.97597 298.97598 150.47597 15047.59794 1.97597 298.97597 150.47597 15047.59700 2020-01-01 2020-01-02 2020-01-01 00:10:58 2020-01-02 03:39:19 2020-01-01 00:10:58.000 2020-01-02 03:39:19.000 658 99559 50108.5 5010850 658 99559 50108.5 5010850 -32512 32626 4888.66 488866 -124 127 4.18 418 -659 2 10649 99560 1.97897 298.97897 150.47897 15047.89789 1.97897 298.97897 150.47897 15047.8976 1.97897 298.97897 150.4789700000001 15047.89700 2020-01-01 2020-01-02 2020-01-01 00:10:59 2020-01-02 03:39:20 2020-01-01 00:10:59.000 2020-01-02 03:39:20.000 659 99560 50109.5 5010950 659 99560 50109.5 5010950 -32511 32627 4889.66 488966 -128 127 2.62 262 -66 2 10056 99966 0.19819 300.19819 150.19819 15170.01801 0.19819 300.1982 150.19819 15170.01808 0.19819 300.19819 150.19818999999987 15170.01719 2020-01-01 2020-01-02 2020-01-01 00:01:06 2020-01-02 03:46:06 2020-01-01 00:01:06.000 2020-01-02 03:46:06.000 66 99966 50016 5051616 66 99966 50016 5051616 -32503 32432 4595.009900990099 464096 -128 127 -0.31683168316831684 -32 -660 2 10650 99561 1.98198 298.98198 150.48198 15048.19819 1.98198 298.982 150.48198 15048.19822 1.98198 298.98198 150.48197999999996 15048.19800 2020-01-01 2020-01-02 2020-01-01 00:11:00 2020-01-02 03:39:21 2020-01-01 00:11:00.000 2020-01-02 03:39:21.000 660 99561 50110.5 5011050 660 99561 50110.5 5011050 -32510 32628 4890.66 489066 -128 127 1.06 106 -661 2 10651 99562 1.98498 298.98498 150.48498 15048.49849 1.98498 298.985 150.48498 15048.49853 1.98498 298.98498 150.4849799999999 15048.49800 2020-01-01 2020-01-02 2020-01-01 00:11:01 2020-01-02 03:39:22 2020-01-01 00:11:01.000 2020-01-02 03:39:22.000 661 99562 50111.5 5011150 661 99562 50111.5 5011150 -32509 32629 4891.66 489166 -128 124 -0.5 -50 -662 2 10652 99563 1.98798 298.98798 150.48798 15048.79879 1.98798 298.98798 150.48798 15048.79882 1.98798 298.98798 150.4879799999999 15048.79800 2020-01-01 2020-01-02 2020-01-01 00:11:02 2020-01-02 03:39:23 2020-01-01 00:11:02.000 2020-01-02 03:39:23.000 662 99563 50112.5 5011250 662 99563 50112.5 5011250 -32508 32630 4892.66 489266 -127 125 0.5 50 -663 2 10653 99564 1.99099 298.99099 150.49099 15049.09909 1.99099 298.991 150.49099 15049.09942 1.99099 298.99099 150.49099000000015 15049.09900 2020-01-01 2020-01-02 2020-01-01 00:11:03 2020-01-02 03:39:24 2020-01-01 00:11:03.000 2020-01-02 03:39:24.000 663 99564 50113.5 5011350 663 99564 50113.5 5011350 -32507 32631 4893.66 489366 -126 126 1.5 150 -664 2 10654 99565 1.99399 298.99399 150.49399 15049.39939 1.99399 298.994 150.49399 15049.39911 1.99399 298.99399 150.49399000000014 15049.39900 2020-01-01 2020-01-02 2020-01-01 00:11:04 2020-01-02 03:39:25 2020-01-01 00:11:04.000 2020-01-02 03:39:25.000 664 99565 50114.5 5011450 664 99565 50114.5 5011450 -32506 32632 4894.66 489466 -125 127 2.5 250 -665 2 10655 99566 1.99699 298.99699 150.49699 15049.69969 1.99699 298.997 150.49699 15049.6997 1.99699 298.99699 150.4969899999998 15049.69900 2020-01-01 2020-01-02 2020-01-01 00:11:05 2020-01-02 03:39:26 2020-01-01 00:11:05.000 2020-01-02 03:39:26.000 665 99566 50115.5 5011550 665 99566 50115.5 5011550 -32505 32633 4895.66 489566 -128 127 0.94 94 +659 2 10649 99560 1.97897 298.97897 150.47897 15047.89789 1.97897 298.97897 150.47897 15047.8976 1.97897 298.97897 150.47897 15047.89700 2020-01-01 2020-01-02 2020-01-01 00:10:59 2020-01-02 03:39:20 2020-01-01 00:10:59.000 2020-01-02 03:39:20.000 659 99560 50109.5 5010950 659 99560 50109.5 5010950 -32511 32627 4889.66 488966 -128 127 2.62 262 +66 2 10056 99966 0.19819 300.19819 150.19819 15170.01801 0.19819 300.1982 150.19819 15170.01808 0.19819 300.19819 150.19819 15170.01719 2020-01-01 2020-01-02 2020-01-01 00:01:06 2020-01-02 03:46:06 2020-01-01 00:01:06.000 2020-01-02 03:46:06.000 66 99966 50016 5051616 66 99966 50016 5051616 -32503 32432 4595.009900990099 464096 -128 127 -0.31683168316831684 -32 +660 2 10650 99561 1.98198 298.98198 150.48198 15048.19819 1.98198 298.982 150.48198 15048.19822 1.98198 298.98198 150.48198 15048.19800 2020-01-01 2020-01-02 2020-01-01 00:11:00 2020-01-02 03:39:21 2020-01-01 00:11:00.000 2020-01-02 03:39:21.000 660 99561 50110.5 5011050 660 99561 50110.5 5011050 -32510 32628 4890.66 489066 -128 127 1.06 106 +661 2 10651 99562 1.98498 298.98498 150.48498 15048.49849 1.98498 298.985 150.48498 15048.49853 1.98498 298.98498 150.48498 15048.49800 2020-01-01 2020-01-02 2020-01-01 00:11:01 2020-01-02 03:39:22 2020-01-01 00:11:01.000 2020-01-02 03:39:22.000 661 99562 50111.5 5011150 661 99562 50111.5 5011150 -32509 32629 4891.66 489166 -128 124 -0.5 -50 +662 2 10652 99563 1.98798 298.98798 150.48798 15048.79879 1.98798 298.98798 150.48798 15048.79882 1.98798 298.98798 150.48798 15048.79800 2020-01-01 2020-01-02 2020-01-01 00:11:02 2020-01-02 03:39:23 2020-01-01 00:11:02.000 2020-01-02 03:39:23.000 662 99563 50112.5 5011250 662 99563 50112.5 5011250 -32508 32630 4892.66 489266 -127 125 0.5 50 +663 2 10653 99564 1.99099 298.99099 150.49099 15049.09909 1.99099 298.991 150.49099 15049.09942 1.99099 298.99099 150.49099 15049.09900 2020-01-01 2020-01-02 2020-01-01 00:11:03 2020-01-02 03:39:24 2020-01-01 00:11:03.000 2020-01-02 03:39:24.000 663 99564 50113.5 5011350 663 99564 50113.5 5011350 -32507 32631 4893.66 489366 -126 126 1.5 150 +664 2 10654 99565 1.99399 298.99399 150.49399 15049.39939 1.99399 298.994 150.49399 15049.39911 1.99399 298.99399 150.49399 15049.39900 2020-01-01 2020-01-02 2020-01-01 00:11:04 2020-01-02 03:39:25 2020-01-01 00:11:04.000 2020-01-02 03:39:25.000 664 99565 50114.5 5011450 664 99565 50114.5 5011450 -32506 32632 4894.66 489466 -125 127 2.5 250 +665 2 10655 99566 1.99699 298.99699 150.49699 15049.69969 1.99699 298.997 150.49699 15049.6997 1.99699 298.99699 150.49699 15049.69900 2020-01-01 2020-01-02 2020-01-01 00:11:05 2020-01-02 03:39:26 2020-01-01 00:11:05.000 2020-01-02 03:39:26.000 665 99566 50115.5 5011550 665 99566 50115.5 5011550 -32505 32633 4895.66 489566 -128 127 0.94 94 666 2 10656 99567 2 299 150.5 15050 2 299 150.5 15050 2.00000 299.00000 150.5 15050.00000 2020-01-01 2020-01-02 2020-01-01 00:11:06 2020-01-02 03:39:27 2020-01-01 00:11:06.000 2020-01-02 03:39:27.000 666 99567 50116.5 5011650 666 99567 50116.5 5011650 -32504 32634 4896.66 489666 -128 127 -0.62 -62 -667 2 10657 99568 2.003 299.003 150.503 15050.3003 2.003 299.003 150.503 15050.30029 2.00300 299.00300 150.50300000000004 15050.30000 2020-01-01 2020-01-02 2020-01-01 00:11:07 2020-01-02 03:39:28 2020-01-01 00:11:07.000 2020-01-02 03:39:28.000 667 99568 50117.5 5011750 667 99568 50117.5 5011750 -32503 32635 4897.66 489766 -128 123 -2.18 -218 -668 2 10658 99569 2.006 299.006 150.506 15050.6006 2.006 299.006 150.506 15050.60089 2.00600 299.00600 150.50599999999991 15050.60000 2020-01-01 2020-01-02 2020-01-01 00:11:08 2020-01-02 03:39:29 2020-01-01 00:11:08.000 2020-01-02 03:39:29.000 668 99569 50118.5 5011850 668 99569 50118.5 5011850 -32502 32636 4898.66 489866 -127 124 -1.18 -118 +667 2 10657 99568 2.003 299.003 150.503 15050.3003 2.003 299.003 150.503 15050.30029 2.00300 299.00300 150.503 15050.30000 2020-01-01 2020-01-02 2020-01-01 00:11:07 2020-01-02 03:39:28 2020-01-01 00:11:07.000 2020-01-02 03:39:28.000 667 99568 50117.5 5011750 667 99568 50117.5 5011750 -32503 32635 4897.66 489766 -128 123 -2.18 -218 +668 2 10658 99569 2.006 299.006 150.506 15050.6006 2.006 299.006 150.506 15050.60089 2.00600 299.00600 150.506 15050.60000 2020-01-01 2020-01-02 2020-01-01 00:11:08 2020-01-02 03:39:29 2020-01-01 00:11:08.000 2020-01-02 03:39:29.000 668 99569 50118.5 5011850 668 99569 50118.5 5011850 -32502 32636 4898.66 489866 -127 124 -1.18 -118 669 2 10659 99570 2.009 299.009 150.509 15050.9009 2.009 299.009 150.509 15050.90057 2.00900 299.00900 150.509 15050.90000 2020-01-01 2020-01-02 2020-01-01 00:11:09 2020-01-02 03:39:30 2020-01-01 00:11:09.000 2020-01-02 03:39:30.000 669 99570 50119.5 5011950 669 99570 50119.5 5011950 -32501 32637 4899.66 489966 -126 125 -0.18 -18 -67 2 10057 99967 0.2012 300.2012 150.2012 15170.32132 0.2012 300.2012 150.2012 15170.32142 0.20120 300.20120 150.20120000000003 15170.32120 2020-01-01 2020-01-02 2020-01-01 00:01:07 2020-01-02 03:46:07 2020-01-01 00:01:07.000 2020-01-02 03:46:07.000 67 99967 50017 5051717 67 99967 50017 5051717 -32502 32433 4596.009900990099 464197 -128 127 -1.8514851485148516 -187 -670 2 10660 99571 2.01201 299.01201 150.51201 15051.2012 2.01201 299.01202 150.51201 15051.20117 2.01201 299.01201 150.51201000000015 15051.20100 2020-01-01 2020-01-02 2020-01-01 00:11:10 2020-01-02 03:39:31 2020-01-01 00:11:10.000 2020-01-02 03:39:31.000 670 99571 50120.5 5012050 670 99571 50120.5 5012050 -32500 32638 4900.66 490066 -125 126 0.82 82 -671 2 10661 99572 2.01501 299.01501 150.51501 15051.5015 2.01501 299.015 150.51501 15051.50146 2.01501 299.01501 150.51500999999985 15051.50100 2020-01-01 2020-01-02 2020-01-01 00:11:11 2020-01-02 03:39:32 2020-01-01 00:11:11.000 2020-01-02 03:39:32.000 671 99572 50121.5 5012150 671 99572 50121.5 5012150 -32499 32639 4901.66 490166 -124 127 1.82 182 -672 2 10662 99573 2.01801 299.01801 150.51801 15051.8018 2.01801 299.018 150.51801 15051.80176 2.01801 299.01801 150.51800999999986 15051.80100 2020-01-01 2020-01-02 2020-01-01 00:11:12 2020-01-02 03:39:33 2020-01-01 00:11:12.000 2020-01-02 03:39:33.000 672 99573 50122.5 5012250 672 99573 50122.5 5012250 -32498 32640 4902.66 490266 -128 127 0.26 26 -673 2 10663 99574 2.02102 299.02102 150.52102 15052.1021 2.02102 299.02103 150.52102 15052.1024 2.02102 299.02102 150.5210200000001 15052.10200 2020-01-01 2020-01-02 2020-01-01 00:11:13 2020-01-02 03:39:34 2020-01-01 00:11:13.000 2020-01-02 03:39:34.000 673 99574 50123.5 5012350 673 99574 50123.5 5012350 -32497 32641 4903.66 490366 -128 123 -1.3 -130 -674 2 10664 99575 2.02402 299.02402 150.52402 15052.4024 2.02402 299.02402 150.52402 15052.40204 2.02402 299.02402 150.52402000000006 15052.40200 2020-01-01 2020-01-02 2020-01-01 00:11:14 2020-01-02 03:39:35 2020-01-01 00:11:14.000 2020-01-02 03:39:35.000 674 99575 50124.5 5012450 674 99575 50124.5 5012450 -32496 32642 4904.66 490466 -127 124 -0.3 -30 -675 2 10665 99576 2.02702 299.02702 150.52702 15052.7027 2.02702 299.02704 150.52702 15052.70264 2.02702 299.02702 150.5270199999999 15052.70200 2020-01-01 2020-01-02 2020-01-01 00:11:15 2020-01-02 03:39:36 2020-01-01 00:11:15.000 2020-01-02 03:39:36.000 675 99576 50125.5 5012550 675 99576 50125.5 5012550 -32495 32643 4905.66 490566 -126 125 0.7 70 -676 2 10666 99577 2.03003 299.03003 150.53003 15053.003 2.03003 299.03003 150.53002 15053.00293 2.03003 299.03003 150.53002999999993 15053.00300 2020-01-01 2020-01-02 2020-01-01 00:11:16 2020-01-02 03:39:37 2020-01-01 00:11:16.000 2020-01-02 03:39:37.000 676 99577 50126.5 5012650 676 99577 50126.5 5012650 -32494 32644 4906.66 490666 -125 126 1.7 170 +67 2 10057 99967 0.2012 300.2012 150.2012 15170.32132 0.2012 300.2012 150.2012 15170.32142 0.20120 300.20120 150.2012 15170.32120 2020-01-01 2020-01-02 2020-01-01 00:01:07 2020-01-02 03:46:07 2020-01-01 00:01:07.000 2020-01-02 03:46:07.000 67 99967 50017 5051717 67 99967 50017 5051717 -32502 32433 4596.009900990099 464197 -128 127 -1.8514851485148516 -187 +670 2 10660 99571 2.01201 299.01201 150.51201 15051.2012 2.01201 299.01202 150.51201 15051.20117 2.01201 299.01201 150.51201 15051.20100 2020-01-01 2020-01-02 2020-01-01 00:11:10 2020-01-02 03:39:31 2020-01-01 00:11:10.000 2020-01-02 03:39:31.000 670 99571 50120.5 5012050 670 99571 50120.5 5012050 -32500 32638 4900.66 490066 -125 126 0.82 82 +671 2 10661 99572 2.01501 299.01501 150.51501 15051.5015 2.01501 299.015 150.51501 15051.50146 2.01501 299.01501 150.51501 15051.50100 2020-01-01 2020-01-02 2020-01-01 00:11:11 2020-01-02 03:39:32 2020-01-01 00:11:11.000 2020-01-02 03:39:32.000 671 99572 50121.5 5012150 671 99572 50121.5 5012150 -32499 32639 4901.66 490166 -124 127 1.82 182 +672 2 10662 99573 2.01801 299.01801 150.51801 15051.8018 2.01801 299.018 150.51801 15051.80176 2.01801 299.01801 150.51801 15051.80100 2020-01-01 2020-01-02 2020-01-01 00:11:12 2020-01-02 03:39:33 2020-01-01 00:11:12.000 2020-01-02 03:39:33.000 672 99573 50122.5 5012250 672 99573 50122.5 5012250 -32498 32640 4902.66 490266 -128 127 0.26 26 +673 2 10663 99574 2.02102 299.02102 150.52102 15052.1021 2.02102 299.02103 150.52102 15052.1024 2.02102 299.02102 150.52102000000002 15052.10200 2020-01-01 2020-01-02 2020-01-01 00:11:13 2020-01-02 03:39:34 2020-01-01 00:11:13.000 2020-01-02 03:39:34.000 673 99574 50123.5 5012350 673 99574 50123.5 5012350 -32497 32641 4903.66 490366 -128 123 -1.3 -130 +674 2 10664 99575 2.02402 299.02402 150.52402 15052.4024 2.02402 299.02402 150.52402 15052.40204 2.02402 299.02402 150.52402 15052.40200 2020-01-01 2020-01-02 2020-01-01 00:11:14 2020-01-02 03:39:35 2020-01-01 00:11:14.000 2020-01-02 03:39:35.000 674 99575 50124.5 5012450 674 99575 50124.5 5012450 -32496 32642 4904.66 490466 -127 124 -0.3 -30 +675 2 10665 99576 2.02702 299.02702 150.52702 15052.7027 2.02702 299.02704 150.52702 15052.70264 2.02702 299.02702 150.52702 15052.70200 2020-01-01 2020-01-02 2020-01-01 00:11:15 2020-01-02 03:39:36 2020-01-01 00:11:15.000 2020-01-02 03:39:36.000 675 99576 50125.5 5012550 675 99576 50125.5 5012550 -32495 32643 4905.66 490566 -126 125 0.7 70 +676 2 10666 99577 2.03003 299.03003 150.53003 15053.003 2.03003 299.03003 150.53002 15053.00293 2.03003 299.03003 150.53003 15053.00300 2020-01-01 2020-01-02 2020-01-01 00:11:16 2020-01-02 03:39:37 2020-01-01 00:11:16.000 2020-01-02 03:39:37.000 676 99577 50126.5 5012650 676 99577 50126.5 5012650 -32494 32644 4906.66 490666 -125 126 1.7 170 677 2 10667 99578 2.03303 299.03303 150.53303 15053.3033 2.03303 299.03302 150.53303 15053.30323 2.03303 299.03303 150.53303 15053.30300 2020-01-01 2020-01-02 2020-01-01 00:11:17 2020-01-02 03:39:38 2020-01-01 00:11:17.000 2020-01-02 03:39:38.000 677 99578 50127.5 5012750 677 99578 50127.5 5012750 -32493 32645 4907.66 490766 -124 127 2.7 270 -678 2 10668 99579 2.03603 299.03603 150.53603 15053.6036 2.03603 299.03604 150.53603 15053.60387 2.03603 299.03603 150.53602999999993 15053.60300 2020-01-01 2020-01-02 2020-01-01 00:11:18 2020-01-02 03:39:39 2020-01-01 00:11:18.000 2020-01-02 03:39:39.000 678 99579 50128.5 5012850 678 99579 50128.5 5012850 -32492 32646 4908.66 490866 -128 127 1.14 114 -679 2 10669 99580 2.03903 299.03903 150.53903 15053.9039 2.03903 299.03903 150.53903 15053.90351 2.03903 299.03903 150.53902999999994 15053.90300 2020-01-01 2020-01-02 2020-01-01 00:11:19 2020-01-02 03:39:40 2020-01-01 00:11:19.000 2020-01-02 03:39:40.000 679 99580 50129.5 5012950 679 99580 50129.5 5012950 -32491 32647 4909.66 490966 -128 123 -0.42 -42 -68 2 10058 99968 0.2042 300.2042 150.2042 15170.62462 0.2042 300.2042 150.2042 15170.62457 0.20420 300.20420 150.2042000000001 15170.62420 2020-01-01 2020-01-02 2020-01-01 00:01:08 2020-01-02 03:46:08 2020-01-01 00:01:08.000 2020-01-02 03:46:08.000 68 99968 50018 5051818 68 99968 50018 5051818 -32501 32434 4597.009900990099 464298 -128 124 -3.386138613861386 -342 -680 2 10670 99581 2.04204 299.04204 150.54204 15054.2042 2.04204 299.04205 150.54204 15054.20426 2.04204 299.04204 150.5420400000001 15054.20400 2020-01-01 2020-01-02 2020-01-01 00:11:20 2020-01-02 03:39:41 2020-01-01 00:11:20.000 2020-01-02 03:39:41.000 680 99581 50130.5 5013050 680 99581 50130.5 5013050 -32490 32648 4910.66 491066 -127 124 0.58 58 -681 2 10671 99582 2.04504 299.04504 150.54504 15054.5045 2.04504 299.04504 150.54504 15054.5044 2.04504 299.04504 150.5450400000002 15054.50400 2020-01-01 2020-01-02 2020-01-01 00:11:21 2020-01-02 03:39:42 2020-01-01 00:11:21.000 2020-01-02 03:39:42.000 681 99582 50131.5 5013150 681 99582 50131.5 5013150 -32489 32649 4911.66 491166 -126 125 1.58 158 -682 2 10672 99583 2.04804 299.04804 150.54804 15054.8048 2.04804 299.04803 150.54804 15054.80474 2.04804 299.04804 150.54803999999984 15054.80400 2020-01-01 2020-01-02 2020-01-01 00:11:22 2020-01-02 03:39:43 2020-01-01 00:11:22.000 2020-01-02 03:39:43.000 682 99583 50132.5 5013250 682 99583 50132.5 5013250 -32488 32650 4912.66 491266 -125 126 2.58 258 +678 2 10668 99579 2.03603 299.03603 150.53603 15053.6036 2.03603 299.03604 150.53603 15053.60387 2.03603 299.03603 150.53602999999998 15053.60300 2020-01-01 2020-01-02 2020-01-01 00:11:18 2020-01-02 03:39:39 2020-01-01 00:11:18.000 2020-01-02 03:39:39.000 678 99579 50128.5 5012850 678 99579 50128.5 5012850 -32492 32646 4908.66 490866 -128 127 1.14 114 +679 2 10669 99580 2.03903 299.03903 150.53903 15053.9039 2.03903 299.03903 150.53903 15053.90351 2.03903 299.03903 150.53903 15053.90300 2020-01-01 2020-01-02 2020-01-01 00:11:19 2020-01-02 03:39:40 2020-01-01 00:11:19.000 2020-01-02 03:39:40.000 679 99580 50129.5 5012950 679 99580 50129.5 5012950 -32491 32647 4909.66 490966 -128 123 -0.42 -42 +68 2 10058 99968 0.2042 300.2042 150.2042 15170.62462 0.2042 300.2042 150.2042 15170.62457 0.20420 300.20420 150.20420000000001 15170.62420 2020-01-01 2020-01-02 2020-01-01 00:01:08 2020-01-02 03:46:08 2020-01-01 00:01:08.000 2020-01-02 03:46:08.000 68 99968 50018 5051818 68 99968 50018 5051818 -32501 32434 4597.009900990099 464298 -128 124 -3.386138613861386 -342 +680 2 10670 99581 2.04204 299.04204 150.54204 15054.2042 2.04204 299.04205 150.54204 15054.20426 2.04204 299.04204 150.54204 15054.20400 2020-01-01 2020-01-02 2020-01-01 00:11:20 2020-01-02 03:39:41 2020-01-01 00:11:20.000 2020-01-02 03:39:41.000 680 99581 50130.5 5013050 680 99581 50130.5 5013050 -32490 32648 4910.66 491066 -127 124 0.58 58 +681 2 10671 99582 2.04504 299.04504 150.54504 15054.5045 2.04504 299.04504 150.54504 15054.5044 2.04504 299.04504 150.54504 15054.50400 2020-01-01 2020-01-02 2020-01-01 00:11:21 2020-01-02 03:39:42 2020-01-01 00:11:21.000 2020-01-02 03:39:42.000 681 99582 50131.5 5013150 681 99582 50131.5 5013150 -32489 32649 4911.66 491166 -126 125 1.58 158 +682 2 10672 99583 2.04804 299.04804 150.54804 15054.8048 2.04804 299.04803 150.54804 15054.80474 2.04804 299.04804 150.54804000000001 15054.80400 2020-01-01 2020-01-02 2020-01-01 00:11:22 2020-01-02 03:39:43 2020-01-01 00:11:22.000 2020-01-02 03:39:43.000 682 99583 50132.5 5013250 682 99583 50132.5 5013250 -32488 32650 4912.66 491266 -125 126 2.58 258 683 2 10673 99584 2.05105 299.05105 150.55105 15055.1051 2.05105 299.05106 150.55105 15055.10533 2.05105 299.05105 150.55105 15055.10500 2020-01-01 2020-01-02 2020-01-01 00:11:23 2020-01-02 03:39:44 2020-01-01 00:11:23.000 2020-01-02 03:39:44.000 683 99584 50133.5 5013350 683 99584 50133.5 5013350 -32487 32651 4913.66 491366 -124 127 3.58 358 -684 2 10674 99585 2.05405 299.05405 150.55405 15055.4054 2.05405 299.05405 150.55404 15055.40498 2.05405 299.05405 150.55405000000013 15055.40500 2020-01-01 2020-01-02 2020-01-01 00:11:24 2020-01-02 03:39:45 2020-01-01 00:11:24.000 2020-01-02 03:39:45.000 684 99585 50134.5 5013450 684 99585 50134.5 5013450 -32486 32652 4914.66 491466 -128 127 2.02 202 -685 2 10675 99586 2.05705 299.05705 150.55705 15055.7057 2.05705 299.05707 150.55705 15055.70573 2.05705 299.05705 150.55704999999992 15055.70500 2020-01-01 2020-01-02 2020-01-01 00:11:25 2020-01-02 03:39:46 2020-01-01 00:11:25.000 2020-01-02 03:39:46.000 685 99586 50135.5 5013550 685 99586 50135.5 5013550 -32485 32653 4915.66 491566 -128 127 0.46 46 -686 2 10676 99587 2.06006 299.06006 150.56006 15056.006 2.06006 299.06006 150.56005 15056.00587 2.06006 299.06006 150.56005999999988 15056.00600 2020-01-01 2020-01-02 2020-01-01 00:11:26 2020-01-02 03:39:47 2020-01-01 00:11:26.000 2020-01-02 03:39:47.000 686 99587 50136.5 5013650 686 99587 50136.5 5013650 -32484 32654 4916.66 491666 -128 124 -1.1 -110 -687 2 10677 99588 2.06306 299.06306 150.56306 15056.3063 2.06306 299.06305 150.56306 15056.30621 2.06306 299.06306 150.5630600000002 15056.30600 2020-01-01 2020-01-02 2020-01-01 00:11:27 2020-01-02 03:39:48 2020-01-01 00:11:27.000 2020-01-02 03:39:48.000 687 99588 50137.5 5013750 687 99588 50137.5 5013750 -32483 32655 4917.66 491766 -127 125 -0.1 -10 -688 2 10678 99589 2.06606 299.06606 150.56606 15056.6066 2.06606 299.06607 150.56606 15056.60681 2.06606 299.06606 150.56605999999988 15056.60600 2020-01-01 2020-01-02 2020-01-01 00:11:28 2020-01-02 03:39:49 2020-01-01 00:11:28.000 2020-01-02 03:39:49.000 688 99589 50138.5 5013850 688 99589 50138.5 5013850 -32482 32656 4918.66 491866 -126 126 0.9 90 -689 2 10679 99590 2.06906 299.06906 150.56906 15056.9069 2.06906 299.06906 150.56907 15056.9071 2.06906 299.06906 150.56905999999987 15056.90600 2020-01-01 2020-01-02 2020-01-01 00:11:29 2020-01-02 03:39:50 2020-01-01 00:11:29.000 2020-01-02 03:39:50.000 689 99590 50139.5 5013950 689 99590 50139.5 5013950 -32481 32657 4919.66 491966 -125 127 1.9 190 -69 2 10059 99969 0.2072 300.2072 150.2072 15170.92792 0.2072 300.2072 150.20721 15170.92832 0.20720 300.20720 150.20720000000009 15170.92720 2020-01-01 2020-01-02 2020-01-01 00:01:09 2020-01-02 03:46:09 2020-01-01 00:01:09.000 2020-01-02 03:46:09.000 69 99969 50019 5051919 69 99969 50019 5051919 -32500 32435 4598.009900990099 464399 -127 125 -2.386138613861386 -241 -690 2 10680 99591 2.07207 299.07207 150.57207 15057.2072 2.07207 299.07208 150.57207 15057.2072 2.07207 299.07207 150.5720700000001 15057.20700 2020-01-01 2020-01-02 2020-01-01 00:11:30 2020-01-02 03:39:51 2020-01-01 00:11:30.000 2020-01-02 03:39:51.000 690 99591 50140.5 5014050 690 99591 50140.5 5014050 -32480 32658 4920.66 492066 -128 127 0.34 34 -691 2 10681 99592 2.07507 299.07507 150.57507 15057.5075 2.07507 299.07507 150.57507 15057.50734 2.07507 299.07507 150.57507000000012 15057.50700 2020-01-01 2020-01-02 2020-01-01 00:11:31 2020-01-02 03:39:52 2020-01-01 00:11:31.000 2020-01-02 03:39:52.000 691 99592 50141.5 5014150 691 99592 50141.5 5014150 -32479 32659 4921.66 492166 -128 127 -1.22 -122 -692 2 10682 99593 2.07807 299.07807 150.57807 15057.8078 2.07807 299.07806 150.57807 15057.80767 2.07807 299.07807 150.5780699999998 15057.80700 2020-01-01 2020-01-02 2020-01-01 00:11:32 2020-01-02 03:39:53 2020-01-01 00:11:32.000 2020-01-02 03:39:53.000 692 99593 50142.5 5014250 692 99593 50142.5 5014250 -32478 32660 4922.66 492266 -128 123 -2.78 -278 -693 2 10683 99594 2.08108 299.08108 150.58108 15058.1081 2.08108 299.0811 150.58108 15058.10827 2.08108 299.08108 150.58107999999996 15058.10800 2020-01-01 2020-01-02 2020-01-01 00:11:33 2020-01-02 03:39:54 2020-01-01 00:11:33.000 2020-01-02 03:39:54.000 693 99594 50143.5 5014350 693 99594 50143.5 5014350 -32477 32661 4923.66 492366 -127 124 -1.78 -178 -694 2 10684 99595 2.08408 299.08408 150.58408 15058.4084 2.08408 299.08408 150.58408 15058.40857 2.08408 299.08408 150.58408000000003 15058.40800 2020-01-01 2020-01-02 2020-01-01 00:11:34 2020-01-02 03:39:55 2020-01-01 00:11:34.000 2020-01-02 03:39:55.000 694 99595 50144.5 5014450 694 99595 50144.5 5014450 -32476 32662 4924.66 492466 -126 125 -0.78 -78 -695 2 10685 99596 2.08708 299.08708 150.58708 15058.7087 2.08708 299.0871 150.58708 15058.70867 2.08708 299.08708 150.5870799999999 15058.70800 2020-01-01 2020-01-02 2020-01-01 00:11:35 2020-01-02 03:39:56 2020-01-01 00:11:35.000 2020-01-02 03:39:56.000 695 99596 50145.5 5014550 695 99596 50145.5 5014550 -32475 32663 4925.66 492566 -125 126 0.22 22 -696 2 10686 99597 2.09009 299.09009 150.59009 15059.009 2.09009 299.0901 150.59008 15059.00885 2.09009 299.09009 150.59008999999986 15059.00900 2020-01-01 2020-01-02 2020-01-01 00:11:36 2020-01-02 03:39:57 2020-01-01 00:11:36.000 2020-01-02 03:39:57.000 696 99597 50146.5 5014650 696 99597 50146.5 5014650 -32474 32664 4926.66 492666 -124 127 1.22 122 -697 2 10687 99598 2.09309 299.09309 150.59309 15059.3093 2.09309 299.09308 150.59309 15059.30915 2.09309 299.09309 150.59309000000013 15059.30900 2020-01-01 2020-01-02 2020-01-01 00:11:37 2020-01-02 03:39:58 2020-01-01 00:11:37.000 2020-01-02 03:39:58.000 697 99598 50147.5 5014750 697 99598 50147.5 5014750 -32473 32665 4927.66 492766 -128 127 -0.34 -34 -698 2 10688 99599 2.09609 299.09609 150.59609 15059.6096 2.09609 299.0961 150.59609 15059.6099 2.09609 299.09609 150.5960899999998 15059.60900 2020-01-01 2020-01-02 2020-01-01 00:11:38 2020-01-02 03:39:59 2020-01-01 00:11:38.000 2020-01-02 03:39:59.000 698 99599 50148.5 5014850 698 99599 50148.5 5014850 -32472 32666 4928.66 492866 -128 123 -1.9 -190 -699 2 10689 99600 2.09909 299.09909 150.59909 15059.9099 2.09909 299.0991 150.5991 15059.91003 2.09909 299.09909 150.59908999999988 15059.90900 2020-01-01 2020-01-02 2020-01-01 00:11:39 2020-01-02 03:40:00 2020-01-01 00:11:39.000 2020-01-02 03:40:00.000 699 99600 50149.5 5014950 699 99600 50149.5 5014950 -32471 32667 4929.66 492966 -127 124 -0.9 -90 -7 2 1006 9997 0.02102 300.02102 150.02102 15152.12312 0.02102 300.02103 150.02102 15152.12342 0.02102 300.02102 150.0210200000001 15152.12302 2020-01-01 2020-01-02 2020-01-01 00:00:07 2020-01-02 03:45:07 2020-01-01 00:00:07.000 2020-01-02 03:45:07.000 7 99907 49957 5045657 7 99907 49957 5045657 -32562 32373 4536.009900990099 458137 -126 125 -1.0198019801980198 -103 -70 2 10060 99970 0.21021 300.21021 150.21021 15171.23123 0.21021 300.2102 150.2102 15171.23097 0.21021 300.21021 150.2102099999999 15171.23121 2020-01-01 2020-01-02 2020-01-01 00:01:10 2020-01-02 03:46:10 2020-01-01 00:01:10.000 2020-01-02 03:46:10.000 70 99970 50020 5052020 70 99970 50020 5052020 -32499 32436 4599.009900990099 464500 -126 126 -1.386138613861386 -140 -700 2 10690 99601 2.1021 299.1021 150.6021 15060.21021 2.1021 299.1021 150.6021 15060.21014 2.10210 299.10210 150.60210000000004 15060.21000 2020-01-01 2020-01-02 2020-01-01 00:11:40 2020-01-02 03:40:01 2020-01-01 00:11:40.000 2020-01-02 03:40:01.000 700 99601 50150.5 5015050 700 99601 50150.5 5015050 -32470 32668 4930.66 493066 -126 125 0.1 10 -701 2 10691 99602 2.1051 299.1051 150.6051 15060.51051 2.1051 299.1051 150.6051 15060.51031 2.10510 299.10510 150.60510000000005 15060.51000 2020-01-01 2020-01-02 2020-01-01 00:11:41 2020-01-02 03:40:02 2020-01-01 00:11:41.000 2020-01-02 03:40:02.000 701 99602 50151.5 5015150 701 99602 50151.5 5015150 -32469 32669 4931.66 493166 -125 126 1.1 110 -702 2 10692 99603 2.1081 299.1081 150.6081 15060.81081 2.1081 299.1081 150.6081 15060.81062 2.10810 299.10810 150.60809999999998 15060.81000 2020-01-01 2020-01-02 2020-01-01 00:11:42 2020-01-02 03:40:03 2020-01-01 00:11:42.000 2020-01-02 03:40:03.000 702 99603 50152.5 5015250 702 99603 50152.5 5015250 -32468 32670 4932.66 493266 -124 127 2.1 210 -703 2 10693 99604 2.11111 299.11111 150.61111 15061.11111 2.11111 299.1111 150.61111 15061.11137 2.11111 299.11111 150.61110999999988 15061.11100 2020-01-01 2020-01-02 2020-01-01 00:11:43 2020-01-02 03:40:04 2020-01-01 00:11:43.000 2020-01-02 03:40:04.000 703 99604 50153.5 5015350 703 99604 50153.5 5015350 -32467 32671 4933.66 493366 -128 127 0.54 54 -704 2 10694 99605 2.11411 299.11411 150.61411 15061.41141 2.11411 299.1141 150.61411 15061.41151 2.11411 299.11411 150.6141100000002 15061.41100 2020-01-01 2020-01-02 2020-01-01 00:11:44 2020-01-02 03:40:05 2020-01-01 00:11:44.000 2020-01-02 03:40:05.000 704 99605 50154.5 5015450 704 99605 50154.5 5015450 -32466 32672 4934.66 493466 -128 123 -1.02 -102 -705 2 10695 99606 2.11711 299.11711 150.61711 15061.71171 2.11711 299.11713 150.61711 15061.71165 2.11711 299.11711 150.6171099999999 15061.71100 2020-01-01 2020-01-02 2020-01-01 00:11:45 2020-01-02 03:40:06 2020-01-01 00:11:45.000 2020-01-02 03:40:06.000 705 99606 50155.5 5015550 705 99606 50155.5 5015550 -32465 32673 4935.66 493566 -127 124 -0.02 -2 -706 2 10696 99607 2.12012 299.12012 150.62012 15062.01201 2.12012 299.12012 150.62011 15062.01179 2.12012 299.12012 150.6201199999998 15062.01200 2020-01-01 2020-01-02 2020-01-01 00:11:46 2020-01-02 03:40:07 2020-01-01 00:11:46.000 2020-01-02 03:40:07.000 706 99607 50156.5 5015650 706 99607 50156.5 5015650 -32464 32674 4936.66 493666 -126 125 0.98 98 -707 2 10697 99608 2.12312 299.12312 150.62312 15062.31231 2.12312 299.1231 150.62312 15062.31208 2.12312 299.12312 150.62312000000014 15062.31200 2020-01-01 2020-01-02 2020-01-01 00:11:47 2020-01-02 03:40:08 2020-01-01 00:11:47.000 2020-01-02 03:40:08.000 707 99608 50157.5 5015750 707 99608 50157.5 5015750 -32463 32675 4937.66 493766 -125 126 1.98 198 -708 2 10698 99609 2.12612 299.12612 150.62612 15062.61261 2.12612 299.12613 150.62612 15062.61283 2.12612 299.12612 150.62612000000013 15062.61200 2020-01-01 2020-01-02 2020-01-01 00:11:48 2020-01-02 03:40:09 2020-01-01 00:11:48.000 2020-01-02 03:40:09.000 708 99609 50158.5 5015850 708 99609 50158.5 5015850 -32462 32676 4938.66 493866 -124 127 2.98 298 -709 2 10699 99610 2.12912 299.12912 150.62912 15062.91291 2.12912 299.12912 150.62912 15062.91298 2.12912 299.12912 150.6291199999998 15062.91200 2020-01-01 2020-01-02 2020-01-01 00:11:49 2020-01-02 03:40:10 2020-01-01 00:11:49.000 2020-01-02 03:40:10.000 709 99610 50159.5 5015950 709 99610 50159.5 5015950 -32461 32677 4939.66 493966 -128 127 1.42 142 -71 2 10061 99971 0.21321 300.21321 150.21321 15171.53453 0.21321 300.21323 150.21321 15171.5346 0.21321 300.21321 150.21320999999992 15171.53421 2020-01-01 2020-01-02 2020-01-01 00:01:11 2020-01-02 03:46:11 2020-01-01 00:01:11.000 2020-01-02 03:46:11.000 71 99971 50021 5052121 71 99971 50021 5052121 -32498 32437 4600.009900990099 464601 -125 127 -0.38613861386138615 -39 +684 2 10674 99585 2.05405 299.05405 150.55405 15055.4054 2.05405 299.05405 150.55404 15055.40498 2.05405 299.05405 150.55405000000002 15055.40500 2020-01-01 2020-01-02 2020-01-01 00:11:24 2020-01-02 03:39:45 2020-01-01 00:11:24.000 2020-01-02 03:39:45.000 684 99585 50134.5 5013450 684 99585 50134.5 5013450 -32486 32652 4914.66 491466 -128 127 2.02 202 +685 2 10675 99586 2.05705 299.05705 150.55705 15055.7057 2.05705 299.05707 150.55705 15055.70573 2.05705 299.05705 150.55705 15055.70500 2020-01-01 2020-01-02 2020-01-01 00:11:25 2020-01-02 03:39:46 2020-01-01 00:11:25.000 2020-01-02 03:39:46.000 685 99586 50135.5 5013550 685 99586 50135.5 5013550 -32485 32653 4915.66 491566 -128 127 0.46 46 +686 2 10676 99587 2.06006 299.06006 150.56006 15056.006 2.06006 299.06006 150.56005 15056.00587 2.06006 299.06006 150.56006 15056.00600 2020-01-01 2020-01-02 2020-01-01 00:11:26 2020-01-02 03:39:47 2020-01-01 00:11:26.000 2020-01-02 03:39:47.000 686 99587 50136.5 5013650 686 99587 50136.5 5013650 -32484 32654 4916.66 491666 -128 124 -1.1 -110 +687 2 10677 99588 2.06306 299.06306 150.56306 15056.3063 2.06306 299.06305 150.56306 15056.30621 2.06306 299.06306 150.56306 15056.30600 2020-01-01 2020-01-02 2020-01-01 00:11:27 2020-01-02 03:39:48 2020-01-01 00:11:27.000 2020-01-02 03:39:48.000 687 99588 50137.5 5013750 687 99588 50137.5 5013750 -32483 32655 4917.66 491766 -127 125 -0.1 -10 +688 2 10678 99589 2.06606 299.06606 150.56606 15056.6066 2.06606 299.06607 150.56606 15056.60681 2.06606 299.06606 150.56606 15056.60600 2020-01-01 2020-01-02 2020-01-01 00:11:28 2020-01-02 03:39:49 2020-01-01 00:11:28.000 2020-01-02 03:39:49.000 688 99589 50138.5 5013850 688 99589 50138.5 5013850 -32482 32656 4918.66 491866 -126 126 0.9 90 +689 2 10679 99590 2.06906 299.06906 150.56906 15056.9069 2.06906 299.06906 150.56907 15056.9071 2.06906 299.06906 150.56906 15056.90600 2020-01-01 2020-01-02 2020-01-01 00:11:29 2020-01-02 03:39:50 2020-01-01 00:11:29.000 2020-01-02 03:39:50.000 689 99590 50139.5 5013950 689 99590 50139.5 5013950 -32481 32657 4919.66 491966 -125 127 1.9 190 +69 2 10059 99969 0.2072 300.2072 150.2072 15170.92792 0.2072 300.2072 150.20721 15170.92832 0.20720 300.20720 150.2072 15170.92720 2020-01-01 2020-01-02 2020-01-01 00:01:09 2020-01-02 03:46:09 2020-01-01 00:01:09.000 2020-01-02 03:46:09.000 69 99969 50019 5051919 69 99969 50019 5051919 -32500 32435 4598.009900990099 464399 -127 125 -2.386138613861386 -241 +690 2 10680 99591 2.07207 299.07207 150.57207 15057.2072 2.07207 299.07208 150.57207 15057.2072 2.07207 299.07207 150.57207 15057.20700 2020-01-01 2020-01-02 2020-01-01 00:11:30 2020-01-02 03:39:51 2020-01-01 00:11:30.000 2020-01-02 03:39:51.000 690 99591 50140.5 5014050 690 99591 50140.5 5014050 -32480 32658 4920.66 492066 -128 127 0.34 34 +691 2 10681 99592 2.07507 299.07507 150.57507 15057.5075 2.07507 299.07507 150.57507 15057.50734 2.07507 299.07507 150.57506999999998 15057.50700 2020-01-01 2020-01-02 2020-01-01 00:11:31 2020-01-02 03:39:52 2020-01-01 00:11:31.000 2020-01-02 03:39:52.000 691 99592 50141.5 5014150 691 99592 50141.5 5014150 -32479 32659 4921.66 492166 -128 127 -1.22 -122 +692 2 10682 99593 2.07807 299.07807 150.57807 15057.8078 2.07807 299.07806 150.57807 15057.80767 2.07807 299.07807 150.57807 15057.80700 2020-01-01 2020-01-02 2020-01-01 00:11:32 2020-01-02 03:39:53 2020-01-01 00:11:32.000 2020-01-02 03:39:53.000 692 99593 50142.5 5014250 692 99593 50142.5 5014250 -32478 32660 4922.66 492266 -128 123 -2.78 -278 +693 2 10683 99594 2.08108 299.08108 150.58108 15058.1081 2.08108 299.0811 150.58108 15058.10827 2.08108 299.08108 150.58108000000001 15058.10800 2020-01-01 2020-01-02 2020-01-01 00:11:33 2020-01-02 03:39:54 2020-01-01 00:11:33.000 2020-01-02 03:39:54.000 693 99594 50143.5 5014350 693 99594 50143.5 5014350 -32477 32661 4923.66 492366 -127 124 -1.78 -178 +694 2 10684 99595 2.08408 299.08408 150.58408 15058.4084 2.08408 299.08408 150.58408 15058.40857 2.08408 299.08408 150.58408 15058.40800 2020-01-01 2020-01-02 2020-01-01 00:11:34 2020-01-02 03:39:55 2020-01-01 00:11:34.000 2020-01-02 03:39:55.000 694 99595 50144.5 5014450 694 99595 50144.5 5014450 -32476 32662 4924.66 492466 -126 125 -0.78 -78 +695 2 10685 99596 2.08708 299.08708 150.58708 15058.7087 2.08708 299.0871 150.58708 15058.70867 2.08708 299.08708 150.58708000000001 15058.70800 2020-01-01 2020-01-02 2020-01-01 00:11:35 2020-01-02 03:39:56 2020-01-01 00:11:35.000 2020-01-02 03:39:56.000 695 99596 50145.5 5014550 695 99596 50145.5 5014550 -32475 32663 4925.66 492566 -125 126 0.22 22 +696 2 10686 99597 2.09009 299.09009 150.59009 15059.009 2.09009 299.0901 150.59008 15059.00885 2.09009 299.09009 150.59009 15059.00900 2020-01-01 2020-01-02 2020-01-01 00:11:36 2020-01-02 03:39:57 2020-01-01 00:11:36.000 2020-01-02 03:39:57.000 696 99597 50146.5 5014650 696 99597 50146.5 5014650 -32474 32664 4926.66 492666 -124 127 1.22 122 +697 2 10687 99598 2.09309 299.09309 150.59309 15059.3093 2.09309 299.09308 150.59309 15059.30915 2.09309 299.09309 150.59309 15059.30900 2020-01-01 2020-01-02 2020-01-01 00:11:37 2020-01-02 03:39:58 2020-01-01 00:11:37.000 2020-01-02 03:39:58.000 697 99598 50147.5 5014750 697 99598 50147.5 5014750 -32473 32665 4927.66 492766 -128 127 -0.34 -34 +698 2 10688 99599 2.09609 299.09609 150.59609 15059.6096 2.09609 299.0961 150.59609 15059.6099 2.09609 299.09609 150.59609 15059.60900 2020-01-01 2020-01-02 2020-01-01 00:11:38 2020-01-02 03:39:59 2020-01-01 00:11:38.000 2020-01-02 03:39:59.000 698 99599 50148.5 5014850 698 99599 50148.5 5014850 -32472 32666 4928.66 492866 -128 123 -1.9 -190 +699 2 10689 99600 2.09909 299.09909 150.59909 15059.9099 2.09909 299.0991 150.5991 15059.91003 2.09909 299.09909 150.59909 15059.90900 2020-01-01 2020-01-02 2020-01-01 00:11:39 2020-01-02 03:40:00 2020-01-01 00:11:39.000 2020-01-02 03:40:00.000 699 99600 50149.5 5014950 699 99600 50149.5 5014950 -32471 32667 4929.66 492966 -127 124 -0.9 -90 +7 2 1006 9997 0.02102 300.02102 150.02102 15152.12312 0.02102 300.02103 150.02102 15152.12342 0.02102 300.02102 150.02102000000002 15152.12302 2020-01-01 2020-01-02 2020-01-01 00:00:07 2020-01-02 03:45:07 2020-01-01 00:00:07.000 2020-01-02 03:45:07.000 7 99907 49957 5045657 7 99907 49957 5045657 -32562 32373 4536.009900990099 458137 -126 125 -1.0198019801980198 -103 +70 2 10060 99970 0.21021 300.21021 150.21021 15171.23123 0.21021 300.2102 150.2102 15171.23097 0.21021 300.21021 150.21021 15171.23121 2020-01-01 2020-01-02 2020-01-01 00:01:10 2020-01-02 03:46:10 2020-01-01 00:01:10.000 2020-01-02 03:46:10.000 70 99970 50020 5052020 70 99970 50020 5052020 -32499 32436 4599.009900990099 464500 -126 126 -1.386138613861386 -140 +700 2 10690 99601 2.1021 299.1021 150.6021 15060.21021 2.1021 299.1021 150.6021 15060.21014 2.10210 299.10210 150.60209999999998 15060.21000 2020-01-01 2020-01-02 2020-01-01 00:11:40 2020-01-02 03:40:01 2020-01-01 00:11:40.000 2020-01-02 03:40:01.000 700 99601 50150.5 5015050 700 99601 50150.5 5015050 -32470 32668 4930.66 493066 -126 125 0.1 10 +701 2 10691 99602 2.1051 299.1051 150.6051 15060.51051 2.1051 299.1051 150.6051 15060.51031 2.10510 299.10510 150.6051 15060.51000 2020-01-01 2020-01-02 2020-01-01 00:11:41 2020-01-02 03:40:02 2020-01-01 00:11:41.000 2020-01-02 03:40:02.000 701 99602 50151.5 5015150 701 99602 50151.5 5015150 -32469 32669 4931.66 493166 -125 126 1.1 110 +702 2 10692 99603 2.1081 299.1081 150.6081 15060.81081 2.1081 299.1081 150.6081 15060.81062 2.10810 299.10810 150.6081 15060.81000 2020-01-01 2020-01-02 2020-01-01 00:11:42 2020-01-02 03:40:03 2020-01-01 00:11:42.000 2020-01-02 03:40:03.000 702 99603 50152.5 5015250 702 99603 50152.5 5015250 -32468 32670 4932.66 493266 -124 127 2.1 210 +703 2 10693 99604 2.11111 299.11111 150.61111 15061.11111 2.11111 299.1111 150.61111 15061.11137 2.11111 299.11111 150.61111 15061.11100 2020-01-01 2020-01-02 2020-01-01 00:11:43 2020-01-02 03:40:04 2020-01-01 00:11:43.000 2020-01-02 03:40:04.000 703 99604 50153.5 5015350 703 99604 50153.5 5015350 -32467 32671 4933.66 493366 -128 127 0.54 54 +704 2 10694 99605 2.11411 299.11411 150.61411 15061.41141 2.11411 299.1141 150.61411 15061.41151 2.11411 299.11411 150.61411 15061.41100 2020-01-01 2020-01-02 2020-01-01 00:11:44 2020-01-02 03:40:05 2020-01-01 00:11:44.000 2020-01-02 03:40:05.000 704 99605 50154.5 5015450 704 99605 50154.5 5015450 -32466 32672 4934.66 493466 -128 123 -1.02 -102 +705 2 10695 99606 2.11711 299.11711 150.61711 15061.71171 2.11711 299.11713 150.61711 15061.71165 2.11711 299.11711 150.61711 15061.71100 2020-01-01 2020-01-02 2020-01-01 00:11:45 2020-01-02 03:40:06 2020-01-01 00:11:45.000 2020-01-02 03:40:06.000 705 99606 50155.5 5015550 705 99606 50155.5 5015550 -32465 32673 4935.66 493566 -127 124 -0.02 -2 +706 2 10696 99607 2.12012 299.12012 150.62012 15062.01201 2.12012 299.12012 150.62011 15062.01179 2.12012 299.12012 150.62012000000001 15062.01200 2020-01-01 2020-01-02 2020-01-01 00:11:46 2020-01-02 03:40:07 2020-01-01 00:11:46.000 2020-01-02 03:40:07.000 706 99607 50156.5 5015650 706 99607 50156.5 5015650 -32464 32674 4936.66 493666 -126 125 0.98 98 +707 2 10697 99608 2.12312 299.12312 150.62312 15062.31231 2.12312 299.1231 150.62312 15062.31208 2.12312 299.12312 150.62312 15062.31200 2020-01-01 2020-01-02 2020-01-01 00:11:47 2020-01-02 03:40:08 2020-01-01 00:11:47.000 2020-01-02 03:40:08.000 707 99608 50157.5 5015750 707 99608 50157.5 5015750 -32463 32675 4937.66 493766 -125 126 1.98 198 +708 2 10698 99609 2.12612 299.12612 150.62612 15062.61261 2.12612 299.12613 150.62612 15062.61283 2.12612 299.12612 150.62612 15062.61200 2020-01-01 2020-01-02 2020-01-01 00:11:48 2020-01-02 03:40:09 2020-01-01 00:11:48.000 2020-01-02 03:40:09.000 708 99609 50158.5 5015850 708 99609 50158.5 5015850 -32462 32676 4938.66 493866 -124 127 2.98 298 +709 2 10699 99610 2.12912 299.12912 150.62912 15062.91291 2.12912 299.12912 150.62912 15062.91298 2.12912 299.12912 150.62912 15062.91200 2020-01-01 2020-01-02 2020-01-01 00:11:49 2020-01-02 03:40:10 2020-01-01 00:11:49.000 2020-01-02 03:40:10.000 709 99610 50159.5 5015950 709 99610 50159.5 5015950 -32461 32677 4939.66 493966 -128 127 1.42 142 +71 2 10061 99971 0.21321 300.21321 150.21321 15171.53453 0.21321 300.21323 150.21321 15171.5346 0.21321 300.21321 150.21321 15171.53421 2020-01-01 2020-01-02 2020-01-01 00:01:11 2020-01-02 03:46:11 2020-01-01 00:01:11.000 2020-01-02 03:46:11.000 71 99971 50021 5052121 71 99971 50021 5052121 -32498 32437 4600.009900990099 464601 -125 127 -0.38613861386138615 -39 710 2 10700 99611 2.13213 299.13213 150.63213 15063.21321 2.13213 299.13214 150.63213 15063.21311 2.13213 299.13213 150.63213 15063.21300 2020-01-01 2020-01-02 2020-01-01 00:11:50 2020-01-02 03:40:11 2020-01-01 00:11:50.000 2020-01-02 03:40:11.000 710 99611 50160.5 5016050 710 99611 50160.5 5016050 -32460 32678 4940.66 494066 -128 127 -0.14 -14 -711 2 10701 99612 2.13513 299.13513 150.63513 15063.51351 2.13513 299.13513 150.63513 15063.51325 2.13513 299.13513 150.63513000000006 15063.51300 2020-01-01 2020-01-02 2020-01-01 00:11:51 2020-01-02 03:40:12 2020-01-01 00:11:51.000 2020-01-02 03:40:12.000 711 99612 50161.5 5016150 711 99612 50161.5 5016150 -32459 32679 4941.66 494166 -128 124 -1.7 -170 -712 2 10702 99613 2.13813 299.13813 150.63813 15063.81381 2.13813 299.13815 150.63814 15063.81401 2.13813 299.13813 150.6381299999999 15063.81300 2020-01-01 2020-01-02 2020-01-01 00:11:52 2020-01-02 03:40:13 2020-01-01 00:11:52.000 2020-01-02 03:40:13.000 712 99613 50162.5 5016250 712 99613 50162.5 5016250 -32458 32680 4942.66 494266 -127 125 -0.7 -70 -713 2 10703 99614 2.14114 299.14114 150.64114 15064.11411 2.14114 299.14114 150.64114 15064.11431 2.14114 299.14114 150.6411399999999 15064.11400 2020-01-01 2020-01-02 2020-01-01 00:11:53 2020-01-02 03:40:14 2020-01-01 00:11:53.000 2020-01-02 03:40:14.000 713 99614 50163.5 5016350 713 99614 50163.5 5016350 -32457 32681 4943.66 494366 -126 126 0.3 30 -714 2 10704 99615 2.14414 299.14414 150.64414 15064.41441 2.14414 299.14413 150.64414 15064.41448 2.14414 299.14414 150.64414000000014 15064.41400 2020-01-01 2020-01-02 2020-01-01 00:11:54 2020-01-02 03:40:15 2020-01-01 00:11:54.000 2020-01-02 03:40:15.000 714 99615 50164.5 5016450 714 99615 50164.5 5016450 -32456 32682 4944.66 494466 -125 127 1.3 130 -715 2 10705 99616 2.14714 299.14714 150.64714 15064.71471 2.14714 299.14716 150.64714 15064.71458 2.14714 299.14714 150.6471399999998 15064.71400 2020-01-01 2020-01-02 2020-01-01 00:11:55 2020-01-02 03:40:16 2020-01-01 00:11:55.000 2020-01-02 03:40:16.000 715 99616 50165.5 5016550 715 99616 50165.5 5016550 -32455 32683 4945.66 494566 -128 127 -0.26 -26 -716 2 10706 99617 2.15015 299.15015 150.65015 15065.01501 2.15015 299.15015 150.65014 15065.01472 2.15015 299.15015 150.65015000000002 15065.01500 2020-01-01 2020-01-02 2020-01-01 00:11:56 2020-01-02 03:40:17 2020-01-01 00:11:56.000 2020-01-02 03:40:17.000 716 99617 50166.5 5016650 716 99617 50166.5 5016650 -32454 32684 4946.66 494666 -128 127 -1.82 -182 -717 2 10707 99618 2.15315 299.15315 150.65315 15065.31531 2.15315 299.15317 150.65315 15065.31547 2.15315 299.15315 150.65315000000007 15065.31500 2020-01-01 2020-01-02 2020-01-01 00:11:57 2020-01-02 03:40:18 2020-01-01 00:11:57.000 2020-01-02 03:40:18.000 717 99618 50167.5 5016750 717 99618 50167.5 5016750 -32453 32685 4947.66 494766 -128 123 -3.38 -338 -718 2 10708 99619 2.15615 299.15615 150.65615 15065.61561 2.15615 299.15616 150.65615 15065.61578 2.15615 299.15615 150.65615000000005 15065.61500 2020-01-01 2020-01-02 2020-01-01 00:11:58 2020-01-02 03:40:19 2020-01-01 00:11:58.000 2020-01-02 03:40:19.000 718 99619 50168.5 5016850 718 99619 50168.5 5016850 -32452 32686 4948.66 494866 -127 124 -2.38 -238 +711 2 10701 99612 2.13513 299.13513 150.63513 15063.51351 2.13513 299.13513 150.63513 15063.51325 2.13513 299.13513 150.63513 15063.51300 2020-01-01 2020-01-02 2020-01-01 00:11:51 2020-01-02 03:40:12 2020-01-01 00:11:51.000 2020-01-02 03:40:12.000 711 99612 50161.5 5016150 711 99612 50161.5 5016150 -32459 32679 4941.66 494166 -128 124 -1.7 -170 +712 2 10702 99613 2.13813 299.13813 150.63813 15063.81381 2.13813 299.13815 150.63814 15063.81401 2.13813 299.13813 150.63813 15063.81300 2020-01-01 2020-01-02 2020-01-01 00:11:52 2020-01-02 03:40:13 2020-01-01 00:11:52.000 2020-01-02 03:40:13.000 712 99613 50162.5 5016250 712 99613 50162.5 5016250 -32458 32680 4942.66 494266 -127 125 -0.7 -70 +713 2 10703 99614 2.14114 299.14114 150.64114 15064.11411 2.14114 299.14114 150.64114 15064.11431 2.14114 299.14114 150.64114 15064.11400 2020-01-01 2020-01-02 2020-01-01 00:11:53 2020-01-02 03:40:14 2020-01-01 00:11:53.000 2020-01-02 03:40:14.000 713 99614 50163.5 5016350 713 99614 50163.5 5016350 -32457 32681 4943.66 494366 -126 126 0.3 30 +714 2 10704 99615 2.14414 299.14414 150.64414 15064.41441 2.14414 299.14413 150.64414 15064.41448 2.14414 299.14414 150.64414 15064.41400 2020-01-01 2020-01-02 2020-01-01 00:11:54 2020-01-02 03:40:15 2020-01-01 00:11:54.000 2020-01-02 03:40:15.000 714 99615 50164.5 5016450 714 99615 50164.5 5016450 -32456 32682 4944.66 494466 -125 127 1.3 130 +715 2 10705 99616 2.14714 299.14714 150.64714 15064.71471 2.14714 299.14716 150.64714 15064.71458 2.14714 299.14714 150.64714 15064.71400 2020-01-01 2020-01-02 2020-01-01 00:11:55 2020-01-02 03:40:16 2020-01-01 00:11:55.000 2020-01-02 03:40:16.000 715 99616 50165.5 5016550 715 99616 50165.5 5016550 -32455 32683 4945.66 494566 -128 127 -0.26 -26 +716 2 10706 99617 2.15015 299.15015 150.65015 15065.01501 2.15015 299.15015 150.65014 15065.01472 2.15015 299.15015 150.65015 15065.01500 2020-01-01 2020-01-02 2020-01-01 00:11:56 2020-01-02 03:40:17 2020-01-01 00:11:56.000 2020-01-02 03:40:17.000 716 99617 50166.5 5016650 716 99617 50166.5 5016650 -32454 32684 4946.66 494666 -128 127 -1.82 -182 +717 2 10707 99618 2.15315 299.15315 150.65315 15065.31531 2.15315 299.15317 150.65315 15065.31547 2.15315 299.15315 150.65315 15065.31500 2020-01-01 2020-01-02 2020-01-01 00:11:57 2020-01-02 03:40:18 2020-01-01 00:11:57.000 2020-01-02 03:40:18.000 717 99618 50167.5 5016750 717 99618 50167.5 5016750 -32453 32685 4947.66 494766 -128 123 -3.38 -338 +718 2 10708 99619 2.15615 299.15615 150.65615 15065.61561 2.15615 299.15616 150.65615 15065.61578 2.15615 299.15615 150.65615 15065.61500 2020-01-01 2020-01-02 2020-01-01 00:11:58 2020-01-02 03:40:19 2020-01-01 00:11:58.000 2020-01-02 03:40:19.000 718 99619 50168.5 5016850 718 99619 50168.5 5016850 -32452 32686 4948.66 494866 -127 124 -2.38 -238 719 2 10709 99620 2.15915 299.15915 150.65915 15065.91591 2.15915 299.15915 150.65915 15065.91595 2.15915 299.15915 150.65915 15065.91500 2020-01-01 2020-01-02 2020-01-01 00:11:59 2020-01-02 03:40:20 2020-01-01 00:11:59.000 2020-01-02 03:40:20.000 719 99620 50169.5 5016950 719 99620 50169.5 5016950 -32451 32687 4949.66 494966 -126 125 -1.38 -138 72 2 10062 99972 0.21621 300.21621 150.21621 15171.83783 0.21621 300.21622 150.21621 15171.83791 0.21621 300.21621 150.21621 15171.83721 2020-01-01 2020-01-02 2020-01-01 00:01:12 2020-01-02 03:46:12 2020-01-01 00:01:12.000 2020-01-02 03:46:12.000 72 99972 50022 5052222 72 99972 50022 5052222 -32497 32438 4601.009900990099 464702 -128 127 -1.9207920792079207 -194 -720 2 10710 99621 2.16216 299.16216 150.66216 15066.21621 2.16216 299.16217 150.66216 15066.21606 2.16216 299.16216 150.6621599999999 15066.21600 2020-01-01 2020-01-02 2020-01-01 00:12:00 2020-01-02 03:40:21 2020-01-01 00:12:00.000 2020-01-02 03:40:21.000 720 99621 50170.5 5017050 720 99621 50170.5 5017050 -32450 32688 4950.66 495066 -125 126 -0.38 -38 -721 2 10711 99622 2.16516 299.16516 150.66516 15066.51651 2.16516 299.16516 150.66516 15066.51635 2.16516 299.16516 150.66515999999996 15066.51600 2020-01-01 2020-01-02 2020-01-01 00:12:01 2020-01-02 03:40:22 2020-01-01 00:12:01.000 2020-01-02 03:40:22.000 721 99622 50171.5 5017150 721 99622 50171.5 5017150 -32449 32689 4951.66 495166 -124 127 0.62 62 -722 2 10712 99623 2.16816 299.16816 150.66816 15066.81681 2.16816 299.16818 150.66816 15066.81695 2.16816 299.16816 150.66815999999992 15066.81600 2020-01-01 2020-01-02 2020-01-01 00:12:02 2020-01-02 03:40:23 2020-01-01 00:12:02.000 2020-01-02 03:40:23.000 722 99623 50172.5 5017250 722 99623 50172.5 5017250 -32448 32690 4952.66 495266 -128 127 -0.94 -94 -723 2 10713 99624 2.17117 299.17117 150.67117 15067.11711 2.17117 299.17117 150.67117 15067.11724 2.17117 299.17117 150.6711699999998 15067.11700 2020-01-01 2020-01-02 2020-01-01 00:12:03 2020-01-02 03:40:24 2020-01-01 00:12:03.000 2020-01-02 03:40:24.000 723 99624 50173.5 5017350 723 99624 50173.5 5017350 -32447 32691 4953.66 495366 -128 123 -2.5 -250 -724 2 10714 99625 2.17417 299.17417 150.67417 15067.41741 2.17417 299.17416 150.67417 15067.41742 2.17417 299.17417 150.67417000000015 15067.41700 2020-01-01 2020-01-02 2020-01-01 00:12:04 2020-01-02 03:40:25 2020-01-01 00:12:04.000 2020-01-02 03:40:25.000 724 99625 50174.5 5017450 724 99625 50174.5 5017450 -32446 32692 4954.66 495466 -127 124 -1.5 -150 -725 2 10715 99626 2.17717 299.17717 150.67717 15067.71771 2.17717 299.1772 150.67717 15067.71752 2.17717 299.17717 150.67717000000016 15067.71700 2020-01-01 2020-01-02 2020-01-01 00:12:05 2020-01-02 03:40:26 2020-01-01 00:12:05.000 2020-01-02 03:40:26.000 725 99626 50175.5 5017550 725 99626 50175.5 5017550 -32445 32693 4955.66 495566 -126 125 -0.5 -50 -726 2 10716 99627 2.18018 299.18018 150.68018 15068.01801 2.18018 299.18018 150.68017 15068.01782 2.18018 299.18018 150.68017999999992 15068.01800 2020-01-01 2020-01-02 2020-01-01 00:12:06 2020-01-02 03:40:27 2020-01-01 00:12:06.000 2020-01-02 03:40:27.000 726 99627 50176.5 5017650 726 99627 50176.5 5017650 -32444 32694 4956.66 495666 -125 126 0.5 50 +720 2 10710 99621 2.16216 299.16216 150.66216 15066.21621 2.16216 299.16217 150.66216 15066.21606 2.16216 299.16216 150.66216 15066.21600 2020-01-01 2020-01-02 2020-01-01 00:12:00 2020-01-02 03:40:21 2020-01-01 00:12:00.000 2020-01-02 03:40:21.000 720 99621 50170.5 5017050 720 99621 50170.5 5017050 -32450 32688 4950.66 495066 -125 126 -0.38 -38 +721 2 10711 99622 2.16516 299.16516 150.66516 15066.51651 2.16516 299.16516 150.66516 15066.51635 2.16516 299.16516 150.66516 15066.51600 2020-01-01 2020-01-02 2020-01-01 00:12:01 2020-01-02 03:40:22 2020-01-01 00:12:01.000 2020-01-02 03:40:22.000 721 99622 50171.5 5017150 721 99622 50171.5 5017150 -32449 32689 4951.66 495166 -124 127 0.62 62 +722 2 10712 99623 2.16816 299.16816 150.66816 15066.81681 2.16816 299.16818 150.66816 15066.81695 2.16816 299.16816 150.66816 15066.81600 2020-01-01 2020-01-02 2020-01-01 00:12:02 2020-01-02 03:40:23 2020-01-01 00:12:02.000 2020-01-02 03:40:23.000 722 99623 50172.5 5017250 722 99623 50172.5 5017250 -32448 32690 4952.66 495266 -128 127 -0.94 -94 +723 2 10713 99624 2.17117 299.17117 150.67117 15067.11711 2.17117 299.17117 150.67117 15067.11724 2.17117 299.17117 150.67117 15067.11700 2020-01-01 2020-01-02 2020-01-01 00:12:03 2020-01-02 03:40:24 2020-01-01 00:12:03.000 2020-01-02 03:40:24.000 723 99624 50173.5 5017350 723 99624 50173.5 5017350 -32447 32691 4953.66 495366 -128 123 -2.5 -250 +724 2 10714 99625 2.17417 299.17417 150.67417 15067.41741 2.17417 299.17416 150.67417 15067.41742 2.17417 299.17417 150.67417 15067.41700 2020-01-01 2020-01-02 2020-01-01 00:12:04 2020-01-02 03:40:25 2020-01-01 00:12:04.000 2020-01-02 03:40:25.000 724 99625 50174.5 5017450 724 99625 50174.5 5017450 -32446 32692 4954.66 495466 -127 124 -1.5 -150 +725 2 10715 99626 2.17717 299.17717 150.67717 15067.71771 2.17717 299.1772 150.67717 15067.71752 2.17717 299.17717 150.67717000000002 15067.71700 2020-01-01 2020-01-02 2020-01-01 00:12:05 2020-01-02 03:40:26 2020-01-01 00:12:05.000 2020-01-02 03:40:26.000 725 99626 50175.5 5017550 725 99626 50175.5 5017550 -32445 32693 4955.66 495566 -126 125 -0.5 -50 +726 2 10716 99627 2.18018 299.18018 150.68018 15068.01801 2.18018 299.18018 150.68017 15068.01782 2.18018 299.18018 150.68018 15068.01800 2020-01-01 2020-01-02 2020-01-01 00:12:06 2020-01-02 03:40:27 2020-01-01 00:12:06.000 2020-01-02 03:40:27.000 726 99627 50176.5 5017650 726 99627 50176.5 5017650 -32444 32694 4956.66 495666 -125 126 0.5 50 727 2 10717 99628 2.18318 299.18318 150.68318 15068.31831 2.18318 299.1832 150.68318 15068.31842 2.18318 299.18318 150.68318 15068.31800 2020-01-01 2020-01-02 2020-01-01 00:12:07 2020-01-02 03:40:28 2020-01-01 00:12:07.000 2020-01-02 03:40:28.000 727 99628 50177.5 5017750 727 99628 50177.5 5017750 -32443 32695 4957.66 495766 -124 127 1.5 150 -728 2 10718 99629 2.18618 299.18618 150.68618 15068.61861 2.18618 299.1862 150.68618 15068.61875 2.18618 299.18618 150.6861800000001 15068.61800 2020-01-01 2020-01-02 2020-01-01 00:12:08 2020-01-02 03:40:29 2020-01-01 00:12:08.000 2020-01-02 03:40:29.000 728 99629 50178.5 5017850 728 99629 50178.5 5017850 -32442 32696 4958.66 495866 -128 127 -0.06 -6 -729 2 10719 99630 2.18918 299.18918 150.68918 15068.91891 2.18918 299.18918 150.68918 15068.91889 2.18918 299.18918 150.6891799999999 15068.91800 2020-01-01 2020-01-02 2020-01-01 00:12:09 2020-01-02 03:40:30 2020-01-01 00:12:09.000 2020-01-02 03:40:30.000 729 99630 50179.5 5017950 729 99630 50179.5 5017950 -32441 32697 4959.66 495966 -128 123 -1.62 -162 -73 2 10063 99973 0.21921 300.21921 150.21921 15172.14114 0.21921 300.2192 150.21921 15172.14121 0.21921 300.21921 150.21920999999992 15172.14021 2020-01-01 2020-01-02 2020-01-01 00:01:13 2020-01-02 03:46:13 2020-01-01 00:01:13.000 2020-01-02 03:46:13.000 73 99973 50023 5052323 73 99973 50023 5052323 -32496 32439 4602.009900990099 464803 -128 127 -3.4554455445544554 -349 -730 2 10720 99631 2.19219 299.19219 150.69219 15069.21921 2.19219 299.1922 150.69219 15069.21965 2.19219 299.19219 150.6921899999999 15069.21900 2020-01-01 2020-01-02 2020-01-01 00:12:10 2020-01-02 03:40:31 2020-01-01 00:12:10.000 2020-01-02 03:40:31.000 730 99631 50180.5 5018050 730 99631 50180.5 5018050 -32440 32698 4960.66 496066 -127 124 -0.62 -62 -731 2 10721 99632 2.19519 299.19519 150.69519 15069.51951 2.19519 299.1952 150.69519 15069.51928 2.19519 299.19519 150.6951900000002 15069.51900 2020-01-01 2020-01-02 2020-01-01 00:12:11 2020-01-02 03:40:32 2020-01-01 00:12:11.000 2020-01-02 03:40:32.000 731 99632 50181.5 5018150 731 99632 50181.5 5018150 -32439 32699 4961.66 496166 -126 125 0.38 38 -732 2 10722 99633 2.19819 299.19819 150.69819 15069.81981 2.19819 299.1982 150.69819 15069.81988 2.19819 299.19819 150.69818999999984 15069.81900 2020-01-01 2020-01-02 2020-01-01 00:12:12 2020-01-02 03:40:33 2020-01-01 00:12:12.000 2020-01-02 03:40:33.000 732 99633 50182.5 5018250 732 99633 50182.5 5018250 -32438 32700 4962.66 496266 -125 126 1.38 138 -733 2 10723 99634 2.2012 299.2012 150.7012 15070.12012 2.2012 299.2012 150.7012 15070.12022 2.20120 299.20120 150.70120000000003 15070.12000 2020-01-01 2020-01-02 2020-01-01 00:12:13 2020-01-02 03:40:34 2020-01-01 00:12:13.000 2020-01-02 03:40:34.000 733 99634 50183.5 5018350 733 99634 50183.5 5018350 -32437 32701 4963.66 496366 -124 127 2.38 238 -734 2 10724 99635 2.2042 299.2042 150.7042 15070.42042 2.2042 299.2042 150.7042 15070.42036 2.20420 299.20420 150.7042000000001 15070.42000 2020-01-01 2020-01-02 2020-01-01 00:12:14 2020-01-02 03:40:35 2020-01-01 00:12:14.000 2020-01-02 03:40:35.000 734 99635 50184.5 5018450 734 99635 50184.5 5018450 -32436 32702 4964.66 496466 -128 127 0.82 82 -735 2 10725 99636 2.2072 299.2072 150.7072 15070.72072 2.2072 299.2072 150.70721 15070.72111 2.20720 299.20720 150.70720000000009 15070.72000 2020-01-01 2020-01-02 2020-01-01 00:12:15 2020-01-02 03:40:36 2020-01-01 00:12:15.000 2020-01-02 03:40:36.000 735 99636 50185.5 5018550 735 99636 50185.5 5018550 -32435 32703 4965.66 496566 -128 127 -0.74 -74 -736 2 10726 99637 2.21021 299.21021 150.71021 15071.02102 2.21021 299.2102 150.7102 15071.02076 2.21021 299.21021 150.71020999999996 15071.02100 2020-01-01 2020-01-02 2020-01-01 00:12:16 2020-01-02 03:40:37 2020-01-01 00:12:16.000 2020-01-02 03:40:37.000 736 99637 50186.5 5018650 736 99637 50186.5 5018650 -32434 32704 4966.66 496666 -128 124 -2.3 -230 -737 2 10727 99638 2.21321 299.21321 150.71321 15071.32132 2.21321 299.21323 150.71321 15071.32139 2.21321 299.21321 150.71320999999992 15071.32100 2020-01-01 2020-01-02 2020-01-01 00:12:17 2020-01-02 03:40:38 2020-01-01 00:12:17.000 2020-01-02 03:40:38.000 737 99638 50187.5 5018750 737 99638 50187.5 5018750 -32433 32705 4967.66 496766 -127 125 -1.3 -130 +728 2 10718 99629 2.18618 299.18618 150.68618 15068.61861 2.18618 299.1862 150.68618 15068.61875 2.18618 299.18618 150.68618 15068.61800 2020-01-01 2020-01-02 2020-01-01 00:12:08 2020-01-02 03:40:29 2020-01-01 00:12:08.000 2020-01-02 03:40:29.000 728 99629 50178.5 5017850 728 99629 50178.5 5017850 -32442 32696 4958.66 495866 -128 127 -0.06 -6 +729 2 10719 99630 2.18918 299.18918 150.68918 15068.91891 2.18918 299.18918 150.68918 15068.91889 2.18918 299.18918 150.68918 15068.91800 2020-01-01 2020-01-02 2020-01-01 00:12:09 2020-01-02 03:40:30 2020-01-01 00:12:09.000 2020-01-02 03:40:30.000 729 99630 50179.5 5017950 729 99630 50179.5 5017950 -32441 32697 4959.66 495966 -128 123 -1.62 -162 +73 2 10063 99973 0.21921 300.21921 150.21921 15172.14114 0.21921 300.2192 150.21921 15172.14121 0.21921 300.21921 150.21921 15172.14021 2020-01-01 2020-01-02 2020-01-01 00:01:13 2020-01-02 03:46:13 2020-01-01 00:01:13.000 2020-01-02 03:46:13.000 73 99973 50023 5052323 73 99973 50023 5052323 -32496 32439 4602.009900990099 464803 -128 127 -3.4554455445544554 -349 +730 2 10720 99631 2.19219 299.19219 150.69219 15069.21921 2.19219 299.1922 150.69219 15069.21965 2.19219 299.19219 150.69218999999998 15069.21900 2020-01-01 2020-01-02 2020-01-01 00:12:10 2020-01-02 03:40:31 2020-01-01 00:12:10.000 2020-01-02 03:40:31.000 730 99631 50180.5 5018050 730 99631 50180.5 5018050 -32440 32698 4960.66 496066 -127 124 -0.62 -62 +731 2 10721 99632 2.19519 299.19519 150.69519 15069.51951 2.19519 299.1952 150.69519 15069.51928 2.19519 299.19519 150.69519 15069.51900 2020-01-01 2020-01-02 2020-01-01 00:12:11 2020-01-02 03:40:32 2020-01-01 00:12:11.000 2020-01-02 03:40:32.000 731 99632 50181.5 5018150 731 99632 50181.5 5018150 -32439 32699 4961.66 496166 -126 125 0.38 38 +732 2 10722 99633 2.19819 299.19819 150.69819 15069.81981 2.19819 299.1982 150.69819 15069.81988 2.19819 299.19819 150.69818999999998 15069.81900 2020-01-01 2020-01-02 2020-01-01 00:12:12 2020-01-02 03:40:33 2020-01-01 00:12:12.000 2020-01-02 03:40:33.000 732 99633 50182.5 5018250 732 99633 50182.5 5018250 -32438 32700 4962.66 496266 -125 126 1.38 138 +733 2 10723 99634 2.2012 299.2012 150.7012 15070.12012 2.2012 299.2012 150.7012 15070.12022 2.20120 299.20120 150.7012 15070.12000 2020-01-01 2020-01-02 2020-01-01 00:12:13 2020-01-02 03:40:34 2020-01-01 00:12:13.000 2020-01-02 03:40:34.000 733 99634 50183.5 5018350 733 99634 50183.5 5018350 -32437 32701 4963.66 496366 -124 127 2.38 238 +734 2 10724 99635 2.2042 299.2042 150.7042 15070.42042 2.2042 299.2042 150.7042 15070.42036 2.20420 299.20420 150.70420000000001 15070.42000 2020-01-01 2020-01-02 2020-01-01 00:12:14 2020-01-02 03:40:35 2020-01-01 00:12:14.000 2020-01-02 03:40:35.000 734 99635 50184.5 5018450 734 99635 50184.5 5018450 -32436 32702 4964.66 496466 -128 127 0.82 82 +735 2 10725 99636 2.2072 299.2072 150.7072 15070.72072 2.2072 299.2072 150.70721 15070.72111 2.20720 299.20720 150.7072 15070.72000 2020-01-01 2020-01-02 2020-01-01 00:12:15 2020-01-02 03:40:36 2020-01-01 00:12:15.000 2020-01-02 03:40:36.000 735 99636 50185.5 5018550 735 99636 50185.5 5018550 -32435 32703 4965.66 496566 -128 127 -0.74 -74 +736 2 10726 99637 2.21021 299.21021 150.71021 15071.02102 2.21021 299.2102 150.7102 15071.02076 2.21021 299.21021 150.71021000000002 15071.02100 2020-01-01 2020-01-02 2020-01-01 00:12:16 2020-01-02 03:40:37 2020-01-01 00:12:16.000 2020-01-02 03:40:37.000 736 99637 50186.5 5018650 736 99637 50186.5 5018650 -32434 32704 4966.66 496666 -128 124 -2.3 -230 +737 2 10727 99638 2.21321 299.21321 150.71321 15071.32132 2.21321 299.21323 150.71321 15071.32139 2.21321 299.21321 150.71321 15071.32100 2020-01-01 2020-01-02 2020-01-01 00:12:17 2020-01-02 03:40:38 2020-01-01 00:12:17.000 2020-01-02 03:40:38.000 737 99638 50187.5 5018750 737 99638 50187.5 5018750 -32433 32705 4967.66 496766 -127 125 -1.3 -130 738 2 10728 99639 2.21621 299.21621 150.71621 15071.62162 2.21621 299.21622 150.71621 15071.62169 2.21621 299.21621 150.71621 15071.62100 2020-01-01 2020-01-02 2020-01-01 00:12:18 2020-01-02 03:40:39 2020-01-01 00:12:18.000 2020-01-02 03:40:39.000 738 99639 50188.5 5018850 738 99639 50188.5 5018850 -32432 32706 4968.66 496866 -126 126 -0.3 -30 -739 2 10729 99640 2.21921 299.21921 150.71921 15071.92192 2.21921 299.2192 150.71921 15071.92199 2.21921 299.21921 150.71920999999995 15071.92100 2020-01-01 2020-01-02 2020-01-01 00:12:19 2020-01-02 03:40:40 2020-01-01 00:12:19.000 2020-01-02 03:40:40.000 739 99640 50189.5 5018950 739 99640 50189.5 5018950 -32431 32707 4969.66 496966 -125 127 0.7 70 -74 2 10064 99974 0.22222 300.22222 150.22222 15172.44444 0.22222 300.22223 150.22222 15172.4448 0.22222 300.22222 150.22221999999985 15172.44422 2020-01-01 2020-01-02 2020-01-01 00:01:14 2020-01-02 03:46:14 2020-01-01 00:01:14.000 2020-01-02 03:46:14.000 74 99974 50024 5052424 74 99974 50024 5052424 -32495 32440 4603.009900990099 464904 -128 123 -4.99009900990099 -504 -740 2 10730 99641 2.22222 299.22222 150.72222 15072.22222 2.22222 299.22223 150.72222 15072.22258 2.22222 299.22222 150.72221999999985 15072.22200 2020-01-01 2020-01-02 2020-01-01 00:12:20 2020-01-02 03:40:41 2020-01-01 00:12:20.000 2020-01-02 03:40:41.000 740 99641 50190.5 5019050 740 99641 50190.5 5019050 -32430 32708 4970.66 497066 -128 127 -0.86 -86 -741 2 10731 99642 2.22522 299.22522 150.72522 15072.52252 2.22522 299.22522 150.72522 15072.52223 2.22522 299.22522 150.7252200000001 15072.52200 2020-01-01 2020-01-02 2020-01-01 00:12:21 2020-01-02 03:40:42 2020-01-01 00:12:21.000 2020-01-02 03:40:42.000 741 99642 50191.5 5019150 741 99642 50191.5 5019150 -32429 32709 4971.66 497166 -128 127 -2.42 -242 -742 2 10732 99643 2.22822 299.22822 150.72822 15072.82282 2.22822 299.22824 150.72822 15072.82286 2.22822 299.22822 150.7282200000002 15072.82200 2020-01-01 2020-01-02 2020-01-01 00:12:22 2020-01-02 03:40:43 2020-01-01 00:12:22.000 2020-01-02 03:40:43.000 742 99643 50192.5 5019250 742 99643 50192.5 5019250 -32428 32710 4972.66 497266 -128 123 -3.98 -398 -743 2 10733 99644 2.23123 299.23123 150.73123 15073.12312 2.23123 299.23123 150.73123 15073.12316 2.23123 299.23123 150.73122999999993 15073.12300 2020-01-01 2020-01-02 2020-01-01 00:12:23 2020-01-02 03:40:44 2020-01-01 00:12:23.000 2020-01-02 03:40:44.000 743 99644 50193.5 5019350 743 99644 50193.5 5019350 -32427 32711 4973.66 497366 -127 124 -2.98 -298 +739 2 10729 99640 2.21921 299.21921 150.71921 15071.92192 2.21921 299.2192 150.71921 15071.92199 2.21921 299.21921 150.71921 15071.92100 2020-01-01 2020-01-02 2020-01-01 00:12:19 2020-01-02 03:40:40 2020-01-01 00:12:19.000 2020-01-02 03:40:40.000 739 99640 50189.5 5018950 739 99640 50189.5 5018950 -32431 32707 4969.66 496966 -125 127 0.7 70 +74 2 10064 99974 0.22222 300.22222 150.22222 15172.44444 0.22222 300.22223 150.22222 15172.4448 0.22222 300.22222 150.22222 15172.44422 2020-01-01 2020-01-02 2020-01-01 00:01:14 2020-01-02 03:46:14 2020-01-01 00:01:14.000 2020-01-02 03:46:14.000 74 99974 50024 5052424 74 99974 50024 5052424 -32495 32440 4603.009900990099 464904 -128 123 -4.99009900990099 -504 +740 2 10730 99641 2.22222 299.22222 150.72222 15072.22222 2.22222 299.22223 150.72222 15072.22258 2.22222 299.22222 150.72222 15072.22200 2020-01-01 2020-01-02 2020-01-01 00:12:20 2020-01-02 03:40:41 2020-01-01 00:12:20.000 2020-01-02 03:40:41.000 740 99641 50190.5 5019050 740 99641 50190.5 5019050 -32430 32708 4970.66 497066 -128 127 -0.86 -86 +741 2 10731 99642 2.22522 299.22522 150.72522 15072.52252 2.22522 299.22522 150.72522 15072.52223 2.22522 299.22522 150.72522 15072.52200 2020-01-01 2020-01-02 2020-01-01 00:12:21 2020-01-02 03:40:42 2020-01-01 00:12:21.000 2020-01-02 03:40:42.000 741 99642 50191.5 5019150 741 99642 50191.5 5019150 -32429 32709 4971.66 497166 -128 127 -2.42 -242 +742 2 10732 99643 2.22822 299.22822 150.72822 15072.82282 2.22822 299.22824 150.72822 15072.82286 2.22822 299.22822 150.72822 15072.82200 2020-01-01 2020-01-02 2020-01-01 00:12:22 2020-01-02 03:40:43 2020-01-01 00:12:22.000 2020-01-02 03:40:43.000 742 99643 50192.5 5019250 742 99643 50192.5 5019250 -32428 32710 4972.66 497266 -128 123 -3.98 -398 +743 2 10733 99644 2.23123 299.23123 150.73123 15073.12312 2.23123 299.23123 150.73123 15073.12316 2.23123 299.23123 150.73122999999998 15073.12300 2020-01-01 2020-01-02 2020-01-01 00:12:23 2020-01-02 03:40:44 2020-01-01 00:12:23.000 2020-01-02 03:40:44.000 743 99644 50193.5 5019350 743 99644 50193.5 5019350 -32427 32711 4973.66 497366 -127 124 -2.98 -298 744 2 10734 99645 2.23423 299.23423 150.73423 15073.42342 2.23423 299.23422 150.73423 15073.42345 2.23423 299.23423 150.73423 15073.42300 2020-01-01 2020-01-02 2020-01-01 00:12:24 2020-01-02 03:40:45 2020-01-01 00:12:24.000 2020-01-02 03:40:45.000 744 99645 50194.5 5019450 744 99645 50194.5 5019450 -32426 32712 4974.66 497466 -126 125 -1.98 -198 -745 2 10735 99646 2.23723 299.23723 150.73723 15073.72372 2.23723 299.23724 150.73724 15073.72405 2.23723 299.23723 150.73723000000012 15073.72300 2020-01-01 2020-01-02 2020-01-01 00:12:25 2020-01-02 03:40:46 2020-01-01 00:12:25.000 2020-01-02 03:40:46.000 745 99646 50195.5 5019550 745 99646 50195.5 5019550 -32425 32713 4975.66 497566 -125 126 -0.98 -98 -746 2 10736 99647 2.24024 299.24024 150.74024 15074.02402 2.24024 299.24023 150.74023 15074.02373 2.24024 299.24024 150.7402399999999 15074.02400 2020-01-01 2020-01-02 2020-01-01 00:12:26 2020-01-02 03:40:47 2020-01-01 00:12:26.000 2020-01-02 03:40:47.000 746 99647 50196.5 5019650 746 99647 50196.5 5019650 -32424 32714 4976.66 497666 -124 127 0.02 2 -747 2 10737 99648 2.24324 299.24324 150.74324 15074.32432 2.24324 299.24326 150.74324 15074.32433 2.24324 299.24324 150.74323999999984 15074.32400 2020-01-01 2020-01-02 2020-01-01 00:12:27 2020-01-02 03:40:48 2020-01-01 00:12:27.000 2020-01-02 03:40:48.000 747 99648 50197.5 5019750 747 99648 50197.5 5019750 -32423 32715 4977.66 497766 -128 127 -1.54 -154 -748 2 10738 99649 2.24624 299.24624 150.74624 15074.62462 2.24624 299.24625 150.74624 15074.62463 2.24624 299.24624 150.74624000000017 15074.62400 2020-01-01 2020-01-02 2020-01-01 00:12:28 2020-01-02 03:40:49 2020-01-01 00:12:28.000 2020-01-02 03:40:49.000 748 99649 50198.5 5019850 748 99649 50198.5 5019850 -32422 32716 4978.66 497866 -128 123 -3.1 -310 -749 2 10739 99650 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74923999999987 15074.92400 2020-01-01 2020-01-02 2020-01-01 00:12:29 2020-01-02 03:40:50 2020-01-01 00:12:29.000 2020-01-02 03:40:50.000 749 99650 50199.5 5019950 749 99650 50199.5 5019950 -32421 32717 4979.66 497966 -127 124 -2.1 -210 -75 2 10065 99975 0.22522 300.22522 150.22522 15172.74774 0.22522 300.22522 150.22522 15172.74745 0.22522 300.22522 150.2252200000001 15172.74722 2020-01-01 2020-01-02 2020-01-01 00:01:15 2020-01-02 03:46:15 2020-01-01 00:01:15.000 2020-01-02 03:46:15.000 75 99975 50025 5052525 75 99975 50025 5052525 -32494 32441 4604.009900990099 465005 -127 124 -3.99009900990099 -403 -750 2 10740 99651 2.25225 299.25225 150.75225 15075.22522 2.25225 299.25226 150.75225 15075.22552 2.25225 299.25225 150.75224999999978 15075.22500 2020-01-01 2020-01-02 2020-01-01 00:12:30 2020-01-02 03:40:51 2020-01-01 00:12:30.000 2020-01-02 03:40:51.000 750 99651 50200.5 5020050 750 99651 50200.5 5020050 -32420 32718 4980.66 498066 -126 125 -1.1 -110 -751 2 10741 99652 2.25525 299.25525 150.75525 15075.52552 2.25525 299.25525 150.75525 15075.5252 2.25525 299.25525 150.7552500000001 15075.52500 2020-01-01 2020-01-02 2020-01-01 00:12:31 2020-01-02 03:40:52 2020-01-01 00:12:31.000 2020-01-02 03:40:52.000 751 99652 50201.5 5020150 751 99652 50201.5 5020150 -32419 32719 4981.66 498166 -125 126 -0.1 -10 -752 2 10742 99653 2.25825 299.25825 150.75825 15075.82582 2.25825 299.25827 150.75825 15075.8258 2.25825 299.25825 150.75825000000015 15075.82500 2020-01-01 2020-01-02 2020-01-01 00:12:32 2020-01-02 03:40:53 2020-01-01 00:12:32.000 2020-01-02 03:40:53.000 752 99653 50202.5 5020250 752 99653 50202.5 5020250 -32418 32720 4982.66 498266 -124 127 0.9 90 -753 2 10743 99654 2.26126 299.26126 150.76126 15076.12612 2.26126 299.26126 150.76126 15076.12609 2.26126 299.26126 150.76125999999994 15076.12600 2020-01-01 2020-01-02 2020-01-01 00:12:33 2020-01-02 03:40:54 2020-01-01 00:12:33.000 2020-01-02 03:40:54.000 753 99654 50203.5 5020350 753 99654 50203.5 5020350 -32417 32721 4983.66 498366 -128 127 -0.66 -66 -754 2 10744 99655 2.26426 299.26426 150.76426 15076.42642 2.26426 299.26425 150.76426 15076.4264 2.26426 299.26426 150.76425999999995 15076.42600 2020-01-01 2020-01-02 2020-01-01 00:12:34 2020-01-02 03:40:55 2020-01-01 00:12:34.000 2020-01-02 03:40:55.000 754 99655 50204.5 5020450 754 99655 50204.5 5020450 -32416 32722 4984.66 498466 -128 123 -2.22 -222 +745 2 10735 99646 2.23723 299.23723 150.73723 15073.72372 2.23723 299.23724 150.73724 15073.72405 2.23723 299.23723 150.73723 15073.72300 2020-01-01 2020-01-02 2020-01-01 00:12:25 2020-01-02 03:40:46 2020-01-01 00:12:25.000 2020-01-02 03:40:46.000 745 99646 50195.5 5019550 745 99646 50195.5 5019550 -32425 32713 4975.66 497566 -125 126 -0.98 -98 +746 2 10736 99647 2.24024 299.24024 150.74024 15074.02402 2.24024 299.24023 150.74023 15074.02373 2.24024 299.24024 150.74024 15074.02400 2020-01-01 2020-01-02 2020-01-01 00:12:26 2020-01-02 03:40:47 2020-01-01 00:12:26.000 2020-01-02 03:40:47.000 746 99647 50196.5 5019650 746 99647 50196.5 5019650 -32424 32714 4976.66 497666 -124 127 0.02 2 +747 2 10737 99648 2.24324 299.24324 150.74324 15074.32432 2.24324 299.24326 150.74324 15074.32433 2.24324 299.24324 150.74324000000001 15074.32400 2020-01-01 2020-01-02 2020-01-01 00:12:27 2020-01-02 03:40:48 2020-01-01 00:12:27.000 2020-01-02 03:40:48.000 747 99648 50197.5 5019750 747 99648 50197.5 5019750 -32423 32715 4977.66 497766 -128 127 -1.54 -154 +748 2 10738 99649 2.24624 299.24624 150.74624 15074.62462 2.24624 299.24625 150.74624 15074.62463 2.24624 299.24624 150.74624 15074.62400 2020-01-01 2020-01-02 2020-01-01 00:12:28 2020-01-02 03:40:49 2020-01-01 00:12:28.000 2020-01-02 03:40:49.000 748 99649 50198.5 5019850 748 99649 50198.5 5019850 -32422 32716 4978.66 497866 -128 123 -3.1 -310 +749 2 10739 99650 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924 15074.92492 2.24924 299.24924 150.74924000000001 15074.92400 2020-01-01 2020-01-02 2020-01-01 00:12:29 2020-01-02 03:40:50 2020-01-01 00:12:29.000 2020-01-02 03:40:50.000 749 99650 50199.5 5019950 749 99650 50199.5 5019950 -32421 32717 4979.66 497966 -127 124 -2.1 -210 +75 2 10065 99975 0.22522 300.22522 150.22522 15172.74774 0.22522 300.22522 150.22522 15172.74745 0.22522 300.22522 150.22521999999998 15172.74722 2020-01-01 2020-01-02 2020-01-01 00:01:15 2020-01-02 03:46:15 2020-01-01 00:01:15.000 2020-01-02 03:46:15.000 75 99975 50025 5052525 75 99975 50025 5052525 -32494 32441 4604.009900990099 465005 -127 124 -3.99009900990099 -403 +750 2 10740 99651 2.25225 299.25225 150.75225 15075.22522 2.25225 299.25226 150.75225 15075.22552 2.25225 299.25225 150.75225 15075.22500 2020-01-01 2020-01-02 2020-01-01 00:12:30 2020-01-02 03:40:51 2020-01-01 00:12:30.000 2020-01-02 03:40:51.000 750 99651 50200.5 5020050 750 99651 50200.5 5020050 -32420 32718 4980.66 498066 -126 125 -1.1 -110 +751 2 10741 99652 2.25525 299.25525 150.75525 15075.52552 2.25525 299.25525 150.75525 15075.5252 2.25525 299.25525 150.75525 15075.52500 2020-01-01 2020-01-02 2020-01-01 00:12:31 2020-01-02 03:40:52 2020-01-01 00:12:31.000 2020-01-02 03:40:52.000 751 99652 50201.5 5020150 751 99652 50201.5 5020150 -32419 32719 4981.66 498166 -125 126 -0.1 -10 +752 2 10742 99653 2.25825 299.25825 150.75825 15075.82582 2.25825 299.25827 150.75825 15075.8258 2.25825 299.25825 150.75825 15075.82500 2020-01-01 2020-01-02 2020-01-01 00:12:32 2020-01-02 03:40:53 2020-01-01 00:12:32.000 2020-01-02 03:40:53.000 752 99653 50202.5 5020250 752 99653 50202.5 5020250 -32418 32720 4982.66 498266 -124 127 0.9 90 +753 2 10743 99654 2.26126 299.26126 150.76126 15076.12612 2.26126 299.26126 150.76126 15076.12609 2.26126 299.26126 150.76126 15076.12600 2020-01-01 2020-01-02 2020-01-01 00:12:33 2020-01-02 03:40:54 2020-01-01 00:12:33.000 2020-01-02 03:40:54.000 753 99654 50203.5 5020350 753 99654 50203.5 5020350 -32417 32721 4983.66 498366 -128 127 -0.66 -66 +754 2 10744 99655 2.26426 299.26426 150.76426 15076.42642 2.26426 299.26425 150.76426 15076.4264 2.26426 299.26426 150.76426 15076.42600 2020-01-01 2020-01-02 2020-01-01 00:12:34 2020-01-02 03:40:55 2020-01-01 00:12:34.000 2020-01-02 03:40:55.000 754 99655 50204.5 5020450 754 99655 50204.5 5020450 -32416 32722 4984.66 498466 -128 123 -2.22 -222 755 2 10745 99656 2.26726 299.26726 150.76726 15076.72672 2.26726 299.26727 150.76727 15076.72703 2.26726 299.26726 150.76726 15076.72600 2020-01-01 2020-01-02 2020-01-01 00:12:35 2020-01-02 03:40:56 2020-01-01 00:12:35.000 2020-01-02 03:40:56.000 755 99656 50205.5 5020550 755 99656 50205.5 5020550 -32415 32723 4985.66 498566 -127 124 -1.22 -122 -756 2 10746 99657 2.27027 299.27027 150.77027 15077.02702 2.27027 299.27026 150.77026 15077.02667 2.27027 299.27027 150.7702700000002 15077.02700 2020-01-01 2020-01-02 2020-01-01 00:12:36 2020-01-02 03:40:57 2020-01-01 00:12:36.000 2020-01-02 03:40:57.000 756 99657 50206.5 5020650 756 99657 50206.5 5020650 -32414 32724 4986.66 498666 -126 125 -0.22 -22 -757 2 10747 99658 2.27327 299.27327 150.77327 15077.32732 2.27327 299.2733 150.77327 15077.32727 2.27327 299.27327 150.77326999999985 15077.32700 2020-01-01 2020-01-02 2020-01-01 00:12:37 2020-01-02 03:40:58 2020-01-01 00:12:37.000 2020-01-02 03:40:58.000 757 99658 50207.5 5020750 757 99658 50207.5 5020750 -32413 32725 4987.66 498766 -125 126 0.78 78 -758 2 10748 99659 2.27627 299.27627 150.77627 15077.62762 2.27627 299.27628 150.77627 15077.62756 2.27627 299.27627 150.77627000000012 15077.62700 2020-01-01 2020-01-02 2020-01-01 00:12:38 2020-01-02 03:40:59 2020-01-01 00:12:38.000 2020-01-02 03:40:59.000 758 99659 50208.5 5020850 758 99659 50208.5 5020850 -32412 32726 4988.66 498866 -124 127 1.78 178 -759 2 10749 99660 2.27927 299.27927 150.77927 15077.92792 2.27927 299.27927 150.77927 15077.92787 2.27927 299.27927 150.7792699999999 15077.92700 2020-01-01 2020-01-02 2020-01-01 00:12:39 2020-01-02 03:41:00 2020-01-01 00:12:39.000 2020-01-02 03:41:00.000 759 99660 50209.5 5020950 759 99660 50209.5 5020950 -32411 32727 4989.66 498966 -128 127 0.22 22 -76 2 10066 99976 0.22822 300.22822 150.22822 15173.05105 0.22822 300.22824 150.22822 15173.05109 0.22822 300.22822 150.2282200000002 15173.05022 2020-01-01 2020-01-02 2020-01-01 00:01:16 2020-01-02 03:46:16 2020-01-01 00:01:16.000 2020-01-02 03:46:16.000 76 99976 50026 5052626 76 99976 50026 5052626 -32493 32442 4605.009900990099 465106 -126 125 -2.99009900990099 -302 -760 2 10750 99661 2.28228 299.28228 150.78228 15078.22822 2.28228 299.2823 150.78228 15078.2285 2.28228 299.28228 150.78228000000001 15078.22800 2020-01-01 2020-01-02 2020-01-01 00:12:40 2020-01-02 03:41:01 2020-01-01 00:12:40.000 2020-01-02 03:41:01.000 760 99661 50210.5 5021050 760 99661 50210.5 5021050 -32410 32728 4990.66 499066 -128 127 -1.34 -134 -761 2 10751 99662 2.28528 299.28528 150.78528 15078.52852 2.28528 299.28528 150.78528 15078.52814 2.28528 299.28528 150.78528000000003 15078.52800 2020-01-01 2020-01-02 2020-01-01 00:12:41 2020-01-02 03:41:02 2020-01-01 00:12:41.000 2020-01-02 03:41:02.000 761 99662 50211.5 5021150 761 99662 50211.5 5021150 -32409 32729 4991.66 499166 -128 124 -2.9 -290 -762 2 10752 99663 2.28828 299.28828 150.78828 15078.82882 2.28828 299.2883 150.78828 15078.82889 2.28828 299.28828 150.78828000000007 15078.82800 2020-01-01 2020-01-02 2020-01-01 00:12:42 2020-01-02 03:41:03 2020-01-01 00:12:42.000 2020-01-02 03:41:03.000 762 99663 50212.5 5021250 762 99663 50212.5 5021250 -32408 32730 4992.66 499266 -127 125 -1.9 -190 -763 2 10753 99664 2.29129 299.29129 150.79129 15079.12912 2.29129 299.2913 150.79129 15079.12904 2.29129 299.29129 150.79128999999992 15079.12900 2020-01-01 2020-01-02 2020-01-01 00:12:43 2020-01-02 03:41:04 2020-01-01 00:12:43.000 2020-01-02 03:41:04.000 763 99664 50213.5 5021350 763 99664 50213.5 5021350 -32407 32731 4993.66 499366 -126 126 -0.9 -90 -764 2 10754 99665 2.29429 299.29429 150.79429 15079.42942 2.29429 299.29428 150.79429 15079.42933 2.29429 299.29429 150.79428999999988 15079.42900 2020-01-01 2020-01-02 2020-01-01 00:12:44 2020-01-02 03:41:05 2020-01-01 00:12:44.000 2020-01-02 03:41:05.000 764 99665 50214.5 5021450 764 99665 50214.5 5021450 -32406 32732 4994.66 499466 -125 127 0.1 10 -765 2 10755 99666 2.29729 299.29729 150.79729 15079.72972 2.29729 299.2973 150.79729 15079.72996 2.29729 299.29729 150.7972900000002 15079.72900 2020-01-01 2020-01-02 2020-01-01 00:12:45 2020-01-02 03:41:06 2020-01-01 00:12:45.000 2020-01-02 03:41:06.000 765 99666 50215.5 5021550 765 99666 50215.5 5021550 -32405 32733 4995.66 499566 -128 127 -1.46 -146 -766 2 10756 99667 2.3003 299.3003 150.8003 15080.03003 2.3003 299.3003 150.80029 15080.02961 2.30030 299.30030 150.8003000000001 15080.03000 2020-01-01 2020-01-02 2020-01-01 00:12:46 2020-01-02 03:41:07 2020-01-01 00:12:46.000 2020-01-02 03:41:07.000 766 99667 50216.5 5021650 766 99667 50216.5 5021650 -32404 32734 4996.66 499666 -128 127 -3.02 -302 -767 2 10757 99668 2.3033 299.3033 150.8033 15080.33033 2.3033 299.3033 150.8033 15080.33036 2.30330 299.30330 150.80329999999978 15080.33000 2020-01-01 2020-01-02 2020-01-01 00:12:47 2020-01-02 03:41:08 2020-01-01 00:12:47.000 2020-01-02 03:41:08.000 767 99668 50217.5 5021750 767 99668 50217.5 5021750 -32403 32735 4997.66 499766 -128 123 -4.58 -458 -768 2 10758 99669 2.3063 299.3063 150.8063 15080.63063 2.3063 299.3063 150.8063 15080.6305 2.30630 299.30630 150.80630000000014 15080.63000 2020-01-01 2020-01-02 2020-01-01 00:12:48 2020-01-02 03:41:09 2020-01-01 00:12:48.000 2020-01-02 03:41:09.000 768 99669 50218.5 5021850 768 99669 50218.5 5021850 -32402 32736 4998.66 499866 -127 124 -3.58 -358 -769 2 10759 99670 2.3093 299.3093 150.8093 15080.93093 2.3093 299.3093 150.8093 15080.93084 2.30930 299.30930 150.80930000000012 15080.93000 2020-01-01 2020-01-02 2020-01-01 00:12:49 2020-01-02 03:41:10 2020-01-01 00:12:49.000 2020-01-02 03:41:10.000 769 99670 50219.5 5021950 769 99670 50219.5 5021950 -32401 32737 4999.66 499966 -126 125 -2.58 -258 -77 2 10067 99977 0.23123 300.23123 150.23123 15173.35435 0.23123 300.23123 150.23123 15173.35439 0.23123 300.23123 150.23122999999993 15173.35423 2020-01-01 2020-01-02 2020-01-01 00:01:17 2020-01-02 03:46:17 2020-01-01 00:01:17.000 2020-01-02 03:46:17.000 77 99977 50027 5052727 77 99977 50027 5052727 -32492 32443 4606.009900990099 465207 -125 126 -1.99009900990099 -201 -770 2 10760 99671 2.31231 299.31231 150.81231 15081.23123 2.31231 299.31232 150.81231 15081.23144 2.31231 299.31231 150.8123099999999 15081.23100 2020-01-01 2020-01-02 2020-01-01 00:12:50 2020-01-02 03:41:11 2020-01-01 00:12:50.000 2020-01-02 03:41:11.000 770 99671 50220.5 5022050 770 99671 50220.5 5022050 -32400 32738 5000.66 500066 -125 126 -1.58 -158 -771 2 10761 99672 2.31531 299.31531 150.81531 15081.53153 2.31531 299.3153 150.81531 15081.53173 2.31531 299.31531 150.81530999999998 15081.53100 2020-01-01 2020-01-02 2020-01-01 00:12:51 2020-01-02 03:41:12 2020-01-01 00:12:51.000 2020-01-02 03:41:12.000 771 99672 50221.5 5022150 771 99672 50221.5 5022150 -32399 32739 5001.66 500166 -124 127 -0.58 -58 -772 2 10762 99673 2.31831 299.31831 150.81831 15081.83183 2.31831 299.31833 150.81831 15081.83183 2.31831 299.31831 150.81831000000003 15081.83100 2020-01-01 2020-01-02 2020-01-01 00:12:52 2020-01-02 03:41:13 2020-01-01 00:12:52.000 2020-01-02 03:41:13.000 772 99673 50222.5 5022250 772 99673 50222.5 5022250 -32398 32740 5002.66 500266 -128 127 -2.14 -214 -773 2 10763 99674 2.32132 299.32132 150.82132 15082.13213 2.32132 299.32132 150.82131 15082.13197 2.32132 299.32132 150.8213199999998 15082.13200 2020-01-01 2020-01-02 2020-01-01 00:12:53 2020-01-02 03:41:14 2020-01-01 00:12:53.000 2020-01-02 03:41:14.000 773 99674 50223.5 5022350 773 99674 50223.5 5022350 -32397 32741 5003.66 500366 -128 123 -3.7 -370 -774 2 10764 99675 2.32432 299.32432 150.82432 15082.43243 2.32432 299.3243 150.82432 15082.43231 2.32432 299.32432 150.8243199999999 15082.43200 2020-01-01 2020-01-02 2020-01-01 00:12:54 2020-01-02 03:41:15 2020-01-01 00:12:54.000 2020-01-02 03:41:15.000 774 99675 50224.5 5022450 774 99675 50224.5 5022450 -32396 32742 5004.66 500466 -127 124 -2.7 -270 -775 2 10765 99676 2.32732 299.32732 150.82732 15082.73273 2.32732 299.32733 150.82732 15082.73291 2.32732 299.32732 150.82732000000016 15082.73200 2020-01-01 2020-01-02 2020-01-01 00:12:55 2020-01-02 03:41:16 2020-01-01 00:12:55.000 2020-01-02 03:41:16.000 775 99676 50225.5 5022550 775 99676 50225.5 5022550 -32395 32743 5005.66 500566 -126 125 -1.7 -170 -776 2 10766 99677 2.33033 299.33033 150.83033 15083.03303 2.33033 299.33032 150.83033 15083.0332 2.33033 299.33033 150.83033000000003 15083.03300 2020-01-01 2020-01-02 2020-01-01 00:12:56 2020-01-02 03:41:17 2020-01-01 00:12:56.000 2020-01-02 03:41:17.000 776 99677 50226.5 5022650 776 99677 50226.5 5022650 -32394 32744 5006.66 500666 -125 126 -0.7 -70 +756 2 10746 99657 2.27027 299.27027 150.77027 15077.02702 2.27027 299.27026 150.77026 15077.02667 2.27027 299.27027 150.77027 15077.02700 2020-01-01 2020-01-02 2020-01-01 00:12:36 2020-01-02 03:40:57 2020-01-01 00:12:36.000 2020-01-02 03:40:57.000 756 99657 50206.5 5020650 756 99657 50206.5 5020650 -32414 32724 4986.66 498666 -126 125 -0.22 -22 +757 2 10747 99658 2.27327 299.27327 150.77327 15077.32732 2.27327 299.2733 150.77327 15077.32727 2.27327 299.27327 150.77327 15077.32700 2020-01-01 2020-01-02 2020-01-01 00:12:37 2020-01-02 03:40:58 2020-01-01 00:12:37.000 2020-01-02 03:40:58.000 757 99658 50207.5 5020750 757 99658 50207.5 5020750 -32413 32725 4987.66 498766 -125 126 0.78 78 +758 2 10748 99659 2.27627 299.27627 150.77627 15077.62762 2.27627 299.27628 150.77627 15077.62756 2.27627 299.27627 150.77627 15077.62700 2020-01-01 2020-01-02 2020-01-01 00:12:38 2020-01-02 03:40:59 2020-01-01 00:12:38.000 2020-01-02 03:40:59.000 758 99659 50208.5 5020850 758 99659 50208.5 5020850 -32412 32726 4988.66 498866 -124 127 1.78 178 +759 2 10749 99660 2.27927 299.27927 150.77927 15077.92792 2.27927 299.27927 150.77927 15077.92787 2.27927 299.27927 150.77927 15077.92700 2020-01-01 2020-01-02 2020-01-01 00:12:39 2020-01-02 03:41:00 2020-01-01 00:12:39.000 2020-01-02 03:41:00.000 759 99660 50209.5 5020950 759 99660 50209.5 5020950 -32411 32727 4989.66 498966 -128 127 0.22 22 +76 2 10066 99976 0.22822 300.22822 150.22822 15173.05105 0.22822 300.22824 150.22822 15173.05109 0.22822 300.22822 150.22822 15173.05022 2020-01-01 2020-01-02 2020-01-01 00:01:16 2020-01-02 03:46:16 2020-01-01 00:01:16.000 2020-01-02 03:46:16.000 76 99976 50026 5052626 76 99976 50026 5052626 -32493 32442 4605.009900990099 465106 -126 125 -2.99009900990099 -302 +760 2 10750 99661 2.28228 299.28228 150.78228 15078.22822 2.28228 299.2823 150.78228 15078.2285 2.28228 299.28228 150.78228 15078.22800 2020-01-01 2020-01-02 2020-01-01 00:12:40 2020-01-02 03:41:01 2020-01-01 00:12:40.000 2020-01-02 03:41:01.000 760 99661 50210.5 5021050 760 99661 50210.5 5021050 -32410 32728 4990.66 499066 -128 127 -1.34 -134 +761 2 10751 99662 2.28528 299.28528 150.78528 15078.52852 2.28528 299.28528 150.78528 15078.52814 2.28528 299.28528 150.78528 15078.52800 2020-01-01 2020-01-02 2020-01-01 00:12:41 2020-01-02 03:41:02 2020-01-01 00:12:41.000 2020-01-02 03:41:02.000 761 99662 50211.5 5021150 761 99662 50211.5 5021150 -32409 32729 4991.66 499166 -128 124 -2.9 -290 +762 2 10752 99663 2.28828 299.28828 150.78828 15078.82882 2.28828 299.2883 150.78828 15078.82889 2.28828 299.28828 150.78828 15078.82800 2020-01-01 2020-01-02 2020-01-01 00:12:42 2020-01-02 03:41:03 2020-01-01 00:12:42.000 2020-01-02 03:41:03.000 762 99663 50212.5 5021250 762 99663 50212.5 5021250 -32408 32730 4992.66 499266 -127 125 -1.9 -190 +763 2 10753 99664 2.29129 299.29129 150.79129 15079.12912 2.29129 299.2913 150.79129 15079.12904 2.29129 299.29129 150.79129 15079.12900 2020-01-01 2020-01-02 2020-01-01 00:12:43 2020-01-02 03:41:04 2020-01-01 00:12:43.000 2020-01-02 03:41:04.000 763 99664 50213.5 5021350 763 99664 50213.5 5021350 -32407 32731 4993.66 499366 -126 126 -0.9 -90 +764 2 10754 99665 2.29429 299.29429 150.79429 15079.42942 2.29429 299.29428 150.79429 15079.42933 2.29429 299.29429 150.79429 15079.42900 2020-01-01 2020-01-02 2020-01-01 00:12:44 2020-01-02 03:41:05 2020-01-01 00:12:44.000 2020-01-02 03:41:05.000 764 99665 50214.5 5021450 764 99665 50214.5 5021450 -32406 32732 4994.66 499466 -125 127 0.1 10 +765 2 10755 99666 2.29729 299.29729 150.79729 15079.72972 2.29729 299.2973 150.79729 15079.72996 2.29729 299.29729 150.79729 15079.72900 2020-01-01 2020-01-02 2020-01-01 00:12:45 2020-01-02 03:41:06 2020-01-01 00:12:45.000 2020-01-02 03:41:06.000 765 99666 50215.5 5021550 765 99666 50215.5 5021550 -32405 32733 4995.66 499566 -128 127 -1.46 -146 +766 2 10756 99667 2.3003 299.3003 150.8003 15080.03003 2.3003 299.3003 150.80029 15080.02961 2.30030 299.30030 150.8003 15080.03000 2020-01-01 2020-01-02 2020-01-01 00:12:46 2020-01-02 03:41:07 2020-01-01 00:12:46.000 2020-01-02 03:41:07.000 766 99667 50216.5 5021650 766 99667 50216.5 5021650 -32404 32734 4996.66 499666 -128 127 -3.02 -302 +767 2 10757 99668 2.3033 299.3033 150.8033 15080.33033 2.3033 299.3033 150.8033 15080.33036 2.30330 299.30330 150.8033 15080.33000 2020-01-01 2020-01-02 2020-01-01 00:12:47 2020-01-02 03:41:08 2020-01-01 00:12:47.000 2020-01-02 03:41:08.000 767 99668 50217.5 5021750 767 99668 50217.5 5021750 -32403 32735 4997.66 499766 -128 123 -4.58 -458 +768 2 10758 99669 2.3063 299.3063 150.8063 15080.63063 2.3063 299.3063 150.8063 15080.6305 2.30630 299.30630 150.8063 15080.63000 2020-01-01 2020-01-02 2020-01-01 00:12:48 2020-01-02 03:41:09 2020-01-01 00:12:48.000 2020-01-02 03:41:09.000 768 99669 50218.5 5021850 768 99669 50218.5 5021850 -32402 32736 4998.66 499866 -127 124 -3.58 -358 +769 2 10759 99670 2.3093 299.3093 150.8093 15080.93093 2.3093 299.3093 150.8093 15080.93084 2.30930 299.30930 150.8093 15080.93000 2020-01-01 2020-01-02 2020-01-01 00:12:49 2020-01-02 03:41:10 2020-01-01 00:12:49.000 2020-01-02 03:41:10.000 769 99670 50219.5 5021950 769 99670 50219.5 5021950 -32401 32737 4999.66 499966 -126 125 -2.58 -258 +77 2 10067 99977 0.23123 300.23123 150.23123 15173.35435 0.23123 300.23123 150.23123 15173.35439 0.23123 300.23123 150.23123 15173.35423 2020-01-01 2020-01-02 2020-01-01 00:01:17 2020-01-02 03:46:17 2020-01-01 00:01:17.000 2020-01-02 03:46:17.000 77 99977 50027 5052727 77 99977 50027 5052727 -32492 32443 4606.009900990099 465207 -125 126 -1.99009900990099 -201 +770 2 10760 99671 2.31231 299.31231 150.81231 15081.23123 2.31231 299.31232 150.81231 15081.23144 2.31231 299.31231 150.81231 15081.23100 2020-01-01 2020-01-02 2020-01-01 00:12:50 2020-01-02 03:41:11 2020-01-01 00:12:50.000 2020-01-02 03:41:11.000 770 99671 50220.5 5022050 770 99671 50220.5 5022050 -32400 32738 5000.66 500066 -125 126 -1.58 -158 +771 2 10761 99672 2.31531 299.31531 150.81531 15081.53153 2.31531 299.3153 150.81531 15081.53173 2.31531 299.31531 150.81531 15081.53100 2020-01-01 2020-01-02 2020-01-01 00:12:51 2020-01-02 03:41:12 2020-01-01 00:12:51.000 2020-01-02 03:41:12.000 771 99672 50221.5 5022150 771 99672 50221.5 5022150 -32399 32739 5001.66 500166 -124 127 -0.58 -58 +772 2 10762 99673 2.31831 299.31831 150.81831 15081.83183 2.31831 299.31833 150.81831 15081.83183 2.31831 299.31831 150.81831 15081.83100 2020-01-01 2020-01-02 2020-01-01 00:12:52 2020-01-02 03:41:13 2020-01-01 00:12:52.000 2020-01-02 03:41:13.000 772 99673 50222.5 5022250 772 99673 50222.5 5022250 -32398 32740 5002.66 500266 -128 127 -2.14 -214 +773 2 10763 99674 2.32132 299.32132 150.82132 15082.13213 2.32132 299.32132 150.82131 15082.13197 2.32132 299.32132 150.82132 15082.13200 2020-01-01 2020-01-02 2020-01-01 00:12:53 2020-01-02 03:41:14 2020-01-01 00:12:53.000 2020-01-02 03:41:14.000 773 99674 50223.5 5022350 773 99674 50223.5 5022350 -32397 32741 5003.66 500366 -128 123 -3.7 -370 +774 2 10764 99675 2.32432 299.32432 150.82432 15082.43243 2.32432 299.3243 150.82432 15082.43231 2.32432 299.32432 150.82432 15082.43200 2020-01-01 2020-01-02 2020-01-01 00:12:54 2020-01-02 03:41:15 2020-01-01 00:12:54.000 2020-01-02 03:41:15.000 774 99675 50224.5 5022450 774 99675 50224.5 5022450 -32396 32742 5004.66 500466 -127 124 -2.7 -270 +775 2 10765 99676 2.32732 299.32732 150.82732 15082.73273 2.32732 299.32733 150.82732 15082.73291 2.32732 299.32732 150.82732 15082.73200 2020-01-01 2020-01-02 2020-01-01 00:12:55 2020-01-02 03:41:16 2020-01-01 00:12:55.000 2020-01-02 03:41:16.000 775 99676 50225.5 5022550 775 99676 50225.5 5022550 -32395 32743 5005.66 500566 -126 125 -1.7 -170 +776 2 10766 99677 2.33033 299.33033 150.83033 15083.03303 2.33033 299.33032 150.83033 15083.0332 2.33033 299.33033 150.83033 15083.03300 2020-01-01 2020-01-02 2020-01-01 00:12:56 2020-01-02 03:41:17 2020-01-01 00:12:56.000 2020-01-02 03:41:17.000 776 99677 50226.5 5022650 776 99677 50226.5 5022650 -32394 32744 5006.66 500666 -125 126 -0.7 -70 777 2 10767 99678 2.33333 299.33333 150.83333 15083.33333 2.33333 299.33334 150.83333 15083.3333 2.33333 299.33333 150.83333000000002 15083.33300 2020-01-01 2020-01-02 2020-01-01 00:12:57 2020-01-02 03:41:18 2020-01-01 00:12:57.000 2020-01-02 03:41:18.000 777 99678 50227.5 5022750 777 99678 50227.5 5022750 -32393 32745 5007.66 500766 -124 127 0.3 30 -778 2 10768 99679 2.33633 299.33633 150.83633 15083.63363 2.33633 299.33633 150.83633 15083.63348 2.33633 299.33633 150.83633000000006 15083.63300 2020-01-01 2020-01-02 2020-01-01 00:12:58 2020-01-02 03:41:19 2020-01-01 00:12:58.000 2020-01-02 03:41:19.000 778 99679 50228.5 5022850 778 99679 50228.5 5022850 -32392 32746 5008.66 500866 -128 127 -1.26 -126 +778 2 10768 99679 2.33633 299.33633 150.83633 15083.63363 2.33633 299.33633 150.83633 15083.63348 2.33633 299.33633 150.83633 15083.63300 2020-01-01 2020-01-02 2020-01-01 00:12:58 2020-01-02 03:41:19 2020-01-01 00:12:58.000 2020-01-02 03:41:19.000 778 99679 50228.5 5022850 778 99679 50228.5 5022850 -32392 32746 5008.66 500866 -128 127 -1.26 -126 779 2 10769 99680 2.33933 299.33933 150.83933 15083.93393 2.33933 299.33932 150.83933 15083.93378 2.33933 299.33933 150.83933000000002 15083.93300 2020-01-01 2020-01-02 2020-01-01 00:12:59 2020-01-02 03:41:20 2020-01-01 00:12:59.000 2020-01-02 03:41:20.000 779 99680 50229.5 5022950 779 99680 50229.5 5022950 -32391 32747 5009.66 500966 -128 123 -2.82 -282 -78 2 10068 99978 0.23423 300.23423 150.23423 15173.65765 0.23423 300.23422 150.23423 15173.65769 0.23423 300.23423 150.23423000000003 15173.65723 2020-01-01 2020-01-02 2020-01-01 00:01:18 2020-01-02 03:46:18 2020-01-01 00:01:18.000 2020-01-02 03:46:18.000 78 99978 50028 5052828 78 99978 50028 5052828 -32491 32444 4607.009900990099 465308 -124 127 -0.9900990099009901 -100 -780 2 10770 99681 2.34234 299.34234 150.84234 15084.23423 2.34234 299.34235 150.84234 15084.23437 2.34234 299.34234 150.8423399999999 15084.23400 2020-01-01 2020-01-02 2020-01-01 00:13:00 2020-01-02 03:41:21 2020-01-01 00:13:00.000 2020-01-02 03:41:21.000 780 99681 50230.5 5023050 780 99681 50230.5 5023050 -32390 32748 5010.66 501066 -127 124 -1.82 -182 -781 2 10771 99682 2.34534 299.34534 150.84534 15084.53453 2.34534 299.34534 150.84534 15084.53467 2.34534 299.34534 150.8453399999999 15084.53400 2020-01-01 2020-01-02 2020-01-01 00:13:01 2020-01-02 03:41:22 2020-01-01 00:13:01.000 2020-01-02 03:41:22.000 781 99682 50231.5 5023150 781 99682 50231.5 5023150 -32389 32749 5011.66 501166 -126 125 -0.82 -82 -782 2 10772 99683 2.34834 299.34834 150.84834 15084.83483 2.34834 299.34836 150.84834 15084.83477 2.34834 299.34834 150.84834000000006 15084.83400 2020-01-01 2020-01-02 2020-01-01 00:13:02 2020-01-02 03:41:23 2020-01-01 00:13:02.000 2020-01-02 03:41:23.000 782 99683 50232.5 5023250 782 99683 50232.5 5023250 -32388 32750 5012.66 501266 -125 126 0.18 18 -783 2 10773 99684 2.35135 299.35135 150.85135 15085.13513 2.35135 299.35135 150.85134 15085.13495 2.35135 299.35135 150.85135000000014 15085.13500 2020-01-01 2020-01-02 2020-01-01 00:13:03 2020-01-02 03:41:24 2020-01-01 00:13:03.000 2020-01-02 03:41:24.000 783 99684 50233.5 5023350 783 99684 50233.5 5023350 -32387 32751 5013.66 501366 -124 127 1.18 118 -784 2 10774 99685 2.35435 299.35435 150.85435 15085.43543 2.35435 299.35434 150.85435 15085.43525 2.35435 299.35435 150.85434999999978 15085.43500 2020-01-01 2020-01-02 2020-01-01 00:13:04 2020-01-02 03:41:25 2020-01-01 00:13:04.000 2020-01-02 03:41:25.000 784 99685 50234.5 5023450 784 99685 50234.5 5023450 -32386 32752 5014.66 501466 -128 127 -0.38 -38 -785 2 10775 99686 2.35735 299.35735 150.85735 15085.73573 2.35735 299.35736 150.85736 15085.736 2.35735 299.35735 150.85735000000017 15085.73500 2020-01-01 2020-01-02 2020-01-01 00:13:05 2020-01-02 03:41:26 2020-01-01 00:13:05.000 2020-01-02 03:41:26.000 785 99686 50235.5 5023550 785 99686 50235.5 5023550 -32385 32753 5015.66 501566 -128 127 -1.94 -194 -786 2 10776 99687 2.36036 299.36036 150.86036 15086.03603 2.36036 299.36035 150.86036 15086.03614 2.36036 299.36036 150.86036000000004 15086.03600 2020-01-01 2020-01-02 2020-01-01 00:13:06 2020-01-02 03:41:27 2020-01-01 00:13:06.000 2020-01-02 03:41:27.000 786 99687 50236.5 5023650 786 99687 50236.5 5023650 -32384 32754 5016.66 501666 -128 124 -3.5 -350 -787 2 10777 99688 2.36336 299.36336 150.86336 15086.33633 2.36336 299.36337 150.86336 15086.33628 2.36336 299.36336 150.86335999999991 15086.33600 2020-01-01 2020-01-02 2020-01-01 00:13:07 2020-01-02 03:41:28 2020-01-01 00:13:07.000 2020-01-02 03:41:28.000 787 99688 50237.5 5023750 787 99688 50237.5 5023750 -32383 32755 5017.66 501766 -127 125 -2.5 -250 -788 2 10778 99689 2.36636 299.36636 150.86636 15086.63663 2.36636 299.36636 150.86636 15086.63641 2.36636 299.36636 150.86636 15086.63600 2020-01-01 2020-01-02 2020-01-01 00:13:08 2020-01-02 03:41:29 2020-01-01 00:13:08.000 2020-01-02 03:41:29.000 788 99689 50238.5 5023850 788 99689 50238.5 5023850 -32382 32756 5018.66 501866 -126 126 -1.5 -150 -789 2 10779 99690 2.36936 299.36936 150.86936 15086.93693 2.36936 299.36935 150.86936 15086.93672 2.36936 299.36936 150.86936000000006 15086.93600 2020-01-01 2020-01-02 2020-01-01 00:13:09 2020-01-02 03:41:30 2020-01-01 00:13:09.000 2020-01-02 03:41:30.000 789 99690 50239.5 5023950 789 99690 50239.5 5023950 -32381 32757 5019.66 501966 -125 127 -0.5 -50 -79 2 10069 99979 0.23723 300.23723 150.23723 15173.96096 0.23723 300.23724 150.23724 15173.96129 0.23723 300.23723 150.23723000000015 15173.96023 2020-01-01 2020-01-02 2020-01-01 00:01:19 2020-01-02 03:46:19 2020-01-01 00:01:19.000 2020-01-02 03:46:19.000 79 99979 50029 5052929 79 99979 50029 5052929 -32490 32445 4608.009900990099 465409 -128 127 -2.5247524752475248 -255 -790 2 10780 99691 2.37237 299.37237 150.87237 15087.23723 2.37237 299.37238 150.87237 15087.23747 2.37237 299.37237 150.87236999999982 15087.23700 2020-01-01 2020-01-02 2020-01-01 00:13:10 2020-01-02 03:41:31 2020-01-01 00:13:10.000 2020-01-02 03:41:31.000 790 99691 50240.5 5024050 790 99691 50240.5 5024050 -32380 32758 5020.66 502066 -128 127 -2.06 -206 -791 2 10781 99692 2.37537 299.37537 150.87537 15087.53753 2.37537 299.37537 150.87537 15087.53761 2.37537 299.37537 150.87536999999986 15087.53700 2020-01-01 2020-01-02 2020-01-01 00:13:11 2020-01-02 03:41:32 2020-01-01 00:13:11.000 2020-01-02 03:41:32.000 791 99692 50241.5 5024150 791 99692 50241.5 5024150 -32379 32759 5021.66 502166 -128 127 -3.62 -362 -792 2 10782 99693 2.37837 299.37837 150.87837 15087.83783 2.37837 299.3784 150.87837 15087.83775 2.37837 299.37837 150.87837000000016 15087.83700 2020-01-01 2020-01-02 2020-01-01 00:13:12 2020-01-02 03:41:33 2020-01-01 00:13:12.000 2020-01-02 03:41:33.000 792 99693 50242.5 5024250 792 99693 50242.5 5024250 -32378 32760 5022.66 502266 -128 123 -5.18 -518 -793 2 10783 99694 2.38138 299.38138 150.88138 15088.13813 2.38138 299.38138 150.88137 15088.13789 2.38138 299.38138 150.8813800000001 15088.13800 2020-01-01 2020-01-02 2020-01-01 00:13:13 2020-01-02 03:41:34 2020-01-01 00:13:13.000 2020-01-02 03:41:34.000 793 99694 50243.5 5024350 793 99694 50243.5 5024350 -32377 32761 5023.66 502366 -127 124 -4.18 -418 -794 2 10784 99695 2.38438 299.38438 150.88438 15088.43843 2.38438 299.3844 150.88438 15088.43864 2.38438 299.38438 150.88438000000005 15088.43800 2020-01-01 2020-01-02 2020-01-01 00:13:14 2020-01-02 03:41:35 2020-01-01 00:13:14.000 2020-01-02 03:41:35.000 794 99695 50244.5 5024450 794 99695 50244.5 5024450 -32376 32762 5024.66 502466 -126 125 -3.18 -318 -795 2 10785 99696 2.38738 299.38738 150.88738 15088.73873 2.38738 299.3874 150.88738 15088.73894 2.38738 299.38738 150.8873800000001 15088.73800 2020-01-01 2020-01-02 2020-01-01 00:13:15 2020-01-02 03:41:36 2020-01-01 00:13:15.000 2020-01-02 03:41:36.000 795 99696 50245.5 5024550 795 99696 50245.5 5024550 -32375 32763 5025.66 502566 -125 126 -2.18 -218 -796 2 10786 99697 2.39039 299.39039 150.89039 15089.03903 2.39039 299.39038 150.89039 15089.03908 2.39039 299.39039 150.89038999999997 15089.03900 2020-01-01 2020-01-02 2020-01-01 00:13:16 2020-01-02 03:41:37 2020-01-01 00:13:16.000 2020-01-02 03:41:37.000 796 99697 50246.5 5024650 796 99697 50246.5 5024650 -32374 32764 5026.66 502666 -124 127 -1.18 -118 -797 2 10787 99698 2.39339 299.39339 150.89339 15089.33933 2.39339 299.3934 150.89339 15089.33921 2.39339 299.39339 150.8933899999999 15089.33900 2020-01-01 2020-01-02 2020-01-01 00:13:17 2020-01-02 03:41:38 2020-01-01 00:13:17.000 2020-01-02 03:41:38.000 797 99698 50247.5 5024750 797 99698 50247.5 5024750 -32373 32765 5027.66 502766 -128 127 -2.74 -274 -798 2 10788 99699 2.39639 299.39639 150.89639 15089.63963 2.39639 299.3964 150.89639 15089.63936 2.39639 299.39639 150.8963899999999 15089.63900 2020-01-01 2020-01-02 2020-01-01 00:13:18 2020-01-02 03:41:39 2020-01-01 00:13:18.000 2020-01-02 03:41:39.000 798 99699 50248.5 5024850 798 99699 50248.5 5024850 -32372 32766 5028.66 502866 -128 123 -4.3 -430 -799 2 10789 99700 2.39939 299.39939 150.89939 15089.93993 2.39939 299.3994 150.8994 15089.94011 2.39939 299.39939 150.89938999999998 15089.93900 2020-01-01 2020-01-02 2020-01-01 00:13:19 2020-01-02 03:41:40 2020-01-01 00:13:19.000 2020-01-02 03:41:40.000 799 99700 50249.5 5024950 799 99700 50249.5 5024950 -32371 32767 5029.66 502966 -127 124 -3.3 -330 -8 2 1007 9998 0.02402 300.02402 150.02402 15152.42642 0.02402 300.02402 150.02402 15152.42607 0.02402 300.02402 150.02402000000012 15152.42602 2020-01-01 2020-01-02 2020-01-01 00:00:08 2020-01-02 03:45:08 2020-01-01 00:00:08.000 2020-01-02 03:45:08.000 8 99908 49958 5045758 8 99908 49958 5045758 -32561 32374 4537.009900990099 458238 -125 126 -0.019801980198019802 -2 -80 2 10070 99980 0.24024 300.24024 150.24024 15174.26426 0.24024 300.24023 150.24023 15174.26397 0.24024 300.24024 150.2402399999998 15174.26424 2020-01-01 2020-01-02 2020-01-01 00:01:20 2020-01-02 03:46:20 2020-01-01 00:01:20.000 2020-01-02 03:46:20.000 80 99980 50030 5053030 80 99980 50030 5053030 -32489 32446 4609.009900990099 465510 -128 123 -4.0594059405940595 -410 -800 2 10790 99701 2.4024 299.4024 150.9024 15090.24024 2.4024 299.4024 150.9024 15090.24041 2.40240 299.40240 150.90240000000017 15090.24000 2020-01-01 2020-01-02 2020-01-01 00:13:20 2020-01-02 03:41:41 2020-01-01 00:13:20.000 2020-01-02 03:41:41.000 800 99701 50250.5 5025050 800 99701 50250.5 5025050 -32768 32167 4375.3 437530 -126 125 -2.3 -230 -801 2 10791 99702 2.4054 299.4054 150.9054 15090.54054 2.4054 299.4054 150.9054 15090.54058 2.40540 299.40540 150.90539999999984 15090.54000 2020-01-01 2020-01-02 2020-01-01 00:13:21 2020-01-02 03:41:42 2020-01-01 00:13:21.000 2020-01-02 03:41:42.000 801 99702 50251.5 5025150 801 99702 50251.5 5025150 -32767 32168 4376.3 437630 -125 126 -1.3 -130 -802 2 10792 99703 2.4084 299.4084 150.9084 15090.84084 2.4084 299.40842 150.9084 15090.84069 2.40840 299.40840 150.90840000000009 15090.84000 2020-01-01 2020-01-02 2020-01-01 00:13:22 2020-01-02 03:41:43 2020-01-01 00:13:22.000 2020-01-02 03:41:43.000 802 99703 50252.5 5025250 802 99703 50252.5 5025250 -32766 32169 4377.3 437730 -124 127 -0.3 -30 -803 2 10793 99704 2.41141 299.41141 150.91141 15091.14114 2.41141 299.4114 150.9114 15091.14098 2.41141 299.41141 150.91141000000007 15091.14100 2020-01-01 2020-01-02 2020-01-01 00:13:23 2020-01-02 03:41:44 2020-01-01 00:13:23.000 2020-01-02 03:41:44.000 803 99704 50253.5 5025350 803 99704 50253.5 5025350 -32765 32170 4378.3 437830 -128 127 -1.86 -186 -804 2 10794 99705 2.41441 299.41441 150.91441 15091.44144 2.41441 299.41443 150.91441 15091.44158 2.41441 299.41441 150.91440999999995 15091.44100 2020-01-01 2020-01-02 2020-01-01 00:13:24 2020-01-02 03:41:45 2020-01-01 00:13:24.000 2020-01-02 03:41:45.000 804 99705 50254.5 5025450 804 99705 50254.5 5025450 -32764 32171 4379.3 437930 -128 123 -3.42 -342 +78 2 10068 99978 0.23423 300.23423 150.23423 15173.65765 0.23423 300.23422 150.23423 15173.65769 0.23423 300.23423 150.23423 15173.65723 2020-01-01 2020-01-02 2020-01-01 00:01:18 2020-01-02 03:46:18 2020-01-01 00:01:18.000 2020-01-02 03:46:18.000 78 99978 50028 5052828 78 99978 50028 5052828 -32491 32444 4607.009900990099 465308 -124 127 -0.9900990099009901 -100 +780 2 10770 99681 2.34234 299.34234 150.84234 15084.23423 2.34234 299.34235 150.84234 15084.23437 2.34234 299.34234 150.84234 15084.23400 2020-01-01 2020-01-02 2020-01-01 00:13:00 2020-01-02 03:41:21 2020-01-01 00:13:00.000 2020-01-02 03:41:21.000 780 99681 50230.5 5023050 780 99681 50230.5 5023050 -32390 32748 5010.66 501066 -127 124 -1.82 -182 +781 2 10771 99682 2.34534 299.34534 150.84534 15084.53453 2.34534 299.34534 150.84534 15084.53467 2.34534 299.34534 150.84534 15084.53400 2020-01-01 2020-01-02 2020-01-01 00:13:01 2020-01-02 03:41:22 2020-01-01 00:13:01.000 2020-01-02 03:41:22.000 781 99682 50231.5 5023150 781 99682 50231.5 5023150 -32389 32749 5011.66 501166 -126 125 -0.82 -82 +782 2 10772 99683 2.34834 299.34834 150.84834 15084.83483 2.34834 299.34836 150.84834 15084.83477 2.34834 299.34834 150.84834 15084.83400 2020-01-01 2020-01-02 2020-01-01 00:13:02 2020-01-02 03:41:23 2020-01-01 00:13:02.000 2020-01-02 03:41:23.000 782 99683 50232.5 5023250 782 99683 50232.5 5023250 -32388 32750 5012.66 501266 -125 126 0.18 18 +783 2 10773 99684 2.35135 299.35135 150.85135 15085.13513 2.35135 299.35135 150.85134 15085.13495 2.35135 299.35135 150.85135 15085.13500 2020-01-01 2020-01-02 2020-01-01 00:13:03 2020-01-02 03:41:24 2020-01-01 00:13:03.000 2020-01-02 03:41:24.000 783 99684 50233.5 5023350 783 99684 50233.5 5023350 -32387 32751 5013.66 501366 -124 127 1.18 118 +784 2 10774 99685 2.35435 299.35435 150.85435 15085.43543 2.35435 299.35434 150.85435 15085.43525 2.35435 299.35435 150.85434999999998 15085.43500 2020-01-01 2020-01-02 2020-01-01 00:13:04 2020-01-02 03:41:25 2020-01-01 00:13:04.000 2020-01-02 03:41:25.000 784 99685 50234.5 5023450 784 99685 50234.5 5023450 -32386 32752 5014.66 501466 -128 127 -0.38 -38 +785 2 10775 99686 2.35735 299.35735 150.85735 15085.73573 2.35735 299.35736 150.85736 15085.736 2.35735 299.35735 150.85735 15085.73500 2020-01-01 2020-01-02 2020-01-01 00:13:05 2020-01-02 03:41:26 2020-01-01 00:13:05.000 2020-01-02 03:41:26.000 785 99686 50235.5 5023550 785 99686 50235.5 5023550 -32385 32753 5015.66 501566 -128 127 -1.94 -194 +786 2 10776 99687 2.36036 299.36036 150.86036 15086.03603 2.36036 299.36035 150.86036 15086.03614 2.36036 299.36036 150.86036000000001 15086.03600 2020-01-01 2020-01-02 2020-01-01 00:13:06 2020-01-02 03:41:27 2020-01-01 00:13:06.000 2020-01-02 03:41:27.000 786 99687 50236.5 5023650 786 99687 50236.5 5023650 -32384 32754 5016.66 501666 -128 124 -3.5 -350 +787 2 10777 99688 2.36336 299.36336 150.86336 15086.33633 2.36336 299.36337 150.86336 15086.33628 2.36336 299.36336 150.86336 15086.33600 2020-01-01 2020-01-02 2020-01-01 00:13:07 2020-01-02 03:41:28 2020-01-01 00:13:07.000 2020-01-02 03:41:28.000 787 99688 50237.5 5023750 787 99688 50237.5 5023750 -32383 32755 5017.66 501766 -127 125 -2.5 -250 +788 2 10778 99689 2.36636 299.36636 150.86636 15086.63663 2.36636 299.36636 150.86636 15086.63641 2.36636 299.36636 150.86636000000001 15086.63600 2020-01-01 2020-01-02 2020-01-01 00:13:08 2020-01-02 03:41:29 2020-01-01 00:13:08.000 2020-01-02 03:41:29.000 788 99689 50238.5 5023850 788 99689 50238.5 5023850 -32382 32756 5018.66 501866 -126 126 -1.5 -150 +789 2 10779 99690 2.36936 299.36936 150.86936 15086.93693 2.36936 299.36935 150.86936 15086.93672 2.36936 299.36936 150.86936 15086.93600 2020-01-01 2020-01-02 2020-01-01 00:13:09 2020-01-02 03:41:30 2020-01-01 00:13:09.000 2020-01-02 03:41:30.000 789 99690 50239.5 5023950 789 99690 50239.5 5023950 -32381 32757 5019.66 501966 -125 127 -0.5 -50 +79 2 10069 99979 0.23723 300.23723 150.23723 15173.96096 0.23723 300.23724 150.23724 15173.96129 0.23723 300.23723 150.23723 15173.96023 2020-01-01 2020-01-02 2020-01-01 00:01:19 2020-01-02 03:46:19 2020-01-01 00:01:19.000 2020-01-02 03:46:19.000 79 99979 50029 5052929 79 99979 50029 5052929 -32490 32445 4608.009900990099 465409 -128 127 -2.5247524752475248 -255 +790 2 10780 99691 2.37237 299.37237 150.87237 15087.23723 2.37237 299.37238 150.87237 15087.23747 2.37237 299.37237 150.87237 15087.23700 2020-01-01 2020-01-02 2020-01-01 00:13:10 2020-01-02 03:41:31 2020-01-01 00:13:10.000 2020-01-02 03:41:31.000 790 99691 50240.5 5024050 790 99691 50240.5 5024050 -32380 32758 5020.66 502066 -128 127 -2.06 -206 +791 2 10781 99692 2.37537 299.37537 150.87537 15087.53753 2.37537 299.37537 150.87537 15087.53761 2.37537 299.37537 150.87537 15087.53700 2020-01-01 2020-01-02 2020-01-01 00:13:11 2020-01-02 03:41:32 2020-01-01 00:13:11.000 2020-01-02 03:41:32.000 791 99692 50241.5 5024150 791 99692 50241.5 5024150 -32379 32759 5021.66 502166 -128 127 -3.62 -362 +792 2 10782 99693 2.37837 299.37837 150.87837 15087.83783 2.37837 299.3784 150.87837 15087.83775 2.37837 299.37837 150.87837 15087.83700 2020-01-01 2020-01-02 2020-01-01 00:13:12 2020-01-02 03:41:33 2020-01-01 00:13:12.000 2020-01-02 03:41:33.000 792 99693 50242.5 5024250 792 99693 50242.5 5024250 -32378 32760 5022.66 502266 -128 123 -5.18 -518 +793 2 10783 99694 2.38138 299.38138 150.88138 15088.13813 2.38138 299.38138 150.88137 15088.13789 2.38138 299.38138 150.88138 15088.13800 2020-01-01 2020-01-02 2020-01-01 00:13:13 2020-01-02 03:41:34 2020-01-01 00:13:13.000 2020-01-02 03:41:34.000 793 99694 50243.5 5024350 793 99694 50243.5 5024350 -32377 32761 5023.66 502366 -127 124 -4.18 -418 +794 2 10784 99695 2.38438 299.38438 150.88438 15088.43843 2.38438 299.3844 150.88438 15088.43864 2.38438 299.38438 150.88438 15088.43800 2020-01-01 2020-01-02 2020-01-01 00:13:14 2020-01-02 03:41:35 2020-01-01 00:13:14.000 2020-01-02 03:41:35.000 794 99695 50244.5 5024450 794 99695 50244.5 5024450 -32376 32762 5024.66 502466 -126 125 -3.18 -318 +795 2 10785 99696 2.38738 299.38738 150.88738 15088.73873 2.38738 299.3874 150.88738 15088.73894 2.38738 299.38738 150.88738 15088.73800 2020-01-01 2020-01-02 2020-01-01 00:13:15 2020-01-02 03:41:36 2020-01-01 00:13:15.000 2020-01-02 03:41:36.000 795 99696 50245.5 5024550 795 99696 50245.5 5024550 -32375 32763 5025.66 502566 -125 126 -2.18 -218 +796 2 10786 99697 2.39039 299.39039 150.89039 15089.03903 2.39039 299.39038 150.89039 15089.03908 2.39039 299.39039 150.89039 15089.03900 2020-01-01 2020-01-02 2020-01-01 00:13:16 2020-01-02 03:41:37 2020-01-01 00:13:16.000 2020-01-02 03:41:37.000 796 99697 50246.5 5024650 796 99697 50246.5 5024650 -32374 32764 5026.66 502666 -124 127 -1.18 -118 +797 2 10787 99698 2.39339 299.39339 150.89339 15089.33933 2.39339 299.3934 150.89339 15089.33921 2.39339 299.39339 150.89339 15089.33900 2020-01-01 2020-01-02 2020-01-01 00:13:17 2020-01-02 03:41:38 2020-01-01 00:13:17.000 2020-01-02 03:41:38.000 797 99698 50247.5 5024750 797 99698 50247.5 5024750 -32373 32765 5027.66 502766 -128 127 -2.74 -274 +798 2 10788 99699 2.39639 299.39639 150.89639 15089.63963 2.39639 299.3964 150.89639 15089.63936 2.39639 299.39639 150.89639 15089.63900 2020-01-01 2020-01-02 2020-01-01 00:13:18 2020-01-02 03:41:39 2020-01-01 00:13:18.000 2020-01-02 03:41:39.000 798 99699 50248.5 5024850 798 99699 50248.5 5024850 -32372 32766 5028.66 502866 -128 123 -4.3 -430 +799 2 10789 99700 2.39939 299.39939 150.89939 15089.93993 2.39939 299.3994 150.8994 15089.94011 2.39939 299.39939 150.89939 15089.93900 2020-01-01 2020-01-02 2020-01-01 00:13:19 2020-01-02 03:41:40 2020-01-01 00:13:19.000 2020-01-02 03:41:40.000 799 99700 50249.5 5024950 799 99700 50249.5 5024950 -32371 32767 5029.66 502966 -127 124 -3.3 -330 +8 2 1007 9998 0.02402 300.02402 150.02402 15152.42642 0.02402 300.02402 150.02402 15152.42607 0.02402 300.02402 150.02402 15152.42602 2020-01-01 2020-01-02 2020-01-01 00:00:08 2020-01-02 03:45:08 2020-01-01 00:00:08.000 2020-01-02 03:45:08.000 8 99908 49958 5045758 8 99908 49958 5045758 -32561 32374 4537.009900990099 458238 -125 126 -0.019801980198019802 -2 +80 2 10070 99980 0.24024 300.24024 150.24024 15174.26426 0.24024 300.24023 150.24023 15174.26397 0.24024 300.24024 150.24024 15174.26424 2020-01-01 2020-01-02 2020-01-01 00:01:20 2020-01-02 03:46:20 2020-01-01 00:01:20.000 2020-01-02 03:46:20.000 80 99980 50030 5053030 80 99980 50030 5053030 -32489 32446 4609.009900990099 465510 -128 123 -4.0594059405940595 -410 +800 2 10790 99701 2.4024 299.4024 150.9024 15090.24024 2.4024 299.4024 150.9024 15090.24041 2.40240 299.40240 150.9024 15090.24000 2020-01-01 2020-01-02 2020-01-01 00:13:20 2020-01-02 03:41:41 2020-01-01 00:13:20.000 2020-01-02 03:41:41.000 800 99701 50250.5 5025050 800 99701 50250.5 5025050 -32768 32167 4375.3 437530 -126 125 -2.3 -230 +801 2 10791 99702 2.4054 299.4054 150.9054 15090.54054 2.4054 299.4054 150.9054 15090.54058 2.40540 299.40540 150.90540000000001 15090.54000 2020-01-01 2020-01-02 2020-01-01 00:13:21 2020-01-02 03:41:42 2020-01-01 00:13:21.000 2020-01-02 03:41:42.000 801 99702 50251.5 5025150 801 99702 50251.5 5025150 -32767 32168 4376.3 437630 -125 126 -1.3 -130 +802 2 10792 99703 2.4084 299.4084 150.9084 15090.84084 2.4084 299.40842 150.9084 15090.84069 2.40840 299.40840 150.9084 15090.84000 2020-01-01 2020-01-02 2020-01-01 00:13:22 2020-01-02 03:41:43 2020-01-01 00:13:22.000 2020-01-02 03:41:43.000 802 99703 50252.5 5025250 802 99703 50252.5 5025250 -32766 32169 4377.3 437730 -124 127 -0.3 -30 +803 2 10793 99704 2.41141 299.41141 150.91141 15091.14114 2.41141 299.4114 150.9114 15091.14098 2.41141 299.41141 150.91141 15091.14100 2020-01-01 2020-01-02 2020-01-01 00:13:23 2020-01-02 03:41:44 2020-01-01 00:13:23.000 2020-01-02 03:41:44.000 803 99704 50253.5 5025350 803 99704 50253.5 5025350 -32765 32170 4378.3 437830 -128 127 -1.86 -186 +804 2 10794 99705 2.41441 299.41441 150.91441 15091.44144 2.41441 299.41443 150.91441 15091.44158 2.41441 299.41441 150.91441 15091.44100 2020-01-01 2020-01-02 2020-01-01 00:13:24 2020-01-02 03:41:45 2020-01-01 00:13:24.000 2020-01-02 03:41:45.000 804 99705 50254.5 5025450 804 99705 50254.5 5025450 -32764 32171 4379.3 437930 -128 123 -3.42 -342 805 2 10795 99706 2.41741 299.41741 150.91741 15091.74174 2.41741 299.41742 150.91741 15091.74188 2.41741 299.41741 150.91741 15091.74100 2020-01-01 2020-01-02 2020-01-01 00:13:25 2020-01-02 03:41:46 2020-01-01 00:13:25.000 2020-01-02 03:41:46.000 805 99706 50255.5 5025550 805 99706 50255.5 5025550 -32763 32172 4380.3 438030 -127 124 -2.42 -242 -806 2 10796 99707 2.42042 299.42042 150.92042 15092.04204 2.42042 299.4204 150.92042 15092.04205 2.42042 299.42042 150.92042000000018 15092.04200 2020-01-01 2020-01-02 2020-01-01 00:13:26 2020-01-02 03:41:47 2020-01-01 00:13:26.000 2020-01-02 03:41:47.000 806 99707 50256.5 5025650 806 99707 50256.5 5025650 -32762 32173 4381.3 438130 -126 125 -1.42 -142 -807 2 10797 99708 2.42342 299.42342 150.92342 15092.34234 2.42342 299.42343 150.92342 15092.34216 2.42342 299.42342 150.92341999999985 15092.34200 2020-01-01 2020-01-02 2020-01-01 00:13:27 2020-01-02 03:41:48 2020-01-01 00:13:27.000 2020-01-02 03:41:48.000 807 99708 50257.5 5025750 807 99708 50257.5 5025750 -32761 32174 4382.3 438230 -125 126 -0.42 -42 -808 2 10798 99709 2.42642 299.42642 150.92642 15092.64264 2.42642 299.42642 150.92642 15092.64245 2.42642 299.42642 150.92641999999987 15092.64200 2020-01-01 2020-01-02 2020-01-01 00:13:28 2020-01-02 03:41:49 2020-01-01 00:13:28.000 2020-01-02 03:41:49.000 808 99709 50258.5 5025850 808 99709 50258.5 5025850 -32760 32175 4383.3 438330 -124 127 0.58 58 -809 2 10799 99710 2.42942 299.42942 150.92942 15092.94294 2.42942 299.42944 150.92943 15092.94305 2.42942 299.42942 150.9294200000002 15092.94200 2020-01-01 2020-01-02 2020-01-01 00:13:29 2020-01-02 03:41:50 2020-01-01 00:13:29.000 2020-01-02 03:41:50.000 809 99710 50259.5 5025950 809 99710 50259.5 5025950 -32759 32176 4384.3 438430 -128 127 -0.98 -98 -81 2 10071 99981 0.24324 300.24324 150.24324 15174.56756 0.24324 300.24326 150.24324 15174.56758 0.24324 300.24324 150.24323999999987 15174.56724 2020-01-01 2020-01-02 2020-01-01 00:01:21 2020-01-02 03:46:21 2020-01-01 00:01:21.000 2020-01-02 03:46:21.000 81 99981 50031 5053131 81 99981 50031 5053131 -32488 32447 4610.009900990099 465611 -127 124 -3.0594059405940595 -309 -810 2 10800 99711 2.43243 299.43243 150.93243 15093.24324 2.43243 299.43243 150.93243 15093.24338 2.43243 299.43243 150.93243000000012 15093.24300 2020-01-01 2020-01-02 2020-01-01 00:13:30 2020-01-02 03:41:51 2020-01-01 00:13:30.000 2020-01-02 03:41:51.000 810 99711 50260.5 5026050 810 99711 50260.5 5026050 -32758 32177 4385.3 438530 -128 127 -2.54 -254 -811 2 10801 99712 2.43543 299.43543 150.93543 15093.54354 2.43543 299.43542 150.93543 15093.54353 2.43543 299.43543 150.9354299999998 15093.54300 2020-01-01 2020-01-02 2020-01-01 00:13:31 2020-01-02 03:41:52 2020-01-01 00:13:31.000 2020-01-02 03:41:52.000 811 99712 50261.5 5026150 811 99712 50261.5 5026150 -32757 32178 4386.3 438630 -128 124 -4.1 -410 -812 2 10802 99713 2.43843 299.43843 150.93843 15093.84384 2.43843 299.43845 150.93844 15093.84428 2.43843 299.43843 150.9384300000001 15093.84300 2020-01-01 2020-01-02 2020-01-01 00:13:32 2020-01-02 03:41:53 2020-01-01 00:13:32.000 2020-01-02 03:41:53.000 812 99713 50262.5 5026250 812 99713 50262.5 5026250 -32756 32179 4387.3 438730 -127 125 -3.1 -310 +806 2 10796 99707 2.42042 299.42042 150.92042 15092.04204 2.42042 299.4204 150.92042 15092.04205 2.42042 299.42042 150.92042 15092.04200 2020-01-01 2020-01-02 2020-01-01 00:13:26 2020-01-02 03:41:47 2020-01-01 00:13:26.000 2020-01-02 03:41:47.000 806 99707 50256.5 5025650 806 99707 50256.5 5025650 -32762 32173 4381.3 438130 -126 125 -1.42 -142 +807 2 10797 99708 2.42342 299.42342 150.92342 15092.34234 2.42342 299.42343 150.92342 15092.34216 2.42342 299.42342 150.92342 15092.34200 2020-01-01 2020-01-02 2020-01-01 00:13:27 2020-01-02 03:41:48 2020-01-01 00:13:27.000 2020-01-02 03:41:48.000 807 99708 50257.5 5025750 807 99708 50257.5 5025750 -32761 32174 4382.3 438230 -125 126 -0.42 -42 +808 2 10798 99709 2.42642 299.42642 150.92642 15092.64264 2.42642 299.42642 150.92642 15092.64245 2.42642 299.42642 150.92642 15092.64200 2020-01-01 2020-01-02 2020-01-01 00:13:28 2020-01-02 03:41:49 2020-01-01 00:13:28.000 2020-01-02 03:41:49.000 808 99709 50258.5 5025850 808 99709 50258.5 5025850 -32760 32175 4383.3 438330 -124 127 0.58 58 +809 2 10799 99710 2.42942 299.42942 150.92942 15092.94294 2.42942 299.42944 150.92943 15092.94305 2.42942 299.42942 150.92942 15092.94200 2020-01-01 2020-01-02 2020-01-01 00:13:29 2020-01-02 03:41:50 2020-01-01 00:13:29.000 2020-01-02 03:41:50.000 809 99710 50259.5 5025950 809 99710 50259.5 5025950 -32759 32176 4384.3 438430 -128 127 -0.98 -98 +81 2 10071 99981 0.24324 300.24324 150.24324 15174.56756 0.24324 300.24326 150.24324 15174.56758 0.24324 300.24324 150.24324000000001 15174.56724 2020-01-01 2020-01-02 2020-01-01 00:01:21 2020-01-02 03:46:21 2020-01-01 00:01:21.000 2020-01-02 03:46:21.000 81 99981 50031 5053131 81 99981 50031 5053131 -32488 32447 4610.009900990099 465611 -127 124 -3.0594059405940595 -309 +810 2 10800 99711 2.43243 299.43243 150.93243 15093.24324 2.43243 299.43243 150.93243 15093.24338 2.43243 299.43243 150.93243 15093.24300 2020-01-01 2020-01-02 2020-01-01 00:13:30 2020-01-02 03:41:51 2020-01-01 00:13:30.000 2020-01-02 03:41:51.000 810 99711 50260.5 5026050 810 99711 50260.5 5026050 -32758 32177 4385.3 438530 -128 127 -2.54 -254 +811 2 10801 99712 2.43543 299.43543 150.93543 15093.54354 2.43543 299.43542 150.93543 15093.54353 2.43543 299.43543 150.93543 15093.54300 2020-01-01 2020-01-02 2020-01-01 00:13:31 2020-01-02 03:41:52 2020-01-01 00:13:31.000 2020-01-02 03:41:52.000 811 99712 50261.5 5026150 811 99712 50261.5 5026150 -32757 32178 4386.3 438630 -128 124 -4.1 -410 +812 2 10802 99713 2.43843 299.43843 150.93843 15093.84384 2.43843 299.43845 150.93844 15093.84428 2.43843 299.43843 150.93843 15093.84300 2020-01-01 2020-01-02 2020-01-01 00:13:32 2020-01-02 03:41:53 2020-01-01 00:13:32.000 2020-01-02 03:41:53.000 812 99713 50262.5 5026250 812 99713 50262.5 5026250 -32756 32179 4387.3 438730 -127 125 -3.1 -310 813 2 10803 99714 2.44144 299.44144 150.94144 15094.14414 2.44144 299.44144 150.94143 15094.14392 2.44144 299.44144 150.94144 15094.14400 2020-01-01 2020-01-02 2020-01-01 00:13:33 2020-01-02 03:41:54 2020-01-01 00:13:33.000 2020-01-02 03:41:54.000 813 99714 50263.5 5026350 813 99714 50263.5 5026350 -32755 32180 4388.3 438830 -126 126 -2.1 -210 -814 2 10804 99715 2.44444 299.44444 150.94444 15094.44444 2.44444 299.44446 150.94444 15094.44452 2.44444 299.44444 150.94443999999996 15094.44400 2020-01-01 2020-01-02 2020-01-01 00:13:34 2020-01-02 03:41:55 2020-01-01 00:13:34.000 2020-01-02 03:41:55.000 814 99715 50264.5 5026450 814 99715 50264.5 5026450 -32754 32181 4389.3 438930 -125 127 -1.1 -110 -815 2 10805 99716 2.44744 299.44744 150.94744 15094.74474 2.44744 299.44745 150.94744 15094.74485 2.44744 299.44744 150.94743999999994 15094.74400 2020-01-01 2020-01-02 2020-01-01 00:13:35 2020-01-02 03:41:56 2020-01-01 00:13:35.000 2020-01-02 03:41:56.000 815 99716 50265.5 5026550 815 99716 50265.5 5026550 -32753 32182 4390.3 439030 -128 127 -2.66 -266 -816 2 10806 99717 2.45045 299.45045 150.95045 15095.04504 2.45045 299.45044 150.95045 15095.045 2.45045 299.45045 150.9504500000001 15095.04500 2020-01-01 2020-01-02 2020-01-01 00:13:36 2020-01-02 03:41:57 2020-01-01 00:13:36.000 2020-01-02 03:41:57.000 816 99717 50266.5 5026650 816 99717 50266.5 5026650 -32752 32183 4391.3 439130 -128 127 -4.22 -422 -817 2 10807 99718 2.45345 299.45345 150.95345 15095.34534 2.45345 299.45346 150.95345 15095.34574 2.45345 299.45345 150.9534500000002 15095.34500 2020-01-01 2020-01-02 2020-01-01 00:13:37 2020-01-02 03:41:58 2020-01-01 00:13:37.000 2020-01-02 03:41:58.000 817 99718 50267.5 5026750 817 99718 50267.5 5026750 -32751 32184 4392.3 439230 -128 123 -5.78 -578 -818 2 10808 99719 2.45645 299.45645 150.95645 15095.64564 2.45645 299.45645 150.95645 15095.64539 2.45645 299.45645 150.95644999999985 15095.64500 2020-01-01 2020-01-02 2020-01-01 00:13:38 2020-01-02 03:41:59 2020-01-01 00:13:38.000 2020-01-02 03:41:59.000 818 99719 50268.5 5026850 818 99719 50268.5 5026850 -32750 32185 4393.3 439330 -127 124 -4.78 -478 -819 2 10809 99720 2.45945 299.45945 150.95945 15095.94594 2.45945 299.45947 150.95946 15095.94602 2.45945 299.45945 150.95945000000012 15095.94500 2020-01-01 2020-01-02 2020-01-01 00:13:39 2020-01-02 03:42:00 2020-01-01 00:13:39.000 2020-01-02 03:42:00.000 819 99720 50269.5 5026950 819 99720 50269.5 5026950 -32749 32186 4394.3 439430 -126 125 -3.78 -378 -82 2 10072 99982 0.24624 300.24624 150.24624 15174.87087 0.24624 300.24625 150.24624 15174.87088 0.24624 300.24624 150.2462400000002 15174.87024 2020-01-01 2020-01-02 2020-01-01 00:01:22 2020-01-02 03:46:22 2020-01-01 00:01:22.000 2020-01-02 03:46:22.000 82 99982 50032 5053232 82 99982 50032 5053232 -32487 32448 4611.009900990099 465712 -126 125 -2.0594059405940595 -208 -820 2 10810 99721 2.46246 299.46246 150.96246 15096.24624 2.46246 299.46246 150.96246 15096.24633 2.46246 299.46246 150.9624600000001 15096.24600 2020-01-01 2020-01-02 2020-01-01 00:13:40 2020-01-02 03:42:01 2020-01-01 00:13:40.000 2020-01-02 03:42:01.000 820 99721 50270.5 5027050 820 99721 50270.5 5027050 -32748 32187 4395.3 439530 -125 126 -2.78 -278 -821 2 10811 99722 2.46546 299.46546 150.96546 15096.54654 2.46546 299.46545 150.96546 15096.54646 2.46546 299.46546 150.96545999999995 15096.54600 2020-01-01 2020-01-02 2020-01-01 00:13:41 2020-01-02 03:42:02 2020-01-01 00:13:41.000 2020-01-02 03:42:02.000 821 99722 50271.5 5027150 821 99722 50271.5 5027150 -32747 32188 4396.3 439630 -124 127 -1.78 -178 -822 2 10812 99723 2.46846 299.46846 150.96846 15096.84684 2.46846 299.46848 150.96847 15096.84721 2.46846 299.46846 150.96846000000002 15096.84600 2020-01-01 2020-01-02 2020-01-01 00:13:42 2020-01-02 03:42:03 2020-01-01 00:13:42.000 2020-01-02 03:42:03.000 822 99723 50272.5 5027250 822 99723 50272.5 5027250 -32746 32189 4397.3 439730 -128 127 -3.34 -334 -823 2 10813 99724 2.47147 299.47147 150.97147 15097.14714 2.47147 299.47147 150.97146 15097.14686 2.47147 299.47147 150.97147000000024 15097.14700 2020-01-01 2020-01-02 2020-01-01 00:13:43 2020-01-02 03:42:04 2020-01-01 00:13:43.000 2020-01-02 03:42:04.000 823 99724 50273.5 5027350 823 99724 50273.5 5027350 -32745 32190 4398.3 439830 -128 123 -4.9 -490 -824 2 10814 99725 2.47447 299.47447 150.97447 15097.44744 2.47447 299.4745 150.97447 15097.44749 2.47447 299.47447 150.97446999999985 15097.44700 2020-01-01 2020-01-02 2020-01-01 00:13:44 2020-01-02 03:42:05 2020-01-01 00:13:44.000 2020-01-02 03:42:05.000 824 99725 50274.5 5027450 824 99725 50274.5 5027450 -32744 32191 4399.3 439930 -127 124 -3.9 -390 -825 2 10815 99726 2.47747 299.47747 150.97747 15097.74774 2.47747 299.47748 150.97747 15097.74779 2.47747 299.47747 150.97746999999987 15097.74700 2020-01-01 2020-01-02 2020-01-01 00:13:45 2020-01-02 03:42:06 2020-01-01 00:13:45.000 2020-01-02 03:42:06.000 825 99726 50275.5 5027550 825 99726 50275.5 5027550 -32743 32192 4400.3 440030 -126 125 -2.9 -290 -826 2 10816 99727 2.48048 299.48048 150.98048 15098.04804 2.48048 299.48047 150.98048 15098.04809 2.48048 299.48048 150.9804800000001 15098.04800 2020-01-01 2020-01-02 2020-01-01 00:13:46 2020-01-02 03:42:07 2020-01-01 00:13:46.000 2020-01-02 03:42:07.000 826 99727 50276.5 5027650 826 99727 50276.5 5027650 -32742 32193 4401.3 440130 -125 126 -1.9 -190 -827 2 10817 99728 2.48348 299.48348 150.98348 15098.34834 2.48348 299.4835 150.98348 15098.34869 2.48348 299.48348 150.98348000000013 15098.34800 2020-01-01 2020-01-02 2020-01-01 00:13:47 2020-01-02 03:42:08 2020-01-01 00:13:47.000 2020-01-02 03:42:08.000 827 99728 50277.5 5027750 827 99728 50277.5 5027750 -32741 32194 4402.3 440230 -124 127 -0.9 -90 -828 2 10818 99729 2.48648 299.48648 150.98648 15098.64864 2.48648 299.48648 150.98648 15098.64837 2.48648 299.48648 150.98647999999977 15098.64800 2020-01-01 2020-01-02 2020-01-01 00:13:48 2020-01-02 03:42:09 2020-01-01 00:13:48.000 2020-01-02 03:42:09.000 828 99729 50278.5 5027850 828 99729 50278.5 5027850 -32740 32195 4403.3 440330 -128 127 -2.46 -246 -829 2 10819 99730 2.48948 299.48948 150.98948 15098.94894 2.48948 299.4895 150.98948 15098.94896 2.48948 299.48948 150.98948000000013 15098.94800 2020-01-01 2020-01-02 2020-01-01 00:13:49 2020-01-02 03:42:10 2020-01-01 00:13:49.000 2020-01-02 03:42:10.000 829 99730 50279.5 5027950 829 99730 50279.5 5027950 -32739 32196 4404.3 440430 -128 123 -4.02 -402 -83 2 10073 99983 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24923999999987 15175.17324 2020-01-01 2020-01-02 2020-01-01 00:01:23 2020-01-02 03:46:23 2020-01-01 00:01:23.000 2020-01-02 03:46:23.000 83 99983 50033 5053333 83 99983 50033 5053333 -32486 32449 4612.009900990099 465813 -125 126 -1.0594059405940595 -107 -830 2 10820 99731 2.49249 299.49249 150.99249 15099.24924 2.49249 299.4925 150.99249 15099.24926 2.49249 299.49249 150.99249000000003 15099.24900 2020-01-01 2020-01-02 2020-01-01 00:13:50 2020-01-02 03:42:11 2020-01-01 00:13:50.000 2020-01-02 03:42:11.000 830 99731 50280.5 5028050 830 99731 50280.5 5028050 -32738 32197 4405.3 440530 -127 124 -3.02 -302 -831 2 10821 99732 2.49549 299.49549 150.99549 15099.54954 2.49549 299.49548 150.99549 15099.54956 2.49549 299.49549 150.9954899999999 15099.54900 2020-01-01 2020-01-02 2020-01-01 00:13:51 2020-01-02 03:42:12 2020-01-01 00:13:51.000 2020-01-02 03:42:12.000 831 99732 50281.5 5028150 831 99732 50281.5 5028150 -32737 32198 4406.3 440630 -126 125 -2.02 -202 -832 2 10822 99733 2.49849 299.49849 150.99849 15099.84984 2.49849 299.4985 150.9985 15099.85016 2.49849 299.49849 150.99848999999998 15099.84900 2020-01-01 2020-01-02 2020-01-01 00:13:52 2020-01-02 03:42:13 2020-01-01 00:13:52.000 2020-01-02 03:42:13.000 832 99733 50282.5 5028250 832 99733 50282.5 5028250 -32736 32199 4407.3 440730 -125 126 -1.02 -102 -833 2 10823 99734 2.5015 299.5015 151.0015 15100.15015 2.5015 299.5015 151.00149 15100.14983 2.50150 299.50150 151.00150000000014 15100.15000 2020-01-01 2020-01-02 2020-01-01 00:13:53 2020-01-02 03:42:14 2020-01-01 00:13:53.000 2020-01-02 03:42:14.000 833 99734 50283.5 5028350 833 99734 50283.5 5028350 -32735 32200 4408.3 440830 -124 127 -0.02 -2 -834 2 10824 99735 2.5045 299.5045 151.0045 15100.45045 2.5045 299.50452 151.0045 15100.45043 2.50450 299.50450 151.00449999999984 15100.45000 2020-01-01 2020-01-02 2020-01-01 00:13:54 2020-01-02 03:42:15 2020-01-01 00:13:54.000 2020-01-02 03:42:15.000 834 99735 50284.5 5028450 834 99735 50284.5 5028450 -32734 32201 4409.3 440930 -128 127 -1.58 -158 -835 2 10825 99736 2.5075 299.5075 151.0075 15100.75075 2.5075 299.5075 151.0075 15100.75073 2.50750 299.50750 151.00749999999988 15100.75000 2020-01-01 2020-01-02 2020-01-01 00:13:55 2020-01-02 03:42:16 2020-01-01 00:13:55.000 2020-01-02 03:42:16.000 835 99736 50285.5 5028550 835 99736 50285.5 5028550 -32733 32202 4410.3 441030 -128 123 -3.14 -314 -836 2 10826 99737 2.51051 299.51051 151.01051 15101.05105 2.51051 299.5105 151.01051 15101.05103 2.51051 299.51051 151.01051000000004 15101.05100 2020-01-01 2020-01-02 2020-01-01 00:13:56 2020-01-02 03:42:17 2020-01-01 00:13:56.000 2020-01-02 03:42:17.000 836 99737 50286.5 5028650 836 99737 50286.5 5028650 -32732 32203 4411.3 441130 -127 124 -2.14 -214 -837 2 10827 99738 2.51351 299.51351 151.01351 15101.35135 2.51351 299.51352 151.01351 15101.35162 2.51351 299.51351 151.01351000000005 15101.35100 2020-01-01 2020-01-02 2020-01-01 00:13:57 2020-01-02 03:42:18 2020-01-01 00:13:57.000 2020-01-02 03:42:18.000 837 99738 50287.5 5028750 837 99738 50287.5 5028750 -32731 32204 4412.3 441230 -126 125 -1.14 -114 -838 2 10828 99739 2.51651 299.51651 151.01651 15101.65165 2.51651 299.5165 151.01651 15101.6513 2.51651 299.51651 151.01650999999995 15101.65100 2020-01-01 2020-01-02 2020-01-01 00:13:58 2020-01-02 03:42:19 2020-01-01 00:13:58.000 2020-01-02 03:42:19.000 838 99739 50288.5 5028850 838 99739 50288.5 5028850 -32730 32205 4413.3 441330 -125 126 -0.14 -14 -839 2 10829 99740 2.51951 299.51951 151.01951 15101.95195 2.51951 299.51953 151.01951 15101.9519 2.51951 299.51951 151.01951000000005 15101.95100 2020-01-01 2020-01-02 2020-01-01 00:13:59 2020-01-02 03:42:20 2020-01-01 00:13:59.000 2020-01-02 03:42:20.000 839 99740 50289.5 5028950 839 99740 50289.5 5028950 -32729 32206 4414.3 441430 -124 127 0.86 86 -84 2 10074 99984 0.25225 300.25225 150.25225 15175.47747 0.25225 300.25226 150.25225 15175.47778 0.25225 300.25225 150.25224999999975 15175.47725 2020-01-01 2020-01-02 2020-01-01 00:01:24 2020-01-02 03:46:24 2020-01-01 00:01:24.000 2020-01-02 03:46:24.000 84 99984 50034 5053434 84 99984 50034 5053434 -32485 32450 4613.009900990099 465914 -124 127 -0.0594059405940594 -6 -840 2 10830 99741 2.52252 299.52252 151.02252 15102.25225 2.52252 299.52252 151.02252 15102.2522 2.52252 299.52252 151.0225200000002 15102.25200 2020-01-01 2020-01-02 2020-01-01 00:14:00 2020-01-02 03:42:21 2020-01-01 00:14:00.000 2020-01-02 03:42:21.000 840 99741 50290.5 5029050 840 99741 50290.5 5029050 -32728 32207 4415.3 441530 -128 127 -0.7 -70 -841 2 10831 99742 2.52552 299.52552 151.02552 15102.55255 2.52552 299.5255 151.02552 15102.5525 2.52552 299.52552 151.02551999999991 15102.55200 2020-01-01 2020-01-02 2020-01-01 00:14:01 2020-01-02 03:42:22 2020-01-01 00:14:01.000 2020-01-02 03:42:22.000 841 99742 50291.5 5029150 841 99742 50291.5 5029150 -32727 32208 4416.3 441630 -128 127 -2.26 -226 -842 2 10832 99743 2.52852 299.52852 151.02852 15102.85285 2.52852 299.52853 151.02853 15102.85313 2.52852 299.52852 151.0285199999999 15102.85200 2020-01-01 2020-01-02 2020-01-01 00:14:02 2020-01-02 03:42:23 2020-01-01 00:14:02.000 2020-01-02 03:42:23.000 842 99743 50292.5 5029250 842 99743 50292.5 5029250 -32726 32209 4417.3 441730 -128 123 -3.82 -382 -843 2 10833 99744 2.53153 299.53153 151.03153 15103.15315 2.53153 299.53152 151.03152 15103.15278 2.53153 299.53153 151.03153000000015 15103.15300 2020-01-01 2020-01-02 2020-01-01 00:14:03 2020-01-02 03:42:24 2020-01-01 00:14:03.000 2020-01-02 03:42:24.000 843 99744 50293.5 5029350 843 99744 50293.5 5029350 -32725 32210 4418.3 441830 -127 124 -2.82 -282 -844 2 10834 99745 2.53453 299.53453 151.03453 15103.45345 2.53453 299.53455 151.03453 15103.45353 2.53453 299.53453 151.03453000000013 15103.45300 2020-01-01 2020-01-02 2020-01-01 00:14:04 2020-01-02 03:42:25 2020-01-01 00:14:04.000 2020-01-02 03:42:25.000 844 99745 50294.5 5029450 844 99745 50294.5 5029450 -32724 32211 4419.3 441930 -126 125 -1.82 -182 -845 2 10835 99746 2.53753 299.53753 151.03753 15103.75375 2.53753 299.53754 151.03753 15103.75366 2.53753 299.53753 151.0375299999998 15103.75300 2020-01-01 2020-01-02 2020-01-01 00:14:05 2020-01-02 03:42:26 2020-01-01 00:14:05.000 2020-01-02 03:42:26.000 845 99746 50295.5 5029550 845 99746 50295.5 5029550 -32723 32212 4420.3 442030 -125 126 -0.82 -82 +814 2 10804 99715 2.44444 299.44444 150.94444 15094.44444 2.44444 299.44446 150.94444 15094.44452 2.44444 299.44444 150.94444 15094.44400 2020-01-01 2020-01-02 2020-01-01 00:13:34 2020-01-02 03:41:55 2020-01-01 00:13:34.000 2020-01-02 03:41:55.000 814 99715 50264.5 5026450 814 99715 50264.5 5026450 -32754 32181 4389.3 438930 -125 127 -1.1 -110 +815 2 10805 99716 2.44744 299.44744 150.94744 15094.74474 2.44744 299.44745 150.94744 15094.74485 2.44744 299.44744 150.94744 15094.74400 2020-01-01 2020-01-02 2020-01-01 00:13:35 2020-01-02 03:41:56 2020-01-01 00:13:35.000 2020-01-02 03:41:56.000 815 99716 50265.5 5026550 815 99716 50265.5 5026550 -32753 32182 4390.3 439030 -128 127 -2.66 -266 +816 2 10806 99717 2.45045 299.45045 150.95045 15095.04504 2.45045 299.45044 150.95045 15095.045 2.45045 299.45045 150.95045 15095.04500 2020-01-01 2020-01-02 2020-01-01 00:13:36 2020-01-02 03:41:57 2020-01-01 00:13:36.000 2020-01-02 03:41:57.000 816 99717 50266.5 5026650 816 99717 50266.5 5026650 -32752 32183 4391.3 439130 -128 127 -4.22 -422 +817 2 10807 99718 2.45345 299.45345 150.95345 15095.34534 2.45345 299.45346 150.95345 15095.34574 2.45345 299.45345 150.95345 15095.34500 2020-01-01 2020-01-02 2020-01-01 00:13:37 2020-01-02 03:41:58 2020-01-01 00:13:37.000 2020-01-02 03:41:58.000 817 99718 50267.5 5026750 817 99718 50267.5 5026750 -32751 32184 4392.3 439230 -128 123 -5.78 -578 +818 2 10808 99719 2.45645 299.45645 150.95645 15095.64564 2.45645 299.45645 150.95645 15095.64539 2.45645 299.45645 150.95645000000002 15095.64500 2020-01-01 2020-01-02 2020-01-01 00:13:38 2020-01-02 03:41:59 2020-01-01 00:13:38.000 2020-01-02 03:41:59.000 818 99719 50268.5 5026850 818 99719 50268.5 5026850 -32750 32185 4393.3 439330 -127 124 -4.78 -478 +819 2 10809 99720 2.45945 299.45945 150.95945 15095.94594 2.45945 299.45947 150.95946 15095.94602 2.45945 299.45945 150.95945 15095.94500 2020-01-01 2020-01-02 2020-01-01 00:13:39 2020-01-02 03:42:00 2020-01-01 00:13:39.000 2020-01-02 03:42:00.000 819 99720 50269.5 5026950 819 99720 50269.5 5026950 -32749 32186 4394.3 439430 -126 125 -3.78 -378 +82 2 10072 99982 0.24624 300.24624 150.24624 15174.87087 0.24624 300.24625 150.24624 15174.87088 0.24624 300.24624 150.24624 15174.87024 2020-01-01 2020-01-02 2020-01-01 00:01:22 2020-01-02 03:46:22 2020-01-01 00:01:22.000 2020-01-02 03:46:22.000 82 99982 50032 5053232 82 99982 50032 5053232 -32487 32448 4611.009900990099 465712 -126 125 -2.0594059405940595 -208 +820 2 10810 99721 2.46246 299.46246 150.96246 15096.24624 2.46246 299.46246 150.96246 15096.24633 2.46246 299.46246 150.96246 15096.24600 2020-01-01 2020-01-02 2020-01-01 00:13:40 2020-01-02 03:42:01 2020-01-01 00:13:40.000 2020-01-02 03:42:01.000 820 99721 50270.5 5027050 820 99721 50270.5 5027050 -32748 32187 4395.3 439530 -125 126 -2.78 -278 +821 2 10811 99722 2.46546 299.46546 150.96546 15096.54654 2.46546 299.46545 150.96546 15096.54646 2.46546 299.46546 150.96546 15096.54600 2020-01-01 2020-01-02 2020-01-01 00:13:41 2020-01-02 03:42:02 2020-01-01 00:13:41.000 2020-01-02 03:42:02.000 821 99722 50271.5 5027150 821 99722 50271.5 5027150 -32747 32188 4396.3 439630 -124 127 -1.78 -178 +822 2 10812 99723 2.46846 299.46846 150.96846 15096.84684 2.46846 299.46848 150.96847 15096.84721 2.46846 299.46846 150.96846 15096.84600 2020-01-01 2020-01-02 2020-01-01 00:13:42 2020-01-02 03:42:03 2020-01-01 00:13:42.000 2020-01-02 03:42:03.000 822 99723 50272.5 5027250 822 99723 50272.5 5027250 -32746 32189 4397.3 439730 -128 127 -3.34 -334 +823 2 10813 99724 2.47147 299.47147 150.97147 15097.14714 2.47147 299.47147 150.97146 15097.14686 2.47147 299.47147 150.97147 15097.14700 2020-01-01 2020-01-02 2020-01-01 00:13:43 2020-01-02 03:42:04 2020-01-01 00:13:43.000 2020-01-02 03:42:04.000 823 99724 50273.5 5027350 823 99724 50273.5 5027350 -32745 32190 4398.3 439830 -128 123 -4.9 -490 +824 2 10814 99725 2.47447 299.47447 150.97447 15097.44744 2.47447 299.4745 150.97447 15097.44749 2.47447 299.47447 150.97447 15097.44700 2020-01-01 2020-01-02 2020-01-01 00:13:44 2020-01-02 03:42:05 2020-01-01 00:13:44.000 2020-01-02 03:42:05.000 824 99725 50274.5 5027450 824 99725 50274.5 5027450 -32744 32191 4399.3 439930 -127 124 -3.9 -390 +825 2 10815 99726 2.47747 299.47747 150.97747 15097.74774 2.47747 299.47748 150.97747 15097.74779 2.47747 299.47747 150.97746999999998 15097.74700 2020-01-01 2020-01-02 2020-01-01 00:13:45 2020-01-02 03:42:06 2020-01-01 00:13:45.000 2020-01-02 03:42:06.000 825 99726 50275.5 5027550 825 99726 50275.5 5027550 -32743 32192 4400.3 440030 -126 125 -2.9 -290 +826 2 10816 99727 2.48048 299.48048 150.98048 15098.04804 2.48048 299.48047 150.98048 15098.04809 2.48048 299.48048 150.98048 15098.04800 2020-01-01 2020-01-02 2020-01-01 00:13:46 2020-01-02 03:42:07 2020-01-01 00:13:46.000 2020-01-02 03:42:07.000 826 99727 50276.5 5027650 826 99727 50276.5 5027650 -32742 32193 4401.3 440130 -125 126 -1.9 -190 +827 2 10817 99728 2.48348 299.48348 150.98348 15098.34834 2.48348 299.4835 150.98348 15098.34869 2.48348 299.48348 150.98348 15098.34800 2020-01-01 2020-01-02 2020-01-01 00:13:47 2020-01-02 03:42:08 2020-01-01 00:13:47.000 2020-01-02 03:42:08.000 827 99728 50277.5 5027750 827 99728 50277.5 5027750 -32741 32194 4402.3 440230 -124 127 -0.9 -90 +828 2 10818 99729 2.48648 299.48648 150.98648 15098.64864 2.48648 299.48648 150.98648 15098.64837 2.48648 299.48648 150.98648 15098.64800 2020-01-01 2020-01-02 2020-01-01 00:13:48 2020-01-02 03:42:09 2020-01-01 00:13:48.000 2020-01-02 03:42:09.000 828 99729 50278.5 5027850 828 99729 50278.5 5027850 -32740 32195 4403.3 440330 -128 127 -2.46 -246 +829 2 10819 99730 2.48948 299.48948 150.98948 15098.94894 2.48948 299.4895 150.98948 15098.94896 2.48948 299.48948 150.98948000000001 15098.94800 2020-01-01 2020-01-02 2020-01-01 00:13:49 2020-01-02 03:42:10 2020-01-01 00:13:49.000 2020-01-02 03:42:10.000 829 99730 50279.5 5027950 829 99730 50279.5 5027950 -32739 32196 4404.3 440430 -128 123 -4.02 -402 +83 2 10073 99983 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924 15175.17417 0.24924 300.24924 150.24924000000001 15175.17324 2020-01-01 2020-01-02 2020-01-01 00:01:23 2020-01-02 03:46:23 2020-01-01 00:01:23.000 2020-01-02 03:46:23.000 83 99983 50033 5053333 83 99983 50033 5053333 -32486 32449 4612.009900990099 465813 -125 126 -1.0594059405940595 -107 +830 2 10820 99731 2.49249 299.49249 150.99249 15099.24924 2.49249 299.4925 150.99249 15099.24926 2.49249 299.49249 150.99249 15099.24900 2020-01-01 2020-01-02 2020-01-01 00:13:50 2020-01-02 03:42:11 2020-01-01 00:13:50.000 2020-01-02 03:42:11.000 830 99731 50280.5 5028050 830 99731 50280.5 5028050 -32738 32197 4405.3 440530 -127 124 -3.02 -302 +831 2 10821 99732 2.49549 299.49549 150.99549 15099.54954 2.49549 299.49548 150.99549 15099.54956 2.49549 299.49549 150.99549000000002 15099.54900 2020-01-01 2020-01-02 2020-01-01 00:13:51 2020-01-02 03:42:12 2020-01-01 00:13:51.000 2020-01-02 03:42:12.000 831 99732 50281.5 5028150 831 99732 50281.5 5028150 -32737 32198 4406.3 440630 -126 125 -2.02 -202 +832 2 10822 99733 2.49849 299.49849 150.99849 15099.84984 2.49849 299.4985 150.9985 15099.85016 2.49849 299.49849 150.99849 15099.84900 2020-01-01 2020-01-02 2020-01-01 00:13:52 2020-01-02 03:42:13 2020-01-01 00:13:52.000 2020-01-02 03:42:13.000 832 99733 50282.5 5028250 832 99733 50282.5 5028250 -32736 32199 4407.3 440730 -125 126 -1.02 -102 +833 2 10823 99734 2.5015 299.5015 151.0015 15100.15015 2.5015 299.5015 151.00149 15100.14983 2.50150 299.50150 151.0015 15100.15000 2020-01-01 2020-01-02 2020-01-01 00:13:53 2020-01-02 03:42:14 2020-01-01 00:13:53.000 2020-01-02 03:42:14.000 833 99734 50283.5 5028350 833 99734 50283.5 5028350 -32735 32200 4408.3 440830 -124 127 -0.02 -2 +834 2 10824 99735 2.5045 299.5045 151.0045 15100.45045 2.5045 299.50452 151.0045 15100.45043 2.50450 299.50450 151.0045 15100.45000 2020-01-01 2020-01-02 2020-01-01 00:13:54 2020-01-02 03:42:15 2020-01-01 00:13:54.000 2020-01-02 03:42:15.000 834 99735 50284.5 5028450 834 99735 50284.5 5028450 -32734 32201 4409.3 440930 -128 127 -1.58 -158 +835 2 10825 99736 2.5075 299.5075 151.0075 15100.75075 2.5075 299.5075 151.0075 15100.75073 2.50750 299.50750 151.0075 15100.75000 2020-01-01 2020-01-02 2020-01-01 00:13:55 2020-01-02 03:42:16 2020-01-01 00:13:55.000 2020-01-02 03:42:16.000 835 99736 50285.5 5028550 835 99736 50285.5 5028550 -32733 32202 4410.3 441030 -128 123 -3.14 -314 +836 2 10826 99737 2.51051 299.51051 151.01051 15101.05105 2.51051 299.5105 151.01051 15101.05103 2.51051 299.51051 151.01050999999998 15101.05100 2020-01-01 2020-01-02 2020-01-01 00:13:56 2020-01-02 03:42:17 2020-01-01 00:13:56.000 2020-01-02 03:42:17.000 836 99737 50286.5 5028650 836 99737 50286.5 5028650 -32732 32203 4411.3 441130 -127 124 -2.14 -214 +837 2 10827 99738 2.51351 299.51351 151.01351 15101.35135 2.51351 299.51352 151.01351 15101.35162 2.51351 299.51351 151.01351 15101.35100 2020-01-01 2020-01-02 2020-01-01 00:13:57 2020-01-02 03:42:18 2020-01-01 00:13:57.000 2020-01-02 03:42:18.000 837 99738 50287.5 5028750 837 99738 50287.5 5028750 -32731 32204 4412.3 441230 -126 125 -1.14 -114 +838 2 10828 99739 2.51651 299.51651 151.01651 15101.65165 2.51651 299.5165 151.01651 15101.6513 2.51651 299.51651 151.01651 15101.65100 2020-01-01 2020-01-02 2020-01-01 00:13:58 2020-01-02 03:42:19 2020-01-01 00:13:58.000 2020-01-02 03:42:19.000 838 99739 50288.5 5028850 838 99739 50288.5 5028850 -32730 32205 4413.3 441330 -125 126 -0.14 -14 +839 2 10829 99740 2.51951 299.51951 151.01951 15101.95195 2.51951 299.51953 151.01951 15101.9519 2.51951 299.51951 151.01951 15101.95100 2020-01-01 2020-01-02 2020-01-01 00:13:59 2020-01-02 03:42:20 2020-01-01 00:13:59.000 2020-01-02 03:42:20.000 839 99740 50289.5 5028950 839 99740 50289.5 5028950 -32729 32206 4414.3 441430 -124 127 0.86 86 +84 2 10074 99984 0.25225 300.25225 150.25225 15175.47747 0.25225 300.25226 150.25225 15175.47778 0.25225 300.25225 150.25225 15175.47725 2020-01-01 2020-01-02 2020-01-01 00:01:24 2020-01-02 03:46:24 2020-01-01 00:01:24.000 2020-01-02 03:46:24.000 84 99984 50034 5053434 84 99984 50034 5053434 -32485 32450 4613.009900990099 465914 -124 127 -0.0594059405940594 -6 +840 2 10830 99741 2.52252 299.52252 151.02252 15102.25225 2.52252 299.52252 151.02252 15102.2522 2.52252 299.52252 151.02252000000001 15102.25200 2020-01-01 2020-01-02 2020-01-01 00:14:00 2020-01-02 03:42:21 2020-01-01 00:14:00.000 2020-01-02 03:42:21.000 840 99741 50290.5 5029050 840 99741 50290.5 5029050 -32728 32207 4415.3 441530 -128 127 -0.7 -70 +841 2 10831 99742 2.52552 299.52552 151.02552 15102.55255 2.52552 299.5255 151.02552 15102.5525 2.52552 299.52552 151.02552 15102.55200 2020-01-01 2020-01-02 2020-01-01 00:14:01 2020-01-02 03:42:22 2020-01-01 00:14:01.000 2020-01-02 03:42:22.000 841 99742 50291.5 5029150 841 99742 50291.5 5029150 -32727 32208 4416.3 441630 -128 127 -2.26 -226 +842 2 10832 99743 2.52852 299.52852 151.02852 15102.85285 2.52852 299.52853 151.02853 15102.85313 2.52852 299.52852 151.02852000000001 15102.85200 2020-01-01 2020-01-02 2020-01-01 00:14:02 2020-01-02 03:42:23 2020-01-01 00:14:02.000 2020-01-02 03:42:23.000 842 99743 50292.5 5029250 842 99743 50292.5 5029250 -32726 32209 4417.3 441730 -128 123 -3.82 -382 +843 2 10833 99744 2.53153 299.53153 151.03153 15103.15315 2.53153 299.53152 151.03152 15103.15278 2.53153 299.53153 151.03153 15103.15300 2020-01-01 2020-01-02 2020-01-01 00:14:03 2020-01-02 03:42:24 2020-01-01 00:14:03.000 2020-01-02 03:42:24.000 843 99744 50293.5 5029350 843 99744 50293.5 5029350 -32725 32210 4418.3 441830 -127 124 -2.82 -282 +844 2 10834 99745 2.53453 299.53453 151.03453 15103.45345 2.53453 299.53455 151.03453 15103.45353 2.53453 299.53453 151.03453 15103.45300 2020-01-01 2020-01-02 2020-01-01 00:14:04 2020-01-02 03:42:25 2020-01-01 00:14:04.000 2020-01-02 03:42:25.000 844 99745 50294.5 5029450 844 99745 50294.5 5029450 -32724 32211 4419.3 441930 -126 125 -1.82 -182 +845 2 10835 99746 2.53753 299.53753 151.03753 15103.75375 2.53753 299.53754 151.03753 15103.75366 2.53753 299.53753 151.03753 15103.75300 2020-01-01 2020-01-02 2020-01-01 00:14:05 2020-01-02 03:42:26 2020-01-01 00:14:05.000 2020-01-02 03:42:26.000 845 99746 50295.5 5029550 845 99746 50295.5 5029550 -32723 32212 4420.3 442030 -125 126 -0.82 -82 846 2 10836 99747 2.54054 299.54054 151.04054 15104.05405 2.54054 299.54053 151.04053 15104.05397 2.54054 299.54054 151.04054 15104.05400 2020-01-01 2020-01-02 2020-01-01 00:14:06 2020-01-02 03:42:27 2020-01-01 00:14:06.000 2020-01-02 03:42:27.000 846 99747 50296.5 5029650 846 99747 50296.5 5029650 -32722 32213 4421.3 442130 -124 127 0.18 18 -847 2 10837 99748 2.54354 299.54354 151.04354 15104.35435 2.54354 299.54355 151.04354 15104.3546 2.54354 299.54354 151.04354000000006 15104.35400 2020-01-01 2020-01-02 2020-01-01 00:14:07 2020-01-02 03:42:28 2020-01-01 00:14:07.000 2020-01-02 03:42:28.000 847 99748 50297.5 5029750 847 99748 50297.5 5029750 -32721 32214 4422.3 442230 -128 127 -1.38 -138 -848 2 10838 99749 2.54654 299.54654 151.04654 15104.65465 2.54654 299.54654 151.04654 15104.65425 2.54654 299.54654 151.0465399999999 15104.65400 2020-01-01 2020-01-02 2020-01-01 00:14:08 2020-01-02 03:42:29 2020-01-01 00:14:08.000 2020-01-02 03:42:29.000 848 99749 50298.5 5029850 848 99749 50298.5 5029850 -32720 32215 4423.3 442330 -128 123 -2.94 -294 -849 2 10839 99750 2.54954 299.54954 151.04954 15104.95495 2.54954 299.54956 151.04954 15104.95499 2.54954 299.54954 151.04953999999998 15104.95400 2020-01-01 2020-01-02 2020-01-01 00:14:09 2020-01-02 03:42:30 2020-01-01 00:14:09.000 2020-01-02 03:42:30.000 849 99750 50299.5 5029950 849 99750 50299.5 5029950 -32719 32216 4424.3 442430 -127 124 -1.94 -194 -85 2 10075 99985 0.25525 300.25525 150.25525 15175.78078 0.25525 300.25525 150.25525 15175.78046 0.25525 300.25525 150.2552500000001 15175.78025 2020-01-01 2020-01-02 2020-01-01 00:01:25 2020-01-02 03:46:25 2020-01-01 00:01:25.000 2020-01-02 03:46:25.000 85 99985 50035 5053535 85 99985 50035 5053535 -32484 32451 4614.009900990099 466015 -128 127 -1.5940594059405941 -161 -850 2 10840 99751 2.55255 299.55255 151.05255 15105.25525 2.55255 299.55255 151.05255 15105.25514 2.55255 299.55255 151.05255000000014 15105.25500 2020-01-01 2020-01-02 2020-01-01 00:14:10 2020-01-02 03:42:31 2020-01-01 00:14:10.000 2020-01-02 03:42:31.000 850 99751 50300.5 5030050 850 99751 50300.5 5030050 -32718 32217 4425.3 442530 -126 125 -0.94 -94 -851 2 10841 99752 2.55555 299.55555 151.05555 15105.55555 2.55555 299.55554 151.05555 15105.55547 2.55555 299.55555 151.0555499999998 15105.55500 2020-01-01 2020-01-02 2020-01-01 00:14:11 2020-01-02 03:42:32 2020-01-01 00:14:11.000 2020-01-02 03:42:32.000 851 99752 50301.5 5030150 851 99752 50301.5 5030150 -32717 32218 4426.3 442630 -125 126 0.06 6 -852 2 10842 99753 2.55855 299.55855 151.05855 15105.85585 2.55855 299.55856 151.05856 15105.85607 2.55855 299.55855 151.0585499999999 15105.85500 2020-01-01 2020-01-02 2020-01-01 00:14:12 2020-01-02 03:42:33 2020-01-01 00:14:12.000 2020-01-02 03:42:33.000 852 99753 50302.5 5030250 852 99753 50302.5 5030250 -32716 32219 4427.3 442730 -124 127 1.06 106 -853 2 10843 99754 2.56156 299.56156 151.06156 15106.15615 2.56156 299.56155 151.06155 15106.15571 2.56156 299.56156 151.06156000000007 15106.15600 2020-01-01 2020-01-02 2020-01-01 00:14:13 2020-01-02 03:42:34 2020-01-01 00:14:13.000 2020-01-02 03:42:34.000 853 99754 50303.5 5030350 853 99754 50303.5 5030350 -32715 32220 4428.3 442830 -128 127 -0.5 -50 -854 2 10844 99755 2.56456 299.56456 151.06456 15106.45645 2.56456 299.56458 151.06456 15106.45646 2.56456 299.56456 151.06456000000009 15106.45600 2020-01-01 2020-01-02 2020-01-01 00:14:14 2020-01-02 03:42:35 2020-01-01 00:14:14.000 2020-01-02 03:42:35.000 854 99755 50304.5 5030450 854 99755 50304.5 5030450 -32714 32221 4429.3 442930 -128 123 -2.06 -206 -855 2 10845 99756 2.56756 299.56756 151.06756 15106.75675 2.56756 299.56757 151.06756 15106.75661 2.56756 299.56756 151.06756000000001 15106.75600 2020-01-01 2020-01-02 2020-01-01 00:14:15 2020-01-02 03:42:36 2020-01-01 00:14:15.000 2020-01-02 03:42:36.000 855 99756 50305.5 5030550 855 99756 50305.5 5030550 -32713 32222 4430.3 443030 -127 124 -1.06 -106 -856 2 10846 99757 2.57057 299.57057 151.07057 15107.05705 2.57057 299.57056 151.07056 15107.05694 2.57057 299.57057 151.0705699999999 15107.05700 2020-01-01 2020-01-02 2020-01-01 00:14:16 2020-01-02 03:42:37 2020-01-01 00:14:16.000 2020-01-02 03:42:37.000 856 99757 50306.5 5030650 856 99757 50306.5 5030650 -32712 32223 4431.3 443130 -126 125 -0.06 -6 +847 2 10837 99748 2.54354 299.54354 151.04354 15104.35435 2.54354 299.54355 151.04354 15104.3546 2.54354 299.54354 151.04354 15104.35400 2020-01-01 2020-01-02 2020-01-01 00:14:07 2020-01-02 03:42:28 2020-01-01 00:14:07.000 2020-01-02 03:42:28.000 847 99748 50297.5 5029750 847 99748 50297.5 5029750 -32721 32214 4422.3 442230 -128 127 -1.38 -138 +848 2 10838 99749 2.54654 299.54654 151.04654 15104.65465 2.54654 299.54654 151.04654 15104.65425 2.54654 299.54654 151.04654 15104.65400 2020-01-01 2020-01-02 2020-01-01 00:14:08 2020-01-02 03:42:29 2020-01-01 00:14:08.000 2020-01-02 03:42:29.000 848 99749 50298.5 5029850 848 99749 50298.5 5029850 -32720 32215 4423.3 442330 -128 123 -2.94 -294 +849 2 10839 99750 2.54954 299.54954 151.04954 15104.95495 2.54954 299.54956 151.04954 15104.95499 2.54954 299.54954 151.04954 15104.95400 2020-01-01 2020-01-02 2020-01-01 00:14:09 2020-01-02 03:42:30 2020-01-01 00:14:09.000 2020-01-02 03:42:30.000 849 99750 50299.5 5029950 849 99750 50299.5 5029950 -32719 32216 4424.3 442430 -127 124 -1.94 -194 +85 2 10075 99985 0.25525 300.25525 150.25525 15175.78078 0.25525 300.25525 150.25525 15175.78046 0.25525 300.25525 150.25525 15175.78025 2020-01-01 2020-01-02 2020-01-01 00:01:25 2020-01-02 03:46:25 2020-01-01 00:01:25.000 2020-01-02 03:46:25.000 85 99985 50035 5053535 85 99985 50035 5053535 -32484 32451 4614.009900990099 466015 -128 127 -1.5940594059405941 -161 +850 2 10840 99751 2.55255 299.55255 151.05255 15105.25525 2.55255 299.55255 151.05255 15105.25514 2.55255 299.55255 151.05255 15105.25500 2020-01-01 2020-01-02 2020-01-01 00:14:10 2020-01-02 03:42:31 2020-01-01 00:14:10.000 2020-01-02 03:42:31.000 850 99751 50300.5 5030050 850 99751 50300.5 5030050 -32718 32217 4425.3 442530 -126 125 -0.94 -94 +851 2 10841 99752 2.55555 299.55555 151.05555 15105.55555 2.55555 299.55554 151.05555 15105.55547 2.55555 299.55555 151.05555 15105.55500 2020-01-01 2020-01-02 2020-01-01 00:14:11 2020-01-02 03:42:32 2020-01-01 00:14:11.000 2020-01-02 03:42:32.000 851 99752 50301.5 5030150 851 99752 50301.5 5030150 -32717 32218 4426.3 442630 -125 126 0.06 6 +852 2 10842 99753 2.55855 299.55855 151.05855 15105.85585 2.55855 299.55856 151.05856 15105.85607 2.55855 299.55855 151.05855 15105.85500 2020-01-01 2020-01-02 2020-01-01 00:14:12 2020-01-02 03:42:33 2020-01-01 00:14:12.000 2020-01-02 03:42:33.000 852 99753 50302.5 5030250 852 99753 50302.5 5030250 -32716 32219 4427.3 442730 -124 127 1.06 106 +853 2 10843 99754 2.56156 299.56156 151.06156 15106.15615 2.56156 299.56155 151.06155 15106.15571 2.56156 299.56156 151.06156000000001 15106.15600 2020-01-01 2020-01-02 2020-01-01 00:14:13 2020-01-02 03:42:34 2020-01-01 00:14:13.000 2020-01-02 03:42:34.000 853 99754 50303.5 5030350 853 99754 50303.5 5030350 -32715 32220 4428.3 442830 -128 127 -0.5 -50 +854 2 10844 99755 2.56456 299.56456 151.06456 15106.45645 2.56456 299.56458 151.06456 15106.45646 2.56456 299.56456 151.06456 15106.45600 2020-01-01 2020-01-02 2020-01-01 00:14:14 2020-01-02 03:42:35 2020-01-01 00:14:14.000 2020-01-02 03:42:35.000 854 99755 50304.5 5030450 854 99755 50304.5 5030450 -32714 32221 4429.3 442930 -128 123 -2.06 -206 +855 2 10845 99756 2.56756 299.56756 151.06756 15106.75675 2.56756 299.56757 151.06756 15106.75661 2.56756 299.56756 151.06756 15106.75600 2020-01-01 2020-01-02 2020-01-01 00:14:15 2020-01-02 03:42:36 2020-01-01 00:14:15.000 2020-01-02 03:42:36.000 855 99756 50305.5 5030550 855 99756 50305.5 5030550 -32713 32222 4430.3 443030 -127 124 -1.06 -106 +856 2 10846 99757 2.57057 299.57057 151.07057 15107.05705 2.57057 299.57056 151.07056 15107.05694 2.57057 299.57057 151.07057 15107.05700 2020-01-01 2020-01-02 2020-01-01 00:14:16 2020-01-02 03:42:37 2020-01-01 00:14:16.000 2020-01-02 03:42:37.000 856 99757 50306.5 5030650 856 99757 50306.5 5030650 -32712 32223 4431.3 443130 -126 125 -0.06 -6 857 2 10847 99758 2.57357 299.57357 151.07357 15107.35735 2.57357 299.57358 151.07357 15107.35754 2.57357 299.57357 151.07357 15107.35700 2020-01-01 2020-01-02 2020-01-01 00:14:17 2020-01-02 03:42:38 2020-01-01 00:14:17.000 2020-01-02 03:42:38.000 857 99758 50307.5 5030750 857 99758 50307.5 5030750 -32711 32224 4432.3 443230 -125 126 0.94 94 -858 2 10848 99759 2.57657 299.57657 151.07657 15107.65765 2.57657 299.57657 151.07657 15107.65783 2.57657 299.57657 151.07656999999995 15107.65700 2020-01-01 2020-01-02 2020-01-01 00:14:18 2020-01-02 03:42:39 2020-01-01 00:14:18.000 2020-01-02 03:42:39.000 858 99759 50308.5 5030850 858 99759 50308.5 5030850 -32710 32225 4433.3 443330 -124 127 1.94 194 -859 2 10849 99760 2.57957 299.57957 151.07957 15107.95795 2.57957 299.5796 151.07957 15107.95794 2.57957 299.57957 151.0795699999999 15107.95700 2020-01-01 2020-01-02 2020-01-01 00:14:19 2020-01-02 03:42:40 2020-01-01 00:14:19.000 2020-01-02 03:42:40.000 859 99760 50309.5 5030950 859 99760 50309.5 5030950 -32709 32226 4434.3 443430 -128 127 0.38 38 -86 2 10076 99986 0.25825 300.25825 150.25825 15176.08408 0.25825 300.25827 150.25825 15176.08406 0.25825 300.25825 150.25825000000012 15176.08325 2020-01-01 2020-01-02 2020-01-01 00:01:26 2020-01-02 03:46:26 2020-01-01 00:01:26.000 2020-01-02 03:46:26.000 86 99986 50036 5053636 86 99986 50036 5053636 -32483 32452 4615.009900990099 466116 -128 123 -3.128712871287129 -316 -860 2 10850 99761 2.58258 299.58258 151.08258 15108.25825 2.58258 299.58258 151.08258 15108.25811 2.58258 299.58258 151.08258000000012 15108.25800 2020-01-01 2020-01-02 2020-01-01 00:14:20 2020-01-02 03:42:41 2020-01-01 00:14:20.000 2020-01-02 03:42:41.000 860 99761 50310.5 5031050 860 99761 50310.5 5031050 -32708 32227 4435.3 443530 -128 123 -1.18 -118 -861 2 10851 99762 2.58558 299.58558 151.08558 15108.55855 2.58558 299.58557 151.08558 15108.55841 2.58558 299.58558 151.08558000000016 15108.55800 2020-01-01 2020-01-02 2020-01-01 00:14:21 2020-01-02 03:42:42 2020-01-01 00:14:21.000 2020-01-02 03:42:42.000 861 99762 50311.5 5031150 861 99762 50311.5 5031150 -32707 32228 4436.3 443630 -127 124 -0.18 -18 -862 2 10852 99763 2.58858 299.58858 151.08858 15108.85885 2.58858 299.5886 151.08859 15108.85901 2.58858 299.58858 151.08857999999984 15108.85800 2020-01-01 2020-01-02 2020-01-01 00:14:22 2020-01-02 03:42:43 2020-01-01 00:14:22.000 2020-01-02 03:42:43.000 862 99763 50312.5 5031250 862 99763 50312.5 5031250 -32706 32229 4437.3 443730 -126 125 0.82 82 +858 2 10848 99759 2.57657 299.57657 151.07657 15107.65765 2.57657 299.57657 151.07657 15107.65783 2.57657 299.57657 151.07657 15107.65700 2020-01-01 2020-01-02 2020-01-01 00:14:18 2020-01-02 03:42:39 2020-01-01 00:14:18.000 2020-01-02 03:42:39.000 858 99759 50308.5 5030850 858 99759 50308.5 5030850 -32710 32225 4433.3 443330 -124 127 1.94 194 +859 2 10849 99760 2.57957 299.57957 151.07957 15107.95795 2.57957 299.5796 151.07957 15107.95794 2.57957 299.57957 151.07957 15107.95700 2020-01-01 2020-01-02 2020-01-01 00:14:19 2020-01-02 03:42:40 2020-01-01 00:14:19.000 2020-01-02 03:42:40.000 859 99760 50309.5 5030950 859 99760 50309.5 5030950 -32709 32226 4434.3 443430 -128 127 0.38 38 +86 2 10076 99986 0.25825 300.25825 150.25825 15176.08408 0.25825 300.25827 150.25825 15176.08406 0.25825 300.25825 150.25825 15176.08325 2020-01-01 2020-01-02 2020-01-01 00:01:26 2020-01-02 03:46:26 2020-01-01 00:01:26.000 2020-01-02 03:46:26.000 86 99986 50036 5053636 86 99986 50036 5053636 -32483 32452 4615.009900990099 466116 -128 123 -3.128712871287129 -316 +860 2 10850 99761 2.58258 299.58258 151.08258 15108.25825 2.58258 299.58258 151.08258 15108.25811 2.58258 299.58258 151.08258 15108.25800 2020-01-01 2020-01-02 2020-01-01 00:14:20 2020-01-02 03:42:41 2020-01-01 00:14:20.000 2020-01-02 03:42:41.000 860 99761 50310.5 5031050 860 99761 50310.5 5031050 -32708 32227 4435.3 443530 -128 123 -1.18 -118 +861 2 10851 99762 2.58558 299.58558 151.08558 15108.55855 2.58558 299.58557 151.08558 15108.55841 2.58558 299.58558 151.08558000000002 15108.55800 2020-01-01 2020-01-02 2020-01-01 00:14:21 2020-01-02 03:42:42 2020-01-01 00:14:21.000 2020-01-02 03:42:42.000 861 99762 50311.5 5031150 861 99762 50311.5 5031150 -32707 32228 4436.3 443630 -127 124 -0.18 -18 +862 2 10852 99763 2.58858 299.58858 151.08858 15108.85885 2.58858 299.5886 151.08859 15108.85901 2.58858 299.58858 151.08858 15108.85800 2020-01-01 2020-01-02 2020-01-01 00:14:22 2020-01-02 03:42:43 2020-01-01 00:14:22.000 2020-01-02 03:42:43.000 862 99763 50312.5 5031250 862 99763 50312.5 5031250 -32706 32229 4437.3 443730 -126 125 0.82 82 863 2 10853 99764 2.59159 299.59159 151.09159 15109.15915 2.59159 299.59158 151.09159 15109.1593 2.59159 299.59159 151.09159 15109.15900 2020-01-01 2020-01-02 2020-01-01 00:14:23 2020-01-02 03:42:44 2020-01-01 00:14:23.000 2020-01-02 03:42:44.000 863 99764 50313.5 5031350 863 99764 50313.5 5031350 -32705 32230 4438.3 443830 -125 126 1.82 182 -864 2 10854 99765 2.59459 299.59459 151.09459 15109.45945 2.59459 299.5946 151.09459 15109.45941 2.59459 299.59459 151.09459000000007 15109.45900 2020-01-01 2020-01-02 2020-01-01 00:14:24 2020-01-02 03:42:45 2020-01-01 00:14:24.000 2020-01-02 03:42:45.000 864 99765 50314.5 5031450 864 99765 50314.5 5031450 -32704 32231 4439.3 443930 -124 127 2.82 282 -865 2 10855 99766 2.59759 299.59759 151.09759 15109.75975 2.59759 299.5976 151.09759 15109.75958 2.59759 299.59759 151.0975899999999 15109.75900 2020-01-01 2020-01-02 2020-01-01 00:14:25 2020-01-02 03:42:46 2020-01-01 00:14:25.000 2020-01-02 03:42:46.000 865 99766 50315.5 5031550 865 99766 50315.5 5031550 -32703 32232 4440.3 444030 -128 127 1.26 126 -866 2 10856 99767 2.6006 299.6006 151.1006 15110.06006 2.6006 299.6006 151.10059 15110.05988 2.60060 299.60060 151.10059999999984 15110.06000 2020-01-01 2020-01-02 2020-01-01 00:14:26 2020-01-02 03:42:47 2020-01-01 00:14:26.000 2020-01-02 03:42:47.000 866 99767 50316.5 5031650 866 99767 50316.5 5031650 -32702 32233 4441.3 444130 -128 127 -0.3 -30 -867 2 10857 99768 2.6036 299.6036 151.1036 15110.36036 2.6036 299.6036 151.1036 15110.36063 2.60360 299.60360 151.10360000000017 15110.36000 2020-01-01 2020-01-02 2020-01-01 00:14:27 2020-01-02 03:42:48 2020-01-01 00:14:27.000 2020-01-02 03:42:48.000 867 99768 50317.5 5031750 867 99768 50317.5 5031750 -32701 32234 4442.3 444230 -128 123 -1.86 -186 -868 2 10858 99769 2.6066 299.6066 151.1066 15110.66066 2.6066 299.6066 151.1066 15110.66078 2.60660 299.60660 151.10659999999984 15110.66000 2020-01-01 2020-01-02 2020-01-01 00:14:28 2020-01-02 03:42:49 2020-01-01 00:14:28.000 2020-01-02 03:42:49.000 868 99769 50318.5 5031850 868 99769 50318.5 5031850 -32700 32235 4443.3 444330 -127 124 -0.86 -86 -869 2 10859 99770 2.6096 299.6096 151.1096 15110.96096 2.6096 299.60962 151.1096 15110.96091 2.60960 299.60960 151.10959999999983 15110.96000 2020-01-01 2020-01-02 2020-01-01 00:14:29 2020-01-02 03:42:50 2020-01-01 00:14:29.000 2020-01-02 03:42:50.000 869 99770 50319.5 5031950 869 99770 50319.5 5031950 -32699 32236 4444.3 444430 -126 125 0.14 14 -87 2 10077 99987 0.26126 300.26126 150.26126 15176.38738 0.26126 300.26126 150.26126 15176.38736 0.26126 300.26126 150.26125999999994 15176.38726 2020-01-01 2020-01-02 2020-01-01 00:01:27 2020-01-02 03:46:27 2020-01-01 00:01:27.000 2020-01-02 03:46:27.000 87 99987 50037 5053737 87 99987 50037 5053737 -32482 32453 4616.009900990099 466217 -127 124 -2.128712871287129 -215 -870 2 10860 99771 2.61261 299.61261 151.11261 15111.26126 2.61261 299.6126 151.11261 15111.26105 2.61261 299.61261 151.1126100000001 15111.26100 2020-01-01 2020-01-02 2020-01-01 00:14:30 2020-01-02 03:42:51 2020-01-01 00:14:30.000 2020-01-02 03:42:51.000 870 99771 50320.5 5032050 870 99771 50320.5 5032050 -32698 32237 4445.3 444530 -125 126 1.14 114 -871 2 10861 99772 2.61561 299.61561 151.11561 15111.56156 2.61561 299.6156 151.11561 15111.56135 2.61561 299.61561 151.11561000000012 15111.56100 2020-01-01 2020-01-02 2020-01-01 00:14:31 2020-01-02 03:42:52 2020-01-01 00:14:31.000 2020-01-02 03:42:52.000 871 99772 50321.5 5032150 871 99772 50321.5 5032150 -32697 32238 4446.3 444630 -124 127 2.14 214 -872 2 10862 99773 2.61861 299.61861 151.11861 15111.86186 2.61861 299.61862 151.11862 15111.8621 2.61861 299.61861 151.1186099999999 15111.86100 2020-01-01 2020-01-02 2020-01-01 00:14:32 2020-01-02 03:42:53 2020-01-01 00:14:32.000 2020-01-02 03:42:53.000 872 99773 50322.5 5032250 872 99773 50322.5 5032250 -32696 32239 4447.3 444730 -128 127 0.58 58 -873 2 10863 99774 2.62162 299.62162 151.12162 15112.16216 2.62162 299.6216 151.12162 15112.16224 2.62162 299.62162 151.12161999999992 15112.16200 2020-01-01 2020-01-02 2020-01-01 00:14:33 2020-01-02 03:42:54 2020-01-01 00:14:33.000 2020-01-02 03:42:54.000 873 99774 50323.5 5032350 873 99774 50323.5 5032350 -32695 32240 4448.3 444830 -128 123 -0.98 -98 +864 2 10854 99765 2.59459 299.59459 151.09459 15109.45945 2.59459 299.5946 151.09459 15109.45941 2.59459 299.59459 151.09459 15109.45900 2020-01-01 2020-01-02 2020-01-01 00:14:24 2020-01-02 03:42:45 2020-01-01 00:14:24.000 2020-01-02 03:42:45.000 864 99765 50314.5 5031450 864 99765 50314.5 5031450 -32704 32231 4439.3 443930 -124 127 2.82 282 +865 2 10855 99766 2.59759 299.59759 151.09759 15109.75975 2.59759 299.5976 151.09759 15109.75958 2.59759 299.59759 151.09759 15109.75900 2020-01-01 2020-01-02 2020-01-01 00:14:25 2020-01-02 03:42:46 2020-01-01 00:14:25.000 2020-01-02 03:42:46.000 865 99766 50315.5 5031550 865 99766 50315.5 5031550 -32703 32232 4440.3 444030 -128 127 1.26 126 +866 2 10856 99767 2.6006 299.6006 151.1006 15110.06006 2.6006 299.6006 151.10059 15110.05988 2.60060 299.60060 151.1006 15110.06000 2020-01-01 2020-01-02 2020-01-01 00:14:26 2020-01-02 03:42:47 2020-01-01 00:14:26.000 2020-01-02 03:42:47.000 866 99767 50316.5 5031650 866 99767 50316.5 5031650 -32702 32233 4441.3 444130 -128 127 -0.3 -30 +867 2 10857 99768 2.6036 299.6036 151.1036 15110.36036 2.6036 299.6036 151.1036 15110.36063 2.60360 299.60360 151.1036 15110.36000 2020-01-01 2020-01-02 2020-01-01 00:14:27 2020-01-02 03:42:48 2020-01-01 00:14:27.000 2020-01-02 03:42:48.000 867 99768 50317.5 5031750 867 99768 50317.5 5031750 -32701 32234 4442.3 444230 -128 123 -1.86 -186 +868 2 10858 99769 2.6066 299.6066 151.1066 15110.66066 2.6066 299.6066 151.1066 15110.66078 2.60660 299.60660 151.1066 15110.66000 2020-01-01 2020-01-02 2020-01-01 00:14:28 2020-01-02 03:42:49 2020-01-01 00:14:28.000 2020-01-02 03:42:49.000 868 99769 50318.5 5031850 868 99769 50318.5 5031850 -32700 32235 4443.3 444330 -127 124 -0.86 -86 +869 2 10859 99770 2.6096 299.6096 151.1096 15110.96096 2.6096 299.60962 151.1096 15110.96091 2.60960 299.60960 151.1096 15110.96000 2020-01-01 2020-01-02 2020-01-01 00:14:29 2020-01-02 03:42:50 2020-01-01 00:14:29.000 2020-01-02 03:42:50.000 869 99770 50319.5 5031950 869 99770 50319.5 5031950 -32699 32236 4444.3 444430 -126 125 0.14 14 +87 2 10077 99987 0.26126 300.26126 150.26126 15176.38738 0.26126 300.26126 150.26126 15176.38736 0.26126 300.26126 150.26126 15176.38726 2020-01-01 2020-01-02 2020-01-01 00:01:27 2020-01-02 03:46:27 2020-01-01 00:01:27.000 2020-01-02 03:46:27.000 87 99987 50037 5053737 87 99987 50037 5053737 -32482 32453 4616.009900990099 466217 -127 124 -2.128712871287129 -215 +870 2 10860 99771 2.61261 299.61261 151.11261 15111.26126 2.61261 299.6126 151.11261 15111.26105 2.61261 299.61261 151.11261000000002 15111.26100 2020-01-01 2020-01-02 2020-01-01 00:14:30 2020-01-02 03:42:51 2020-01-01 00:14:30.000 2020-01-02 03:42:51.000 870 99771 50320.5 5032050 870 99771 50320.5 5032050 -32698 32237 4445.3 444530 -125 126 1.14 114 +871 2 10861 99772 2.61561 299.61561 151.11561 15111.56156 2.61561 299.6156 151.11561 15111.56135 2.61561 299.61561 151.11561 15111.56100 2020-01-01 2020-01-02 2020-01-01 00:14:31 2020-01-02 03:42:52 2020-01-01 00:14:31.000 2020-01-02 03:42:52.000 871 99772 50321.5 5032150 871 99772 50321.5 5032150 -32697 32238 4446.3 444630 -124 127 2.14 214 +872 2 10862 99773 2.61861 299.61861 151.11861 15111.86186 2.61861 299.61862 151.11862 15111.8621 2.61861 299.61861 151.11861000000002 15111.86100 2020-01-01 2020-01-02 2020-01-01 00:14:32 2020-01-02 03:42:53 2020-01-01 00:14:32.000 2020-01-02 03:42:53.000 872 99773 50322.5 5032250 872 99773 50322.5 5032250 -32696 32239 4447.3 444730 -128 127 0.58 58 +873 2 10863 99774 2.62162 299.62162 151.12162 15112.16216 2.62162 299.6216 151.12162 15112.16224 2.62162 299.62162 151.12162 15112.16200 2020-01-01 2020-01-02 2020-01-01 00:14:33 2020-01-02 03:42:54 2020-01-01 00:14:33.000 2020-01-02 03:42:54.000 873 99774 50323.5 5032350 873 99774 50323.5 5032350 -32695 32240 4448.3 444830 -128 123 -0.98 -98 874 2 10864 99775 2.62462 299.62462 151.12462 15112.46246 2.62462 299.62463 151.12462 15112.46238 2.62462 299.62462 151.12462 15112.46200 2020-01-01 2020-01-02 2020-01-01 00:14:34 2020-01-02 03:42:55 2020-01-01 00:14:34.000 2020-01-02 03:42:55.000 874 99775 50324.5 5032450 874 99775 50324.5 5032450 -32694 32241 4449.3 444930 -127 124 0.02 2 -875 2 10865 99776 2.62762 299.62762 151.12762 15112.76276 2.62762 299.62762 151.12762 15112.76252 2.62762 299.62762 151.12761999999992 15112.76200 2020-01-01 2020-01-02 2020-01-01 00:14:35 2020-01-02 03:42:56 2020-01-01 00:14:35.000 2020-01-02 03:42:56.000 875 99776 50325.5 5032550 875 99776 50325.5 5032550 -32693 32242 4450.3 445030 -126 125 1.02 102 -876 2 10866 99777 2.63063 299.63063 151.13063 15113.06306 2.63063 299.63065 151.13063 15113.06327 2.63063 299.63063 151.13062999999985 15113.06300 2020-01-01 2020-01-02 2020-01-01 00:14:36 2020-01-02 03:42:57 2020-01-01 00:14:36.000 2020-01-02 03:42:57.000 876 99777 50326.5 5032650 876 99777 50326.5 5032650 -32692 32243 4451.3 445130 -125 126 2.02 202 -877 2 10867 99778 2.63363 299.63363 151.13363 15113.36336 2.63363 299.63364 151.13363 15113.36358 2.63363 299.63363 151.1336300000001 15113.36300 2020-01-01 2020-01-02 2020-01-01 00:14:37 2020-01-02 03:42:58 2020-01-01 00:14:37.000 2020-01-02 03:42:58.000 877 99778 50327.5 5032750 877 99778 50327.5 5032750 -32691 32244 4452.3 445230 -124 127 3.02 302 -878 2 10868 99779 2.63663 299.63663 151.13663 15113.66366 2.63663 299.63663 151.13663 15113.66371 2.63663 299.63663 151.13663000000017 15113.66300 2020-01-01 2020-01-02 2020-01-01 00:14:38 2020-01-02 03:42:59 2020-01-01 00:14:38.000 2020-01-02 03:42:59.000 878 99779 50328.5 5032850 878 99779 50328.5 5032850 -32690 32245 4453.3 445330 -128 127 1.46 146 -879 2 10869 99780 2.63963 299.63963 151.13963 15113.96396 2.63963 299.63965 151.13963 15113.96385 2.63963 299.63963 151.13962999999984 15113.96300 2020-01-01 2020-01-02 2020-01-01 00:14:39 2020-01-02 03:43:00 2020-01-01 00:14:39.000 2020-01-02 03:43:00.000 879 99780 50329.5 5032950 879 99780 50329.5 5032950 -32689 32246 4454.3 445430 -128 123 -0.1 -10 -88 2 10078 99988 0.26426 300.26426 150.26426 15176.69069 0.26426 300.26425 150.26426 15176.69066 0.26426 300.26426 150.26425999999995 15176.69026 2020-01-01 2020-01-02 2020-01-01 00:01:28 2020-01-02 03:46:28 2020-01-01 00:01:28.000 2020-01-02 03:46:28.000 88 99988 50038 5053838 88 99988 50038 5053838 -32481 32454 4617.009900990099 466318 -126 125 -1.1287128712871286 -114 +875 2 10865 99776 2.62762 299.62762 151.12762 15112.76276 2.62762 299.62762 151.12762 15112.76252 2.62762 299.62762 151.12762 15112.76200 2020-01-01 2020-01-02 2020-01-01 00:14:35 2020-01-02 03:42:56 2020-01-01 00:14:35.000 2020-01-02 03:42:56.000 875 99776 50325.5 5032550 875 99776 50325.5 5032550 -32693 32242 4450.3 445030 -126 125 1.02 102 +876 2 10866 99777 2.63063 299.63063 151.13063 15113.06306 2.63063 299.63065 151.13063 15113.06327 2.63063 299.63063 151.13063 15113.06300 2020-01-01 2020-01-02 2020-01-01 00:14:36 2020-01-02 03:42:57 2020-01-01 00:14:36.000 2020-01-02 03:42:57.000 876 99777 50326.5 5032650 876 99777 50326.5 5032650 -32692 32243 4451.3 445130 -125 126 2.02 202 +877 2 10867 99778 2.63363 299.63363 151.13363 15113.36336 2.63363 299.63364 151.13363 15113.36358 2.63363 299.63363 151.13362999999998 15113.36300 2020-01-01 2020-01-02 2020-01-01 00:14:37 2020-01-02 03:42:58 2020-01-01 00:14:37.000 2020-01-02 03:42:58.000 877 99778 50327.5 5032750 877 99778 50327.5 5032750 -32691 32244 4452.3 445230 -124 127 3.02 302 +878 2 10868 99779 2.63663 299.63663 151.13663 15113.66366 2.63663 299.63663 151.13663 15113.66371 2.63663 299.63663 151.13663 15113.66300 2020-01-01 2020-01-02 2020-01-01 00:14:38 2020-01-02 03:42:59 2020-01-01 00:14:38.000 2020-01-02 03:42:59.000 878 99779 50328.5 5032850 878 99779 50328.5 5032850 -32690 32245 4453.3 445330 -128 127 1.46 146 +879 2 10869 99780 2.63963 299.63963 151.13963 15113.96396 2.63963 299.63965 151.13963 15113.96385 2.63963 299.63963 151.13963 15113.96300 2020-01-01 2020-01-02 2020-01-01 00:14:39 2020-01-02 03:43:00 2020-01-01 00:14:39.000 2020-01-02 03:43:00.000 879 99780 50329.5 5032950 879 99780 50329.5 5032950 -32689 32246 4454.3 445430 -128 123 -0.1 -10 +88 2 10078 99988 0.26426 300.26426 150.26426 15176.69069 0.26426 300.26425 150.26426 15176.69066 0.26426 300.26426 150.26426 15176.69026 2020-01-01 2020-01-02 2020-01-01 00:01:28 2020-01-02 03:46:28 2020-01-01 00:01:28.000 2020-01-02 03:46:28.000 88 99988 50038 5053838 88 99988 50038 5053838 -32481 32454 4617.009900990099 466318 -126 125 -1.1287128712871286 -114 880 2 10870 99781 2.64264 299.64264 151.14264 15114.26426 2.64264 299.64264 151.14263 15114.26399 2.64264 299.64264 151.14264 15114.26400 2020-01-01 2020-01-02 2020-01-01 00:14:40 2020-01-02 03:43:01 2020-01-01 00:14:40.000 2020-01-02 03:43:01.000 880 99781 50330.5 5033050 880 99781 50330.5 5033050 -32688 32247 4455.3 445530 -127 124 0.9 90 -881 2 10871 99782 2.64564 299.64564 151.14564 15114.56456 2.64564 299.64566 151.14564 15114.56474 2.64564 299.64564 151.1456400000001 15114.56400 2020-01-01 2020-01-02 2020-01-01 00:14:41 2020-01-02 03:43:02 2020-01-01 00:14:41.000 2020-01-02 03:43:02.000 881 99782 50331.5 5033150 881 99782 50331.5 5033150 -32687 32248 4456.3 445630 -126 125 1.9 190 -882 2 10872 99783 2.64864 299.64864 151.14864 15114.86486 2.64864 299.64865 151.14865 15114.86504 2.64864 299.64864 151.14863999999992 15114.86400 2020-01-01 2020-01-02 2020-01-01 00:14:42 2020-01-02 03:43:03 2020-01-01 00:14:42.000 2020-01-02 03:43:03.000 882 99783 50332.5 5033250 882 99783 50332.5 5033250 -32686 32249 4457.3 445730 -125 126 2.9 290 -883 2 10873 99784 2.65165 299.65165 151.15165 15115.16516 2.65165 299.65164 151.15165 15115.16522 2.65165 299.65165 151.15164999999988 15115.16500 2020-01-01 2020-01-02 2020-01-01 00:14:43 2020-01-02 03:43:04 2020-01-01 00:14:43.000 2020-01-02 03:43:04.000 883 99784 50333.5 5033350 883 99784 50333.5 5033350 -32685 32250 4458.3 445830 -124 127 3.9 390 -884 2 10874 99785 2.65465 299.65465 151.15465 15115.46546 2.65465 299.65466 151.15465 15115.46532 2.65465 299.65465 151.1546500000002 15115.46500 2020-01-01 2020-01-02 2020-01-01 00:14:44 2020-01-02 03:43:05 2020-01-01 00:14:44.000 2020-01-02 03:43:05.000 884 99785 50334.5 5033450 884 99785 50334.5 5033450 -32684 32251 4459.3 445930 -128 127 2.34 234 -885 2 10875 99786 2.65765 299.65765 151.15765 15115.76576 2.65765 299.65765 151.15765 15115.76562 2.65765 299.65765 151.15764999999988 15115.76500 2020-01-01 2020-01-02 2020-01-01 00:14:45 2020-01-02 03:43:06 2020-01-01 00:14:45.000 2020-01-02 03:43:06.000 885 99786 50335.5 5033550 885 99786 50335.5 5033550 -32683 32252 4460.3 446030 -128 123 0.78 78 -886 2 10876 99787 2.66066 299.66066 151.16066 15116.06606 2.66066 299.66068 151.16066 15116.06621 2.66066 299.66066 151.16065999999978 15116.06600 2020-01-01 2020-01-02 2020-01-01 00:14:46 2020-01-02 03:43:07 2020-01-01 00:14:46.000 2020-01-02 03:43:07.000 886 99787 50336.5 5033650 886 99787 50336.5 5033650 -32682 32253 4461.3 446130 -127 124 1.78 178 -887 2 10877 99788 2.66366 299.66366 151.16366 15116.36636 2.66366 299.66367 151.16366 15116.36651 2.66366 299.66366 151.1636600000001 15116.36600 2020-01-01 2020-01-02 2020-01-01 00:14:47 2020-01-02 03:43:08 2020-01-01 00:14:47.000 2020-01-02 03:43:08.000 887 99788 50337.5 5033750 887 99788 50337.5 5033750 -32681 32254 4462.3 446230 -126 125 2.78 278 -888 2 10878 99789 2.66666 299.66666 151.16666 15116.66666 2.66666 299.66666 151.16666 15116.66669 2.66666 299.66666 151.1666600000001 15116.66600 2020-01-01 2020-01-02 2020-01-01 00:14:48 2020-01-02 03:43:09 2020-01-01 00:14:48.000 2020-01-02 03:43:09.000 888 99789 50338.5 5033850 888 99789 50338.5 5033850 -32680 32255 4463.3 446330 -125 126 3.78 378 -889 2 10879 99790 2.66966 299.66966 151.16966 15116.96696 2.66966 299.66968 151.16966 15116.96679 2.66966 299.66966 151.1696599999998 15116.96600 2020-01-01 2020-01-02 2020-01-01 00:14:49 2020-01-02 03:43:10 2020-01-01 00:14:49.000 2020-01-02 03:43:10.000 889 99790 50339.5 5033950 889 99790 50339.5 5033950 -32679 32256 4464.3 446430 -124 127 4.78 478 -89 2 10079 99989 0.26726 300.26726 150.26726 15176.99399 0.26726 300.26727 150.26727 15176.9943 0.26726 300.26726 150.26726000000005 15176.99326 2020-01-01 2020-01-02 2020-01-01 00:01:29 2020-01-02 03:46:29 2020-01-01 00:01:29.000 2020-01-02 03:46:29.000 89 99989 50039 5053939 89 99989 50039 5053939 -32480 32455 4618.009900990099 466419 -125 126 -0.12871287128712872 -13 -890 2 10880 99791 2.67267 299.67267 151.17267 15117.26726 2.67267 299.67267 151.17267 15117.26708 2.67267 299.67267 151.17266999999995 15117.26700 2020-01-01 2020-01-02 2020-01-01 00:14:50 2020-01-02 03:43:11 2020-01-01 00:14:50.000 2020-01-02 03:43:11.000 890 99791 50340.5 5034050 890 99791 50340.5 5034050 -32678 32257 4465.3 446530 -128 127 3.22 322 +881 2 10871 99782 2.64564 299.64564 151.14564 15114.56456 2.64564 299.64566 151.14564 15114.56474 2.64564 299.64564 151.14564000000001 15114.56400 2020-01-01 2020-01-02 2020-01-01 00:14:41 2020-01-02 03:43:02 2020-01-01 00:14:41.000 2020-01-02 03:43:02.000 881 99782 50331.5 5033150 881 99782 50331.5 5033150 -32687 32248 4456.3 445630 -126 125 1.9 190 +882 2 10872 99783 2.64864 299.64864 151.14864 15114.86486 2.64864 299.64865 151.14865 15114.86504 2.64864 299.64864 151.14864 15114.86400 2020-01-01 2020-01-02 2020-01-01 00:14:42 2020-01-02 03:43:03 2020-01-01 00:14:42.000 2020-01-02 03:43:03.000 882 99783 50332.5 5033250 882 99783 50332.5 5033250 -32686 32249 4457.3 445730 -125 126 2.9 290 +883 2 10873 99784 2.65165 299.65165 151.15165 15115.16516 2.65165 299.65164 151.15165 15115.16522 2.65165 299.65165 151.15165000000002 15115.16500 2020-01-01 2020-01-02 2020-01-01 00:14:43 2020-01-02 03:43:04 2020-01-01 00:14:43.000 2020-01-02 03:43:04.000 883 99784 50333.5 5033350 883 99784 50333.5 5033350 -32685 32250 4458.3 445830 -124 127 3.9 390 +884 2 10874 99785 2.65465 299.65465 151.15465 15115.46546 2.65465 299.65466 151.15465 15115.46532 2.65465 299.65465 151.15465 15115.46500 2020-01-01 2020-01-02 2020-01-01 00:14:44 2020-01-02 03:43:05 2020-01-01 00:14:44.000 2020-01-02 03:43:05.000 884 99785 50334.5 5033450 884 99785 50334.5 5033450 -32684 32251 4459.3 445930 -128 127 2.34 234 +885 2 10875 99786 2.65765 299.65765 151.15765 15115.76576 2.65765 299.65765 151.15765 15115.76562 2.65765 299.65765 151.15765 15115.76500 2020-01-01 2020-01-02 2020-01-01 00:14:45 2020-01-02 03:43:06 2020-01-01 00:14:45.000 2020-01-02 03:43:06.000 885 99786 50335.5 5033550 885 99786 50335.5 5033550 -32683 32252 4460.3 446030 -128 123 0.78 78 +886 2 10876 99787 2.66066 299.66066 151.16066 15116.06606 2.66066 299.66068 151.16066 15116.06621 2.66066 299.66066 151.16066 15116.06600 2020-01-01 2020-01-02 2020-01-01 00:14:46 2020-01-02 03:43:07 2020-01-01 00:14:46.000 2020-01-02 03:43:07.000 886 99787 50336.5 5033650 886 99787 50336.5 5033650 -32682 32253 4461.3 446130 -127 124 1.78 178 +887 2 10877 99788 2.66366 299.66366 151.16366 15116.36636 2.66366 299.66367 151.16366 15116.36651 2.66366 299.66366 151.16366 15116.36600 2020-01-01 2020-01-02 2020-01-01 00:14:47 2020-01-02 03:43:08 2020-01-01 00:14:47.000 2020-01-02 03:43:08.000 887 99788 50337.5 5033750 887 99788 50337.5 5033750 -32681 32254 4462.3 446230 -126 125 2.78 278 +888 2 10878 99789 2.66666 299.66666 151.16666 15116.66666 2.66666 299.66666 151.16666 15116.66669 2.66666 299.66666 151.16665999999998 15116.66600 2020-01-01 2020-01-02 2020-01-01 00:14:48 2020-01-02 03:43:09 2020-01-01 00:14:48.000 2020-01-02 03:43:09.000 888 99789 50338.5 5033850 888 99789 50338.5 5033850 -32680 32255 4463.3 446330 -125 126 3.78 378 +889 2 10879 99790 2.66966 299.66966 151.16966 15116.96696 2.66966 299.66968 151.16966 15116.96679 2.66966 299.66966 151.16966 15116.96600 2020-01-01 2020-01-02 2020-01-01 00:14:49 2020-01-02 03:43:10 2020-01-01 00:14:49.000 2020-01-02 03:43:10.000 889 99790 50339.5 5033950 889 99790 50339.5 5033950 -32679 32256 4464.3 446430 -124 127 4.78 478 +89 2 10079 99989 0.26726 300.26726 150.26726 15176.99399 0.26726 300.26727 150.26727 15176.9943 0.26726 300.26726 150.26726 15176.99326 2020-01-01 2020-01-02 2020-01-01 00:01:29 2020-01-02 03:46:29 2020-01-01 00:01:29.000 2020-01-02 03:46:29.000 89 99989 50039 5053939 89 99989 50039 5053939 -32480 32455 4618.009900990099 466419 -125 126 -0.12871287128712872 -13 +890 2 10880 99791 2.67267 299.67267 151.17267 15117.26726 2.67267 299.67267 151.17267 15117.26708 2.67267 299.67267 151.17267 15117.26700 2020-01-01 2020-01-02 2020-01-01 00:14:50 2020-01-02 03:43:11 2020-01-01 00:14:50.000 2020-01-02 03:43:11.000 890 99791 50340.5 5034050 890 99791 50340.5 5034050 -32678 32257 4465.3 446530 -128 127 3.22 322 891 2 10881 99792 2.67567 299.67567 151.17567 15117.56756 2.67567 299.6757 151.17567 15117.56768 2.67567 299.67567 151.17567 15117.56700 2020-01-01 2020-01-02 2020-01-01 00:14:51 2020-01-02 03:43:12 2020-01-01 00:14:51.000 2020-01-02 03:43:12.000 891 99792 50341.5 5034150 891 99792 50341.5 5034150 -32677 32258 4466.3 446630 -128 127 1.66 166 -892 2 10882 99793 2.67867 299.67867 151.17867 15117.86786 2.67867 299.67868 151.17868 15117.86802 2.67867 299.67867 151.17866999999993 15117.86700 2020-01-01 2020-01-02 2020-01-01 00:14:52 2020-01-02 03:43:13 2020-01-01 00:14:52.000 2020-01-02 03:43:13.000 892 99793 50342.5 5034250 892 99793 50342.5 5034250 -32676 32259 4467.3 446730 -128 124 0.1 10 -893 2 10883 99794 2.68168 299.68168 151.18168 15118.16816 2.68168 299.68167 151.18168 15118.16816 2.68168 299.68168 151.18167999999986 15118.16800 2020-01-01 2020-01-02 2020-01-01 00:14:53 2020-01-02 03:43:14 2020-01-01 00:14:53.000 2020-01-02 03:43:14.000 893 99794 50343.5 5034350 893 99794 50343.5 5034350 -32675 32260 4468.3 446830 -127 125 1.1 110 -894 2 10884 99795 2.68468 299.68468 151.18468 15118.46846 2.68468 299.6847 151.18468 15118.46826 2.68468 299.68468 151.18468000000013 15118.46800 2020-01-01 2020-01-02 2020-01-01 00:14:54 2020-01-02 03:43:15 2020-01-01 00:14:54.000 2020-01-02 03:43:15.000 894 99795 50344.5 5034450 894 99795 50344.5 5034450 -32674 32261 4469.3 446930 -126 126 2.1 210 -895 2 10885 99796 2.68768 299.68768 151.18768 15118.76876 2.68768 299.68768 151.18768 15118.76855 2.68768 299.68768 151.1876799999998 15118.76800 2020-01-01 2020-01-02 2020-01-01 00:14:55 2020-01-02 03:43:16 2020-01-01 00:14:55.000 2020-01-02 03:43:16.000 895 99796 50345.5 5034550 895 99796 50345.5 5034550 -32673 32262 4470.3 447030 -125 127 3.1 310 -896 2 10886 99797 2.69069 299.69069 151.19069 15119.06906 2.69069 299.6907 151.19069 15119.06915 2.69069 299.69069 151.19068999999993 15119.06900 2020-01-01 2020-01-02 2020-01-01 00:14:56 2020-01-02 03:43:17 2020-01-01 00:14:56.000 2020-01-02 03:43:17.000 896 99797 50346.5 5034650 896 99797 50346.5 5034650 -32672 32263 4471.3 447130 -128 127 1.54 154 -897 2 10887 99798 2.69369 299.69369 151.19369 15119.36936 2.69369 299.6937 151.19369 15119.36949 2.69369 299.69369 151.19369000000003 15119.36900 2020-01-01 2020-01-02 2020-01-01 00:14:57 2020-01-02 03:43:18 2020-01-01 00:14:57.000 2020-01-02 03:43:18.000 897 99798 50347.5 5034750 897 99798 50347.5 5034750 -32671 32264 4472.3 447230 -128 127 -0.02 -2 -898 2 10888 99799 2.69669 299.69669 151.19669 15119.66966 2.69669 299.6967 151.19669 15119.66963 2.69669 299.69669 151.1966900000001 15119.66900 2020-01-01 2020-01-02 2020-01-01 00:14:58 2020-01-02 03:43:19 2020-01-01 00:14:58.000 2020-01-02 03:43:19.000 898 99799 50348.5 5034850 898 99799 50348.5 5034850 -32670 32265 4473.3 447330 -128 123 -1.58 -158 -899 2 10889 99800 2.69969 299.69969 151.19969 15119.96996 2.69969 299.6997 151.1997 15119.97038 2.69969 299.69969 151.19968999999998 15119.96900 2020-01-01 2020-01-02 2020-01-01 00:14:59 2020-01-02 03:43:20 2020-01-01 00:14:59.000 2020-01-02 03:43:20.000 899 99800 50349.5 5034950 899 99800 50349.5 5034950 -32669 32266 4474.3 447430 -127 124 -0.58 -58 -9 2 1008 9999 0.02702 300.02702 150.02702 15152.72972 0.02702 300.02704 150.02702 15152.72966 0.02702 300.02702 150.02701999999985 15152.72902 2020-01-01 2020-01-02 2020-01-01 00:00:09 2020-01-02 03:45:09 2020-01-01 00:00:09.000 2020-01-02 03:45:09.000 9 99909 49959 5045859 9 99909 49959 5045859 -32560 32375 4538.009900990099 458339 -124 127 0.9801980198019802 99 -90 2 10080 99990 0.27027 300.27027 150.27027 15177.29729 0.27027 300.27026 150.27026 15177.29694 0.27027 300.27027 150.27027000000024 15177.29727 2020-01-01 2020-01-02 2020-01-01 00:01:30 2020-01-02 03:46:30 2020-01-01 00:01:30.000 2020-01-02 03:46:30.000 90 99990 50040 5054040 90 99990 50040 5054040 -32479 32456 4619.009900990099 466520 -124 127 0.8712871287128713 88 -900 2 10890 99801 2.7027 299.7027 151.2027 15120.27027 2.7027 299.7027 151.2027 15120.27003 2.70270 299.70270 151.20269999999988 15120.27000 2020-01-01 2020-01-02 2020-01-01 00:15:00 2020-01-02 03:43:21 2020-01-01 00:15:00.000 2020-01-02 03:43:21.000 900 99801 50350.5 5035050 900 99801 50350.5 5035050 -32668 32267 4475.3 447530 -126 125 0.42 42 -901 2 10891 99802 2.7057 299.7057 151.2057 15120.57057 2.7057 299.70572 151.2057 15120.57066 2.70570 299.70570 151.2057000000002 15120.57000 2020-01-01 2020-01-02 2020-01-01 00:15:01 2020-01-02 03:43:22 2020-01-01 00:15:01.000 2020-01-02 03:43:22.000 901 99802 50351.5 5035150 901 99802 50351.5 5035150 -32667 32268 4476.3 447630 -125 126 1.42 142 -902 2 10892 99803 2.7087 299.7087 151.2087 15120.87087 2.7087 299.7087 151.2087 15120.87095 2.70870 299.70870 151.2086999999999 15120.87000 2020-01-01 2020-01-02 2020-01-01 00:15:02 2020-01-02 03:43:23 2020-01-01 00:15:02.000 2020-01-02 03:43:23.000 902 99803 50352.5 5035250 902 99803 50352.5 5035250 -32666 32269 4477.3 447730 -124 127 2.42 242 -903 2 10893 99804 2.71171 299.71171 151.21171 15121.17117 2.71171 299.7117 151.21171 15121.1711 2.71171 299.71171 151.21170999999978 15121.17100 2020-01-01 2020-01-02 2020-01-01 00:15:03 2020-01-02 03:43:24 2020-01-01 00:15:03.000 2020-01-02 03:43:24.000 903 99804 50353.5 5035350 903 99804 50353.5 5035350 -32665 32270 4478.3 447830 -128 127 0.86 86 -904 2 10894 99805 2.71471 299.71471 151.21471 15121.47147 2.71471 299.71472 151.21471 15121.47185 2.71471 299.71471 151.2147100000001 15121.47100 2020-01-01 2020-01-02 2020-01-01 00:15:04 2020-01-02 03:43:25 2020-01-01 00:15:04.000 2020-01-02 03:43:25.000 904 99805 50354.5 5035450 904 99805 50354.5 5035450 -32664 32271 4479.3 447930 -128 123 -0.7 -70 -905 2 10895 99806 2.71771 299.71771 151.21771 15121.77177 2.71771 299.7177 151.21771 15121.77149 2.71771 299.71771 151.21771000000012 15121.77100 2020-01-01 2020-01-02 2020-01-01 00:15:05 2020-01-02 03:43:26 2020-01-01 00:15:05.000 2020-01-02 03:43:26.000 905 99806 50355.5 5035550 905 99806 50355.5 5035550 -32663 32272 4480.3 448030 -127 124 0.3 30 -906 2 10896 99807 2.72072 299.72072 151.22072 15122.07207 2.72072 299.72073 151.22072 15122.07212 2.72072 299.72072 151.2207199999999 15122.07200 2020-01-01 2020-01-02 2020-01-01 00:15:06 2020-01-02 03:43:27 2020-01-01 00:15:06.000 2020-01-02 03:43:27.000 906 99807 50356.5 5035650 906 99807 50356.5 5035650 -32662 32273 4481.3 448130 -126 125 1.3 130 +892 2 10882 99793 2.67867 299.67867 151.17867 15117.86786 2.67867 299.67868 151.17868 15117.86802 2.67867 299.67867 151.17867 15117.86700 2020-01-01 2020-01-02 2020-01-01 00:14:52 2020-01-02 03:43:13 2020-01-01 00:14:52.000 2020-01-02 03:43:13.000 892 99793 50342.5 5034250 892 99793 50342.5 5034250 -32676 32259 4467.3 446730 -128 124 0.1 10 +893 2 10883 99794 2.68168 299.68168 151.18168 15118.16816 2.68168 299.68167 151.18168 15118.16816 2.68168 299.68168 151.18168 15118.16800 2020-01-01 2020-01-02 2020-01-01 00:14:53 2020-01-02 03:43:14 2020-01-01 00:14:53.000 2020-01-02 03:43:14.000 893 99794 50343.5 5034350 893 99794 50343.5 5034350 -32675 32260 4468.3 446830 -127 125 1.1 110 +894 2 10884 99795 2.68468 299.68468 151.18468 15118.46846 2.68468 299.6847 151.18468 15118.46826 2.68468 299.68468 151.18468000000001 15118.46800 2020-01-01 2020-01-02 2020-01-01 00:14:54 2020-01-02 03:43:15 2020-01-01 00:14:54.000 2020-01-02 03:43:15.000 894 99795 50344.5 5034450 894 99795 50344.5 5034450 -32674 32261 4469.3 446930 -126 126 2.1 210 +895 2 10885 99796 2.68768 299.68768 151.18768 15118.76876 2.68768 299.68768 151.18768 15118.76855 2.68768 299.68768 151.18768 15118.76800 2020-01-01 2020-01-02 2020-01-01 00:14:55 2020-01-02 03:43:16 2020-01-01 00:14:55.000 2020-01-02 03:43:16.000 895 99796 50345.5 5034550 895 99796 50345.5 5034550 -32673 32262 4470.3 447030 -125 127 3.1 310 +896 2 10886 99797 2.69069 299.69069 151.19069 15119.06906 2.69069 299.6907 151.19069 15119.06915 2.69069 299.69069 151.19069 15119.06900 2020-01-01 2020-01-02 2020-01-01 00:14:56 2020-01-02 03:43:17 2020-01-01 00:14:56.000 2020-01-02 03:43:17.000 896 99797 50346.5 5034650 896 99797 50346.5 5034650 -32672 32263 4471.3 447130 -128 127 1.54 154 +897 2 10887 99798 2.69369 299.69369 151.19369 15119.36936 2.69369 299.6937 151.19369 15119.36949 2.69369 299.69369 151.19369 15119.36900 2020-01-01 2020-01-02 2020-01-01 00:14:57 2020-01-02 03:43:18 2020-01-01 00:14:57.000 2020-01-02 03:43:18.000 897 99798 50347.5 5034750 897 99798 50347.5 5034750 -32671 32264 4472.3 447230 -128 127 -0.02 -2 +898 2 10888 99799 2.69669 299.69669 151.19669 15119.66966 2.69669 299.6967 151.19669 15119.66963 2.69669 299.69669 151.19669 15119.66900 2020-01-01 2020-01-02 2020-01-01 00:14:58 2020-01-02 03:43:19 2020-01-01 00:14:58.000 2020-01-02 03:43:19.000 898 99799 50348.5 5034850 898 99799 50348.5 5034850 -32670 32265 4473.3 447330 -128 123 -1.58 -158 +899 2 10889 99800 2.69969 299.69969 151.19969 15119.96996 2.69969 299.6997 151.1997 15119.97038 2.69969 299.69969 151.19969 15119.96900 2020-01-01 2020-01-02 2020-01-01 00:14:59 2020-01-02 03:43:20 2020-01-01 00:14:59.000 2020-01-02 03:43:20.000 899 99800 50349.5 5034950 899 99800 50349.5 5034950 -32669 32266 4474.3 447430 -127 124 -0.58 -58 +9 2 1008 9999 0.02702 300.02702 150.02702 15152.72972 0.02702 300.02704 150.02702 15152.72966 0.02702 300.02702 150.02702 15152.72902 2020-01-01 2020-01-02 2020-01-01 00:00:09 2020-01-02 03:45:09 2020-01-01 00:00:09.000 2020-01-02 03:45:09.000 9 99909 49959 5045859 9 99909 49959 5045859 -32560 32375 4538.009900990099 458339 -124 127 0.9801980198019802 99 +90 2 10080 99990 0.27027 300.27027 150.27027 15177.29729 0.27027 300.27026 150.27026 15177.29694 0.27027 300.27027 150.27026999999998 15177.29727 2020-01-01 2020-01-02 2020-01-01 00:01:30 2020-01-02 03:46:30 2020-01-01 00:01:30.000 2020-01-02 03:46:30.000 90 99990 50040 5054040 90 99990 50040 5054040 -32479 32456 4619.009900990099 466520 -124 127 0.8712871287128713 88 +900 2 10890 99801 2.7027 299.7027 151.2027 15120.27027 2.7027 299.7027 151.2027 15120.27003 2.70270 299.70270 151.2027 15120.27000 2020-01-01 2020-01-02 2020-01-01 00:15:00 2020-01-02 03:43:21 2020-01-01 00:15:00.000 2020-01-02 03:43:21.000 900 99801 50350.5 5035050 900 99801 50350.5 5035050 -32668 32267 4475.3 447530 -126 125 0.42 42 +901 2 10891 99802 2.7057 299.7057 151.2057 15120.57057 2.7057 299.70572 151.2057 15120.57066 2.70570 299.70570 151.2057 15120.57000 2020-01-01 2020-01-02 2020-01-01 00:15:01 2020-01-02 03:43:22 2020-01-01 00:15:01.000 2020-01-02 03:43:22.000 901 99802 50351.5 5035150 901 99802 50351.5 5035150 -32667 32268 4476.3 447630 -125 126 1.42 142 +902 2 10892 99803 2.7087 299.7087 151.2087 15120.87087 2.7087 299.7087 151.2087 15120.87095 2.70870 299.70870 151.20870000000002 15120.87000 2020-01-01 2020-01-02 2020-01-01 00:15:02 2020-01-02 03:43:23 2020-01-01 00:15:02.000 2020-01-02 03:43:23.000 902 99803 50352.5 5035250 902 99803 50352.5 5035250 -32666 32269 4477.3 447730 -124 127 2.42 242 +903 2 10893 99804 2.71171 299.71171 151.21171 15121.17117 2.71171 299.7117 151.21171 15121.1711 2.71171 299.71171 151.21171 15121.17100 2020-01-01 2020-01-02 2020-01-01 00:15:03 2020-01-02 03:43:24 2020-01-01 00:15:03.000 2020-01-02 03:43:24.000 903 99804 50353.5 5035350 903 99804 50353.5 5035350 -32665 32270 4478.3 447830 -128 127 0.86 86 +904 2 10894 99805 2.71471 299.71471 151.21471 15121.47147 2.71471 299.71472 151.21471 15121.47185 2.71471 299.71471 151.21471 15121.47100 2020-01-01 2020-01-02 2020-01-01 00:15:04 2020-01-02 03:43:25 2020-01-01 00:15:04.000 2020-01-02 03:43:25.000 904 99805 50354.5 5035450 904 99805 50354.5 5035450 -32664 32271 4479.3 447930 -128 123 -0.7 -70 +905 2 10895 99806 2.71771 299.71771 151.21771 15121.77177 2.71771 299.7177 151.21771 15121.77149 2.71771 299.71771 151.21771 15121.77100 2020-01-01 2020-01-02 2020-01-01 00:15:05 2020-01-02 03:43:26 2020-01-01 00:15:05.000 2020-01-02 03:43:26.000 905 99806 50355.5 5035550 905 99806 50355.5 5035550 -32663 32272 4480.3 448030 -127 124 0.3 30 +906 2 10896 99807 2.72072 299.72072 151.22072 15122.07207 2.72072 299.72073 151.22072 15122.07212 2.72072 299.72072 151.22072 15122.07200 2020-01-01 2020-01-02 2020-01-01 00:15:06 2020-01-02 03:43:27 2020-01-01 00:15:06.000 2020-01-02 03:43:27.000 906 99807 50356.5 5035650 906 99807 50356.5 5035650 -32662 32273 4481.3 448130 -126 125 1.3 130 907 2 10897 99808 2.72372 299.72372 151.22372 15122.37237 2.72372 299.72372 151.22372 15122.37243 2.72372 299.72372 151.22372 15122.37200 2020-01-01 2020-01-02 2020-01-01 00:15:07 2020-01-02 03:43:28 2020-01-01 00:15:07.000 2020-01-02 03:43:28.000 907 99808 50357.5 5035750 907 99808 50357.5 5035750 -32661 32274 4482.3 448230 -125 126 2.3 230 -908 2 10898 99809 2.72672 299.72672 151.22672 15122.67267 2.72672 299.7267 151.22672 15122.67272 2.72672 299.72672 151.22672000000006 15122.67200 2020-01-01 2020-01-02 2020-01-01 00:15:08 2020-01-02 03:43:29 2020-01-01 00:15:08.000 2020-01-02 03:43:29.000 908 99809 50358.5 5035850 908 99809 50358.5 5035850 -32660 32275 4483.3 448330 -124 127 3.3 330 -909 2 10899 99810 2.72972 299.72972 151.22972 15122.97297 2.72972 299.72974 151.22973 15122.97332 2.72972 299.72972 151.22971999999987 15122.97200 2020-01-01 2020-01-02 2020-01-01 00:15:09 2020-01-02 03:43:30 2020-01-01 00:15:09.000 2020-01-02 03:43:30.000 909 99810 50359.5 5035950 909 99810 50359.5 5035950 -32659 32276 4484.3 448430 -128 127 1.74 174 -91 2 10081 99991 0.27327 300.27327 150.27327 15177.6006 0.27327 300.2733 150.27327 15177.60054 0.27327 300.27327 150.27326999999985 15177.60027 2020-01-01 2020-01-02 2020-01-01 00:01:31 2020-01-02 03:46:31 2020-01-01 00:01:31.000 2020-01-02 03:46:31.000 91 99991 50041 5054141 91 99991 50041 5054141 -32478 32457 4620.009900990099 466621 -128 127 -0.6633663366336634 -67 -910 2 10900 99811 2.73273 299.73273 151.23273 15123.27327 2.73273 299.73273 151.23272 15123.27296 2.73273 299.73273 151.2327299999999 15123.27300 2020-01-01 2020-01-02 2020-01-01 00:15:10 2020-01-02 03:43:31 2020-01-01 00:15:10.000 2020-01-02 03:43:31.000 910 99811 50360.5 5036050 910 99811 50360.5 5036050 -32658 32277 4485.3 448530 -128 123 0.18 18 -911 2 10901 99812 2.73573 299.73573 151.23573 15123.57357 2.73573 299.73575 151.23573 15123.57359 2.73573 299.73573 151.23573000000016 15123.57300 2020-01-01 2020-01-02 2020-01-01 00:15:11 2020-01-02 03:43:32 2020-01-01 00:15:11.000 2020-01-02 03:43:32.000 911 99812 50361.5 5036150 911 99812 50361.5 5036150 -32657 32278 4486.3 448630 -127 124 1.18 118 -912 2 10902 99813 2.73873 299.73873 151.23873 15123.87387 2.73873 299.73874 151.23873 15123.8739 2.73873 299.73873 151.2387299999998 15123.87300 2020-01-01 2020-01-02 2020-01-01 00:15:12 2020-01-02 03:43:33 2020-01-01 00:15:12.000 2020-01-02 03:43:33.000 912 99813 50362.5 5036250 912 99813 50362.5 5036250 -32656 32279 4487.3 448730 -126 125 2.18 218 -913 2 10903 99814 2.74174 299.74174 151.24174 15124.17417 2.74174 299.74173 151.24174 15124.17419 2.74174 299.74174 151.24174 15124.17400 2020-01-01 2020-01-02 2020-01-01 00:15:13 2020-01-02 03:43:34 2020-01-01 00:15:13.000 2020-01-02 03:43:34.000 913 99814 50363.5 5036350 913 99814 50363.5 5036350 -32655 32280 4488.3 448830 -125 126 3.18 318 -914 2 10904 99815 2.74474 299.74474 151.24474 15124.47447 2.74474 299.74475 151.24474 15124.47479 2.74474 299.74474 151.24474000000006 15124.47400 2020-01-01 2020-01-02 2020-01-01 00:15:14 2020-01-02 03:43:35 2020-01-01 00:15:14.000 2020-01-02 03:43:35.000 914 99815 50364.5 5036450 914 99815 50364.5 5036450 -32654 32281 4489.3 448930 -124 127 4.18 418 -915 2 10905 99816 2.74774 299.74774 151.24774 15124.77477 2.74774 299.74774 151.24774 15124.77447 2.74774 299.74774 151.24774000000008 15124.77400 2020-01-01 2020-01-02 2020-01-01 00:15:15 2020-01-02 03:43:36 2020-01-01 00:15:15.000 2020-01-02 03:43:36.000 915 99816 50365.5 5036550 915 99816 50365.5 5036550 -32653 32282 4490.3 449030 -128 127 2.62 262 -916 2 10906 99817 2.75075 299.75075 151.25075 15125.07507 2.75075 299.75076 151.25075 15125.07507 2.75075 299.75075 151.25074999999987 15125.07500 2020-01-01 2020-01-02 2020-01-01 00:15:16 2020-01-02 03:43:37 2020-01-01 00:15:16.000 2020-01-02 03:43:37.000 916 99817 50366.5 5036650 916 99817 50366.5 5036650 -32652 32283 4491.3 449130 -128 127 1.06 106 -917 2 10907 99818 2.75375 299.75375 151.25375 15125.37537 2.75375 299.75375 151.25375 15125.37536 2.75375 299.75375 151.2537499999999 15125.37500 2020-01-01 2020-01-02 2020-01-01 00:15:17 2020-01-02 03:43:38 2020-01-01 00:15:17.000 2020-01-02 03:43:38.000 917 99818 50367.5 5036750 917 99818 50367.5 5036750 -32651 32284 4492.3 449230 -128 124 -0.5 -50 -918 2 10908 99819 2.75675 299.75675 151.25675 15125.67567 2.75675 299.75674 151.25675 15125.67566 2.75675 299.75675 151.25675000000004 15125.67500 2020-01-01 2020-01-02 2020-01-01 00:15:18 2020-01-02 03:43:39 2020-01-01 00:15:18.000 2020-01-02 03:43:39.000 918 99819 50368.5 5036850 918 99819 50368.5 5036850 -32650 32285 4493.3 449330 -127 125 0.5 50 -919 2 10909 99820 2.75975 299.75975 151.25975 15125.97597 2.75975 299.75977 151.25976 15125.97626 2.75975 299.75975 151.25974999999994 15125.97500 2020-01-01 2020-01-02 2020-01-01 00:15:19 2020-01-02 03:43:40 2020-01-01 00:15:19.000 2020-01-02 03:43:40.000 919 99820 50369.5 5036950 919 99820 50369.5 5036950 -32649 32286 4494.3 449430 -126 126 1.5 150 -92 2 10082 99992 0.27627 300.27627 150.27627 15177.9039 0.27627 300.27628 150.27627 15177.90384 0.27627 300.27627 150.27627000000012 15177.90327 2020-01-01 2020-01-02 2020-01-01 00:01:32 2020-01-02 03:46:32 2020-01-01 00:01:32.000 2020-01-02 03:46:32.000 92 99992 50042 5054242 92 99992 50042 5054242 -32477 32458 4621.009900990099 466722 -128 123 -2.198019801980198 -222 -920 2 10910 99821 2.76276 299.76276 151.26276 15126.27627 2.76276 299.76276 151.26275 15126.27594 2.76276 299.76276 151.2627599999998 15126.27600 2020-01-01 2020-01-02 2020-01-01 00:15:20 2020-01-02 03:43:41 2020-01-01 00:15:20.000 2020-01-02 03:43:41.000 920 99821 50370.5 5037050 920 99821 50370.5 5037050 -32648 32287 4495.3 449530 -125 127 2.5 250 -921 2 10911 99822 2.76576 299.76576 151.26576 15126.57657 2.76576 299.76578 151.26576 15126.57654 2.76576 299.76576 151.26576000000017 15126.57600 2020-01-01 2020-01-02 2020-01-01 00:15:21 2020-01-02 03:43:42 2020-01-01 00:15:21.000 2020-01-02 03:43:42.000 921 99822 50371.5 5037150 921 99822 50371.5 5037150 -32647 32288 4496.3 449630 -128 127 0.94 94 -922 2 10912 99823 2.76876 299.76876 151.26876 15126.87687 2.76876 299.76877 151.26876 15126.87683 2.76876 299.76876 151.26876000000016 15126.87600 2020-01-01 2020-01-02 2020-01-01 00:15:22 2020-01-02 03:43:43 2020-01-01 00:15:22.000 2020-01-02 03:43:43.000 922 99823 50372.5 5037250 922 99823 50372.5 5037250 -32646 32289 4497.3 449730 -128 127 -0.62 -62 -923 2 10913 99824 2.77177 299.77177 151.27177 15127.17717 2.77177 299.77176 151.27177 15127.17713 2.77177 299.77177 151.27176999999992 15127.17700 2020-01-01 2020-01-02 2020-01-01 00:15:23 2020-01-02 03:43:44 2020-01-01 00:15:23.000 2020-01-02 03:43:44.000 923 99824 50373.5 5037350 923 99824 50373.5 5037350 -32645 32290 4498.3 449830 -128 123 -2.18 -218 -924 2 10914 99825 2.77477 299.77477 151.27477 15127.47747 2.77477 299.77478 151.27477 15127.47776 2.77477 299.77477 151.27477 15127.47700 2020-01-01 2020-01-02 2020-01-01 00:15:24 2020-01-02 03:43:45 2020-01-01 00:15:24.000 2020-01-02 03:43:45.000 924 99825 50374.5 5037450 924 99825 50374.5 5037450 -32644 32291 4499.3 449930 -127 124 -1.18 -118 -925 2 10915 99826 2.77777 299.77777 151.27777 15127.77777 2.77777 299.77777 151.27777 15127.77741 2.77777 299.77777 151.2777700000001 15127.77700 2020-01-01 2020-01-02 2020-01-01 00:15:25 2020-01-02 03:43:46 2020-01-01 00:15:25.000 2020-01-02 03:43:46.000 925 99826 50375.5 5037550 925 99826 50375.5 5037550 -32643 32292 4500.3 450030 -126 125 -0.18 -18 -926 2 10916 99827 2.78078 299.78078 151.28078 15128.07807 2.78078 299.7808 151.28078 15128.078 2.78078 299.78078 151.28077999999982 15128.07800 2020-01-01 2020-01-02 2020-01-01 00:15:26 2020-01-02 03:43:47 2020-01-01 00:15:26.000 2020-01-02 03:43:47.000 926 99827 50376.5 5037650 926 99827 50376.5 5037650 -32642 32293 4501.3 450130 -125 126 0.82 82 -927 2 10917 99828 2.78378 299.78378 151.28378 15128.37837 2.78378 299.78378 151.28378 15128.3783 2.78378 299.78378 151.28377999999987 15128.37800 2020-01-01 2020-01-02 2020-01-01 00:15:27 2020-01-02 03:43:48 2020-01-01 00:15:27.000 2020-01-02 03:43:48.000 927 99828 50377.5 5037750 927 99828 50377.5 5037750 -32641 32294 4502.3 450230 -124 127 1.82 182 -928 2 10918 99829 2.78678 299.78678 151.28678 15128.67867 2.78678 299.78677 151.28678 15128.6786 2.78678 299.78678 151.2867800000002 15128.67800 2020-01-01 2020-01-02 2020-01-01 00:15:28 2020-01-02 03:43:49 2020-01-01 00:15:28.000 2020-01-02 03:43:49.000 928 99829 50378.5 5037850 928 99829 50378.5 5037850 -32640 32295 4503.3 450330 -128 127 0.26 26 -929 2 10919 99830 2.78978 299.78978 151.28978 15128.97897 2.78978 299.7898 151.28979 15128.97923 2.78978 299.78978 151.28977999999984 15128.97800 2020-01-01 2020-01-02 2020-01-01 00:15:29 2020-01-02 03:43:50 2020-01-01 00:15:29.000 2020-01-02 03:43:50.000 929 99830 50379.5 5037950 929 99830 50379.5 5037950 -32639 32296 4504.3 450430 -128 123 -1.3 -130 -93 2 10083 99993 0.27927 300.27927 150.27927 15178.2072 0.27927 300.27927 150.27927 15178.20715 0.27927 300.27927 150.27926999999983 15178.20627 2020-01-01 2020-01-02 2020-01-01 00:01:33 2020-01-02 03:46:33 2020-01-01 00:01:33.000 2020-01-02 03:46:33.000 93 99993 50043 5054343 93 99993 50043 5054343 -32476 32459 4622.009900990099 466823 -127 124 -1.198019801980198 -121 +908 2 10898 99809 2.72672 299.72672 151.22672 15122.67267 2.72672 299.7267 151.22672 15122.67272 2.72672 299.72672 151.22672 15122.67200 2020-01-01 2020-01-02 2020-01-01 00:15:08 2020-01-02 03:43:29 2020-01-01 00:15:08.000 2020-01-02 03:43:29.000 908 99809 50358.5 5035850 908 99809 50358.5 5035850 -32660 32275 4483.3 448330 -124 127 3.3 330 +909 2 10899 99810 2.72972 299.72972 151.22972 15122.97297 2.72972 299.72974 151.22973 15122.97332 2.72972 299.72972 151.22972 15122.97200 2020-01-01 2020-01-02 2020-01-01 00:15:09 2020-01-02 03:43:30 2020-01-01 00:15:09.000 2020-01-02 03:43:30.000 909 99810 50359.5 5035950 909 99810 50359.5 5035950 -32659 32276 4484.3 448430 -128 127 1.74 174 +91 2 10081 99991 0.27327 300.27327 150.27327 15177.6006 0.27327 300.2733 150.27327 15177.60054 0.27327 300.27327 150.27327 15177.60027 2020-01-01 2020-01-02 2020-01-01 00:01:31 2020-01-02 03:46:31 2020-01-01 00:01:31.000 2020-01-02 03:46:31.000 91 99991 50041 5054141 91 99991 50041 5054141 -32478 32457 4620.009900990099 466621 -128 127 -0.6633663366336634 -67 +910 2 10900 99811 2.73273 299.73273 151.23273 15123.27327 2.73273 299.73273 151.23272 15123.27296 2.73273 299.73273 151.23273 15123.27300 2020-01-01 2020-01-02 2020-01-01 00:15:10 2020-01-02 03:43:31 2020-01-01 00:15:10.000 2020-01-02 03:43:31.000 910 99811 50360.5 5036050 910 99811 50360.5 5036050 -32658 32277 4485.3 448530 -128 123 0.18 18 +911 2 10901 99812 2.73573 299.73573 151.23573 15123.57357 2.73573 299.73575 151.23573 15123.57359 2.73573 299.73573 151.23573 15123.57300 2020-01-01 2020-01-02 2020-01-01 00:15:11 2020-01-02 03:43:32 2020-01-01 00:15:11.000 2020-01-02 03:43:32.000 911 99812 50361.5 5036150 911 99812 50361.5 5036150 -32657 32278 4486.3 448630 -127 124 1.18 118 +912 2 10902 99813 2.73873 299.73873 151.23873 15123.87387 2.73873 299.73874 151.23873 15123.8739 2.73873 299.73873 151.23873 15123.87300 2020-01-01 2020-01-02 2020-01-01 00:15:12 2020-01-02 03:43:33 2020-01-01 00:15:12.000 2020-01-02 03:43:33.000 912 99813 50362.5 5036250 912 99813 50362.5 5036250 -32656 32279 4487.3 448730 -126 125 2.18 218 +913 2 10903 99814 2.74174 299.74174 151.24174 15124.17417 2.74174 299.74173 151.24174 15124.17419 2.74174 299.74174 151.24174000000002 15124.17400 2020-01-01 2020-01-02 2020-01-01 00:15:13 2020-01-02 03:43:34 2020-01-01 00:15:13.000 2020-01-02 03:43:34.000 913 99814 50363.5 5036350 913 99814 50363.5 5036350 -32655 32280 4488.3 448830 -125 126 3.18 318 +914 2 10904 99815 2.74474 299.74474 151.24474 15124.47447 2.74474 299.74475 151.24474 15124.47479 2.74474 299.74474 151.24474 15124.47400 2020-01-01 2020-01-02 2020-01-01 00:15:14 2020-01-02 03:43:35 2020-01-01 00:15:14.000 2020-01-02 03:43:35.000 914 99815 50364.5 5036450 914 99815 50364.5 5036450 -32654 32281 4489.3 448930 -124 127 4.18 418 +915 2 10905 99816 2.74774 299.74774 151.24774 15124.77477 2.74774 299.74774 151.24774 15124.77447 2.74774 299.74774 151.24774 15124.77400 2020-01-01 2020-01-02 2020-01-01 00:15:15 2020-01-02 03:43:36 2020-01-01 00:15:15.000 2020-01-02 03:43:36.000 915 99816 50365.5 5036550 915 99816 50365.5 5036550 -32653 32282 4490.3 449030 -128 127 2.62 262 +916 2 10906 99817 2.75075 299.75075 151.25075 15125.07507 2.75075 299.75076 151.25075 15125.07507 2.75075 299.75075 151.25075 15125.07500 2020-01-01 2020-01-02 2020-01-01 00:15:16 2020-01-02 03:43:37 2020-01-01 00:15:16.000 2020-01-02 03:43:37.000 916 99817 50366.5 5036650 916 99817 50366.5 5036650 -32652 32283 4491.3 449130 -128 127 1.06 106 +917 2 10907 99818 2.75375 299.75375 151.25375 15125.37537 2.75375 299.75375 151.25375 15125.37536 2.75375 299.75375 151.25375 15125.37500 2020-01-01 2020-01-02 2020-01-01 00:15:17 2020-01-02 03:43:38 2020-01-01 00:15:17.000 2020-01-02 03:43:38.000 917 99818 50367.5 5036750 917 99818 50367.5 5036750 -32651 32284 4492.3 449230 -128 124 -0.5 -50 +918 2 10908 99819 2.75675 299.75675 151.25675 15125.67567 2.75675 299.75674 151.25675 15125.67566 2.75675 299.75675 151.25674999999998 15125.67500 2020-01-01 2020-01-02 2020-01-01 00:15:18 2020-01-02 03:43:39 2020-01-01 00:15:18.000 2020-01-02 03:43:39.000 918 99819 50368.5 5036850 918 99819 50368.5 5036850 -32650 32285 4493.3 449330 -127 125 0.5 50 +919 2 10909 99820 2.75975 299.75975 151.25975 15125.97597 2.75975 299.75977 151.25976 15125.97626 2.75975 299.75975 151.25975 15125.97500 2020-01-01 2020-01-02 2020-01-01 00:15:19 2020-01-02 03:43:40 2020-01-01 00:15:19.000 2020-01-02 03:43:40.000 919 99820 50369.5 5036950 919 99820 50369.5 5036950 -32649 32286 4494.3 449430 -126 126 1.5 150 +92 2 10082 99992 0.27627 300.27627 150.27627 15177.9039 0.27627 300.27628 150.27627 15177.90384 0.27627 300.27627 150.27627 15177.90327 2020-01-01 2020-01-02 2020-01-01 00:01:32 2020-01-02 03:46:32 2020-01-01 00:01:32.000 2020-01-02 03:46:32.000 92 99992 50042 5054242 92 99992 50042 5054242 -32477 32458 4621.009900990099 466722 -128 123 -2.198019801980198 -222 +920 2 10910 99821 2.76276 299.76276 151.26276 15126.27627 2.76276 299.76276 151.26275 15126.27594 2.76276 299.76276 151.26276 15126.27600 2020-01-01 2020-01-02 2020-01-01 00:15:20 2020-01-02 03:43:41 2020-01-01 00:15:20.000 2020-01-02 03:43:41.000 920 99821 50370.5 5037050 920 99821 50370.5 5037050 -32648 32287 4495.3 449530 -125 127 2.5 250 +921 2 10911 99822 2.76576 299.76576 151.26576 15126.57657 2.76576 299.76578 151.26576 15126.57654 2.76576 299.76576 151.26576 15126.57600 2020-01-01 2020-01-02 2020-01-01 00:15:21 2020-01-02 03:43:42 2020-01-01 00:15:21.000 2020-01-02 03:43:42.000 921 99822 50371.5 5037150 921 99822 50371.5 5037150 -32647 32288 4496.3 449630 -128 127 0.94 94 +922 2 10912 99823 2.76876 299.76876 151.26876 15126.87687 2.76876 299.76877 151.26876 15126.87683 2.76876 299.76876 151.26876000000001 15126.87600 2020-01-01 2020-01-02 2020-01-01 00:15:22 2020-01-02 03:43:43 2020-01-01 00:15:22.000 2020-01-02 03:43:43.000 922 99823 50372.5 5037250 922 99823 50372.5 5037250 -32646 32289 4497.3 449730 -128 127 -0.62 -62 +923 2 10913 99824 2.77177 299.77177 151.27177 15127.17717 2.77177 299.77176 151.27177 15127.17713 2.77177 299.77177 151.27177 15127.17700 2020-01-01 2020-01-02 2020-01-01 00:15:23 2020-01-02 03:43:44 2020-01-01 00:15:23.000 2020-01-02 03:43:44.000 923 99824 50373.5 5037350 923 99824 50373.5 5037350 -32645 32290 4498.3 449830 -128 123 -2.18 -218 +924 2 10914 99825 2.77477 299.77477 151.27477 15127.47747 2.77477 299.77478 151.27477 15127.47776 2.77477 299.77477 151.27477000000002 15127.47700 2020-01-01 2020-01-02 2020-01-01 00:15:24 2020-01-02 03:43:45 2020-01-01 00:15:24.000 2020-01-02 03:43:45.000 924 99825 50374.5 5037450 924 99825 50374.5 5037450 -32644 32291 4499.3 449930 -127 124 -1.18 -118 +925 2 10915 99826 2.77777 299.77777 151.27777 15127.77777 2.77777 299.77777 151.27777 15127.77741 2.77777 299.77777 151.27777 15127.77700 2020-01-01 2020-01-02 2020-01-01 00:15:25 2020-01-02 03:43:46 2020-01-01 00:15:25.000 2020-01-02 03:43:46.000 925 99826 50375.5 5037550 925 99826 50375.5 5037550 -32643 32292 4500.3 450030 -126 125 -0.18 -18 +926 2 10916 99827 2.78078 299.78078 151.28078 15128.07807 2.78078 299.7808 151.28078 15128.078 2.78078 299.78078 151.28078 15128.07800 2020-01-01 2020-01-02 2020-01-01 00:15:26 2020-01-02 03:43:47 2020-01-01 00:15:26.000 2020-01-02 03:43:47.000 926 99827 50376.5 5037650 926 99827 50376.5 5037650 -32642 32293 4501.3 450130 -125 126 0.82 82 +927 2 10917 99828 2.78378 299.78378 151.28378 15128.37837 2.78378 299.78378 151.28378 15128.3783 2.78378 299.78378 151.28378 15128.37800 2020-01-01 2020-01-02 2020-01-01 00:15:27 2020-01-02 03:43:48 2020-01-01 00:15:27.000 2020-01-02 03:43:48.000 927 99828 50377.5 5037750 927 99828 50377.5 5037750 -32641 32294 4502.3 450230 -124 127 1.82 182 +928 2 10918 99829 2.78678 299.78678 151.28678 15128.67867 2.78678 299.78677 151.28678 15128.6786 2.78678 299.78678 151.28678 15128.67800 2020-01-01 2020-01-02 2020-01-01 00:15:28 2020-01-02 03:43:49 2020-01-01 00:15:28.000 2020-01-02 03:43:49.000 928 99829 50378.5 5037850 928 99829 50378.5 5037850 -32640 32295 4503.3 450330 -128 127 0.26 26 +929 2 10919 99830 2.78978 299.78978 151.28978 15128.97897 2.78978 299.7898 151.28979 15128.97923 2.78978 299.78978 151.28977999999998 15128.97800 2020-01-01 2020-01-02 2020-01-01 00:15:29 2020-01-02 03:43:50 2020-01-01 00:15:29.000 2020-01-02 03:43:50.000 929 99830 50379.5 5037950 929 99830 50379.5 5037950 -32639 32296 4504.3 450430 -128 123 -1.3 -130 +93 2 10083 99993 0.27927 300.27927 150.27927 15178.2072 0.27927 300.27927 150.27927 15178.20715 0.27927 300.27927 150.27927 15178.20627 2020-01-01 2020-01-02 2020-01-01 00:01:33 2020-01-02 03:46:33 2020-01-01 00:01:33.000 2020-01-02 03:46:33.000 93 99993 50043 5054343 93 99993 50043 5054343 -32476 32459 4622.009900990099 466823 -127 124 -1.198019801980198 -121 930 2 10920 99831 2.79279 299.79279 151.29279 15129.27927 2.79279 299.7928 151.29278 15129.27888 2.79279 299.79279 151.29279 15129.27900 2020-01-01 2020-01-02 2020-01-01 00:15:30 2020-01-02 03:43:51 2020-01-01 00:15:30.000 2020-01-02 03:43:51.000 930 99831 50380.5 5038050 930 99831 50380.5 5038050 -32638 32297 4505.3 450530 -127 124 -0.3 -30 -931 2 10921 99832 2.79579 299.79579 151.29579 15129.57957 2.79579 299.7958 151.29579 15129.57963 2.79579 299.79579 151.2957900000001 15129.57900 2020-01-01 2020-01-02 2020-01-01 00:15:31 2020-01-02 03:43:52 2020-01-01 00:15:31.000 2020-01-02 03:43:52.000 931 99832 50381.5 5038150 931 99832 50381.5 5038150 -32637 32298 4506.3 450630 -126 125 0.7 70 -932 2 10922 99833 2.79879 299.79879 151.29879 15129.87987 2.79879 299.7988 151.29879 15129.87977 2.79879 299.79879 151.2987900000001 15129.87900 2020-01-01 2020-01-02 2020-01-01 00:15:32 2020-01-02 03:43:53 2020-01-01 00:15:32.000 2020-01-02 03:43:53.000 932 99833 50382.5 5038250 932 99833 50382.5 5038250 -32636 32299 4507.3 450730 -125 126 1.7 170 -933 2 10923 99834 2.8018 299.8018 151.3018 15130.18018 2.8018 299.8018 151.3018 15130.18011 2.80180 299.80180 151.3017999999999 15130.18000 2020-01-01 2020-01-02 2020-01-01 00:15:33 2020-01-02 03:43:54 2020-01-01 00:15:33.000 2020-01-02 03:43:54.000 933 99834 50383.5 5038350 933 99834 50383.5 5038350 -32635 32300 4508.3 450830 -124 127 2.7 270 -934 2 10924 99835 2.8048 299.8048 151.3048 15130.48048 2.8048 299.8048 151.3048 15130.48071 2.80480 299.80480 151.30479999999991 15130.48000 2020-01-01 2020-01-02 2020-01-01 00:15:34 2020-01-02 03:43:55 2020-01-01 00:15:34.000 2020-01-02 03:43:55.000 934 99835 50384.5 5038450 934 99835 50384.5 5038450 -32634 32301 4509.3 450930 -128 127 1.14 114 +931 2 10921 99832 2.79579 299.79579 151.29579 15129.57957 2.79579 299.7958 151.29579 15129.57963 2.79579 299.79579 151.29579 15129.57900 2020-01-01 2020-01-02 2020-01-01 00:15:31 2020-01-02 03:43:52 2020-01-01 00:15:31.000 2020-01-02 03:43:52.000 931 99832 50381.5 5038150 931 99832 50381.5 5038150 -32637 32298 4506.3 450630 -126 125 0.7 70 +932 2 10922 99833 2.79879 299.79879 151.29879 15129.87987 2.79879 299.7988 151.29879 15129.87977 2.79879 299.79879 151.29879 15129.87900 2020-01-01 2020-01-02 2020-01-01 00:15:32 2020-01-02 03:43:53 2020-01-01 00:15:32.000 2020-01-02 03:43:53.000 932 99833 50382.5 5038250 932 99833 50382.5 5038250 -32636 32299 4507.3 450730 -125 126 1.7 170 +933 2 10923 99834 2.8018 299.8018 151.3018 15130.18018 2.8018 299.8018 151.3018 15130.18011 2.80180 299.80180 151.30180000000001 15130.18000 2020-01-01 2020-01-02 2020-01-01 00:15:33 2020-01-02 03:43:54 2020-01-01 00:15:33.000 2020-01-02 03:43:54.000 933 99834 50383.5 5038350 933 99834 50383.5 5038350 -32635 32300 4508.3 450830 -124 127 2.7 270 +934 2 10924 99835 2.8048 299.8048 151.3048 15130.48048 2.8048 299.8048 151.3048 15130.48071 2.80480 299.80480 151.3048 15130.48000 2020-01-01 2020-01-02 2020-01-01 00:15:34 2020-01-02 03:43:55 2020-01-01 00:15:34.000 2020-01-02 03:43:55.000 934 99835 50384.5 5038450 934 99835 50384.5 5038450 -32634 32301 4509.3 450930 -128 127 1.14 114 935 2 10925 99836 2.8078 299.8078 151.3078 15130.78078 2.8078 299.8078 151.3078 15130.78034 2.80780 299.80780 151.30780000000001 15130.78000 2020-01-01 2020-01-02 2020-01-01 00:15:35 2020-01-02 03:43:56 2020-01-01 00:15:35.000 2020-01-02 03:43:56.000 935 99836 50385.5 5038550 935 99836 50385.5 5038550 -32633 32302 4510.3 451030 -128 123 -0.42 -42 -936 2 10926 99837 2.81081 299.81081 151.31081 15131.08108 2.81081 299.81082 151.31081 15131.0811 2.81081 299.81081 151.31081000000017 15131.08100 2020-01-01 2020-01-02 2020-01-01 00:15:36 2020-01-02 03:43:57 2020-01-01 00:15:36.000 2020-01-02 03:43:57.000 936 99837 50386.5 5038650 936 99837 50386.5 5038650 -32632 32303 4511.3 451130 -127 124 0.58 58 -937 2 10927 99838 2.81381 299.81381 151.31381 15131.38138 2.81381 299.8138 151.31381 15131.38124 2.81381 299.81381 151.31380999999985 15131.38100 2020-01-01 2020-01-02 2020-01-01 00:15:37 2020-01-02 03:43:58 2020-01-01 00:15:37.000 2020-01-02 03:43:58.000 937 99838 50387.5 5038750 937 99838 50387.5 5038750 -32631 32304 4512.3 451230 -126 125 1.58 158 -938 2 10928 99839 2.81681 299.81681 151.31681 15131.68168 2.81681 299.8168 151.31681 15131.68157 2.81681 299.81681 151.3168100000001 15131.68100 2020-01-01 2020-01-02 2020-01-01 00:15:38 2020-01-02 03:43:59 2020-01-01 00:15:38.000 2020-01-02 03:43:59.000 938 99839 50388.5 5038850 938 99839 50388.5 5038850 -32630 32305 4513.3 451330 -125 126 2.58 258 -939 2 10929 99840 2.81981 299.81981 151.31981 15131.98198 2.81982 299.81982 151.31982 15131.98217 2.81981 299.81981 151.3198100000002 15131.98100 2020-01-01 2020-01-02 2020-01-01 00:15:39 2020-01-02 03:44:00 2020-01-01 00:15:39.000 2020-01-02 03:44:00.000 939 99840 50389.5 5038950 939 99840 50389.5 5038950 -32629 32306 4514.3 451430 -124 127 3.58 358 -94 2 10084 99994 0.28228 300.28228 150.28228 15178.51051 0.28228 300.2823 150.28228 15178.51078 0.28228 300.28228 150.28227999999996 15178.51028 2020-01-01 2020-01-02 2020-01-01 00:01:34 2020-01-02 03:46:34 2020-01-01 00:01:34.000 2020-01-02 03:46:34.000 94 99994 50044 5054444 94 99994 50044 5054444 -32475 32460 4623.009900990099 466924 -126 125 -0.19801980198019803 -20 -940 2 10930 99841 2.82282 299.82282 151.32282 15132.28228 2.82282 299.8228 151.32282 15132.28247 2.82282 299.82282 151.32281999999995 15132.28200 2020-01-01 2020-01-02 2020-01-01 00:15:40 2020-01-02 03:44:01 2020-01-01 00:15:40.000 2020-01-02 03:44:01.000 940 99841 50390.5 5039050 940 99841 50390.5 5039050 -32628 32307 4515.3 451530 -128 127 2.02 202 +936 2 10926 99837 2.81081 299.81081 151.31081 15131.08108 2.81081 299.81082 151.31081 15131.0811 2.81081 299.81081 151.31081 15131.08100 2020-01-01 2020-01-02 2020-01-01 00:15:36 2020-01-02 03:43:57 2020-01-01 00:15:36.000 2020-01-02 03:43:57.000 936 99837 50386.5 5038650 936 99837 50386.5 5038650 -32632 32303 4511.3 451130 -127 124 0.58 58 +937 2 10927 99838 2.81381 299.81381 151.31381 15131.38138 2.81381 299.8138 151.31381 15131.38124 2.81381 299.81381 151.31381 15131.38100 2020-01-01 2020-01-02 2020-01-01 00:15:37 2020-01-02 03:43:58 2020-01-01 00:15:37.000 2020-01-02 03:43:58.000 937 99838 50387.5 5038750 937 99838 50387.5 5038750 -32631 32304 4512.3 451230 -126 125 1.58 158 +938 2 10928 99839 2.81681 299.81681 151.31681 15131.68168 2.81681 299.8168 151.31681 15131.68157 2.81681 299.81681 151.31681 15131.68100 2020-01-01 2020-01-02 2020-01-01 00:15:38 2020-01-02 03:43:59 2020-01-01 00:15:38.000 2020-01-02 03:43:59.000 938 99839 50388.5 5038850 938 99839 50388.5 5038850 -32630 32305 4513.3 451330 -125 126 2.58 258 +939 2 10929 99840 2.81981 299.81981 151.31981 15131.98198 2.81982 299.81982 151.31982 15131.98217 2.81981 299.81981 151.31981 15131.98100 2020-01-01 2020-01-02 2020-01-01 00:15:39 2020-01-02 03:44:00 2020-01-01 00:15:39.000 2020-01-02 03:44:00.000 939 99840 50389.5 5038950 939 99840 50389.5 5038950 -32629 32306 4514.3 451430 -124 127 3.58 358 +94 2 10084 99994 0.28228 300.28228 150.28228 15178.51051 0.28228 300.2823 150.28228 15178.51078 0.28228 300.28228 150.28228000000001 15178.51028 2020-01-01 2020-01-02 2020-01-01 00:01:34 2020-01-02 03:46:34 2020-01-01 00:01:34.000 2020-01-02 03:46:34.000 94 99994 50044 5054444 94 99994 50044 5054444 -32475 32460 4623.009900990099 466924 -126 125 -0.19801980198019803 -20 +940 2 10930 99841 2.82282 299.82282 151.32282 15132.28228 2.82282 299.8228 151.32282 15132.28247 2.82282 299.82282 151.32281999999998 15132.28200 2020-01-01 2020-01-02 2020-01-01 00:15:40 2020-01-02 03:44:01 2020-01-01 00:15:40.000 2020-01-02 03:44:01.000 940 99841 50390.5 5039050 940 99841 50390.5 5039050 -32628 32307 4515.3 451530 -128 127 2.02 202 941 2 10931 99842 2.82582 299.82582 151.32582 15132.58258 2.82582 299.82584 151.32582 15132.58257 2.82582 299.82582 151.32582 15132.58200 2020-01-01 2020-01-02 2020-01-01 00:15:41 2020-01-02 03:44:02 2020-01-01 00:15:41.000 2020-01-02 03:44:02.000 941 99842 50391.5 5039150 941 99842 50391.5 5039150 -32627 32308 4516.3 451630 -128 127 0.46 46 -942 2 10932 99843 2.82882 299.82882 151.32882 15132.88288 2.82882 299.82883 151.32882 15132.88275 2.82882 299.82882 151.3288200000001 15132.88200 2020-01-01 2020-01-02 2020-01-01 00:15:42 2020-01-02 03:44:03 2020-01-01 00:15:42.000 2020-01-02 03:44:03.000 942 99843 50392.5 5039250 942 99843 50392.5 5039250 -32626 32309 4517.3 451730 -128 124 -1.1 -110 -943 2 10933 99844 2.83183 299.83183 151.33183 15133.18318 2.83183 299.83182 151.33183 15133.18304 2.83183 299.83183 151.33182999999985 15133.18300 2020-01-01 2020-01-02 2020-01-01 00:15:43 2020-01-02 03:44:04 2020-01-01 00:15:43.000 2020-01-02 03:44:04.000 943 99844 50393.5 5039350 943 99844 50393.5 5039350 -32625 32310 4518.3 451830 -127 125 -0.1 -10 -944 2 10934 99845 2.83483 299.83483 151.33483 15133.48348 2.83483 299.83484 151.33483 15133.48364 2.83483 299.83483 151.33482999999984 15133.48300 2020-01-01 2020-01-02 2020-01-01 00:15:44 2020-01-02 03:44:05 2020-01-01 00:15:44.000 2020-01-02 03:44:05.000 944 99845 50394.5 5039450 944 99845 50394.5 5039450 -32624 32311 4519.3 451930 -126 126 0.9 90 -945 2 10935 99846 2.83783 299.83783 151.33783 15133.78378 2.83783 299.83783 151.33783 15133.78393 2.83783 299.83783 151.3378300000002 15133.78300 2020-01-01 2020-01-02 2020-01-01 00:15:45 2020-01-02 03:44:06 2020-01-01 00:15:45.000 2020-01-02 03:44:06.000 945 99846 50395.5 5039550 945 99846 50395.5 5039550 -32623 32312 4520.3 452030 -125 127 1.9 190 -946 2 10936 99847 2.84084 299.84084 151.34084 15134.08408 2.84084 299.84085 151.34084 15134.08404 2.84084 299.84084 151.34084000000007 15134.08400 2020-01-01 2020-01-02 2020-01-01 00:15:46 2020-01-02 03:44:07 2020-01-01 00:15:46.000 2020-01-02 03:44:07.000 946 99847 50396.5 5039650 946 99847 50396.5 5039650 -32622 32313 4521.3 452130 -128 127 0.34 34 -947 2 10937 99848 2.84384 299.84384 151.34384 15134.38438 2.84384 299.84384 151.34384 15134.38421 2.84384 299.84384 151.34383999999977 15134.38400 2020-01-01 2020-01-02 2020-01-01 00:15:47 2020-01-02 03:44:08 2020-01-01 00:15:47.000 2020-01-02 03:44:08.000 947 99848 50397.5 5039750 947 99848 50397.5 5039750 -32621 32314 4522.3 452230 -128 127 -1.22 -122 -948 2 10938 99849 2.84684 299.84684 151.34684 15134.68468 2.84684 299.84683 151.34684 15134.68452 2.84684 299.84684 151.3468400000001 15134.68400 2020-01-01 2020-01-02 2020-01-01 00:15:48 2020-01-02 03:44:09 2020-01-01 00:15:48.000 2020-01-02 03:44:09.000 948 99849 50398.5 5039850 948 99849 50398.5 5039850 -32620 32315 4523.3 452330 -128 123 -2.78 -278 -949 2 10939 99850 2.84984 299.84984 151.34984 15134.98498 2.84985 299.84985 151.34985 15134.98527 2.84984 299.84984 151.34984000000009 15134.98400 2020-01-01 2020-01-02 2020-01-01 00:15:49 2020-01-02 03:44:10 2020-01-01 00:15:49.000 2020-01-02 03:44:10.000 949 99850 50399.5 5039950 949 99850 50399.5 5039950 -32619 32316 4524.3 452430 -127 124 -1.78 -178 -95 2 10085 99995 0.28528 300.28528 150.28528 15178.81381 0.28528 300.28528 150.28528 15178.81343 0.28528 300.28528 150.28528000000003 15178.81328 2020-01-01 2020-01-02 2020-01-01 00:01:35 2020-01-02 03:46:35 2020-01-01 00:01:35.000 2020-01-02 03:46:35.000 95 99995 50045 5054545 95 99995 50045 5054545 -32474 32461 4624.009900990099 467025 -125 126 0.801980198019802 81 -950 2 10940 99851 2.85285 299.85285 151.35285 15135.28528 2.85285 299.85284 151.35285 15135.28541 2.85285 299.85285 151.35284999999996 15135.28500 2020-01-01 2020-01-02 2020-01-01 00:15:50 2020-01-02 03:44:11 2020-01-01 00:15:50.000 2020-01-02 03:44:11.000 950 99851 50400.5 5040050 950 99851 50400.5 5040050 -32618 32317 4525.3 452530 -126 125 -0.78 -78 -951 2 10941 99852 2.85585 299.85585 151.35585 15135.58558 2.85585 299.85587 151.35585 15135.58551 2.85585 299.85585 151.35584999999995 15135.58500 2020-01-01 2020-01-02 2020-01-01 00:15:51 2020-01-02 03:44:12 2020-01-01 00:15:51.000 2020-01-02 03:44:12.000 951 99852 50401.5 5040150 951 99852 50401.5 5040150 -32617 32318 4526.3 452630 -125 126 0.22 22 -952 2 10942 99853 2.85885 299.85885 151.35885 15135.88588 2.85885 299.85886 151.35885 15135.88568 2.85885 299.85885 151.35885000000002 15135.88500 2020-01-01 2020-01-02 2020-01-01 00:15:52 2020-01-02 03:44:13 2020-01-01 00:15:52.000 2020-01-02 03:44:13.000 952 99853 50402.5 5040250 952 99853 50402.5 5040250 -32616 32319 4527.3 452730 -124 127 1.22 122 -953 2 10943 99854 2.86186 299.86186 151.36186 15136.18618 2.86186 299.86185 151.36185 15136.18598 2.86186 299.86186 151.36186000000018 15136.18600 2020-01-01 2020-01-02 2020-01-01 00:15:53 2020-01-02 03:44:14 2020-01-01 00:15:53.000 2020-01-02 03:44:14.000 953 99854 50403.5 5040350 953 99854 50403.5 5040350 -32615 32320 4528.3 452830 -128 127 -0.34 -34 -954 2 10944 99855 2.86486 299.86486 151.36486 15136.48648 2.86486 299.86487 151.36486 15136.48674 2.86486 299.86486 151.36485999999985 15136.48600 2020-01-01 2020-01-02 2020-01-01 00:15:54 2020-01-02 03:44:15 2020-01-01 00:15:54.000 2020-01-02 03:44:15.000 954 99855 50404.5 5040450 954 99855 50404.5 5040450 -32614 32321 4529.3 452930 -128 123 -1.9 -190 -955 2 10945 99856 2.86786 299.86786 151.36786 15136.78678 2.86786 299.86786 151.36786 15136.78688 2.86786 299.86786 151.36786000000012 15136.78600 2020-01-01 2020-01-02 2020-01-01 00:15:55 2020-01-02 03:44:16 2020-01-01 00:15:55.000 2020-01-02 03:44:16.000 955 99856 50405.5 5040550 955 99856 50405.5 5040550 -32613 32322 4530.3 453030 -127 124 -0.9 -90 -956 2 10946 99857 2.87087 299.87087 151.37087 15137.08708 2.87087 299.87088 151.37087 15137.08701 2.87087 299.87087 151.37087000000014 15137.08700 2020-01-01 2020-01-02 2020-01-01 00:15:56 2020-01-02 03:44:17 2020-01-01 00:15:56.000 2020-01-02 03:44:17.000 956 99857 50406.5 5040650 956 99857 50406.5 5040650 -32612 32323 4531.3 453130 -126 125 0.1 10 -957 2 10947 99858 2.87387 299.87387 151.37387 15137.38738 2.87387 299.87387 151.37387 15137.38716 2.87387 299.87387 151.37386999999995 15137.38700 2020-01-01 2020-01-02 2020-01-01 00:15:57 2020-01-02 03:44:18 2020-01-01 00:15:57.000 2020-01-02 03:44:18.000 957 99858 50407.5 5040750 957 99858 50407.5 5040750 -32611 32324 4532.3 453230 -125 126 1.1 110 -958 2 10948 99859 2.87687 299.87687 151.37687 15137.68768 2.87687 299.8769 151.37687 15137.68791 2.87687 299.87687 151.37687000000003 15137.68700 2020-01-01 2020-01-02 2020-01-01 00:15:58 2020-01-02 03:44:19 2020-01-01 00:15:58.000 2020-01-02 03:44:19.000 958 99859 50408.5 5040850 958 99859 50408.5 5040850 -32610 32325 4533.3 453330 -124 127 2.1 210 -959 2 10949 99860 2.87987 299.87987 151.37987 15137.98798 2.87988 299.87988 151.37988 15137.9882 2.87987 299.87987 151.3798700000001 15137.98700 2020-01-01 2020-01-02 2020-01-01 00:15:59 2020-01-02 03:44:20 2020-01-01 00:15:59.000 2020-01-02 03:44:20.000 959 99860 50409.5 5040950 959 99860 50409.5 5040950 -32609 32326 4534.3 453430 -128 127 0.54 54 -96 2 10086 99996 0.28828 300.28828 150.28828 15179.11711 0.28828 300.2883 150.28828 15179.11718 0.28828 300.28828 150.28828000000007 15179.11628 2020-01-01 2020-01-02 2020-01-01 00:01:36 2020-01-02 03:46:36 2020-01-01 00:01:36.000 2020-01-02 03:46:36.000 96 99996 50046 5054646 96 99996 50046 5054646 -32473 32462 4625.009900990099 467126 -124 127 1.801980198019802 182 -960 2 10950 99861 2.88288 299.88288 151.38288 15138.28828 2.88288 299.88287 151.38288 15138.28834 2.88288 299.88288 151.3828799999999 15138.28800 2020-01-01 2020-01-02 2020-01-01 00:16:00 2020-01-02 03:44:21 2020-01-01 00:16:00.000 2020-01-02 03:44:21.000 960 99861 50410.5 5041050 960 99861 50410.5 5041050 -32608 32327 4535.3 453530 -128 123 -1.02 -102 -961 2 10951 99862 2.88588 299.88588 151.38588 15138.58858 2.88588 299.8859 151.38588 15138.58848 2.88588 299.88588 151.38587999999987 15138.58800 2020-01-01 2020-01-02 2020-01-01 00:16:01 2020-01-02 03:44:22 2020-01-01 00:16:01.000 2020-01-02 03:44:22.000 961 99862 50411.5 5041150 961 99862 50411.5 5041150 -32607 32328 4536.3 453630 -127 124 -0.02 -2 -962 2 10952 99863 2.88888 299.88888 151.38888 15138.88888 2.88888 299.8889 151.38888 15138.88862 2.88888 299.88888 151.3888800000002 15138.88800 2020-01-01 2020-01-02 2020-01-01 00:16:02 2020-01-02 03:44:23 2020-01-01 00:16:02.000 2020-01-02 03:44:23.000 962 99863 50412.5 5041250 962 99863 50412.5 5041250 -32606 32329 4537.3 453730 -126 125 0.98 98 -963 2 10953 99864 2.89189 299.89189 151.39189 15139.18918 2.89189 299.8919 151.39189 15139.18937 2.89189 299.89189 151.39189000000013 15139.18900 2020-01-01 2020-01-02 2020-01-01 00:16:03 2020-01-02 03:44:24 2020-01-01 00:16:03.000 2020-01-02 03:44:24.000 963 99864 50413.5 5041350 963 99864 50413.5 5041350 -32605 32330 4538.3 453830 -125 126 1.98 198 -964 2 10954 99865 2.89489 299.89489 151.39489 15139.48948 2.89489 299.8949 151.39489 15139.48968 2.89489 299.89489 151.39488999999978 15139.48900 2020-01-01 2020-01-02 2020-01-01 00:16:04 2020-01-02 03:44:25 2020-01-01 00:16:04.000 2020-01-02 03:44:25.000 964 99865 50414.5 5041450 964 99865 50414.5 5041450 -32604 32331 4539.3 453930 -124 127 2.98 298 -965 2 10955 99866 2.89789 299.89789 151.39789 15139.78978 2.89789 299.8979 151.39789 15139.78985 2.89789 299.89789 151.39789000000013 15139.78900 2020-01-01 2020-01-02 2020-01-01 00:16:05 2020-01-02 03:44:26 2020-01-01 00:16:05.000 2020-01-02 03:44:26.000 965 99866 50415.5 5041550 965 99866 50415.5 5041550 -32603 32332 4540.3 454030 -128 127 1.42 142 +942 2 10932 99843 2.82882 299.82882 151.32882 15132.88288 2.82882 299.82883 151.32882 15132.88275 2.82882 299.82882 151.32882 15132.88200 2020-01-01 2020-01-02 2020-01-01 00:15:42 2020-01-02 03:44:03 2020-01-01 00:15:42.000 2020-01-02 03:44:03.000 942 99843 50392.5 5039250 942 99843 50392.5 5039250 -32626 32309 4517.3 451730 -128 124 -1.1 -110 +943 2 10933 99844 2.83183 299.83183 151.33183 15133.18318 2.83183 299.83182 151.33183 15133.18304 2.83183 299.83183 151.33183 15133.18300 2020-01-01 2020-01-02 2020-01-01 00:15:43 2020-01-02 03:44:04 2020-01-01 00:15:43.000 2020-01-02 03:44:04.000 943 99844 50393.5 5039350 943 99844 50393.5 5039350 -32625 32310 4518.3 451830 -127 125 -0.1 -10 +944 2 10934 99845 2.83483 299.83483 151.33483 15133.48348 2.83483 299.83484 151.33483 15133.48364 2.83483 299.83483 151.33483 15133.48300 2020-01-01 2020-01-02 2020-01-01 00:15:44 2020-01-02 03:44:05 2020-01-01 00:15:44.000 2020-01-02 03:44:05.000 944 99845 50394.5 5039450 944 99845 50394.5 5039450 -32624 32311 4519.3 451930 -126 126 0.9 90 +945 2 10935 99846 2.83783 299.83783 151.33783 15133.78378 2.83783 299.83783 151.33783 15133.78393 2.83783 299.83783 151.33783 15133.78300 2020-01-01 2020-01-02 2020-01-01 00:15:45 2020-01-02 03:44:06 2020-01-01 00:15:45.000 2020-01-02 03:44:06.000 945 99846 50395.5 5039550 945 99846 50395.5 5039550 -32623 32312 4520.3 452030 -125 127 1.9 190 +946 2 10936 99847 2.84084 299.84084 151.34084 15134.08408 2.84084 299.84085 151.34084 15134.08404 2.84084 299.84084 151.34084000000001 15134.08400 2020-01-01 2020-01-02 2020-01-01 00:15:46 2020-01-02 03:44:07 2020-01-01 00:15:46.000 2020-01-02 03:44:07.000 946 99847 50396.5 5039650 946 99847 50396.5 5039650 -32622 32313 4521.3 452130 -128 127 0.34 34 +947 2 10937 99848 2.84384 299.84384 151.34384 15134.38438 2.84384 299.84384 151.34384 15134.38421 2.84384 299.84384 151.34384 15134.38400 2020-01-01 2020-01-02 2020-01-01 00:15:47 2020-01-02 03:44:08 2020-01-01 00:15:47.000 2020-01-02 03:44:08.000 947 99848 50397.5 5039750 947 99848 50397.5 5039750 -32621 32314 4522.3 452230 -128 127 -1.22 -122 +948 2 10938 99849 2.84684 299.84684 151.34684 15134.68468 2.84684 299.84683 151.34684 15134.68452 2.84684 299.84684 151.34684 15134.68400 2020-01-01 2020-01-02 2020-01-01 00:15:48 2020-01-02 03:44:09 2020-01-01 00:15:48.000 2020-01-02 03:44:09.000 948 99849 50398.5 5039850 948 99849 50398.5 5039850 -32620 32315 4523.3 452330 -128 123 -2.78 -278 +949 2 10939 99850 2.84984 299.84984 151.34984 15134.98498 2.84985 299.84985 151.34985 15134.98527 2.84984 299.84984 151.34984 15134.98400 2020-01-01 2020-01-02 2020-01-01 00:15:49 2020-01-02 03:44:10 2020-01-01 00:15:49.000 2020-01-02 03:44:10.000 949 99850 50399.5 5039950 949 99850 50399.5 5039950 -32619 32316 4524.3 452430 -127 124 -1.78 -178 +95 2 10085 99995 0.28528 300.28528 150.28528 15178.81381 0.28528 300.28528 150.28528 15178.81343 0.28528 300.28528 150.28528 15178.81328 2020-01-01 2020-01-02 2020-01-01 00:01:35 2020-01-02 03:46:35 2020-01-01 00:01:35.000 2020-01-02 03:46:35.000 95 99995 50045 5054545 95 99995 50045 5054545 -32474 32461 4624.009900990099 467025 -125 126 0.801980198019802 81 +950 2 10940 99851 2.85285 299.85285 151.35285 15135.28528 2.85285 299.85284 151.35285 15135.28541 2.85285 299.85285 151.35285 15135.28500 2020-01-01 2020-01-02 2020-01-01 00:15:50 2020-01-02 03:44:11 2020-01-01 00:15:50.000 2020-01-02 03:44:11.000 950 99851 50400.5 5040050 950 99851 50400.5 5040050 -32618 32317 4525.3 452530 -126 125 -0.78 -78 +951 2 10941 99852 2.85585 299.85585 151.35585 15135.58558 2.85585 299.85587 151.35585 15135.58551 2.85585 299.85585 151.35585 15135.58500 2020-01-01 2020-01-02 2020-01-01 00:15:51 2020-01-02 03:44:12 2020-01-01 00:15:51.000 2020-01-02 03:44:12.000 951 99852 50401.5 5040150 951 99852 50401.5 5040150 -32617 32318 4526.3 452630 -125 126 0.22 22 +952 2 10942 99853 2.85885 299.85885 151.35885 15135.88588 2.85885 299.85886 151.35885 15135.88568 2.85885 299.85885 151.35885 15135.88500 2020-01-01 2020-01-02 2020-01-01 00:15:52 2020-01-02 03:44:13 2020-01-01 00:15:52.000 2020-01-02 03:44:13.000 952 99853 50402.5 5040250 952 99853 50402.5 5040250 -32616 32319 4527.3 452730 -124 127 1.22 122 +953 2 10943 99854 2.86186 299.86186 151.36186 15136.18618 2.86186 299.86185 151.36185 15136.18598 2.86186 299.86186 151.36186 15136.18600 2020-01-01 2020-01-02 2020-01-01 00:15:53 2020-01-02 03:44:14 2020-01-01 00:15:53.000 2020-01-02 03:44:14.000 953 99854 50403.5 5040350 953 99854 50403.5 5040350 -32615 32320 4528.3 452830 -128 127 -0.34 -34 +954 2 10944 99855 2.86486 299.86486 151.36486 15136.48648 2.86486 299.86487 151.36486 15136.48674 2.86486 299.86486 151.36486000000002 15136.48600 2020-01-01 2020-01-02 2020-01-01 00:15:54 2020-01-02 03:44:15 2020-01-01 00:15:54.000 2020-01-02 03:44:15.000 954 99855 50404.5 5040450 954 99855 50404.5 5040450 -32614 32321 4529.3 452930 -128 123 -1.9 -190 +955 2 10945 99856 2.86786 299.86786 151.36786 15136.78678 2.86786 299.86786 151.36786 15136.78688 2.86786 299.86786 151.36786 15136.78600 2020-01-01 2020-01-02 2020-01-01 00:15:55 2020-01-02 03:44:16 2020-01-01 00:15:55.000 2020-01-02 03:44:16.000 955 99856 50405.5 5040550 955 99856 50405.5 5040550 -32613 32322 4530.3 453030 -127 124 -0.9 -90 +956 2 10946 99857 2.87087 299.87087 151.37087 15137.08708 2.87087 299.87088 151.37087 15137.08701 2.87087 299.87087 151.37087 15137.08700 2020-01-01 2020-01-02 2020-01-01 00:15:56 2020-01-02 03:44:17 2020-01-01 00:15:56.000 2020-01-02 03:44:17.000 956 99857 50406.5 5040650 956 99857 50406.5 5040650 -32612 32323 4531.3 453130 -126 125 0.1 10 +957 2 10947 99858 2.87387 299.87387 151.37387 15137.38738 2.87387 299.87387 151.37387 15137.38716 2.87387 299.87387 151.37387 15137.38700 2020-01-01 2020-01-02 2020-01-01 00:15:57 2020-01-02 03:44:18 2020-01-01 00:15:57.000 2020-01-02 03:44:18.000 957 99858 50407.5 5040750 957 99858 50407.5 5040750 -32611 32324 4532.3 453230 -125 126 1.1 110 +958 2 10948 99859 2.87687 299.87687 151.37687 15137.68768 2.87687 299.8769 151.37687 15137.68791 2.87687 299.87687 151.37687 15137.68700 2020-01-01 2020-01-02 2020-01-01 00:15:58 2020-01-02 03:44:19 2020-01-01 00:15:58.000 2020-01-02 03:44:19.000 958 99859 50408.5 5040850 958 99859 50408.5 5040850 -32610 32325 4533.3 453330 -124 127 2.1 210 +959 2 10949 99860 2.87987 299.87987 151.37987 15137.98798 2.87988 299.87988 151.37988 15137.9882 2.87987 299.87987 151.37986999999998 15137.98700 2020-01-01 2020-01-02 2020-01-01 00:15:59 2020-01-02 03:44:20 2020-01-01 00:15:59.000 2020-01-02 03:44:20.000 959 99860 50409.5 5040950 959 99860 50409.5 5040950 -32609 32326 4534.3 453430 -128 127 0.54 54 +96 2 10086 99996 0.28828 300.28828 150.28828 15179.11711 0.28828 300.2883 150.28828 15179.11718 0.28828 300.28828 150.28828000000001 15179.11628 2020-01-01 2020-01-02 2020-01-01 00:01:36 2020-01-02 03:46:36 2020-01-01 00:01:36.000 2020-01-02 03:46:36.000 96 99996 50046 5054646 96 99996 50046 5054646 -32473 32462 4625.009900990099 467126 -124 127 1.801980198019802 182 +960 2 10950 99861 2.88288 299.88288 151.38288 15138.28828 2.88288 299.88287 151.38288 15138.28834 2.88288 299.88288 151.38288 15138.28800 2020-01-01 2020-01-02 2020-01-01 00:16:00 2020-01-02 03:44:21 2020-01-01 00:16:00.000 2020-01-02 03:44:21.000 960 99861 50410.5 5041050 960 99861 50410.5 5041050 -32608 32327 4535.3 453530 -128 123 -1.02 -102 +961 2 10951 99862 2.88588 299.88588 151.38588 15138.58858 2.88588 299.8859 151.38588 15138.58848 2.88588 299.88588 151.38588 15138.58800 2020-01-01 2020-01-02 2020-01-01 00:16:01 2020-01-02 03:44:22 2020-01-01 00:16:01.000 2020-01-02 03:44:22.000 961 99862 50411.5 5041150 961 99862 50411.5 5041150 -32607 32328 4536.3 453630 -127 124 -0.02 -2 +962 2 10952 99863 2.88888 299.88888 151.38888 15138.88888 2.88888 299.8889 151.38888 15138.88862 2.88888 299.88888 151.38888 15138.88800 2020-01-01 2020-01-02 2020-01-01 00:16:02 2020-01-02 03:44:23 2020-01-01 00:16:02.000 2020-01-02 03:44:23.000 962 99863 50412.5 5041250 962 99863 50412.5 5041250 -32606 32329 4537.3 453730 -126 125 0.98 98 +963 2 10953 99864 2.89189 299.89189 151.39189 15139.18918 2.89189 299.8919 151.39189 15139.18937 2.89189 299.89189 151.39189 15139.18900 2020-01-01 2020-01-02 2020-01-01 00:16:03 2020-01-02 03:44:24 2020-01-01 00:16:03.000 2020-01-02 03:44:24.000 963 99864 50413.5 5041350 963 99864 50413.5 5041350 -32605 32330 4538.3 453830 -125 126 1.98 198 +964 2 10954 99865 2.89489 299.89489 151.39489 15139.48948 2.89489 299.8949 151.39489 15139.48968 2.89489 299.89489 151.39489 15139.48900 2020-01-01 2020-01-02 2020-01-01 00:16:04 2020-01-02 03:44:25 2020-01-01 00:16:04.000 2020-01-02 03:44:25.000 964 99865 50414.5 5041450 964 99865 50414.5 5041450 -32604 32331 4539.3 453930 -124 127 2.98 298 +965 2 10955 99866 2.89789 299.89789 151.39789 15139.78978 2.89789 299.8979 151.39789 15139.78985 2.89789 299.89789 151.39789000000002 15139.78900 2020-01-01 2020-01-02 2020-01-01 00:16:05 2020-01-02 03:44:26 2020-01-01 00:16:05.000 2020-01-02 03:44:26.000 965 99866 50415.5 5041550 965 99866 50415.5 5041550 -32603 32332 4540.3 454030 -128 127 1.42 142 966 2 10956 99867 2.9009 299.9009 151.4009 15140.09009 2.9009 299.9009 151.40089 15140.08996 2.90090 299.90090 151.4009 15140.09000 2020-01-01 2020-01-02 2020-01-01 00:16:06 2020-01-02 03:44:27 2020-01-01 00:16:06.000 2020-01-02 03:44:27.000 966 99867 50416.5 5041650 966 99867 50416.5 5041650 -32602 32333 4541.3 454130 -128 127 -0.14 -14 -967 2 10957 99868 2.9039 299.9039 151.4039 15140.39039 2.9039 299.9039 151.4039 15140.39009 2.90390 299.90390 151.4038999999999 15140.39000 2020-01-01 2020-01-02 2020-01-01 00:16:07 2020-01-02 03:44:28 2020-01-01 00:16:07.000 2020-01-02 03:44:28.000 967 99868 50417.5 5041750 967 99868 50417.5 5041750 -32601 32334 4542.3 454230 -128 124 -1.7 -170 -968 2 10958 99869 2.9069 299.9069 151.4069 15140.69069 2.9069 299.90692 151.4069 15140.69084 2.90690 299.90690 151.40689999999998 15140.69000 2020-01-01 2020-01-02 2020-01-01 00:16:08 2020-01-02 03:44:29 2020-01-01 00:16:08.000 2020-01-02 03:44:29.000 968 99869 50418.5 5041850 968 99869 50418.5 5041850 -32600 32335 4543.3 454330 -127 125 -0.7 -70 -969 2 10959 99870 2.9099 299.9099 151.4099 15140.99099 2.90991 299.9099 151.40991 15140.99114 2.90990 299.90990 151.40990000000002 15140.99000 2020-01-01 2020-01-02 2020-01-01 00:16:09 2020-01-02 03:44:30 2020-01-01 00:16:09.000 2020-01-02 03:44:30.000 969 99870 50419.5 5041950 969 99870 50419.5 5041950 -32599 32336 4544.3 454430 -126 126 0.3 30 -97 2 10087 99997 0.29129 300.29129 150.29129 15179.42042 0.29129 300.2913 150.29129 15179.42033 0.29129 300.29129 150.29128999999986 15179.42029 2020-01-01 2020-01-02 2020-01-01 00:01:37 2020-01-02 03:46:37 2020-01-01 00:01:37.000 2020-01-02 03:46:37.000 97 99997 50047 5054747 97 99997 50047 5054747 -32472 32463 4626.009900990099 467227 -128 127 0.26732673267326734 27 -970 2 10960 99871 2.91291 299.91291 151.41291 15141.29129 2.91291 299.9129 151.41291 15141.29132 2.91291 299.91291 151.41290999999984 15141.29100 2020-01-01 2020-01-02 2020-01-01 00:16:10 2020-01-02 03:44:31 2020-01-01 00:16:10.000 2020-01-02 03:44:31.000 970 99871 50420.5 5042050 970 99871 50420.5 5042050 -32598 32337 4545.3 454530 -125 127 1.3 130 -971 2 10961 99872 2.91591 299.91591 151.41591 15141.59159 2.91591 299.91592 151.41591 15141.59142 2.91591 299.91591 151.41590999999988 15141.59100 2020-01-01 2020-01-02 2020-01-01 00:16:11 2020-01-02 03:44:32 2020-01-01 00:16:11.000 2020-01-02 03:44:32.000 971 99872 50421.5 5042150 971 99872 50421.5 5042150 -32597 32338 4546.3 454630 -128 127 -0.26 -26 -972 2 10962 99873 2.91891 299.91891 151.41891 15141.89189 2.91891 299.9189 151.41891 15141.89172 2.91891 299.91891 151.41891000000015 15141.89100 2020-01-01 2020-01-02 2020-01-01 00:16:12 2020-01-02 03:44:33 2020-01-01 00:16:12.000 2020-01-02 03:44:33.000 972 99873 50422.5 5042250 972 99873 50422.5 5042250 -32596 32339 4547.3 454730 -128 127 -1.82 -182 -973 2 10963 99874 2.92192 299.92192 151.42192 15142.19219 2.92192 299.92194 151.42192 15142.19232 2.92192 299.92192 151.42192000000006 15142.19200 2020-01-01 2020-01-02 2020-01-01 00:16:13 2020-01-02 03:44:34 2020-01-01 00:16:13.000 2020-01-02 03:44:34.000 973 99874 50423.5 5042350 973 99874 50423.5 5042350 -32595 32340 4548.3 454830 -128 123 -3.38 -338 -974 2 10964 99875 2.92492 299.92492 151.42492 15142.49249 2.92492 299.92493 151.42492 15142.49265 2.92492 299.92492 151.42492 15142.49200 2020-01-01 2020-01-02 2020-01-01 00:16:14 2020-01-02 03:44:35 2020-01-01 00:16:14.000 2020-01-02 03:44:35.000 974 99875 50424.5 5042450 974 99875 50424.5 5042450 -32594 32341 4549.3 454930 -127 124 -2.38 -238 -975 2 10965 99876 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792000000006 15142.79200 2020-01-01 2020-01-02 2020-01-01 00:16:15 2020-01-02 03:44:36 2020-01-01 00:16:15.000 2020-01-02 03:44:36.000 975 99876 50425.5 5042550 975 99876 50425.5 5042550 -32593 32342 4550.3 455030 -126 125 -1.38 -138 -976 2 10966 99877 2.93093 299.93093 151.43093 15143.09309 2.93093 299.93094 151.43092 15143.09289 2.93093 299.93093 151.43093000000022 15143.09300 2020-01-01 2020-01-02 2020-01-01 00:16:16 2020-01-02 03:44:37 2020-01-01 00:16:16.000 2020-01-02 03:44:37.000 976 99877 50426.5 5042650 976 99877 50426.5 5042650 -32592 32343 4551.3 455130 -125 126 -0.38 -38 -977 2 10967 99878 2.93393 299.93393 151.43393 15143.39339 2.93393 299.93393 151.43393 15143.39318 2.93393 299.93393 151.4339299999999 15143.39300 2020-01-01 2020-01-02 2020-01-01 00:16:17 2020-01-02 03:44:38 2020-01-01 00:16:17.000 2020-01-02 03:44:38.000 977 99878 50427.5 5042750 977 99878 50427.5 5042750 -32591 32344 4552.3 455230 -124 127 0.62 62 -978 2 10968 99879 2.93693 299.93693 151.43693 15143.69369 2.93693 299.93695 151.43693 15143.69378 2.93693 299.93693 151.4369299999999 15143.69300 2020-01-01 2020-01-02 2020-01-01 00:16:18 2020-01-02 03:44:39 2020-01-01 00:16:18.000 2020-01-02 03:44:39.000 978 99879 50428.5 5042850 978 99879 50428.5 5042850 -32590 32345 4553.3 455330 -128 127 -0.94 -94 -979 2 10969 99880 2.93993 299.93993 151.43993 15143.99399 2.93994 299.93994 151.43994 15143.99412 2.93993 299.93993 151.43993000000017 15143.99300 2020-01-01 2020-01-02 2020-01-01 00:16:19 2020-01-02 03:44:40 2020-01-01 00:16:19.000 2020-01-02 03:44:40.000 979 99880 50429.5 5042950 979 99880 50429.5 5042950 -32589 32346 4554.3 455430 -128 123 -2.5 -250 -98 2 10088 99998 0.29429 300.29429 150.29429 15179.72372 0.29429 300.29428 150.29429 15179.72363 0.29429 300.29429 150.29428999999988 15179.72329 2020-01-01 2020-01-02 2020-01-01 00:01:38 2020-01-02 03:46:38 2020-01-01 00:01:38.000 2020-01-02 03:46:38.000 98 99998 50048 5054848 98 99998 50048 5054848 -32471 32464 4627.009900990099 467328 -128 127 -1.2673267326732673 -128 -980 2 10970 99881 2.94294 299.94294 151.44294 15144.29429 2.94294 299.94293 151.44294 15144.29426 2.94294 299.94294 151.44294000000014 15144.29400 2020-01-01 2020-01-02 2020-01-01 00:16:20 2020-01-02 03:44:41 2020-01-01 00:16:20.000 2020-01-02 03:44:41.000 980 99881 50430.5 5043050 980 99881 50430.5 5043050 -32588 32347 4555.3 455530 -127 124 -1.5 -150 -981 2 10971 99882 2.94594 299.94594 151.44594 15144.59459 2.94594 299.94595 151.44595 15144.59501 2.94594 299.94594 151.4459399999998 15144.59400 2020-01-01 2020-01-02 2020-01-01 00:16:21 2020-01-02 03:44:42 2020-01-01 00:16:21.000 2020-01-02 03:44:42.000 981 99882 50431.5 5043150 981 99882 50431.5 5043150 -32587 32348 4556.3 455630 -126 125 -0.5 -50 -982 2 10972 99883 2.94894 299.94894 151.44894 15144.89489 2.94894 299.94894 151.44894 15144.89466 2.94894 299.94894 151.44894000000014 15144.89400 2020-01-01 2020-01-02 2020-01-01 00:16:22 2020-01-02 03:44:43 2020-01-01 00:16:22.000 2020-01-02 03:44:43.000 982 99883 50432.5 5043250 982 99883 50432.5 5043250 -32586 32349 4557.3 455730 -125 126 0.5 50 -983 2 10973 99884 2.95195 299.95195 151.45195 15145.19519 2.95195 299.95197 151.45195 15145.19525 2.95195 299.95195 151.45195000000004 15145.19500 2020-01-01 2020-01-02 2020-01-01 00:16:23 2020-01-02 03:44:44 2020-01-01 00:16:23.000 2020-01-02 03:44:44.000 983 99884 50433.5 5043350 983 99884 50433.5 5043350 -32585 32350 4558.3 455830 -124 127 1.5 150 -984 2 10974 99885 2.95495 299.95495 151.45495 15145.49549 2.95495 299.95496 151.45495 15145.49559 2.95495 299.95495 151.4549499999999 15145.49500 2020-01-01 2020-01-02 2020-01-01 00:16:24 2020-01-02 03:44:45 2020-01-01 00:16:24.000 2020-01-02 03:44:45.000 984 99885 50434.5 5043450 984 99885 50434.5 5043450 -32584 32351 4559.3 455930 -128 127 -0.06 -6 -985 2 10975 99886 2.95795 299.95795 151.45795 15145.79579 2.95795 299.95795 151.45795 15145.79573 2.95795 299.95795 151.45794999999998 15145.79500 2020-01-01 2020-01-02 2020-01-01 00:16:25 2020-01-02 03:44:46 2020-01-01 00:16:25.000 2020-01-02 03:44:46.000 985 99886 50435.5 5043550 985 99886 50435.5 5043550 -32583 32352 4560.3 456030 -128 123 -1.62 -162 -986 2 10976 99887 2.96096 299.96096 151.46096 15146.09609 2.96096 299.96097 151.46096 15146.09648 2.96096 299.96096 151.46096000000014 15146.09600 2020-01-01 2020-01-02 2020-01-01 00:16:26 2020-01-02 03:44:47 2020-01-01 00:16:26.000 2020-01-02 03:44:47.000 986 99887 50436.5 5043650 986 99887 50436.5 5043650 -32582 32353 4561.3 456130 -127 124 -0.62 -62 -987 2 10977 99888 2.96396 299.96396 151.46396 15146.39639 2.96396 299.96396 151.46396 15146.39612 2.96396 299.96396 151.46395999999982 15146.39600 2020-01-01 2020-01-02 2020-01-01 00:16:27 2020-01-02 03:44:48 2020-01-01 00:16:27.000 2020-01-02 03:44:48.000 987 99888 50437.5 5043750 987 99888 50437.5 5043750 -32581 32354 4562.3 456230 -126 125 0.38 38 -988 2 10978 99889 2.96696 299.96696 151.46696 15146.69669 2.96696 299.96698 151.46696 15146.69676 2.96696 299.96696 151.46695999999986 15146.69600 2020-01-01 2020-01-02 2020-01-01 00:16:28 2020-01-02 03:44:49 2020-01-01 00:16:28.000 2020-01-02 03:44:49.000 988 99889 50438.5 5043850 988 99889 50438.5 5043850 -32580 32355 4563.3 456330 -125 126 1.38 138 -989 2 10979 99890 2.96996 299.96996 151.46996 15146.99699 2.96997 299.96997 151.46997 15146.99706 2.96996 299.96996 151.46996000000016 15146.99600 2020-01-01 2020-01-02 2020-01-01 00:16:29 2020-01-02 03:44:50 2020-01-01 00:16:29.000 2020-01-02 03:44:50.000 989 99890 50439.5 5043950 989 99890 50439.5 5043950 -32579 32356 4564.3 456430 -124 127 2.38 238 -99 2 10089 99999 0.29729 300.29729 150.29729 15180.02702 0.29729 300.2973 150.29729 15180.02726 0.29729 300.29729 150.29729000000023 15180.02629 2020-01-01 2020-01-02 2020-01-01 00:01:39 2020-01-02 03:46:39 2020-01-01 00:01:39.000 2020-01-02 03:46:39.000 99 99999 50049 5054949 99 99999 50049 5054949 -32470 32465 4628.009900990099 467429 -128 123 -2.801980198019802 -283 -990 2 10980 99891 2.97297 299.97297 151.47297 15147.29729 2.97297 299.97296 151.47297 15147.29735 2.97297 299.97297 151.4729700000001 15147.29700 2020-01-01 2020-01-02 2020-01-01 00:16:30 2020-01-02 03:44:51 2020-01-01 00:16:30.000 2020-01-02 03:44:51.000 990 99891 50440.5 5044050 990 99891 50440.5 5044050 -32578 32357 4565.3 456530 -128 127 0.82 82 +967 2 10957 99868 2.9039 299.9039 151.4039 15140.39039 2.9039 299.9039 151.4039 15140.39009 2.90390 299.90390 151.4039 15140.39000 2020-01-01 2020-01-02 2020-01-01 00:16:07 2020-01-02 03:44:28 2020-01-01 00:16:07.000 2020-01-02 03:44:28.000 967 99868 50417.5 5041750 967 99868 50417.5 5041750 -32601 32334 4542.3 454230 -128 124 -1.7 -170 +968 2 10958 99869 2.9069 299.9069 151.4069 15140.69069 2.9069 299.90692 151.4069 15140.69084 2.90690 299.90690 151.4069 15140.69000 2020-01-01 2020-01-02 2020-01-01 00:16:08 2020-01-02 03:44:29 2020-01-01 00:16:08.000 2020-01-02 03:44:29.000 968 99869 50418.5 5041850 968 99869 50418.5 5041850 -32600 32335 4543.3 454330 -127 125 -0.7 -70 +969 2 10959 99870 2.9099 299.9099 151.4099 15140.99099 2.90991 299.9099 151.40991 15140.99114 2.90990 299.90990 151.4099 15140.99000 2020-01-01 2020-01-02 2020-01-01 00:16:09 2020-01-02 03:44:30 2020-01-01 00:16:09.000 2020-01-02 03:44:30.000 969 99870 50419.5 5041950 969 99870 50419.5 5041950 -32599 32336 4544.3 454430 -126 126 0.3 30 +97 2 10087 99997 0.29129 300.29129 150.29129 15179.42042 0.29129 300.2913 150.29129 15179.42033 0.29129 300.29129 150.29129 15179.42029 2020-01-01 2020-01-02 2020-01-01 00:01:37 2020-01-02 03:46:37 2020-01-01 00:01:37.000 2020-01-02 03:46:37.000 97 99997 50047 5054747 97 99997 50047 5054747 -32472 32463 4626.009900990099 467227 -128 127 0.26732673267326734 27 +970 2 10960 99871 2.91291 299.91291 151.41291 15141.29129 2.91291 299.9129 151.41291 15141.29132 2.91291 299.91291 151.41290999999998 15141.29100 2020-01-01 2020-01-02 2020-01-01 00:16:10 2020-01-02 03:44:31 2020-01-01 00:16:10.000 2020-01-02 03:44:31.000 970 99871 50420.5 5042050 970 99871 50420.5 5042050 -32598 32337 4545.3 454530 -125 127 1.3 130 +971 2 10961 99872 2.91591 299.91591 151.41591 15141.59159 2.91591 299.91592 151.41591 15141.59142 2.91591 299.91591 151.41591 15141.59100 2020-01-01 2020-01-02 2020-01-01 00:16:11 2020-01-02 03:44:32 2020-01-01 00:16:11.000 2020-01-02 03:44:32.000 971 99872 50421.5 5042150 971 99872 50421.5 5042150 -32597 32338 4546.3 454630 -128 127 -0.26 -26 +972 2 10962 99873 2.91891 299.91891 151.41891 15141.89189 2.91891 299.9189 151.41891 15141.89172 2.91891 299.91891 151.41890999999998 15141.89100 2020-01-01 2020-01-02 2020-01-01 00:16:12 2020-01-02 03:44:33 2020-01-01 00:16:12.000 2020-01-02 03:44:33.000 972 99873 50422.5 5042250 972 99873 50422.5 5042250 -32596 32339 4547.3 454730 -128 127 -1.82 -182 +973 2 10963 99874 2.92192 299.92192 151.42192 15142.19219 2.92192 299.92194 151.42192 15142.19232 2.92192 299.92192 151.42192 15142.19200 2020-01-01 2020-01-02 2020-01-01 00:16:13 2020-01-02 03:44:34 2020-01-01 00:16:13.000 2020-01-02 03:44:34.000 973 99874 50423.5 5042350 973 99874 50423.5 5042350 -32595 32340 4548.3 454830 -128 123 -3.38 -338 +974 2 10964 99875 2.92492 299.92492 151.42492 15142.49249 2.92492 299.92493 151.42492 15142.49265 2.92492 299.92492 151.42492000000001 15142.49200 2020-01-01 2020-01-02 2020-01-01 00:16:14 2020-01-02 03:44:35 2020-01-01 00:16:14.000 2020-01-02 03:44:35.000 974 99875 50424.5 5042450 974 99875 50424.5 5042450 -32594 32341 4549.3 454930 -127 124 -2.38 -238 +975 2 10965 99876 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792 15142.79279 2.92792 299.92792 151.42792 15142.79200 2020-01-01 2020-01-02 2020-01-01 00:16:15 2020-01-02 03:44:36 2020-01-01 00:16:15.000 2020-01-02 03:44:36.000 975 99876 50425.5 5042550 975 99876 50425.5 5042550 -32593 32342 4550.3 455030 -126 125 -1.38 -138 +976 2 10966 99877 2.93093 299.93093 151.43093 15143.09309 2.93093 299.93094 151.43092 15143.09289 2.93093 299.93093 151.43093000000002 15143.09300 2020-01-01 2020-01-02 2020-01-01 00:16:16 2020-01-02 03:44:37 2020-01-01 00:16:16.000 2020-01-02 03:44:37.000 976 99877 50426.5 5042650 976 99877 50426.5 5042650 -32592 32343 4551.3 455130 -125 126 -0.38 -38 +977 2 10967 99878 2.93393 299.93393 151.43393 15143.39339 2.93393 299.93393 151.43393 15143.39318 2.93393 299.93393 151.43393 15143.39300 2020-01-01 2020-01-02 2020-01-01 00:16:17 2020-01-02 03:44:38 2020-01-01 00:16:17.000 2020-01-02 03:44:38.000 977 99878 50427.5 5042750 977 99878 50427.5 5042750 -32591 32344 4552.3 455230 -124 127 0.62 62 +978 2 10968 99879 2.93693 299.93693 151.43693 15143.69369 2.93693 299.93695 151.43693 15143.69378 2.93693 299.93693 151.43693 15143.69300 2020-01-01 2020-01-02 2020-01-01 00:16:18 2020-01-02 03:44:39 2020-01-01 00:16:18.000 2020-01-02 03:44:39.000 978 99879 50428.5 5042850 978 99879 50428.5 5042850 -32590 32345 4553.3 455330 -128 127 -0.94 -94 +979 2 10969 99880 2.93993 299.93993 151.43993 15143.99399 2.93994 299.93994 151.43994 15143.99412 2.93993 299.93993 151.43993 15143.99300 2020-01-01 2020-01-02 2020-01-01 00:16:19 2020-01-02 03:44:40 2020-01-01 00:16:19.000 2020-01-02 03:44:40.000 979 99880 50429.5 5042950 979 99880 50429.5 5042950 -32589 32346 4554.3 455430 -128 123 -2.5 -250 +98 2 10088 99998 0.29429 300.29429 150.29429 15179.72372 0.29429 300.29428 150.29429 15179.72363 0.29429 300.29429 150.29429 15179.72329 2020-01-01 2020-01-02 2020-01-01 00:01:38 2020-01-02 03:46:38 2020-01-01 00:01:38.000 2020-01-02 03:46:38.000 98 99998 50048 5054848 98 99998 50048 5054848 -32471 32464 4627.009900990099 467328 -128 127 -1.2673267326732673 -128 +980 2 10970 99881 2.94294 299.94294 151.44294 15144.29429 2.94294 299.94293 151.44294 15144.29426 2.94294 299.94294 151.44294 15144.29400 2020-01-01 2020-01-02 2020-01-01 00:16:20 2020-01-02 03:44:41 2020-01-01 00:16:20.000 2020-01-02 03:44:41.000 980 99881 50430.5 5043050 980 99881 50430.5 5043050 -32588 32347 4555.3 455530 -127 124 -1.5 -150 +981 2 10971 99882 2.94594 299.94594 151.44594 15144.59459 2.94594 299.94595 151.44595 15144.59501 2.94594 299.94594 151.44593999999998 15144.59400 2020-01-01 2020-01-02 2020-01-01 00:16:21 2020-01-02 03:44:42 2020-01-01 00:16:21.000 2020-01-02 03:44:42.000 981 99882 50431.5 5043150 981 99882 50431.5 5043150 -32587 32348 4556.3 455630 -126 125 -0.5 -50 +982 2 10972 99883 2.94894 299.94894 151.44894 15144.89489 2.94894 299.94894 151.44894 15144.89466 2.94894 299.94894 151.44894 15144.89400 2020-01-01 2020-01-02 2020-01-01 00:16:22 2020-01-02 03:44:43 2020-01-01 00:16:22.000 2020-01-02 03:44:43.000 982 99883 50432.5 5043250 982 99883 50432.5 5043250 -32586 32349 4557.3 455730 -125 126 0.5 50 +983 2 10973 99884 2.95195 299.95195 151.45195 15145.19519 2.95195 299.95197 151.45195 15145.19525 2.95195 299.95195 151.45195 15145.19500 2020-01-01 2020-01-02 2020-01-01 00:16:23 2020-01-02 03:44:44 2020-01-01 00:16:23.000 2020-01-02 03:44:44.000 983 99884 50433.5 5043350 983 99884 50433.5 5043350 -32585 32350 4558.3 455830 -124 127 1.5 150 +984 2 10974 99885 2.95495 299.95495 151.45495 15145.49549 2.95495 299.95496 151.45495 15145.49559 2.95495 299.95495 151.45495 15145.49500 2020-01-01 2020-01-02 2020-01-01 00:16:24 2020-01-02 03:44:45 2020-01-01 00:16:24.000 2020-01-02 03:44:45.000 984 99885 50434.5 5043450 984 99885 50434.5 5043450 -32584 32351 4559.3 455930 -128 127 -0.06 -6 +985 2 10975 99886 2.95795 299.95795 151.45795 15145.79579 2.95795 299.95795 151.45795 15145.79573 2.95795 299.95795 151.45795 15145.79500 2020-01-01 2020-01-02 2020-01-01 00:16:25 2020-01-02 03:44:46 2020-01-01 00:16:25.000 2020-01-02 03:44:46.000 985 99886 50435.5 5043550 985 99886 50435.5 5043550 -32583 32352 4560.3 456030 -128 123 -1.62 -162 +986 2 10976 99887 2.96096 299.96096 151.46096 15146.09609 2.96096 299.96097 151.46096 15146.09648 2.96096 299.96096 151.46096 15146.09600 2020-01-01 2020-01-02 2020-01-01 00:16:26 2020-01-02 03:44:47 2020-01-01 00:16:26.000 2020-01-02 03:44:47.000 986 99887 50436.5 5043650 986 99887 50436.5 5043650 -32582 32353 4561.3 456130 -127 124 -0.62 -62 +987 2 10977 99888 2.96396 299.96396 151.46396 15146.39639 2.96396 299.96396 151.46396 15146.39612 2.96396 299.96396 151.46396000000001 15146.39600 2020-01-01 2020-01-02 2020-01-01 00:16:27 2020-01-02 03:44:48 2020-01-01 00:16:27.000 2020-01-02 03:44:48.000 987 99888 50437.5 5043750 987 99888 50437.5 5043750 -32581 32354 4562.3 456230 -126 125 0.38 38 +988 2 10978 99889 2.96696 299.96696 151.46696 15146.69669 2.96696 299.96698 151.46696 15146.69676 2.96696 299.96696 151.46696 15146.69600 2020-01-01 2020-01-02 2020-01-01 00:16:28 2020-01-02 03:44:49 2020-01-01 00:16:28.000 2020-01-02 03:44:49.000 988 99889 50438.5 5043850 988 99889 50438.5 5043850 -32580 32355 4563.3 456330 -125 126 1.38 138 +989 2 10979 99890 2.96996 299.96996 151.46996 15146.99699 2.96997 299.96997 151.46997 15146.99706 2.96996 299.96996 151.46996 15146.99600 2020-01-01 2020-01-02 2020-01-01 00:16:29 2020-01-02 03:44:50 2020-01-01 00:16:29.000 2020-01-02 03:44:50.000 989 99890 50439.5 5043950 989 99890 50439.5 5043950 -32579 32356 4564.3 456430 -124 127 2.38 238 +99 2 10089 99999 0.29729 300.29729 150.29729 15180.02702 0.29729 300.2973 150.29729 15180.02726 0.29729 300.29729 150.29729 15180.02629 2020-01-01 2020-01-02 2020-01-01 00:01:39 2020-01-02 03:46:39 2020-01-01 00:01:39.000 2020-01-02 03:46:39.000 99 99999 50049 5054949 99 99999 50049 5054949 -32470 32465 4628.009900990099 467429 -128 123 -2.801980198019802 -283 +990 2 10980 99891 2.97297 299.97297 151.47297 15147.29729 2.97297 299.97296 151.47297 15147.29735 2.97297 299.97297 151.47297 15147.29700 2020-01-01 2020-01-02 2020-01-01 00:16:30 2020-01-02 03:44:51 2020-01-01 00:16:30.000 2020-01-02 03:44:51.000 990 99891 50440.5 5044050 990 99891 50440.5 5044050 -32578 32357 4565.3 456530 -128 127 0.82 82 991 2 10981 99892 2.97597 299.97597 151.47597 15147.59759 2.97597 299.97598 151.47597 15147.59795 2.97597 299.97597 151.47597 15147.59700 2020-01-01 2020-01-02 2020-01-01 00:16:31 2020-01-02 03:44:52 2020-01-01 00:16:31.000 2020-01-02 03:44:52.000 991 99892 50441.5 5044150 991 99892 50441.5 5044150 -32577 32358 4566.3 456630 -128 127 -0.74 -74 -992 2 10982 99893 2.97897 299.97897 151.47897 15147.89789 2.97897 299.97897 151.47897 15147.89759 2.97897 299.97897 151.4789700000001 15147.89700 2020-01-01 2020-01-02 2020-01-01 00:16:32 2020-01-02 03:44:53 2020-01-01 00:16:32.000 2020-01-02 03:44:53.000 992 99893 50442.5 5044250 992 99893 50442.5 5044250 -32576 32359 4567.3 456730 -128 124 -2.3 -230 +992 2 10982 99893 2.97897 299.97897 151.47897 15147.89789 2.97897 299.97897 151.47897 15147.89759 2.97897 299.97897 151.47897 15147.89700 2020-01-01 2020-01-02 2020-01-01 00:16:32 2020-01-02 03:44:53 2020-01-01 00:16:32.000 2020-01-02 03:44:53.000 992 99893 50442.5 5044250 992 99893 50442.5 5044250 -32576 32359 4567.3 456730 -128 124 -2.3 -230 993 2 10983 99894 2.98198 299.98198 151.48198 15148.19819 2.98198 299.982 151.48198 15148.19823 2.98198 299.98198 151.48198 15148.19800 2020-01-01 2020-01-02 2020-01-01 00:16:33 2020-01-02 03:44:54 2020-01-01 00:16:33.000 2020-01-02 03:44:54.000 993 99894 50443.5 5044350 993 99894 50443.5 5044350 -32575 32360 4568.3 456830 -127 125 -1.3 -130 -994 2 10984 99895 2.98498 299.98498 151.48498 15148.49849 2.98498 299.985 151.48498 15148.49853 2.98498 299.98498 151.48497999999995 15148.49800 2020-01-01 2020-01-02 2020-01-01 00:16:34 2020-01-02 03:44:55 2020-01-01 00:16:34.000 2020-01-02 03:44:55.000 994 99895 50444.5 5044450 994 99895 50444.5 5044450 -32574 32361 4569.3 456930 -126 126 -0.3 -30 -995 2 10985 99896 2.98798 299.98798 151.48798 15148.79879 2.98798 299.98798 151.48798 15148.79882 2.98798 299.98798 151.4879799999999 15148.79800 2020-01-01 2020-01-02 2020-01-01 00:16:35 2020-01-02 03:44:56 2020-01-01 00:16:35.000 2020-01-02 03:44:56.000 995 99896 50445.5 5044550 995 99896 50445.5 5044550 -32573 32362 4570.3 457030 -125 127 0.7 70 -996 2 10986 99897 2.99099 299.99099 151.49099 15149.09909 2.99099 299.991 151.49099 15149.09942 2.99099 299.99099 151.49099000000015 15149.09900 2020-01-01 2020-01-02 2020-01-01 00:16:36 2020-01-02 03:44:57 2020-01-01 00:16:36.000 2020-01-02 03:44:57.000 996 99897 50446.5 5044650 996 99897 50446.5 5044650 -32572 32363 4571.3 457130 -128 127 -0.86 -86 -997 2 10987 99898 2.99399 299.99399 151.49399 15149.39939 2.99399 299.994 151.49399 15149.3991 2.99399 299.99399 151.49399000000017 15149.39900 2020-01-01 2020-01-02 2020-01-01 00:16:37 2020-01-02 03:44:58 2020-01-01 00:16:37.000 2020-01-02 03:44:58.000 997 99898 50447.5 5044750 997 99898 50447.5 5044750 -32571 32364 4572.3 457230 -128 127 -2.42 -242 -998 2 10988 99899 2.99699 299.99699 151.49699 15149.69969 2.99699 299.997 151.49699 15149.6997 2.99699 299.99699 151.49698999999984 15149.69900 2020-01-01 2020-01-02 2020-01-01 00:16:38 2020-01-02 03:44:59 2020-01-01 00:16:38.000 2020-01-02 03:44:59.000 998 99899 50448.5 5044850 998 99899 50448.5 5044850 -32570 32365 4573.3 457330 -128 123 -3.98 -398 +994 2 10984 99895 2.98498 299.98498 151.48498 15148.49849 2.98498 299.985 151.48498 15148.49853 2.98498 299.98498 151.48498 15148.49800 2020-01-01 2020-01-02 2020-01-01 00:16:34 2020-01-02 03:44:55 2020-01-01 00:16:34.000 2020-01-02 03:44:55.000 994 99895 50444.5 5044450 994 99895 50444.5 5044450 -32574 32361 4569.3 456930 -126 126 -0.3 -30 +995 2 10985 99896 2.98798 299.98798 151.48798 15148.79879 2.98798 299.98798 151.48798 15148.79882 2.98798 299.98798 151.48798 15148.79800 2020-01-01 2020-01-02 2020-01-01 00:16:35 2020-01-02 03:44:56 2020-01-01 00:16:35.000 2020-01-02 03:44:56.000 995 99896 50445.5 5044550 995 99896 50445.5 5044550 -32573 32362 4570.3 457030 -125 127 0.7 70 +996 2 10986 99897 2.99099 299.99099 151.49099 15149.09909 2.99099 299.991 151.49099 15149.09942 2.99099 299.99099 151.49099 15149.09900 2020-01-01 2020-01-02 2020-01-01 00:16:36 2020-01-02 03:44:57 2020-01-01 00:16:36.000 2020-01-02 03:44:57.000 996 99897 50446.5 5044650 996 99897 50446.5 5044650 -32572 32363 4571.3 457130 -128 127 -0.86 -86 +997 2 10987 99898 2.99399 299.99399 151.49399 15149.39939 2.99399 299.994 151.49399 15149.3991 2.99399 299.99399 151.49399 15149.39900 2020-01-01 2020-01-02 2020-01-01 00:16:37 2020-01-02 03:44:58 2020-01-01 00:16:37.000 2020-01-02 03:44:58.000 997 99898 50447.5 5044750 997 99898 50447.5 5044750 -32571 32364 4572.3 457230 -128 127 -2.42 -242 +998 2 10988 99899 2.99699 299.99699 151.49699 15149.69969 2.99699 299.997 151.49699 15149.6997 2.99699 299.99699 151.49699 15149.69900 2020-01-01 2020-01-02 2020-01-01 00:16:38 2020-01-02 03:44:59 2020-01-01 00:16:38.000 2020-01-02 03:44:59.000 998 99899 50448.5 5044850 998 99899 50448.5 5044850 -32570 32365 4573.3 457330 -128 123 -3.98 -398 ---- select row with nulls without states ---- -2 1 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N ---- select row with nulls with states ---- From ee2c600130f04ef8409d0499c9ea2489a4f52540 Mon Sep 17 00:00:00 2001 From: Grigory Buteyko Date: Wed, 4 Nov 2020 18:31:27 +0300 Subject: [PATCH 0111/1088] better comments --- src/AggregateFunctions/QuantileTDigest.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index c09797573c4..c547a93a257 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -189,7 +189,7 @@ public: } ++r; } - count = sum + l_count; // Update count, changed due inaccurancy + count = sum + l_count; // Update count, it might be different due to += inaccuracy /// At the end of the loop, all values to the right of l were "eaten". centroids.resize(l - centroids.begin() + 1); @@ -204,7 +204,7 @@ public: void add(T x, UInt64 cnt = 1) { if (cnt == 0) - return; // Count 0 breaks compress() assumptions + return; // Count 0 breaks compress() assumptions, we treat it as no sample addCentroid(Centroid(Value(x), Count(cnt))); } @@ -238,7 +238,7 @@ public: Centroid & c = centroids[i]; if (c.count <= 0 || std::isnan(c.count) || std::isnan(c.mean)) // invalid count breaks compress(), invalid mean breaks sort() { - centroids.resize(i); // Exception safety, without this line we will end up with TDigest with invalid centroids + centroids.resize(i); // Exception safety, without this line caller will end up with TDigest object in broken invariant state throw std::runtime_error("Invalid centroid " + std::to_string(c.count) + ":" + std::to_string(c.mean)); } count += c.count; From a9529e8d6558f54f171906c7f906c7cea91b49fb Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 4 Nov 2020 21:54:55 +0300 Subject: [PATCH 0112/1088] Bad code but all events --- src/Common/ZooKeeper/TestKeeperStorage.cpp | 347 +++++++++++++++++++-- src/Server/TestKeeperTCPHandler.cpp | 58 +--- src/Server/TestKeeperTCPHandler.h | 5 +- 3 files changed, 328 insertions(+), 82 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Common/ZooKeeper/TestKeeperStorage.cpp index 50564e6ae54..a3e637a4ce8 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.cpp +++ b/src/Common/ZooKeeper/TestKeeperStorage.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace zkutil { @@ -17,6 +18,11 @@ static String parentPath(const String & path) return "/"; } +static String baseName(const String & path) +{ + auto rslash_pos = path.rfind('/'); + return path.substr(rslash_pos + 1); +} TestKeeperStorage::TestKeeperStorage() { @@ -152,6 +158,255 @@ struct TestKeeperStorageGetRequest final : public TestKeeperStorageRequest } }; +struct TestKeeperStorageRemoveRequest final : public TestKeeperStorageRequest +{ + using TestKeeperStorageRequest::TestKeeperStorageRequest; + std::pair process(TestKeeperStorage::Container & container, int64_t /*zxid*/) const override + { + Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); + Coordination::ZooKeeperRemoveRequest & request = dynamic_cast(*zk_request); + Undo undo; + + auto it = container.find(request.path); + if (it == container.end()) + { + response.error = Coordination::Error::ZNONODE; + } + else if (request.version != -1 && request.version != it->second.stat.version) + { + response.error = Coordination::Error::ZBADVERSION; + } + else if (it->second.stat.numChildren) + { + response.error = Coordination::Error::ZNOTEMPTY; + } + else + { + auto prev_node = it->second; + container.erase(it); + auto & parent = container.at(parentPath(request.path)); + --parent.stat.numChildren; + ++parent.stat.cversion; + response.error = Coordination::Error::ZOK; + + undo = [prev_node, &container, path = request.path] + { + container.emplace(path, prev_node); + auto & undo_parent = container.at(parentPath(path)); + ++undo_parent.stat.numChildren; + --undo_parent.stat.cversion; + }; + } + + return { response_ptr, undo }; + } +}; + +struct TestKeeperStorageExistsRequest final : public TestKeeperStorageRequest +{ + using TestKeeperStorageRequest::TestKeeperStorageRequest; + std::pair process(TestKeeperStorage::Container & container, int64_t /*zxid*/) const override + { + Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); + Coordination::ZooKeeperExistsRequest & request = dynamic_cast(*zk_request); + + auto it = container.find(request.path); + if (it != container.end()) + { + response.stat = it->second.stat; + response.error = Coordination::Error::ZOK; + } + else + { + response.error = Coordination::Error::ZNONODE; + } + + return { response_ptr, {} }; + } +}; + +struct TestKeeperStorageSetRequest final : public TestKeeperStorageRequest +{ + using TestKeeperStorageRequest::TestKeeperStorageRequest; + std::pair process(TestKeeperStorage::Container & container, int64_t zxid) const override + { + Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperSetResponse & response = dynamic_cast(*response_ptr); + Coordination::ZooKeeperSetRequest & request = dynamic_cast(*zk_request); + Undo undo; + + auto it = container.find(request.path); + if (it == container.end()) + { + response.error = Coordination::Error::ZNONODE; + } + else if (request.version == -1 || request.version == it->second.stat.version) + { + auto prev_node = it->second; + + it->second.data = request.data; + ++it->second.stat.version; + it->second.stat.mzxid = zxid; + it->second.stat.mtime = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1); + it->second.data = request.data; + ++container.at(parentPath(request.path)).stat.cversion; + response.stat = it->second.stat; + response.error = Coordination::Error::ZOK; + + undo = [prev_node, &container, path = request.path] + { + container.at(path) = prev_node; + --container.at(parentPath(path)).stat.cversion; + }; + } + else + { + response.error = Coordination::Error::ZBADVERSION; + } + + return { response_ptr, {} }; + } +}; + +struct TestKeeperStorageListRequest final : public TestKeeperStorageRequest +{ + using TestKeeperStorageRequest::TestKeeperStorageRequest; + std::pair process(TestKeeperStorage::Container & container, int64_t /*zxid*/) const override + { + Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); + Coordination::ZooKeeperListRequest & request = dynamic_cast(*zk_request); + Undo undo; + auto it = container.find(request.path); + if (it == container.end()) + { + response.error = Coordination::Error::ZNONODE; + } + else + { + auto path_prefix = request.path; + if (path_prefix.empty()) + throw Coordination::Exception("Logical error: path cannot be empty", Coordination::Error::ZSESSIONEXPIRED); + + if (path_prefix.back() != '/') + path_prefix += '/'; + + /// Fairly inefficient. + for (auto child_it = container.upper_bound(path_prefix); + child_it != container.end() && startsWith(child_it->first, path_prefix); + ++child_it) + { + if (parentPath(child_it->first) == request.path) + response.names.emplace_back(baseName(child_it->first)); + } + + response.stat = it->second.stat; + response.error = Coordination::Error::ZOK; + } + + return { response_ptr, {} }; + } +}; + +struct TestKeeperStorageCheckRequest final : public TestKeeperStorageRequest +{ + using TestKeeperStorageRequest::TestKeeperStorageRequest; + std::pair process(TestKeeperStorage::Container & container, int64_t /*zxid*/) const override + { + Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); + Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*zk_request); + auto it = container.find(request.path); + if (it == container.end()) + { + response.error = Coordination::Error::ZNONODE; + } + else if (request.version != -1 && request.version != it->second.stat.version) + { + response.error = Coordination::Error::ZBADVERSION; + } + else + { + response.error = Coordination::Error::ZOK; + } + + return { response_ptr, {} }; + } +}; + +struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest +{ + std::vector concrete_requests; + TestKeeperStorageMultiRequest(const Coordination::ZooKeeperRequestPtr & zk_request_) + : TestKeeperStorageRequest(zk_request_) + { + Coordination::ZooKeeperMultiRequest & request = dynamic_cast(*zk_request); + concrete_requests.reserve(request.requests.size()); + + for (const auto & zk_request : request.requests) + { + if (const auto * concrete_request_create = dynamic_cast(zk_request.get())) + { + concrete_requests.push_back(std::make_shared(dynamic_pointer_cast(zk_request))); + } + else if (const auto * concrete_request_remove = dynamic_cast(zk_request.get())) + { + concrete_requests.push_back(std::make_shared(dynamic_pointer_cast(zk_request))); + } + else if (const auto * concrete_request_set = dynamic_cast(zk_request.get())) + { + concrete_requests.push_back(std::make_shared(dynamic_pointer_cast(zk_request))); + } + else if (const auto * concrete_request_check = dynamic_cast(zk_request.get())) + { + concrete_requests.push_back(std::make_shared(dynamic_pointer_cast(zk_request))); + } + else + throw Coordination::Exception("Illegal command as part of multi ZooKeeper request", Coordination::Error::ZBADARGUMENTS); + } + } + + std::pair process(TestKeeperStorage::Container & container, int64_t zxid) const override + { + Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); + std::vector undo_actions; + + try + { + for (const auto & concrete_request : concrete_requests) + { + auto [ cur_response, undo_action ] = concrete_request->process(container, zxid); + response.responses.emplace_back(cur_response); + if (cur_response->error != Coordination::Error::ZOK) + { + response.error = cur_response->error; + + for (auto it = undo_actions.rbegin(); it != undo_actions.rend(); ++it) + if (*it) + (*it)(); + + return { response_ptr, {} }; + } + else + undo_actions.emplace_back(std::move(undo_action)); + } + + response.error = Coordination::Error::ZOK; + return { response_ptr, {} }; + } + catch (...) + { + for (auto it = undo_actions.rbegin(); it != undo_actions.rend(); ++it) + if (*it) + (*it)(); + throw; + } + } +}; + void TestKeeperStorage::processingThread() { setThreadName("TestKeeperSProc"); @@ -174,24 +429,11 @@ void TestKeeperStorage::processingThread() auto zk_request = info.request->zk_request; LOG_DEBUG(&Poco::Logger::get("STORAGE"), "GOT REQUEST {}", zk_request->getOpNum()); - Coordination::ZooKeeperResponsePtr response; - if (zk_request->xid == -2) - { - response = std::make_shared(); - response->xid = zk_request->xid; - response->zxid = zxid; - } - else - { - zk_request->addRootPath(root_path); - LOG_DEBUG(&Poco::Logger::get("STORAGE"), "PROCESSING REQUEST"); - std::tie(response, std::ignore) = info.request->process(container, zxid); - response->xid = zk_request->xid; - LOG_DEBUG(&Poco::Logger::get("STORAGE"), "SENDING XID {}", response->xid); - response->zxid = zxid; - response->removeRootPath(root_path); - } + auto [response, _] = info.request->process(container, zxid); + response->xid = zk_request->xid; + response->zxid = zxid; + response->removeRootPath(root_path); LOG_DEBUG(&Poco::Logger::get("STORAGE"), "SENDING RESPONSE"); info.response_callback(response); @@ -237,23 +479,74 @@ void TestKeeperStorage::finalize() { tryLogCurrentException(__PRETTY_FUNCTION__); } +} + + + + +class TestKeeperWrapperFactory final : private boost::noncopyable +{ + +public: + using Creator = std::function; + using OpNumToRequest = std::unordered_map; + + static TestKeeperWrapperFactory & instance() + { + static TestKeeperWrapperFactory factory; + return factory; + } + + TestKeeperStorageRequestPtr get(const Coordination::ZooKeeperRequestPtr & zk_request) const + { + auto it = op_num_to_request.find(zk_request->getOpNum()); + if (it == op_num_to_request.end()) + throw Coordination::Exception("Unknown operation type " + std::to_string(zk_request->getOpNum()), Coordination::Error::ZBADARGUMENTS); + + return it->second(zk_request); + } + + void registerRequest(int32_t op_num, Creator creator) + { + if (!op_num_to_request.try_emplace(op_num, creator).second) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Request with op num {} already registered", op_num); + } + +private: + OpNumToRequest op_num_to_request; + +private: + TestKeeperWrapperFactory(); +}; + +template +void registerTestKeeperRequestWrapper(TestKeeperWrapperFactory & factory) +{ + factory.registerRequest(num, [] (const Coordination::ZooKeeperRequestPtr & zk_request) { return std::make_shared(zk_request); }); +} + + +TestKeeperWrapperFactory::TestKeeperWrapperFactory() +{ + registerTestKeeperRequestWrapper<11, TestKeeperStorageHeartbeatRequest>(*this); + //registerTestKeeperRequestWrapper<100, TestKeeperStorageAuthRequest>(*this); + //registerTestKeeperRequestWrapper<-11, TestKeeperStorageCloseRequest>(*this); + registerTestKeeperRequestWrapper<1, TestKeeperStorageCreateRequest>(*this); + registerTestKeeperRequestWrapper<2, TestKeeperStorageRemoveRequest>(*this); + registerTestKeeperRequestWrapper<3, TestKeeperStorageExistsRequest>(*this); + registerTestKeeperRequestWrapper<4, TestKeeperStorageGetRequest>(*this); + registerTestKeeperRequestWrapper<5, TestKeeperStorageSetRequest>(*this); + registerTestKeeperRequestWrapper<12, TestKeeperStorageListRequest>(*this); + registerTestKeeperRequestWrapper<13, TestKeeperStorageCheckRequest>(*this); + registerTestKeeperRequestWrapper<14, TestKeeperStorageMultiRequest>(*this); } TestKeeperStorage::AsyncResponse TestKeeperStorage::putRequest(const Coordination::ZooKeeperRequestPtr & request) { auto promise = std::make_shared>(); auto future = promise->get_future(); - TestKeeperStorageRequestPtr storage_request; - if (request->xid == -2) - storage_request = std::make_shared(request); - else if (request->getOpNum() == 1) - storage_request = std::make_shared(request); - else if (request->getOpNum() == 4) - storage_request = std::make_shared(request); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "UNKNOWN EVENT WITH OPNUM {}", request->getOpNum()) -; + TestKeeperStorageRequestPtr storage_request = TestKeeperWrapperFactory::instance().get(request); RequestInfo request_info; request_info.time = clock::now(); request_info.request = storage_request; diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index baf78e1fc59..0ca106d0e32 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -289,19 +289,16 @@ void TestKeeperTCPHandler::runImpl() } if (in->poll(max_wait)) - { - receiveHeartbeatRequest(); - } + receiveRequest(); } } -bool TestKeeperTCPHandler::receiveHeartbeatRequest() +void TestKeeperTCPHandler::receiveRequest() { LOG_DEBUG(log, "Receiving heartbeat event"); int32_t length; read(length, *in); - int32_t total_count = in->count(); LOG_DEBUG(log, "RECEIVED LENGTH {}", length); int32_t xid; LOG_DEBUG(log, "READING XID"); @@ -309,56 +306,15 @@ bool TestKeeperTCPHandler::receiveHeartbeatRequest() LOG_DEBUG(log, "Received xid {}", xid); - Coordination::ZooKeeperRequestPtr request; - if (xid == -2) - { - int32_t opnum; - read(opnum, *in); - LOG_DEBUG(log, "RRECEIVED OP NUM {}", opnum); - request = std::make_shared(); - request->xid = xid; - request->readImpl(*in); - int32_t readed = in->count() - total_count; - if (readed != length) - LOG_DEBUG(log, "EXPECTED TO READ {}, BUT GOT {}", length, readed); - } - else - { - int32_t opnum; - read(opnum, *in); - LOG_DEBUG(log, "RRECEIVED OP NUM {}", opnum); - if (opnum == 1) - request = std::make_shared(); - else if (opnum == 4) - request = std::make_shared(); - request->readImpl(*in); - request->xid = xid; - int32_t readed = in->count() - total_count; - if (readed != length) - LOG_DEBUG(log, "EXPECTED TO READ {}, BUT GOT {}", length, readed); - LOG_DEBUG(log, "REQUEST PUTTED TO STORAGE"); - } - + int32_t opnum; + read(opnum, *in); + Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(opnum); + request->xid = xid; + request->readImpl(*in); responses.push(test_keeper_storage->putRequest(request)); LOG_DEBUG(log, "Event received"); - return false; } -void TestKeeperTCPHandler::sendHeartbeatResponse() -{ - LOG_DEBUG(log, "Sending heartbeat event"); - int32_t length = sizeof(int32_t) + sizeof(int64_t) + sizeof(Coordination::Error); - write(length, *out); - int64_t zxid = test_keeper_storage->getZXID(); - int32_t xid = -2; - write(xid, *out); - write(zxid, *out); - write(Coordination::Error::ZOK, *out); - auto response = std::make_shared(); - response->writeImpl(*out); - out->next(); -} - } diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h index fd2c6227e73..2a796daa4e9 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/TestKeeperTCPHandler.h @@ -45,10 +45,7 @@ private: void sendHandshake(); void receiveHandshake(); - bool receiveHeartbeatRequest(); - void sendHeartbeatResponse(); - - void receiveCreateRequest(); + void receiveRequest(); }; } From c09fcf846dcb858d5b03cd30ef7724f6a56385c7 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 5 Nov 2020 11:12:04 +0300 Subject: [PATCH 0113/1088] added proper test for avg() --- tests/queries/0_stateless/01035_avg.reference | 7 ++- tests/queries/0_stateless/01035_avg.sql | 48 ++++++++++++++++--- 2 files changed, 46 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/01035_avg.reference b/tests/queries/0_stateless/01035_avg.reference index 8e5b888b523..f76c532ea69 100644 --- a/tests/queries/0_stateless/01035_avg.reference +++ b/tests/queries/0_stateless/01035_avg.reference @@ -1,2 +1,5 @@ -nan -499.5 +nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan +-0.5 -0.5 -0.5 -0.5 -0.5 -0.5 127.493856 32355.57552 499999.5 499999.5 499999.5 499999.5 499999.5 -0.000500002 0.49999949943727 -0.000005 1.7009999999999985e-8 +-2767.546272 +999999 +-0.50000449943727 diff --git a/tests/queries/0_stateless/01035_avg.sql b/tests/queries/0_stateless/01035_avg.sql index ee58587736f..a185ce816fb 100644 --- a/tests/queries/0_stateless/01035_avg.sql +++ b/tests/queries/0_stateless/01035_avg.sql @@ -1,9 +1,43 @@ -CREATE TABLE IF NOT EXISTS test_01035 ( - t UInt16 -) ENGINE = Memory; +SET allow_experimental_bigint_types=1; -SELECT avg(t) FROM test_01035; -INSERT INTO test_01035 SELECT * FROM system.numbers LIMIT 1000; -SELECT avg(t) FROM test_01035; +-- no UInt128 as for now +CREATE TABLE IF NOT EXISTS test_01035_avg ( + i8 Int8 DEFAULT i64, + i16 Int16 DEFAULT i64, + i32 Int32 DEFAULT i64, + i64 Int64 DEFAULT if(u64 % 2 = 0, toInt64(u64), toInt64(-u64)), + i128 Int128 DEFAULT i64, + i256 Int256 DEFAULT i64, -DROP TABLE IF EXISTS test_01035 + u8 UInt8 DEFAULT u64, + u16 UInt16 DEFAULT u64, + u32 UInt32 DEFAULT u64, + u64 UInt64, + u256 UInt256 DEFAULT u64, + + f32 Float32 DEFAULT u64, + f64 Float64 DEFAULT u64, + + d32 Decimal32(4) DEFAULT toDecimal32(i32 / 1000, 4), + d64 Decimal64(18) DEFAULT toDecimal64(u64 / 1000000, 8), + d128 Decimal128(20) DEFAULT toDecimal128(i128 / 100000, 20), + d256 Decimal256(40) DEFAULT toDecimal256(i256 / 100000, 40) +) ENGINE = MergeTree() ORDER BY i64; + +SELECT avg(i8), avg(i16), avg(i32), avg(i64), avg(i128), avg(i256), + avg(u8), avg(u16), avg(u32), avg(u64), avg(u256), + avg(f32), avg(f64), + avg(d32), avg(d64), avg(d128), avg(d256) FROM test_01035_avg; + +INSERT INTO test_01035_avg (u64) SELECT number FROM system.numbers LIMIT 1000000; + +SELECT avg(i8), avg(i16), avg(i32), avg(i64), avg(i128), avg(i256), + avg(u8), avg(u16), avg(u32), avg(u64), avg(u256), + avg(f32), avg(f64), + avg(d32), avg(d64), avg(d128), avg(d256) FROM test_01035_avg; + +SELECT avg(i8 * i16) FROM test_01035_avg; +SELECT avg(f32 + f64) FROM test_01035_avg; +SELECT avg(d128 - d64) FROM test_01035_avg; + +DROP TABLE IF EXISTS test_01035_avg; From bcd8f1896f79a1218d5388b98bfe9742a38459f2 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 5 Nov 2020 12:10:26 +0300 Subject: [PATCH 0114/1088] fixed the MSan exception on ColumnConst(ColumnDecimal) ctor --- src/Columns/ColumnDecimal.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 1939d87e357..1f96ab00647 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -127,7 +127,9 @@ public: bool isNumeric() const override { return false; } bool canBeInsideNullable() const override { return true; } - bool isFixedAndContiguous() const override { return true; } + + bool isFixedAndContiguous() const final { return is_POD; } + size_t sizeOfValueIfFixed() const override { return sizeof(T); } size_t size() const override { return data.size(); } From ddd40fc4ce15e8277fd0978d3fbb08212b14e38a Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 5 Nov 2020 12:20:05 +0300 Subject: [PATCH 0115/1088] fixed the perf test --- tests/performance/avg_weighted.xml | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/performance/avg_weighted.xml b/tests/performance/avg_weighted.xml index 13bc7e58c44..a55fe245399 100644 --- a/tests/performance/avg_weighted.xml +++ b/tests/performance/avg_weighted.xml @@ -1,20 +1,22 @@ + + hits_100m_single + + DROP TABLE IF EXISTS perf_avg SET allow_experimental_bigint_types=1 CREATE TABLE perf_avg( num UInt64, num_u Decimal256(75) DEFAULT toDecimal256(num / 400000, 75), - num_f Float64 DEFAULT num + num_f Float64 DEFAULT num / 100 ) ENGINE = MergeTree() ORDER BY tuple() INSERT INTO perf_avg(num) - SELECT number / r - FROM system.numbers - ARRAY JOIN range(1, 400000) AS r - LIMIT 200000000 + SELECT toUInt64(UserID / (WatchID + 1) * 1000000) + FROM hits_100m_single SELECT avg(num) FROM perf_avg From 1ee417b12582220e7ef778a6b31a6756569f8d9c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 5 Nov 2020 12:20:51 +0300 Subject: [PATCH 0116/1088] Update 00087_math_functions.sql --- tests/queries/0_stateless/00087_math_functions.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00087_math_functions.sql b/tests/queries/0_stateless/00087_math_functions.sql index 8b4b84ead9b..8bb883a4808 100644 --- a/tests/queries/0_stateless/00087_math_functions.sql +++ b/tests/queries/0_stateless/00087_math_functions.sql @@ -92,7 +92,7 @@ select atan2(0, 1) = 0; select atan2(0, 2) = 0; select atan2(1, 0) = pi() / 2; select atan2(1, 1) = pi() / 4; -select atan2(-1, -1) = -3 * pi() / 4); +select atan2(-1, -1) = -3 * pi() / 4; select hypot(0, 1) = 1; select hypot(1, 0) = 1; From 2a0d9da5e4bd3a2f95615a0bb23cc47c7efb12c4 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 5 Nov 2020 12:25:42 +0300 Subject: [PATCH 0117/1088] cleanup --- docs/en/sql-reference/statements/select/union-all.md | 2 +- src/Common/ErrorCodes.cpp | 8 -------- src/Interpreters/executeQuery.cpp | 11 +++++++++-- 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/statements/select/union-all.md b/docs/en/sql-reference/statements/select/union-all.md index 1784da37c9a..f150efbdc80 100644 --- a/docs/en/sql-reference/statements/select/union-all.md +++ b/docs/en/sql-reference/statements/select/union-all.md @@ -29,7 +29,7 @@ Queries that are parts of `UNION ALL` can’t be enclosed in round brackets. [OR The difference between `UNION ALL` and `UNION DISTINCT` is that `UNION DISTINCT` will do a distinct transform for union result, it is equivalent to `SELECT DISTINCT` from a subquery containing `UNION ALL`. # UNION Clause {#union-clause} -By defaul, `UNION` has same react as `UNION DISTINCT`, but you can specify union mode by setting `union_default_mode`, values can be 'ALL', 'DISTINCT' or empty string. However, if you use `UNION` with setting `union_default_mode` to empty string, it will throw an exception. +By default, `UNION` has the same behavior as `UNION DISTINCT`, but you can specify union mode by setting `union_default_mode`, values can be 'ALL', 'DISTINCT' or empty string. However, if you use `UNION` with setting `union_default_mode` to empty string, it will throw an exception. ## Implementation Details {#implementation-details} diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 501f655154b..6f46dd861d0 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -559,15 +559,7 @@ namespace ErrorCodes return error_codes_names.names[error_code]; } -<<<<<<< HEAD - extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND = 2001; - extern const int ILLEGAL_PROJECTION_MANIPULATOR = 2002; - extern const int UNRECOGNIZED_ARGUMENTS = 2003; - extern const int UNKNOWN_UNION = 2004; - extern const int EXPECTED_ALL_OR_DISTINCT = 2005; -======= ErrorCode end() { return END+1; } ->>>>>>> origin/master } } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 8faccf7bc7b..3551d58444a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -349,8 +349,15 @@ static std::tuple executeQueryImpl( { if (!select_with_union_query->list_of_selects->children.empty()) { - if (auto new_settings = select_with_union_query->list_of_selects->children.back()->as()->settings()) - InterpreterSetQuery(new_settings, context).executeForCurrentContext(); + select_with_union_query->dumpTree(std::cerr); + // We might have an arbitrarily complex UNION tree, so just give + // up if the last first-order child is not a plain SELECT. + // It is flattened later, when we process UNION ALL/DISTINCT. + const auto * last_select = select_with_union_query->list_of_selects->children.back()->as(); + if (last_select && last_select->settings()) + { + InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext(); + } } } else if (const auto * query_with_output = dynamic_cast(ast.get())) From 6b35657f59fdcca5571a80edcb3dc98eca24968d Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 5 Nov 2020 12:38:46 +0300 Subject: [PATCH 0118/1088] updated the mem check code --- src/Columns/ColumnConst.cpp | 9 --------- src/Columns/ColumnDecimal.h | 4 +--- 2 files changed, 1 insertion(+), 12 deletions(-) diff --git a/src/Columns/ColumnConst.cpp b/src/Columns/ColumnConst.cpp index 550a44a23a2..36b8a74ec94 100644 --- a/src/Columns/ColumnConst.cpp +++ b/src/Columns/ColumnConst.cpp @@ -32,15 +32,6 @@ ColumnConst::ColumnConst(const ColumnPtr & data_, size_t s_) if (data->size() != 1) throw Exception("Incorrect size of nested column in constructor of ColumnConst: " + toString(data->size()) + ", must be 1.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - - /// Check that the value is initialized. We do it earlier, before it will be used, to ease debugging. -#if defined(MEMORY_SANITIZER) - if (data->isFixedAndContiguous()) - { - StringRef value = data->getDataAt(0); - __msan_check_mem_is_initialized(value.data, value.size); - } -#endif } ColumnPtr ColumnConst::convertToFullColumn() const diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 1f96ab00647..85ea7de88bc 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -127,9 +127,7 @@ public: bool isNumeric() const override { return false; } bool canBeInsideNullable() const override { return true; } - - bool isFixedAndContiguous() const final { return is_POD; } - + bool isFixedAndContiguous() const final { return true; } size_t sizeOfValueIfFixed() const override { return sizeof(T); } size_t size() const override { return data.size(); } From 8cf203c21c803db0b21e89dc781d86454695da16 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 5 Nov 2020 12:47:04 +0300 Subject: [PATCH 0119/1088] simplified the ColumnDecimal --- src/Columns/ColumnConst.cpp | 9 +++++ src/Columns/ColumnDecimal.h | 81 +++++-------------------------------- 2 files changed, 18 insertions(+), 72 deletions(-) diff --git a/src/Columns/ColumnConst.cpp b/src/Columns/ColumnConst.cpp index 36b8a74ec94..550a44a23a2 100644 --- a/src/Columns/ColumnConst.cpp +++ b/src/Columns/ColumnConst.cpp @@ -32,6 +32,15 @@ ColumnConst::ColumnConst(const ColumnPtr & data_, size_t s_) if (data->size() != 1) throw Exception("Incorrect size of nested column in constructor of ColumnConst: " + toString(data->size()) + ", must be 1.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + /// Check that the value is initialized. We do it earlier, before it will be used, to ease debugging. +#if defined(MEMORY_SANITIZER) + if (data->isFixedAndContiguous()) + { + StringRef value = data->getDataAt(0); + __msan_check_mem_is_initialized(value.data, value.size); + } +#endif } ColumnPtr ColumnConst::convertToFullColumn() const diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 85ea7de88bc..2242c0eecbc 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -55,43 +55,6 @@ private: UInt32 scale; }; -/// std::vector extended by Decimal scale -template -class DecimalVector : public std::vector -{ -public: - using Base = std::vector; - using Base::operator[]; - - DecimalVector(size_t size, UInt32 scale_) - : Base(size), - scale(scale_) - {} - - DecimalVector(const DecimalVector & other) - : Base(other.begin(), other.end()), - scale(other.scale) - {} - - DecimalVector(DecimalVector && other) - { - this->swap(other); - std::swap(scale, other.scale); - } - - DecimalVector & operator=(DecimalVector && other) - { - this->swap(other); - std::swap(scale, other.scale); - return *this; - } - - UInt32 getScale() const { return scale; } - -private: - UInt32 scale; -}; - /// A ColumnVector for Decimals template class ColumnDecimal final : public COWHelper> @@ -105,10 +68,6 @@ private: public: using ValueType = T; using NativeT = typename T::NativeType; - static constexpr bool is_POD = !is_big_int_v; - using Container = std::conditional_t, - DecimalVector>; private: ColumnDecimal(const size_t n, UInt32 scale_) @@ -132,18 +91,8 @@ public: size_t size() const override { return data.size(); } size_t byteSize() const override { return data.size() * sizeof(data[0]); } - size_t allocatedBytes() const override - { - if constexpr (is_POD) - return data.allocated_bytes(); - else - return data.capacity() * sizeof(data[0]); - } - void protect() override - { - if constexpr (is_POD) - data.protect(); - } + size_t allocatedBytes() const override { return data.allocated_bytes(); } + void protect() override { data.protect(); } void reserve(size_t n) override { data.reserve(n); } void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast(src).getData()[n]); } @@ -151,36 +100,24 @@ public: void insertDefault() override { data.push_back(T()); } virtual void insertManyDefaults(size_t length) override { - if constexpr (is_POD) - data.resize_fill(data.size() + length); - else - data.resize(data.size() + length); + data.resize_fill(data.size() + length); } void insert(const Field & x) override { data.push_back(DB::get>(x)); } void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; void popBack(size_t n) override { - if constexpr (is_POD) - data.resize_assume_reserved(data.size() - n); - else - data.resize(data.size() - n); + data.resize_assume_reserved(data.size() - n); } StringRef getRawData() const override { - if constexpr (is_POD) - return StringRef(reinterpret_cast(data.data()), byteSize()); - else - throw Exception("getRawData() is not implemented for big integers", ErrorCodes::NOT_IMPLEMENTED); + return StringRef(reinterpret_cast(data.data()), byteSize()); } StringRef getDataAt(size_t n) const override { - if constexpr (is_POD) - return StringRef(reinterpret_cast(&data[n]), sizeof(data[n])); - else - throw Exception("getDataAt() is not implemented for big integers", ErrorCodes::NOT_IMPLEMENTED); + return StringRef(reinterpret_cast(&data[n]), sizeof(data[n])); } Float64 getFloat64(size_t n) const final { return DecimalUtils::convertTo(data[n], scale); } @@ -232,15 +169,15 @@ public: void insertValue(const T value) { data.push_back(value); } - Container & getData() { return data; } - const Container & getData() const { return data; } + DecimalPaddedPODArray & getData() { return data; } + const DecimalPaddedPODArray & getData() const { return data; } const T & getElement(size_t n) const { return data[n]; } T & getElement(size_t n) { return data[n]; } UInt32 getScale() const {return scale;} protected: - Container data; + DecimalPaddedPODArray data; UInt32 scale; template From d2fcd8548ef5a8f630336393e4e8899fdd3b91f4 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 5 Nov 2020 13:01:41 +0300 Subject: [PATCH 0120/1088] Update 00087_math_functions.sql --- tests/queries/0_stateless/00087_math_functions.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00087_math_functions.sql b/tests/queries/0_stateless/00087_math_functions.sql index 8bb883a4808..1c8fa03ff47 100644 --- a/tests/queries/0_stateless/00087_math_functions.sql +++ b/tests/queries/0_stateless/00087_math_functions.sql @@ -115,7 +115,7 @@ select (sum(abs(cosh(x) * cosh(x) - sinh(x) * sinh(x) - 1)) < 1e-9) / count() fr select asinh(0) = 0; select asinh(1) = -asinh(-1); select abs(asinh(1) - ln(1 + sqrt(2))) < 1e-9; -select (asinh(sinh(1)) - 1) < 1e-9 +select (asinh(sinh(1)) - 1) < 1e-9; select sum(abs(asinh(sinh(x)) - x) < 1e-9) / count() from system.one array join range(1000) as x; select acosh(1) = 0; @@ -127,7 +127,7 @@ select sum(abs(acosh(cosh(x)) - x) < 1e-9) / count() from system.one array join select atanh(0) = 0; select atanh(0.5) = -asinh(-0.5); select abs(atanh(0.9) - 0.5 * ln(19)) < 1e-9; -select (atanh(tanh(1)) - 1) < 1e-9 +select (atanh(tanh(1)) - 1) < 1e-9; select sum(abs(asinh(sinh(x)) - x) < 1e-9) / count() from system.one array join range(1000) as x; select erf(0) = 0; From 680e2a5af155d0059ebad5b1ece08755ca635f8d Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 5 Nov 2020 13:11:56 +0300 Subject: [PATCH 0121/1088] fixing Decimal is_POD --- src/Columns/ColumnDecimal.cpp | 69 ++++++++--------------------------- src/Columns/ColumnDecimal.h | 13 ++----- src/Core/DecimalComparison.h | 1 + 3 files changed, 20 insertions(+), 63 deletions(-) diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index b9549175f6c..ce2475f06c2 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -53,32 +53,16 @@ void ColumnDecimal::compareColumn(const IColumn & rhs, size_t rhs_row_num, template StringRef ColumnDecimal::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const { - if constexpr (is_POD) - { - auto * pos = arena.allocContinue(sizeof(T), begin); - memcpy(pos, &data[n], sizeof(T)); - return StringRef(pos, sizeof(T)); - } - else - { - char * pos = arena.allocContinue(BigInt::size, begin); - return BigInt::serialize(data[n], pos); - } + auto * pos = arena.allocContinue(sizeof(T), begin); + memcpy(pos, &data[n], sizeof(T)); + return StringRef(pos, sizeof(T)); } template const char * ColumnDecimal::deserializeAndInsertFromArena(const char * pos) { - if constexpr (is_POD) - { - data.push_back(unalignedLoad(pos)); - return pos + sizeof(T); - } - else - { - data.push_back(BigInt::deserialize(pos)); - return pos + BigInt::size; - } + data.push_back(unalignedLoad(pos)); + return pos + sizeof(T); } template @@ -251,24 +235,13 @@ MutableColumnPtr ColumnDecimal::cloneResized(size_t size) const new_col.data.resize(size); size_t count = std::min(this->size(), size); - if constexpr (is_POD) - { - memcpy(new_col.data.data(), data.data(), count * sizeof(data[0])); - if (size > count) - { - void * tail = &new_col.data[count]; - memset(tail, 0, (size - count) * sizeof(T)); - } - } - else - { - for (size_t i = 0; i < count; i++) - new_col.data[i] = data[i]; + memcpy(new_col.data.data(), data.data(), count * sizeof(data[0])); - if (size > count) - for (size_t i = count; i < size; i++) - new_col.data[i] = T{}; + if (size > count) + { + void * tail = &new_col.data[count]; + memset(tail, 0, (size - count) * sizeof(T)); } } @@ -278,16 +251,9 @@ MutableColumnPtr ColumnDecimal::cloneResized(size_t size) const template void ColumnDecimal::insertData(const char * src, size_t /*length*/) { - if constexpr (is_POD) - { - T tmp; - memcpy(&tmp, src, sizeof(T)); - data.emplace_back(tmp); - } - else - { - data.push_back(BigInt::deserialize(src)); - } + T tmp; + memcpy(&tmp, src, sizeof(T)); + data.emplace_back(tmp); } template @@ -302,13 +268,8 @@ void ColumnDecimal::insertRangeFrom(const IColumn & src, size_t start, size_t size_t old_size = data.size(); data.resize(old_size + length); - if constexpr (is_POD) - memcpy(data.data() + old_size, &src_vec.data[start], length * sizeof(data[0])); - else - { - for (size_t i = 0; i < length; i++) - data[old_size + i] = src_vec.data[start + i]; - } + + memcpy(data.data() + old_size, &src_vec.data[start], length * sizeof(data[0])); } template diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 2242c0eecbc..bcbf85e0ff4 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -12,12 +12,6 @@ namespace DB { - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - /// PaddedPODArray extended by Decimal scale template class DecimalPaddedPODArray : public PaddedPODArray @@ -68,6 +62,7 @@ private: public: using ValueType = T; using NativeT = typename T::NativeType; + using Container = DecimalPaddedPODArray; private: ColumnDecimal(const size_t n, UInt32 scale_) @@ -169,15 +164,15 @@ public: void insertValue(const T value) { data.push_back(value); } - DecimalPaddedPODArray & getData() { return data; } - const DecimalPaddedPODArray & getData() const { return data; } + Container & getData() { return data; } + const Container & getData() const { return data; } const T & getElement(size_t n) const { return data[n]; } T & getElement(size_t n) { return data[n]; } UInt32 getScale() const {return scale;} protected: - DecimalPaddedPODArray data; + Container data; UInt32 scale; template diff --git a/src/Core/DecimalComparison.h b/src/Core/DecimalComparison.h index 674ed31683b..6da1fc00b7c 100644 --- a/src/Core/DecimalComparison.h +++ b/src/Core/DecimalComparison.h @@ -57,6 +57,7 @@ public: using Op = Operation; using ColVecA = std::conditional_t, ColumnDecimal, ColumnVector>; using ColVecB = std::conditional_t, ColumnDecimal, ColumnVector>; + using ArrayA = typename ColVecA::Container; using ArrayB = typename ColVecB::Container; From fcb61251a435d04926c5bf84c3c5810e5b0c0dd8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 5 Nov 2020 15:36:25 +0300 Subject: [PATCH 0122/1088] Update 00087_math_functions.sql --- tests/queries/0_stateless/00087_math_functions.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00087_math_functions.sql b/tests/queries/0_stateless/00087_math_functions.sql index 1c8fa03ff47..984a8347268 100644 --- a/tests/queries/0_stateless/00087_math_functions.sql +++ b/tests/queries/0_stateless/00087_math_functions.sql @@ -121,7 +121,7 @@ select sum(abs(asinh(sinh(x)) - x) < 1e-9) / count() from system.one array join select acosh(1) = 0; select acosh(2) = acosh(-2); select abs(acosh(2) - ln(2 + sqrt(3))) < 1e-9; -select (acosh(cosh(2)) - 2) < 1e-9 +select (acosh(cosh(2)) - 2) < 1e-9; select sum(abs(acosh(cosh(x)) - x) < 1e-9) / count() from system.one array join range(1, 1001) as x; select atanh(0) = 0; From 4bf7b54dffac29be4c0fc8daddd4b8b3648ec353 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 6 Nov 2020 11:35:26 +0000 Subject: [PATCH 0123/1088] fix --- src/Interpreters/executeQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 3551d58444a..1fa28389bb7 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -349,7 +349,6 @@ static std::tuple executeQueryImpl( { if (!select_with_union_query->list_of_selects->children.empty()) { - select_with_union_query->dumpTree(std::cerr); // We might have an arbitrarily complex UNION tree, so just give // up if the last first-order child is not a plain SELECT. // It is flattened later, when we process UNION ALL/DISTINCT. From 34990f7ee4a2f4eaa48fc7795f48d82c36169168 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 6 Nov 2020 21:27:10 +0800 Subject: [PATCH 0124/1088] ISSUES-16574 trigger CI From cbe5f3d667771773e78eb37f91d3a27890e30e11 Mon Sep 17 00:00:00 2001 From: Konstantin Malanchev Date: Fri, 6 Nov 2020 16:22:06 +0100 Subject: [PATCH 0125/1088] Reduce range to test --- tests/queries/0_stateless/00087_math_functions.sql | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/00087_math_functions.sql b/tests/queries/0_stateless/00087_math_functions.sql index 984a8347268..49c495ed359 100644 --- a/tests/queries/0_stateless/00087_math_functions.sql +++ b/tests/queries/0_stateless/00087_math_functions.sql @@ -104,31 +104,31 @@ select sinh(0) = 0; select sinh(1) = -sinh(-1); select abs(sinh(1) - 0.5 * (e() - exp(-1))) < 1e-9; select abs(sinh(2) - 0.5 * (exp(2) - exp(-2))) < 1e-9; -select (sum(abs(sinh(x) - 0.5 * (exp(x) - exp(-x)))) < 1e-9) / count() from system.one array join range(1000) as x; +select (sum(abs(sinh(x) - 0.5 * (exp(x) - exp(-x)))) < 1e-9) / count() from system.one array join range(100) as x; select cosh(0) = 1; select cosh(1) = cosh(-1); select abs(cosh(1) - 0.5 * (e() + exp(-1))) < 1e-9; select abs(pow(cosh(1), 2) - pow(sinh(1), 2) - 1) < 1e-9; -select (sum(abs(cosh(x) * cosh(x) - sinh(x) * sinh(x) - 1)) < 1e-9) / count() from system.one array join range(1000) as x; +select (sum(abs(cosh(x) * cosh(x) - sinh(x) * sinh(x) - 1)) < 1e-9) / count() from system.one array join range(10) as x; select asinh(0) = 0; select asinh(1) = -asinh(-1); select abs(asinh(1) - ln(1 + sqrt(2))) < 1e-9; select (asinh(sinh(1)) - 1) < 1e-9; -select sum(abs(asinh(sinh(x)) - x) < 1e-9) / count() from system.one array join range(1000) as x; +select sum(abs(asinh(sinh(x)) - x) < 1e-9) / count() from system.one array join range(100) as x; select acosh(1) = 0; select acosh(2) = acosh(-2); select abs(acosh(2) - ln(2 + sqrt(3))) < 1e-9; select (acosh(cosh(2)) - 2) < 1e-9; -select sum(abs(acosh(cosh(x)) - x) < 1e-9) / count() from system.one array join range(1, 1001) as x; +select sum(abs(acosh(cosh(x)) - x) < 1e-9) / count() from system.one array join range(1, 101) as x; select atanh(0) = 0; select atanh(0.5) = -asinh(-0.5); select abs(atanh(0.9) - 0.5 * ln(19)) < 1e-9; select (atanh(tanh(1)) - 1) < 1e-9; -select sum(abs(asinh(sinh(x)) - x) < 1e-9) / count() from system.one array join range(1000) as x; +select sum(abs(atanh(tanh(x)) - x) < 1e-9) / count() from system.one array join range(10) as x; select erf(0) = 0; select erf(-10) = -1; From dabb23b6680f064f8ad5f712e71e6baef4f5d8d0 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 5 Nov 2020 21:07:14 +0300 Subject: [PATCH 0126/1088] done --- src/DataStreams/ParallelParsingBlockInputStream.cpp | 2 +- src/DataStreams/ParallelParsingBlockInputStream.h | 2 +- src/Formats/FormatFactory.cpp | 5 ++++- .../01548_parallel_parsing_max_memory.reference | 1 + .../0_stateless/01548_parallel_parsing_max_memory.sh | 8 ++++++++ 5 files changed, 15 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01548_parallel_parsing_max_memory.reference create mode 100755 tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh diff --git a/src/DataStreams/ParallelParsingBlockInputStream.cpp b/src/DataStreams/ParallelParsingBlockInputStream.cpp index 050a0d8ef8a..19b04d36fc1 100644 --- a/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -17,7 +17,7 @@ ParallelParsingBlockInputStream::ParallelParsingBlockInputStream(const Params & // Subtract one thread that we use for segmentation and one for // reading. After that, must have at least two threads left for // parsing. See the assertion below. - pool(std::max(2, params.max_threads - 2)), + pool(std::max(2, static_cast(params.max_threads) - 2)), file_segmentation_engine(params.file_segmentation_engine) { // See comment above. diff --git a/src/DataStreams/ParallelParsingBlockInputStream.h b/src/DataStreams/ParallelParsingBlockInputStream.h index 4c110f8a937..c882acd9ddd 100644 --- a/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/src/DataStreams/ParallelParsingBlockInputStream.h @@ -69,7 +69,7 @@ public: const InputProcessorCreator & input_processor_creator; const InputCreatorParams & input_creator_params; FormatFactory::FileSegmentationEngine file_segmentation_engine; - int max_threads; + size_t max_threads; size_t min_chunk_bytes; }; diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 4dc5b816420..728f9ae5a24 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -166,6 +166,9 @@ BlockInputStreamPtr FormatFactory::getInput( // (segmentator + two parsers + reader). bool parallel_parsing = settings.input_format_parallel_parsing && file_segmentation_engine && settings.max_threads >= 4; + if (settings.min_chunk_bytes_for_parallel_parsing * settings.max_threads * 2 > settings.max_memory_usage) + parallel_parsing = false; + if (parallel_parsing && name == "JSONEachRow") { /// FIXME ParallelParsingBlockInputStream doesn't support formats with non-trivial readPrefix() and readSuffix() @@ -195,7 +198,7 @@ BlockInputStreamPtr FormatFactory::getInput( auto input_creator_params = ParallelParsingBlockInputStream::InputCreatorParams{sample, row_input_format_params, format_settings}; ParallelParsingBlockInputStream::Params params{buf, input_getter, input_creator_params, file_segmentation_engine, - static_cast(settings.max_threads), + settings.max_threads, settings.min_chunk_bytes_for_parallel_parsing}; return std::make_shared(params); } diff --git a/tests/queries/0_stateless/01548_parallel_parsing_max_memory.reference b/tests/queries/0_stateless/01548_parallel_parsing_max_memory.reference new file mode 100644 index 00000000000..cf77cd33536 --- /dev/null +++ b/tests/queries/0_stateless/01548_parallel_parsing_max_memory.reference @@ -0,0 +1 @@ +19884108 diff --git a/tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh b/tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh new file mode 100755 index 00000000000..884d5b6e058 --- /dev/null +++ b/tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +yes http://foobarfoobarfoobarfoobarfoobarfoobarfoobar.com | head -c1G > 1g.csv + +$CLICKHOUSE_LOCAL --stacktrace --input_format_parallel_parsing=1 --max_memory_usage=100Mi -q "select count() from file('1g.csv', 'TSV', 'URL String')" \ No newline at end of file From 2febfd43e55654b1562efbdf1fdd6ab7d8cdeaee Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Fri, 6 Nov 2020 17:06:52 +0300 Subject: [PATCH 0127/1088] rewrite format line as string --- src/Formats/FormatFactory.cpp | 1 - .../Impl/LineAsStringRowInputFormat.cpp | 69 +++++-------------- .../Formats/Impl/LineAsStringRowInputFormat.h | 2 - 3 files changed, 19 insertions(+), 53 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 728f9ae5a24..e62b0fb8517 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -415,7 +415,6 @@ FormatFactory::FormatFactory() registerFileSegmentationEngineJSONEachRow(*this); registerFileSegmentationEngineRegexp(*this); registerFileSegmentationEngineJSONAsString(*this); - registerFileSegmentationEngineLineAsString(*this); registerInputFormatNative(*this); registerOutputFormatNative(*this); diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index 8f5eee4bb1b..1bcba78d434 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -12,7 +12,7 @@ namespace ErrorCodes } LineAsStringRowInputFormat::LineAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) : - IRowInputFormat(header_, in_, std::move(params_)), buf(in) + IRowInputFormat(header_, in_, std::move(params_)) { if (header_.columns() > 1 || header_.getDataTypes()[0]->getTypeId() != TypeIndex::String) { @@ -23,42 +23,37 @@ LineAsStringRowInputFormat::LineAsStringRowInputFormat(const Block & header_, Re void LineAsStringRowInputFormat::resetParser() { IRowInputFormat::resetParser(); - buf.reset(); } void LineAsStringRowInputFormat::readLineObject(IColumn & column) { - PeekableReadBufferCheckpoint checkpoint{buf}; - bool newline = true; - bool over = false; + DB::Memory<> object; - char * pos; + char * pos = in.position(); + bool need_more_data = true; - while (newline) + while (loadAtPosition(in, object, pos) && need_more_data) { - pos = find_first_symbols<'\n'>(buf.position(), buf.buffer().end()); - buf.position() = pos; - if (buf.position() == buf.buffer().end()) - { - over = true; - break; - } - else if (*buf.position() == '\n') - { - newline = false; - } + pos = find_first_symbols<'\n'>(pos, in.buffer().end()); + if (pos == in.buffer().end()) + continue; + + if (*pos == '\n') + need_more_data = false; + + ++pos; } - buf.makeContinuousMemoryFromCheckpointToPos(); - char * end = over ? buf.position(): ++buf.position(); - buf.rollbackToCheckpoint(); - column.insertData(buf.position(), end - (over ? 0 : 1) - buf.position()); - buf.position() = end; + saveUpToPosition(in, object, pos); + loadAtPosition(in, object, pos); + + /// Last character is always \n. + column.insertData(object.data(), object.size() - 1); } bool LineAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { - if (buf.eof()) + if (in.eof()) return false; readLineObject(*columns[0]); @@ -78,30 +73,4 @@ void registerInputFormatProcessorLineAsString(FormatFactory & factory) }); } -static bool fileSegmentationEngineLineAsStringpImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) -{ - char * pos = in.position(); - bool need_more_data = true; - - while (loadAtPosition(in, memory, pos) && need_more_data) - { - pos = find_first_symbols<'\n'>(pos, in.buffer().end()); - if (pos == in.buffer().end()) - continue; - - if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) - need_more_data = false; - - ++pos; - } - - saveUpToPosition(in, memory, pos); - return loadAtPosition(in, memory, pos); -} - -void registerFileSegmentationEngineLineAsString(FormatFactory & factory) -{ - factory.registerFileSegmentationEngine("LineAsString", &fileSegmentationEngineLineAsStringpImpl); -} - } diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h index a31dce1cc4a..9afb722705a 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h @@ -24,8 +24,6 @@ public: private: void readLineObject(IColumn & column); - - PeekableReadBuffer buf; }; } From f1709f9a80e0331112713c44a80f124935614ff7 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Fri, 6 Nov 2020 17:07:28 +0300 Subject: [PATCH 0128/1088] better --- src/Processors/Formats/Impl/LineAsStringRowInputFormat.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h index 9afb722705a..7c0187bc3ff 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h @@ -2,7 +2,6 @@ #include #include -#include namespace DB { From 9803565af77cece2f8fff904e6b183b90c980d11 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Fri, 6 Nov 2020 21:59:16 +0300 Subject: [PATCH 0129/1088] style --- src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index 1bcba78d434..01c2c8cd614 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -47,7 +47,7 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) saveUpToPosition(in, object, pos); loadAtPosition(in, object, pos); - /// Last character is always \n. + /// Last character is always \n. column.insertData(object.data(), object.size() - 1); } @@ -72,5 +72,4 @@ void registerInputFormatProcessorLineAsString(FormatFactory & factory) return std::make_shared(sample, buf, params); }); } - } From 125eb0272674f7e086d73faecd7a8c9d4378b279 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 7 Nov 2020 11:38:20 +0000 Subject: [PATCH 0130/1088] nomalize ASTSelectWithUnionQuery --- .../InterpreterSelectWithUnionQuery.cpp | 25 +++- .../InterpreterSelectWithUnionQuery.h | 2 +- src/Parsers/ASTSelectWithUnionQuery.cpp | 25 ++-- src/Parsers/ASTSelectWithUnionQuery.h | 4 +- src/Parsers/ExpressionListParsers.cpp | 2 +- src/Parsers/ExpressionListParsers.h | 2 +- src/Parsers/ParserSelectWithUnionQuery.cpp | 116 +++++++++++++++- ...t_and_setting_union_default_mode.reference | 23 ---- ..._explain_select_with_union_query.reference | 126 ++++++++++++++++++ .../01556_explain_select_with_union_query.sql | 13 ++ 10 files changed, 284 insertions(+), 54 deletions(-) create mode 100644 tests/queries/0_stateless/01556_explain_select_with_union_query.reference create mode 100644 tests/queries/0_stateless/01556_explain_select_with_union_query.sql diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 7cc0e890fc2..976dcaddd9c 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -170,6 +170,7 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_, return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock(); } +#if 0 size_t InterpreterSelectWithUnionQuery::optimizeUnionList() { auto union_distinct_num = 0; @@ -213,10 +214,11 @@ size_t InterpreterSelectWithUnionQuery::optimizeUnionList() } return union_distinct_num; } +#endif void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) { - auto num_distinct_union = optimizeUnionList(); + // auto num_distinct_union = optimizeUnionList(); size_t num_plans = nested_interpreters.size(); /// Skip union for single interpreter. @@ -227,8 +229,8 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) } /// All UNION streams in the chain does not need to do DISTINCT transform - if (num_distinct_union == 0) - { + // if (num_distinct_union == 0) + // { std::vector> plans(num_plans); DataStreams data_streams(num_plans); @@ -243,9 +245,23 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) auto union_step = std::make_unique(std::move(data_streams), result_header, max_threads); query_plan.unitePlans(std::move(union_step), std::move(plans)); - } + + const auto & query = query_ptr->as(); + if (query.union_mode == ASTSelectWithUnionQuery::Mode::DISTINCT) + { + /// Add distinct transform + const Settings & settings = context->getSettingsRef(); + SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); + + auto distinct_step + = std::make_unique(query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false); + + query_plan.addStep(std::move(distinct_step)); + } + // } /// The first union_distinct_num UNION streams need to do a DISTINCT transform after unite +#if 0 else { QueryPlan distinct_query_plan; @@ -298,6 +314,7 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) auto final_union_step = std::make_unique(std::move(final_data_streams), result_header, max_threads); query_plan.unitePlans(std::move(final_union_step), std::move(final_plans)); } +#endif } BlockIO InterpreterSelectWithUnionQuery::execute() diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index 06d31c92a67..9a3035f117c 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -49,7 +49,7 @@ private: std::unique_ptr buildCurrentChildInterpreter(const ASTPtr & ast_ptr_, const Names & current_required_result_column_names); - size_t optimizeUnionList(); + // size_t optimizeUnionList(); }; } diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 639c8ec1b6e..610c82ee03a 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -14,7 +14,7 @@ ASTPtr ASTSelectWithUnionQuery::clone() const res->list_of_selects = list_of_selects->clone(); res->children.push_back(res->list_of_selects); - res->union_modes = union_modes; + res->union_mode = union_mode; cloneOutputOptions(*res); return res; @@ -38,24 +38,15 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it) { if (it != list_of_selects->children.begin()) - settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION " - << mode_to_str(union_modes[it - list_of_selects->children.begin() - 1]) << (settings.hilite ? hilite_none : ""); + settings.ostr + << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") + << "UNION " + << mode_to_str(union_mode) << (settings.hilite ? hilite_none : ""); if (auto * node = (*it)->as()) { - // just one child in subquery, () is not need - if (node->list_of_selects->children.size() == 1) - { - if (it != list_of_selects->children.begin()) - settings.ostr << settings.nl_or_ws; - node->list_of_selects->children.at(0)->formatImpl(settings, state, frame); - } - // more than one child in subquery - else - { - auto sub_query = std::make_shared(); - sub_query->children.push_back(*it); - sub_query->formatImpl(settings, state, frame); - } + auto sub_query = std::make_shared(); + sub_query->children.push_back(*it); + sub_query->formatImpl(settings, state, frame); } else { diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index 5600dd4b43a..fd5eeae2d7a 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -23,9 +23,9 @@ public: DISTINCT }; - using Modes = std::vector; + using UnionModes = std::vector; - Modes union_modes; + Mode union_mode; ASTPtr list_of_selects; }; diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 1cc72f5fb8b..4eecc9754bf 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -128,7 +128,7 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } // SELECT ... UNION SELECT ... else - union_modes.push_back(ASTSelectWithUnionQuery::Mode::Unspecified); + union_modes.push_back(ASTSelectWithUnionQuery::Mode::DISTINCT); return true; } return false; diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index d93952923a9..a239676b7e2 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -119,7 +119,7 @@ private: ParserPtr s_union_parser; ParserPtr s_all_parser; ParserPtr s_distinct_parser; - ASTSelectWithUnionQuery::Modes union_modes; + ASTSelectWithUnionQuery::UnionModes union_modes; }; /** An expression with an infix binary left-associative operator. diff --git a/src/Parsers/ParserSelectWithUnionQuery.cpp b/src/Parsers/ParserSelectWithUnionQuery.cpp index db586867db0..49f5aa719e5 100644 --- a/src/Parsers/ParserSelectWithUnionQuery.cpp +++ b/src/Parsers/ParserSelectWithUnionQuery.cpp @@ -6,6 +6,84 @@ namespace DB { +static void getSelectsFromUnionListNode(ASTPtr & ast_select, ASTs & selects) +{ + if (auto * inner_union = ast_select->as()) + { + /// We need flatten from last to first + for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend(); ++child) + getSelectsFromUnionListNode(*child, selects); + + return; + } + + selects.push_back(std::move(ast_select)); +} + +void normalizeSelectList(ASTs & select_list, const ASTSelectWithUnionQuery::UnionModes & union_modes, ASTs & selects) +{ + int i; + for (i = union_modes.size() - 1; i >= 0; --i) + { + if (union_modes[i] == ASTSelectWithUnionQuery::Mode::ALL) + { + if (auto * inner_union = select_list[i + 1]->as()) + { + /// If inner_union is an UNION ALL list, just lift up + if (inner_union->union_mode == ASTSelectWithUnionQuery::Mode::ALL) + { + for (auto child = inner_union->list_of_selects->children.rbegin(); + child != inner_union->list_of_selects->children.rend(); + ++child) + selects.push_back(std::move(*child)); + } + /// inner_union is an UNION DISTINCT list, + // we cann't lift up + else + selects.push_back(std::move(select_list[i + 1])); + } + else + selects.push_back(std::move(select_list[i + 1])); + } + /// flatten all left nodes and current node to a UNION DISTINCT list + else if (union_modes[i] == ASTSelectWithUnionQuery::Mode::DISTINCT) + { + auto distinct_list = std::make_shared(); + distinct_list->list_of_selects = std::make_shared(); + distinct_list->children.push_back(distinct_list->list_of_selects); + for (int j = i + 1; j >= 0; j--) + { + getSelectsFromUnionListNode(select_list[j], distinct_list->list_of_selects->children); + } + distinct_list->union_mode = ASTSelectWithUnionQuery::Mode::DISTINCT; + // Reverse children list + std::reverse(distinct_list->list_of_selects->children.begin(), distinct_list->list_of_selects->children.end()); + selects.push_back(std::move(distinct_list)); + return; + } + } + /// No UNION DISTINCT or only one SELECT in select_list + if (i == -1) + { + if (auto * inner_union = select_list[0]->as()) + { + /// If inner_union is an UNION ALL list, just lift it up + if (inner_union->union_mode == ASTSelectWithUnionQuery::Mode::ALL) + { + for (auto child = inner_union->list_of_selects->children.rbegin(); + child != inner_union->list_of_selects->children.rend(); + ++child) + selects.push_back(std::move(*child)); + } + /// inner_union is an UNION DISTINCT list, + // we cann't lift it up + else + selects.push_back(std::move(select_list[i + 1])); + } + else + selects.push_back(std::move(select_list[0])); + } +} bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { @@ -20,16 +98,44 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & if (!parser.parse(pos, list_node, expected)) return false; + /// NOTE: We cann't simply flatten inner union query now, since we may have different union mode in query, + /// so flatten may change it's semantics. For example: + /// flatten `SELECT 1 UNION (SELECT 1 UNION ALL SELETC 1)` -> `SELECT 1 UNION SELECT 1 UNION ALL SELECT 1` + + /// Before normalize, if we got only one child which is ASTSelectWithUnionQuery, just lift it up + auto & expr_list = list_node->as(); + if (expr_list.children.size() == 1) + { + if (expr_list.children.at(0)->as()) + { + node = std::move(expr_list.children.at(0)); + return true; + } + } + auto select_with_union_query = std::make_shared(); node = select_with_union_query; - select_with_union_query->list_of_selects = list_node; + select_with_union_query->list_of_selects = std::make_shared(); select_with_union_query->children.push_back(select_with_union_query->list_of_selects); - select_with_union_query->union_modes = parser.getUnionModes(); - /// NOTE: We cann't flatten inner union query now, since we may have different union mode in query, - /// so flatten may change it's semantics. For example: - /// flatten `SELECT 1 UNION (SELECT 1 UNION ALL SELETC 1)` -> `SELECT 1 UNION SELECT 1 UNION ALL SELECT 1` + auto union_modes = parser.getUnionModes(); + + normalizeSelectList(expr_list.children, union_modes, select_with_union_query->list_of_selects->children); + /// We need reverse children list + std::reverse(select_with_union_query->list_of_selects->children.begin(), select_with_union_query->list_of_selects->children.end()); + + select_with_union_query->union_mode = ASTSelectWithUnionQuery::Mode::ALL; + + /// After normalize, if we only have one ASTSelectWithUnionQuery child, lift if up + if (select_with_union_query->list_of_selects->children.size() == 1) + { + if (select_with_union_query->list_of_selects->children.at(0)->as()) + { + node = std::move(select_with_union_query->list_of_selects->children.at(0)); + } + } + return true; } diff --git a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference index f9f3ee818e9..ff0086583fa 100644 --- a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference +++ b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference @@ -44,16 +44,6 @@ all 1 1 1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -all all all 1 @@ -77,16 +67,3 @@ all 1 1 1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/01556_explain_select_with_union_query.reference b/tests/queries/0_stateless/01556_explain_select_with_union_query.reference new file mode 100644 index 00000000000..c1b07cedd05 --- /dev/null +++ b/tests/queries/0_stateless/01556_explain_select_with_union_query.reference @@ -0,0 +1,126 @@ +Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) diff --git a/tests/queries/0_stateless/01556_explain_select_with_union_query.sql b/tests/queries/0_stateless/01556_explain_select_with_union_query.sql new file mode 100644 index 00000000000..abb7f602af5 --- /dev/null +++ b/tests/queries/0_stateless/01556_explain_select_with_union_query.sql @@ -0,0 +1,13 @@ +EXPLAIN SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1; +EXPLAIN (SELECT 1 UNION ALL SELECT 1) UNION ALL SELECT 1; +EXPLAIN SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1); + +EXPLAIN SELECT 1 UNION DISTINCT SELECT 1 UNION DISTINCT SELECT 1; +EXPLAIN (SELECT 1 UNION DISTINCT SELECT 1) UNION DISTINCT SELECT 1; +EXPLAIN SELECT 1 UNION DISTINCT (SELECT 1 UNION DISTINCT SELECT 1); + +EXPLAIN (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 UNION DISTINCT SELECT 1))) UNION ALL (((SELECT 1) UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 ) UNION DISTINCT SELECT 1)))); + +EXPLAIN (((((((((((((((SELECT 1 UNION ALL SELECT 1) UNION ALL SELECT 1)))))))))))))); +EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION ALL SELECT 1))))))))))))))))))))))))))))); +EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION DISTINCT SELECT 1))))))))))))))))))))))))))))); From a67f5b780f0582f88951bbd38e9f884ab874bbd7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Sun, 8 Nov 2020 19:01:12 +0300 Subject: [PATCH 0131/1088] Use sorted ip array instead of trie in TrieDictionary --- programs/server/config.xml | 2 +- src/Common/IPv6ToBinary.cpp | 27 ++ src/Common/IPv6ToBinary.h | 5 + src/Dictionaries/TrieDictionary.cpp | 274 ++++++++---------- src/Dictionaries/TrieDictionary.h | 37 ++- .../01018_ddl_dictionaries_special.reference | 5 + .../01018_ddl_dictionaries_special.sql | 8 +- 7 files changed, 200 insertions(+), 158 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index e17b59671af..a03270aa7b9 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -675,7 +675,7 @@ *_dictionary.xml diff --git a/src/Common/IPv6ToBinary.cpp b/src/Common/IPv6ToBinary.cpp index bfa6992de9e..00c1b520a7a 100644 --- a/src/Common/IPv6ToBinary.cpp +++ b/src/Common/IPv6ToBinary.cpp @@ -1,5 +1,7 @@ #include "IPv6ToBinary.h" #include +#include + #include @@ -28,4 +30,29 @@ std::array IPv6ToBinary(const Poco::Net::IPAddress & address) return res; } + +UInt32 IPv4ToBinary(const Poco::Net::IPAddress & address, bool & success) +{ + if (!address.isIPv4Mapped()) + { + success = false; + return 0; + } + + success = true; + if (Poco::Net::IPAddress::IPv6 == address.family()) + { + auto raw = reinterpret_cast(address.addr()); + return *reinterpret_cast(&raw[12]); + } + else if (Poco::Net::IPAddress::IPv4 == address.family()) + { + auto raw = reinterpret_cast(address.addr()); + return *reinterpret_cast(raw); + } + + success = false; + return 0; +} + } diff --git a/src/Common/IPv6ToBinary.h b/src/Common/IPv6ToBinary.h index e95dfa10223..4f2cdd0ea21 100644 --- a/src/Common/IPv6ToBinary.h +++ b/src/Common/IPv6ToBinary.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace Poco { namespace Net { class IPAddress; }} @@ -9,4 +10,8 @@ namespace DB /// Convert IP address to 16-byte array with IPv6 data (big endian). If it's an IPv4, map it to IPv6. std::array IPv6ToBinary(const Poco::Net::IPAddress & address); +/// Convert IP address to UInt32 (big endian) if it's IPv4 or IPv4 mapped to IPv6. +/// Sets success variable to true if succeed. +UInt32 IPv4ToBinary(const Poco::Net::IPAddress & address, bool & success); + } diff --git a/src/Dictionaries/TrieDictionary.cpp b/src/Dictionaries/TrieDictionary.cpp index d8267047b92..a2e2bcf0bd8 100644 --- a/src/Dictionaries/TrieDictionary.cpp +++ b/src/Dictionaries/TrieDictionary.cpp @@ -3,11 +3,11 @@ #include #include #include +#include #include #include #include #include -#include #include #include #include @@ -15,14 +15,6 @@ #include "DictionaryBlockInputStream.h" #include "DictionaryFactory.h" -#ifdef __clang__ - #pragma clang diagnostic ignored "-Wold-style-cast" - #pragma clang diagnostic ignored "-Wnewline-eof" -#endif - -#include - - namespace DB { namespace ErrorCodes @@ -31,7 +23,6 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; extern const int BAD_ARGUMENTS; extern const int DICTIONARY_IS_EMPTY; - extern const int NOT_IMPLEMENTED; } static void validateKeyTypes(const DataTypes & key_types) @@ -45,6 +36,18 @@ static void validateKeyTypes(const DataTypes & key_types) throw Exception{"Key does not match, expected either UInt32 or FixedString(16)", ErrorCodes::TYPE_MISMATCH}; } +/// Create IPAddress from 16 byte array converting to ipv4 if possible +static Poco::Net::IPAddress ip4or6fromBytes(const uint8_t * data) +{ + Poco::Net::IPAddress ipaddr(reinterpret_cast(data), IPV6_BINARY_LENGTH); + + // try to consider as ipv4 + bool is_v4 = false; + if (auto addr_v4 = IPv4ToBinary(ipaddr, is_v4); is_v4) + return Poco::Net::IPAddress(reinterpret_cast(&addr_v4), IPV4_BINARY_LENGTH); + + return ipaddr; +} TrieDictionary::TrieDictionary( const StorageID & dict_id_, @@ -57,17 +60,16 @@ TrieDictionary::TrieDictionary( , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) , require_nonempty(require_nonempty_) + , total_ip_length(0) , logger(&Poco::Logger::get("TrieDictionary")) { createAttributes(); - trie = btrie_create(); loadData(); calculateBytesAllocated(); } TrieDictionary::~TrieDictionary() { - btrie_destroy(trie); } #define DECLARE(TYPE) \ @@ -305,7 +307,8 @@ void TrieDictionary::loadData() /// created upfront to avoid excess allocations const auto keys_size = dict_struct.key->size(); - StringRefs keys(keys_size); + + ip_records.reserve(keys_size); const auto attributes_size = attributes.size(); @@ -331,11 +334,42 @@ void TrieDictionary::loadData() { const auto & attribute_column = *attribute_column_ptrs[attribute_idx]; auto & attribute = attributes[attribute_idx]; - setAttributeValue(attribute, key_column->getDataAt(row_idx), attribute_column[row_idx]); + + setAttributeValue(attribute, attribute_column[row_idx]); } + + size_t row_number = ip_records.size(); + + std::string addr_str(key_column->getDataAt(row_idx).toString()); + size_t pos = addr_str.find('/'); + if (pos != std::string::npos) + { + IPAddress addr(addr_str.substr(0, pos)); + UInt8 prefix = std::stoi(addr_str.substr(pos + 1), nullptr, 10); + addr = addr & IPAddress(prefix, addr.family()); + ip_records.emplace_back(IPRecord{addr, prefix, row_number}); + } + else + { + IPAddress addr(addr_str); + UInt8 prefix = addr.length() * 8; + ip_records.emplace_back(IPRecord{addr, prefix, row_number}); + } + total_ip_length += ip_records.back().addr.length(); } } + LOG_TRACE(logger, "{} ip records are read", ip_records.size()); + + std::sort(ip_records.begin(), ip_records.end(), [](const auto & a, const auto & b) + { + if (a.addr.family() != b.addr.family()) + return a.addr.family() < b.addr.family(); + if (a.addr == b.addr) + return a.prefix > b.prefix; + return a.addr < b.addr; + }); + stream->readSuffix(); if (require_nonempty && 0 == element_count) @@ -352,6 +386,8 @@ void TrieDictionary::addAttributeSize(const Attribute & attribute) void TrieDictionary::calculateBytesAllocated() { + bytes_allocated += ip_records.size() * sizeof(ip_records.front()); + bytes_allocated += total_ip_length; bytes_allocated += attributes.size() * sizeof(attributes.front()); for (const auto & attribute : attributes) @@ -411,8 +447,6 @@ void TrieDictionary::calculateBytesAllocated() } } } - - bytes_allocated += btrie_allocated(trie); } @@ -494,16 +528,15 @@ void TrieDictionary::getItemsImpl( const auto first_column = key_columns.front(); const auto rows = first_column->size(); + if (first_column->isNumeric()) { for (const auto i : ext::range(0, rows)) { - auto addr = Int32(first_column->get64(i)); - uintptr_t slot = btrie_find(trie, addr); -#pragma GCC diagnostic push -#pragma GCC diagnostic warning "-Wold-style-cast" - set_value(i, slot != BTRIE_NULL ? static_cast(vec[slot]) : get_default(i)); -#pragma GCC diagnostic pop + auto addr = Poco::ByteOrder::toNetwork(UInt32(first_column->get64(i))); + auto ipaddr = IPAddress(reinterpret_cast(&addr), IPV4_BINARY_LENGTH); + auto found = lookupIPRecord(ipaddr); + set_value(i, (found != ipRecordNotFound()) ? static_cast(vec[found->row]) : get_default(i)); } } else @@ -511,107 +544,66 @@ void TrieDictionary::getItemsImpl( for (const auto i : ext::range(0, rows)) { auto addr = first_column->getDataAt(i); - if (addr.size != 16) + if (addr.size != IPV6_BINARY_LENGTH) throw Exception("Expected key to be FixedString(16)", ErrorCodes::LOGICAL_ERROR); - uintptr_t slot = btrie_find_a6(trie, reinterpret_cast(addr.data)); -#pragma GCC diagnostic push -#pragma GCC diagnostic warning "-Wold-style-cast" - set_value(i, slot != BTRIE_NULL ? static_cast(vec[slot]) : get_default(i)); -#pragma GCC diagnostic pop + auto ipaddr = ip4or6fromBytes(reinterpret_cast(addr.data)); + auto found = lookupIPRecord(ipaddr); + set_value(i, (found != ipRecordNotFound()) ? static_cast(vec[found->row]) : get_default(i)); } } query_count.fetch_add(rows, std::memory_order_relaxed); } - template -bool TrieDictionary::setAttributeValueImpl(Attribute & attribute, const StringRef key, const T value) +void TrieDictionary::setAttributeValueImpl(Attribute & attribute, const T value) { - // Insert value into appropriate vector type auto & vec = std::get>(attribute.maps); - size_t row = vec.size(); vec.push_back(value); - - // Parse IP address and subnet length from string (e.g. 2a02:6b8::3/64) - Poco::Net::IPAddress addr, mask; - std::string addr_str(key.toString()); - size_t pos = addr_str.find('/'); - if (pos != std::string::npos) - { - addr = Poco::Net::IPAddress(addr_str.substr(0, pos)); - mask = Poco::Net::IPAddress(std::stoi(addr_str.substr(pos + 1), nullptr, 10), addr.family()); - } - else - { - addr = Poco::Net::IPAddress(addr_str); - mask = Poco::Net::IPAddress(addr.length() * 8, addr.family()); - } - - /* - * Here we might overwrite the same key with the same slot as each key can map to multiple attributes. - * However, all columns have equal number of rows so it is okay to store only row number for each key - * instead of building a trie for each column. This comes at the cost of additional lookup in attribute - * vector on lookup time to return cell from row + column. The reason for this is to save space, - * and build only single trie instead of trie for each column. - */ - if (addr.family() == Poco::Net::IPAddress::IPv4) - { - UInt32 addr_v4 = Poco::ByteOrder::toNetwork(*reinterpret_cast(addr.addr())); - UInt32 mask_v4 = Poco::ByteOrder::toNetwork(*reinterpret_cast(mask.addr())); - return btrie_insert(trie, addr_v4, mask_v4, row) == 0; - } - - const uint8_t * addr_v6 = reinterpret_cast(addr.addr()); - const uint8_t * mask_v6 = reinterpret_cast(mask.addr()); - return btrie_insert_a6(trie, addr_v6, mask_v6, row) == 0; } -bool TrieDictionary::setAttributeValue(Attribute & attribute, const StringRef key, const Field & value) +void TrieDictionary::setAttributeValue(Attribute & attribute, const Field & value) { switch (attribute.type) { case AttributeUnderlyingType::utUInt8: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utUInt16: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utUInt32: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utUInt64: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utUInt128: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utInt8: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utInt16: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utInt32: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utInt64: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utFloat32: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utFloat64: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utDecimal32: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utDecimal64: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utDecimal128: - return setAttributeValueImpl(attribute, key, value.get()); + return setAttributeValueImpl(attribute, value.get()); case AttributeUnderlyingType::utString: { const auto & string = value.get(); const auto * string_in_arena = attribute.string_arena->insert(string.data(), string.size()); - setAttributeValueImpl(attribute, key, StringRef{string_in_arena, string.size()}); - return true; + return setAttributeValueImpl(attribute, StringRef{string_in_arena, string.size()}); } } - - return {}; } const TrieDictionary::Attribute & TrieDictionary::getAttribute(const std::string & attribute_name) const @@ -633,11 +625,9 @@ void TrieDictionary::has(const Attribute &, const Columns & key_columns, PaddedP for (const auto i : ext::range(0, rows)) { auto addr = Int32(first_column->get64(i)); - uintptr_t slot = btrie_find(trie, addr); -#pragma GCC diagnostic push -#pragma GCC diagnostic warning "-Wold-style-cast" - out[i] = (slot != BTRIE_NULL); -#pragma GCC diagnostic pop + auto ipaddr = IPAddress(reinterpret_cast(&addr), IPV4_BINARY_LENGTH); + auto found = lookupIPRecord(ipaddr); + out[i] = (found != ipRecordNotFound()); } } else @@ -648,78 +638,27 @@ void TrieDictionary::has(const Attribute &, const Columns & key_columns, PaddedP if (unlikely(addr.size != 16)) throw Exception("Expected key to be FixedString(16)", ErrorCodes::LOGICAL_ERROR); - uintptr_t slot = btrie_find_a6(trie, reinterpret_cast(addr.data)); -#pragma GCC diagnostic push -#pragma GCC diagnostic warning "-Wold-style-cast" - out[i] = (slot != BTRIE_NULL); -#pragma GCC diagnostic pop + auto ipaddr = ip4or6fromBytes(reinterpret_cast(addr.data)); + auto found = lookupIPRecord(ipaddr); + out[i] = (found != ipRecordNotFound()); } } query_count.fetch_add(rows, std::memory_order_relaxed); } -template -static void trieTraverse(const btrie_t * trie, Getter && getter) -{ - KeyType key = 0; - const KeyType high_bit = ~((~key) >> 1); - - btrie_node_t * node; - node = trie->root; - - std::stack stack; - while (node) - { - stack.push(node); - node = node->left; - } - - auto get_bit = [&high_bit](size_t size) { return size ? (high_bit >> (size - 1)) : 0; }; - - while (!stack.empty()) - { - node = stack.top(); - stack.pop(); -#pragma GCC diagnostic push -#pragma GCC diagnostic warning "-Wold-style-cast" - if (node && node->value != BTRIE_NULL) -#pragma GCC diagnostic pop - getter(key, stack.size()); - - if (node && node->right) - { - stack.push(nullptr); - key |= get_bit(stack.size()); - stack.push(node->right); - while (stack.top()->left) - stack.push(stack.top()->left); - } - else - key &= ~get_bit(stack.size()); - } -} - Columns TrieDictionary::getKeyColumns() const { auto ip_column = ColumnFixedString::create(IPV6_BINARY_LENGTH); auto mask_column = ColumnVector::create(); -#if defined(__SIZEOF_INT128__) - auto getter = [&ip_column, &mask_column](__uint128_t ip, size_t mask) + for (const auto & record : ip_records) { - Poco::UInt64 * ip_array = reinterpret_cast(&ip); // Poco:: for old poco + macos - ip_array[0] = Poco::ByteOrder::fromNetwork(ip_array[0]); - ip_array[1] = Poco::ByteOrder::fromNetwork(ip_array[1]); - std::swap(ip_array[0], ip_array[1]); - ip_column->insertData(reinterpret_cast(ip_array), IPV6_BINARY_LENGTH); - mask_column->insertValue(static_cast(mask)); - }; + auto ip_array = IPv6ToBinary(record.addr); + ip_column->insertData(ip_array.data(), IPV6_BINARY_LENGTH); + mask_column->insertValue(record.prefix); + } - trieTraverse(trie, std::move(getter)); -#else - throw Exception("TrieDictionary::getKeyColumns is not implemented for 32bit arch", ErrorCodes::NOT_IMPLEMENTED); -#endif return {std::move(ip_column), std::move(mask_column)}; } @@ -755,6 +694,45 @@ BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_nam shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)); } +int TrieDictionary::matchIPAddrWithRecord(const IPAddress & ipaddr, const IPRecord & record) const +{ + if (ipaddr.family() != record.addr.family()) + return ipaddr.family() < record.addr.family() ? -1 : 1; + + auto masked_ipaddr = ipaddr & IPAddress(record.prefix, record.addr.family()); + if (masked_ipaddr < record.addr) + return -1; + if (masked_ipaddr == record.addr) + return 0; + return 1; +} + +TrieDictionary::IPRecordConstIt TrieDictionary::ipRecordNotFound() const +{ + return ip_records.end(); +} + +TrieDictionary::IPRecordConstIt TrieDictionary::lookupIPRecord(const IPAddress & target) const +{ + if (ip_records.empty()) + return ipRecordNotFound(); + + auto comp = [&](const IPAddress & needle, const IPRecord & record) -> bool + { + return matchIPAddrWithRecord(needle, record) < 0; + }; + + auto next_it = std::upper_bound(ip_records.begin(), ip_records.end(), target, comp); + + if (next_it == ip_records.begin()) + return ipRecordNotFound(); + + auto found = next_it - 1; + if (matchIPAddrWithRecord(target, *found) == 0) + return found; + + return ipRecordNotFound(); +} void registerDictionaryTrie(DictionaryFactory & factory) { diff --git a/src/Dictionaries/TrieDictionary.h b/src/Dictionaries/TrieDictionary.h index 1849f161935..891dadd3be8 100644 --- a/src/Dictionaries/TrieDictionary.h +++ b/src/Dictionaries/TrieDictionary.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -14,9 +15,6 @@ #include "IDictionary.h" #include "IDictionarySource.h" -struct btrie_s; -typedef struct btrie_s btrie_t; - namespace DB { class TrieDictionary final : public IDictionaryBase @@ -150,9 +148,22 @@ public: BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; private: + template using ContainerType = std::vector; + using IPAddress = Poco::Net::IPAddress; + + struct IPRecord; + using IPRecordConstIt = ContainerType::const_iterator; + + struct IPRecord final + { + IPAddress addr; + UInt8 prefix; + size_t row; + }; + struct Attribute final { AttributeUnderlyingType type; @@ -212,11 +223,10 @@ private: void getItemsImpl(const Attribute & attribute, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const; - template - bool setAttributeValueImpl(Attribute & attribute, const StringRef key, const T value); + void setAttributeValueImpl(Attribute & attribute, const T value); - bool setAttributeValue(Attribute & attribute, const StringRef key, const Field & value); + void setAttributeValue(Attribute & attribute, const Field & value); const Attribute & getAttribute(const std::string & attribute_name) const; @@ -225,14 +235,27 @@ private: Columns getKeyColumns() const; + /** + * Compare ip addresses. + * + * @return negative value if ipaddr less than address in record + * @return zero if ipaddr in record subnet + * @return positive value if ipaddr greater than address in record + */ + int matchIPAddrWithRecord(const IPAddress & ipaddr, const IPRecord & record) const; + + IPRecordConstIt ipRecordNotFound() const; + IPRecordConstIt lookupIPRecord(const IPAddress & target) const; + const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; const bool require_nonempty; const std::string key_description{dict_struct.getKeyDescription()}; + ContainerType ip_records; + size_t total_ip_length; - btrie_t * trie = nullptr; std::map attribute_index_by_name; std::vector attributes; diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_special.reference b/tests/queries/0_stateless/01018_ddl_dictionaries_special.reference index c6c6993faa8..a6332b85f4e 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_special.reference +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_special.reference @@ -10,6 +10,11 @@ 0 ***ip trie dict*** 17501 +17501 +17502 +0 +11211 +11211 NP ***hierarchy dict*** Moscow diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql index ede5897bdf7..6c4a325a3b5 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql @@ -82,8 +82,7 @@ CREATE TABLE database_for_dict.table_ip_trie ) engine = TinyLog; -INSERT INTO database_for_dict.table_ip_trie VALUES ('202.79.32.0/20', 17501, 'NP'), ('2620:0:870::/48', 3856, 'US'), ('2a02:6b8:1::/48', 13238, 'RU'), ('2001:db8::/32', 65536, 'ZZ'); - +INSERT INTO database_for_dict.table_ip_trie VALUES ('202.79.32.0/20', 17501, 'NP'), ('202.79.32.2', 17502, 'NP'), ('101.79.55.22', 11211, 'UK'), ('2620:0:870::/48', 3856, 'US'), ('2a02:6b8:1::/48', 13238, 'RU'), ('2001:db8::/32', 65536, 'ZZ'); CREATE DICTIONARY database_for_dict.dict_ip_trie ( @@ -97,6 +96,11 @@ LAYOUT(IP_TRIE()) LIFETIME(MIN 10 MAX 100); SELECT dictGetUInt32('database_for_dict.dict_ip_trie', 'asn', tuple(IPv4StringToNum('202.79.32.0'))); +SELECT dictGetUInt32('database_for_dict.dict_ip_trie', 'asn', tuple(IPv4StringToNum('202.79.32.1'))); +SELECT dictGetUInt32('database_for_dict.dict_ip_trie', 'asn', tuple(IPv4StringToNum('202.79.32.2'))); +SELECT dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('654f:3716::'))); +SELECT dictGetUInt32('database_for_dict.dict_ip_trie', 'asn', tuple(IPv6StringToNum('::ffff:654f:3716'))); +SELECT dictGetUInt32('database_for_dict.dict_ip_trie', 'asn', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); SELECT dictGetString('database_for_dict.dict_ip_trie', 'cca2', tuple(IPv4StringToNum('202.79.32.0'))); SELECT '***hierarchy dict***'; From 81671ceb39c0d4bfedb1a36c298499aad345dd1c Mon Sep 17 00:00:00 2001 From: vdimir Date: Sun, 8 Nov 2020 21:32:22 +0300 Subject: [PATCH 0132/1088] Add tests/performance/ip_trie.xml [wip] --- tests/performance/ip_trie.xml | 66 +++++++++++++++++++++++++++++++++++ 1 file changed, 66 insertions(+) create mode 100644 tests/performance/ip_trie.xml diff --git a/tests/performance/ip_trie.xml b/tests/performance/ip_trie.xml new file mode 100644 index 00000000000..41b6ef99abb --- /dev/null +++ b/tests/performance/ip_trie.xml @@ -0,0 +1,66 @@ + + + CREATE TABLE table_ip_trie + ( + ip String, + val Float32 + ) ENGINE = TinyLog + + + + INSERT INTO table_ip_trie + SELECT + IPv4NumToString(ipv4) || '/' || toString(rand() % 32 + 1) as ip, + val + FROM generateRandom('ipv4 UInt32, val Float32', 0, 30, 30) + LIMIT 1000000 + + + + INSERT INTO table_ip_trie + SELECT + IPv6NumToString(ipv6) || '/' || toString(rand() % 128 + 1) as ip, + val + FROM generateRandom('ipv6 FixedString(16), val Float32', 0, 30, 30) + LIMIT 1000000 + + + + CREATE DICTIONARY dict_ip_trie + ( + ip String, + val Float32 + ) + PRIMARY KEY ip + SOURCE(CLICKHOUSE( + HOST 'localhost' + PORT 9000 + USER 'default' + DB 'default' + TABLE 'table_ip_trie')) + LAYOUT(IP_TRIE()) + LIFETIME(300) + + + + CREATE TABLE dict_ip_trie_table + ( + `id` String, + `val` Float32 + ) ENGINE = Dictionary(default.dict_ip_trie) + + + + SELECT dictGetFloat32('default.dict_ip_trie', 'value', tuple(rand32())) + FROM numbers(500000) + + + + SELECT dictGetFloat32('default.dict_ip_trie', 'value', tuple(randomFixedString(16))) + FROM numbers(500000) + + + DROP DICTIONARY IF EXISTS default.dict_ip_trie + DROP TABLE IF EXISTS table_ip_trie + DROP TABLE IF EXISTS dict_ip_trie_table + From c306902fdf2fe778dbe9d82063631814541855f6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Sun, 8 Nov 2020 23:21:13 +0300 Subject: [PATCH 0133/1088] Fix ip subnet comparison --- src/Dictionaries/TrieDictionary.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Dictionaries/TrieDictionary.cpp b/src/Dictionaries/TrieDictionary.cpp index a2e2bcf0bd8..3fd87e42445 100644 --- a/src/Dictionaries/TrieDictionary.cpp +++ b/src/Dictionaries/TrieDictionary.cpp @@ -365,8 +365,10 @@ void TrieDictionary::loadData() { if (a.addr.family() != b.addr.family()) return a.addr.family() < b.addr.family(); + + // prefer IPs with more narrow subnet if (a.addr == b.addr) - return a.prefix > b.prefix; + return a.prefix < b.prefix; return a.addr < b.addr; }); From 13685e83dfb34ed4c2fbda76eb0dd88610783f12 Mon Sep 17 00:00:00 2001 From: vdimir Date: Sun, 8 Nov 2020 23:50:49 +0300 Subject: [PATCH 0134/1088] Exact ip records comparison for getBlockInputStream --- src/Dictionaries/TrieDictionary.cpp | 82 +++++++++++++++++++++++------ src/Dictionaries/TrieDictionary.h | 1 + 2 files changed, 66 insertions(+), 17 deletions(-) diff --git a/src/Dictionaries/TrieDictionary.cpp b/src/Dictionaries/TrieDictionary.cpp index 3fd87e42445..28769780027 100644 --- a/src/Dictionaries/TrieDictionary.cpp +++ b/src/Dictionaries/TrieDictionary.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -27,13 +28,23 @@ namespace ErrorCodes static void validateKeyTypes(const DataTypes & key_types) { - if (key_types.size() != 1) - throw Exception{"Expected a single IP address", ErrorCodes::TYPE_MISMATCH}; + if (key_types.size() < 1 && 2 < key_types.size()) + throw Exception{"Expected a single IP address or IP with mask", + ErrorCodes::TYPE_MISMATCH}; - const auto & actual_type = key_types[0]->getName(); + if (key_types.size() == 1) + { + const auto & actual_type = key_types[0]->getName(); + if (actual_type != "UInt32" && actual_type != "FixedString(16)") + throw Exception{"Key does not match, expected either UInt32 or FixedString(16)", ErrorCodes::TYPE_MISMATCH}; + return; + } - if (actual_type != "UInt32" && actual_type != "FixedString(16)") - throw Exception{"Key does not match, expected either UInt32 or FixedString(16)", ErrorCodes::TYPE_MISMATCH}; + const auto * ip_col_type = typeid_cast(key_types[0].get()); + const auto * mask_col_type = typeid_cast(key_types[1].get()); + bool type_ok = ip_col_type && mask_col_type && ip_col_type->getN() == IPV6_BINARY_LENGTH; + if (!type_ok) + throw Exception{"Keys do not match, {FixedString(16), UInt8}", ErrorCodes::TYPE_MISMATCH}; } /// Create IPAddress from 16 byte array converting to ipv4 if possible @@ -361,16 +372,7 @@ void TrieDictionary::loadData() LOG_TRACE(logger, "{} ip records are read", ip_records.size()); - std::sort(ip_records.begin(), ip_records.end(), [](const auto & a, const auto & b) - { - if (a.addr.family() != b.addr.family()) - return a.addr.family() < b.addr.family(); - - // prefer IPs with more narrow subnet - if (a.addr == b.addr) - return a.prefix < b.prefix; - return a.addr < b.addr; - }); + std::sort(ip_records.begin(), ip_records.end(), lessIPRecords); stream->readSuffix(); @@ -531,6 +533,39 @@ void TrieDictionary::getItemsImpl( const auto first_column = key_columns.front(); const auto rows = first_column->size(); + // special case for getBlockInputStream + if (unlikely(key_columns.size() == 2)) + { + const auto & ip_column = assert_cast(*key_columns.front()); + const auto & mask_column = assert_cast &>(*key_columns.back()); + + for (const auto i : ext::range(0, rows)) + { + const auto second_column = key_columns.back(); + + auto addr_data = ip_column.getDataAt(i).data; + auto ipaddr = ip4or6fromBytes(reinterpret_cast(addr_data)); + + UInt8 mask = mask_column.getElement(i); + + auto target = IPRecord{ipaddr, mask, 0}; + auto found_it = std::lower_bound(ip_records.begin(), ip_records.end(), target, lessIPRecords); + + if (likely(found_it != ip_records.end() && + found_it->addr == target.addr && + found_it->prefix == target.prefix)) + { + set_value(i, static_cast(vec[found_it->row])); + } + else + { + set_value(i, get_default(i)); + } + } + query_count.fetch_add(rows, std::memory_order_relaxed); + return; + } + if (first_column->isNumeric()) { for (const auto i : ext::range(0, rows)) @@ -671,8 +706,10 @@ BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_nam auto get_keys = [](const Columns & columns, const std::vector & dict_attributes) { const auto & attr = dict_attributes.front(); - return ColumnsWithTypeAndName( - {ColumnWithTypeAndName(columns.front(), std::make_shared(IPV6_BINARY_LENGTH), attr.name)}); + return ColumnsWithTypeAndName({ + ColumnWithTypeAndName(columns.front(), std::make_shared(IPV6_BINARY_LENGTH), attr.name), + ColumnWithTypeAndName(columns.back(), std::make_shared(), attr.name + ".mask") + }); }; auto get_view = [](const Columns & columns, const std::vector & dict_attributes) { @@ -709,6 +746,17 @@ int TrieDictionary::matchIPAddrWithRecord(const IPAddress & ipaddr, const IPReco return 1; } +bool TrieDictionary::lessIPRecords(const IPRecord & a, const IPRecord & b) const +{ + if (a.addr.family() != b.addr.family()) + return a.addr.family() < b.addr.family(); + + // prefer IPs with more narrow subnet + if (a.addr == b.addr) + return a.prefix < b.prefix; + return a.addr < b.addr; +} + TrieDictionary::IPRecordConstIt TrieDictionary::ipRecordNotFound() const { return ip_records.end(); diff --git a/src/Dictionaries/TrieDictionary.h b/src/Dictionaries/TrieDictionary.h index 891dadd3be8..9fe304a786a 100644 --- a/src/Dictionaries/TrieDictionary.h +++ b/src/Dictionaries/TrieDictionary.h @@ -243,6 +243,7 @@ private: * @return positive value if ipaddr greater than address in record */ int matchIPAddrWithRecord(const IPAddress & ipaddr, const IPRecord & record) const; + bool lessIPRecords(const IPRecord & a, const IPRecord & b) const; IPRecordConstIt ipRecordNotFound() const; IPRecordConstIt lookupIPRecord(const IPAddress & target) const; From 08a266cf72e23126213354dd0e6f3d625fa675f1 Mon Sep 17 00:00:00 2001 From: Grigory Buteyko Date: Mon, 9 Nov 2020 00:44:41 +0300 Subject: [PATCH 0135/1088] removed dependence of invariants on floating point black box properties. Now more than 2048 centroids will never be written, on read, however, we allow more centroids and just compress them. if there is too much. Thus we allow loading TDigests with different error, as number of centroids is dependent on error. --- src/AggregateFunctions/QuantileTDigest.h | 224 +++++++++++++---------- 1 file changed, 124 insertions(+), 100 deletions(-) diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index c547a93a257..ad2121f7f72 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -65,21 +65,29 @@ class TDigest /** :param epsilon: value \delta from the article - error in the range * quantile 0.5 (default is 0.01, i.e. 1%) + * if you change epsilon, you must also change max_centroids + * :param max_centroids: depends on epsilon, the better accuracy, the more centroids you need + * to describe data with this accuracy. Read article before changing. * :param max_unmerged: when accumulating count of new points beyond this * value centroid compression is triggered * (default is 2048, the higher the value - the * more memory is required, but amortization of execution time increases) + * Change freely anytime. */ struct Params { Value epsilon = 0.01; + size_t max_centroids = 2048; size_t max_unmerged = 2048; }; + /** max_centroids_deserialize should be >= all max_centroids ever used in production. + * This is security parameter, preventing allocation of too much centroids in deserialize, so can be relatively large. + */ + static constexpr size_t max_centroids_deserialize = 65536; - Params params; + static constexpr Params params{}; - /// The memory will be allocated to several elements at once, so that the state occupies 64 bytes. - static constexpr size_t bytes_in_arena = 128 - sizeof(PODArray) - sizeof(Count) - sizeof(UInt32); + static constexpr size_t bytes_in_arena = 128 - sizeof(PODArray) - sizeof(BetterFloat) - sizeof(size_t); // If alignment is imperfect, sizeof(TDigest) will be more than naively expected using Centroids = PODArrayWithStackMemory; Centroids centroids; @@ -112,9 +120,48 @@ class TDigest centroids.push_back(c); count += c.count; ++unmerged; - if (unmerged >= params.max_unmerged) + if (unmerged > params.max_unmerged) compress(); } + void compress_brute() { + if (centroids.size() <= params.max_centroids) + return; + const size_t batch_size = (centroids.size() + params.max_centroids - 1) / params.max_centroids; // at least 2 + + auto l = centroids.begin(); + auto r = std::next(l); + BetterFloat sum = 0; + BetterFloat l_mean = l->mean; // We have high-precision temporaries for numeric stability + BetterFloat l_count = l->count; + size_t batch_pos = 0; + for (;r != centroids.end(); ++r) + { + if (batch_pos < batch_size - 1) + { + /// The left column "eats" the right. Middle of the batch + l_count += r->count; + l_mean += r->count * (r->mean - l_mean) / l_count; // Symmetric algo (M1*C1 + M2*C2)/(C1+C2) is numerically better, but slower + l->mean = l_mean; + l->count = l_count; + batch_pos += 1; + } + else + { + // End of the batch, start the next one + sum += l->count; // Not l_count, otherwise actual sum of elements will be different + ++l; + + /// We skip all the values "eaten" earlier. + *l = *r; + l_mean = l->mean; + l_count = l->count; + batch_pos = 0; + } + } + count = sum + l_count; // Update count, it might be different due to += inaccuracy + centroids.resize(l - centroids.begin() + 1); + // Here centroids.size() <= params.max_centroids + } public: /** Performs compression of accumulated centroids @@ -123,89 +170,89 @@ public: */ void compress() { - if (unmerged > 0) + if (unmerged > 0 || centroids.size() > params.max_centroids) { + // unmerged > 0 implies centroids.size() > 0, hence *l is valid below RadixSort::executeLSD(centroids.data(), centroids.size()); - if (centroids.size() > 3) + /// A pair of consecutive bars of the histogram. + auto l = centroids.begin(); + auto r = std::next(l); + + const BetterFloat count_epsilon_4 = count * params.epsilon * 4; // Compiler is unable to do this optimization + BetterFloat sum = 0; + BetterFloat l_mean = l->mean; // We have high-precision temporaries for numeric stability + BetterFloat l_count = l->count; + while (r != centroids.end()) { - /// A pair of consecutive bars of the histogram. - auto l = centroids.begin(); - auto r = std::next(l); - - const BetterFloat count_epsilon_4 = count * params.epsilon * 4; // Compiler is unable to do this optimization - BetterFloat sum = 0; - BetterFloat l_mean = l->mean; // We have high-precision temporaries for numeric stability - BetterFloat l_count = l->count; - while (r != centroids.end()) + if (l->mean == r->mean) // Perfect aggregation (fast). We compare l->mean, not l_mean, to avoid identical elements after compress { - if (l->mean == r->mean) // Perfect aggregation (fast). We compare l->mean, not l_mean, to avoid identical elements after compress - { - l_count += r->count; - l->count = l_count; - ++r; - continue; - } - // we use quantile which gives us the smallest error - - /// The ratio of the part of the histogram to l, including the half l to the entire histogram. That is, what level quantile in position l. - BetterFloat ql = (sum + l_count * 0.5) / count; - BetterFloat err = ql * (1 - ql); - - /// The ratio of the portion of the histogram to l, including l and half r to the entire histogram. That is, what level is the quantile in position r. - BetterFloat qr = (sum + l_count + r->count * 0.5) / count; - BetterFloat err2 = qr * (1 - qr); - - if (err > err2) - err = err2; - - BetterFloat k = count_epsilon_4 * err; - - /** The ratio of the weight of the glued column pair to all values is not greater, - * than epsilon multiply by a certain quadratic coefficient, which in the median is 1 (4 * 1/2 * 1/2), - * and at the edges decreases and is approximately equal to the distance to the edge * 4. - */ - - if (l_count + r->count <= k) - { - // it is possible to merge left and right - /// The left column "eats" the right. - l_count += r->count; - l_mean += r->count * (r->mean - l_mean) / l_count; // Symmetric algo (M1*C1 + M2*C2)/(C1+C2) is numerically better, but slower - l->mean = l_mean; - l->count = l_count; - } - else - { - // not enough capacity, check the next pair - sum += l->count; // Not l_count, otherwise actual sum of elements will be different - ++l; - - /// We skip all the values "eaten" earlier. - if (l != r) - *l = *r; - l_mean = l->mean; - l_count = l->count; - } + l_count += r->count; + l->count = l_count; ++r; + continue; } - count = sum + l_count; // Update count, it might be different due to += inaccuracy + // we use quantile which gives us the smallest error - /// At the end of the loop, all values to the right of l were "eaten". - centroids.resize(l - centroids.begin() + 1); + /// The ratio of the part of the histogram to l, including the half l to the entire histogram. That is, what level quantile in position l. + BetterFloat ql = (sum + l_count * 0.5) / count; + BetterFloat err = ql * (1 - ql); + + /// The ratio of the portion of the histogram to l, including l and half r to the entire histogram. That is, what level is the quantile in position r. + BetterFloat qr = (sum + l_count + r->count * 0.5) / count; + BetterFloat err2 = qr * (1 - qr); + + if (err > err2) + err = err2; + + BetterFloat k = count_epsilon_4 * err; + + /** The ratio of the weight of the glued column pair to all values is not greater, + * than epsilon multiply by a certain quadratic coefficient, which in the median is 1 (4 * 1/2 * 1/2), + * and at the edges decreases and is approximately equal to the distance to the edge * 4. + */ + + if (l_count + r->count <= k) + { + // it is possible to merge left and right + /// The left column "eats" the right. + l_count += r->count; + l_mean += r->count * (r->mean - l_mean) / l_count; // Symmetric algo (M1*C1 + M2*C2)/(C1+C2) is numerically better, but slower + l->mean = l_mean; + l->count = l_count; + } + else + { + // not enough capacity, check the next pair + sum += l->count; // Not l_count, otherwise actual sum of elements will be different + ++l; + + /// We skip all the values "eaten" earlier. + if (l != r) + *l = *r; + l_mean = l->mean; + l_count = l->count; + } + ++r; } + count = sum + l_count; // Update count, it might be different due to += inaccuracy + /// At the end of the loop, all values to the right of l were "eaten". + centroids.resize(l - centroids.begin() + 1); unmerged = 0; } + // Ensures centroids.size() < max_centroids, independent of unprovable floating point blackbox above + compress_brute(); } /** Adds to the digest a change in `x` with a weight of `cnt` (default 1) */ void add(T x, UInt64 cnt = 1) { - if (cnt == 0) - return; // Count 0 breaks compress() assumptions, we treat it as no sample - addCentroid(Centroid(Value(x), Count(cnt))); + auto vx = static_cast(x); + if (cnt == 0 || std::isnan(vx)) + return; // Count 0 breaks compress() assumptions, Nan breaks sort(). We treat them as no sample. + addCentroid(Centroid{vx, static_cast(cnt)}); } void merge(const TDigest & other) @@ -226,23 +273,23 @@ public: size_t size = 0; readVarUInt(size, buf); - if (size > params.max_unmerged) + if (size > max_centroids_deserialize) throw Exception("Too large t-digest centroids size", ErrorCodes::TOO_LARGE_ARRAY_SIZE); + count = 0; + unmerged = 0; + centroids.resize(size); + // From now, TDigest will be in invalid state if exception is thrown. buf.read(reinterpret_cast(centroids.data()), size * sizeof(centroids[0])); - count = 0; - for (size_t i = 0; i != centroids.size(); ++i) + for (const auto & c : centroids) { - Centroid & c = centroids[i]; if (c.count <= 0 || std::isnan(c.count) || std::isnan(c.mean)) // invalid count breaks compress(), invalid mean breaks sort() - { - centroids.resize(i); // Exception safety, without this line caller will end up with TDigest object in broken invariant state throw std::runtime_error("Invalid centroid " + std::to_string(c.count) + ":" + std::to_string(c.mean)); - } count += c.count; } + compress(); // Allows reading/writing TDigests with different epsilon/max_centroids params } Count getCount() @@ -269,14 +316,7 @@ class QuantileTDigest using Value = Float32; using Count = Float32; - /** We store two t-digests. When an amount of elements in sub_tdigest become more than merge_threshold - * we merge sub_tdigest in main_tdigest and reset sub_tdigest. This method is needed to decrease an amount of - * centroids in t-digest (experiments show that after merge_threshold the size of t-digest significantly grows, - * but merging two big t-digest decreases it). - */ TDigest main_tdigest; - TDigest sub_tdigest; - size_t merge_threshold = 1e7; /** Linear interpolation at the point x on the line (x1, y1)..(x2, y2) */ @@ -286,36 +326,24 @@ class QuantileTDigest return y1 + k * (y2 - y1); } - void mergeTDigests() - { - main_tdigest.merge(sub_tdigest); - sub_tdigest.reset(); - } - public: void add(T x, UInt64 cnt = 1) { - if (sub_tdigest.getCount() >= merge_threshold) - mergeTDigests(); - sub_tdigest.add(x, cnt); + main_tdigest.add(x, cnt); } void merge(const QuantileTDigest & other) { - mergeTDigests(); main_tdigest.merge(other.main_tdigest); - main_tdigest.merge(other.sub_tdigest); } void serialize(WriteBuffer & buf) { - mergeTDigests(); main_tdigest.serialize(buf); } void deserialize(ReadBuffer & buf) { - sub_tdigest.reset(); main_tdigest.deserialize(buf); } @@ -325,8 +353,6 @@ public: template ResultType getImpl(Float64 level) { - mergeTDigests(); - auto & centroids = main_tdigest.getCentroids(); if (centroids.empty()) return std::is_floating_point_v ? NAN : 0; @@ -364,8 +390,6 @@ public: template void getManyImpl(const Float64 * levels, const size_t * levels_permutation, size_t size, ResultType * result) { - mergeTDigests(); - auto & centroids = main_tdigest.getCentroids(); if (centroids.empty()) { From 1726fb0f4e43f6b99e191529689382af3125bcbc Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 9 Nov 2020 12:40:28 +0800 Subject: [PATCH 0136/1088] ISSUES-16574 try fix test failure --- src/AggregateFunctions/AggregateFunctionIf.cpp | 2 +- tests/queries/0_stateless/01556_if_null.reference | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index d654387e5db..276abb90920 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -195,7 +195,7 @@ AggregateFunctionPtr AggregateFunctionIf::getOwnNullAdapter( if (serialize_flag) return std::make_shared>(nested_function, arguments, params); else - return std::make_shared>(nested_function, arguments, params); + return std::make_shared>(nested_function, arguments, params); } } } diff --git a/tests/queries/0_stateless/01556_if_null.reference b/tests/queries/0_stateless/01556_if_null.reference index a0c5e7faf40..adc63ecf47b 100644 --- a/tests/queries/0_stateless/01556_if_null.reference +++ b/tests/queries/0_stateless/01556_if_null.reference @@ -1 +1 @@ -([1],[5]) 4 4 +([1],[4]) 4 4 From ad8eac492919915dac9219bd64cd6d69d4d3b792 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 9 Nov 2020 13:58:32 +0800 Subject: [PATCH 0137/1088] build error and style error fix --- src/Parsers/ASTColumnsTransformers.cpp | 10 +++++----- src/Parsers/ExpressionElementParsers.cpp | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index b28644383d1..7af9786bfdb 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -43,7 +43,7 @@ void ASTColumnsApplyTransformer::formatImpl(const FormatSettings & settings, For parameters->formatImpl(settings, state, frame); if (!column_name_prefix.empty()) - settings.ostr << ", '" << column_name_prefix << "')"; + settings.ostr << ", '" << column_name_prefix << "')"; } void ASTColumnsApplyTransformer::transform(ASTs & nodes) const @@ -214,12 +214,12 @@ void ASTColumnsReplaceTransformer::transform(ASTs & nodes) const if (is_strict && !replace_map.empty()) { - String expected_columns = ""; - for (auto it = replace_map.begin(); it != replace_map.end(); ++it) + String expected_columns; + for (auto & elem: replace_map) { - if (expected_columns != "") + if (!expected_columns.empty()) expected_columns += ", "; - expected_columns += it->first; + expected_columns += elem.first; } throw Exception( "Columns transformer REPLACE expects following column(s) : " + expected_columns, diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 0a0fca53545..47fc1423d3e 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1317,7 +1317,7 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e if (pos->type != TokenType::ClosingRoundBracket) return false; ++pos; - } + } auto res = std::make_shared(); res->func_name = getIdentifierName(func_name); From 5622882553ce92fc0abf6cd308c533e54b74a9dc Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 9 Nov 2020 14:21:50 +0800 Subject: [PATCH 0138/1088] remove unused code --- src/Parsers/ASTColumnsTransformers.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 7af9786bfdb..2d4d2304bd7 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -48,7 +48,6 @@ void ASTColumnsApplyTransformer::formatImpl(const FormatSettings & settings, For void ASTColumnsApplyTransformer::transform(ASTs & nodes) const { - std::cout << "\033[31m" << __FILE__ << ":"<<__LINE__ << "\033[39m" << std::endl; for (auto & column : nodes) { String name; From 456b0b94c1661a37d52b2261e6725710c693f393 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 9 Nov 2020 09:54:35 +0300 Subject: [PATCH 0139/1088] Remove some debug --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 2 -- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 3 --- 2 files changed, 5 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 5ab22474d72..b1a11b2d473 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -37,8 +37,6 @@ static void write(bool x, WriteBuffer & out) static void write(const String & s, WriteBuffer & out) { - - LOG_DEBUG(&Poco::Logger::get("LOG"), "S SIZE {}", s.size()); write(int32_t(s.size()), out); out.write(s.data(), s.size()); } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 30b30695eea..33204686dce 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -751,12 +751,9 @@ void ZooKeeper::receiveEvent() Error err; read(length); - std::cerr << "RECEIVED LENGTH " << length << std::endl; size_t count_before_event = in->count(); read(xid); - std::cerr << "RECEIVED XID " << xid << std::endl; read(zxid); - std::cerr << "RECEIVED ZXID " << zxid << std::endl; read(err); RequestInfo request_info; From 2b240029c148e3bfdd1bf534217b54979f573146 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 9 Nov 2020 17:13:27 +0800 Subject: [PATCH 0140/1088] Add space for format Impl --- src/Parsers/ASTColumnsTransformers.cpp | 41 +++++++++++++------------- 1 file changed, 20 insertions(+), 21 deletions(-) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 2d4d2304bd7..9ae32aa3f74 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -71,10 +71,10 @@ void ASTColumnsApplyTransformer::transform(ASTs & nodes) const void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXCEPT" << (is_strict ? " STRICT " : "") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXCEPT" << (is_strict ? " STRICT " : " ") << (settings.hilite ? hilite_none : ""); if (children.size() > 1) - settings.ostr << " ("; + settings.ostr << "("; for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { @@ -93,26 +93,25 @@ void ASTColumnsExceptTransformer::transform(ASTs & nodes) const { ASTs expected_columns(children); - nodes.erase( - std::remove_if( - nodes.begin(), - nodes.end(), - [&](const ASTPtr & node_child) + for (auto it = nodes.begin(); it != nodes.end();) + { + bool removed = false; + if (const auto * id = it->get()->as()) + { + for (int i = expected_columns.size() - 1; i >= 0; --i) { - if (const auto * id = node_child->as()) + if (expected_columns[i]->as().name() == id->shortName()) { - for (int i = expected_columns.size() - 1; i >= 0; --i) - { - if (expected_columns[i]->as().name() == id->shortName()) - { - expected_columns.erase(expected_columns.begin() + i); - return true; - } - } + removed = true; + expected_columns.erase(expected_columns.begin() + i); + it = nodes.erase(it); } - return false; - }), - nodes.end()); + } + } + + if (!removed) + ++it; + } if (is_strict && !expected_columns.empty()) { @@ -139,10 +138,10 @@ void ASTColumnsReplaceTransformer::Replacement::formatImpl( void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "REPLACE" << (is_strict ? " STRICT " : "") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "REPLACE" << (is_strict ? " STRICT " : " ") << (settings.hilite ? hilite_none : ""); if (children.size() > 1) - settings.ostr << " ("; + settings.ostr << "("; for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { From 26229ed231fc2dcb3dc3e17265197574881d9fff Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 9 Nov 2020 18:07:38 +0300 Subject: [PATCH 0141/1088] tmp spans for threads (doesn't compile because of json metadata changes) --- src/Common/ThreadStatus.cpp | 1 + src/Common/ThreadStatus.h | 82 +++++++++-------- src/Core/Settings.h | 2 +- src/Formats/FormatSettings.h | 2 + src/Interpreters/InterpreterFactory.cpp | 3 + src/Interpreters/OpenTelemetrySpanLog.cpp | 92 +++++++++++++++++++ src/Interpreters/OpenTelemetrySpanLog.h | 10 ++ src/Interpreters/ThreadStatusExt.cpp | 29 ++++++ src/Interpreters/executeQuery.cpp | 6 +- src/Parsers/parseQuery.cpp | 2 + src/Processors/Executors/PipelineExecutor.cpp | 2 + src/Server/TCPHandler.cpp | 3 + 12 files changed, 195 insertions(+), 39 deletions(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index f5ad28f57af..c7551d42dfe 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 1b4d20e9721..934654cf385 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -31,6 +31,7 @@ class ThreadStatus; class QueryProfilerReal; class QueryProfilerCpu; class QueryThreadLog; +struct OpenTelemetrySpanHolder; class TasksStatsCounters; struct RUsageCounters; struct PerfEventsCounters; @@ -86,9 +87,6 @@ extern thread_local ThreadStatus * current_thread; class ThreadStatus : public boost::noncopyable { public: - ThreadStatus(); - ~ThreadStatus(); - /// Linux's PID (or TGID) (the same id is shown by ps util) const UInt64 thread_id = 0; /// Also called "nice" value. If it was changed to non-zero (when attaching query) - will be reset to zero when query is detached. @@ -110,6 +108,50 @@ public: using Deleter = std::function; Deleter deleter; + __uint128_t opentelemetry_trace_id; + UInt64 opentelemetry_current_span_id; + std::unique_ptr opentelemetry_thread_span; + +protected: + ThreadGroupStatusPtr thread_group; + + std::atomic thread_state{ThreadState::DetachedFromQuery}; + + /// Is set once + Context * global_context = nullptr; + /// Use it only from current thread + Context * query_context = nullptr; + + String query_id; + + /// A logs queue used by TCPHandler to pass logs to a client + InternalTextLogsQueueWeakPtr logs_queue_ptr; + + bool performance_counters_finalized = false; + UInt64 query_start_time_nanoseconds = 0; + UInt64 query_start_time_microseconds = 0; + time_t query_start_time = 0; + size_t queries_started = 0; + + // CPU and Real time query profilers + std::unique_ptr query_profiler_real; + std::unique_ptr query_profiler_cpu; + + Poco::Logger * log = nullptr; + + friend class CurrentThread; + + /// Use ptr not to add extra dependencies in the header + std::unique_ptr last_rusage; + std::unique_ptr taskstats; + + /// Is used to send logs from logs_queue to client in case of fatal errors. + std::function fatal_error_callback; + +public: + ThreadStatus(); + ~ThreadStatus(); + ThreadGroupStatusPtr getThreadGroup() const { return thread_group; @@ -176,40 +218,6 @@ protected: void assertState(const std::initializer_list & permitted_states, const char * description = nullptr) const; - ThreadGroupStatusPtr thread_group; - - std::atomic thread_state{ThreadState::DetachedFromQuery}; - - /// Is set once - Context * global_context = nullptr; - /// Use it only from current thread - Context * query_context = nullptr; - - String query_id; - - /// A logs queue used by TCPHandler to pass logs to a client - InternalTextLogsQueueWeakPtr logs_queue_ptr; - - bool performance_counters_finalized = false; - UInt64 query_start_time_nanoseconds = 0; - UInt64 query_start_time_microseconds = 0; - time_t query_start_time = 0; - size_t queries_started = 0; - - // CPU and Real time query profilers - std::unique_ptr query_profiler_real; - std::unique_ptr query_profiler_cpu; - - Poco::Logger * log = nullptr; - - friend class CurrentThread; - - /// Use ptr not to add extra dependencies in the header - std::unique_ptr last_rusage; - std::unique_ptr taskstats; - - /// Is used to send logs from logs_queue to client in case of fatal errors. - std::function fatal_error_callback; private: void setupState(const ThreadGroupStatusPtr & thread_group_); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f974a514bc0..71b248edfdd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -517,7 +517,7 @@ struct Settings : public BaseSettings }; /* - * User-specified file format settings for File and ULR engines. + * User-specified file format settings for File and URL engines. */ DECLARE_SETTINGS_TRAITS(FormatFactorySettingsTraits, FORMAT_FACTORY_SETTINGS) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index b3c01ddcf14..0a897c96896 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -86,9 +86,11 @@ struct FormatSettings struct { + bool array_of_rows = false; bool quote_64bit_integers = true; bool quote_denormals = true; bool escape_forward_slashes = true; + bool named_tuple_as_object = false; bool serialize_as_strings = false; } json; diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index a18d3ab8a6f..7505c017953 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -66,6 +66,7 @@ #include #include #include +#include #include @@ -93,6 +94,8 @@ namespace ErrorCodes std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & context, QueryProcessingStage::Enum stage) { + OpenTelemetrySpanHolder span(__FUNCTION__); + ProfileEvents::increment(ProfileEvents::Query); if (query->as()) diff --git a/src/Interpreters/OpenTelemetrySpanLog.cpp b/src/Interpreters/OpenTelemetrySpanLog.cpp index 6c22165546d..013913d9f2d 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.cpp +++ b/src/Interpreters/OpenTelemetrySpanLog.cpp @@ -43,5 +43,97 @@ void OpenTelemetrySpanLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(attribute_values); } +OpenTelemetrySpanHolder::OpenTelemetrySpanHolder(const std::string & _operation_name) +{ + auto & thread = CurrentThread::get(); + + trace_id = thread.opentelemetry_trace_id; + if (!trace_id) + { + return; + } + + parent_span_id = thread.opentelemetry_current_span_id; + span_id = thread_local_rng(); + operation_name = _operation_name; + start_time_us = std::chrono::duration_cast( + std::chrono::system_clock::now().time_since_epoch()).count(); + + // *** remove this + attribute_names.push_back("start.stacktrace"); + attribute_values.push_back(StackTrace().toString()); + + thread.opentelemetry_current_span_id = span_id; +} + +OpenTelemetrySpanHolder::~OpenTelemetrySpanHolder() +{ + try + { + fmt::print(stderr, "{}\n", StackTrace().toString()); + + if (!trace_id) + { + return; + } + + // First of all, return old value of current span. + auto & thread = CurrentThread::get(); + assert(thread.opentelemetry_current_span_id = span_id); + thread.opentelemetry_current_span_id = parent_span_id; + + // Not sure what's the best way to access the log from here. + auto * thread_group = CurrentThread::getGroup().get(); + // Not sure whether and when this can be null. + if (!thread_group) + { + return; + } + + fmt::print(stderr, "1\n"); + + auto * context = thread_group->query_context; + if (!context) + { + // Both global and query contexts can be null when executing a + // background task, and global context can be null for some + // queries. + return; + } + + //******** remove this + attribute_names.push_back("clickhouse.query_id"); + attribute_values.push_back(context->getCurrentQueryId()); + + fmt::print(stderr, "2\n"); + + auto log = context->getOpenTelemetrySpanLog(); + if (!log) + { + // The log might be disabled. + return; + } + + fmt::print(stderr, "3\n"); + + finish_time_us = std::chrono::duration_cast( + std::chrono::system_clock::now().time_since_epoch()).count(); + + // We should use a high resolution monotonic clock for calculating + // duration, but this way will do for now. + duration_ns = (finish_time_us - start_time_us) * 1000; + + + log->add(OpenTelemetrySpanLogElement( + static_cast(*this))); + + fmt::print(stderr, "4\n"); + } + catch (...) + { + tryLogCurrentException(__FUNCTION__); + } +} + } diff --git a/src/Interpreters/OpenTelemetrySpanLog.h b/src/Interpreters/OpenTelemetrySpanLog.h index 271d02804f4..a1c198559b8 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.h +++ b/src/Interpreters/OpenTelemetrySpanLog.h @@ -23,6 +23,10 @@ struct OpenTelemetrySpan struct OpenTelemetrySpanLogElement : public OpenTelemetrySpan { + OpenTelemetrySpanLogElement() = default; + OpenTelemetrySpanLogElement(const OpenTelemetrySpan & span) + : OpenTelemetrySpan(span) {} + static std::string name() { return "OpenTelemetrySpanLog"; } static Block createBlock(); void appendToBlock(MutableColumns & columns) const; @@ -36,4 +40,10 @@ public: using SystemLog::SystemLog; }; +struct OpenTelemetrySpanHolder : public OpenTelemetrySpan +{ + OpenTelemetrySpanHolder(const std::string & _operation_name); + ~OpenTelemetrySpanHolder(); +}; + } diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index adb9a38b10d..1689a1598ba 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -108,8 +109,26 @@ void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_) } if (query_context) + { applyQuerySettings(); + opentelemetry_trace_id = query_context->getClientInfo().opentelemetry_trace_id; + opentelemetry_current_span_id = query_context->getClientInfo().opentelemetry_span_id; + + if (opentelemetry_trace_id) + { + // Register the span for our thread. We might not know the name yet + // -- there are no strong constraints on when it is set relative to + // attaching the thread to query. Will set the name when the span ends. + opentelemetry_thread_span.reset(new OpenTelemetrySpanHolder("")); + } + } + else + { + opentelemetry_trace_id = 0; + opentelemetry_current_span_id = 0; + } + initPerformanceCounters(); thread_state = ThreadState::AttachedToQuery; @@ -300,6 +319,13 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) assertState({ThreadState::AttachedToQuery}, __PRETTY_FUNCTION__); + if (opentelemetry_thread_span) + { + opentelemetry_thread_span->operation_name = getThreadName(); + opentelemetry_thread_span->attribute_names.push_back("clickhouse.thread_id"); + opentelemetry_thread_span->attribute_values.push_back(thread_id); + } + finalizeQueryProfiler(); finalizePerformanceCounters(); @@ -312,6 +338,9 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) query_id.clear(); query_context = nullptr; + opentelemetry_thread_span = nullptr; + opentelemetry_trace_id = 0; + opentelemetry_current_span_id = 0; thread_group.reset(); thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index cdb3d9b7d7b..dbe3d23ca40 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -479,7 +479,11 @@ static std::tuple executeQueryImpl( limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); } - res = interpreter->execute(); + { + OpenTelemetrySpanHolder span("execute interpreter"); + res = interpreter->execute(); + } + QueryPipeline & pipeline = res.pipeline; bool use_processors = pipeline.initialized(); diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index f4e4c195506..5ecdd091e11 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -1,4 +1,6 @@ #include + +#include #include #include #include diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 271903add86..f940ea148c7 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #ifndef NDEBUG #include @@ -75,6 +76,7 @@ static void executeJob(IProcessor * processor) { try { + OpenTelemetrySpanHolder span(demangle(typeid(*processor).name())); processor->work(); } catch (Exception & exception) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 4dceb0aa905..6f484f6df91 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -521,6 +522,8 @@ void TCPHandler::processInsertQuery(const Settings & connection_settings) void TCPHandler::processOrdinaryQuery() { + OpenTelemetrySpanHolder span(__FUNCTION__); + /// Pull query execution result, if exists, and send it to network. if (state.io.in) { From 0568daf4fc03256051788563bd86ffa74f62b221 Mon Sep 17 00:00:00 2001 From: Aleksandrov Vladimir Date: Mon, 9 Nov 2020 18:55:41 +0300 Subject: [PATCH 0142/1088] Update collapsingmergetree.md formatting fix --- .../table-engines/mergetree-family/collapsingmergetree.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md b/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md index 4bfb9dc200e..ea0b265d652 100644 --- a/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md @@ -273,13 +273,15 @@ SELECT sum(Duration) AS Duration FROM UAct GROUP BY UserID -```text +``` + +``` text ┌──────────────UserID─┬─PageViews─┬─Duration─┐ │ 4324182021466249494 │ 6 │ 185 │ └─────────────────────┴───────────┴──────────┘ ``` -``` sqk +``` sql select count() FROM UAct ``` From b4c933e585a482a31450252078cadd442310ba12 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 9 Nov 2020 15:44:11 +0000 Subject: [PATCH 0143/1088] fix fix test --- .../InterpreterSelectWithUnionQuery.cpp | 250 +++++++++--------- .../InterpreterSelectWithUnionQuery.h | 1 - src/Parsers/ASTSelectWithUnionQuery.cpp | 27 +- src/Parsers/ASTSelectWithUnionQuery.h | 4 + src/Parsers/ExpressionListParsers.cpp | 2 +- src/Parsers/ParserSelectWithUnionQuery.cpp | 100 +------ ...t_and_setting_union_default_mode.reference | 40 +++ ...istinct_and_setting_union_default_mode.sql | 33 +++ ..._explain_select_with_union_query.reference | 162 ++++++++++-- .../01556_explain_select_with_union_query.sql | 30 ++- 10 files changed, 396 insertions(+), 253 deletions(-) diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 976dcaddd9c..f12c949eb92 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -11,6 +11,8 @@ #include #include +#include + namespace DB { @@ -21,11 +23,120 @@ namespace ErrorCodes extern const int EXPECTED_ALL_OR_DISTINCT; } +struct CustomizeASTSelectWithUnionQueryNormalize +{ + using TypeToVisit = ASTSelectWithUnionQuery; + + const UnionMode & union_default_mode; + + static void getSelectsFromUnionListNode(ASTPtr & ast_select, ASTs & selects) + { + if (auto * inner_union = ast_select->as()) + { + /// We need flatten from last to first + for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend(); + ++child) + getSelectsFromUnionListNode(*child, selects); + + return; + } + + selects.push_back(std::move(ast_select)); + } + + void visit(ASTSelectWithUnionQuery & ast, ASTPtr &) + { + auto & union_modes = ast.list_of_modes; + ASTs selects; + auto & select_list = ast.list_of_selects->children; + + int i; + for (i = union_modes.size() - 1; i >= 0; --i) + { + /// Rewrite UNION Mode + if (union_modes[i] == ASTSelectWithUnionQuery::Mode::Unspecified) + { + if (union_default_mode == UnionMode::ALL) + union_modes[i] = ASTSelectWithUnionQuery::Mode::ALL; + else if (union_default_mode == UnionMode::DISTINCT) + union_modes[i] = ASTSelectWithUnionQuery::Mode::DISTINCT; + else + throw Exception( + "Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty", + DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); + } + + if (union_modes[i] == ASTSelectWithUnionQuery::Mode::ALL) + { + if (auto * inner_union = select_list[i + 1]->as()) + { + /// Inner_union is an UNION ALL list, just lift up + for (auto child = inner_union->list_of_selects->children.rbegin(); + child != inner_union->list_of_selects->children.rend(); + ++child) + selects.push_back(std::move(*child)); + } + else + selects.push_back(std::move(select_list[i + 1])); + } + /// flatten all left nodes and current node to a UNION DISTINCT list + else if (union_modes[i] == ASTSelectWithUnionQuery::Mode::DISTINCT) + { + auto distinct_list = std::make_shared(); + distinct_list->list_of_selects = std::make_shared(); + distinct_list->children.push_back(distinct_list->list_of_selects); + for (int j = i + 1; j >= 0; j--) + { + getSelectsFromUnionListNode(select_list[j], distinct_list->list_of_selects->children); + } + distinct_list->union_mode = ASTSelectWithUnionQuery::Mode::DISTINCT; + // Reverse children list + std::reverse(distinct_list->list_of_selects->children.begin(), distinct_list->list_of_selects->children.end()); + distinct_list->is_normalized = true; + selects.push_back(std::move(distinct_list)); + break; + } + } + + /// No UNION DISTINCT or only one child in select_list + if (i == -1) + { + if (auto * inner_union = select_list[0]->as()) + { + /// Inner_union is an UNION ALL list, just lift it up + for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend(); + ++child) + selects.push_back(std::move(*child)); + } + else + selects.push_back(std::move(select_list[0])); + } + + // reverse children list + std::reverse(selects.begin(), selects.end()); + + ast.is_normalized = true; + ast.union_mode = ASTSelectWithUnionQuery::Mode::ALL; + + ast.list_of_selects->children = std::move(selects); + } +}; + +using CustomizeASTSelectWithUnionQueryNormalizeVisitor + = InDepthNodeVisitor, false>; + InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( const ASTPtr & query_ptr_, const Context & context_, const SelectQueryOptions & options_, const Names & required_result_column_names) : IInterpreterUnionOrSelectQuery(query_ptr_, context_, options_) { - const auto & ast = query_ptr->as(); + auto & ast = query_ptr->as(); + + /// Normalize AST Tree + if (!ast.is_normalized) + { + CustomizeASTSelectWithUnionQueryNormalizeVisitor::Data union_default_mode{context->getSettingsRef().union_default_mode}; + CustomizeASTSelectWithUnionQueryNormalizeVisitor(union_default_mode).visit(query_ptr); + } size_t num_children = ast.list_of_selects->children.size(); if (!num_children) @@ -170,51 +281,6 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_, return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock(); } -#if 0 -size_t InterpreterSelectWithUnionQuery::optimizeUnionList() -{ - auto union_distinct_num = 0; - - auto union_default_mode = context->getSettingsRef().union_default_mode; - auto & ast = query_ptr->as(); - size_t num_selects = ast.list_of_selects->children.size(); - - if (!num_selects) - throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR); - - if (num_selects > 1) - { - for (auto & mode : ast.union_modes) - { - if (mode == ASTSelectWithUnionQuery::Mode::Unspecified) - { - if (union_default_mode == UnionMode::ALL) - mode = ASTSelectWithUnionQuery::Mode::ALL; - else if (union_default_mode == UnionMode::DISTINCT) - mode = ASTSelectWithUnionQuery::Mode::DISTINCT; - else - throw Exception( - "Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty", - DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); - } - } - /// Optimize general cases: if there is UNION DISTINCT, all previous UNION DISTINCT can be rewritten to UNION ALL. - /// Therefore we have at most one UNION DISTINCT in a sequence. - for (auto rit = ast.union_modes.rbegin(); rit != ast.union_modes.rend(); ++rit) - { - if (*rit == ASTSelectWithUnionQuery::Mode::DISTINCT) - { - /// Number of streams need to do a DISTINCT transform after unite - union_distinct_num = ast.union_modes.rend() - rit + 1; - for (auto mode_to_modify = ++rit; mode_to_modify != ast.union_modes.rend(); ++mode_to_modify) - *mode_to_modify = ASTSelectWithUnionQuery::Mode::ALL; - break; - } - } - } - return union_distinct_num; -} -#endif void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) { @@ -228,93 +294,33 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) return; } - /// All UNION streams in the chain does not need to do DISTINCT transform - // if (num_distinct_union == 0) - // { - std::vector> plans(num_plans); - DataStreams data_streams(num_plans); + std::vector> plans(num_plans); + DataStreams data_streams(num_plans); - for (size_t i = 0; i < num_plans; ++i) - { - plans[i] = std::make_unique(); - nested_interpreters[i]->buildQueryPlan(*plans[i]); - data_streams[i] = plans[i]->getCurrentDataStream(); - } - - auto max_threads = context->getSettingsRef().max_threads; - auto union_step = std::make_unique(std::move(data_streams), result_header, max_threads); - - query_plan.unitePlans(std::move(union_step), std::move(plans)); - - const auto & query = query_ptr->as(); - if (query.union_mode == ASTSelectWithUnionQuery::Mode::DISTINCT) - { - /// Add distinct transform - const Settings & settings = context->getSettingsRef(); - SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); - - auto distinct_step - = std::make_unique(query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false); - - query_plan.addStep(std::move(distinct_step)); - } - // } - - /// The first union_distinct_num UNION streams need to do a DISTINCT transform after unite -#if 0 - else + for (size_t i = 0; i < num_plans; ++i) { - QueryPlan distinct_query_plan; + plans[i] = std::make_unique(); + nested_interpreters[i]->buildQueryPlan(*plans[i]); + data_streams[i] = plans[i]->getCurrentDataStream(); + } - std::vector> plans(num_distinct_union); - DataStreams data_streams(num_distinct_union); + auto max_threads = context->getSettingsRef().max_threads; + auto union_step = std::make_unique(std::move(data_streams), result_header, max_threads); - for (size_t i = 0; i < num_distinct_union; ++i) - { - plans[i] = std::make_unique(); - nested_interpreters[i]->buildQueryPlan(*plans[i]); - data_streams[i] = plans[i]->getCurrentDataStream(); - } - - auto max_threads = context->getSettingsRef().max_threads; - auto union_step = std::make_unique(std::move(data_streams), result_header, max_threads); - - distinct_query_plan.unitePlans(std::move(union_step), std::move(plans)); + query_plan.unitePlans(std::move(union_step), std::move(plans)); + const auto & query = query_ptr->as(); + if (query.union_mode == ASTSelectWithUnionQuery::Mode::DISTINCT) + { /// Add distinct transform const Settings & settings = context->getSettingsRef(); SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); - auto distinct_step - = std::make_unique(distinct_query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false); + auto distinct_step = std::make_unique(query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false); - distinct_query_plan.addStep(std::move(distinct_step)); - - /// No other UNION streams after DISTINCT stream - if (num_plans == num_distinct_union) - { - query_plan = std::move(distinct_query_plan); - return; - } - - /// Build final UNION step - std::vector> final_plans(num_plans - num_distinct_union + 1); - DataStreams final_data_streams(num_plans - num_distinct_union + 1); - - final_plans[0] = std::make_unique(std::move(distinct_query_plan)); - final_data_streams[0] = final_plans[0]->getCurrentDataStream(); - - for (size_t i = 1; i < num_plans - num_distinct_union + 1; ++i) - { - final_plans[i] = std::make_unique(); - nested_interpreters[num_distinct_union + i - 1]->buildQueryPlan(*final_plans[i]); - final_data_streams[i] = final_plans[i]->getCurrentDataStream(); - } - - auto final_union_step = std::make_unique(std::move(final_data_streams), result_header, max_threads); - query_plan.unitePlans(std::move(final_union_step), std::move(final_plans)); + query_plan.addStep(std::move(distinct_step)); } -#endif + } BlockIO InterpreterSelectWithUnionQuery::execute() diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index 9a3035f117c..cd089a51970 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -49,7 +49,6 @@ private: std::unique_ptr buildCurrentChildInterpreter(const ASTPtr & ast_ptr_, const Names & current_required_result_column_names); - // size_t optimizeUnionList(); }; } diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 610c82ee03a..f51f998efc5 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -3,6 +3,8 @@ #include #include +#include + namespace DB { @@ -16,6 +18,8 @@ ASTPtr ASTSelectWithUnionQuery::clone() const res->union_mode = union_mode; + res->list_of_modes = list_of_modes; + cloneOutputOptions(*res); return res; } @@ -38,15 +42,24 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it) { if (it != list_of_selects->children.begin()) - settings.ostr - << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") - << "UNION " - << mode_to_str(union_mode) << (settings.hilite ? hilite_none : ""); + settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION " + << mode_to_str((is_normalized) ? union_mode : list_of_modes[it - list_of_selects->children.begin() - 1]) + << (settings.hilite ? hilite_none : ""); + if (auto * node = (*it)->as()) { - auto sub_query = std::make_shared(); - sub_query->children.push_back(*it); - sub_query->formatImpl(settings, state, frame); + if (node->list_of_selects->children.size() == 1) + { + if (it != list_of_selects->children.begin()) + settings.ostr << settings.nl_or_ws; + (node->list_of_selects->children.at(0))->formatImpl(settings, state, frame); + } + else + { + auto sub_query = std::make_shared(); + sub_query->children.push_back(*it); + sub_query->formatImpl(settings, state, frame); + } } else { diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index fd5eeae2d7a..ecf03bb6a05 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -27,6 +27,10 @@ public: Mode union_mode; + UnionModes list_of_modes; + + bool is_normalized = false; + ASTPtr list_of_selects; }; diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 4eecc9754bf..1cc72f5fb8b 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -128,7 +128,7 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } // SELECT ... UNION SELECT ... else - union_modes.push_back(ASTSelectWithUnionQuery::Mode::DISTINCT); + union_modes.push_back(ASTSelectWithUnionQuery::Mode::Unspecified); return true; } return false; diff --git a/src/Parsers/ParserSelectWithUnionQuery.cpp b/src/Parsers/ParserSelectWithUnionQuery.cpp index 49f5aa719e5..efda8e43ca9 100644 --- a/src/Parsers/ParserSelectWithUnionQuery.cpp +++ b/src/Parsers/ParserSelectWithUnionQuery.cpp @@ -6,84 +6,6 @@ namespace DB { -static void getSelectsFromUnionListNode(ASTPtr & ast_select, ASTs & selects) -{ - if (auto * inner_union = ast_select->as()) - { - /// We need flatten from last to first - for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend(); ++child) - getSelectsFromUnionListNode(*child, selects); - - return; - } - - selects.push_back(std::move(ast_select)); -} - -void normalizeSelectList(ASTs & select_list, const ASTSelectWithUnionQuery::UnionModes & union_modes, ASTs & selects) -{ - int i; - for (i = union_modes.size() - 1; i >= 0; --i) - { - if (union_modes[i] == ASTSelectWithUnionQuery::Mode::ALL) - { - if (auto * inner_union = select_list[i + 1]->as()) - { - /// If inner_union is an UNION ALL list, just lift up - if (inner_union->union_mode == ASTSelectWithUnionQuery::Mode::ALL) - { - for (auto child = inner_union->list_of_selects->children.rbegin(); - child != inner_union->list_of_selects->children.rend(); - ++child) - selects.push_back(std::move(*child)); - } - /// inner_union is an UNION DISTINCT list, - // we cann't lift up - else - selects.push_back(std::move(select_list[i + 1])); - } - else - selects.push_back(std::move(select_list[i + 1])); - } - /// flatten all left nodes and current node to a UNION DISTINCT list - else if (union_modes[i] == ASTSelectWithUnionQuery::Mode::DISTINCT) - { - auto distinct_list = std::make_shared(); - distinct_list->list_of_selects = std::make_shared(); - distinct_list->children.push_back(distinct_list->list_of_selects); - for (int j = i + 1; j >= 0; j--) - { - getSelectsFromUnionListNode(select_list[j], distinct_list->list_of_selects->children); - } - distinct_list->union_mode = ASTSelectWithUnionQuery::Mode::DISTINCT; - // Reverse children list - std::reverse(distinct_list->list_of_selects->children.begin(), distinct_list->list_of_selects->children.end()); - selects.push_back(std::move(distinct_list)); - return; - } - } - /// No UNION DISTINCT or only one SELECT in select_list - if (i == -1) - { - if (auto * inner_union = select_list[0]->as()) - { - /// If inner_union is an UNION ALL list, just lift it up - if (inner_union->union_mode == ASTSelectWithUnionQuery::Mode::ALL) - { - for (auto child = inner_union->list_of_selects->children.rbegin(); - child != inner_union->list_of_selects->children.rend(); - ++child) - selects.push_back(std::move(*child)); - } - /// inner_union is an UNION DISTINCT list, - // we cann't lift it up - else - selects.push_back(std::move(select_list[i + 1])); - } - else - selects.push_back(std::move(select_list[0])); - } -} bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { @@ -102,7 +24,7 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & /// so flatten may change it's semantics. For example: /// flatten `SELECT 1 UNION (SELECT 1 UNION ALL SELETC 1)` -> `SELECT 1 UNION SELECT 1 UNION ALL SELECT 1` - /// Before normalize, if we got only one child which is ASTSelectWithUnionQuery, just lift it up + /// If we got only one child which is ASTSelectWithUnionQuery, just lift it up auto & expr_list = list_node->as(); if (expr_list.children.size() == 1) { @@ -116,25 +38,9 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & auto select_with_union_query = std::make_shared(); node = select_with_union_query; - select_with_union_query->list_of_selects = std::make_shared(); + select_with_union_query->list_of_selects = list_node; //std::make_shared(); select_with_union_query->children.push_back(select_with_union_query->list_of_selects); - - auto union_modes = parser.getUnionModes(); - - normalizeSelectList(expr_list.children, union_modes, select_with_union_query->list_of_selects->children); - /// We need reverse children list - std::reverse(select_with_union_query->list_of_selects->children.begin(), select_with_union_query->list_of_selects->children.end()); - - select_with_union_query->union_mode = ASTSelectWithUnionQuery::Mode::ALL; - - /// After normalize, if we only have one ASTSelectWithUnionQuery child, lift if up - if (select_with_union_query->list_of_selects->children.size() == 1) - { - if (select_with_union_query->list_of_selects->children.at(0)->as()) - { - node = std::move(select_with_union_query->list_of_selects->children.at(0)); - } - } + select_with_union_query->list_of_modes = parser.getUnionModes(); return true; } diff --git a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference index ff0086583fa..9c6ef1adb09 100644 --- a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference +++ b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.reference @@ -8,6 +8,11 @@ 1 1 1 +a +a +a +a +a 1 1 all @@ -44,6 +49,28 @@ all 1 1 1 +1 +1 +1 +1 +1 +a +a +a +a +a +a +a +a +a +a +a +a +1 +1 +1 +1 +all all all 1 @@ -67,3 +94,16 @@ all 1 1 1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql index 6e45c150508..e29e43f64ba 100644 --- a/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql +++ b/tests/queries/0_stateless/01529_union_distinct_and_setting_union_default_mode.sql @@ -1,26 +1,59 @@ SELECT 1; + (((((((SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1; + (((((((SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1; + +SELECT 'a' UNION ALL SELECT 'a' UNION ALL SELECT 'a' UNION SELECT 'a'; + +SELECT 'a' UNION ALL (SELECT 'a' UNION ALL SELECT 'a' UNION SELECT 'a'); + +SELECT 'a' UNION SELECT 'a' UNION SELECT 'a' UNION ALL SELECT'a'; + SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION ALL SELECT 1; + SELECT 'all' UNION SELECT 'all' UNION ALL SELECT 'all'; + SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1; + SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION ALL SELECT 1; + SELECT 1 UNION (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1); + SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1); + SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1))))))); + SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1))))))); + SELECT * FROM (SELECT 1 UNION ALL (SELECT 1 UNION SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1))); SET union_default_mode='ALL'; (((((((SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1; + (((((((SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1; + +SELECT 'a' UNION ALL SELECT 'a' UNION ALL SELECT 'a' UNION SELECT 'a'; + +SELECT 'a' UNION ALL (SELECT 'a' UNION ALL SELECT 'a' UNION SELECT 'a'); + +SELECT 'a' UNION SELECT 'a' UNION SELECT 'a' UNION ALL SELECT'a'; + SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION ALL SELECT 1; + SELECT 'all' UNION SELECT 'all' UNION ALL SELECT 'all'; + SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1; + SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION ALL SELECT 1; + SELECT 1 UNION (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1); + SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1); + SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1))))))); + SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL(SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1))))))); + SELECT * FROM (SELECT 1 UNION ALL (SELECT 1 UNION SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1))); diff --git a/tests/queries/0_stateless/01556_explain_select_with_union_query.reference b/tests/queries/0_stateless/01556_explain_select_with_union_query.reference index c1b07cedd05..cf892c2c591 100644 --- a/tests/queries/0_stateless/01556_explain_select_with_union_query.reference +++ b/tests/queries/0_stateless/01556_explain_select_with_union_query.reference @@ -18,16 +18,17 @@ Union Expression (Projection) Expression (Before ORDER BY and SELECT) ReadFromStorage (Read from SystemOne) -Union - Expression (Projection) - Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) - Expression (Projection) - Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) - Expression (Projection) - Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) +Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) Distinct Union Expression (Projection) @@ -79,6 +80,132 @@ Union Expression (Projection) Expression (Before ORDER BY and SELECT) ReadFromStorage (Read from SystemOne) + Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Distinct + Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) +Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) ReadFromStorage (Read from SystemOne) @@ -116,11 +243,10 @@ Union Expression (Projection) Expression (Before ORDER BY and SELECT) ReadFromStorage (Read from SystemOne) -Distinct - Union - Expression (Projection) - Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) - Expression (Projection) - Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) +Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) + Expression (Projection) + Expression (Before ORDER BY and SELECT) + ReadFromStorage (Read from SystemOne) diff --git a/tests/queries/0_stateless/01556_explain_select_with_union_query.sql b/tests/queries/0_stateless/01556_explain_select_with_union_query.sql index abb7f602af5..16271113b5f 100644 --- a/tests/queries/0_stateless/01556_explain_select_with_union_query.sql +++ b/tests/queries/0_stateless/01556_explain_select_with_union_query.sql @@ -1,13 +1,29 @@ EXPLAIN SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1; EXPLAIN (SELECT 1 UNION ALL SELECT 1) UNION ALL SELECT 1; -EXPLAIN SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1); +EXPLAIN SELECT 1 UNION (SELECT 1 UNION ALL SELECT 1); -EXPLAIN SELECT 1 UNION DISTINCT SELECT 1 UNION DISTINCT SELECT 1; +EXPLAIN SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1; EXPLAIN (SELECT 1 UNION DISTINCT SELECT 1) UNION DISTINCT SELECT 1; -EXPLAIN SELECT 1 UNION DISTINCT (SELECT 1 UNION DISTINCT SELECT 1); +EXPLAIN SELECT 1 UNION DISTINCT (SELECT 1 UNION SELECT 1); -EXPLAIN (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 UNION DISTINCT SELECT 1))) UNION ALL (((SELECT 1) UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 ) UNION DISTINCT SELECT 1)))); +EXPLAIN (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1))) UNION ALL (((SELECT 1) UNION (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION SELECT 1 ) UNION DISTINCT SELECT 1)))); -EXPLAIN (((((((((((((((SELECT 1 UNION ALL SELECT 1) UNION ALL SELECT 1)))))))))))))); -EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION ALL SELECT 1))))))))))))))))))))))))))))); -EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION DISTINCT SELECT 1))))))))))))))))))))))))))))); +EXPLAIN (((((((((((((((SELECT 1 UNION ALL SELECT 1) UNION SELECT 1)))))))))))))); +EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION SELECT 1))))))))))))))))))))))))))))); +EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION SELECT 1))))))))))))))))))))))))))))); + +SET union_default_mode='ALL'; + +EXPLAIN SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1; +EXPLAIN (SELECT 1 UNION ALL SELECT 1) UNION ALL SELECT 1; +EXPLAIN SELECT 1 UNION (SELECT 1 UNION ALL SELECT 1); + +EXPLAIN SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1; +EXPLAIN (SELECT 1 UNION DISTINCT SELECT 1) UNION DISTINCT SELECT 1; +EXPLAIN SELECT 1 UNION DISTINCT (SELECT 1 UNION SELECT 1); + +EXPLAIN (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1))) UNION ALL (((SELECT 1) UNION (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION SELECT 1 ) UNION DISTINCT SELECT 1)))); + +EXPLAIN (((((((((((((((SELECT 1 UNION ALL SELECT 1) UNION SELECT 1)))))))))))))); +EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION SELECT 1))))))))))))))))))))))))))))); +EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION SELECT 1))))))))))))))))))))))))))))); From 68bdde87bbfd014c859c8f3b439cded6efc199b5 Mon Sep 17 00:00:00 2001 From: myrrc Date: Mon, 9 Nov 2020 19:07:59 +0300 Subject: [PATCH 0144/1088] patched thw cerevra/int lib with the no-UB patch --- base/common/wide_integer_impl.h | 41 +++++++++++++++++++++++++-------- 1 file changed, 32 insertions(+), 9 deletions(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index d90bde30a43..16ba13f9342 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -5,6 +5,7 @@ /// (See at http://www.boost.org/LICENSE_1_0.txt) #include "throwError.h" +#include namespace wide { @@ -192,7 +193,7 @@ struct integer::_impl } template - constexpr static auto to_Integral(T f) noexcept + __attribute__((no_sanitize("undefined"))) constexpr static auto to_Integral(T f) noexcept { if constexpr (std::is_same_v) return f; @@ -227,26 +228,48 @@ struct integer::_impl constexpr static void wide_integer_from_bultin(integer & self, double rhs) noexcept { - if ((rhs > 0 && rhs < std::numeric_limits::max()) || (rhs < 0 && rhs > std::numeric_limits::min())) - { + constexpr uint64_t max_uint = std::numeric_limits::max(); + constexpr int64_t max_int = std::numeric_limits::max(); + + if ((rhs > 0 && rhs < max_uint) || + (rhs < 0 && rhs > std::numeric_limits::min())) { self = to_Integral(rhs); return; } long double r = rhs; - if (r < 0) + if (r < 0) { r = -r; + } - size_t count = r / std::numeric_limits::max(); + size_t count = r / max_uint; self = count; - self *= std::numeric_limits::max(); + self *= max_uint; long double to_diff = count; - to_diff *= std::numeric_limits::max(); + to_diff *= max_uint; - self += to_Integral(r - to_diff); + /// There are values in int64 that have more than 53 significant bits (in terms of double + /// representation). Such values, being promoted to double, are rounded up or down. If they are rounded up, + /// the result may not fit in 64 bits. + /// The example of such a number is 9.22337e+18. + /// As to_Integral does a static_cast to int64_t, it may result in UB. + /// The necessary check here is that long double has enough significant (mantissa) bits to store the + /// int64_t max value precisely. + static_assert(LDBL_MANT_DIG >= 64, + "On your system long double has less than 64 precision bits," + "which may result in UB when initializing double from int64_t"); - if (rhs < 0) + if (r - to_diff > static_cast(max_int)) + { + self += max_int; + self += static_cast(r - to_diff - max_int); + } + else + self += to_Integral(r - to_diff); + + if (rhs < 0) { self = -self; + } } template From 45b089425250edcafc4ef3ce982134c0b448e7db Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 9 Nov 2020 20:50:39 +0300 Subject: [PATCH 0145/1088] Some fixes --- src/Common/ZooKeeper/TestKeeperStorage.cpp | 8 ++++---- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 2 ++ src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 ++++ 3 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Common/ZooKeeper/TestKeeperStorage.cpp index a3e637a4ce8..22a06f67988 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.cpp +++ b/src/Common/ZooKeeper/TestKeeperStorage.cpp @@ -376,12 +376,14 @@ struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest try { + size_t i = 0; for (const auto & concrete_request : concrete_requests) { auto [ cur_response, undo_action ] = concrete_request->process(container, zxid); - response.responses.emplace_back(cur_response); + response.responses[i] = cur_response; if (cur_response->error != Coordination::Error::ZOK) { + std::cerr << "GOT ERROR ON: " << i << " error" << static_cast(cur_response->error) << std::endl; response.error = cur_response->error; for (auto it = undo_actions.rbegin(); it != undo_actions.rend(); ++it) @@ -392,6 +394,7 @@ struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest } else undo_actions.emplace_back(std::move(undo_action)); + ++i; } response.error = Coordination::Error::ZOK; @@ -482,9 +485,6 @@ void TestKeeperStorage::finalize() } - - - class TestKeeperWrapperFactory final : private boost::noncopyable { diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index b1a11b2d473..f26a72a3d55 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -544,12 +544,14 @@ void ZooKeeperMultiResponse::readImpl(ReadBuffer & in) void ZooKeeperMultiResponse::writeImpl(WriteBuffer & out) const { + std::cerr << "WRITING MULTIRESPONSE " << responses.size() << std::endl; for (auto & response : responses) { const ZooKeeperResponse & zk_response = dynamic_cast(*response); OpNum op_num = zk_response.getOpNum(); bool done = false; Error op_error = zk_response.error; + std::cerr << "WRITING OP ERROR:" << static_cast(op_error) << std::endl; Coordination::write(op_num, out); Coordination::write(done, out); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 33204686dce..4d81748ca3d 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -824,11 +824,15 @@ void ZooKeeper::receiveEvent() try { + std::cerr << "READING RESPONSE FOR REQUEST ID:" << request_info.request->getOpNum() << std::endl; if (!response) response = request_info.request->makeResponse(); if (err != Error::ZOK) + { + std::cerr << "GOT ERROR:" << static_cast(err) << std::endl; response->error = err; + } else { response->readImpl(*in); From ba449d7c929a8be63d09290e8f790b5733738f00 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 9 Nov 2020 20:51:22 +0300 Subject: [PATCH 0146/1088] Add small test --- utils/zookeeper-test/CMakeLists.txt | 3 + utils/zookeeper-test/main.cpp | 129 ++++++++++++++++++++++++++++ 2 files changed, 132 insertions(+) create mode 100644 utils/zookeeper-test/CMakeLists.txt create mode 100644 utils/zookeeper-test/main.cpp diff --git a/utils/zookeeper-test/CMakeLists.txt b/utils/zookeeper-test/CMakeLists.txt new file mode 100644 index 00000000000..aa26c840ba3 --- /dev/null +++ b/utils/zookeeper-test/CMakeLists.txt @@ -0,0 +1,3 @@ +add_executable(zk-test main.cpp) +target_link_libraries(zk-test PRIVATE clickhouse_common_zookeeper) +INSTALL(TARGETS zk-test RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse-utils) diff --git a/utils/zookeeper-test/main.cpp b/utils/zookeeper-test/main.cpp new file mode 100644 index 00000000000..694a6205adb --- /dev/null +++ b/utils/zookeeper-test/main.cpp @@ -0,0 +1,129 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +using namespace std; + +void checkEq(zkutil::ZooKeeper & zk, const std::string & path, const std::string & expected) +{ + auto result = zk.get(path); + if (result != expected) + throw std::runtime_error(fmt::format("Data on path '{}' = '{}' doesn't match expected '{}'", + path, result, expected)); +} + +void checkExists(zkutil::ZooKeeper & zk, const std::string & path) +{ + if (!zk.exists(path)) + throw std::runtime_error(fmt::format("Path '{}' doesn't exists", path)); +} + +void testCreateGetExistsNode(zkutil::ZooKeeper & zk) +{ + zk.create("/data", "test_string", zkutil::CreateMode::Persistent); + zk.create("/data/seq-", "another_string", zkutil::CreateMode::PersistentSequential); + checkEq(zk, "/data", "test_string"); + checkExists(zk, "/data/seq-0000000000"); + checkEq(zk, "/data/seq-0000000000", "another_string"); +} + +void testCreateSetNode(zkutil::ZooKeeper & zk) +{ + zk.create("/data/set", "sssss", zkutil::CreateMode::Persistent); + checkEq(zk, "/data/set", "sssss"); + zk.set("/data/set", "qqqqq"); + checkEq(zk, "/data/set", "qqqqq"); +} + +void testCreateList(zkutil::ZooKeeper & zk) +{ + zk.create("/data/lst", "", zkutil::CreateMode::Persistent); + zk.create("/data/lst/d1", "", zkutil::CreateMode::Persistent); + zk.create("/data/lst/d2", "", zkutil::CreateMode::Persistent); + zk.create("/data/lst/d3", "", zkutil::CreateMode::Persistent); + auto children = zk.getChildren("/data/lst"); + if (children.size() != 3) + throw std::runtime_error("Children of /data/lst doesn't equal to three"); + for (size_t i = 0; i < children.size(); ++i) + { + if (children[i] != "d" + std::to_string(i + 1)) + throw std::runtime_error(fmt::format("Incorrect children #{} got {}, expected {}", i, children[i], "d" + std::to_string(i + 1))); + } +} + +void testCreateSetVersionRequest(zkutil::ZooKeeper & zk) +{ + zk.create("/data/check_data", "d", zkutil::CreateMode::Persistent); + Coordination::Stat stat; + std::string result = zk.get("/data/check_data", &stat); + try + { + zk.set("/data/check_data", "e", stat.version + 2); + std::terminate(); + } + catch (...) + { + std::cerr << "Got exception on incorrect version (it's ok)\n"; + } + + checkEq(zk, "/data/check_data", "d"); + zk.set("/data/check_data", "e", stat.version); + + checkEq(zk, "/data/check_data", "e"); +} + +void testMultiRequest(zkutil::ZooKeeper & zk) +{ + Coordination::Requests requests; + requests.push_back(zkutil::makeCreateRequest("/data/multirequest", "aaa", zkutil::CreateMode::Persistent)); + requests.push_back(zkutil::makeSetRequest("/data/multirequest", "bbb", -1)); + zk.multi(requests); + + try + { + requests.clear(); + requests.push_back(zkutil::makeCreateRequest("/data/multirequest", "qweqwe", zkutil::CreateMode::Persistent)); + requests.push_back(zkutil::makeSetRequest("/data/multirequest", "bbb", -1)); + requests.push_back(zkutil::makeSetRequest("/data/multirequest", "ccc", -1)); + zk.multi(requests); + std::terminate(); + } + catch(...) + { + std::cerr << "Got exception on multy request (it's ok)\n"; + } + + checkEq(zk, "/data/multirequest", "bbb"); +} + +int main(int argc, char *argv[]) { + if (argc != 2) + { + std::cerr << "usage: " << argv[0] << " hosts" << std::endl; + return 2; + } + Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); + + zkutil::ZooKeeper zk(argv[1]); + + testCreateGetExistsNode(zk); + testCreateSetNode(zk); + testCreateList(zk); + testCreateSetVersionRequest(zk); + testMultiRequest(zk); + + //zk.removeRecursive("/data"); + return 0; +} From 09fac0da9d37b3365bd9a574535816cb4123277b Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 9 Nov 2020 21:16:02 +0300 Subject: [PATCH 0147/1088] Better --- src/Common/ZooKeeper/TestKeeperStorage.cpp | 1 - src/Common/ZooKeeper/ZooKeeperCommon.cpp | 3 ++- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 1 + utils/CMakeLists.txt | 1 + utils/zookeeper-test/main.cpp | 22 +++++++++++++++------- 5 files changed, 19 insertions(+), 9 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Common/ZooKeeper/TestKeeperStorage.cpp index 22a06f67988..5ac8daf2e09 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.cpp +++ b/src/Common/ZooKeeper/TestKeeperStorage.cpp @@ -384,7 +384,6 @@ struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest if (cur_response->error != Coordination::Error::ZOK) { std::cerr << "GOT ERROR ON: " << i << " error" << static_cast(cur_response->error) << std::endl; - response.error = cur_response->error; for (auto it = undo_actions.rbegin(); it != undo_actions.rend(); ++it) if (*it) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index f26a72a3d55..604bb2137c4 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -556,7 +556,8 @@ void ZooKeeperMultiResponse::writeImpl(WriteBuffer & out) const Coordination::write(op_num, out); Coordination::write(done, out); Coordination::write(op_error, out); - zk_response.writeImpl(out); + if (op_error == Error::ZOK) + zk_response.writeImpl(out); } /// Footer. diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 4d81748ca3d..fef0b7063f5 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -835,6 +835,7 @@ void ZooKeeper::receiveEvent() } else { + std::cerr << "NO ERROR RECEIVED\n"; response->readImpl(*in); response->removeRootPath(root_path); } diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 0987d64abed..9e1872ded7b 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -19,6 +19,7 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (iotest) add_subdirectory (corrector_utf8) add_subdirectory (zookeeper-cli) + add_subdirectory (zookeeper-test) add_subdirectory (zookeeper-dump-tree) add_subdirectory (zookeeper-remove-by-list) add_subdirectory (zookeeper-create-entry-to-download-part) diff --git a/utils/zookeeper-test/main.cpp b/utils/zookeeper-test/main.cpp index 694a6205adb..802fbc17708 100644 --- a/utils/zookeeper-test/main.cpp +++ b/utils/zookeeper-test/main.cpp @@ -107,6 +107,7 @@ void testMultiRequest(zkutil::ZooKeeper & zk) } int main(int argc, char *argv[]) { + if (argc != 2) { std::cerr << "usage: " << argv[0] << " hosts" << std::endl; @@ -118,12 +119,19 @@ int main(int argc, char *argv[]) { zkutil::ZooKeeper zk(argv[1]); - testCreateGetExistsNode(zk); - testCreateSetNode(zk); - testCreateList(zk); - testCreateSetVersionRequest(zk); - testMultiRequest(zk); - - //zk.removeRecursive("/data"); + try + { + zk.tryRemoveRecursive("/data"); + testCreateGetExistsNode(zk); + testCreateSetNode(zk); + testCreateList(zk); + testCreateSetVersionRequest(zk); + testMultiRequest(zk); + } + catch(...) + { + zk.tryRemoveRecursive("/data"); + throw; + } return 0; } From 2fba03ba4316ad89b634a2f1b35601c2c1904865 Mon Sep 17 00:00:00 2001 From: myrrc Date: Tue, 10 Nov 2020 00:30:52 +0300 Subject: [PATCH 0148/1088] fixing another UB and style errors --- base/common/wide_integer_impl.h | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index 16ba13f9342..7d7830e3ee0 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -6,6 +6,7 @@ #include "throwError.h" #include +#include namespace wide { @@ -232,17 +233,19 @@ struct integer::_impl constexpr int64_t max_int = std::numeric_limits::max(); if ((rhs > 0 && rhs < max_uint) || - (rhs < 0 && rhs > std::numeric_limits::min())) { + (rhs < 0 && rhs > std::numeric_limits::min())) + { self = to_Integral(rhs); return; } long double r = rhs; - if (r < 0) { - r = -r; - } - size_t count = r / max_uint; + if (r < 0) + r = -r; + + /// r / max_uint may not fit in size_t + size_t count = std::clamp(r / max_uint, std::numeric_limits::max(), 0); self = count; self *= max_uint; long double to_diff = count; @@ -267,9 +270,8 @@ struct integer::_impl else self += to_Integral(r - to_diff); - if (rhs < 0) { + if (rhs < 0) self = -self; - } } template From b7e716fb2b993218c14568c9876e95a0a84335e9 Mon Sep 17 00:00:00 2001 From: myrrc Date: Tue, 10 Nov 2020 00:43:10 +0300 Subject: [PATCH 0149/1088] another UB fix --- base/common/wide_integer_impl.h | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index 7d7830e3ee0..1d5deb3051e 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -231,6 +231,7 @@ struct integer::_impl { constexpr uint64_t max_uint = std::numeric_limits::max(); constexpr int64_t max_int = std::numeric_limits::max(); + constexpr size_t max_sizet = std::numeric_limits::max(); if ((rhs > 0 && rhs < max_uint) || (rhs < 0 && rhs > std::numeric_limits::min())) @@ -244,8 +245,13 @@ struct integer::_impl if (r < 0) r = -r; + const long double div = r / max_int; + size_t count = max_sizet; + /// r / max_uint may not fit in size_t - size_t count = std::clamp(r / max_uint, std::numeric_limits::max(), 0); + if (div <= static_cast(max_sizet)) + count = div; + self = count; self *= max_uint; long double to_diff = count; From 5ef262b1a3a417f4bbdac9483e78276ab6f97e67 Mon Sep 17 00:00:00 2001 From: myrrc Date: Tue, 10 Nov 2020 02:06:11 +0300 Subject: [PATCH 0150/1088] fixed test results after fixing UB --- tests/queries/0_stateless/01035_avg.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01035_avg.reference b/tests/queries/0_stateless/01035_avg.reference index f76c532ea69..42d23e98908 100644 --- a/tests/queries/0_stateless/01035_avg.reference +++ b/tests/queries/0_stateless/01035_avg.reference @@ -1,5 +1,5 @@ nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan --0.5 -0.5 -0.5 -0.5 -0.5 -0.5 127.493856 32355.57552 499999.5 499999.5 499999.5 499999.5 499999.5 -0.000500002 0.49999949943727 -0.000005 1.7009999999999985e-8 +-0.5 -0.5 -0.5 -0.5 -0.5 -0.5 127.493856 32355.57552 499999.5 499999.5 499999.5 499999.5 499999.5 -0.000500002 0.49999949943727 -0.000005 -1.702e-8 -2767.546272 999999 -0.50000449943727 From 694ad1f45274d569a4f42635b52e1e8f82f42b90 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 10 Nov 2020 11:14:41 +0800 Subject: [PATCH 0151/1088] Modify except column transformer's error log --- src/Parsers/ASTColumnsTransformers.cpp | 26 ++++++++----------- .../01470_columns_transformers.reference | 2 ++ .../01470_columns_transformers.sql | 1 + 3 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 9ae32aa3f74..cd28d8bfae9 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -91,21 +91,21 @@ void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, Fo void ASTColumnsExceptTransformer::transform(ASTs & nodes) const { - ASTs expected_columns(children); + std::set expected_columns; + for (size_t i = 0; i < children.size(); ++i) + expected_columns.insert(children[i]->as().name()); for (auto it = nodes.begin(); it != nodes.end();) { bool removed = false; if (const auto * id = it->get()->as()) { - for (int i = expected_columns.size() - 1; i >= 0; --i) + auto expected_column = expected_columns.find(id->shortName()); + if (expected_column != expected_columns.end()) { - if (expected_columns[i]->as().name() == id->shortName()) - { - removed = true; - expected_columns.erase(expected_columns.begin() + i); - it = nodes.erase(it); - } + removed = true; + expected_columns.erase(expected_column); + it = nodes.erase(it); } } @@ -116,15 +116,11 @@ void ASTColumnsExceptTransformer::transform(ASTs & nodes) const if (is_strict && !expected_columns.empty()) { String expected_columns_str; - for (size_t i = 0; i < expected_columns.size(); ++i) - { - if (i > 0) - expected_columns_str += ", "; - expected_columns_str += expected_columns[i]->as().name(); - } + std::for_each(expected_columns.begin(), expected_columns.end(), + [&](String x) { expected_columns_str += (" " + x) ; }); throw Exception( - "Columns transformer EXCEPT expects following column(s) : " + expected_columns_str, + "Columns transformer EXCEPT expects following column(s) :" + expected_columns_str, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); } } diff --git a/tests/queries/0_stateless/01470_columns_transformers.reference b/tests/queries/0_stateless/01470_columns_transformers.reference index cfe93c927bf..a103d62167b 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.reference +++ b/tests/queries/0_stateless/01470_columns_transformers.reference @@ -9,6 +9,8 @@ 1970-04-11 1970-01-11 1970-11-21 10 324 8 23 +324 +23 101 10 324 121 8 23 222 18 347 diff --git a/tests/queries/0_stateless/01470_columns_transformers.sql b/tests/queries/0_stateless/01470_columns_transformers.sql index bae0a0e5237..2da2f6e9c67 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.sql +++ b/tests/queries/0_stateless/01470_columns_transformers.sql @@ -15,6 +15,7 @@ SELECT columns_transformers.* EXCEPT(j) APPLY(avg) from columns_transformers; SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a; SELECT * EXCEPT STRICT i from columns_transformers; +SELECT * EXCEPT STRICT (i, j) from columns_transformers; SELECT * EXCEPT STRICT i, j1 from columns_transformers; -- { serverError 47 } SELECT * EXCEPT STRICT(i, j1) from columns_transformers; -- { serverError 16 } SELECT * REPLACE STRICT i + 1 AS i from columns_transformers; From 81a720d01c7c1241e25f8661dfa64e965e08f595 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 10 Nov 2020 04:28:27 +0000 Subject: [PATCH 0152/1088] fix special build --- src/Interpreters/InterpreterSelectWithUnionQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index f12c949eb92..936f2138c83 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -44,7 +44,7 @@ struct CustomizeASTSelectWithUnionQueryNormalize selects.push_back(std::move(ast_select)); } - void visit(ASTSelectWithUnionQuery & ast, ASTPtr &) + void visit(ASTSelectWithUnionQuery & ast, ASTPtr &) const { auto & union_modes = ast.list_of_modes; ASTs selects; From 0530c40cd8fdc940e2ad589c08cdb8457ac96641 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 10 Nov 2020 08:50:32 +0300 Subject: [PATCH 0153/1088] fixes --- src/Common/ThreadStatus.h | 1 - src/Formats/FormatFactory.cpp | 1 + src/Interpreters/OpenTelemetrySpanLog.cpp | 27 +-- src/Interpreters/ThreadStatusExt.cpp | 47 +++-- .../Formats/Impl/JSONRowOutputFormat.cpp | 173 +++++------------- .../Formats/Impl/JSONRowOutputFormat.h | 13 +- 6 files changed, 93 insertions(+), 169 deletions(-) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 934654cf385..0162a6946c6 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -110,7 +110,6 @@ public: __uint128_t opentelemetry_trace_id; UInt64 opentelemetry_current_span_id; - std::unique_ptr opentelemetry_thread_span; protected: ThreadGroupStatusPtr thread_group; diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 1ff2f0e2a9b..65d1e3ce9fb 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -79,6 +79,7 @@ FormatSettings getFormatSettings(const Context & context, format_settings.input_allow_errors_num = settings.input_format_allow_errors_num; format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio; format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes; + format_settings.json.named_tuple_as_object = settings.output_format_json_named_tuple_as_object; format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; format_settings.null_as_default = settings.input_format_null_as_default; diff --git a/src/Interpreters/OpenTelemetrySpanLog.cpp b/src/Interpreters/OpenTelemetrySpanLog.cpp index 013913d9f2d..186b067c251 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.cpp +++ b/src/Interpreters/OpenTelemetrySpanLog.cpp @@ -40,7 +40,16 @@ void OpenTelemetrySpanLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(finish_time_us); columns[i++]->insert(DateLUT::instance().toDayNum(finish_time_us / 1000000)); columns[i++]->insert(attribute_names); - columns[i++]->insert(attribute_values); + // The user might add some ints values, and we will have Int Field, and the + // insert will fail because the column requires Strings. Convert the fields + // here, because it's hard to remember to convert them in all other places. + Array string_values; + string_values.reserve(attribute_values.size()); + for (auto & value : attribute_values) + { + string_values.push_back(toString(value)); + } + columns[i++]->insert(string_values); } OpenTelemetrySpanHolder::OpenTelemetrySpanHolder(const std::string & _operation_name) @@ -59,8 +68,8 @@ OpenTelemetrySpanHolder::OpenTelemetrySpanHolder(const std::string & _operation_ start_time_us = std::chrono::duration_cast( std::chrono::system_clock::now().time_since_epoch()).count(); - // *** remove this - attribute_names.push_back("start.stacktrace"); + // ****** remove this + attribute_names.push_back("clickhouse.start.stacktrace"); attribute_values.push_back(StackTrace().toString()); thread.opentelemetry_current_span_id = span_id; @@ -70,8 +79,6 @@ OpenTelemetrySpanHolder::~OpenTelemetrySpanHolder() { try { - fmt::print(stderr, "{}\n", StackTrace().toString()); - if (!trace_id) { return; @@ -90,8 +97,6 @@ OpenTelemetrySpanHolder::~OpenTelemetrySpanHolder() return; } - fmt::print(stderr, "1\n"); - auto * context = thread_group->query_context; if (!context) { @@ -104,8 +109,8 @@ OpenTelemetrySpanHolder::~OpenTelemetrySpanHolder() //******** remove this attribute_names.push_back("clickhouse.query_id"); attribute_values.push_back(context->getCurrentQueryId()); - - fmt::print(stderr, "2\n"); + attribute_names.push_back("clickhouse.end.stacktrace"); + attribute_values.push_back(StackTrace().toString()); auto log = context->getOpenTelemetrySpanLog(); if (!log) @@ -114,8 +119,6 @@ OpenTelemetrySpanHolder::~OpenTelemetrySpanHolder() return; } - fmt::print(stderr, "3\n"); - finish_time_us = std::chrono::duration_cast( std::chrono::system_clock::now().time_since_epoch()).count(); @@ -126,8 +129,6 @@ OpenTelemetrySpanHolder::~OpenTelemetrySpanHolder() log->add(OpenTelemetrySpanLogElement( static_cast(*this))); - - fmt::print(stderr, "4\n"); } catch (...) { diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 1689a1598ba..3d56182a2f7 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -113,14 +113,13 @@ void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_) applyQuerySettings(); opentelemetry_trace_id = query_context->getClientInfo().opentelemetry_trace_id; - opentelemetry_current_span_id = query_context->getClientInfo().opentelemetry_span_id; - if (opentelemetry_trace_id) { - // Register the span for our thread. We might not know the name yet - // -- there are no strong constraints on when it is set relative to - // attaching the thread to query. Will set the name when the span ends. - opentelemetry_thread_span.reset(new OpenTelemetrySpanHolder("")); + opentelemetry_current_span_id = thread_local_rng(); + } + else + { + opentelemetry_current_span_id = 0; } } else @@ -319,11 +318,38 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) assertState({ThreadState::AttachedToQuery}, __PRETTY_FUNCTION__); - if (opentelemetry_thread_span) + std::shared_ptr opentelemetry_span_log; + if (opentelemetry_trace_id && query_context) { - opentelemetry_thread_span->operation_name = getThreadName(); - opentelemetry_thread_span->attribute_names.push_back("clickhouse.thread_id"); - opentelemetry_thread_span->attribute_values.push_back(thread_id); + opentelemetry_span_log = query_context->getOpenTelemetrySpanLog(); + } + + if (opentelemetry_span_log) + { + // Log the current thread span. + // We do this manually, because we can't use OpenTelemetrySpanHolder as a + // ThreadStatus member, because of linking issues. This file is linked + // separately, so we can reference OpenTelemetrySpanLog here, but if we had + // the span holder as a field, we would have to reference it in the + // destructor, which is in another library. + OpenTelemetrySpanLogElement span; + + span.trace_id = opentelemetry_trace_id; + // Might be problematic if some span holder isn't finished by the time + // we detach this thread... + span.span_id = opentelemetry_current_span_id; + span.parent_span_id = query_context->getClientInfo().opentelemetry_span_id; + span.operation_name = getThreadName(); + span.start_time_us = query_start_time_microseconds; + span.finish_time_us = + std::chrono::duration_cast( + std::chrono::system_clock::now().time_since_epoch()).count(); + // We could use a more precise and monotonic counter for this. + span.duration_ns = (span.finish_time_us - span.start_time_us) * 1000; + span.attribute_names.push_back("clickhouse.thread_id"); + span.attribute_values.push_back(thread_id); + + opentelemetry_span_log->add(span); } finalizeQueryProfiler(); @@ -338,7 +364,6 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) query_id.clear(); query_context = nullptr; - opentelemetry_thread_span = nullptr; opentelemetry_trace_id = 0; opentelemetry_current_span_id = 0; thread_group.reset(); diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index babb217ea15..517f126060f 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -1,50 +1,12 @@ -#include - -#include -#include -#include -#include #include #include +#include #include namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -void JSONRowOutputFormat::addColumn(String name, DataTypePtr type, - bool & need_validate_utf8, std::string tabs) -{ - if (!type->textCanContainOnlyValidUTF8()) - need_validate_utf8 = true; - - WriteBufferFromOwnString buf; - writeJSONString(name, buf, settings); - - const auto * as_tuple = typeid_cast(type.get()); - const bool recurse = settings.json.named_tuple_as_object - && as_tuple && as_tuple->haveExplicitNames(); - - fields.emplace_back(FieldInfo{buf.str(), type, recurse, tabs}); - - if (recurse) - { - const auto & element_types = as_tuple->getElements(); - const auto & names = as_tuple->getElementNames(); - - assert(element_types.size() == names.size()); - for (size_t i = 0; i < element_types.size(); i++) - { - addColumn(names[i], element_types[i], need_validate_utf8, tabs + "\t"); - } - } -} - JSONRowOutputFormat::JSONRowOutputFormat( WriteBuffer & out_, const Block & header, @@ -55,21 +17,19 @@ JSONRowOutputFormat::JSONRowOutputFormat( { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); + fields.assign(columns.begin(), columns.end()); - fields.reserve(columns.size()); - - const std::string initial_tabs = settings.json.write_metadata ? "\t\t\t" : "\t\t"; bool need_validate_utf8 = false; - for (const auto & column : columns) + for (size_t i = 0; i < sample.columns(); ++i) { - addColumn(column.name, column.type, need_validate_utf8, initial_tabs); - } + if (!sample.getByPosition(i).type->textCanContainOnlyValidUTF8()) + need_validate_utf8 = true; -// for (size_t i = 0; i < fields.size(); i++) -// { -// fmt::print(stderr, "{}: '{}' '{}' '{}\n", -// i, fields[i].name, fields[i].type->getName(), fields[i].recurse); -// } + WriteBufferFromOwnString buf; + writeJSONString(fields[i].name, buf, settings); + + fields[i].name = buf.str(); + } if (need_validate_utf8) { @@ -83,76 +43,40 @@ JSONRowOutputFormat::JSONRowOutputFormat( void JSONRowOutputFormat::writePrefix() { - if (settings.json.write_metadata) + writeCString("{\n", *ostr); + writeCString("\t\"meta\":\n", *ostr); + writeCString("\t[\n", *ostr); + + for (size_t i = 0; i < fields.size(); ++i) { - writeCString("{\n", *ostr); - writeCString("\t\"meta\":\n", *ostr); - writeCString("\t[\n", *ostr); + writeCString("\t\t{\n", *ostr); - for (size_t i = 0; i < fields.size(); ++i) - { - writeCString("\t\t{\n", *ostr); - - writeCString("\t\t\t\"name\": ", *ostr); - writeString(fields[i].name, *ostr); - writeCString(",\n", *ostr); - writeCString("\t\t\t\"type\": ", *ostr); - writeJSONString(fields[i].type->getName(), *ostr, settings); - writeChar('\n', *ostr); - - writeCString("\t\t}", *ostr); - if (i + 1 < fields.size()) - writeChar(',', *ostr); - writeChar('\n', *ostr); - } - - writeCString("\t],\n", *ostr); + writeCString("\t\t\t\"name\": ", *ostr); + writeString(fields[i].name, *ostr); + writeCString(",\n", *ostr); + writeCString("\t\t\t\"type\": ", *ostr); + writeJSONString(fields[i].type->getName(), *ostr, settings); + writeChar('\n', *ostr); + + writeCString("\t\t}", *ostr); + if (i + 1 < fields.size()) + writeChar(',', *ostr); writeChar('\n', *ostr); - writeCString("\t\"data\":\n", *ostr); - writeCString("\t", *ostr); } - writeCString("[\n", *ostr); + + writeCString("\t],\n", *ostr); + writeChar('\n', *ostr); + writeCString("\t\"data\":\n", *ostr); + writeCString("\t[\n", *ostr); } + void JSONRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) { -// fmt::print(stderr, "write field column '{}' type '{}'\n", -// column.getName(), type.getName()); - - writeString(fields[field_number].tabs, *ostr); + writeCString("\t\t\t", *ostr); writeString(fields[field_number].name, *ostr); writeCString(": ", *ostr); - // Sanity check: the input column type is the same as in header block. - // If I don't write out the raw pointer explicitly, for some reason clang - // complains about side effect in dereferencing the pointer: - // src/Processors/Formats/Impl/JSONRowOutputFormat.cpp:120:35: warning: expression with side effects will be evaluated despite being used as an operand to 'typeid' [-Wpotentially-evaluated-expression] - [[maybe_unused]] const IDataType * raw_ptr = fields[field_number].type.get(); - assert(typeid(type) == typeid(*raw_ptr)); - - if (fields[field_number].recurse) - { - const auto & tabs = fields[field_number].tabs; - ++field_number; - const auto & tuple_column = assert_cast(column); - const auto & nested_columns = tuple_column.getColumns(); - writeCString("{\n", *ostr); - for (size_t i = 0; i < nested_columns.size(); i++) - { - // field_number is incremented inside, and should match the nested - // columns. - writeField(*nested_columns[i], *fields[field_number].type, row_num); - if (i + 1 < nested_columns.size()) - { - writeCString(",", *ostr); - } - writeCString("\n", *ostr); - } - writeString(tabs, *ostr); - writeCString("}", *ostr); - return; - } - if (yield_strings) { WriteBufferFromOwnString buf; @@ -220,12 +144,6 @@ void JSONRowOutputFormat::writeSuffix() void JSONRowOutputFormat::writeBeforeTotals() { - if (!settings.json.write_metadata) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Cannot output totals in JSON format without metadata"); - } - writeCString(",\n", *ostr); writeChar('\n', *ostr); writeCString("\t\"totals\":\n", *ostr); @@ -254,12 +172,6 @@ void JSONRowOutputFormat::writeAfterTotals() void JSONRowOutputFormat::writeBeforeExtremes() { - if (!settings.json.write_metadata) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Cannot output extremes in JSON format without metadata"); - } - writeCString(",\n", *ostr); writeChar('\n', *ostr); writeCString("\t\"extremes\":\n", *ostr); @@ -305,20 +217,17 @@ void JSONRowOutputFormat::writeAfterExtremes() void JSONRowOutputFormat::writeLastSuffix() { - if (settings.json.write_metadata) - { - writeCString(",\n\n", *ostr); - writeCString("\t\"rows\": ", *ostr); - writeIntText(row_count, *ostr); + writeCString(",\n\n", *ostr); + writeCString("\t\"rows\": ", *ostr); + writeIntText(row_count, *ostr); - writeRowsBeforeLimitAtLeast(); + writeRowsBeforeLimitAtLeast(); - if (settings.write_statistics) - writeStatistics(); + if (settings.write_statistics) + writeStatistics(); - writeChar('\n', *ostr); - writeCString("}\n", *ostr); - } + writeChar('\n', *ostr); + writeCString("}\n", *ostr); ostr->next(); } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.h b/src/Processors/Formats/Impl/JSONRowOutputFormat.h index a4593663aeb..88b74afbabd 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.h @@ -70,8 +70,6 @@ protected: void writeRowsBeforeLimitAtLeast(); void writeStatistics(); - void addColumn(String name, DataTypePtr type, bool & need_validate_utf8, - std::string tabs); std::unique_ptr validating_ostr; /// Validates UTF-8 sequences, replaces bad sequences with replacement character. WriteBuffer * ostr; @@ -80,16 +78,7 @@ protected: size_t row_count = 0; bool applied_limit = false; size_t rows_before_limit = 0; - - struct FieldInfo - { - String name; - DataTypePtr type; - bool recurse = false; - std::string tabs; - }; - - std::vector fields; + NamesAndTypes fields; Progress progress; Stopwatch watch; From 293ae54fab9cae6347a16c6e1f82945395781f98 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 10 Nov 2020 15:18:12 +0800 Subject: [PATCH 0154/1088] Modify codes to make logical clearly --- src/Parsers/ASTColumnsTransformers.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index cd28d8bfae9..00a3ed68fd8 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -92,24 +92,23 @@ void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, Fo void ASTColumnsExceptTransformer::transform(ASTs & nodes) const { std::set expected_columns; - for (size_t i = 0; i < children.size(); ++i) - expected_columns.insert(children[i]->as().name()); + for (const auto & child : children) + expected_columns.insert(child->as().name()); for (auto it = nodes.begin(); it != nodes.end();) { - bool removed = false; if (const auto * id = it->get()->as()) { auto expected_column = expected_columns.find(id->shortName()); if (expected_column != expected_columns.end()) { - removed = true; expected_columns.erase(expected_column); it = nodes.erase(it); } + else + ++it; } - - if (!removed) + else ++it; } From b3a061443b69f0e960e053f53c81dee0839fd796 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 10 Nov 2020 21:09:05 +0800 Subject: [PATCH 0155/1088] ISSUES-16574 trigger CI From d8f515c4f31026fb4089ce147eed95fde4f42053 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 10 Nov 2020 16:43:10 +0300 Subject: [PATCH 0156/1088] Almost watches --- src/Common/ZooKeeper/TestKeeperStorage.cpp | 85 +++++++++++++++++++++- src/Common/ZooKeeper/TestKeeperStorage.h | 13 +++- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 9 ++- src/Server/TestKeeperTCPHandler.cpp | 30 ++++++-- src/Server/TestKeeperTCPHandler.h | 3 +- 5 files changed, 122 insertions(+), 18 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Common/ZooKeeper/TestKeeperStorage.cpp index 5ac8daf2e09..9ddb7a7e82f 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.cpp +++ b/src/Common/ZooKeeper/TestKeeperStorage.cpp @@ -24,6 +24,37 @@ static String baseName(const String & path) return path.substr(rslash_pos + 1); } +static void processWatchesImpl(const String & path, TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) +{ + Coordination::ZooKeeperWatchResponse watch_response; + watch_response.path = path; + watch_response.xid = -1; + + auto it = watches.find(watch_response.path); + if (it != watches.end()) + { + for (auto & callback : it->second) + if (callback) + callback(std::make_shared(watch_response)); + + watches.erase(it); + } + + Coordination::ZooKeeperWatchResponse watch_list_response; + watch_list_response.path = parentPath(path); + watch_list_response.xid = -1; + + it = list_watches.find(watch_list_response.path); + if (it != list_watches.end()) + { + for (auto & callback : it->second) + if (callback) + callback(std::make_shared(watch_list_response)); + + list_watches.erase(it); + } +} + TestKeeperStorage::TestKeeperStorage() { container.emplace("/", Node()); @@ -41,6 +72,8 @@ struct TestKeeperStorageRequest : zk_request(zk_request_) {} virtual std::pair process(TestKeeperStorage::Container & container, int64_t zxid) const = 0; + virtual void processWatches(TestKeeperStorage::Watches & /*watches*/, TestKeeperStorage::Watches & /*list_watches*/) const {} + virtual ~TestKeeperStorageRequest() {} }; @@ -57,6 +90,12 @@ struct TestKeeperStorageHeartbeatRequest final : public TestKeeperStorageRequest struct TestKeeperStorageCreateRequest final : public TestKeeperStorageRequest { using TestKeeperStorageRequest::TestKeeperStorageRequest; + + void processWatches(TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) const override + { + processWatchesImpl(zk_request->getPath(), watches, list_watches); + } + std::pair process(TestKeeperStorage::Container & container, int64_t zxid) const override { LOG_DEBUG(&Poco::Logger::get("STORAGE"), "EXECUTING CREATE REQUEST"); @@ -201,6 +240,11 @@ struct TestKeeperStorageRemoveRequest final : public TestKeeperStorageRequest return { response_ptr, undo }; } + + void processWatches(TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) const override + { + processWatchesImpl(zk_request->getPath(), watches, list_watches); + } }; struct TestKeeperStorageExistsRequest final : public TestKeeperStorageRequest @@ -268,6 +312,12 @@ struct TestKeeperStorageSetRequest final : public TestKeeperStorageRequest return { response_ptr, {} }; } + + void processWatches(TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) const override + { + processWatchesImpl(zk_request->getPath(), watches, list_watches); + } + }; struct TestKeeperStorageListRequest final : public TestKeeperStorageRequest @@ -407,6 +457,12 @@ struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest throw; } } + + void processWatches(TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) const override + { + for (const auto & generic_request : concrete_requests) + generic_request->processWatches(watches, list_watches); + } }; void TestKeeperStorage::processingThread() @@ -427,12 +483,26 @@ void TestKeeperStorage::processingThread() if (shutdown) break; - ++zxid; + if (info.watch_callback) + { + auto & watches_type = dynamic_cast(info.request->zk_request.get()) + ? list_watches + : watches; + + watches_type[info.request->zk_request->getPath()].emplace_back(std::move(info.watch_callback)); + } auto zk_request = info.request->zk_request; LOG_DEBUG(&Poco::Logger::get("STORAGE"), "GOT REQUEST {}", zk_request->getOpNum()); + info.request->zk_request->addRootPath(root_path); auto [response, _] = info.request->process(container, zxid); + if (response->error == Coordination::Error::ZOK) + { + info.request->processWatches(watches, list_watches); + } + + ++zxid; response->xid = zk_request->xid; response->zxid = zxid; response->removeRootPath(root_path); @@ -541,7 +611,7 @@ TestKeeperWrapperFactory::TestKeeperWrapperFactory() registerTestKeeperRequestWrapper<14, TestKeeperStorageMultiRequest>(*this); } -TestKeeperStorage::AsyncResponse TestKeeperStorage::putRequest(const Coordination::ZooKeeperRequestPtr & request) +TestKeeperStorage::ResponsePair TestKeeperStorage::putRequest(const Coordination::ZooKeeperRequestPtr & request) { auto promise = std::make_shared>(); auto future = promise->get_future(); @@ -550,12 +620,19 @@ TestKeeperStorage::AsyncResponse TestKeeperStorage::putRequest(const Coordinatio request_info.time = clock::now(); request_info.request = storage_request; request_info.response_callback = [promise] (const Coordination::ZooKeeperResponsePtr & response) { promise->set_value(response); }; + std::optional watch_future; + if (request->has_watch) + { + auto watch_promise = std::make_shared>(); + watch_future.emplace(watch_promise->get_future()); + request_info.watch_callback = [watch_promise] (const Coordination::ZooKeeperResponsePtr & response) { watch_promise->set_value(response); }; + } std::lock_guard lock(push_request_mutex); if (!requests_queue.tryPush(std::move(request_info), operation_timeout.totalMilliseconds())) throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::LOGICAL_ERROR); - LOG_DEBUG(&Poco::Logger::get("STORAGE"), "PUSHED"); - return future; + //LOG_DEBUG(&Poco::Logger::get("STORAGE"), "PUSHED"); + return ResponsePair{std::move(future), std::move(watch_future)}; } diff --git a/src/Common/ZooKeeper/TestKeeperStorage.h b/src/Common/ZooKeeper/TestKeeperStorage.h index 86be2e0eeaf..edcae28af85 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.h +++ b/src/Common/ZooKeeper/TestKeeperStorage.h @@ -12,6 +12,7 @@ namespace zkutil using namespace DB; struct TestKeeperStorageRequest; using TestKeeperStorageRequestPtr = std::shared_ptr; +using ResponseCallback = std::function; class TestKeeperStorage { @@ -33,7 +34,7 @@ public: using Container = std::map; - using WatchCallbacks = std::vector; + using WatchCallbacks = std::vector; using Watches = std::map; Container container; @@ -51,7 +52,8 @@ public: struct RequestInfo { TestKeeperStorageRequestPtr request; - std::function response_callback; + ResponseCallback response_callback; + ResponseCallback watch_callback; clock::time_point time; }; std::mutex push_request_mutex; @@ -68,7 +70,12 @@ public: using AsyncResponse = std::future; TestKeeperStorage(); ~TestKeeperStorage(); - AsyncResponse putRequest(const Coordination::ZooKeeperRequestPtr & request); + struct ResponsePair + { + AsyncResponse response; + std::optional watch_response; + }; + ResponsePair putRequest(const Coordination::ZooKeeperRequestPtr & request); int64_t getSessionID() { diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index fef0b7063f5..093b2bec7df 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -654,11 +654,14 @@ void ZooKeeper::sendThread() } if (expired) + { break; + } info.request->addRootPath(root_path); info.request->probably_sent = true; + std::cerr << "SENDING GENERAL REQUEST\n"; info.request->write(*out); if (info.request->xid == close_xid) @@ -899,7 +902,6 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) if (expired) return; - expired = true; } active_session_metric_increment.destroy(); @@ -922,6 +924,11 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) send_thread.join(); } + { + std::lock_guard lock(push_request_mutex); + expired = true; + } + try { /// This will also wakeup the receiving thread. diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index 0ca106d0e32..38d9f7e54c5 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -289,31 +289,45 @@ void TestKeeperTCPHandler::runImpl() } if (in->poll(max_wait)) - receiveRequest(); + { + bool close_received = receiveRequest(); + if (close_received) + { + LOG_DEBUG(log, "Received close request"); + break; + } + } } } -void TestKeeperTCPHandler::receiveRequest() +bool TestKeeperTCPHandler::receiveRequest() { - LOG_DEBUG(log, "Receiving heartbeat event"); + LOG_DEBUG(log, "Receiving event"); int32_t length; read(length, *in); - LOG_DEBUG(log, "RECEIVED LENGTH {}", length); + //LOG_DEBUG(log, "RECEIVED LENGTH {}", length); int32_t xid; - LOG_DEBUG(log, "READING XID"); + //LOG_DEBUG(log, "READING XID"); read(xid, *in); - LOG_DEBUG(log, "Received xid {}", xid); + //LOG_DEBUG(log, "Received xid {}", xid); int32_t opnum; read(opnum, *in); + if (opnum == -11) + return true; + Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(opnum); request->xid = xid; request->readImpl(*in); - responses.push(test_keeper_storage->putRequest(request)); + auto request_future_responses = test_keeper_storage->putRequest(request); + responses.push(std::move(request_future_responses.response)); + if (request_future_responses.watch_response) + responses.push(std::move(*request_future_responses.watch_response)); - LOG_DEBUG(log, "Event received"); + return false; + //LOG_DEBUG(log, "Event received"); } diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h index 2a796daa4e9..b50af3363c1 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/TestKeeperTCPHandler.h @@ -35,7 +35,6 @@ private: std::queue responses; - /// Streams for reading/writing from/to client connection socket. std::shared_ptr in; std::shared_ptr out; @@ -45,7 +44,7 @@ private: void sendHandshake(); void receiveHandshake(); - void receiveRequest(); + bool receiveRequest(); }; } From 9d3788f264b1287815e0e69d58acbac8900dd17a Mon Sep 17 00:00:00 2001 From: myrrc Date: Tue, 10 Nov 2020 16:51:23 +0300 Subject: [PATCH 0157/1088] fixing UB overflow in line 274 in wide_integer_impl.h --- base/common/wide_integer_impl.h | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index 1d5deb3051e..1940129ca98 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -232,6 +232,7 @@ struct integer::_impl constexpr uint64_t max_uint = std::numeric_limits::max(); constexpr int64_t max_int = std::numeric_limits::max(); constexpr size_t max_sizet = std::numeric_limits::max(); + constexpr long double max_int_long_double = static_cast(max_int); if ((rhs > 0 && rhs < max_uint) || (rhs < 0 && rhs > std::numeric_limits::min())) @@ -268,13 +269,18 @@ struct integer::_impl "On your system long double has less than 64 precision bits," "which may result in UB when initializing double from int64_t"); - if (r - to_diff > static_cast(max_int)) + if (long double diff = r - to_diff; diff > max_int_long_double) { - self += max_int; - self += static_cast(r - to_diff - max_int); + uint64_t diff_multiplier = max_uint; + + if (const long double multiplier = diff / max_int_long_double; multiplier < max_uint) + diff_multiplier = multiplier; + + self += max_int_long_double * diff_multiplier; + self += static_cast(diff - max_int_long_double * diff_multiplier); } else - self += to_Integral(r - to_diff); + self += static_cast(diff); if (rhs < 0) self = -self; From ec2c2ec576f11ed96c0d5072bdd99f5cf90de2f8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 10 Nov 2020 17:09:32 +0300 Subject: [PATCH 0158/1088] Set expire after we set close --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 22 ++++++++++++++-------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index f5c57781eef..4df4882a25b 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1114,6 +1114,7 @@ void ZooKeeper::sendThread() info.request->probably_sent = true; info.request->write(*out); + /// We sent close request, exit if (info.request->xid == close_xid) break; } @@ -1342,13 +1343,8 @@ void ZooKeeper::receiveEvent() void ZooKeeper::finalize(bool error_send, bool error_receive) { - { - std::lock_guard lock(push_request_mutex); - - if (expired) - return; - expired = true; - } + if (expired) + return; active_session_metric_increment.destroy(); @@ -1356,7 +1352,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) { if (!error_send) { - /// Send close event. This also signals sending thread to wakeup and then stop. + /// Send close event. This also signals sending thread to stop. try { close(); @@ -1364,12 +1360,22 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) catch (...) { /// This happens for example, when "Cannot push request to queue within operation timeout". + /// Just mark session expired in case of error on close request. + std::lock_guard lock(push_request_mutex); + expired = true; tryLogCurrentException(__PRETTY_FUNCTION__); } + /// Send thread will exit after sending close request or on expired flag send_thread.join(); } + /// Set expired flag after we sent close event + { + std::lock_guard lock(push_request_mutex); + expired = true; + } + try { /// This will also wakeup the receiving thread. From 9120189d8a6457aff678609892b8a114a8f5d29b Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 10 Nov 2020 17:42:56 +0300 Subject: [PATCH 0159/1088] Add SelectPartsDecision enum class --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 24 ++++++------- .../MergeTree/MergeTreeDataMergerMutator.h | 14 +++++--- src/Storages/StorageMergeTree.cpp | 25 ++++++------- src/Storages/StorageReplicatedMergeTree.cpp | 36 +++++++------------ 4 files changed, 43 insertions(+), 56 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 624c5f156aa..bb999bc71c7 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -203,7 +203,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() const return 0; } -bool MergeTreeDataMergerMutator::selectPartsToMerge( +SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( FutureMergedMutatedPart & future_part, bool aggressive, size_t max_total_size_to_merge, @@ -219,7 +219,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( { if (out_disable_reason) *out_disable_reason = "There are no parts in the table"; - return false; + return SelectPartsDecision::CANNOT_SELECT; } time_t current_time = std::time(nullptr); @@ -335,7 +335,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( { if (out_disable_reason) *out_disable_reason = "There is no need to merge parts according to merge selector algorithm"; - return false; + return SelectPartsDecision::CANNOT_SELECT; } } @@ -349,29 +349,28 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( LOG_DEBUG(log, "Selected {} parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); future_part.assign(std::move(parts)); - return true; + return SelectPartsDecision::SELECTED; } -bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( +SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( FutureMergedMutatedPart & future_part, UInt64 & available_disk_space, const AllowedMergingPredicate & can_merge, const String & partition_id, bool final, - bool * is_single_merged_part, const StorageMetadataPtr & metadata_snapshot, String * out_disable_reason) { MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id); if (parts.empty()) - return false; + return SelectPartsDecision::NOTHING_TO_MERGE; if (!final && parts.size() == 1) { if (out_disable_reason) *out_disable_reason = "There is only one part inside partition"; - return false; + return SelectPartsDecision::CANNOT_SELECT; } /// If final, optimize_skip_merged_partitions is true and we have only one part in partition with level > 0 @@ -379,8 +378,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0 && (!metadata_snapshot->hasAnyTTL() || parts[0]->checkAllTTLCalculated(metadata_snapshot))) { - *is_single_merged_part = true; - return false; + return SelectPartsDecision::NOTHING_TO_MERGE; } auto it = parts.begin(); @@ -392,7 +390,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( /// For the case of one part, we check that it can be merged "with itself". if ((it != parts.begin() || parts.size() == 1) && !can_merge(*prev_it, *it, out_disable_reason)) { - return false; + return SelectPartsDecision::NOTHING_TO_MERGE; } sum_bytes += (*it)->getBytesOnDisk(); @@ -422,14 +420,14 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( if (out_disable_reason) *out_disable_reason = fmt::format("Insufficient available disk space, required {}", ReadableSize(required_disk_space)); - return false; + return SelectPartsDecision::CANNOT_SELECT; } LOG_DEBUG(log, "Selected {} parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); future_part.assign(std::move(parts)); available_disk_space -= required_disk_space; - return true; + return SelectPartsDecision::SELECTED; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index a3f240f86de..4af8ab6bb6b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -16,6 +16,13 @@ namespace DB class MergeListEntry; class MergeProgressCallback; +enum class SelectPartsDecision +{ + SELECTED = 0, + CANNOT_SELECT = 1, + NOTHING_TO_MERGE = 2, +}; + /// Auxiliary struct holding metainformation for the future merged or mutated part. struct FutureMergedMutatedPart { @@ -79,7 +86,7 @@ public: * - Parts between which another part can still appear can not be merged. Refer to METR-7001. * - A part that already merges with something in one place, you can not start to merge into something else in another place. */ - bool selectPartsToMerge( + SelectPartsDecision selectPartsToMerge( FutureMergedMutatedPart & future_part, bool aggressive, size_t max_total_size_to_merge, @@ -90,15 +97,14 @@ public: /** Select all the parts in the specified partition for merge, if possible. * final - choose to merge even a single part - that is, allow to merge one part "with itself", * but if setting optimize_skip_merged_partitions is true (it's true as default) than single part with level > 0 - * won't be merged with itself. + * and without expired TTL won't be merged with itself. */ - bool selectAllPartsToMergeWithinPartition( + SelectPartsDecision selectAllPartsToMergeWithinPartition( FutureMergedMutatedPart & future_part, UInt64 & available_disk_space, const AllowedMergingPredicate & can_merge, const String & partition_id, bool final, - bool * is_single_merged_part, const StorageMetadataPtr & metadata_snapshot, String * out_disable_reason = nullptr); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 69558b13d9b..44715e51a99 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -653,11 +653,7 @@ bool StorageMergeTree::merge( && getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock); }; - bool selected = false; - - /// This flag is true when there is only one part in partition, it's level > 0 - /// and setting optimize_skip_merged_partitions is true - bool is_single_merged_part = false; + SelectPartsDecision select_decision = SelectPartsDecision::NOTHING_TO_MERGE; if (partition_id.empty()) { @@ -669,7 +665,7 @@ bool StorageMergeTree::merge( /// possible. if (max_source_parts_size > 0) { - selected = merger_mutator.selectPartsToMerge( + select_decision = merger_mutator.selectPartsToMerge( future_part, aggressive, max_source_parts_size, @@ -685,13 +681,13 @@ bool StorageMergeTree::merge( while (true) { UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); - selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, metadata_snapshot, out_disable_reason); + select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( + future_part, disk_space, can_merge, partition_id, final, metadata_snapshot, out_disable_reason); /// If final - we will wait for currently processing merges to finish and continue. /// TODO Respect query settings for timeout if (final - && !selected + && select_decision != SelectPartsDecision::SELECTED && !currently_merging_mutating_parts.empty() && out_disable_reason && out_disable_reason->empty()) @@ -711,13 +707,12 @@ bool StorageMergeTree::merge( } } - if (!selected) + /// If final and there is nothing to merge we treat this merge as successful. + if (final && select_decision == SelectPartsDecision::NOTHING_TO_MERGE) + return true; + + if (select_decision != SelectPartsDecision::SELECTED) { - /// If is_single_merged_part is true we treat this part as already merged - if (final && is_single_merged_part) - { - return true; - } if (out_disable_reason) { if (!out_disable_reason->empty()) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 09d36ddd4e4..c9b8bafdb5a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2681,7 +2681,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() FutureMergedMutatedPart future_merged_part; if (max_source_parts_size_for_merge > 0 && - merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, merge_with_ttl_allowed, nullptr)) + merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, merge_with_ttl_allowed, nullptr) == SelectPartsDecision::SELECTED) { create_result = createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, future_merged_part.type, deduplicate, nullptr, merge_pred.getVersion(), future_merged_part.merge_type); @@ -3762,20 +3762,11 @@ bool StorageReplicatedMergeTree::optimize( ReplicatedMergeTreeMergePredicate can_merge = queue.getMergePredicate(zookeeper); FutureMergedMutatedPart future_merged_part; + SelectPartsDecision select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( + future_merged_part, disk_space, can_merge, partition_id, true, metadata_snapshot, nullptr); - /// This flag is true when there is only one part in partition, it's level > 0 - /// and setting optimize_skip_merged_partitions is true - bool is_single_merged_part = false; - bool selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, true, &is_single_merged_part, metadata_snapshot, nullptr); - - if (!selected) - { - /// If is_single_merged_part is true we treat this part as already merged - if (is_single_merged_part) - return true; + if (select_decision != SelectPartsDecision::SELECTED) break; - } ReplicatedMergeTreeLogEntryData merge_entry; CreateMergeEntryResult create_result = createLogEntryToMergeParts( @@ -3807,29 +3798,26 @@ bool StorageReplicatedMergeTree::optimize( FutureMergedMutatedPart future_merged_part; String disable_reason; - bool selected = false; + SelectPartsDecision select_decision = SelectPartsDecision::NOTHING_TO_MERGE; - /// This flag is true when there is only one part in partition, it's level > 0 - /// and setting optimize_skip_merged_partitions is true - bool is_single_merged_part = false; if (!partition) { - selected = merger_mutator.selectPartsToMerge( + select_decision = merger_mutator.selectPartsToMerge( future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, false, &disable_reason); } else { UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); String partition_id = getPartitionIDFromQuery(partition, query_context); - selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, metadata_snapshot, &disable_reason); + select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( + future_merged_part, disk_space, can_merge, partition_id, final, metadata_snapshot, &disable_reason); } - if (!selected) + if (final && select_decision == SelectPartsDecision::NOTHING_TO_MERGE) + break; + + if (select_decision != SelectPartsDecision::SELECTED) { - /// If is_single_merged_part is true we treat this part as already merged - if (final && is_single_merged_part) - return true; std::stringstream message; message << "Cannot select parts for optimization"; if (!disable_reason.empty()) From c80e05f18eeca26b3ef69333b1c286bf6c596d19 Mon Sep 17 00:00:00 2001 From: myrrc Date: Tue, 10 Nov 2020 19:35:14 +0300 Subject: [PATCH 0160/1088] another attempt to fix UB in wide int lib --- base/common/wide_integer_impl.h | 63 ++++++++++++++++----------------- 1 file changed, 31 insertions(+), 32 deletions(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index 1940129ca98..4b4d81ff4e3 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -7,6 +7,7 @@ #include "throwError.h" #include #include +#include namespace wide { @@ -229,35 +230,19 @@ struct integer::_impl constexpr static void wide_integer_from_bultin(integer & self, double rhs) noexcept { - constexpr uint64_t max_uint = std::numeric_limits::max(); constexpr int64_t max_int = std::numeric_limits::max(); - constexpr size_t max_sizet = std::numeric_limits::max(); - constexpr long double max_int_long_double = static_cast(max_int); + constexpr int64_t min_int = std::numeric_limits::min(); - if ((rhs > 0 && rhs < max_uint) || - (rhs < 0 && rhs > std::numeric_limits::min())) + constexpr long double max_int_long_double = static_cast(max_int); + constexpr long double min_int_long_double = static_cast(min_int); + + if ((rhs > 0 && rhs < max_int) || + (rhs < 0 && rhs > min_int)) { self = to_Integral(rhs); return; } - long double r = rhs; - - if (r < 0) - r = -r; - - const long double div = r / max_int; - size_t count = max_sizet; - - /// r / max_uint may not fit in size_t - if (div <= static_cast(max_sizet)) - count = div; - - self = count; - self *= max_uint; - long double to_diff = count; - to_diff *= max_uint; - /// There are values in int64 that have more than 53 significant bits (in terms of double /// representation). Such values, being promoted to double, are rounded up or down. If they are rounded up, /// the result may not fit in 64 bits. @@ -269,18 +254,32 @@ struct integer::_impl "On your system long double has less than 64 precision bits," "which may result in UB when initializing double from int64_t"); - if (long double diff = r - to_diff; diff > max_int_long_double) - { - uint64_t diff_multiplier = max_uint; + /// Always >= 0 + const long double rhs_long_double = (static_cast(rhs) < 0) + ? -static_cast(rhs) + : rhs; - if (const long double multiplier = diff / max_int_long_double; multiplier < max_uint) - diff_multiplier = multiplier; + const long double rhs_max_int_count = rhs_long_double / max_int; - self += max_int_long_double * diff_multiplier; - self += static_cast(diff - max_int_long_double * diff_multiplier); - } - else - self += static_cast(diff); + // Won't fit only if long double can hold values >= 2^(64 * 3). + const uint64_t rhs_max_int_count_max_int_count = rhs_max_int_count / max_int; + + long double rhs_max_int_count_acc = rhs_max_int_count; + + self = 0; + + for (uint64_t i = 0; i < rhs_max_int_count_max_int_count; ++i) + self += max_int; + + self *= max_int; + + const long double rhs_div_max_int = rhs_max_int_count * max_int; + const long double rhs_mod_max_int = rhs_long_double - rhs_div_max_int; + + assert(rhs_mod_max_int < max_int_long_double); + assert(rhs_mod_max_int > min_int_long_double); + + self += static_cast(rhs_mod_max_int); if (rhs < 0) self = -self; From 96598dcbb8158b8ed8962432822fb41e5ed4e242 Mon Sep 17 00:00:00 2001 From: myrrc Date: Tue, 10 Nov 2020 19:48:32 +0300 Subject: [PATCH 0161/1088] fixed the UB loop --- base/common/wide_integer_impl.h | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index 4b4d81ff4e3..f2d94bcd80c 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -261,15 +261,16 @@ struct integer::_impl const long double rhs_max_int_count = rhs_long_double / max_int; - // Won't fit only if long double can hold values >= 2^(64 * 3). - const uint64_t rhs_max_int_count_max_int_count = rhs_max_int_count / max_int; - + // We can't just get the number of iterations like rhs_max_int_count / max_int as it may not fit it int64_t. long double rhs_max_int_count_acc = rhs_max_int_count; self = 0; - for (uint64_t i = 0; i < rhs_max_int_count_max_int_count; ++i) + while (rhs_max_int_count_acc > max_int_long_double) + { self += max_int; + rhs_max_int_count_acc -= max_int_long_double; + } self *= max_int; From 21dd9a3ec3eadccd484f873cdcf53f977d76164a Mon Sep 17 00:00:00 2001 From: myrrc Date: Tue, 10 Nov 2020 20:35:05 +0300 Subject: [PATCH 0162/1088] fix unused var --- base/common/wide_integer_impl.h | 1 - 1 file changed, 1 deletion(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index f2d94bcd80c..f6915b64cf7 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -234,7 +234,6 @@ struct integer::_impl constexpr int64_t min_int = std::numeric_limits::min(); constexpr long double max_int_long_double = static_cast(max_int); - constexpr long double min_int_long_double = static_cast(min_int); if ((rhs > 0 && rhs < max_int) || (rhs < 0 && rhs > min_int)) From a0b6aa5d9f3ffb1bd99cbe55318d25b05d079103 Mon Sep 17 00:00:00 2001 From: myrrc Date: Tue, 10 Nov 2020 21:14:40 +0300 Subject: [PATCH 0163/1088] fixing the test after UB fix --- tests/queries/0_stateless/01035_avg.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01035_avg.reference b/tests/queries/0_stateless/01035_avg.reference index 42d23e98908..9ad7fc76be1 100644 --- a/tests/queries/0_stateless/01035_avg.reference +++ b/tests/queries/0_stateless/01035_avg.reference @@ -1,5 +1,5 @@ nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan --0.5 -0.5 -0.5 -0.5 -0.5 -0.5 127.493856 32355.57552 499999.5 499999.5 499999.5 499999.5 499999.5 -0.000500002 0.49999949943727 -0.000005 -1.702e-8 +-0.5 -0.5 -0.5 -0.5 -0.5 -0.5 127.493856 32355.57552 499999.5 499999.5 499999.5 499999.5 499999.5 -0.000500002 0.49999949943727 -0.000005 -0.000005291390805620593 -2767.546272 999999 -0.50000449943727 From cfa95d64fc475c02df6779bd3aa547912b8d4ef7 Mon Sep 17 00:00:00 2001 From: myrrc Date: Tue, 10 Nov 2020 21:21:50 +0300 Subject: [PATCH 0164/1088] fixing the debug build assert --- base/common/wide_integer_impl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index f6915b64cf7..5b13c56e2f7 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -277,7 +277,7 @@ struct integer::_impl const long double rhs_mod_max_int = rhs_long_double - rhs_div_max_int; assert(rhs_mod_max_int < max_int_long_double); - assert(rhs_mod_max_int > min_int_long_double); + assert(rhs_mod_max_int > static_cast(min_int)); self += static_cast(rhs_mod_max_int); From c898ec00d98e69fedc29f72b69c2ea4f6256ce33 Mon Sep 17 00:00:00 2001 From: Grigory Buteyko Date: Tue, 10 Nov 2020 23:03:36 +0300 Subject: [PATCH 0165/1088] Exception instead of std::runtime_error --- src/AggregateFunctions/QuantileTDigest.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index ad2121f7f72..ee71c128c44 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -286,7 +286,7 @@ public: for (const auto & c : centroids) { if (c.count <= 0 || std::isnan(c.count) || std::isnan(c.mean)) // invalid count breaks compress(), invalid mean breaks sort() - throw std::runtime_error("Invalid centroid " + std::to_string(c.count) + ":" + std::to_string(c.mean)); + throw Exception("Invalid centroid " + std::to_string(c.count) + ":" + std::to_string(c.mean), ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED); count += c.count; } compress(); // Allows reading/writing TDigests with different epsilon/max_centroids params From 1c35b47073a8dc7999b10721777639cd024463ec Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 10 Nov 2020 23:01:43 +0300 Subject: [PATCH 0166/1088] Add checking select decision in merge --- src/Storages/StorageMergeTree.cpp | 11 +++++++++-- src/Storages/StorageMergeTree.h | 10 ++++++++-- 2 files changed, 17 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 63cf512fa6e..2abdafd3433 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -635,7 +635,7 @@ void StorageMergeTree::loadMutations() } std::shared_ptr StorageMergeTree::selectPartsToMerge( - const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */) + const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */, SelectPartsDecision * select_decision_out) { std::unique_lock lock(currently_processing_in_background_mutex); auto data_settings = getSettings(); @@ -743,7 +743,14 @@ bool StorageMergeTree::merge( auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder); + SelectPartsDecision select_decision; + + auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder, &select_decision); + + /// If final and there is nothing to merge then we treat this merge as successful + if (final && select_decision == SelectPartsDecision::NOTHING_TO_MERGE) + return true; + if (!merge_mutate_entry) return false; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 6cddfe77fee..b5341f636a4 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -179,8 +179,14 @@ private: {} }; - std::shared_ptr selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason, TableLockHolder & table_lock_holder); - bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); + std::shared_ptr selectPartsToMerge( + const StorageMetadataPtr & metadata_snapshot, + bool aggressive, + const String & partition_id, + bool final, + String * disable_reason, + TableLockHolder & table_lock_holder, + SelectPartsDecision * select_decision_out = nullptr); bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); std::shared_ptr selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder); bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); From dd06ff4f942f5432e58276b9e143056d6005a600 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 11 Nov 2020 00:02:11 +0300 Subject: [PATCH 0167/1088] Minor change --- src/Storages/StorageMergeTree.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- .../0_stateless/01533_optimize_skip_merged_partitions.sql | 8 ++++---- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2abdafd3433..b4722af62a1 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -653,10 +653,10 @@ std::shared_ptr StorageMergeTree::se if (!left) return !currently_merging_mutating_parts.count(right); return !currently_merging_mutating_parts.count(left) && !currently_merging_mutating_parts.count(right) - && getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock); + && getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock); }; - SelectPartsDecision select_decision = SelectPartsDecision::NOTHING_TO_MERGE; + SelectPartsDecision select_decision; if (partition_id.empty()) { @@ -696,7 +696,7 @@ std::shared_ptr StorageMergeTree::se && out_disable_reason->empty()) { LOG_DEBUG(log, "Waiting for currently running merges ({} parts are merging right now) to perform OPTIMIZE FINAL", - currently_merging_mutating_parts.size()); + currently_merging_mutating_parts.size()); if (std::cv_status::timeout == currently_processing_in_background_condition.wait_for( lock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC))) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fbe96c200f3..ef30f0987e4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3831,7 +3831,7 @@ bool StorageReplicatedMergeTree::optimize( } if (try_no == max_retries) return handle_noop("Can't create merge queue node in ZooKeeper, because log was updated in every of " - + toString(max_retries) + " tries"); + + toString(max_retries) + " tries"); } } else @@ -3844,7 +3844,7 @@ bool StorageReplicatedMergeTree::optimize( FutureMergedMutatedPart future_merged_part; String disable_reason; - SelectPartsDecision select_decision = SelectPartsDecision::NOTHING_TO_MERGE; + SelectPartsDecision select_decision; if (!partition) { diff --git a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql index 70f73d960fc..26d60480114 100644 --- a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql +++ b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql @@ -2,13 +2,13 @@ DROP TABLE IF EXISTS optimize_final; CREATE TABLE optimize_final(t DateTime, x Int32) ENGINE = MergeTree() PARTITION BY toYYYYMM(t) ORDER BY x SETTINGS optimize_skip_merged_partitions=1; -INSERT INTO optimize_final SELECT toDate('2000-01-01'), number FROM numbers(5); -INSERT INTO optimize_final SELECT toDate('2000-01-01'), number + 5 FROM numbers(5); +INSERT INTO optimize_final SELECT toDate('2020-01-01'), number FROM numbers(5); +INSERT INTO optimize_final SELECT toDate('2020-01-01'), number + 5 FROM numbers(5); OPTIMIZE TABLE optimize_final FINAL; -INSERT INTO optimize_final SELECT toDate('2020-01-01'), number FROM numbers(5); -INSERT INTO optimize_final SELECT toDate('2020-01-01'), number + 5 FROM numbers(5); +INSERT INTO optimize_final SELECT toDate('2000-01-01'), number FROM numbers(5); +INSERT INTO optimize_final SELECT toDate('2000-01-01'), number + 5 FROM numbers(5); OPTIMIZE TABLE optimize_final FINAL; From 0f9065a91f94237d56ae63743baaaef2cbd1bcb3 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 10 Nov 2020 06:42:38 +0000 Subject: [PATCH 0168/1088] fix fix --- src/Interpreters/InterpreterInsertQuery.cpp | 11 +++++++++-- src/Interpreters/InterpreterSelectWithUnionQuery.cpp | 9 +++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index e2830322024..bf8849bd6dc 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -282,10 +282,17 @@ BlockIO InterpreterInsertQuery::execute() if (settings.optimize_trivial_insert_select) { - const auto & selects = query.select->as().list_of_selects->children; + const auto & select_query = query.select->as(); + const auto & selects = select_query.list_of_selects->children; + const auto & union_modes = select_query.list_of_modes; + /// ASTSelectWithUnionQuery is not normalized now, so it may pass some querys which can be Trivial select querys is_trivial_insert_select - = std::all_of(selects.begin(), selects.end(), [](const ASTPtr & select) { return isTrivialSelect(select); }); + = std::all_of( + union_modes.begin(), + union_modes.end(), + [](const ASTSelectWithUnionQuery::Mode & mode) { return mode == ASTSelectWithUnionQuery::Mode::ALL; }) + && std::all_of(selects.begin(), selects.end(), [](const ASTPtr & select) { return isTrivialSelect(select); }); } if (is_trivial_insert_select) diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 936f2138c83..1932f49ddc9 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -122,6 +122,7 @@ struct CustomizeASTSelectWithUnionQueryNormalize } }; +/// We need normalize children first, so we should visit AST tree bottom up using CustomizeASTSelectWithUnionQueryNormalizeVisitor = InDepthNodeVisitor, false>; @@ -136,6 +137,14 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( { CustomizeASTSelectWithUnionQueryNormalizeVisitor::Data union_default_mode{context->getSettingsRef().union_default_mode}; CustomizeASTSelectWithUnionQueryNormalizeVisitor(union_default_mode).visit(query_ptr); + + /// After normalization, if it only has one ASTSelectWithUnionQuery child, + /// we can lift it up, this can reduce one unnecessary recursion later. + if (ast.list_of_selects->children.size() == 1 && ast.list_of_selects->children.at(0)->as()) + { + query_ptr = std::move(ast.list_of_selects->children.at(0)); + ast = query_ptr->as(); + } } size_t num_children = ast.list_of_selects->children.size(); From bb3b420057609620fa39d3f7468b929f0da3e65b Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Nov 2020 11:51:54 +0300 Subject: [PATCH 0169/1088] Ugly working code --- src/Common/ZooKeeper/TestKeeperStorage.cpp | 12 +++++++ src/Common/ZooKeeper/ZooKeeperImpl.cpp | 16 ++++++--- src/Server/TestKeeperTCPHandler.cpp | 29 +++++++++++++--- src/Server/TestKeeperTCPHandler.h | 1 + utils/zookeeper-test/main.cpp | 39 ++++++++++++++++++++++ 5 files changed, 88 insertions(+), 9 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Common/ZooKeeper/TestKeeperStorage.cpp index 9ddb7a7e82f..8f80fa1a0c5 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.cpp +++ b/src/Common/ZooKeeper/TestKeeperStorage.cpp @@ -30,12 +30,18 @@ static void processWatchesImpl(const String & path, TestKeeperStorage::Watches & watch_response.path = path; watch_response.xid = -1; + std::cerr << "WATCHES SIZE:" << watches.size() << " path:" << path << std::endl; auto it = watches.find(watch_response.path); if (it != watches.end()) { for (auto & callback : it->second) + { if (callback) + { + std::cerr << "CALLING WATCH CALLBACK\n"; callback(std::make_shared(watch_response)); + } + } watches.erase(it); } @@ -44,12 +50,18 @@ static void processWatchesImpl(const String & path, TestKeeperStorage::Watches & watch_list_response.path = parentPath(path); watch_list_response.xid = -1; + std::cerr << "LIST WATCHES SIZE:" << list_watches.size() << " path:" << path << std::endl; it = list_watches.find(watch_list_response.path); if (it != list_watches.end()) { for (auto & callback : it->second) + { if (callback) + { + std::cerr << "Calling list watch callback\n" << std::endl; callback(std::make_shared(watch_list_response)); + } + } list_watches.erase(it); } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 17999044a77..8e10adb4500 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -650,6 +650,7 @@ void ZooKeeper::sendThread() if (info.watch) { info.request->has_watch = true; + std::cerr << "REQUEST" << info.request->getOpNum() << " HAS WATCH" << std::endl; CurrentMetrics::add(CurrentMetrics::ZooKeeperWatch); } @@ -661,7 +662,7 @@ void ZooKeeper::sendThread() info.request->addRootPath(root_path); info.request->probably_sent = true; - std::cerr << "SENDING GENERAL REQUEST\n"; + std::cerr << "SENDING GENERAL REQUEST:" << info.request->getOpNum() << std::endl; info.request->write(*out); /// We sent close request, exit @@ -729,7 +730,9 @@ void ZooKeeper::receiveThread() else { if (earliest_operation) - throw Exception("Operation timeout (no response) for path: " + earliest_operation->request->getPath(), Error::ZOPERATIONTIMEOUT); + { + throw Exception("Operation timeout (no response) for request " + std::to_string(earliest_operation->request->getOpNum()) + " for path: " + earliest_operation->request->getPath(), Error::ZOPERATIONTIMEOUT); + } waited += max_wait; if (waited >= session_timeout.totalMicroseconds()) throw Exception("Nothing is received in session timeout", Error::ZOPERATIONTIMEOUT); @@ -772,6 +775,7 @@ void ZooKeeper::receiveEvent() } else if (xid == watch_xid) { + std::cerr << "Receiving watch\n"; ProfileEvents::increment(ProfileEvents::ZooKeeperWatchResponse); response = std::make_shared(); @@ -828,21 +832,23 @@ void ZooKeeper::receiveEvent() try { - std::cerr << "READING RESPONSE FOR REQUEST ID:" << request_info.request->getOpNum() << std::endl; + std::cerr << "READING RESPONSE FOR REQUEST\n"; if (!response) response = request_info.request->makeResponse(); if (err != Error::ZOK) { - std::cerr << "GOT ERROR:" << static_cast(err) << std::endl; + //std::cerr << "GOT ERROR:" << static_cast(err) << std::endl; response->error = err; } else { - std::cerr << "NO ERROR RECEIVED\n"; + //std::cerr << "NO ERROR RECEIVED\n"; response->readImpl(*in); response->removeRootPath(root_path); } + if (request_info.request) + std::cerr << "Response Request ID" << request_info.request->getOpNum() << std::endl; /// Instead of setting the watch in sendEvent, set it in receiveEvent because need to check the response. /// The watch shouldn't be set if the node does not exist and it will never exist like sequential ephemeral nodes. diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index 38d9f7e54c5..82949dfc547 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -279,16 +279,33 @@ void TestKeeperTCPHandler::runImpl() while (true) { - UInt64 max_wait = operation_timeout.totalMicroseconds(); + //UInt64 max_wait = operation_timeout.totalMicroseconds(); using namespace std::chrono_literals; + LOG_DEBUG(log, "TRYING TO GET RESPONSE (size {})", responses.size()); if (!responses.empty() && responses.front().wait_for(100ms) == std::future_status::ready) { auto response = responses.front().get(); + + LOG_DEBUG(log, "Writing response bytes to socket {}", response->getOpNum()); response->write(*out); responses.pop(); + LOG_DEBUG(log, "Responses size {}", responses.size()); + } + for (auto it = watch_responses.begin(); it != watch_responses.end();) + { + if (it->wait_for(0s) == std::future_status::ready) + { + it->get()->write(*out); + it = watch_responses.erase(it); + } + else + { + ++it; + } } - if (in->poll(max_wait)) + LOG_DEBUG(log, "WAITING ON POLL"); + if (in->poll(100 * 1000)) { bool close_received = receiveRequest(); if (close_received) @@ -297,6 +314,10 @@ void TestKeeperTCPHandler::runImpl() break; } } + else + { + //LOG_DEBUG(log, "NOTHING POLLED"); + } } } @@ -324,10 +345,10 @@ bool TestKeeperTCPHandler::receiveRequest() auto request_future_responses = test_keeper_storage->putRequest(request); responses.push(std::move(request_future_responses.response)); if (request_future_responses.watch_response) - responses.push(std::move(*request_future_responses.watch_response)); + watch_responses.emplace_back(std::move(*request_future_responses.watch_response)); + LOG_DEBUG(log, "Responses size {}", responses.size()); return false; - //LOG_DEBUG(log, "Event received"); } diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h index b50af3363c1..7eea5419006 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/TestKeeperTCPHandler.h @@ -34,6 +34,7 @@ private: Poco::Timespan operation_timeout; std::queue responses; + std::vector watch_responses; /// Streams for reading/writing from/to client connection socket. std::shared_ptr in; diff --git a/utils/zookeeper-test/main.cpp b/utils/zookeeper-test/main.cpp index 802fbc17708..a924465d02a 100644 --- a/utils/zookeeper-test/main.cpp +++ b/utils/zookeeper-test/main.cpp @@ -82,6 +82,43 @@ void testCreateSetVersionRequest(zkutil::ZooKeeper & zk) checkEq(zk, "/data/check_data", "e"); } +void testCreateSetWatchEvent(zkutil::ZooKeeper & zk) +{ + + std::shared_ptr event = std::make_shared(); + zk.create("/data/nodeforwatch", "", zkutil::CreateMode::Persistent); + Coordination::Stat stat; + zk.get("/data/nodeforwatch", &stat, event); + + if (event->tryWait(300)) + throw std::runtime_error(fmt::format("Event for path {} was set without any actions", "/data/nodeforwatch")); + + zk.set("/data/nodeforwatch", "x"); + if (!event->tryWait(300)) + throw std::runtime_error(fmt::format("Event for path {} was not set after set", "/data/nodeforwatch")); + else + std::cerr << "Event was set well\n"; +} + +void testCreateListWatchEvent(zkutil::ZooKeeper & zk) +{ + std::shared_ptr event = std::make_shared(); + std::string path = "/data/pathforwatch"; + zk.create(path, "", zkutil::CreateMode::Persistent); + zk.create(path + "/n1", "", zkutil::CreateMode::Persistent); + zk.create(path + "/n2", "", zkutil::CreateMode::Persistent); + zk.getChildren(path, nullptr, event); + + if (event->tryWait(300)) + throw std::runtime_error(fmt::format("ListEvent for path {} was set without any actions", path)); + + zk.create(path + "/n3", "", zkutil::CreateMode::Persistent); + if (!event->tryWait(300)) + throw std::runtime_error(fmt::format("ListEvent for path {} was not set after create", path)); + else + std::cerr << "ListEvent was set well\n"; +} + void testMultiRequest(zkutil::ZooKeeper & zk) { Coordination::Requests requests; @@ -127,6 +164,8 @@ int main(int argc, char *argv[]) { testCreateList(zk); testCreateSetVersionRequest(zk); testMultiRequest(zk); + testCreateSetWatchEvent(zk); + testCreateListWatchEvent(zk); } catch(...) { From 0388006ed672a570925993ca43c5a60860ab63de Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Nov 2020 12:05:45 +0300 Subject: [PATCH 0170/1088] Less garbage --- src/Common/ZooKeeper/TestKeeperStorage.cpp | 21 +-------------------- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 6 ++---- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 9 ++------- src/Server/TestKeeperTCPHandler.cpp | 13 +++++++------ 4 files changed, 12 insertions(+), 37 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Common/ZooKeeper/TestKeeperStorage.cpp index 8f80fa1a0c5..b60144f4a87 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.cpp +++ b/src/Common/ZooKeeper/TestKeeperStorage.cpp @@ -30,18 +30,12 @@ static void processWatchesImpl(const String & path, TestKeeperStorage::Watches & watch_response.path = path; watch_response.xid = -1; - std::cerr << "WATCHES SIZE:" << watches.size() << " path:" << path << std::endl; auto it = watches.find(watch_response.path); if (it != watches.end()) { for (auto & callback : it->second) - { if (callback) - { - std::cerr << "CALLING WATCH CALLBACK\n"; callback(std::make_shared(watch_response)); - } - } watches.erase(it); } @@ -50,18 +44,12 @@ static void processWatchesImpl(const String & path, TestKeeperStorage::Watches & watch_list_response.path = parentPath(path); watch_list_response.xid = -1; - std::cerr << "LIST WATCHES SIZE:" << list_watches.size() << " path:" << path << std::endl; it = list_watches.find(watch_list_response.path); if (it != list_watches.end()) { for (auto & callback : it->second) - { if (callback) - { - std::cerr << "Calling list watch callback\n" << std::endl; callback(std::make_shared(watch_list_response)); - } - } list_watches.erase(it); } @@ -110,7 +98,6 @@ struct TestKeeperStorageCreateRequest final : public TestKeeperStorageRequest std::pair process(TestKeeperStorage::Container & container, int64_t zxid) const override { - LOG_DEBUG(&Poco::Logger::get("STORAGE"), "EXECUTING CREATE REQUEST"); Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Undo undo; Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); @@ -188,7 +175,7 @@ struct TestKeeperStorageGetRequest final : public TestKeeperStorageRequest using TestKeeperStorageRequest::TestKeeperStorageRequest; std::pair process(TestKeeperStorage::Container & container, int64_t /* zxid */) const override { - LOG_DEBUG(&Poco::Logger::get("STORAGE"), "EXECUTING GET REQUEST"); + //LOG_DEBUG(&Poco::Logger::get("STORAGE"), "EXECUTING GET REQUEST"); Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); @@ -445,8 +432,6 @@ struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest response.responses[i] = cur_response; if (cur_response->error != Coordination::Error::ZOK) { - std::cerr << "GOT ERROR ON: " << i << " error" << static_cast(cur_response->error) << std::endl; - for (auto it = undo_actions.rbegin(); it != undo_actions.rend(); ++it) if (*it) (*it)(); @@ -481,7 +466,6 @@ void TestKeeperStorage::processingThread() { setThreadName("TestKeeperSProc"); - LOG_DEBUG(&Poco::Logger::get("STORAGE"), "LOOPING IN THREAD"); try { while (!shutdown) @@ -505,7 +489,6 @@ void TestKeeperStorage::processingThread() } auto zk_request = info.request->zk_request; - LOG_DEBUG(&Poco::Logger::get("STORAGE"), "GOT REQUEST {}", zk_request->getOpNum()); info.request->zk_request->addRootPath(root_path); auto [response, _] = info.request->process(container, zxid); @@ -519,9 +502,7 @@ void TestKeeperStorage::processingThread() response->zxid = zxid; response->removeRootPath(root_path); - LOG_DEBUG(&Poco::Logger::get("STORAGE"), "SENDING RESPONSE"); info.response_callback(response); - LOG_DEBUG(&Poco::Logger::get("STORAGE"), "DONE"); } } } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 604bb2137c4..cb61f22fb3c 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -175,13 +175,13 @@ void ZooKeeperResponse::write(WriteBuffer & out) const { /// Excessive copy to calculate length. WriteBufferFromOwnString buf; - LOG_DEBUG(&Poco::Logger::get("LOG"), "WRITING {}", xid); + //LOG_DEBUG(&Poco::Logger::get("LOG"), "WRITING {}", xid); Coordination::write(xid, buf); Coordination::write(zxid, buf); Coordination::write(error, buf); if (error == Error::ZOK) writeImpl(buf); - LOG_DEBUG(&Poco::Logger::get("LOG"), "BUFFER LENGTH {}", buf.str().length()); + //LOG_DEBUG(&Poco::Logger::get("LOG"), "BUFFER LENGTH {}", buf.str().length()); Coordination::write(buf.str(), out); out.next(); } @@ -544,14 +544,12 @@ void ZooKeeperMultiResponse::readImpl(ReadBuffer & in) void ZooKeeperMultiResponse::writeImpl(WriteBuffer & out) const { - std::cerr << "WRITING MULTIRESPONSE " << responses.size() << std::endl; for (auto & response : responses) { const ZooKeeperResponse & zk_response = dynamic_cast(*response); OpNum op_num = zk_response.getOpNum(); bool done = false; Error op_error = zk_response.error; - std::cerr << "WRITING OP ERROR:" << static_cast(op_error) << std::endl; Coordination::write(op_num, out); Coordination::write(done, out); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 8e10adb4500..23a5c6c4301 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -650,7 +650,7 @@ void ZooKeeper::sendThread() if (info.watch) { info.request->has_watch = true; - std::cerr << "REQUEST" << info.request->getOpNum() << " HAS WATCH" << std::endl; + //std::cerr << "REQUEST" << info.request->getOpNum() << " HAS WATCH" << std::endl; CurrentMetrics::add(CurrentMetrics::ZooKeeperWatch); } @@ -662,7 +662,7 @@ void ZooKeeper::sendThread() info.request->addRootPath(root_path); info.request->probably_sent = true; - std::cerr << "SENDING GENERAL REQUEST:" << info.request->getOpNum() << std::endl; + //std::cerr << "SENDING GENERAL REQUEST:" << info.request->getOpNum() << std::endl; info.request->write(*out); /// We sent close request, exit @@ -775,7 +775,6 @@ void ZooKeeper::receiveEvent() } else if (xid == watch_xid) { - std::cerr << "Receiving watch\n"; ProfileEvents::increment(ProfileEvents::ZooKeeperWatchResponse); response = std::make_shared(); @@ -832,7 +831,6 @@ void ZooKeeper::receiveEvent() try { - std::cerr << "READING RESPONSE FOR REQUEST\n"; if (!response) response = request_info.request->makeResponse(); @@ -847,9 +845,6 @@ void ZooKeeper::receiveEvent() response->readImpl(*in); response->removeRootPath(root_path); } - if (request_info.request) - std::cerr << "Response Request ID" << request_info.request->getOpNum() << std::endl; - /// Instead of setting the watch in sendEvent, set it in receiveEvent because need to check the response. /// The watch shouldn't be set if the node does not exist and it will never exist like sequential ephemeral nodes. /// By using getData() instead of exists(), a watch won't be set if the node doesn't exist. diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index 82949dfc547..1e7f69dc14f 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -273,6 +273,7 @@ void TestKeeperTCPHandler::runImpl() catch (const Exception & e) /// Typical for an incorrect username, password, or address. { LOG_DEBUG(log, "Cannot receive handshake {}", e.displayText()); + return; } sendHandshake(); @@ -281,15 +282,15 @@ void TestKeeperTCPHandler::runImpl() { //UInt64 max_wait = operation_timeout.totalMicroseconds(); using namespace std::chrono_literals; - LOG_DEBUG(log, "TRYING TO GET RESPONSE (size {})", responses.size()); + //LOG_DEBUG(log, "TRYING TO GET RESPONSE (size {})", responses.size()); if (!responses.empty() && responses.front().wait_for(100ms) == std::future_status::ready) { auto response = responses.front().get(); - LOG_DEBUG(log, "Writing response bytes to socket {}", response->getOpNum()); + //LOG_DEBUG(log, "Writing response bytes to socket {}", response->getOpNum()); response->write(*out); responses.pop(); - LOG_DEBUG(log, "Responses size {}", responses.size()); + //LOG_DEBUG(log, "Responses size {}", responses.size()); } for (auto it = watch_responses.begin(); it != watch_responses.end();) { @@ -304,7 +305,7 @@ void TestKeeperTCPHandler::runImpl() } } - LOG_DEBUG(log, "WAITING ON POLL"); + //LOG_DEBUG(log, "WAITING ON POLL"); if (in->poll(100 * 1000)) { bool close_received = receiveRequest(); @@ -324,7 +325,7 @@ void TestKeeperTCPHandler::runImpl() bool TestKeeperTCPHandler::receiveRequest() { - LOG_DEBUG(log, "Receiving event"); + //LOG_DEBUG(log, "Receiving event"); int32_t length; read(length, *in); //LOG_DEBUG(log, "RECEIVED LENGTH {}", length); @@ -347,7 +348,7 @@ bool TestKeeperTCPHandler::receiveRequest() if (request_future_responses.watch_response) watch_responses.emplace_back(std::move(*request_future_responses.watch_response)); - LOG_DEBUG(log, "Responses size {}", responses.size()); + //LOG_DEBUG(log, "Responses size {}", responses.size()); return false; } From adfe966c465e46acf77d387231ace671fe302aa3 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 11 Nov 2020 12:30:32 +0300 Subject: [PATCH 0171/1088] Support HTTP proxy and HTTPS S3 endpoint configuration. --- src/IO/S3/PocoHTTPClient.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 49ccb6dc1b3..00cc5f497dc 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -168,12 +168,16 @@ void PocoHTTPClient::makeRequestInternal( auto request_configuration = per_request_configuration(request); if (!request_configuration.proxyHost.empty()) + { + /// Turn on tunnel mode if proxy scheme is HTTP while endpoint scheme is HTTPS. + bool use_tunnel = request_configuration.proxyScheme == Aws::Http::Scheme::HTTP && poco_uri.getScheme() == "https"; session->setProxy( request_configuration.proxyHost, request_configuration.proxyPort, Aws::Http::SchemeMapper::ToString(request_configuration.proxyScheme), - false /// Disable proxy tunneling by default + use_tunnel ); + } Poco::Net::HTTPRequest poco_request(Poco::Net::HTTPRequest::HTTP_1_1); From bd3f9e2a22966d6cf92b3b7c6895515e19662119 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 11 Nov 2020 13:09:48 +0300 Subject: [PATCH 0172/1088] Fix strange code in InterpreterShowAccessQuery --- src/Interpreters/InterpreterShowAccessQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterShowAccessQuery.cpp b/src/Interpreters/InterpreterShowAccessQuery.cpp index 5f28c49c0bc..ecac962878c 100644 --- a/src/Interpreters/InterpreterShowAccessQuery.cpp +++ b/src/Interpreters/InterpreterShowAccessQuery.cpp @@ -78,7 +78,7 @@ ASTs InterpreterShowAccessQuery::getCreateAndGrantQueries() const for (const auto & entity : entities) { create_queries.push_back(InterpreterShowCreateAccessEntityQuery::getCreateQuery(*entity, access_control)); - if (entity->isTypeOf(EntityType::USER) || entity->isTypeOf(EntityType::USER)) + if (entity->isTypeOf(EntityType::USER) || entity->isTypeOf(EntityType::ROLE)) boost::range::push_back(grant_queries, InterpreterShowGrantsQuery::getGrantQueries(*entity, access_control)); } From 287d71b3635262a677d3ef0c41ad230edd4fa135 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 11 Nov 2020 13:34:32 +0300 Subject: [PATCH 0173/1088] Fix build --- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b4722af62a1..17459c7bdb0 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -656,7 +656,7 @@ std::shared_ptr StorageMergeTree::se && getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock); }; - SelectPartsDecision select_decision; + SelectPartsDecision select_decision = SelectPartsDecision::CANNOT_SELECT; if (partition_id.empty()) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ef30f0987e4..7290088056e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3844,7 +3844,7 @@ bool StorageReplicatedMergeTree::optimize( FutureMergedMutatedPart future_merged_part; String disable_reason; - SelectPartsDecision select_decision; + SelectPartsDecision select_decision = SelectPartsDecision::CANNOT_SELECT; if (!partition) { From 148dc71d0eba876a38f54bec069644edfba61282 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 11 Nov 2020 13:38:00 +0300 Subject: [PATCH 0174/1088] Fix StorageMergeTree.h --- src/Storages/StorageMergeTree.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index b5341f636a4..a5992614247 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -186,7 +186,8 @@ private: bool final, String * disable_reason, TableLockHolder & table_lock_holder, - SelectPartsDecision * select_decision_out = nullptr); bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); + SelectPartsDecision * select_decision_out = nullptr); + bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); std::shared_ptr selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder); bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); From f04bd0643d3df34b932fa8b3a4770390a0cc9fa2 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 11 Nov 2020 03:45:06 +0000 Subject: [PATCH 0175/1088] fix test fix fix --- .../InterpreterSelectWithUnionQuery.cpp | 12 +- src/Parsers/ParserSelectWithUnionQuery.cpp | 2 +- ..._explain_select_with_union_query.reference | 216 ++++++++++++------ 3 files changed, 150 insertions(+), 80 deletions(-) diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 1932f49ddc9..a836dc8b271 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -33,10 +33,8 @@ struct CustomizeASTSelectWithUnionQueryNormalize { if (auto * inner_union = ast_select->as()) { - /// We need flatten from last to first - for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend(); - ++child) - getSelectsFromUnionListNode(*child, selects); + for (auto & child : inner_union->list_of_selects->children) + getSelectsFromUnionListNode(child, selects); return; } @@ -85,13 +83,13 @@ struct CustomizeASTSelectWithUnionQueryNormalize auto distinct_list = std::make_shared(); distinct_list->list_of_selects = std::make_shared(); distinct_list->children.push_back(distinct_list->list_of_selects); - for (int j = i + 1; j >= 0; j--) + + for (int j = 0; j <= i + 1; ++j) { getSelectsFromUnionListNode(select_list[j], distinct_list->list_of_selects->children); } + distinct_list->union_mode = ASTSelectWithUnionQuery::Mode::DISTINCT; - // Reverse children list - std::reverse(distinct_list->list_of_selects->children.begin(), distinct_list->list_of_selects->children.end()); distinct_list->is_normalized = true; selects.push_back(std::move(distinct_list)); break; diff --git a/src/Parsers/ParserSelectWithUnionQuery.cpp b/src/Parsers/ParserSelectWithUnionQuery.cpp index efda8e43ca9..9a644d8e937 100644 --- a/src/Parsers/ParserSelectWithUnionQuery.cpp +++ b/src/Parsers/ParserSelectWithUnionQuery.cpp @@ -38,7 +38,7 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & auto select_with_union_query = std::make_shared(); node = select_with_union_query; - select_with_union_query->list_of_selects = list_node; //std::make_shared(); + select_with_union_query->list_of_selects = list_node; select_with_union_query->children.push_back(select_with_union_query->list_of_selects); select_with_union_query->list_of_modes = parser.getUnionModes(); diff --git a/tests/queries/0_stateless/01556_explain_select_with_union_query.reference b/tests/queries/0_stateless/01556_explain_select_with_union_query.reference index cf892c2c591..2d09a1f8625 100644 --- a/tests/queries/0_stateless/01556_explain_select_with_union_query.reference +++ b/tests/queries/0_stateless/01556_explain_select_with_union_query.reference @@ -1,252 +1,324 @@ Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Distinct Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Distinct Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Distinct Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Distinct Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Distinct Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Distinct Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Distinct Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Distinct Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Distinct Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Distinct Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Distinct Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Distinct Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Distinct Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Union Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) Expression (Projection) Expression (Before ORDER BY and SELECT) - ReadFromStorage (Read from SystemOne) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemOne) From d907afa77e2967d5588bbe6f5544d2d8f97d076f Mon Sep 17 00:00:00 2001 From: taichong Date: Tue, 27 Oct 2020 19:17:35 +0800 Subject: [PATCH 0176/1088] throw exception when mysql sync user privs error. --- src/Common/ErrorCodes.cpp | 1 + src/Databases/MySQL/MaterializeMetadata.cpp | 45 ++++++++++++++++ .../MySQL/MaterializeMySQLSyncThread.cpp | 31 +++++++++-- .../MySQL/MaterializeMySQLSyncThread.h | 5 ++ .../materialize_with_ddl.py | 53 ++++++++++++++++++- .../test_materialize_mysql_database/test.py | 9 ++++ 6 files changed, 139 insertions(+), 5 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f069b27827e..7093ed773fa 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -519,6 +519,7 @@ M(550, CONDITIONAL_TREE_PARENT_NOT_FOUND) \ M(551, ILLEGAL_PROJECTION_MANIPULATOR) \ M(552, UNRECOGNIZED_ARGUMENTS) \ + M(553, SYNC_MYSQL_USER_ACCESS_ERROR)\ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 3c5bfdec594..f672e5fb24f 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -19,6 +19,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int SYNC_MYSQL_USER_ACCESS_ERR; } static std::unordered_map fetchTablesCreateQuery( @@ -64,6 +65,7 @@ static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entr return tables_in_db; } + void MaterializeMetadata::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & connection) { Block header{ @@ -105,6 +107,47 @@ static Block getShowMasterLogHeader(const String & mysql_version) }; } +static bool rightSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, std::ostream & out) +{ + Block sync_user_privs_header{ + {std::make_shared(), "current_user_grants"} + }; + + String grants_query, sub_privs; + MySQLBlockInputStream input(connection, "SHOW GRANTS FOR CURRENT_USER();", sync_user_privs_header, DEFAULT_BLOCK_SIZE); + while (Block block = input.read()) + { + for (size_t index = 0; index < block.rows(); ++index) + { + out << (*block.getByPosition(0).column)[index].safeGet() + "; "; + grants_query = (*block.getByPosition(0).column)[index].safeGet(); + sub_privs = grants_query.substr(0, grants_query.find(" ON ")); + if (sub_privs.find("ALL PRIVILEGES") == std::string::npos) + { + if ((sub_privs.find("RELOAD") != std::string::npos and + sub_privs.find("REPLICATION SLAVE") != std::string::npos and + sub_privs.find("REPLICATION CLIENT") != std::string::npos)) + return true; + } + else + { + return true; + } + } + } + return false; +} + +static void rightSyncUserPriv(mysqlxx::PoolWithFailover::Entry & connection) +{ + std::stringstream out; + if (!rightSyncUserPrivImpl(connection, out)) + throw Exception("MySQL SYNC USER ACCESS ERR: mysql sync user needs " + "at least GLOBAL PRIVILEGES:'RELOAD, REPLICATION SLAVE, REPLICATION CLIENT' " + "and SELECT PRIVILEGE on MySQL Database." + "But the SYNC USER grant query is: " + out.str(), ErrorCodes::SYNC_MYSQL_USER_ACCESS_ERR); +} + bool MaterializeMetadata::checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection, const String & mysql_version) const { MySQLBlockInputStream input(connection, "SHOW MASTER LOGS", getShowMasterLogHeader(mysql_version), DEFAULT_BLOCK_SIZE); @@ -167,6 +210,8 @@ MaterializeMetadata::MaterializeMetadata( const String & database, bool & opened_transaction, const String & mysql_version) : persistent_path(path_) { + rightSyncUserPriv(connection); + if (Poco::File(persistent_path).exists()) { ReadBufferFromFile in(persistent_path, DBMS_DEFAULT_BUFFER_SIZE); diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 2a3de25c24f..d5d034eaa8a 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -5,7 +5,6 @@ #if USE_MYSQL #include - # include # include # include @@ -34,6 +33,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; extern const int ILLEGAL_MYSQL_VARIABLE; + extern const int SYNC_MYSQL_USER_ACCESS_ERR; + extern const int UNKNOWN_DATABASE; } static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; @@ -215,10 +216,32 @@ void MaterializeMySQLSyncThread::stopSynchronization() void MaterializeMySQLSyncThread::startSynchronization() { - const auto & mysql_server_version = checkVariableAndGetVersion(pool.get()); + try{ + const auto & mysql_server_version = checkVariableAndGetVersion(pool.get()); - background_thread_pool = std::make_unique( - [this, mysql_server_version = mysql_server_version]() { synchronization(mysql_server_version); }); + background_thread_pool = std::make_unique( + [this, mysql_server_version = mysql_server_version]() { synchronization(mysql_server_version); }); + } + catch (...) + { + try + { + throw; + } + catch (mysqlxx::ConnectionFailed & e) + { + if (e.errnum() == ER_ACCESS_DENIED_ERROR + || e.errnum() == ER_DBACCESS_DENIED_ERROR) + throw Exception("MySQL SYNC USER ACCESS ERR: mysql sync user needs " + "at least GLOBAL PRIVILEGES:'RELOAD, REPLICATION SLAVE, REPLICATION CLIENT' " + "and SELECT PRIVILEGE on Database " + mysql_database_name + , ErrorCodes::SYNC_MYSQL_USER_ACCESS_ERR); + else if (e.errnum() == ER_BAD_DB_ERROR) + throw Exception("Unknown database '" + mysql_database_name + "' on MySQL", ErrorCodes::UNKNOWN_DATABASE); + else + throw; + } + } } static inline void cleanOutdatedTables(const String & database_name, const Context & context) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index 9a0df4823e5..c1f1fc5a391 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -20,6 +20,7 @@ # include # include + namespace DB { @@ -63,6 +64,10 @@ private: MaterializeMySQLSettings * settings; String query_prefix; + const int ER_ACCESS_DENIED_ERROR = 1045; + const int ER_DBACCESS_DENIED_ERROR = 1044; + const int ER_BAD_DB_ERROR = 1049; + struct Buffers { String database; diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index b97a1563212..534d7ff895e 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -2,6 +2,8 @@ import time import pymysql.cursors +import pytest +from helpers.client import QueryRuntimeException def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seconds=3): lastest_result = '' @@ -164,7 +166,6 @@ def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node, serv clickhouse_node.query("DROP DATABASE test_database") mysql_node.query("DROP DATABASE test_database") - def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created @@ -461,3 +462,53 @@ def select_without_columns(clickhouse_node, mysql_node, service_name): clickhouse_node.query("DROP VIEW v") clickhouse_node.query("DROP DATABASE db") mysql_node.query("DROP DATABASE db") + + +def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (4), (5), (6);") + + mysql_node.query("CREATE USER 'test'@'%' IDENTIFIED BY '123'") + + with pytest.raises(QueryRuntimeException) as exception: + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + + assert 'MySQL SYNC USER ACCESS ERR:' in str(exception.value) + assert "test_database" not in clickhouse_node.query("SHOW DATABASES") + + mysql_node.query("GRANT SELECT ON test_database.* TO 'test'@'%'") + + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") + clickhouse_node.query("DROP DATABASE test_database") + + mysql_node.query("GRANT REPLICATION CLIENT, RELOAD ON *.* TO 'test'@'%'") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") + clickhouse_node.query("DROP DATABASE test_database") + + mysql_node.query("GRANT REPLICATION SLAVE ON *.* TO 'test'@'%'") + + # wait mysql grant done + time.sleep(15) + + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + + check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 5, 5) + mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") + check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") + + clickhouse_node.query("DROP DATABASE test_database;") + mysql_node.query("DROP DATABASE test_database;") + mysql_node.query("DROP USER 'test'@'%';") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 6df831e1e7d..04053e09ee1 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -153,3 +153,12 @@ def test_select_without_columns_5_7(started_cluster, started_mysql_5_7): def test_select_without_columns_8_0(started_cluster, started_mysql_8_0): materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_8_0, "mysql8_0") + + +def test_materialize_database_err_sync_user_privs_5_7(started_cluster, started_mysql_5_7): + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + + +def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_mysql_8_0): + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + From 2f6510d75ba539c0d20d0dd60b9c402ea09cf3b4 Mon Sep 17 00:00:00 2001 From: taichong Date: Thu, 29 Oct 2020 16:09:28 +0800 Subject: [PATCH 0177/1088] try to fix ci failed --- .../materialize_with_ddl.py | 59 +++++++++---------- .../test_materialize_mysql_database/test.py | 5 ++ 2 files changed, 34 insertions(+), 30 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 534d7ff895e..a4f1884c546 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -470,36 +470,10 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (4), (5), (6);") mysql_node.query("CREATE USER 'test'@'%' IDENTIFIED BY '123'") - - with pytest.raises(QueryRuntimeException) as exception: - clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( - service_name)) - - assert 'MySQL SYNC USER ACCESS ERR:' in str(exception.value) - assert "test_database" not in clickhouse_node.query("SHOW DATABASES") - + mysql_node.query("GRANT REPLICATION SLAVE, REPLICATION CLIENT, RELOAD ON *.* TO 'test'@'%'") mysql_node.query("GRANT SELECT ON test_database.* TO 'test'@'%'") - - clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( - service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") - clickhouse_node.query("DROP DATABASE test_database") - - mysql_node.query("GRANT REPLICATION CLIENT, RELOAD ON *.* TO 'test'@'%'") - clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( - service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") - clickhouse_node.query("DROP DATABASE test_database") - - mysql_node.query("GRANT REPLICATION SLAVE ON *.* TO 'test'@'%'") - - # wait mysql grant done - time.sleep(15) + print('Eason test') + mysql_node.result("SHOW GRANTS FOR 'test'@'%'") clickhouse_node.query( "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( @@ -508,7 +482,32 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 5, 5) mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") - clickhouse_node.query("DROP DATABASE test_database;") + + mysql_node.query("REVOKE REPLICATION SLAVE ON *.* FROM 'test'@'%'") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") + clickhouse_node.query("DROP DATABASE test_database") + + mysql_node.query("REVOKE REPLICATION CLIENT, RELOAD ON *.* FROM 'test'@'%'") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") + clickhouse_node.query("DROP DATABASE test_database") + + mysql_node.query("REVOKE SELECT ON test_database.* FROM 'test'@'%'") + with pytest.raises(QueryRuntimeException) as exception: + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + + assert 'MySQL SYNC USER ACCESS ERR:' in str(exception.value) + assert "test_database" not in clickhouse_node.query("SHOW DATABASES") + mysql_node.query("DROP DATABASE test_database;") mysql_node.query("DROP USER 'test'@'%';") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 04053e09ee1..267621812ad 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -41,6 +41,11 @@ class MySQLNodeInstance: with self.alloc_connection().cursor() as cursor: cursor.execute(execution_query) + def result(self, execution_query): + with self.alloc_connection().cursor() as cursor: + cursor.execute(execution_query) + print(cursor.fetchone()) + def close(self): if self.mysql_connection is not None: self.mysql_connection.close() From aecb9716e1c2175b593f7f3ff83ef7d370bffc97 Mon Sep 17 00:00:00 2001 From: taichong Date: Fri, 30 Oct 2020 09:31:45 +0800 Subject: [PATCH 0178/1088] print mysql result for test --- .../materialize_with_ddl.py | 10 +++---- .../test_materialize_mysql_database/test.py | 26 ++++++++++++++++--- 2 files changed, 27 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index a4f1884c546..6711b36ca13 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -469,16 +469,16 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (4), (5), (6);") - mysql_node.query("CREATE USER 'test'@'%' IDENTIFIED BY '123'") - mysql_node.query("GRANT REPLICATION SLAVE, REPLICATION CLIENT, RELOAD ON *.* TO 'test'@'%'") - mysql_node.query("GRANT SELECT ON test_database.* TO 'test'@'%'") - print('Eason test') - mysql_node.result("SHOW GRANTS FOR 'test'@'%'") + mysql_node.result("SHOW GRANTS FOR 'test'@'%';") clickhouse_node.query( "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( service_name)) + # wait MaterializeMySQL read binlog events + time.sleep(90) + + assert "test_table_1" in clickhouse_node.query("SHOW TABLES FROM test_database") check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 5, 5) mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 267621812ad..72310ea30cb 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -41,10 +41,16 @@ class MySQLNodeInstance: with self.alloc_connection().cursor() as cursor: cursor.execute(execution_query) + def create_min_priv_user(self): + self.query("CREATE USER 'test'@'%' IDENTIFIED BY '123'") + self.query("GRANT REPLICATION SLAVE, REPLICATION CLIENT, RELOAD ON *.* TO 'test'@'%'") + self.query("GRANT SELECT ON test_database.* TO 'test'@'%'") + def result(self, execution_query): with self.alloc_connection().cursor() as cursor: - cursor.execute(execution_query) - print(cursor.fetchone()) + result = cursor.execute(execution_query) + if result is not None: + print(cursor.fetchall()) def close(self): if self.mysql_connection is not None: @@ -56,6 +62,8 @@ class MySQLNodeInstance: try: self.alloc_connection() print("Mysql Started") + self.create_min_priv_user() + print("min priv user created") return except Exception as ex: print("Can't connect to MySQL " + str(ex)) @@ -161,9 +169,19 @@ def test_select_without_columns_8_0(started_cluster, started_mysql_8_0): def test_materialize_database_err_sync_user_privs_5_7(started_cluster, started_mysql_5_7): - materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + try: + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + except: + print((clickhouse_node.query( + "select '\n', thread_id, query_id, arrayStringConcat(arrayMap(x -> concat(demangle(addressToSymbol(x)), '\n ', addressToLine(x)), trace), '\n') AS sym from system.stack_trace format TSVRaw"))) + raise def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_mysql_8_0): - materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + try: + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + except: + print((clickhouse_node.query( + "select '\n', thread_id, query_id, arrayStringConcat(arrayMap(x -> concat(demangle(addressToSymbol(x)), '\n ', addressToLine(x)), trace), '\n') AS sym from system.stack_trace format TSVRaw"))) + raise From 5c1a01897ac79e23e5c5224605a43270750cfd4d Mon Sep 17 00:00:00 2001 From: taichong Date: Tue, 3 Nov 2020 11:12:05 +0800 Subject: [PATCH 0179/1088] Revise the code according to the review --- src/Databases/MySQL/MaterializeMetadata.cpp | 11 ++++++----- src/Databases/MySQL/MaterializeMySQLSyncThread.cpp | 3 ++- src/Databases/MySQL/MaterializeMySQLSyncThread.h | 2 ++ .../materialize_with_ddl.py | 5 +---- 4 files changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index f672e5fb24f..01a1493fa22 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -107,9 +107,10 @@ static Block getShowMasterLogHeader(const String & mysql_version) }; } -static bool rightSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, std::ostream & out) +static bool checkSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, std::ostream & out) { - Block sync_user_privs_header{ + Block sync_user_privs_header + { {std::make_shared(), "current_user_grants"} }; @@ -138,10 +139,10 @@ static bool rightSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, return false; } -static void rightSyncUserPriv(mysqlxx::PoolWithFailover::Entry & connection) +static void checkSyncUserPriv(mysqlxx::PoolWithFailover::Entry & connection) { std::stringstream out; - if (!rightSyncUserPrivImpl(connection, out)) + if (!checkSyncUserPrivImpl(connection, out)) throw Exception("MySQL SYNC USER ACCESS ERR: mysql sync user needs " "at least GLOBAL PRIVILEGES:'RELOAD, REPLICATION SLAVE, REPLICATION CLIENT' " "and SELECT PRIVILEGE on MySQL Database." @@ -210,7 +211,7 @@ MaterializeMetadata::MaterializeMetadata( const String & database, bool & opened_transaction, const String & mysql_version) : persistent_path(path_) { - rightSyncUserPriv(connection); + checkSyncUserPriv(connection); if (Poco::File(persistent_path).exists()) { diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index d5d034eaa8a..f76e6b77448 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -216,7 +216,8 @@ void MaterializeMySQLSyncThread::stopSynchronization() void MaterializeMySQLSyncThread::startSynchronization() { - try{ + try + { const auto & mysql_server_version = checkVariableAndGetVersion(pool.get()); background_thread_pool = std::make_unique( diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index c1f1fc5a391..323ae5beb80 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -64,6 +64,8 @@ private: MaterializeMySQLSettings * settings; String query_prefix; + // USE MySQL ERROR CODE: + // https://dev.mysql.com/doc/mysql-errors/5.7/en/server-error-reference.html const int ER_ACCESS_DENIED_ERROR = 1045; const int ER_DBACCESS_DENIED_ERROR = 1044; const int ER_BAD_DB_ERROR = 1049; diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 6711b36ca13..8c9256cb1c7 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -476,10 +476,7 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n service_name)) # wait MaterializeMySQL read binlog events - time.sleep(90) - - assert "test_table_1" in clickhouse_node.query("SHOW TABLES FROM test_database") - check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 5, 5) + check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 30, 5) mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") clickhouse_node.query("DROP DATABASE test_database;") From 64acfea79c0e662a842a01ccc5240a2b86c01956 Mon Sep 17 00:00:00 2001 From: taichong Date: Tue, 27 Oct 2020 19:17:35 +0800 Subject: [PATCH 0180/1088] throw exception when mysql sync user privs error. --- tests/integration/test_materialize_mysql_database/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 72310ea30cb..543ff7e1bdd 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -184,4 +184,3 @@ def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_m print((clickhouse_node.query( "select '\n', thread_id, query_id, arrayStringConcat(arrayMap(x -> concat(demangle(addressToSymbol(x)), '\n ', addressToLine(x)), trace), '\n') AS sym from system.stack_trace format TSVRaw"))) raise - From e5da2d4086d33505ad3b181baba744d01e1191bf Mon Sep 17 00:00:00 2001 From: taichong Date: Fri, 30 Oct 2020 09:31:45 +0800 Subject: [PATCH 0181/1088] print mysql result for test --- .../test_materialize_mysql_database/materialize_with_ddl.py | 1 + tests/integration/test_materialize_mysql_database/test.py | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 8c9256cb1c7..9ea4cfbda00 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -477,6 +477,7 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n # wait MaterializeMySQL read binlog events check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 30, 5) + print mysql result for test mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") clickhouse_node.query("DROP DATABASE test_database;") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 543ff7e1bdd..72310ea30cb 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -184,3 +184,4 @@ def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_m print((clickhouse_node.query( "select '\n', thread_id, query_id, arrayStringConcat(arrayMap(x -> concat(demangle(addressToSymbol(x)), '\n ', addressToLine(x)), trace), '\n') AS sym from system.stack_trace format TSVRaw"))) raise + From 6c8b5b573bf0d670145424c776404f242a8d84b4 Mon Sep 17 00:00:00 2001 From: taichong Date: Tue, 3 Nov 2020 13:22:39 +0800 Subject: [PATCH 0182/1088] modify test for Integration Tests Flaky Check --- .../test_materialize_mysql_database/materialize_with_ddl.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 9ea4cfbda00..dc9eca07b27 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -5,7 +5,7 @@ import pymysql.cursors import pytest from helpers.client import QueryRuntimeException -def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seconds=3): +def check_query(clickhouse_node, query, result_set, retry_count=5, interval_seconds=30): lastest_result = '' for index in range(retry_count): lastest_result = clickhouse_node.query(query) @@ -477,7 +477,6 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n # wait MaterializeMySQL read binlog events check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 30, 5) - print mysql result for test mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") clickhouse_node.query("DROP DATABASE test_database;") From f4f437916873b6c73be12e4f9fe2f7dff4e61c09 Mon Sep 17 00:00:00 2001 From: taichong Date: Tue, 3 Nov 2020 18:47:32 +0800 Subject: [PATCH 0183/1088] modify test case add precheck --- .../materialize_with_ddl.py | 28 ++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index dc9eca07b27..52119add336 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -5,7 +5,7 @@ import pymysql.cursors import pytest from helpers.client import QueryRuntimeException -def check_query(clickhouse_node, query, result_set, retry_count=5, interval_seconds=30): +def check_query(clickhouse_node, query, result_set, retry_count=60, interval_seconds=3): lastest_result = '' for index in range(retry_count): lastest_result = clickhouse_node.query(query) @@ -20,6 +20,8 @@ def check_query(clickhouse_node, query, result_set, retry_count=5, interval_seco def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created @@ -102,6 +104,8 @@ def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_nam def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") @@ -134,6 +138,8 @@ def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_ def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") @@ -167,6 +173,8 @@ def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node, serv mysql_node.query("DROP DATABASE test_database") def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") @@ -195,6 +203,8 @@ def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node, se def rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") @@ -215,6 +225,8 @@ def rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, se def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") @@ -256,6 +268,8 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") @@ -288,6 +302,8 @@ def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_nod def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # maybe should test rename primary key? @@ -323,6 +339,8 @@ def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_n def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # maybe should test rename primary key? @@ -367,6 +385,8 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n # pass def alter_rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") @@ -402,6 +422,8 @@ def alter_rename_table_with_materialize_mysql_database(clickhouse_node, mysql_no def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database") mysql_node.query("RESET MASTER") @@ -434,6 +456,8 @@ def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name mysql_node.query("DROP DATABASE test_database") def select_without_columns(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS db") + clickhouse_node.query("DROP DATABASE IF EXISTS db") mysql_node.query("CREATE DATABASE db") mysql_node.query("CREATE TABLE db.t (a INT PRIMARY KEY, b INT)") clickhouse_node.query( @@ -465,6 +489,8 @@ def select_without_columns(clickhouse_node, mysql_node, service_name): def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (4), (5), (6);") From 66236d6ebb8af22eed790ddea1aa2f83d5ef404d Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Nov 2020 16:07:06 +0300 Subject: [PATCH 0184/1088] Refactored a little --- programs/server/Server.cpp | 274 ++++++++++---------- programs/server/Server.h | 14 + src/Common/ZooKeeper/TestKeeperStorage.cpp | 70 +++-- src/Common/ZooKeeper/TestKeeperStorage.h | 3 +- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 197 ++------------ src/Common/ZooKeeper/ZooKeeperCommon.h | 56 ++-- src/Common/ZooKeeper/ZooKeeperConstants.cpp | 64 +++++ src/Common/ZooKeeper/ZooKeeperConstants.h | 47 ++++ src/Common/ZooKeeper/ZooKeeperIO.cpp | 135 ++++++++++ src/Common/ZooKeeper/ZooKeeperIO.h | 73 ++++++ src/Common/ZooKeeper/ZooKeeperImpl.cpp | 134 ++-------- src/Common/ZooKeeper/ZooKeeperImpl.h | 3 - src/Server/TestKeeperTCPHandler.cpp | 246 +++--------------- src/Server/TestKeeperTCPHandler.h | 5 +- 14 files changed, 625 insertions(+), 696 deletions(-) create mode 100644 src/Common/ZooKeeper/ZooKeeperConstants.cpp create mode 100644 src/Common/ZooKeeper/ZooKeeperConstants.h create mode 100644 src/Common/ZooKeeper/ZooKeeperIO.cpp create mode 100644 src/Common/ZooKeeper/ZooKeeperIO.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 344082086e9..597b6c43fe5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -180,6 +180,85 @@ static std::string getUserName(uid_t user_id) return toString(user_id); } +Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, Poco::Logger * log) +{ + Poco::Net::SocketAddress socket_address; + try + { + socket_address = Poco::Net::SocketAddress(host, port); + } + catch (const Poco::Net::DNSException & e) + { + const auto code = e.code(); + if (code == EAI_FAMILY +#if defined(EAI_ADDRFAMILY) + || code == EAI_ADDRFAMILY +#endif + ) + { + LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. " + "If it is an IPv6 address and your host has disabled IPv6, then consider to " + "specify IPv4 address to listen in element of configuration " + "file. Example: 0.0.0.0", + host, e.code(), e.message()); + } + + throw; + } + return socket_address; +} + +Poco::Net::SocketAddress Server::socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure) +{ + auto address = makeSocketAddress(host, port, &logger()); +#if !defined(POCO_CLICKHOUSE_PATCH) || POCO_VERSION < 0x01090100 + if (secure) + /// Bug in old (<1.9.1) poco, listen() after bind() with reusePort param will fail because have no implementation in SecureServerSocketImpl + /// https://github.com/pocoproject/poco/pull/2257 + socket.bind(address, /* reuseAddress = */ true); + else +#endif +#if POCO_VERSION < 0x01080000 + socket.bind(address, /* reuseAddress = */ true); +#else + socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config().getBool("listen_reuse_port", false)); +#endif + + socket.listen(/* backlog = */ config().getUInt("listen_backlog", 64)); + + return address; +} + +void Server::createServer(const std::string & listen_host, const char * port_name, bool listen_try, CreateServerFunc && func) +{ + /// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file. + if (!config().has(port_name)) + return; + + auto port = config().getInt(port_name); + try + { + func(port); + } + catch (const Poco::Exception &) + { + std::string message = "Listen [" + listen_host + "]:" + std::to_string(port) + " failed: " + getCurrentExceptionMessage(false); + + if (listen_try) + { + LOG_WARNING(&logger(), "{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to " + "specify not disabled IPv4 or IPv6 address to listen in element of configuration " + "file. Example for disabled IPv6: 0.0.0.0 ." + " Example for disabled IPv4: ::", + message); + } + else + { + throw Exception{message, ErrorCodes::NETWORK_ERROR}; + } + } +} + void Server::uninitialize() { logger().information("shutting down"); @@ -667,6 +746,47 @@ int Server::main(const std::vector & /*args*/) total_memory_tracker.setDescription("(total)"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); + Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0); + + Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024)); + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(settings.http_receive_timeout); + http_params->setKeepAliveTimeout(keep_alive_timeout); + + std::vector> servers; + + std::vector listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host"); + + bool listen_try = config().getBool("listen_try", false); + if (listen_hosts.empty()) + { + listen_hosts.emplace_back("::1"); + listen_hosts.emplace_back("127.0.0.1"); + listen_try = true; + } + + for (const auto & listen_host : listen_hosts) + { + /// TCP TestKeeper + createServer(listen_host, "test_keeper_tcp_port", listen_try, [&](UInt16 port) + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + servers.emplace_back(std::make_unique( + new TCPHandlerFactory(*this, false, true), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + + LOG_INFO(log, "Listening for connections to fake zookeeper (tcp): {}", address.toString()); + }); + } + + for (auto & server : servers) + server->start(); + /// Set current database name before loading tables and databases because /// system logs may copy global context. global_context->setCurrentDatabaseNameInGlobalContext(default_database); @@ -797,74 +917,6 @@ int Server::main(const std::vector & /*args*/) #endif { - Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0); - - Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024)); - Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; - http_params->setTimeout(settings.http_receive_timeout); - http_params->setKeepAliveTimeout(keep_alive_timeout); - - std::vector> servers; - - std::vector listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host"); - - bool listen_try = config().getBool("listen_try", false); - if (listen_hosts.empty()) - { - listen_hosts.emplace_back("::1"); - listen_hosts.emplace_back("127.0.0.1"); - listen_try = true; - } - - auto make_socket_address = [&](const std::string & host, UInt16 port) - { - Poco::Net::SocketAddress socket_address; - try - { - socket_address = Poco::Net::SocketAddress(host, port); - } - catch (const Poco::Net::DNSException & e) - { - const auto code = e.code(); - if (code == EAI_FAMILY -#if defined(EAI_ADDRFAMILY) - || code == EAI_ADDRFAMILY -#endif - ) - { - LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. " - "If it is an IPv6 address and your host has disabled IPv6, then consider to " - "specify IPv4 address to listen in element of configuration " - "file. Example: 0.0.0.0", - host, e.code(), e.message()); - } - - throw; - } - return socket_address; - }; - - auto socket_bind_listen = [&](auto & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) - { - auto address = make_socket_address(host, port); -#if !defined(POCO_CLICKHOUSE_PATCH) || POCO_VERSION < 0x01090100 - if (secure) - /// Bug in old (<1.9.1) poco, listen() after bind() with reusePort param will fail because have no implementation in SecureServerSocketImpl - /// https://github.com/pocoproject/poco/pull/2257 - socket.bind(address, /* reuseAddress = */ true); - else -#endif -#if POCO_VERSION < 0x01080000 - socket.bind(address, /* reuseAddress = */ true); -#else - socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config().getBool("listen_reuse_port", false)); -#endif - - socket.listen(/* backlog = */ config().getUInt("listen_backlog", 64)); - - return address; - }; - /// This object will periodically calculate some metrics. AsynchronousMetrics async_metrics(*global_context, config().getUInt("asynchronous_metrics_update_period_s", 60)); @@ -872,41 +924,11 @@ int Server::main(const std::vector & /*args*/) for (const auto & listen_host : listen_hosts) { - auto create_server = [&](const char * port_name, auto && func) - { - /// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file. - if (!config().has(port_name)) - return; - - auto port = config().getInt(port_name); - try - { - func(port); - } - catch (const Poco::Exception &) - { - std::string message = "Listen [" + listen_host + "]:" + std::to_string(port) + " failed: " + getCurrentExceptionMessage(false); - - if (listen_try) - { - LOG_WARNING(log, "{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to " - "specify not disabled IPv4 or IPv6 address to listen in element of configuration " - "file. Example for disabled IPv6: 0.0.0.0 ." - " Example for disabled IPv4: ::", - message); - } - else - { - throw Exception{message, ErrorCodes::NETWORK_ERROR}; - } - } - }; - /// HTTP - create_server("http_port", [&](UInt16 port) + createServer(listen_host, "http_port", listen_try, [&](UInt16 port) { Poco::Net::ServerSocket socket; - auto address = socket_bind_listen(socket, listen_host, port); + auto address = socketBindListen(socket, listen_host, port); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); @@ -917,11 +939,11 @@ int Server::main(const std::vector & /*args*/) }); /// HTTPS - create_server("https_port", [&](UInt16 port) + createServer(listen_host, "https_port", listen_try, [&](UInt16 port) { #if USE_SSL Poco::Net::SecureServerSocket socket; - auto address = socket_bind_listen(socket, listen_host, port, /* secure = */ true); + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( @@ -936,10 +958,10 @@ int Server::main(const std::vector & /*args*/) }); /// TCP - create_server("tcp_port", [&](UInt16 port) + createServer(listen_host, "tcp_port", listen_try, [&](UInt16 port) { Poco::Net::ServerSocket socket; - auto address = socket_bind_listen(socket, listen_host, port); + auto address = socketBindListen(socket, listen_host, port); socket.setReceiveTimeout(settings.receive_timeout); socket.setSendTimeout(settings.send_timeout); servers.emplace_back(std::make_unique( @@ -951,28 +973,12 @@ int Server::main(const std::vector & /*args*/) LOG_INFO(log, "Listening for connections with native protocol (tcp): {}", address.toString()); }); - /// TCP TestKeeper - create_server("test_keeper_tcp_port", [&](UInt16 port) - { - Poco::Net::ServerSocket socket; - auto address = socket_bind_listen(socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - servers.emplace_back(std::make_unique( - new TCPHandlerFactory(*this, false, true), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - - LOG_INFO(log, "Listening for connections to fake zookeeper (tcp): {}", address.toString()); - }); - /// TCP with SSL - create_server("tcp_port_secure", [&](UInt16 port) + createServer(listen_host, "tcp_port_secure", listen_try, [&](UInt16 port) { #if USE_SSL Poco::Net::SecureServerSocket socket; - auto address = socket_bind_listen(socket, listen_host, port, /* secure = */ true); + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); socket.setReceiveTimeout(settings.receive_timeout); socket.setSendTimeout(settings.send_timeout); servers.emplace_back(std::make_unique( @@ -989,10 +995,10 @@ int Server::main(const std::vector & /*args*/) }); /// Interserver IO HTTP - create_server("interserver_http_port", [&](UInt16 port) + createServer(listen_host, "interserver_http_port", listen_try, [&](UInt16 port) { Poco::Net::ServerSocket socket; - auto address = socket_bind_listen(socket, listen_host, port); + auto address = socketBindListen(socket, listen_host, port); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( @@ -1001,11 +1007,11 @@ int Server::main(const std::vector & /*args*/) LOG_INFO(log, "Listening for replica communication (interserver): http://{}", address.toString()); }); - create_server("interserver_https_port", [&](UInt16 port) + createServer(listen_host, "interserver_https_port", listen_try, [&](UInt16 port) { #if USE_SSL Poco::Net::SecureServerSocket socket; - auto address = socket_bind_listen(socket, listen_host, port, /* secure = */ true); + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( @@ -1019,10 +1025,10 @@ int Server::main(const std::vector & /*args*/) #endif }); - create_server("mysql_port", [&](UInt16 port) + createServer(listen_host, "mysql_port", listen_try, [&](UInt16 port) { Poco::Net::ServerSocket socket; - auto address = socket_bind_listen(socket, listen_host, port, /* secure = */ true); + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); socket.setReceiveTimeout(Poco::Timespan()); socket.setSendTimeout(settings.send_timeout); servers.emplace_back(std::make_unique( @@ -1034,10 +1040,10 @@ int Server::main(const std::vector & /*args*/) LOG_INFO(log, "Listening for MySQL compatibility protocol: {}", address.toString()); }); - create_server("postgresql_port", [&](UInt16 port) + createServer(listen_host, "postgresql_port", listen_try, [&](UInt16 port) { Poco::Net::ServerSocket socket; - auto address = socket_bind_listen(socket, listen_host, port, /* secure = */ true); + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); socket.setReceiveTimeout(Poco::Timespan()); socket.setSendTimeout(settings.send_timeout); servers.emplace_back(std::make_unique( @@ -1050,10 +1056,10 @@ int Server::main(const std::vector & /*args*/) }); /// Prometheus (if defined and not setup yet with http_port) - create_server("prometheus.port", [&](UInt16 port) + createServer(listen_host, "prometheus.port", listen_try, [&](UInt16 port) { Poco::Net::ServerSocket socket; - auto address = socket_bind_listen(socket, listen_host, port); + auto address = socketBindListen(socket, listen_host, port); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( @@ -1069,8 +1075,8 @@ int Server::main(const std::vector & /*args*/) global_context->enableNamedSessions(); - for (auto & server : servers) - server->start(); + for (size_t i = 1; i < servers.size(); ++i) + servers[i]->start(); { String level_str = config().getString("text_log.level", ""); diff --git a/programs/server/Server.h b/programs/server/Server.h index ad9e51c881c..57fcea3cee8 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -14,6 +14,13 @@ * 3. Interserver HTTP - for replication. */ +namespace Poco +{ + namespace Net + { + class ServerSocket; + } +} namespace DB { @@ -57,6 +64,13 @@ protected: private: Context * global_context_ptr = nullptr; + +private: + + Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false); + + using CreateServerFunc = std::function; + void createServer(const std::string & listen_host, const char * port_name, bool listen_try, CreateServerFunc && func); }; } diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Common/ZooKeeper/TestKeeperStorage.cpp index b60144f4a87..4e291679b78 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.cpp +++ b/src/Common/ZooKeeper/TestKeeperStorage.cpp @@ -175,7 +175,6 @@ struct TestKeeperStorageGetRequest final : public TestKeeperStorageRequest using TestKeeperStorageRequest::TestKeeperStorageRequest; std::pair process(TestKeeperStorage::Container & container, int64_t /* zxid */) const override { - //LOG_DEBUG(&Poco::Logger::get("STORAGE"), "EXECUTING GET REQUEST"); Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); @@ -309,7 +308,7 @@ struct TestKeeperStorageSetRequest final : public TestKeeperStorageRequest response.error = Coordination::Error::ZBADVERSION; } - return { response_ptr, {} }; + return { response_ptr, undo }; } void processWatches(TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) const override @@ -327,7 +326,6 @@ struct TestKeeperStorageListRequest final : public TestKeeperStorageRequest Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); Coordination::ZooKeeperListRequest & request = dynamic_cast(*zk_request); - Undo undo; auto it = container.find(request.path); if (it == container.end()) { @@ -490,17 +488,14 @@ void TestKeeperStorage::processingThread() auto zk_request = info.request->zk_request; - info.request->zk_request->addRootPath(root_path); auto [response, _] = info.request->process(container, zxid); if (response->error == Coordination::Error::ZOK) { info.request->processWatches(watches, list_watches); } - ++zxid; response->xid = zk_request->xid; - response->zxid = zxid; - response->removeRootPath(root_path); + response->zxid = getZXID(); info.response_callback(response); } @@ -525,6 +520,36 @@ void TestKeeperStorage::finalize() } try { + { + auto finish_watch = [] (const auto & watch_pair) + { + Coordination::ZooKeeperWatchResponse response; + response.type = Coordination::SESSION; + response.state = Coordination::EXPIRED_SESSION; + response.error = Coordination::Error::ZSESSIONEXPIRED; + + for (auto & callback : watch_pair.second) + { + if (callback) + { + try + { + callback(std::make_shared(response)); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + } + }; + for (auto & path_watch : watches) + finish_watch(path_watch); + watches.clear(); + for (auto & path_watch : list_watches) + finish_watch(path_watch); + list_watches.clear(); + } RequestInfo info; while (requests_queue.tryPop(info)) { @@ -552,7 +577,7 @@ class TestKeeperWrapperFactory final : private boost::noncopyable public: using Creator = std::function; - using OpNumToRequest = std::unordered_map; + using OpNumToRequest = std::unordered_map; static TestKeeperWrapperFactory & instance() { @@ -564,12 +589,12 @@ public: { auto it = op_num_to_request.find(zk_request->getOpNum()); if (it == op_num_to_request.end()) - throw Coordination::Exception("Unknown operation type " + std::to_string(zk_request->getOpNum()), Coordination::Error::ZBADARGUMENTS); + throw Coordination::Exception("Unknown operation type " + toString(zk_request->getOpNum()), Coordination::Error::ZBADARGUMENTS); return it->second(zk_request); } - void registerRequest(int32_t op_num, Creator creator) + void registerRequest(Coordination::OpNum op_num, Creator creator) { if (!op_num_to_request.try_emplace(op_num, creator).second) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Request with op num {} already registered", op_num); @@ -582,7 +607,7 @@ private: TestKeeperWrapperFactory(); }; -template +template void registerTestKeeperRequestWrapper(TestKeeperWrapperFactory & factory) { factory.registerRequest(num, [] (const Coordination::ZooKeeperRequestPtr & zk_request) { return std::make_shared(zk_request); }); @@ -591,17 +616,17 @@ void registerTestKeeperRequestWrapper(TestKeeperWrapperFactory & factory) TestKeeperWrapperFactory::TestKeeperWrapperFactory() { - registerTestKeeperRequestWrapper<11, TestKeeperStorageHeartbeatRequest>(*this); - //registerTestKeeperRequestWrapper<100, TestKeeperStorageAuthRequest>(*this); - //registerTestKeeperRequestWrapper<-11, TestKeeperStorageCloseRequest>(*this); - registerTestKeeperRequestWrapper<1, TestKeeperStorageCreateRequest>(*this); - registerTestKeeperRequestWrapper<2, TestKeeperStorageRemoveRequest>(*this); - registerTestKeeperRequestWrapper<3, TestKeeperStorageExistsRequest>(*this); - registerTestKeeperRequestWrapper<4, TestKeeperStorageGetRequest>(*this); - registerTestKeeperRequestWrapper<5, TestKeeperStorageSetRequest>(*this); - registerTestKeeperRequestWrapper<12, TestKeeperStorageListRequest>(*this); - registerTestKeeperRequestWrapper<13, TestKeeperStorageCheckRequest>(*this); - registerTestKeeperRequestWrapper<14, TestKeeperStorageMultiRequest>(*this); + registerTestKeeperRequestWrapper(*this); + //registerTestKeeperRequestWrapper(*this); + //registerTestKeeperRequestWrapper(*this); + registerTestKeeperRequestWrapper(*this); + registerTestKeeperRequestWrapper(*this); + registerTestKeeperRequestWrapper(*this); + registerTestKeeperRequestWrapper(*this); + registerTestKeeperRequestWrapper(*this); + registerTestKeeperRequestWrapper(*this); + registerTestKeeperRequestWrapper(*this); + registerTestKeeperRequestWrapper(*this); } TestKeeperStorage::ResponsePair TestKeeperStorage::putRequest(const Coordination::ZooKeeperRequestPtr & request) @@ -624,7 +649,6 @@ TestKeeperStorage::ResponsePair TestKeeperStorage::putRequest(const Coordination std::lock_guard lock(push_request_mutex); if (!requests_queue.tryPush(std::move(request_info), operation_timeout.totalMilliseconds())) throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::LOGICAL_ERROR); - //LOG_DEBUG(&Poco::Logger::get("STORAGE"), "PUSHED"); return ResponsePair{std::move(future), std::move(watch_future)}; } diff --git a/src/Common/ZooKeeper/TestKeeperStorage.h b/src/Common/ZooKeeper/TestKeeperStorage.h index edcae28af85..875cb9f0253 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.h +++ b/src/Common/ZooKeeper/TestKeeperStorage.h @@ -39,8 +39,6 @@ public: Container container; - String root_path; - std::atomic zxid{0}; std::atomic shutdown{false}; @@ -56,6 +54,7 @@ public: ResponseCallback watch_callback; clock::time_point time; }; + std::mutex push_request_mutex; using RequestsQueue = ConcurrentBoundedQueue; RequestsQueue requests_queue{1}; diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index cb61f22fb3c..7b162492d1a 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -12,165 +13,6 @@ namespace Coordination using namespace DB; -/// ZooKeeper has 1 MB node size and serialization limit by default, -/// but it can be raised up, so we have a slightly larger limit on our side. -#define MAX_STRING_OR_ARRAY_SIZE (1 << 28) /// 256 MiB - -/// Assuming we are at little endian. - -static void write(int64_t x, WriteBuffer & out) -{ - x = __builtin_bswap64(x); - writeBinary(x, out); -} - -static void write(int32_t x, WriteBuffer & out) -{ - x = __builtin_bswap32(x); - writeBinary(x, out); -} - -static void write(bool x, WriteBuffer & out) -{ - writeBinary(x, out); -} - -static void write(const String & s, WriteBuffer & out) -{ - write(int32_t(s.size()), out); - out.write(s.data(), s.size()); -} - -template void write(std::array s, WriteBuffer & out) -{ - write(int32_t(N), out); - out.write(s.data(), N); -} - -template void write(const std::vector & arr, WriteBuffer & out) -{ - write(int32_t(arr.size()), out); - for (const auto & elem : arr) - write(elem, out); -} - -static void write(const ACL & acl, WriteBuffer & out) -{ - write(acl.permissions, out); - write(acl.scheme, out); - write(acl.id, out); -} - -static void write(const Stat & stat, WriteBuffer & out) -{ - write(stat.czxid, out); - write(stat.mzxid, out); - write(stat.ctime, out); - write(stat.mtime, out); - write(stat.version, out); - write(stat.cversion, out); - write(stat.aversion, out); - write(stat.ephemeralOwner, out); - write(stat.dataLength, out); - write(stat.numChildren, out); - write(stat.pzxid, out); -} - -static void write(const Error & x, WriteBuffer & out) -{ - write(static_cast(x), out); -} - -static void read(int64_t & x, ReadBuffer & in) -{ - readBinary(x, in); - x = __builtin_bswap64(x); -} - -static void read(int32_t & x, ReadBuffer & in) -{ - readBinary(x, in); - x = __builtin_bswap32(x); -} - -static void read(Error & x, ReadBuffer & in) -{ - int32_t code; - read(code, in); - x = Error(code); -} - -static void read(bool & x, ReadBuffer & in) -{ - readBinary(x, in); -} - -static void read(String & s, ReadBuffer & in) -{ - int32_t size = 0; - read(size, in); - - if (size == -1) - { - /// It means that zookeeper node has NULL value. We will treat it like empty string. - s.clear(); - return; - } - - if (size < 0) - throw Exception("Negative size while reading string from ZooKeeper", Error::ZMARSHALLINGERROR); - - if (size > MAX_STRING_OR_ARRAY_SIZE) - throw Exception("Too large string size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); - - s.resize(size); - in.read(s.data(), size); -} - -template void read(std::array & s, ReadBuffer & in) -{ - int32_t size = 0; - read(size, in); - if (size != N) - throw Exception("Unexpected array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); - in.read(s.data(), N); -} - -static void read(Stat & stat, ReadBuffer & in) -{ - read(stat.czxid, in); - read(stat.mzxid, in); - read(stat.ctime, in); - read(stat.mtime, in); - read(stat.version, in); - read(stat.cversion, in); - read(stat.aversion, in); - read(stat.ephemeralOwner, in); - read(stat.dataLength, in); - read(stat.numChildren, in); - read(stat.pzxid, in); -} - -template void read(std::vector & arr, ReadBuffer & in) -{ - int32_t size = 0; - read(size, in); - if (size < 0) - throw Exception("Negative size while reading array from ZooKeeper", Error::ZMARSHALLINGERROR); - if (size > MAX_STRING_OR_ARRAY_SIZE) - throw Exception("Too large array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); - arr.resize(size); - for (auto & elem : arr) - read(elem, in); -} - -static void read(ACL & acl, ReadBuffer & in) -{ - read(acl.permissions, in); - read(acl.scheme, in); - read(acl.id, in); -} - void ZooKeeperResponse::write(WriteBuffer & out) const { /// Excessive copy to calculate length. @@ -263,7 +105,6 @@ void ZooKeeperCreateResponse::readImpl(ReadBuffer & in) void ZooKeeperCreateResponse::writeImpl(WriteBuffer & out) const { - LOG_DEBUG(&Poco::Logger::get("LOG"), "WRITE IMPL ON: {}", path_created); Coordination::write(path_created, out); } @@ -448,7 +289,7 @@ void ZooKeeperMultiRequest::writeImpl(WriteBuffer & out) const zk_request.writeImpl(out); } - OpNum op_num = -1; + OpNum op_num = OpNum::Error; bool done = true; int32_t error = -1; @@ -471,7 +312,7 @@ void ZooKeeperMultiRequest::readImpl(ReadBuffer & in) if (done) { - if (op_num != -1) + if (op_num != OpNum::Error) throw Exception("Unexpected op_num received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); if (error != -1) throw Exception("Unexpected error value received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); @@ -505,7 +346,7 @@ void ZooKeeperMultiResponse::readImpl(ReadBuffer & in) /// op_num == -1 is special for multi transaction. /// For unknown reason, error code is duplicated in header and in response body. - if (op_num == -1) + if (op_num == OpNum::Error) response = std::make_shared(); if (op_error != Error::ZOK) @@ -519,7 +360,7 @@ void ZooKeeperMultiResponse::readImpl(ReadBuffer & in) error = op_error; } - if (op_error == Error::ZOK || op_num == -1) + if (op_error == Error::ZOK || op_num == OpNum::Error) dynamic_cast(*response).readImpl(in); } @@ -535,7 +376,7 @@ void ZooKeeperMultiResponse::readImpl(ReadBuffer & in) if (!done) throw Exception("Too many results received for multi transaction", Error::ZMARSHALLINGERROR); - if (op_num != -1) + if (op_num != OpNum::Error) throw Exception("Unexpected op_num received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); if (error_read != -1) throw Exception("Unexpected error value received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); @@ -560,7 +401,7 @@ void ZooKeeperMultiResponse::writeImpl(WriteBuffer & out) const /// Footer. { - OpNum op_num = -1; + OpNum op_num = OpNum::Error; bool done = true; int32_t error_read = - 1; @@ -606,7 +447,7 @@ ZooKeeperRequestPtr ZooKeeperRequestFactory::get(OpNum op_num) const { auto it = op_num_to_request.find(op_num); if (it == op_num_to_request.end()) - throw Exception("Unknown operation type " + std::to_string(op_num), Error::ZBADARGUMENTS); + throw Exception("Unknown operation type " + toString(op_num), Error::ZBADARGUMENTS); return it->second(); } @@ -625,17 +466,17 @@ void registerZooKeeperRequest(ZooKeeperRequestFactory & factory) ZooKeeperRequestFactory::ZooKeeperRequestFactory() { - registerZooKeeperRequest<11, ZooKeeperHeartbeatRequest>(*this); - registerZooKeeperRequest<100, ZooKeeperAuthRequest>(*this); - registerZooKeeperRequest<-11, ZooKeeperCloseRequest>(*this); - registerZooKeeperRequest<1, ZooKeeperCreateRequest>(*this); - registerZooKeeperRequest<2, ZooKeeperRemoveRequest>(*this); - registerZooKeeperRequest<3, ZooKeeperExistsRequest>(*this); - registerZooKeeperRequest<4, ZooKeeperGetRequest>(*this); - registerZooKeeperRequest<5, ZooKeeperSetRequest>(*this); - registerZooKeeperRequest<12, ZooKeeperListRequest>(*this); - registerZooKeeperRequest<13, ZooKeeperCheckRequest>(*this); - registerZooKeeperRequest<14, ZooKeeperMultiRequest>(*this); + registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); } } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index f40de116da2..286330bd769 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -22,9 +23,6 @@ namespace Coordination { -using XID = int32_t; -using OpNum = int32_t; - struct ZooKeeperResponse : virtual Response { XID xid = 0; @@ -70,7 +68,7 @@ using ZooKeeperRequestPtr = std::shared_ptr; struct ZooKeeperHeartbeatRequest final : ZooKeeperRequest { String getPath() const override { return {}; } - OpNum getOpNum() const override { return 11; } + OpNum getOpNum() const override { return OpNum::Heartbeat; } void writeImpl(WriteBuffer &) const override {} void readImpl(ReadBuffer &) override {} ZooKeeperResponsePtr makeResponse() const override; @@ -80,7 +78,7 @@ struct ZooKeeperHeartbeatResponse final : ZooKeeperResponse { void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} - OpNum getOpNum() const override { return 11; } + OpNum getOpNum() const override { return OpNum::Heartbeat; } }; struct ZooKeeperWatchResponse final : WatchResponse, ZooKeeperResponse @@ -89,8 +87,10 @@ struct ZooKeeperWatchResponse final : WatchResponse, ZooKeeperResponse void writeImpl(WriteBuffer & out) const override; - /// TODO FIXME alesap - OpNum getOpNum() const override { return 0; } + OpNum getOpNum() const override + { + throw Exception("OpNum for watch response doesn't exist", Error::ZRUNTIMEINCONSISTENCY); + } }; struct ZooKeeperAuthRequest final : ZooKeeperRequest @@ -100,7 +100,7 @@ struct ZooKeeperAuthRequest final : ZooKeeperRequest String data; String getPath() const override { return {}; } - OpNum getOpNum() const override { return 100; } + OpNum getOpNum() const override { return OpNum::Auth; } void writeImpl(WriteBuffer & out) const override; void readImpl(ReadBuffer & in) override; @@ -112,13 +112,13 @@ struct ZooKeeperAuthResponse final : ZooKeeperResponse void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} - OpNum getOpNum() const override { return 100; } + OpNum getOpNum() const override { return OpNum::Auth; } }; struct ZooKeeperCloseRequest final : ZooKeeperRequest { String getPath() const override { return {}; } - OpNum getOpNum() const override { return -11; } + OpNum getOpNum() const override { return OpNum::Close; } void writeImpl(WriteBuffer &) const override {} void readImpl(ReadBuffer &) override {} @@ -134,7 +134,7 @@ struct ZooKeeperCloseResponse final : ZooKeeperResponse void writeImpl(WriteBuffer &) const override {} - OpNum getOpNum() const override { return -11; } + OpNum getOpNum() const override { return OpNum::Close; } }; struct ZooKeeperCreateRequest final : public CreateRequest, ZooKeeperRequest @@ -142,7 +142,7 @@ struct ZooKeeperCreateRequest final : public CreateRequest, ZooKeeperRequest ZooKeeperCreateRequest() = default; explicit ZooKeeperCreateRequest(const CreateRequest & base) : CreateRequest(base) {} - OpNum getOpNum() const override { return 1; } + OpNum getOpNum() const override { return OpNum::Create; } void writeImpl(WriteBuffer & out) const override; void readImpl(ReadBuffer & in) override; @@ -155,7 +155,7 @@ struct ZooKeeperCreateResponse final : CreateResponse, ZooKeeperResponse void writeImpl(WriteBuffer & out) const override; - OpNum getOpNum() const override { return 1; } + OpNum getOpNum() const override { return OpNum::Create; } }; struct ZooKeeperRemoveRequest final : RemoveRequest, ZooKeeperRequest @@ -163,7 +163,7 @@ struct ZooKeeperRemoveRequest final : RemoveRequest, ZooKeeperRequest ZooKeeperRemoveRequest() = default; explicit ZooKeeperRemoveRequest(const RemoveRequest & base) : RemoveRequest(base) {} - OpNum getOpNum() const override { return 2; } + OpNum getOpNum() const override { return OpNum::Remove; } void writeImpl(WriteBuffer & out) const override; void readImpl(ReadBuffer & in) override; @@ -174,12 +174,12 @@ struct ZooKeeperRemoveResponse final : RemoveResponse, ZooKeeperResponse { void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} - OpNum getOpNum() const override { return 2; } + OpNum getOpNum() const override { return OpNum::Remove; } }; struct ZooKeeperExistsRequest final : ExistsRequest, ZooKeeperRequest { - OpNum getOpNum() const override { return 3; } + OpNum getOpNum() const override { return OpNum::Exists; } void writeImpl(WriteBuffer & out) const override; void readImpl(ReadBuffer & in) override; @@ -190,12 +190,12 @@ struct ZooKeeperExistsResponse final : ExistsResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; - OpNum getOpNum() const override { return 3; } + OpNum getOpNum() const override { return OpNum::Exists; } }; struct ZooKeeperGetRequest final : GetRequest, ZooKeeperRequest { - OpNum getOpNum() const override { return 4; } + OpNum getOpNum() const override { return OpNum::Get; } void writeImpl(WriteBuffer & out) const override; void readImpl(ReadBuffer & in) override; @@ -206,7 +206,7 @@ struct ZooKeeperGetResponse final : GetResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; - OpNum getOpNum() const override { return 4; } + OpNum getOpNum() const override { return OpNum::Get; } }; struct ZooKeeperSetRequest final : SetRequest, ZooKeeperRequest @@ -214,7 +214,7 @@ struct ZooKeeperSetRequest final : SetRequest, ZooKeeperRequest ZooKeeperSetRequest() = default; explicit ZooKeeperSetRequest(const SetRequest & base) : SetRequest(base) {} - OpNum getOpNum() const override { return 5; } + OpNum getOpNum() const override { return OpNum::Set; } void writeImpl(WriteBuffer & out) const override; void readImpl(ReadBuffer & in) override; ZooKeeperResponsePtr makeResponse() const override; @@ -224,12 +224,12 @@ struct ZooKeeperSetResponse final : SetResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; - OpNum getOpNum() const override { return 5; } + OpNum getOpNum() const override { return OpNum::Set; } }; struct ZooKeeperListRequest final : ListRequest, ZooKeeperRequest { - OpNum getOpNum() const override { return 12; } + OpNum getOpNum() const override { return OpNum::List; } void writeImpl(WriteBuffer & out) const override; void readImpl(ReadBuffer & in) override; ZooKeeperResponsePtr makeResponse() const override; @@ -239,7 +239,7 @@ struct ZooKeeperListResponse final : ListResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; - OpNum getOpNum() const override { return 12; } + OpNum getOpNum() const override { return OpNum::List; } }; struct ZooKeeperCheckRequest final : CheckRequest, ZooKeeperRequest @@ -247,7 +247,7 @@ struct ZooKeeperCheckRequest final : CheckRequest, ZooKeeperRequest ZooKeeperCheckRequest() = default; explicit ZooKeeperCheckRequest(const CheckRequest & base) : CheckRequest(base) {} - OpNum getOpNum() const override { return 13; } + OpNum getOpNum() const override { return OpNum::Check; } void writeImpl(WriteBuffer & out) const override; void readImpl(ReadBuffer & in) override; @@ -258,7 +258,7 @@ struct ZooKeeperCheckResponse final : CheckResponse, ZooKeeperResponse { void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} - OpNum getOpNum() const override { return 13; } + OpNum getOpNum() const override { return OpNum::Check; } }; /// This response may be received only as an element of responses in MultiResponse. @@ -267,12 +267,12 @@ struct ZooKeeperErrorResponse final : ErrorResponse, ZooKeeperResponse void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; - OpNum getOpNum() const override { return -1; } + OpNum getOpNum() const override { return OpNum::Error; } }; struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest { - OpNum getOpNum() const override { return 14; } + OpNum getOpNum() const override { return OpNum::Multi; } ZooKeeperMultiRequest() = default; ZooKeeperMultiRequest(const Requests & generic_requests, const ACLs & default_acls); @@ -285,7 +285,7 @@ struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest struct ZooKeeperMultiResponse final : MultiResponse, ZooKeeperResponse { - OpNum getOpNum() const override { return 14; } + OpNum getOpNum() const override { return OpNum::Multi; } explicit ZooKeeperMultiResponse(const Requests & requests) { diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.cpp b/src/Common/ZooKeeper/ZooKeeperConstants.cpp new file mode 100644 index 00000000000..9a1a43d43a7 --- /dev/null +++ b/src/Common/ZooKeeper/ZooKeeperConstants.cpp @@ -0,0 +1,64 @@ +#include +#include +#include + +namespace Coordination +{ + +static const std::unordered_set VALID_OPERATIONS = +{ + static_cast(OpNum::Close), + static_cast(OpNum::Error), + static_cast(OpNum::Create), + static_cast(OpNum::Remove), + static_cast(OpNum::Exists), + static_cast(OpNum::Get), + static_cast(OpNum::Set), + static_cast(OpNum::Heartbeat), + static_cast(OpNum::List), + static_cast(OpNum::Check), + static_cast(OpNum::Multi), + static_cast(OpNum::Auth), +}; + +std::string toString(OpNum op_num) +{ + switch (op_num) + { + case OpNum::Close: + return "Close"; + case OpNum::Error: + return "Error"; + case OpNum::Create: + return "Create"; + case OpNum::Remove: + return "Remove"; + case OpNum::Exists: + return "Exists"; + case OpNum::Get: + return "Get"; + case OpNum::Set: + return "Set"; + case OpNum::List: + return "List"; + case OpNum::Check: + return "Check"; + case OpNum::Multi: + return "Multi"; + case OpNum::Heartbeat: + return "Heartbeat"; + case OpNum::Auth: + return "Auth"; + } + int32_t raw_op = static_cast(op_num); + throw Exception("Operation " + std::to_string(raw_op) + " is unknown", Error::ZUNIMPLEMENTED); +} + +OpNum getOpNum(int32_t raw_op_num) +{ + if (!VALID_OPERATIONS.count(raw_op_num)) + throw Exception("Operation " + std::to_string(raw_op_num) + " is unknown", Error::ZUNIMPLEMENTED); + return static_cast(raw_op_num); +} + +} diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h new file mode 100644 index 00000000000..47b597f1c9d --- /dev/null +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include + + +namespace Coordination +{ + +using XID = int32_t; + +static constexpr XID WATCH_XID = -1; +static constexpr XID PING_XID = -2; +static constexpr XID AUTH_XID = -4; +static constexpr XID CLOSE_XID = 0x7FFFFFFF; + +enum class OpNum : int32_t +{ + Close = -11, + Error = -1, + Create = 1, + Remove = 2, + Exists = 3, + Get = 4, + Set = 5, + Heartbeat = 11, + List = 12, + Check = 13, + Multi = 14, + Auth = 100, +}; + +std::string toString(OpNum op_num); +OpNum getOpNum(int32_t raw_op_num); + +static constexpr int32_t ZOOKEEPER_PROTOCOL_VERSION = 0; +static constexpr int32_t CLIENT_HANDSHAKE_LENGTH = 44; +static constexpr int32_t SERVER_HANDSHAKE_LENGTH = 36; +static constexpr int32_t PASSWORD_LENGTH = 16; + +/// ZooKeeper has 1 MB node size and serialization limit by default, +/// but it can be raised up, so we have a slightly larger limit on our side. +static constexpr int32_t MAX_STRING_OR_ARRAY_SIZE = 1 << 28; /// 256 MiB +static constexpr int32_t DEFAULT_SESSION_TIMEOUT = 30000; +static constexpr int32_t DEFAULT_OPERATION_TIMEOUT = 10000; + +} diff --git a/src/Common/ZooKeeper/ZooKeeperIO.cpp b/src/Common/ZooKeeper/ZooKeeperIO.cpp new file mode 100644 index 00000000000..157325cb8e6 --- /dev/null +++ b/src/Common/ZooKeeper/ZooKeeperIO.cpp @@ -0,0 +1,135 @@ +#include + +namespace Coordination +{ + +void write(int64_t x, WriteBuffer & out) +{ + x = __builtin_bswap64(x); + writeBinary(x, out); +} +void write(int32_t x, WriteBuffer & out) +{ + x = __builtin_bswap32(x); + writeBinary(x, out); +} + +void write(OpNum x, WriteBuffer & out) +{ + write(static_cast(x), out); +} + +void write(bool x, WriteBuffer & out) +{ + writeBinary(x, out); +} + +void write(const std::string & s, WriteBuffer & out) +{ + write(int32_t(s.size()), out); + out.write(s.data(), s.size()); +} + +void write(const ACL & acl, WriteBuffer & out) +{ + write(acl.permissions, out); + write(acl.scheme, out); + write(acl.id, out); +} + +void write(const Stat & stat, WriteBuffer & out) +{ + write(stat.czxid, out); + write(stat.mzxid, out); + write(stat.ctime, out); + write(stat.mtime, out); + write(stat.version, out); + write(stat.cversion, out); + write(stat.aversion, out); + write(stat.ephemeralOwner, out); + write(stat.dataLength, out); + write(stat.numChildren, out); + write(stat.pzxid, out); +} + +void write(const Error & x, WriteBuffer & out) +{ + write(static_cast(x), out); +} + +void read(int64_t & x, ReadBuffer & in) +{ + readBinary(x, in); + x = __builtin_bswap64(x); +} + +void read(int32_t & x, ReadBuffer & in) +{ + readBinary(x, in); + x = __builtin_bswap32(x); +} + +void read(OpNum & x, ReadBuffer & in) +{ + int32_t raw_op_num; + read(raw_op_num, in); + x = getOpNum(raw_op_num); +} + +void read(bool & x, ReadBuffer & in) +{ + readBinary(x, in); +} + +void read(std::string & s, ReadBuffer & in) +{ + int32_t size = 0; + read(size, in); + + if (size == -1) + { + /// It means that zookeeper node has NULL value. We will treat it like empty string. + s.clear(); + return; + } + + if (size < 0) + throw Exception("Negative size while reading string from ZooKeeper", Error::ZMARSHALLINGERROR); + + if (size > MAX_STRING_OR_ARRAY_SIZE) + throw Exception("Too large string size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); + + s.resize(size); + in.read(s.data(), size); +} + +void read(ACL & acl, ReadBuffer & in) +{ + read(acl.permissions, in); + read(acl.scheme, in); + read(acl.id, in); +} + +void read(Stat & stat, ReadBuffer & in) +{ + read(stat.czxid, in); + read(stat.mzxid, in); + read(stat.ctime, in); + read(stat.mtime, in); + read(stat.version, in); + read(stat.cversion, in); + read(stat.aversion, in); + read(stat.ephemeralOwner, in); + read(stat.dataLength, in); + read(stat.numChildren, in); + read(stat.pzxid, in); +} + +void read(Error & x, ReadBuffer & in) +{ + int32_t code; + read(code, in); + x = Coordination::Error(code); +} + +} diff --git a/src/Common/ZooKeeper/ZooKeeperIO.h b/src/Common/ZooKeeper/ZooKeeperIO.h new file mode 100644 index 00000000000..bd6d6a6f849 --- /dev/null +++ b/src/Common/ZooKeeper/ZooKeeperIO.h @@ -0,0 +1,73 @@ +#pragma once +#include +#include +#include +#include +#include +#include +#include +#include + +namespace Coordination +{ + +using namespace DB; + +void write(int64_t x, WriteBuffer & out); +void write(int32_t x, WriteBuffer & out); +void write(OpNum x, WriteBuffer & out); +void write(bool x, WriteBuffer & out); +void write(const std::string & s, WriteBuffer & out); +void write(const ACL & acl, WriteBuffer & out); +void write(const Stat & stat, WriteBuffer & out); +void write(const Error & x, WriteBuffer & out); + +template +void write(const std::array s, WriteBuffer & out) +{ + write(int32_t(N), out); + out.write(s.data(), N); +} + +template +void write(const std::vector & arr, WriteBuffer & out) +{ + write(int32_t(arr.size()), out); + for (const auto & elem : arr) + write(elem, out); +} + +void read(int64_t & x, ReadBuffer & in); +void read(int32_t & x, ReadBuffer & in); +void read(OpNum & x, ReadBuffer & in); +void read(bool & x, ReadBuffer & in); +void read(std::string & s, ReadBuffer & in); +void read(ACL & acl, ReadBuffer & in); +void read(Stat & stat, ReadBuffer & in); +void read(Error & x, ReadBuffer & in); + +template +void read(std::array & s, ReadBuffer & in) +{ + int32_t size = 0; + read(size, in); + if (size != N) + throw Exception("Unexpected array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); + in.read(s.data(), N); +} + +template +void read(std::vector & arr, ReadBuffer & in) +{ + int32_t size = 0; + read(size, in); + if (size < 0) + throw Exception("Negative size while reading array from ZooKeeper", Error::ZMARSHALLINGERROR); + if (size > MAX_STRING_OR_ARRAY_SIZE) + throw Exception("Too large array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); + arr.resize(size); + for (auto & elem : arr) + read(elem, in); +} + +} diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 23a5c6c4301..c9ef2a26edd 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -2,7 +2,7 @@ #include #include #include - +#include #include #include #include @@ -20,11 +20,6 @@ #include -/// ZooKeeper has 1 MB node size and serialization limit by default, -/// but it can be raised up, so we have a slightly larger limit on our side. -#define MAX_STRING_OR_ARRAY_SIZE (1 << 28) /// 256 MiB - - namespace ProfileEvents { extern const Event ZooKeeperInit; @@ -267,75 +262,6 @@ namespace Coordination using namespace DB; - -/// Assuming we are at little endian. - -static void write(int64_t x, WriteBuffer & out) -{ - x = __builtin_bswap64(x); - writeBinary(x, out); -} - -static void write(int32_t x, WriteBuffer & out) -{ - x = __builtin_bswap32(x); - writeBinary(x, out); -} - -template void write(std::array s, WriteBuffer & out) -{ - write(int32_t(N), out); - out.write(s.data(), N); -} - -template void write(const std::vector & arr, WriteBuffer & out) -{ - write(int32_t(arr.size()), out); - for (const auto & elem : arr) - write(elem, out); -} - -static void read(int64_t & x, ReadBuffer & in) -{ - readBinary(x, in); - x = __builtin_bswap64(x); -} - -static void read(int32_t & x, ReadBuffer & in) -{ - readBinary(x, in); - x = __builtin_bswap32(x); -} - -static void read(Error & x, ReadBuffer & in) -{ - int32_t code; - read(code, in); - x = Error(code); -} - -template void read(std::array & s, ReadBuffer & in) -{ - int32_t size = 0; - read(size, in); - if (size != N) - throw Exception("Unexpected array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); - in.read(s.data(), N); -} - -template void read(std::vector & arr, ReadBuffer & in) -{ - int32_t size = 0; - read(size, in); - if (size < 0) - throw Exception("Negative size while reading array from ZooKeeper", Error::ZMARSHALLINGERROR); - if (size > MAX_STRING_OR_ARRAY_SIZE) - throw Exception("Too large array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); - arr.resize(size); - for (auto & elem : arr) - read(elem, in); -} - template void ZooKeeper::write(const T & x) { @@ -359,15 +285,6 @@ static void removeRootPath(String & path, const String & root_path) path = path.substr(root_path.size()); } -static constexpr int32_t protocol_version = 0; - -static constexpr ZooKeeper::XID watch_xid = -1; -static constexpr ZooKeeper::XID ping_xid = -2; -static constexpr ZooKeeper::XID auth_xid = -4; - -static constexpr ZooKeeper::XID close_xid = 0x7FFFFFFF; - - ZooKeeper::~ZooKeeper() { try @@ -542,7 +459,7 @@ void ZooKeeper::sendHandshake() std::array passwd {}; write(handshake_length); - write(protocol_version); + write(ZOOKEEPER_PROTOCOL_VERSION); write(last_zxid_seen); write(timeout); write(previous_session_id); @@ -557,16 +474,15 @@ void ZooKeeper::receiveHandshake() int32_t handshake_length; int32_t protocol_version_read; int32_t timeout; - constexpr int32_t passwd_len = 16; - std::array passwd; + std::array passwd; read(handshake_length); - if (handshake_length != 36) - throw Exception("Unexpected handshake length received: " + toString(handshake_length), Error::ZMARSHALLINGERROR); + if (handshake_length != SERVER_HANDSHAKE_LENGTH) + throw Exception("Unexpected handshake length received: " + DB::toString(handshake_length), Error::ZMARSHALLINGERROR); read(protocol_version_read); - if (protocol_version_read != protocol_version) - throw Exception("Unexpected protocol version: " + toString(protocol_version_read), Error::ZMARSHALLINGERROR); + if (protocol_version_read != ZOOKEEPER_PROTOCOL_VERSION) + throw Exception("Unexpected protocol version: " + DB::toString(protocol_version_read), Error::ZMARSHALLINGERROR); read(timeout); if (timeout != session_timeout.totalMilliseconds()) @@ -583,7 +499,7 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) ZooKeeperAuthRequest request; request.scheme = scheme; request.data = data; - request.xid = auth_xid; + request.xid = AUTH_XID; request.write(*out); int32_t length; @@ -597,17 +513,17 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) read(zxid); read(err); - if (read_xid != auth_xid) - throw Exception("Unexpected event received in reply to auth request: " + toString(read_xid), + if (read_xid != AUTH_XID) + throw Exception("Unexpected event received in reply to auth request: " + DB::toString(read_xid), Error::ZMARSHALLINGERROR); int32_t actual_length = in->count() - count_before_event; if (length != actual_length) - throw Exception("Response length doesn't match. Expected: " + toString(length) + ", actual: " + toString(actual_length), + throw Exception("Response length doesn't match. Expected: " + DB::toString(length) + ", actual: " + DB::toString(actual_length), Error::ZMARSHALLINGERROR); if (err != Error::ZOK) - throw Exception("Error received in reply to auth request. Code: " + toString(int32_t(err)) + ". Message: " + String(errorMessage(err)), + throw Exception("Error received in reply to auth request. Code: " + DB::toString(int32_t(err)) + ". Message: " + String(errorMessage(err)), Error::ZMARSHALLINGERROR); } @@ -640,7 +556,7 @@ void ZooKeeper::sendThread() /// After we popped element from the queue, we must register callbacks (even in the case when expired == true right now), /// because they must not be lost (callbacks must be called because the user will wait for them). - if (info.request->xid != close_xid) + if (info.request->xid != CLOSE_XID) { CurrentMetrics::add(CurrentMetrics::ZooKeeperRequest); std::lock_guard lock(operations_mutex); @@ -650,7 +566,6 @@ void ZooKeeper::sendThread() if (info.watch) { info.request->has_watch = true; - //std::cerr << "REQUEST" << info.request->getOpNum() << " HAS WATCH" << std::endl; CurrentMetrics::add(CurrentMetrics::ZooKeeperWatch); } @@ -662,11 +577,10 @@ void ZooKeeper::sendThread() info.request->addRootPath(root_path); info.request->probably_sent = true; - //std::cerr << "SENDING GENERAL REQUEST:" << info.request->getOpNum() << std::endl; info.request->write(*out); /// We sent close request, exit - if (info.request->xid == close_xid) + if (info.request->xid == CLOSE_XID) break; } } @@ -676,7 +590,7 @@ void ZooKeeper::sendThread() prev_heartbeat_time = clock::now(); ZooKeeperHeartbeatRequest request; - request.xid = ping_xid; + request.xid = PING_XID; request.write(*out); } @@ -731,7 +645,7 @@ void ZooKeeper::receiveThread() { if (earliest_operation) { - throw Exception("Operation timeout (no response) for request " + std::to_string(earliest_operation->request->getOpNum()) + " for path: " + earliest_operation->request->getPath(), Error::ZOPERATIONTIMEOUT); + throw Exception("Operation timeout (no response) for request " + toString(earliest_operation->request->getOpNum()) + " for path: " + earliest_operation->request->getPath(), Error::ZOPERATIONTIMEOUT); } waited += max_wait; if (waited >= session_timeout.totalMicroseconds()) @@ -766,14 +680,14 @@ void ZooKeeper::receiveEvent() RequestInfo request_info; ZooKeeperResponsePtr response; - if (xid == ping_xid) + if (xid == PING_XID) { if (err != Error::ZOK) throw Exception("Received error in heartbeat response: " + String(errorMessage(err)), Error::ZRUNTIMEINCONSISTENCY); response = std::make_shared(); } - else if (xid == watch_xid) + else if (xid == WATCH_XID) { ProfileEvents::increment(ProfileEvents::ZooKeeperWatchResponse); response = std::make_shared(); @@ -814,7 +728,7 @@ void ZooKeeper::receiveEvent() auto it = operations.find(xid); if (it == operations.end()) - throw Exception("Received response for unknown xid " + toString(xid), Error::ZRUNTIMEINCONSISTENCY); + throw Exception("Received response for unknown xid " + DB::toString(xid), Error::ZRUNTIMEINCONSISTENCY); /// After this point, we must invoke callback, that we've grabbed from 'operations'. /// Invariant: all callbacks are invoked either in case of success or in case of error. @@ -836,12 +750,10 @@ void ZooKeeper::receiveEvent() if (err != Error::ZOK) { - //std::cerr << "GOT ERROR:" << static_cast(err) << std::endl; response->error = err; } else { - //std::cerr << "NO ERROR RECEIVED\n"; response->readImpl(*in); response->removeRootPath(root_path); } @@ -854,7 +766,7 @@ void ZooKeeper::receiveEvent() /// 3 indicates the ZooKeeperExistsRequest. // For exists, we set the watch on both node exist and nonexist case. // For other case like getData, we only set the watch when node exists. - if (request_info.request->getOpNum() == 3) + if (request_info.request->getOpNum() == OpNum::Exists) add_watch = (response->error == Error::ZOK || response->error == Error::ZNONODE); else add_watch = response->error == Error::ZOK; @@ -871,7 +783,7 @@ void ZooKeeper::receiveEvent() int32_t actual_length = in->count() - count_before_event; if (length != actual_length) - throw Exception("Response length doesn't match. Expected: " + toString(length) + ", actual: " + toString(actual_length), Error::ZMARSHALLINGERROR); + throw Exception("Response length doesn't match. Expected: " + DB::toString(length) + ", actual: " + DB::toString(actual_length), Error::ZMARSHALLINGERROR); } catch (...) { @@ -1058,7 +970,7 @@ void ZooKeeper::pushRequest(RequestInfo && info) if (!info.request->xid) { info.request->xid = next_xid.fetch_add(1); - if (info.request->xid == close_xid) + if (info.request->xid == CLOSE_XID) throw Exception("xid equal to close_xid", Error::ZSESSIONEXPIRED); if (info.request->xid < 0) throw Exception("XID overflow", Error::ZSESSIONEXPIRED); @@ -1238,7 +1150,7 @@ void ZooKeeper::multi( void ZooKeeper::close() { ZooKeeperCloseRequest request; - request.xid = close_xid; + request.xid = CLOSE_XID; RequestInfo request_info; request_info.request = std::make_shared(std::move(request)); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index c96d7d2f0cb..65f8fe91793 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -99,9 +99,6 @@ public: using Nodes = std::vector; - using XID = int32_t; - using OpNum = int32_t; - /** Connection to nodes is performed in order. If you want, shuffle them manually. * Operation timeout couldn't be greater than session timeout. * Operation timeout applies independently for network read, network write, waiting for events and synchronization. diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index 1e7f69dc14f..7afd1af1aa0 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -1,9 +1,6 @@ #include +#include #include -#include -#include -#include -#include #include #include #include @@ -23,187 +20,15 @@ namespace ErrorCodes } -#ifdef __clang__ -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wunused-function" -#endif - -/// ZooKeeper has 1 MB node size and serialization limit by default, -/// but it can be raised up, so we have a slightly larger limit on our side. -#define MAX_STRING_OR_ARRAY_SIZE (1 << 28) /// 256 MiB - -/// Assuming we are at little endian. - -static void write(int64_t x, WriteBuffer & out) -{ - x = __builtin_bswap64(x); - writeBinary(x, out); -} - -static void write(int32_t x, WriteBuffer & out) -{ - x = __builtin_bswap32(x); - writeBinary(x, out); -} - -static void write(bool x, WriteBuffer & out) -{ - writeBinary(x, out); -} - -static void write(const String & s, WriteBuffer & out) -{ - write(int32_t(s.size()), out); - out.write(s.data(), s.size()); -} - -template void write(std::array s, WriteBuffer & out) -{ - write(int32_t(N), out); - out.write(s.data(), N); -} - -template void write(const std::vector & arr, WriteBuffer & out) -{ - write(int32_t(arr.size()), out); - for (const auto & elem : arr) - write(elem, out); -} - -static void write(const Coordination::ACL & acl, WriteBuffer & out) -{ - write(acl.permissions, out); - write(acl.scheme, out); - write(acl.id, out); -} - -static void write(const Coordination::Stat & stat, WriteBuffer & out) -{ - write(stat.czxid, out); - write(stat.mzxid, out); - write(stat.ctime, out); - write(stat.mtime, out); - write(stat.version, out); - write(stat.cversion, out); - write(stat.aversion, out); - write(stat.ephemeralOwner, out); - write(stat.dataLength, out); - write(stat.numChildren, out); - write(stat.pzxid, out); -} - -static void write(const Coordination::Error & x, WriteBuffer & out) -{ - write(static_cast(x), out); -} - -static void read(int64_t & x, ReadBuffer & in) -{ - readBinary(x, in); - x = __builtin_bswap64(x); -} - -static void read(int32_t & x, ReadBuffer & in) -{ - readBinary(x, in); - x = __builtin_bswap32(x); -} - -static void read(Coordination::Error & x, ReadBuffer & in) -{ - int32_t code; - read(code, in); - x = Coordination::Error(code); -} - -static void read(bool & x, ReadBuffer & in) -{ - readBinary(x, in); -} - -static void read(String & s, ReadBuffer & in) -{ - int32_t size = 0; - read(size, in); - - if (size == -1) - { - /// It means that zookeeper node has NULL value. We will treat it like empty string. - s.clear(); - return; - } - - if (size < 0) - throw Exception("Negative size while reading string from ZooKeeper", ErrorCodes::LOGICAL_ERROR); - - if (size > MAX_STRING_OR_ARRAY_SIZE) - throw Exception("Too large string size while reading from ZooKeeper", ErrorCodes::LOGICAL_ERROR); - - s.resize(size); - in.read(s.data(), size); -} - -template void read(std::array & s, ReadBuffer & in) -{ - int32_t size = 0; - read(size, in); - if (size != N) - throw Exception("Unexpected array size while reading from ZooKeeper", ErrorCodes::LOGICAL_ERROR); - in.read(s.data(), N); -} - -static void read(Coordination::Stat & stat, ReadBuffer & in) -{ - read(stat.czxid, in); - read(stat.mzxid, in); - read(stat.ctime, in); - read(stat.mtime, in); - read(stat.version, in); - read(stat.cversion, in); - read(stat.aversion, in); - read(stat.ephemeralOwner, in); - read(stat.dataLength, in); - read(stat.numChildren, in); - read(stat.pzxid, in); -} - -template void read(std::vector & arr, ReadBuffer & in) -{ - int32_t size = 0; - read(size, in); - if (size < 0) - throw Exception("Negative size while reading array from ZooKeeper", ErrorCodes::LOGICAL_ERROR); - if (size > MAX_STRING_OR_ARRAY_SIZE) - throw Exception("Too large array size while reading from ZooKeeper", ErrorCodes::LOGICAL_ERROR); - arr.resize(size); - for (auto & elem : arr) - read(elem, in); -} - -static void read(Coordination::ACL & acl, ReadBuffer & in) -{ - read(acl.permissions, in); - read(acl.scheme, in); - read(acl.id, in); -} - -#ifdef __clang__ -#pragma clang diagnostic pop -#endif - void TestKeeperTCPHandler::sendHandshake() { - static constexpr int32_t handshake_length = 36; - static constexpr int32_t protocol_version = 0; - static constexpr int32_t DEFAULT_SESSION_TIMEOUT = 30000; - write(handshake_length, *out); - write(protocol_version, *out); - write(DEFAULT_SESSION_TIMEOUT, *out); - write(test_keeper_storage->getSessionID(), *out); - constexpr int32_t passwd_len = 16; - std::array passwd{}; - write(passwd, *out); + Coordination::write(Coordination::SERVER_HANDSHAKE_LENGTH, *out); + Coordination::write(Coordination::ZOOKEEPER_PROTOCOL_VERSION, *out); + Coordination::write(Coordination::DEFAULT_SESSION_TIMEOUT, *out); + Coordination::write(test_keeper_storage->getSessionID(), *out); + std::array passwd{}; + Coordination::write(passwd, *out); out->next(); } @@ -219,30 +44,29 @@ void TestKeeperTCPHandler::receiveHandshake() int64_t last_zxid_seen; int32_t timeout; int64_t previous_session_id = 0; /// We don't support session restore. So previous session_id is always zero. - constexpr int32_t passwd_len = 16; - std::array passwd {}; + std::array passwd {}; - read(handshake_length, *in); - if (handshake_length != 44) + Coordination::read(handshake_length, *in); + if (handshake_length != Coordination::CLIENT_HANDSHAKE_LENGTH) throw Exception("Unexpected handshake length received: " + toString(handshake_length), ErrorCodes::LOGICAL_ERROR); - read(protocol_version, *in); + Coordination::read(protocol_version, *in); - if (protocol_version != 0) + if (protocol_version != Coordination::ZOOKEEPER_PROTOCOL_VERSION) throw Exception("Unexpected protocol version: " + toString(protocol_version), ErrorCodes::LOGICAL_ERROR); - read(last_zxid_seen, *in); + Coordination::read(last_zxid_seen, *in); if (last_zxid_seen != 0) throw Exception("Non zero last_zxid_seen is not supported", ErrorCodes::LOGICAL_ERROR); - read(timeout, *in); - read(previous_session_id, *in); + Coordination::read(timeout, *in); + Coordination::read(previous_session_id, *in); if (previous_session_id != 0) throw Exception("Non zero previous session id is not supported", ErrorCodes::LOGICAL_ERROR); - read(passwd, *in); + Coordination::read(passwd, *in); } @@ -272,7 +96,7 @@ void TestKeeperTCPHandler::runImpl() } catch (const Exception & e) /// Typical for an incorrect username, password, or address. { - LOG_DEBUG(log, "Cannot receive handshake {}", e.displayText()); + LOG_WARNING(log, "Cannot receive handshake {}", e.displayText()); return; } @@ -280,18 +104,17 @@ void TestKeeperTCPHandler::runImpl() while (true) { - //UInt64 max_wait = operation_timeout.totalMicroseconds(); using namespace std::chrono_literals; - //LOG_DEBUG(log, "TRYING TO GET RESPONSE (size {})", responses.size()); - if (!responses.empty() && responses.front().wait_for(100ms) == std::future_status::ready) + while(!responses.empty()) { - auto response = responses.front().get(); + if (responses.front().wait_for(10ms) != std::future_status::ready) + break; - //LOG_DEBUG(log, "Writing response bytes to socket {}", response->getOpNum()); + auto response = responses.front().get(); response->write(*out); responses.pop(); - //LOG_DEBUG(log, "Responses size {}", responses.size()); } + for (auto it = watch_responses.begin(); it != watch_responses.end();) { if (it->wait_for(0s) == std::future_status::ready) @@ -305,8 +128,9 @@ void TestKeeperTCPHandler::runImpl() } } - //LOG_DEBUG(log, "WAITING ON POLL"); - if (in->poll(100 * 1000)) + long poll_wait = responses.empty() ? session_timeout.totalMicroseconds() : 10000; + + if (in->poll(poll_wait)) { bool close_received = receiveRequest(); if (close_received) @@ -315,29 +139,20 @@ void TestKeeperTCPHandler::runImpl() break; } } - else - { - //LOG_DEBUG(log, "NOTHING POLLED"); - } } } bool TestKeeperTCPHandler::receiveRequest() { - //LOG_DEBUG(log, "Receiving event"); int32_t length; - read(length, *in); - //LOG_DEBUG(log, "RECEIVED LENGTH {}", length); + Coordination::read(length, *in); int32_t xid; - //LOG_DEBUG(log, "READING XID"); - read(xid, *in); + Coordination::read(xid, *in); - //LOG_DEBUG(log, "Received xid {}", xid); - - int32_t opnum; - read(opnum, *in); - if (opnum == -11) + Coordination::OpNum opnum; + Coordination::read(opnum, *in); + if (opnum == Coordination::OpNum::Close) return true; Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(opnum); @@ -348,7 +163,6 @@ bool TestKeeperTCPHandler::receiveRequest() if (request_future_responses.watch_response) watch_responses.emplace_back(std::move(*request_future_responses.watch_response)); - //LOG_DEBUG(log, "Responses size {}", responses.size()); return false; } diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h index 7eea5419006..06b3c102df5 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/TestKeeperTCPHandler.h @@ -4,6 +4,7 @@ #include "IServer.h" #include #include +#include #include #include #include @@ -21,7 +22,8 @@ public: , log(&Poco::Logger::get("TestKeeperTCPHandler")) , global_context(server.context()) , test_keeper_storage(global_context.getTestKeeperStorage()) - , operation_timeout(10000) + , operation_timeout(Coordination::DEFAULT_OPERATION_TIMEOUT) + , session_timeout(Coordination::DEFAULT_SESSION_TIMEOUT) { } @@ -32,6 +34,7 @@ private: Context global_context; std::shared_ptr test_keeper_storage; Poco::Timespan operation_timeout; + Poco::Timespan session_timeout; std::queue responses; std::vector watch_responses; From d83c68fca806af5c40060c857ba47951905b0330 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Nov 2020 16:55:28 +0300 Subject: [PATCH 0185/1088] Fix timeouts --- programs/server/Server.cpp | 20 ++++++++++---------- src/Common/ZooKeeper/TestKeeperStorage.cpp | 12 ++++++++++++ src/Common/ZooKeeper/TestKeeperStorage.h | 4 ++-- src/Common/ZooKeeper/ZooKeeper.cpp | 4 ++-- src/Common/ZooKeeper/ZooKeeper.h | 8 +++----- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 6 ++---- src/Common/ZooKeeper/ZooKeeperConstants.h | 4 ++-- src/Common/ZooKeeper/ZooKeeperIO.cpp | 2 +- src/Common/ya.make | 4 ++++ src/Server/TestKeeperTCPHandler.cpp | 4 ++-- src/Server/TestKeeperTCPHandler.h | 4 ++-- src/Server/ya.make | 1 + utils/zookeeper-test/main.cpp | 7 ++++--- 13 files changed, 47 insertions(+), 33 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 597b6c43fe5..064fa391813 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -195,17 +195,17 @@ Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port || code == EAI_ADDRFAMILY #endif ) - { - LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. " - "If it is an IPv6 address and your host has disabled IPv6, then consider to " - "specify IPv4 address to listen in element of configuration " - "file. Example: 0.0.0.0", - host, e.code(), e.message()); - } + { + LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. " + "If it is an IPv6 address and your host has disabled IPv6, then consider to " + "specify IPv4 address to listen in element of configuration " + "file. Example: 0.0.0.0", + host, e.code(), e.message()); + } - throw; - } - return socket_address; + throw; + } + return socket_address; } Poco::Net::SocketAddress Server::socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure) diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Common/ZooKeeper/TestKeeperStorage.cpp index 4e291679b78..19d5fe6ad22 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.cpp +++ b/src/Common/ZooKeeper/TestKeeperStorage.cpp @@ -6,10 +6,22 @@ #include #include +namespace DB +{ + namespace ErrorCodes + { + extern const int LOGICAL_ERROR; + } +} + namespace zkutil { + + using namespace DB; + + static String parentPath(const String & path) { auto rslash_pos = path.rfind('/'); diff --git a/src/Common/ZooKeeper/TestKeeperStorage.h b/src/Common/ZooKeeper/TestKeeperStorage.h index 875cb9f0253..02fdc21cc58 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.h +++ b/src/Common/ZooKeeper/TestKeeperStorage.h @@ -19,7 +19,7 @@ class TestKeeperStorage public: - Poco::Timespan operation_timeout{10000}; + Poco::Timespan operation_timeout{0, Coordination::DEFAULT_OPERATION_TIMEOUT_MS * 1000}; std::atomic session_id_counter{0}; struct Node @@ -85,5 +85,5 @@ public: return zxid.fetch_add(1); } }; - + } diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index bee875d1c74..b3a0a082a9f 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -129,8 +129,8 @@ struct ZooKeeperArgs std::vector hosts_strings; - session_timeout_ms = DEFAULT_SESSION_TIMEOUT; - operation_timeout_ms = DEFAULT_OPERATION_TIMEOUT; + session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; + operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; implementation = "zookeeper"; for (const auto & key : keys) { diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index b1a69646db5..0d9dc104c48 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -11,6 +11,7 @@ #include #include #include +#include #include @@ -28,9 +29,6 @@ namespace CurrentMetrics namespace zkutil { -const UInt32 DEFAULT_SESSION_TIMEOUT = 30000; -const UInt32 DEFAULT_OPERATION_TIMEOUT = 10000; - /// Preferred size of multi() command (in number of ops) constexpr size_t MULTI_BATCH_SIZE = 100; @@ -53,8 +51,8 @@ public: using Ptr = std::shared_ptr; ZooKeeper(const std::string & hosts_, const std::string & identity_ = "", - int32_t session_timeout_ms_ = DEFAULT_SESSION_TIMEOUT, - int32_t operation_timeout_ms_ = DEFAULT_OPERATION_TIMEOUT, + int32_t session_timeout_ms_ = Coordination::DEFAULT_SESSION_TIMEOUT_MS, + int32_t operation_timeout_ms_ = Coordination::DEFAULT_OPERATION_TIMEOUT_MS, const std::string & chroot_ = "", const std::string & implementation_ = "zookeeper"); diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 7b162492d1a..d1c6b1ee80b 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -17,13 +17,11 @@ void ZooKeeperResponse::write(WriteBuffer & out) const { /// Excessive copy to calculate length. WriteBufferFromOwnString buf; - //LOG_DEBUG(&Poco::Logger::get("LOG"), "WRITING {}", xid); Coordination::write(xid, buf); Coordination::write(zxid, buf); Coordination::write(error, buf); if (error == Error::ZOK) writeImpl(buf); - //LOG_DEBUG(&Poco::Logger::get("LOG"), "BUFFER LENGTH {}", buf.str().length()); Coordination::write(buf.str(), out); out.next(); } @@ -426,7 +424,7 @@ ZooKeeperResponsePtr ZooKeeperCloseRequest::makeResponse() const { return std::m void ZooKeeperRequestFactory::registerRequest(OpNum op_num, Creator creator) { if (!op_num_to_request.try_emplace(op_num, creator).second) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Request with op num {} already registered", op_num); + throw Coordination::Exception("Request type " + toString(op_num) + " already registered", Coordination::Error::ZRUNTIMEINCONSISTENCY); } std::shared_ptr ZooKeeperRequest::read(ReadBuffer & in) @@ -478,5 +476,5 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory() registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); } - + } diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h index 47b597f1c9d..292bccd86b4 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.h +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -41,7 +41,7 @@ static constexpr int32_t PASSWORD_LENGTH = 16; /// ZooKeeper has 1 MB node size and serialization limit by default, /// but it can be raised up, so we have a slightly larger limit on our side. static constexpr int32_t MAX_STRING_OR_ARRAY_SIZE = 1 << 28; /// 256 MiB -static constexpr int32_t DEFAULT_SESSION_TIMEOUT = 30000; -static constexpr int32_t DEFAULT_OPERATION_TIMEOUT = 10000; +static constexpr int32_t DEFAULT_SESSION_TIMEOUT_MS = 30000; +static constexpr int32_t DEFAULT_OPERATION_TIMEOUT_MS = 10000; } diff --git a/src/Common/ZooKeeper/ZooKeeperIO.cpp b/src/Common/ZooKeeper/ZooKeeperIO.cpp index 157325cb8e6..07227669366 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.cpp +++ b/src/Common/ZooKeeper/ZooKeeperIO.cpp @@ -16,7 +16,7 @@ void write(int32_t x, WriteBuffer & out) void write(OpNum x, WriteBuffer & out) { - write(static_cast(x), out); + write(static_cast(x), out); } void write(bool x, WriteBuffer & out) diff --git a/src/Common/ya.make b/src/Common/ya.make index 0d6caa22f3a..2d8110b13d0 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -78,7 +78,11 @@ SRCS( WeakHash.cpp ZooKeeper/IKeeper.cpp ZooKeeper/TestKeeper.cpp + ZooKeeper/TestKeeperStorage.cpp ZooKeeper/ZooKeeper.cpp + ZooKeeper/ZooKeeperCommon.cpp + ZooKeeper/ZooKeeperConstants.cpp + ZooKeeper/ZooKeeperIO.cpp ZooKeeper/ZooKeeperImpl.cpp ZooKeeper/ZooKeeperNodeCache.cpp checkStackSize.cpp diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index 7afd1af1aa0..cdac4fdd9b2 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -25,7 +25,7 @@ void TestKeeperTCPHandler::sendHandshake() Coordination::write(Coordination::SERVER_HANDSHAKE_LENGTH, *out); Coordination::write(Coordination::ZOOKEEPER_PROTOCOL_VERSION, *out); - Coordination::write(Coordination::DEFAULT_SESSION_TIMEOUT, *out); + Coordination::write(Coordination::DEFAULT_SESSION_TIMEOUT_MS, *out); Coordination::write(test_keeper_storage->getSessionID(), *out); std::array passwd{}; Coordination::write(passwd, *out); @@ -105,7 +105,7 @@ void TestKeeperTCPHandler::runImpl() while (true) { using namespace std::chrono_literals; - while(!responses.empty()) + while (!responses.empty()) { if (responses.front().wait_for(10ms) != std::future_status::ready) break; diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h index 06b3c102df5..ec44a481522 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/TestKeeperTCPHandler.h @@ -22,8 +22,8 @@ public: , log(&Poco::Logger::get("TestKeeperTCPHandler")) , global_context(server.context()) , test_keeper_storage(global_context.getTestKeeperStorage()) - , operation_timeout(Coordination::DEFAULT_OPERATION_TIMEOUT) - , session_timeout(Coordination::DEFAULT_SESSION_TIMEOUT) + , operation_timeout(0, Coordination::DEFAULT_OPERATION_TIMEOUT_MS * 1000) + , session_timeout(0, Coordination::DEFAULT_SESSION_TIMEOUT_MS * 1000) { } diff --git a/src/Server/ya.make b/src/Server/ya.make index 8a9bbd3bbc2..a83b642dc8b 100644 --- a/src/Server/ya.make +++ b/src/Server/ya.make @@ -21,6 +21,7 @@ SRCS( ReplicasStatusHandler.cpp StaticRequestHandler.cpp TCPHandler.cpp + TestKeeperTCPHandler.cpp WebUIRequestHandler.cpp ) diff --git a/utils/zookeeper-test/main.cpp b/utils/zookeeper-test/main.cpp index a924465d02a..1bf17a035bf 100644 --- a/utils/zookeeper-test/main.cpp +++ b/utils/zookeeper-test/main.cpp @@ -135,7 +135,7 @@ void testMultiRequest(zkutil::ZooKeeper & zk) zk.multi(requests); std::terminate(); } - catch(...) + catch (...) { std::cerr << "Got exception on multy request (it's ok)\n"; } @@ -143,7 +143,8 @@ void testMultiRequest(zkutil::ZooKeeper & zk) checkEq(zk, "/data/multirequest", "bbb"); } -int main(int argc, char *argv[]) { +int main(int argc, char *argv[]) +{ if (argc != 2) { @@ -167,7 +168,7 @@ int main(int argc, char *argv[]) { testCreateSetWatchEvent(zk); testCreateListWatchEvent(zk); } - catch(...) + catch (...) { zk.tryRemoveRecursive("/data"); throw; From df02bb23151af67364dc6a3695cc703b698b7a6a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Nov 2020 17:00:28 +0300 Subject: [PATCH 0186/1088] Change test config --- tests/config/config.d/zookeeper.xml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/config/config.d/zookeeper.xml b/tests/config/config.d/zookeeper.xml index 68c85788c98..06ed7fcd39f 100644 --- a/tests/config/config.d/zookeeper.xml +++ b/tests/config/config.d/zookeeper.xml @@ -1,5 +1,8 @@ - testkeeper + + localhost + 9181 + From d57b57156ecda30fcdb01aa7cf2f0ff07311bbbf Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Nov 2020 17:22:11 +0300 Subject: [PATCH 0187/1088] Fix style check --- src/Common/ZooKeeper/TestKeeperStorage.cpp | 5 ----- src/Common/ZooKeeper/ZooKeeperCommon.h | 1 - 2 files changed, 6 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Common/ZooKeeper/TestKeeperStorage.cpp index 19d5fe6ad22..70702dbecc9 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.cpp +++ b/src/Common/ZooKeeper/TestKeeperStorage.cpp @@ -17,11 +17,8 @@ namespace DB namespace zkutil { - using namespace DB; - - static String parentPath(const String & path) { auto rslash_pos = path.rfind('/'); @@ -679,6 +676,4 @@ TestKeeperStorage::~TestKeeperStorage() } } - - } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 286330bd769..fbf3e13fbfe 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -19,7 +19,6 @@ #include - namespace Coordination { From ade14da63484a8c44c034f8c4bdb8a48f4a471e4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Nov 2020 18:45:10 +0300 Subject: [PATCH 0188/1088] Remove some redundant files --- src/Common/ZooKeeper/TestKeeper.h | 2 -- utils/zookeeper-test/main.cpp | 3 ++- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 01c92c98778..ca9f584304f 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -125,8 +125,6 @@ private: Watches watches; Watches list_watches; /// Watches for 'list' request (watches on children). - void createWatchCallBack(const String & path); - using RequestsQueue = ConcurrentBoundedQueue; RequestsQueue requests_queue{1}; diff --git a/utils/zookeeper-test/main.cpp b/utils/zookeeper-test/main.cpp index 1bf17a035bf..fe7bf93fd15 100644 --- a/utils/zookeeper-test/main.cpp +++ b/utils/zookeeper-test/main.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -14,6 +13,8 @@ using namespace std; +/// TODO: Remove ME + void checkEq(zkutil::ZooKeeper & zk, const std::string & path, const std::string & expected) { auto result = zk.get(path); From 57ac63ca340901e5802ed62a4973f687b7578797 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 11 Nov 2020 19:58:54 +0300 Subject: [PATCH 0189/1088] Minor change in query profiler --- src/Common/QueryProfiler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 07e145359d8..504d884dce0 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -176,7 +176,7 @@ template class QueryProfilerBase; template class QueryProfilerBase; QueryProfilerReal::QueryProfilerReal(const UInt64 thread_id, const UInt32 period) - : QueryProfilerBase(thread_id, CLOCK_REALTIME, period, SIGUSR1) + : QueryProfilerBase(thread_id, CLOCK_MONOTONIC, period, SIGUSR1) {} void QueryProfilerReal::signalHandler(int sig, siginfo_t * info, void * context) From 7d32873cb9a75ed3ad730bba85317d8cbe50e761 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Nov 2020 19:59:13 +0300 Subject: [PATCH 0190/1088] Add missed file --- tests/config/config.d/test_keeper_port.xml | 3 +++ tests/config/install.sh | 1 + 2 files changed, 4 insertions(+) create mode 100644 tests/config/config.d/test_keeper_port.xml diff --git a/tests/config/config.d/test_keeper_port.xml b/tests/config/config.d/test_keeper_port.xml new file mode 100644 index 00000000000..23b744630f2 --- /dev/null +++ b/tests/config/config.d/test_keeper_port.xml @@ -0,0 +1,3 @@ + + 9181 + diff --git a/tests/config/install.sh b/tests/config/install.sh index f6fae181ac8..4eb665b01ba 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -28,6 +28,7 @@ ln -sf $SRC_PATH/config.d/clusters.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/graphite.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/database_atomic.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/test_cluster_with_incorrect_pw.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/test_keeper_port.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/access_management.xml $DEST_SERVER_PATH/users.d/ From dbec289c9a60a3c94869657147eaa499b8b8951f Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 12 Nov 2020 00:58:30 +0300 Subject: [PATCH 0191/1088] [wip] rewrite ip_dict data struct, fix bugs, add tests --- src/Common/IPv6ToBinary.cpp | 25 - src/Common/IPv6ToBinary.h | 4 - src/Dictionaries/TrieDictionary.cpp | 587 +++++++++++++----- src/Dictionaries/TrieDictionary.h | 46 +- .../01018_ddl_dictionaries_special.reference | 8 - .../01018_ddl_dictionaries_special.sql | 31 - .../0_stateless/01018_ip_dictionary.reference | 0 .../0_stateless/01018_ip_dictionary.sql | 345 ++++++++++ 8 files changed, 810 insertions(+), 236 deletions(-) create mode 100644 tests/queries/0_stateless/01018_ip_dictionary.reference create mode 100644 tests/queries/0_stateless/01018_ip_dictionary.sql diff --git a/src/Common/IPv6ToBinary.cpp b/src/Common/IPv6ToBinary.cpp index 00c1b520a7a..e3d14c796ce 100644 --- a/src/Common/IPv6ToBinary.cpp +++ b/src/Common/IPv6ToBinary.cpp @@ -30,29 +30,4 @@ std::array IPv6ToBinary(const Poco::Net::IPAddress & address) return res; } - -UInt32 IPv4ToBinary(const Poco::Net::IPAddress & address, bool & success) -{ - if (!address.isIPv4Mapped()) - { - success = false; - return 0; - } - - success = true; - if (Poco::Net::IPAddress::IPv6 == address.family()) - { - auto raw = reinterpret_cast(address.addr()); - return *reinterpret_cast(&raw[12]); - } - else if (Poco::Net::IPAddress::IPv4 == address.family()) - { - auto raw = reinterpret_cast(address.addr()); - return *reinterpret_cast(raw); - } - - success = false; - return 0; -} - } diff --git a/src/Common/IPv6ToBinary.h b/src/Common/IPv6ToBinary.h index 4f2cdd0ea21..a3e82ee08f5 100644 --- a/src/Common/IPv6ToBinary.h +++ b/src/Common/IPv6ToBinary.h @@ -10,8 +10,4 @@ namespace DB /// Convert IP address to 16-byte array with IPv6 data (big endian). If it's an IPv4, map it to IPv6. std::array IPv6ToBinary(const Poco::Net::IPAddress & address); -/// Convert IP address to UInt32 (big endian) if it's IPv4 or IPv4 mapped to IPv6. -/// Sets success variable to true if succeed. -UInt32 IPv4ToBinary(const Poco::Net::IPAddress & address, bool & success); - } diff --git a/src/Dictionaries/TrieDictionary.cpp b/src/Dictionaries/TrieDictionary.cpp index 28769780027..c475672f39b 100644 --- a/src/Dictionaries/TrieDictionary.cpp +++ b/src/Dictionaries/TrieDictionary.cpp @@ -1,9 +1,9 @@ #include "TrieDictionary.h" #include -#include -#include #include #include +#include +#include #include #include #include @@ -26,38 +26,126 @@ namespace ErrorCodes extern const int DICTIONARY_IS_EMPTY; } +namespace +{ + struct IPRecord + { + Poco::Net::IPAddress addr; + UInt8 prefix; + size_t row; + }; + + struct IPv4Subnet + { + UInt32 addr; + UInt8 prefix; + }; + + struct IPv6Subnet + { + const uint8_t * addr; + UInt8 prefix; + }; +} + static void validateKeyTypes(const DataTypes & key_types) { if (key_types.size() < 1 && 2 < key_types.size()) - throw Exception{"Expected a single IP address or IP with mask", - ErrorCodes::TYPE_MISMATCH}; + throw Exception{"Expected a single IP address or IP with mask", ErrorCodes::TYPE_MISMATCH}; - if (key_types.size() == 1) + const auto & actual_type = key_types[0]->getName(); + if (actual_type != "UInt32" && actual_type != "FixedString(16)") + throw Exception{"Key does not match, expected either UInt32 or FixedString(16)", ErrorCodes::TYPE_MISMATCH}; + + if (key_types.size() > 1) { - const auto & actual_type = key_types[0]->getName(); - if (actual_type != "UInt32" && actual_type != "FixedString(16)") - throw Exception{"Key does not match, expected either UInt32 or FixedString(16)", ErrorCodes::TYPE_MISMATCH}; - return; + const auto * mask_col_type = typeid_cast(key_types[1].get()); + if (mask_col_type == nullptr) + throw Exception{"Mask do not match, expected UInt8", ErrorCodes::TYPE_MISMATCH}; } - - const auto * ip_col_type = typeid_cast(key_types[0].get()); - const auto * mask_col_type = typeid_cast(key_types[1].get()); - bool type_ok = ip_col_type && mask_col_type && ip_col_type->getN() == IPV6_BINARY_LENGTH; - if (!type_ok) - throw Exception{"Keys do not match, {FixedString(16), UInt8}", ErrorCodes::TYPE_MISMATCH}; } -/// Create IPAddress from 16 byte array converting to ipv4 if possible -static Poco::Net::IPAddress ip4or6fromBytes(const uint8_t * data) +static inline bool compPrefixes(UInt8 a, UInt8 b) { - Poco::Net::IPAddress ipaddr(reinterpret_cast(data), IPV6_BINARY_LENGTH); + return a < b; +} - // try to consider as ipv4 - bool is_v4 = false; - if (auto addr_v4 = IPv4ToBinary(ipaddr, is_v4); is_v4) - return Poco::Net::IPAddress(reinterpret_cast(&addr_v4), IPV4_BINARY_LENGTH); +/// Convert mapped IPv6 to IPv4 if possible +inline static UInt32 mappedIPv4ToBinary(const uint8_t * addr, bool & success) +{ + const UInt16* words = reinterpret_cast(addr); + auto has_zero_prefix = words[0] == 0 && words[1] == 0 && words[2] == 0 && words[3] == 0 && words[4] == 0; + success = has_zero_prefix && Poco::ByteOrder::fromNetwork(words[5]) == 0xFFFF; + if (!success) + return 0; + return Poco::ByteOrder::fromNetwork(*reinterpret_cast(&addr[6])); +} - return ipaddr; +/// Convert IPv4 to IPv6-mapped and save results to buf +inline static void mapIPv4ToIPv6(UInt32 addr, uint8_t * buf) +{ + memset(buf, 0, 10); + buf[10] = '\xFF'; + buf[11] = '\xFF'; + addr = Poco::ByteOrder::toNetwork(addr); + memcpy(&buf[12], reinterpret_cast(&addr), 4); +} + +/* +static UInt32 applyMask32(UInt32 val, UInt8 prefix) +{ + UInt32 mask = (prefix >= 32) ? 0xffffffff : ~(0xffffffff >> prefix); + return val & mask; +} + +static void applyMask128(const uint8_t * val, uint8_t * out, UInt8 prefix) +{ + if (prefix >= 128) + prefix = 128; + + size_t i = 0; + + for (; prefix >= 8; ++i, prefix -= 8) + out[i] = val[i]; + + if (i >= 16) + return; + + uint8_t mask = ~(0xff >> prefix); + out[i] = val[i] & mask; + + i++; + memset(&out[i], 0, 16 - i); +} +*/ + +static bool matchIPv4Subnet(UInt32 target, UInt32 addr, UInt8 prefix) +{ + UInt32 mask = (prefix >= 32) ? 0xffffffff : ~(0xffffffff >> prefix); + return (target & mask) == addr; +} + +static bool matchIPv6Subnet(const uint8_t * target, const uint8_t * addr, UInt8 prefix) +{ + if (prefix > IPV6_BINARY_LENGTH * 8) + prefix = IPV6_BINARY_LENGTH * 8; + + size_t i = 0; + for (; prefix >= 8; ++i, prefix -= 8) + { + if (target[i] != addr[i]) + return false; + } + if (prefix == 0) + return true; + + auto mask = ~(0xff >> prefix); + return (addr[i] & mask) == target[i]; +} + +const uint8_t * TrieDictionary::getIPv6FromOffset(const TrieDictionary::IPv6Container & ipv6_col, size_t i) +{ + return reinterpret_cast(&ipv6_col[i * IPV6_BINARY_LENGTH]); } TrieDictionary::TrieDictionary( @@ -71,18 +159,14 @@ TrieDictionary::TrieDictionary( , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) , require_nonempty(require_nonempty_) - , total_ip_length(0) , logger(&Poco::Logger::get("TrieDictionary")) { createAttributes(); + loadData(); calculateBytesAllocated(); } -TrieDictionary::~TrieDictionary() -{ -} - #define DECLARE(TYPE) \ void TrieDictionary::get##TYPE( \ const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType & out) const \ @@ -319,10 +403,15 @@ void TrieDictionary::loadData() /// created upfront to avoid excess allocations const auto keys_size = dict_struct.key->size(); - ip_records.reserve(keys_size); - const auto attributes_size = attributes.size(); + std::vector ip_records; + + row_idx.reserve(keys_size); + mask_column.reserve(keys_size); + + bool has_ipv6 = false; + while (const auto block = stream->read()) { const auto rows = block.rows(); @@ -336,7 +425,7 @@ void TrieDictionary::loadData() return block.safeGetByPosition(keys_size + attribute_idx).column; }); - for (const auto row_idx : ext::range(0, rows)) + for (const auto row : ext::range(0, rows)) { /// calculate key once per row const auto key_column = key_column_ptrs.front(); @@ -346,35 +435,115 @@ void TrieDictionary::loadData() const auto & attribute_column = *attribute_column_ptrs[attribute_idx]; auto & attribute = attributes[attribute_idx]; - setAttributeValue(attribute, attribute_column[row_idx]); + setAttributeValue(attribute, attribute_column[row]); } size_t row_number = ip_records.size(); - std::string addr_str(key_column->getDataAt(row_idx).toString()); + std::string addr_str(key_column->getDataAt(row).toString()); size_t pos = addr_str.find('/'); if (pos != std::string::npos) { IPAddress addr(addr_str.substr(0, pos)); + has_ipv6 = has_ipv6 || (addr.family() == Poco::Net::IPAddress::IPv6); + UInt8 prefix = std::stoi(addr_str.substr(pos + 1), nullptr, 10); + addr = addr & IPAddress(prefix, addr.family()); ip_records.emplace_back(IPRecord{addr, prefix, row_number}); } else { IPAddress addr(addr_str); + has_ipv6 = has_ipv6 || (addr.family() == Poco::Net::IPAddress::IPv6); UInt8 prefix = addr.length() * 8; ip_records.emplace_back(IPRecord{addr, prefix, row_number}); } - total_ip_length += ip_records.back().addr.length(); } } + stream->readSuffix(); + LOG_TRACE(logger, "{} ip records are read", ip_records.size()); - std::sort(ip_records.begin(), ip_records.end(), lessIPRecords); + if (has_ipv6) + { + std::sort(ip_records.begin(), ip_records.end(), + [](const auto & record_a, const auto & record_b) + { + auto a = IPv6ToBinary(record_a.addr); + auto b = IPv6ToBinary(record_b.addr); + auto cmpres = memcmp16(reinterpret_cast(a.data()), + reinterpret_cast(b.data())); - stream->readSuffix(); + if (cmpres == 0) + return compPrefixes(record_a.prefix, record_b.prefix); + return cmpres < 0; + }); + + auto & ipv6_col = ip_column.emplace(); + ipv6_col.resize_fill(IPV6_BINARY_LENGTH * ip_records.size()); + + for (const auto & record : ip_records) + { + auto ip_array = IPv6ToBinary(record.addr); + + size_t i = row_idx.size(); + memcpySmallAllowReadWriteOverflow15(&ipv6_col[i * IPV6_BINARY_LENGTH], + reinterpret_cast(ip_array.data()), + IPV6_BINARY_LENGTH); + mask_column.push_back(record.prefix); + row_idx.push_back(record.row); + } + } + else + { + std::sort(ip_records.begin(), ip_records.end(), + [](const auto & record_a, const auto & record_b) + { + UInt32 a = *reinterpret_cast(record_a.addr.addr()); + a = Poco::ByteOrder::fromNetwork(a); + + UInt32 b = *reinterpret_cast(record_b.addr.addr()); + b = Poco::ByteOrder::fromNetwork(b); + + if (a == b) + return compPrefixes(record_a.prefix, record_b.prefix); + return a < b; + }); + + auto & ipv4_col = ip_column.emplace(); + ipv4_col.reserve(ip_records.size()); + for (const auto & record : ip_records) + { + auto addr = Poco::ByteOrder::fromNetwork(*reinterpret_cast(record.addr.addr())); + ipv4_col.push_back(addr); + mask_column.push_back(record.prefix); + row_idx.push_back(record.row); + } + } + + parent_subnet.resize(ip_records.size()); + std::stack subnets_stack; + for (const auto i : ext::range(0, ip_records.size())) + { + parent_subnet[i] = i; + + const auto & cur_address = ip_records[i].addr; + while (!subnets_stack.empty()) + { + size_t subnet_idx = subnets_stack.top(); + const auto cur_subnet = ip_records[subnet_idx]; + auto cur_addr_masked = cur_address & IPAddress(cur_subnet.prefix, cur_address.family()); + if (cur_subnet.addr == cur_addr_masked) + { + parent_subnet[i] = subnet_idx; + break; + } + subnets_stack.pop(); + } + subnets_stack.push(i); + } if (require_nonempty && 0 == element_count) throw Exception{full_name + ": dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY}; @@ -390,8 +559,16 @@ void TrieDictionary::addAttributeSize(const Attribute & attribute) void TrieDictionary::calculateBytesAllocated() { - bytes_allocated += ip_records.size() * sizeof(ip_records.front()); - bytes_allocated += total_ip_length; + if (auto * ipv4_col = std::get_if(&ip_column)) + { + bytes_allocated += ipv4_col->size() * sizeof((*ipv4_col)[0]); + } + else if (auto * ipv6_col = std::get_if(&ip_column)) + { + bytes_allocated += ipv6_col->size() * sizeof((*ipv6_col)[0]); + } + bytes_allocated += mask_column.size() * sizeof(mask_column[0]); + bytes_allocated += row_idx.size() * sizeof(row_idx[0]); bytes_allocated += attributes.size() * sizeof(attributes.front()); for (const auto & attribute : attributes) @@ -523,57 +700,111 @@ TrieDictionary::Attribute TrieDictionary::createAttributeWithType(const Attribut return attr; } +template +void TrieDictionary::getItemsByTwoKeyColumnsImpl( + const Attribute & attribute, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const +{ + const auto first_column = key_columns.front(); + const auto rows = first_column->size(); + auto & vec = std::get>(attribute.maps); + + if (auto ipv4_col = std::get_if(&ip_column)) + { + const auto * key_ip_column_ptr = typeid_cast *>(&*key_columns.front()); + if (key_ip_column_ptr == nullptr) + throw Exception{"Expected a UInt32 IP column", ErrorCodes::TYPE_MISMATCH}; + + const auto & key_mask_column = assert_cast &>(*key_columns.back()); + + auto comp_v4 = [&](size_t elem, IPv4Subnet target) + { + UInt32 addr = (*ipv4_col)[elem]; + if (addr == target.addr) + return compPrefixes(mask_column[elem], target.prefix); + return addr < target.addr; + }; + + for (const auto i : ext::range(0, rows)) + { + UInt32 addr = key_ip_column_ptr->getElement(i); + UInt8 mask = key_mask_column.getElement(i); + + auto range = ext::range(0, row_idx.size()); + auto found_it = std::lower_bound(range.begin(), range.end(), IPv4Subnet{addr, mask}, comp_v4); + + if (likely(found_it != range.end() && + (*ipv4_col)[*found_it] == addr && + mask_column[*found_it] == mask)) + { + set_value(i, static_cast(vec[row_idx[*found_it]])); + } + else + set_value(i, get_default(i)); + } + return; + } + + const auto * key_ip_column_ptr = typeid_cast(&*key_columns.front()); + if (key_ip_column_ptr == nullptr) + throw Exception{"Expected a UInt32 IP column", ErrorCodes::TYPE_MISMATCH}; + + const auto & key_mask_column = assert_cast &>(*key_columns.back()); + + auto ipv6_col = std::get_if(&ip_column); + auto comp_v6 = [&](size_t i, IPv6Subnet target) + { + auto cmpres = memcmp16(getIPv6FromOffset(*ipv6_col, i), target.addr); + if (cmpres == 0) + return compPrefixes(mask_column[i], target.prefix); + return cmpres < 0; + }; + + for (const auto i : ext::range(0, rows)) + { + auto addr = key_ip_column_ptr->getDataAt(i); + UInt8 mask = key_mask_column.getElement(i); + + IPv6Subnet target{reinterpret_cast(addr.data), mask}; + + auto range = ext::range(0, row_idx.size()); + auto found_it = std::lower_bound(range.begin(), range.end(), target, comp_v6); + + if (likely(found_it != range.end() && + memcmp16(getIPv6FromOffset(*ipv6_col, *found_it), target.addr) == 0 && + mask_column[*found_it] == mask)) + set_value(i, static_cast(vec[row_idx[*found_it]])); + else + set_value(i, get_default(i)); + } +} template void TrieDictionary::getItemsImpl( const Attribute & attribute, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const { - auto & vec = std::get>(attribute.maps); - const auto first_column = key_columns.front(); const auto rows = first_column->size(); // special case for getBlockInputStream if (unlikely(key_columns.size() == 2)) { - const auto & ip_column = assert_cast(*key_columns.front()); - const auto & mask_column = assert_cast &>(*key_columns.back()); - - for (const auto i : ext::range(0, rows)) - { - const auto second_column = key_columns.back(); - - auto addr_data = ip_column.getDataAt(i).data; - auto ipaddr = ip4or6fromBytes(reinterpret_cast(addr_data)); - - UInt8 mask = mask_column.getElement(i); - - auto target = IPRecord{ipaddr, mask, 0}; - auto found_it = std::lower_bound(ip_records.begin(), ip_records.end(), target, lessIPRecords); - - if (likely(found_it != ip_records.end() && - found_it->addr == target.addr && - found_it->prefix == target.prefix)) - { - set_value(i, static_cast(vec[found_it->row])); - } - else - { - set_value(i, get_default(i)); - } - } + getItemsByTwoKeyColumnsImpl( + attribute, key_columns, std::forward(set_value), std::forward(get_default)); query_count.fetch_add(rows, std::memory_order_relaxed); return; } + auto & vec = std::get>(attribute.maps); + if (first_column->isNumeric()) { + uint8_t addrv6_buf[IPV6_BINARY_LENGTH]; for (const auto i : ext::range(0, rows)) { - auto addr = Poco::ByteOrder::toNetwork(UInt32(first_column->get64(i))); - auto ipaddr = IPAddress(reinterpret_cast(&addr), IPV4_BINARY_LENGTH); - auto found = lookupIPRecord(ipaddr); - set_value(i, (found != ipRecordNotFound()) ? static_cast(vec[found->row]) : get_default(i)); + // addrv4 has native endianness + auto addrv4 = UInt32(first_column->get64(i)); + auto found = tryLookupIPv4(addrv4, addrv6_buf); + set_value(i, (found != ipNotFound()) ? static_cast(vec[*found]) : get_default(i)); } } else @@ -584,9 +815,8 @@ void TrieDictionary::getItemsImpl( if (addr.size != IPV6_BINARY_LENGTH) throw Exception("Expected key to be FixedString(16)", ErrorCodes::LOGICAL_ERROR); - auto ipaddr = ip4or6fromBytes(reinterpret_cast(addr.data)); - auto found = lookupIPRecord(ipaddr); - set_value(i, (found != ipRecordNotFound()) ? static_cast(vec[found->row]) : get_default(i)); + auto found = tryLookupIPv6(reinterpret_cast(addr.data)); + set_value(i, (found != ipNotFound()) ? static_cast(vec[*found]) : get_default(i)); } } @@ -659,12 +889,12 @@ void TrieDictionary::has(const Attribute &, const Columns & key_columns, PaddedP const auto rows = first_column->size(); if (first_column->isNumeric()) { + uint8_t addrv6_buf[IPV6_BINARY_LENGTH]; for (const auto i : ext::range(0, rows)) { - auto addr = Int32(first_column->get64(i)); - auto ipaddr = IPAddress(reinterpret_cast(&addr), IPV4_BINARY_LENGTH); - auto found = lookupIPRecord(ipaddr); - out[i] = (found != ipRecordNotFound()); + auto addrv4 = UInt32(first_column->get64(i)); + auto found = tryLookupIPv4(addrv4, addrv6_buf); + out[i] = (found != ipNotFound()); } } else @@ -672,12 +902,11 @@ void TrieDictionary::has(const Attribute &, const Columns & key_columns, PaddedP for (const auto i : ext::range(0, rows)) { auto addr = first_column->getDataAt(i); - if (unlikely(addr.size != 16)) + if (unlikely(addr.size != IPV6_BINARY_LENGTH)) throw Exception("Expected key to be FixedString(16)", ErrorCodes::LOGICAL_ERROR); - auto ipaddr = ip4or6fromBytes(reinterpret_cast(addr.data)); - auto found = lookupIPRecord(ipaddr); - out[i] = (found != ipRecordNotFound()); + auto found = tryLookupIPv6(reinterpret_cast(addr.data)); + out[i] = (found != ipNotFound()); } } @@ -686,42 +915,50 @@ void TrieDictionary::has(const Attribute &, const Columns & key_columns, PaddedP Columns TrieDictionary::getKeyColumns() const { - auto ip_column = ColumnFixedString::create(IPV6_BINARY_LENGTH); - auto mask_column = ColumnVector::create(); - - for (const auto & record : ip_records) + auto ipv4_col = std::get_if(&ip_column); + if (ipv4_col) { - auto ip_array = IPv6ToBinary(record.addr); - ip_column->insertData(ip_array.data(), IPV6_BINARY_LENGTH); - mask_column->insertValue(record.prefix); + auto key_ip_column = ColumnVector::create(); + auto key_mask_column = ColumnVector::create(); + for (size_t row : ext::range(0, row_idx.size())) + { + key_ip_column->insertValue((*ipv4_col)[row]); + key_mask_column->insertValue(mask_column[row]); + } + return {std::move(key_ip_column), std::move(key_mask_column)}; } - return {std::move(ip_column), std::move(mask_column)}; + auto ipv6_col = std::get_if(&ip_column); + + auto key_ip_column = ColumnFixedString::create(IPV6_BINARY_LENGTH); + auto key_mask_column = ColumnVector::create(); + + for (size_t row : ext::range(0, row_idx.size())) + { + auto data = reinterpret_cast(getIPv6FromOffset(*ipv6_col, row)); + key_ip_column->insertData(data, IPV6_BINARY_LENGTH); + key_mask_column->insertValue(mask_column[row]); + } + return {std::move(key_ip_column), std::move(key_mask_column)}; } -BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +template +static auto keyViewGetter() { - using BlockInputStreamType = DictionaryBlockInputStream; - - auto get_keys = [](const Columns & columns, const std::vector & dict_attributes) - { - const auto & attr = dict_attributes.front(); - return ColumnsWithTypeAndName({ - ColumnWithTypeAndName(columns.front(), std::make_shared(IPV6_BINARY_LENGTH), attr.name), - ColumnWithTypeAndName(columns.back(), std::make_shared(), attr.name + ".mask") - }); - }; - auto get_view = [](const Columns & columns, const std::vector & dict_attributes) + return [](const Columns & columns, const std::vector & dict_attributes) { auto column = ColumnString::create(); - const auto & ip_column = assert_cast(*columns.front()); - const auto & mask_column = assert_cast &>(*columns.back()); + const auto & key_ip_column = assert_cast(*columns.front()); + const auto & key_mask_column = assert_cast &>(*columns.back()); char buffer[48]; - for (size_t row : ext::range(0, ip_column.size())) + for (size_t row : ext::range(0, key_ip_column.size())) { - UInt8 mask = mask_column.getElement(row); + UInt8 mask = key_mask_column.getElement(row); char * ptr = buffer; - formatIPv6(reinterpret_cast(ip_column.getDataAt(row).data), ptr); + if constexpr (IsIPv4) + formatIPv4(reinterpret_cast(&key_ip_column.getElement(row)), ptr); + else + formatIPv6(reinterpret_cast(key_ip_column.getDataAt(row).data), ptr); *(ptr - 1) = '/'; ptr = itoa(mask, ptr); column->insertData(buffer, ptr - buffer); @@ -729,59 +966,119 @@ BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_nam return ColumnsWithTypeAndName{ ColumnWithTypeAndName(std::move(column), std::make_shared(), dict_attributes.front().name)}; }; +} + +BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +{ + using BlockInputStreamType = DictionaryBlockInputStream; + + + const bool isIPv4 = std::get_if(&ip_column) != nullptr; + + auto get_keys = [isIPv4](const Columns & columns, const std::vector & dict_attributes) + { + const auto & attr = dict_attributes.front(); + std::shared_ptr key_typ; + if (isIPv4) + key_typ = std::make_shared(); + else + key_typ = std::make_shared(IPV6_BINARY_LENGTH); + + return ColumnsWithTypeAndName({ + ColumnWithTypeAndName(columns.front(), key_typ, attr.name), + ColumnWithTypeAndName(columns.back(), std::make_shared(), attr.name + ".mask") + }); + }; + + if (isIPv4) + { + auto get_view = keyViewGetter, true>(); + return std::make_shared( + shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)); + } + + auto get_view = keyViewGetter(); return std::make_shared( shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)); } -int TrieDictionary::matchIPAddrWithRecord(const IPAddress & ipaddr, const IPRecord & record) const +TrieDictionary::RowIdxConstIter TrieDictionary::ipNotFound() const { - if (ipaddr.family() != record.addr.family()) - return ipaddr.family() < record.addr.family() ? -1 : 1; - - auto masked_ipaddr = ipaddr & IPAddress(record.prefix, record.addr.family()); - if (masked_ipaddr < record.addr) - return -1; - if (masked_ipaddr == record.addr) - return 0; - return 1; + return row_idx.end(); } -bool TrieDictionary::lessIPRecords(const IPRecord & a, const IPRecord & b) const +TrieDictionary::RowIdxConstIter TrieDictionary::tryLookupIPv4(UInt32 addr, uint8_t * buf) const { - if (a.addr.family() != b.addr.family()) - return a.addr.family() < b.addr.family(); - - // prefer IPs with more narrow subnet - if (a.addr == b.addr) - return a.prefix < b.prefix; - return a.addr < b.addr; -} - -TrieDictionary::IPRecordConstIt TrieDictionary::ipRecordNotFound() const -{ - return ip_records.end(); -} - -TrieDictionary::IPRecordConstIt TrieDictionary::lookupIPRecord(const IPAddress & target) const -{ - if (ip_records.empty()) - return ipRecordNotFound(); - - auto comp = [&](const IPAddress & needle, const IPRecord & record) -> bool + if (std::get_if(&ip_column)) { - return matchIPAddrWithRecord(needle, record) < 0; + mapIPv4ToIPv6(addr, buf); + return lookupIP(buf); + } + return lookupIP(addr); +} + +TrieDictionary::RowIdxConstIter TrieDictionary::tryLookupIPv6(const uint8_t * addr) const +{ + if (std::get_if(&ip_column)) + { + bool is_mapped = false; + UInt32 addrv4 = mappedIPv4ToBinary(addr, is_mapped); + if (!is_mapped) + return ipNotFound(); + return lookupIP(addrv4); + } + return lookupIP(addr); +} + +template +TrieDictionary::RowIdxConstIter TrieDictionary::lookupIP(IPValueType target) const +{ + if (row_idx.empty()) + return ipNotFound(); + + auto ipv4or6_col = std::get_if(&ip_column); + if (ipv4or6_col == nullptr) + return ipNotFound(); + + auto comp = [&](auto value, auto idx) -> bool + { + if constexpr (std::is_same_v) + return value < (*ipv4or6_col)[idx]; + else + return memcmp16(value, getIPv6FromOffset(*ipv4or6_col, idx)) < 0; }; - auto next_it = std::upper_bound(ip_records.begin(), ip_records.end(), target, comp); + auto range = ext::range(0, row_idx.size()); + auto found_it = std::upper_bound(range.begin(), range.end(), target, comp); - if (next_it == ip_records.begin()) - return ipRecordNotFound(); + if (found_it == range.begin()) + return ipNotFound(); - auto found = next_it - 1; - if (matchIPAddrWithRecord(target, *found) == 0) - return found; + --found_it; + if constexpr (std::is_same_v) + { + for (auto idx = *found_it;; idx = parent_subnet[idx]) + { + if (matchIPv4Subnet(target, (*ipv4or6_col)[idx], mask_column[idx])) + return row_idx.begin() + idx; - return ipRecordNotFound(); + if (idx == parent_subnet[idx]) + return ipNotFound(); + } + } + else + { + for (auto idx = *found_it;; idx = parent_subnet[idx]) + { + if (matchIPv6Subnet(target, getIPv6FromOffset(*ipv4or6_col, idx), mask_column[idx])) + return row_idx.begin() + idx; + + if (idx == parent_subnet[idx]) + return ipNotFound(); + } + } + + return ipNotFound(); } void registerDictionaryTrie(DictionaryFactory & factory) diff --git a/src/Dictionaries/TrieDictionary.h b/src/Dictionaries/TrieDictionary.h index 9fe304a786a..c6b28fdfe8d 100644 --- a/src/Dictionaries/TrieDictionary.h +++ b/src/Dictionaries/TrieDictionary.h @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include #include #include @@ -27,8 +29,6 @@ public: const DictionaryLifetime dict_lifetime_, bool require_nonempty_); - ~TrieDictionary() override; - std::string getKeyDescription() const { return key_description; } std::string getTypeName() const override { return "Trie"; } @@ -154,15 +154,14 @@ private: using IPAddress = Poco::Net::IPAddress; - struct IPRecord; - using IPRecordConstIt = ContainerType::const_iterator; + using IPv4Container = PODArray; + using IPv6Container = PaddedPODArray; + using IPMaskContainer = PODArray; + using RowIdxConstIter = ContainerType::const_iterator; - struct IPRecord final - { - IPAddress addr; - UInt8 prefix; - size_t row; - }; + template struct IPContainerToValueType {}; + template <> struct IPContainerToValueType { using type = UInt32; }; + template <> struct IPContainerToValueType { using type = const uint8_t *; }; struct Attribute final { @@ -218,6 +217,9 @@ private: Attribute createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value); + template + void getItemsByTwoKeyColumnsImpl( + const Attribute & attribute, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const; template void @@ -234,19 +236,15 @@ private: void has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray & out) const; Columns getKeyColumns() const; + RowIdxConstIter ipNotFound() const; + RowIdxConstIter tryLookupIPv4(UInt32 addr, uint8_t * buf) const; + RowIdxConstIter tryLookupIPv6(const uint8_t * addr) const; - /** - * Compare ip addresses. - * - * @return negative value if ipaddr less than address in record - * @return zero if ipaddr in record subnet - * @return positive value if ipaddr greater than address in record - */ - int matchIPAddrWithRecord(const IPAddress & ipaddr, const IPRecord & record) const; - bool lessIPRecords(const IPRecord & a, const IPRecord & b) const; + template ::value> + RowIdxConstIter lookupIP(IPValueType target) const; - IPRecordConstIt ipRecordNotFound() const; - IPRecordConstIt lookupIPRecord(const IPAddress & target) const; + static const uint8_t * getIPv6FromOffset(const IPv6Container & ipv6_col, size_t i); const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; @@ -254,8 +252,10 @@ private: const bool require_nonempty; const std::string key_description{dict_struct.getKeyDescription()}; - ContainerType ip_records; - size_t total_ip_length; + std::variant ip_column; + IPMaskContainer mask_column; + ContainerType parent_subnet; + ContainerType row_idx; std::map attribute_index_by_name; std::vector attributes; diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_special.reference b/tests/queries/0_stateless/01018_ddl_dictionaries_special.reference index a6332b85f4e..b62018ec525 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_special.reference +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_special.reference @@ -8,14 +8,6 @@ 0.42 0.46 0 -***ip trie dict*** -17501 -17501 -17502 -0 -11211 -11211 -NP ***hierarchy dict*** Moscow [3,2,1,10000] diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql index 6c4a325a3b5..ace3d107cb0 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql @@ -72,37 +72,6 @@ SELECT dictGetFloat64('database_for_dict.dict2', 'Tax', toUInt64(1), toDateTime( SELECT dictGetFloat64('database_for_dict.dict2', 'Tax', toUInt64(2), toDateTime('2019-05-29 00:00:00')); SELECT dictGetFloat64('database_for_dict.dict2', 'Tax', toUInt64(2), toDateTime('2019-05-31 00:00:00')); -SELECT '***ip trie dict***'; - -CREATE TABLE database_for_dict.table_ip_trie -( - prefix String, - asn UInt32, - cca2 String -) -engine = TinyLog; - -INSERT INTO database_for_dict.table_ip_trie VALUES ('202.79.32.0/20', 17501, 'NP'), ('202.79.32.2', 17502, 'NP'), ('101.79.55.22', 11211, 'UK'), ('2620:0:870::/48', 3856, 'US'), ('2a02:6b8:1::/48', 13238, 'RU'), ('2001:db8::/32', 65536, 'ZZ'); - -CREATE DICTIONARY database_for_dict.dict_ip_trie -( - prefix String, - asn UInt32, - cca2 String -) -PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ip_trie')) -LAYOUT(IP_TRIE()) -LIFETIME(MIN 10 MAX 100); - -SELECT dictGetUInt32('database_for_dict.dict_ip_trie', 'asn', tuple(IPv4StringToNum('202.79.32.0'))); -SELECT dictGetUInt32('database_for_dict.dict_ip_trie', 'asn', tuple(IPv4StringToNum('202.79.32.1'))); -SELECT dictGetUInt32('database_for_dict.dict_ip_trie', 'asn', tuple(IPv4StringToNum('202.79.32.2'))); -SELECT dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('654f:3716::'))); -SELECT dictGetUInt32('database_for_dict.dict_ip_trie', 'asn', tuple(IPv6StringToNum('::ffff:654f:3716'))); -SELECT dictGetUInt32('database_for_dict.dict_ip_trie', 'asn', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); -SELECT dictGetString('database_for_dict.dict_ip_trie', 'cca2', tuple(IPv4StringToNum('202.79.32.0'))); - SELECT '***hierarchy dict***'; CREATE TABLE database_for_dict.table_with_hierarchy diff --git a/tests/queries/0_stateless/01018_ip_dictionary.reference b/tests/queries/0_stateless/01018_ip_dictionary.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01018_ip_dictionary.sql b/tests/queries/0_stateless/01018_ip_dictionary.sql new file mode 100644 index 00000000000..4a93927e14b --- /dev/null +++ b/tests/queries/0_stateless/01018_ip_dictionary.sql @@ -0,0 +1,345 @@ +SET send_logs_level = 'fatal'; + +DROP DATABASE IF EXISTS database_for_dict; + +CREATE DATABASE database_for_dict; + +SELECT '***ipv4 trie dict***'; +CREATE TABLE database_for_dict.table_ipv4_trie +( + prefix String, + asn UInt32, + cca2 String +) +engine = TinyLog; + +-- numbers reordered to test sorting criteria too +INSERT INTO database_for_dict.table_ipv4_trie +SELECT + '255.255.255.255/' || toString((number + 1) * 13 % 33) AS prefix, + toUInt32((number + 1) * 13 % 33) AS asn, + 'NA' as cca2 +FROM system.numbers LIMIT 33; + +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.2', 1272, 'RU'); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.0/8', 1270, 'RU'); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('101.79.55.22', 11212, 'UK'); + +CREATE DICTIONARY database_for_dict.dict_ipv4_trie +( + prefix String, + asn UInt32, + cca2 String +) +PRIMARY KEY prefix +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ipv4_trie')) +LAYOUT(IP_TRIE()) +LIFETIME(MIN 10 MAX 100); + +SELECT 0 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('0.0.0.0'))); +SELECT 1 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('128.0.0.0'))); +SELECT 2 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('192.0.0.0'))); +SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('224.0.0.0'))); +SELECT 4 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('240.0.0.0'))); +SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('248.0.0.0'))); +SELECT 6 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('252.0.0.0'))); +SELECT 7 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('254.0.0.0'))); +SELECT 8 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.0.0.0'))); +SELECT 9 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.128.0.0'))); +SELECT 10 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.192.0.0'))); +SELECT 11 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.224.0.0'))); +SELECT 12 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.240.0.0'))); +SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.248.0.0'))); +SELECT 14 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.252.0.0'))); +SELECT 15 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.254.0.0'))); +SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.0.0'))); +SELECT 17 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.128.0'))); +SELECT 18 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.192.0'))); +SELECT 19 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.224.0'))); +SELECT 20 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.240.0'))); +SELECT 21 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.248.0'))); +SELECT 22 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.252.0'))); +SELECT 23 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.254.0'))); +SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.0'))); +SELECT 25 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.128'))); +SELECT 26 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.192'))); +SELECT 27 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.224'))); +SELECT 28 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.240'))); +SELECT 29 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.248'))); +SELECT 30 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.252'))); +SELECT 31 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.254'))); +SELECT 32 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.255'))); + +SELECT 'RU' == dictGetString('database_for_dict.dict_ipv4_trie', 'cca2', tuple(IPv4StringToNum('127.0.0.1'))); + +SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 1272 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.2'))); +SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.3'))); +SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.255'))); + +-- SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0'))); +-- SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1'))); +-- SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.2'))); +-- SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.3'))); +-- SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.255'))); +-- SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.127.127.127'))); +-- SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.127.127.0'))); + +-- SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('101.79.55.22'))); +-- SELECT 'x', dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:654f:3716'))); +-- SELECT '11212', dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); + +SELECT 11211 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('202.79.32.2'))); + +CREATE TABLE database_for_dict.table_from_ipv4_trie_dict +( + prefix String, + asn UInt32, + cca2 String +) ENGINE = Dictionary(database_for_dict.dict_ipv4_trie); + +SELECT 1272 == asn AND 'RU' == cca2 FROM database_for_dict.table_from_ipv4_trie_dict +WHERE prefix == '127.0.0.2/32'; + +SELECT 37 == COUNT(*) FROM database_for_dict.table_from_ipv4_trie_dict; + +DROP DICTIONARY IF EXISTS database_for_dict.dict_ipv4_trie; +DROP TABLE IF EXISTS database_for_dict.table_from_ipv4_trie_dict; +DROP TABLE IF EXISTS database_for_dict.table_ipv4_trie; + +SELECT '***ipv4 trie dict pt2***'; + +CREATE TABLE database_for_dict.table_ipv4_trie ( prefix String, val UInt32 ) engine = TinyLog; + +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.0/8', 1); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.0/16', 2); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.0/24', 3); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.1/32', 4); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.127.0/32', 5); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.128.1/32', 6); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.255.0/32', 7); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.255.1/32', 8); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.255.255/32', 9); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.1.0.0/16', 10); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.1.1.0', 11); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.1.255.0/24', 12); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.254.0.0/15', 13); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.254.0.127', 14); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.0.0/16', 15); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.128.0/24', 16); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.128.1/32', 17); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.128.10/32', 18); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.128.128/25', 19); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.255.128/32', 20); +INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.255.255/32', 21); + +CREATE DICTIONARY database_for_dict.dict_ipv4_trie ( prefix String, val UInt32 ) +PRIMARY KEY prefix +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ipv4_trie')) +LAYOUT(IP_TRIE()) +LIFETIME(MIN 10 MAX 100); + +SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT 4 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.127'))); +SELECT 2 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.255.127'))); +SELECT 15 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.127.127'))); +SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.9'))); +SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.127'))); +SELECT 18 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.10'))); +SELECT 19 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.255'))); +SELECT 20 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.255.128'))); + +SELECT '***ipv6 trie dict***'; + +CREATE TABLE database_for_dict.table_ip_trie +( + prefix String, + val String +) +engine = TinyLog; + +INSERT INTO database_for_dict.table_ip_trie VALUES ('101.79.55.22', 'JA'), ('127.0.0.1', 'RU'), ('2620:0:870::/48', 'US'), ('2a02:6b8:1::/48', 'UK'), ('2001:db8::/32', 'ZZ'); + +INSERT INTO database_for_dict.table_ip_trie +SELECT + 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/' || toString((number + 1) * 13 % 129) AS prefix, + toString((number + 1) * 13 % 129) AS val +FROM system.numbers LIMIT 129; + +CREATE DICTIONARY database_for_dict.dict_ip_trie +( + prefix String, + val String +) +PRIMARY KEY prefix +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ip_trie')) +LAYOUT(IP_TRIE()) +LIFETIME(MIN 10 MAX 100); + +SELECT 'US' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('2620:0:870::'))); +SELECT 'UK' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('2a02:6b8:1::'))); +SELECT 'ZZ' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('2001:db8::'))); +SELECT 'ZZ' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('2001:db8:ffff::'))); + +SELECT 1 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff::'))); +SELECT 1 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff:ffff::'))); +SELECT 1 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff:1::'))); + +SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('654f:3716::'))); +SELECT 0 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('654f:3716::'))); +SELECT 0 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('654f:3716:ffff::'))); + +SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:654f:3716'))); +SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); +SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('101.79.55.22'))); + +SELECT '0' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::0'))); +SELECT '1' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('8000::'))); +SELECT '2' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('c000::'))); +SELECT '3' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('e000::'))); +SELECT '4' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('f000::'))); +SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('f800::'))); +SELECT '6' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fc00::'))); +SELECT '7' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fe00::'))); +SELECT '8' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff00::'))); +SELECT '9' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff80::'))); +SELECT '10' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffc0::'))); +SELECT '11' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffe0::'))); +SELECT '12' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff0::'))); +SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff8::'))); +SELECT '14' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffc::'))); +SELECT '15' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffe::'))); +SELECT '16' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff::'))); +SELECT '17' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:8000::'))); +SELECT '18' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); +SELECT '19' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); +SELECT '20' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); +SELECT '21' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); +SELECT '22' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); +SELECT '18' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); +SELECT '19' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); +SELECT '20' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); +SELECT '21' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); +SELECT '22' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); +SELECT '23' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fe00::'))); +SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff00::'))); +SELECT '25' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff80::'))); +SELECT '26' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffc0::'))); +SELECT '27' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffe0::'))); +SELECT '28' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff0::'))); +SELECT '29' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff8::'))); +SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffc::'))); +SELECT '31' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffe::'))); +SELECT '32' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff::'))); +SELECT '33' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:8000::'))); +SELECT '34' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:c000::'))); +SELECT '35' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:e000::'))); +SELECT '36' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f000::'))); +SELECT '37' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f800::'))); +SELECT '38' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fc00::'))); +SELECT '39' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fe00::'))); +SELECT '40' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff00::'))); +SELECT '41' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff80::'))); +SELECT '42' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffc0::'))); +SELECT '43' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffe0::'))); +SELECT '44' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff0::'))); +SELECT '45' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff8::'))); +SELECT '46' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffc::'))); +SELECT '47' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffe::'))); +SELECT '48' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff::'))); +SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:8000::'))); +SELECT '50' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:c000::'))); +SELECT '51' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:e000::'))); +SELECT '52' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f000::'))); +SELECT '53' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f800::'))); +SELECT '54' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fc00::'))); +SELECT '55' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fe00::'))); +SELECT '56' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff00::'))); +SELECT '57' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff80::'))); +SELECT '58' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffc0::'))); +SELECT '59' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffe0::'))); +SELECT '60' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff0::'))); +SELECT '61' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff8::'))); +SELECT '62' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffc::'))); +SELECT '63' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffe::'))); +SELECT '64' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff::'))); +SELECT '65' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:8000::'))); +SELECT '66' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:c000::'))); +SELECT '67' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:e000::'))); +SELECT '68' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f000::'))); +SELECT '69' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f800::'))); +SELECT '70' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fc00::'))); +SELECT '71' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fe00::'))); +SELECT '72' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff00::'))); +SELECT '73' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff80::'))); +SELECT '74' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffc0::'))); +SELECT '75' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffe0::'))); +SELECT '76' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff0::'))); +SELECT '77' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff8::'))); +SELECT '78' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffc::'))); +SELECT '79' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffe::'))); +SELECT '80' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff::'))); +SELECT '81' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:8000::'))); +SELECT '82' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:c000::'))); +SELECT '83' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:e000::'))); +SELECT '84' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f000::'))); +SELECT '85' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f800::'))); +SELECT '86' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fc00::'))); +SELECT '87' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fe00::'))); +SELECT '88' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff00::'))); +SELECT '89' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff80::'))); +SELECT '90' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffc0::'))); +SELECT '91' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffe0::'))); +SELECT '92' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff0::'))); +SELECT '93' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff8::'))); +SELECT '94' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffc::'))); +SELECT '95' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffe::'))); +SELECT '96' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff::'))); +SELECT '97' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:8000:0'))); +SELECT '98' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:c000:0'))); +SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:e000:0'))); +SELECT '100' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f000:0'))); +SELECT '101' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f800:0'))); +SELECT '102' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fc00:0'))); +SELECT '103' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fe00:0'))); +SELECT '104' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff00:0'))); +SELECT '105' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff80:0'))); +SELECT '106' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffc0:0'))); +SELECT '107' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffe0:0'))); +SELECT '108' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff0:0'))); +SELECT '109' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff8:0'))); +SELECT '110' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffc:0'))); +SELECT '111' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffe:0'))); +SELECT '112' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:0'))); +SELECT '113' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:8000'))); +SELECT '114' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:c000'))); +SELECT '115' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:e000'))); +SELECT '116' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f000'))); +SELECT '117' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f800'))); +SELECT '118' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fc00'))); +SELECT '119' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fe00'))); +SELECT '120' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff00'))); +SELECT '121' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff80'))); +SELECT '122' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffc0'))); +SELECT '123' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffe0'))); +SELECT '124' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff0'))); +SELECT '125' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff8'))); +SELECT '126' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffc'))); +SELECT '127' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe'))); +SELECT '128' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'))); + +CREATE TABLE database_for_dict.table_from_ip_trie_dict +( + prefix String, + val String +) ENGINE = Dictionary(database_for_dict.dict_ip_trie); + +SELECT MIN(val == 'US') FROM database_for_dict.table_from_ip_trie_dict +WHERE prefix == '2620:0:870::/48'; + +SELECT 134 == COUNT(*) FROM database_for_dict.table_from_ip_trie_dict; + +DROP DATABASE IF EXISTS database_for_dict; From e2a9fe44b021d4b001fcd5ed6f744cc700fc315b Mon Sep 17 00:00:00 2001 From: taichong Date: Wed, 11 Nov 2020 23:37:44 +0800 Subject: [PATCH 0192/1088] modify test error --- src/Databases/MySQL/MaterializeMetadata.cpp | 4 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 4 +- .../materialize_with_ddl.py | 50 +++++++++---------- .../test_materialize_mysql_database/test.py | 13 +++-- 4 files changed, 37 insertions(+), 34 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 01a1493fa22..fdc9d7aa9ea 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -19,7 +19,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int SYNC_MYSQL_USER_ACCESS_ERR; + extern const int SYNC_MYSQL_USER_ACCESS_ERROR; } static std::unordered_map fetchTablesCreateQuery( @@ -146,7 +146,7 @@ static void checkSyncUserPriv(mysqlxx::PoolWithFailover::Entry & connection) throw Exception("MySQL SYNC USER ACCESS ERR: mysql sync user needs " "at least GLOBAL PRIVILEGES:'RELOAD, REPLICATION SLAVE, REPLICATION CLIENT' " "and SELECT PRIVILEGE on MySQL Database." - "But the SYNC USER grant query is: " + out.str(), ErrorCodes::SYNC_MYSQL_USER_ACCESS_ERR); + "But the SYNC USER grant query is: " + out.str(), ErrorCodes::SYNC_MYSQL_USER_ACCESS_ERROR); } bool MaterializeMetadata::checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection, const String & mysql_version) const diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index f76e6b77448..f2a71bb16a5 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -33,7 +33,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; extern const int ILLEGAL_MYSQL_VARIABLE; - extern const int SYNC_MYSQL_USER_ACCESS_ERR; + extern const int SYNC_MYSQL_USER_ACCESS_ERROR; extern const int UNKNOWN_DATABASE; } @@ -236,7 +236,7 @@ void MaterializeMySQLSyncThread::startSynchronization() throw Exception("MySQL SYNC USER ACCESS ERR: mysql sync user needs " "at least GLOBAL PRIVILEGES:'RELOAD, REPLICATION SLAVE, REPLICATION CLIENT' " "and SELECT PRIVILEGE on Database " + mysql_database_name - , ErrorCodes::SYNC_MYSQL_USER_ACCESS_ERR); + , ErrorCodes::SYNC_MYSQL_USER_ACCESS_ERROR); else if (e.errnum() == ER_BAD_DB_ERROR) throw Exception("Unknown database '" + mysql_database_name + "' on MySQL", ErrorCodes::UNKNOWN_DATABASE); else diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 52119add336..c248caf40a4 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -489,48 +489,48 @@ def select_without_columns(clickhouse_node, mysql_node, service_name): def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (4), (5), (6);") + mysql_node.query("DROP DATABASE IF EXISTS priv_err_db") + clickhouse_node.query("DROP DATABASE IF EXISTS priv_err_db") + mysql_node.query("CREATE DATABASE priv_err_db DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE priv_err_db.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO priv_err_db.test_table_1 VALUES(1);") mysql_node.result("SHOW GRANTS FOR 'test'@'%';") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + "CREATE DATABASE priv_err_db ENGINE = MaterializeMySQL('{}:3306', 'priv_err_db', 'test', '123')".format( service_name)) # wait MaterializeMySQL read binlog events - check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 30, 5) - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") - check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") - clickhouse_node.query("DROP DATABASE test_database;") + check_query(clickhouse_node, "SELECT count() FROM priv_err_db.test_table_1 FORMAT TSV", "1\n", 30, 5) + mysql_node.query("INSERT INTO priv_err_db.test_table_1 VALUES(2);") + check_query(clickhouse_node, "SELECT count() FROM priv_err_db.test_table_1 FORMAT TSV", "2\n") + clickhouse_node.query("DROP DATABASE priv_err_db;") mysql_node.query("REVOKE REPLICATION SLAVE ON *.* FROM 'test'@'%'") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + "CREATE DATABASE priv_err_db ENGINE = MaterializeMySQL('{}:3306', 'priv_err_db', 'test', '123')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") - clickhouse_node.query("DROP DATABASE test_database") + assert "priv_err_db" in clickhouse_node.query("SHOW DATABASES") + assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM priv_err_db") + clickhouse_node.query("DROP DATABASE priv_err_db") mysql_node.query("REVOKE REPLICATION CLIENT, RELOAD ON *.* FROM 'test'@'%'") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + "CREATE DATABASE priv_err_db ENGINE = MaterializeMySQL('{}:3306', 'priv_err_db', 'test', '123')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") - clickhouse_node.query("DROP DATABASE test_database") + assert "priv_err_db" in clickhouse_node.query("SHOW DATABASES") + assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM priv_err_db") + clickhouse_node.query("DETACH DATABASE priv_err_db") + + mysql_node.query("REVOKE SELECT ON priv_err_db.* FROM 'test'@'%'") + time.sleep(3) - mysql_node.query("REVOKE SELECT ON test_database.* FROM 'test'@'%'") with pytest.raises(QueryRuntimeException) as exception: - clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( - service_name)) + clickhouse_node.query("ATTACH DATABASE priv_err_db") assert 'MySQL SYNC USER ACCESS ERR:' in str(exception.value) - assert "test_database" not in clickhouse_node.query("SHOW DATABASES") + assert "priv_err_db" not in clickhouse_node.query("SHOW DATABASES") - mysql_node.query("DROP DATABASE test_database;") - mysql_node.query("DROP USER 'test'@'%';") + mysql_node.query("DROP DATABASE priv_err_db;") + mysql_node.grant_min_priv_for_user("test") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 72310ea30cb..3224b3fd19e 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -41,10 +41,13 @@ class MySQLNodeInstance: with self.alloc_connection().cursor() as cursor: cursor.execute(execution_query) - def create_min_priv_user(self): - self.query("CREATE USER 'test'@'%' IDENTIFIED BY '123'") - self.query("GRANT REPLICATION SLAVE, REPLICATION CLIENT, RELOAD ON *.* TO 'test'@'%'") - self.query("GRANT SELECT ON test_database.* TO 'test'@'%'") + def create_min_priv_user(self, user, password): + self.query("CREATE USER '" + user + "'@'%' IDENTIFIED BY '" + password + "'") + self.grant_min_priv_for_user(user) + + def grant_min_priv_for_user(self, user, db='test_database'): + self.query("GRANT REPLICATION SLAVE, REPLICATION CLIENT, RELOAD ON *.* TO '" + user + "'@'%'") + self.query("GRANT SELECT ON " + db + ".* TO '" + user + "'@'%'") def result(self, execution_query): with self.alloc_connection().cursor() as cursor: @@ -62,7 +65,7 @@ class MySQLNodeInstance: try: self.alloc_connection() print("Mysql Started") - self.create_min_priv_user() + self.create_min_priv_user("test", "123") print("min priv user created") return except Exception as ex: From 091df85cdc8f279376315a7587867fff40d49417 Mon Sep 17 00:00:00 2001 From: taichong Date: Thu, 12 Nov 2020 09:49:12 +0800 Subject: [PATCH 0193/1088] modify test db name --- tests/integration/test_materialize_mysql_database/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 3224b3fd19e..3befd708b2e 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -45,7 +45,7 @@ class MySQLNodeInstance: self.query("CREATE USER '" + user + "'@'%' IDENTIFIED BY '" + password + "'") self.grant_min_priv_for_user(user) - def grant_min_priv_for_user(self, user, db='test_database'): + def grant_min_priv_for_user(self, user, db='priv_err_db'): self.query("GRANT REPLICATION SLAVE, REPLICATION CLIENT, RELOAD ON *.* TO '" + user + "'@'%'") self.query("GRANT SELECT ON " + db + ".* TO '" + user + "'@'%'") From 010e12eb4b15f936c964358b9961284bf06cf40e Mon Sep 17 00:00:00 2001 From: taichong Date: Thu, 12 Nov 2020 13:49:56 +0800 Subject: [PATCH 0194/1088] Replace std::*stringstreams --- src/Databases/MySQL/MaterializeMetadata.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index fdc9d7aa9ea..eed4bddb588 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB { @@ -107,7 +108,7 @@ static Block getShowMasterLogHeader(const String & mysql_version) }; } -static bool checkSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, std::ostream & out) +static bool checkSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, WriteBuffer & out) { Block sync_user_privs_header { @@ -120,8 +121,8 @@ static bool checkSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, { for (size_t index = 0; index < block.rows(); ++index) { - out << (*block.getByPosition(0).column)[index].safeGet() + "; "; grants_query = (*block.getByPosition(0).column)[index].safeGet(); + out << grants_query << "; "; sub_privs = grants_query.substr(0, grants_query.find(" ON ")); if (sub_privs.find("ALL PRIVILEGES") == std::string::npos) { @@ -141,7 +142,8 @@ static bool checkSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, static void checkSyncUserPriv(mysqlxx::PoolWithFailover::Entry & connection) { - std::stringstream out; + WriteBufferFromOwnString out; + if (!checkSyncUserPrivImpl(connection, out)) throw Exception("MySQL SYNC USER ACCESS ERR: mysql sync user needs " "at least GLOBAL PRIVILEGES:'RELOAD, REPLICATION SLAVE, REPLICATION CLIENT' " From 7c19ad5ac7a60f964122b171ea5180a8a07d1bd6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 12 Nov 2020 10:37:27 +0300 Subject: [PATCH 0195/1088] Fix mappedIPv4ToBinary, add test reference for ip_dict --- src/Dictionaries/TrieDictionary.cpp | 39 +-- .../0_stateless/01018_ip_dictionary.reference | 228 ++++++++++++++++++ .../0_stateless/01018_ip_dictionary.sql | 36 ++- 3 files changed, 261 insertions(+), 42 deletions(-) diff --git a/src/Dictionaries/TrieDictionary.cpp b/src/Dictionaries/TrieDictionary.cpp index c475672f39b..18e0b1e9bb1 100644 --- a/src/Dictionaries/TrieDictionary.cpp +++ b/src/Dictionaries/TrieDictionary.cpp @@ -73,12 +73,15 @@ static inline bool compPrefixes(UInt8 a, UInt8 b) /// Convert mapped IPv6 to IPv4 if possible inline static UInt32 mappedIPv4ToBinary(const uint8_t * addr, bool & success) { - const UInt16* words = reinterpret_cast(addr); - auto has_zero_prefix = words[0] == 0 && words[1] == 0 && words[2] == 0 && words[3] == 0 && words[4] == 0; - success = has_zero_prefix && Poco::ByteOrder::fromNetwork(words[5]) == 0xFFFF; + success = addr[0] == 0x0 && addr[1] == 0x0 && + addr[2] == 0x0 && addr[3] == 0x0 && + addr[4] == 0x0 && addr[5] == 0x0 && + addr[6] == 0x0 && addr[7] == 0x0 && + addr[8] == 0x0 && addr[9] == 0x0 && + addr[10] == 0xff && addr[11] == 0xff; if (!success) return 0; - return Poco::ByteOrder::fromNetwork(*reinterpret_cast(&addr[6])); + return Poco::ByteOrder::fromNetwork(*reinterpret_cast(&addr[12])); } /// Convert IPv4 to IPv6-mapped and save results to buf @@ -91,34 +94,6 @@ inline static void mapIPv4ToIPv6(UInt32 addr, uint8_t * buf) memcpy(&buf[12], reinterpret_cast(&addr), 4); } -/* -static UInt32 applyMask32(UInt32 val, UInt8 prefix) -{ - UInt32 mask = (prefix >= 32) ? 0xffffffff : ~(0xffffffff >> prefix); - return val & mask; -} - -static void applyMask128(const uint8_t * val, uint8_t * out, UInt8 prefix) -{ - if (prefix >= 128) - prefix = 128; - - size_t i = 0; - - for (; prefix >= 8; ++i, prefix -= 8) - out[i] = val[i]; - - if (i >= 16) - return; - - uint8_t mask = ~(0xff >> prefix); - out[i] = val[i] & mask; - - i++; - memset(&out[i], 0, 16 - i); -} -*/ - static bool matchIPv4Subnet(UInt32 target, UInt32 addr, UInt8 prefix) { UInt32 mask = (prefix >= 32) ? 0xffffffff : ~(0xffffffff >> prefix); diff --git a/tests/queries/0_stateless/01018_ip_dictionary.reference b/tests/queries/0_stateless/01018_ip_dictionary.reference index e69de29bb2d..6023701a407 100644 --- a/tests/queries/0_stateless/01018_ip_dictionary.reference +++ b/tests/queries/0_stateless/01018_ip_dictionary.reference @@ -0,0 +1,228 @@ +***ipv4 trie dict*** +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +***ipv4 trie dict pt2*** +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +***ipv6 trie dict*** +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/01018_ip_dictionary.sql b/tests/queries/0_stateless/01018_ip_dictionary.sql index 4a93927e14b..f57f5ef9302 100644 --- a/tests/queries/0_stateless/01018_ip_dictionary.sql +++ b/tests/queries/0_stateless/01018_ip_dictionary.sql @@ -79,17 +79,15 @@ SELECT 1272 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IP SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.3'))); SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.255'))); --- SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0'))); --- SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1'))); --- SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.2'))); --- SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.3'))); --- SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.255'))); --- SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.127.127.127'))); --- SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.127.127.0'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.2'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.3'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.255'))); --- SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('101.79.55.22'))); --- SELECT 'x', dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:654f:3716'))); --- SELECT '11212', dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); +SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('101.79.55.22'))); +SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:654f:3716'))); +SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); SELECT 11211 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('202.79.32.2'))); @@ -152,6 +150,24 @@ SELECT 18 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4 SELECT 19 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.255'))); SELECT 20 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.255.128'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.127'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.255.127'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.127.127'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.9'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.127'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.10'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.255'))); +SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.255.128'))); + +SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('128.127.127.127'))); +SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('128.127.127.0'))); +SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('255.127.127.0'))); +SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('255.0.0.0'))); +SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('0.0.0.0'))); +SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('1.1.1.1'))); + SELECT '***ipv6 trie dict***'; CREATE TABLE database_for_dict.table_ip_trie From bbcd10f415da4d5000698742a7a0eef082071bad Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Nov 2020 13:58:14 +0300 Subject: [PATCH 0196/1088] Don't visit functions with multiple arguments in MonotonicityCheckMatcher --- src/Interpreters/MonotonicityCheckVisitor.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index 405ec1cb479..0b3660eb85c 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -131,8 +131,11 @@ public: data.reject(); } - static bool needChildVisit(const ASTPtr &, const ASTPtr &) + static bool needChildVisit(const ASTPtr & parent, const ASTPtr &) { + if (const auto * func = typeid_cast(parent.get())) + return func->children.size() < 2; + return true; } }; From 37c70f4f6aa3b0e82c61f9f468cd53ed744e0ce1 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 12 Nov 2020 14:53:30 +0300 Subject: [PATCH 0197/1088] improved the wide int constructor --- base/common/wide_integer_impl.h | 63 ++++++++++++--------------------- 1 file changed, 22 insertions(+), 41 deletions(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index 5b13c56e2f7..48a2a278df9 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -228,58 +228,39 @@ struct integer::_impl self.items[i] = 0; } - constexpr static void wide_integer_from_bultin(integer & self, double rhs) noexcept - { + /** + * N.B. t is constructed from double, so max(t) = max(double) ~ 2^310 + * the recursive call happens when t / 2^64 > 2^64, so there won't be more than 5 of them. + */ + template + constexpr static void set_multiplier(wide_integer & self, T t) noexcept { + constexpr uint64_t max_int = std::numeric_limits::max(); + const T alpha = t / max_int; + + if (alpha <= max_int) + for (uint64_t i = 0; i < static_cast(alpha); ++i) + self *= max_int; + else // max(double) / 2^64 will surely contain less than 52 precision bits, so speed up computations. + set_multiplier(self, static_cast(alpha)); + + self += static_cast(t - alpha * max_int); + } + + constexpr static void wide_integer_from_bultin(wide_integer& self, double rhs) noexcept { constexpr int64_t max_int = std::numeric_limits::max(); constexpr int64_t min_int = std::numeric_limits::min(); - constexpr long double max_int_long_double = static_cast(max_int); - - if ((rhs > 0 && rhs < max_int) || - (rhs < 0 && rhs > min_int)) - { - self = to_Integral(rhs); + if ((rhs > 0 && rhs < max_int) || (rhs < 0 && rhs > min_int)) { + self = static_cast(rhs); return; } - /// There are values in int64 that have more than 53 significant bits (in terms of double - /// representation). Such values, being promoted to double, are rounded up or down. If they are rounded up, - /// the result may not fit in 64 bits. - /// The example of such a number is 9.22337e+18. - /// As to_Integral does a static_cast to int64_t, it may result in UB. - /// The necessary check here is that long double has enough significant (mantissa) bits to store the - /// int64_t max value precisely. - static_assert(LDBL_MANT_DIG >= 64, - "On your system long double has less than 64 precision bits," - "which may result in UB when initializing double from int64_t"); - - /// Always >= 0 const long double rhs_long_double = (static_cast(rhs) < 0) ? -static_cast(rhs) : rhs; - const long double rhs_max_int_count = rhs_long_double / max_int; - - // We can't just get the number of iterations like rhs_max_int_count / max_int as it may not fit it int64_t. - long double rhs_max_int_count_acc = rhs_max_int_count; - self = 0; - - while (rhs_max_int_count_acc > max_int_long_double) - { - self += max_int; - rhs_max_int_count_acc -= max_int_long_double; - } - - self *= max_int; - - const long double rhs_div_max_int = rhs_max_int_count * max_int; - const long double rhs_mod_max_int = rhs_long_double - rhs_div_max_int; - - assert(rhs_mod_max_int < max_int_long_double); - assert(rhs_mod_max_int > static_cast(min_int)); - - self += static_cast(rhs_mod_max_int); + set_multiplier(self, rhs_long_double); if (rhs < 0) self = -self; From 5e1c84e04fc6a681aba3b29111a81f69244243d5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 12 Nov 2020 15:11:13 +0300 Subject: [PATCH 0198/1088] Fix segfault --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 4df4882a25b..9e20035047d 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1346,8 +1346,6 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) if (expired) return; - active_session_metric_increment.destroy(); - try { if (!error_send) @@ -1376,6 +1374,8 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) expired = true; } + active_session_metric_increment.destroy(); + try { /// This will also wakeup the receiving thread. From e27afbee2a5770ab0be2cec959aa6b1635b86cce Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Nov 2020 15:12:52 +0300 Subject: [PATCH 0199/1088] Don't visit functions with multiple arguments in MonotonicityCheckMatcher --- src/Interpreters/MonotonicityCheckVisitor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index 0b3660eb85c..1e29964f04b 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -134,7 +134,7 @@ public: static bool needChildVisit(const ASTPtr & parent, const ASTPtr &) { if (const auto * func = typeid_cast(parent.get())) - return func->children.size() < 2; + return func->arguments->children.size() < 2; return true; } From 8c4bb8cd0293f54f4d0dfa10c94c152dbd9fc6aa Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 12 Nov 2020 15:17:34 +0300 Subject: [PATCH 0200/1088] naming fix --- base/common/wide_integer_impl.h | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index 48a2a278df9..e18a27df48a 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -233,7 +233,7 @@ struct integer::_impl * the recursive call happens when t / 2^64 > 2^64, so there won't be more than 5 of them. */ template - constexpr static void set_multiplier(wide_integer & self, T t) noexcept { + constexpr static void set_multiplier(integer & self, T t) noexcept { constexpr uint64_t max_int = std::numeric_limits::max(); const T alpha = t / max_int; @@ -241,15 +241,26 @@ struct integer::_impl for (uint64_t i = 0; i < static_cast(alpha); ++i) self *= max_int; else // max(double) / 2^64 will surely contain less than 52 precision bits, so speed up computations. - set_multiplier(self, static_cast(alpha)); + set_multiplier(self, alpha); self += static_cast(t - alpha * max_int); } - constexpr static void wide_integer_from_bultin(wide_integer& self, double rhs) noexcept { + constexpr static void wide_integer_from_bultin(integer& self, double rhs) noexcept { constexpr int64_t max_int = std::numeric_limits::max(); constexpr int64_t min_int = std::numeric_limits::min(); + /// There are values in int64 that have more than 53 significant bits (in terms of double + /// representation). Such values, being promoted to double, are rounded up or down. If they are rounded up, + /// the result may not fit in 64 bits. + /// The example of such a number is 9.22337e+18. + /// As to_Integral does a static_cast to int64_t, it may result in UB. + /// The necessary check here is that long double has enough significant (mantissa) bits to store the + /// int64_t max value precisely. + static_assert(LDBL_MANT_DIG >= 64, + "On your system long double has less than 64 precision bits," + "which may result in UB when initializing double from int64_t"); + if ((rhs > 0 && rhs < max_int) || (rhs < 0 && rhs > min_int)) { self = static_cast(rhs); return; From 1fec1de199398daa228840eac1cd7d805d2890da Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Nov 2020 15:39:35 +0300 Subject: [PATCH 0201/1088] Sdded test --- ...tonicity_check_multiple_args_bug.reference | 4 ++++ ...0_monotonicity_check_multiple_args_bug.sql | 19 +++++++++++++++++++ 2 files changed, 23 insertions(+) create mode 100644 tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference create mode 100644 tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql diff --git a/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference new file mode 100644 index 00000000000..b0a59aa58cc --- /dev/null +++ b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference @@ -0,0 +1,4 @@ +2020-11-12 20.12.1.1 +2020-11-13 20.12.1.1 +2020-11-12 20.12.1.1 +2020-11-13 20.12.1.1 diff --git a/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql new file mode 100644 index 00000000000..d2afd4b29be --- /dev/null +++ b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql @@ -0,0 +1,19 @@ +WITH arrayJoin(range(2)) AS delta +SELECT + toDate(time) + toIntervalDay(delta) AS dt, + version() +FROM +( + SELECT NOW() AS time +) +ORDER BY dt ASC; + +WITH arrayJoin([0, 1]) AS delta +SELECT + toDate(time) + toIntervalDay(delta) AS dt, + version() +FROM +( + SELECT NOW() AS time +) +ORDER BY dt ASC; From 70867eb187d34decff04158135d1da31ae4799b7 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 12 Nov 2020 16:13:22 +0300 Subject: [PATCH 0202/1088] fixing the recursive call case --- base/common/wide_integer_impl.h | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index e18a27df48a..ade02ee1e78 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -231,6 +231,10 @@ struct integer::_impl /** * N.B. t is constructed from double, so max(t) = max(double) ~ 2^310 * the recursive call happens when t / 2^64 > 2^64, so there won't be more than 5 of them. + * + * t = a1 * max_int + b1, a1 > max_int, b1 < max_int + * a1 = a2 * max_int + b2, a2 > max_int, b2 < max_int + * a_(n - 1) = a_n * max_int + b2, a_n <= max_int <- base case. */ template constexpr static void set_multiplier(integer & self, T t) noexcept { @@ -240,10 +244,13 @@ struct integer::_impl if (alpha <= max_int) for (uint64_t i = 0; i < static_cast(alpha); ++i) self *= max_int; - else // max(double) / 2^64 will surely contain less than 52 precision bits, so speed up computations. + else + { // max(double) / 2^64 will surely contain less than 52 precision bits, so speed up computations. set_multiplier(self, alpha); + self *= max_int; + }; - self += static_cast(t - alpha * max_int); + self += static_cast(t - alpha * max_int); // += b_i } constexpr static void wide_integer_from_bultin(integer& self, double rhs) noexcept { @@ -261,7 +268,8 @@ struct integer::_impl "On your system long double has less than 64 precision bits," "which may result in UB when initializing double from int64_t"); - if ((rhs > 0 && rhs < max_int) || (rhs < 0 && rhs > min_int)) { + if ((rhs > 0 && rhs < max_int) || (rhs < 0 && rhs > min_int)) + { self = static_cast(rhs); return; } @@ -270,7 +278,7 @@ struct integer::_impl ? -static_cast(rhs) : rhs; - self = 0; + self = (rhs_long_double / max_int > 0) ? 1 : 0; set_multiplier(self, rhs_long_double); if (rhs < 0) From 38c126ecfdab9bea02c7f0ad7385eaa23007c358 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 12 Nov 2020 16:13:30 +0300 Subject: [PATCH 0203/1088] Fix style and build error --- src/AggregateFunctions/QuantileTDigest.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index ee71c128c44..c5b1c7fd2d6 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -14,6 +14,7 @@ namespace DB namespace ErrorCodes { extern const int TOO_LARGE_ARRAY_SIZE; + extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; } @@ -123,7 +124,8 @@ class TDigest if (unmerged > params.max_unmerged) compress(); } - void compress_brute() { + void compress_brute() + { if (centroids.size() <= params.max_centroids) return; const size_t batch_size = (centroids.size() + params.max_centroids - 1) / params.max_centroids; // at least 2 From 869a6f6aa040af26ba45e6751e47637a1f409c34 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Thu, 12 Nov 2020 16:25:05 +0300 Subject: [PATCH 0204/1088] Add EN and RU description --- docs/en/operations/settings/settings.md | 10 ++++++++++ docs/ru/operations/settings/settings.md | 10 ++++++++++ 2 files changed, 20 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f9c3c8a5d75..f720024f524 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2233,4 +2233,14 @@ Possible values: Default value: `1`. +## output_format_tsv_null_representation {#output_format_tsv_null_representation} + +Allows configurable `NULL` representation for [TSV](../../interfaces/formats.md#tabseparated) output format. The setting only controls output format and `\N` is the only supported `NULL` representation for TSV input format. + +Possible values: + +- `\N` — Enabled. + +Default value: `\N`. + [Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 39a996cb44e..9e192348f51 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -2095,4 +2095,14 @@ SELECT CAST(toNullable(toInt32(0)) AS Int32) as x, toTypeName(x); Значение по умолчанию: `1`. +## output_format_tsv_null_representation {#output_format_tsv_null_representation} + +Позволяет настраивать представление `NULL` для формата выходных данных [TSV](../../interfaces/formats.md#tabseparated). Настройка управляет форматом выходных данных, `\N` является единственным поддерживаемым представлением для формата входных данных TSV. + +Возможные значения: + +- `\N` — включено. + +Значение по умолчанию: `\N`. + [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) From 862f8da331ab18e092709a9a07aa1712fe63facc Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 12 Nov 2020 16:27:07 +0300 Subject: [PATCH 0205/1088] typo --- base/common/wide_integer_impl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index ade02ee1e78..2b55d101be5 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -248,7 +248,7 @@ struct integer::_impl { // max(double) / 2^64 will surely contain less than 52 precision bits, so speed up computations. set_multiplier(self, alpha); self *= max_int; - }; + } self += static_cast(t - alpha * max_int); // += b_i } From fefa2a25480bc54329b57f85eac48132af031198 Mon Sep 17 00:00:00 2001 From: taichong Date: Thu, 12 Nov 2020 22:24:22 +0800 Subject: [PATCH 0206/1088] add log std out --- .../test_materialize_mysql_database/materialize_with_ddl.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index c248caf40a4..6d69d536bea 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -500,7 +500,8 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n clickhouse_node.query( "CREATE DATABASE priv_err_db ENGINE = MaterializeMySQL('{}:3306', 'priv_err_db', 'test', '123')".format( service_name)) - + print("\n=== print log for CI test err ===\n") + print(clickhouse_node.exec_in_container(["bash", "-c", 'cat /var/log/clickhouse-server/clickhouse-server.log'])) # wait MaterializeMySQL read binlog events check_query(clickhouse_node, "SELECT count() FROM priv_err_db.test_table_1 FORMAT TSV", "1\n", 30, 5) mysql_node.query("INSERT INTO priv_err_db.test_table_1 VALUES(2);") From e19d1430dbb7bb014d95be78db6da5d6949ef95a Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 12 Nov 2020 17:43:16 +0300 Subject: [PATCH 0207/1088] Fix livelock --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 38 ++++++++++++++++++++------ 1 file changed, 29 insertions(+), 9 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 9e20035047d..addb6c01504 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1070,6 +1070,7 @@ void ZooKeeper::sendThread() setThreadName("ZooKeeperSend"); auto prev_heartbeat_time = clock::now(); + bool tried_to_send_close = false; try { @@ -1099,6 +1100,15 @@ void ZooKeeper::sendThread() std::lock_guard lock(operations_mutex); operations[info.request->xid] = info; } + else + { + /// We set this variable only once. If we will + /// successfully send close, than this thread will just + /// finish. If we will got an exception while sending + /// close, than thread will also finish and finalization + /// will be completed by some other thread. + tried_to_send_close = true; + } if (info.watch) { @@ -1135,7 +1145,13 @@ void ZooKeeper::sendThread() catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); - finalize(true, false); + /// If we have tried to send close and got an exception than + /// finalization is already started by receiveThread and we cannot do + /// anything better than just exit. + /// + /// Otherwise we should correctly finalize + if (!tried_to_send_close) + finalize(true, false); } } @@ -1346,6 +1362,16 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) if (expired) return; + auto expire_session = [&] + { + std::lock_guard lock(push_request_mutex); + if (!expired) + { + expired = true; + active_session_metric_increment.destroy(); + } + }; + try { if (!error_send) @@ -1359,8 +1385,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) { /// This happens for example, when "Cannot push request to queue within operation timeout". /// Just mark session expired in case of error on close request. - std::lock_guard lock(push_request_mutex); - expired = true; + expire_session(); tryLogCurrentException(__PRETTY_FUNCTION__); } @@ -1369,12 +1394,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) } /// Set expired flag after we sent close event - { - std::lock_guard lock(push_request_mutex); - expired = true; - } - - active_session_metric_increment.destroy(); + expire_session(); try { From eb46951e75f49776a096073ef1f45699c10d7fb1 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 12 Nov 2020 15:55:26 +0000 Subject: [PATCH 0208/1088] aggregate_functions_null_for_empty move OrNull ahead --- src/Interpreters/TreeRewriter.cpp | 27 ++++++++++++++ .../01562_agg_null_for_empty_ahead.reference | 20 +++++++++++ .../01562_agg_null_for_empty_ahead.sql | 35 +++++++++++++++++++ 3 files changed, 82 insertions(+) create mode 100644 tests/queries/0_stateless/01562_agg_null_for_empty_ahead.reference create mode 100644 tests/queries/0_stateless/01562_agg_null_for_empty_ahead.sql diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index b2dbd027191..8724d2979c6 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -125,7 +125,34 @@ struct CustomizeAggregateFunctionsSuffixData { auto properties = instance.tryGetProperties(func.name); if (properties && !properties->returns_default_when_only_null) + { + auto name_size = func.name.size(); + if (endsWith(func.name, "MergeState")) + { + func.name = func.name.substr(0, name_size - 10) + customized_func_suffix + "MergeState"; + return; + } + + if (endsWith(func.name, "Merge")) + { + func.name = func.name.substr(0, name_size - 5) + customized_func_suffix + "Merge"; + return; + } + + if (endsWith(func.name, "State")) + { + func.name = func.name.substr(0, name_size - 5) + customized_func_suffix + "State"; + return; + } + + if (endsWith(func.name, "If")) + { + func.name = func.name.substr(0, name_size - 2) + customized_func_suffix + "If"; + return; + } + func.name = func.name + customized_func_suffix; + } } } }; diff --git a/tests/queries/0_stateless/01562_agg_null_for_empty_ahead.reference b/tests/queries/0_stateless/01562_agg_null_for_empty_ahead.reference new file mode 100644 index 00000000000..31133cfb3a1 --- /dev/null +++ b/tests/queries/0_stateless/01562_agg_null_for_empty_ahead.reference @@ -0,0 +1,20 @@ +0 +0 +0 +0 +0 +1 +0 +\N +1 +\N +\N +0 +\N +0 +\N +1 +0 +\N +1 +\N diff --git a/tests/queries/0_stateless/01562_agg_null_for_empty_ahead.sql b/tests/queries/0_stateless/01562_agg_null_for_empty_ahead.sql new file mode 100644 index 00000000000..50ce4a84ee0 --- /dev/null +++ b/tests/queries/0_stateless/01562_agg_null_for_empty_ahead.sql @@ -0,0 +1,35 @@ +SELECT sumMerge(s) FROM (SELECT sumState(number) s FROM numbers(0)); +SELECT sumMerge(s) FROM (SELECT sumState(number) s FROM numbers(1)); + +SELECT sumMerge(s) FROM (SELECT sumMergeState(n) s FROM (SELECT sumState(number) n FROM numbers(0))); +SELECT sumMerge(s) FROM (SELECT sumMergeState(n) s FROM (SELECT sumState(number) n FROM numbers(1))); + +SELECT sumIf(1, 0); + +SELECT sumIf(1, 1); + +SELECT sumIfOrNull(1, 0); +SELECT sumOrNullIf(1, 0); + +SELECT nullIf(1, 0); + +SELECT nullIf(1, 1); + +SET aggregate_functions_null_for_empty=1; + +SELECT sumMerge(s) FROM (SELECT sumState(number) s FROM numbers(0)); +SELECT sumMerge(s) FROM (SELECT sumState(number) s FROM numbers(1)); + +SELECT sumMerge(s) FROM (SELECT sumMergeState(n) s FROM (SELECT sumState(number) n FROM numbers(0))); +SELECT sumMerge(s) FROM (SELECT sumMergeState(n) s FROM (SELECT sumState(number) n FROM numbers(1))); + +SELECT sumIf(1, 0); + +SELECT sumIf(1, 1); + +SELECT sumIfOrNull(1, 0); +SELECT sumOrNullIf(1, 0); + +SELECT nullIf(1, 0); + +SELECT nullIf(1, 1); From b88402876c3e8ab27e5d9351c2c6c172b3b15e0c Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 12 Nov 2020 19:01:20 +0300 Subject: [PATCH 0209/1088] fix loop --- base/common/wide_integer_impl.h | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index 2b55d101be5..74ed69faeab 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -242,14 +242,11 @@ struct integer::_impl const T alpha = t / max_int; if (alpha <= max_int) - for (uint64_t i = 0; i < static_cast(alpha); ++i) - self *= max_int; - else - { // max(double) / 2^64 will surely contain less than 52 precision bits, so speed up computations. + self = static_cast(alpha); + else // max(double) / 2^64 will surely contain less than 52 precision bits, so speed up computations. set_multiplier(self, alpha); - self *= max_int; - } + self *= max_int; self += static_cast(t - alpha * max_int); // += b_i } From 071c946236369fe37da2414cdfdfd5aa0a02b4c7 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 12 Nov 2020 19:02:27 +0300 Subject: [PATCH 0210/1088] removed useless self init --- base/common/wide_integer_impl.h | 1 - 1 file changed, 1 deletion(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index 74ed69faeab..2a889819c11 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -275,7 +275,6 @@ struct integer::_impl ? -static_cast(rhs) : rhs; - self = (rhs_long_double / max_int > 0) ? 1 : 0; set_multiplier(self, rhs_long_double); if (rhs < 0) From e031e805c44ff2e3b05e105de8970193fc759b24 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Nov 2020 19:04:35 +0300 Subject: [PATCH 0211/1088] Fix test. --- ...1560_monotonicity_check_multiple_args_bug.reference | 8 ++++---- .../01560_monotonicity_check_multiple_args_bug.sql | 10 ++++------ 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference index b0a59aa58cc..5d632f2f5b8 100644 --- a/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference +++ b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference @@ -1,4 +1,4 @@ -2020-11-12 20.12.1.1 -2020-11-13 20.12.1.1 -2020-11-12 20.12.1.1 -2020-11-13 20.12.1.1 +2020-11-12 +2020-11-13 +2020-11-12 +2020-11-13 diff --git a/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql index d2afd4b29be..befc13be8eb 100644 --- a/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql +++ b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql @@ -1,19 +1,17 @@ WITH arrayJoin(range(2)) AS delta SELECT - toDate(time) + toIntervalDay(delta) AS dt, - version() + toDate(time) + toIntervalDay(delta) AS dt FROM ( - SELECT NOW() AS time + SELECT toDateTime('2020.11.12 19:02:04') AS time ) ORDER BY dt ASC; WITH arrayJoin([0, 1]) AS delta SELECT - toDate(time) + toIntervalDay(delta) AS dt, - version() + toDate(time) + toIntervalDay(delta) AS dt FROM ( - SELECT NOW() AS time + SELECT toDateTime('2020.11.12 19:02:04') AS time ) ORDER BY dt ASC; From 766883a0e381b2db352dc6de5170749f573cae04 Mon Sep 17 00:00:00 2001 From: satanson Date: Fri, 13 Nov 2020 00:47:59 +0800 Subject: [PATCH 0212/1088] A very fast O(logn) repeat impl --- src/Functions/repeat.cpp | 33 ++++++++++++++++++++++++++++++--- 1 file changed, 30 insertions(+), 3 deletions(-) diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index 63af62af9af..347402a6ac8 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -124,12 +124,39 @@ struct RepeatImpl } private: + // A very fast repeat implementation, only invoke memcpy for O(log(n)) times. + // as the calling times decreases, more data will be copied for each memcpy, thus + // SIMD optimization will be more efficient. static void process(const UInt8 * src, UInt8 * dst, UInt64 size, UInt64 repeat_time) { - for (UInt64 i = 0; i < repeat_time; ++i) + if (BOOST_UNLIKELY(repeat_time <= 0)) { - memcpy(dst, src, size - 1); - dst += size - 1; + *dst = 0; + return; + } + + size -= 1; + UInt64 k = 0; + UInt64 last_bit = repeat_time & 1; + repeat_time >>= 1; + + const UInt8 * dst_hdr = dst; + memcpy(dst, src, size); + dst += size; + + while (repeat_time > 0) + { + UInt64 cpy_size = size * (1 << k); + memcpy(dst, dst_hdr, cpy_size); + dst += cpy_size; + if (last_bit) + { + memcpy(dst, dst_hdr, cpy_size); + dst += cpy_size; + } + k += 1; + last_bit = repeat_time & 1; + repeat_time >>= 1; } *dst = 0; } From aeab02a02f492c3ad9c81c61a16a63a0730dcecc Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 12 Nov 2020 21:01:51 +0300 Subject: [PATCH 0213/1088] fix test --- tests/queries/0_stateless/01035_avg.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01035_avg.reference b/tests/queries/0_stateless/01035_avg.reference index 9ad7fc76be1..ec79205c6f8 100644 --- a/tests/queries/0_stateless/01035_avg.reference +++ b/tests/queries/0_stateless/01035_avg.reference @@ -1,5 +1,5 @@ nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan --0.5 -0.5 -0.5 -0.5 -0.5 -0.5 127.493856 32355.57552 499999.5 499999.5 499999.5 499999.5 499999.5 -0.000500002 0.49999949943727 -0.000005 -0.000005291390805620593 +-0.5 -0.5 -0.5 -0.5 -0.5 -0.5 127.493856 32355.57552 499999.5 499999.5 499999.5 499999.5 499999.5 -0.000500002 0.49999949943727 -0.000005 -0.000005257366687274546 -2767.546272 999999 -0.50000449943727 From 7990c8cbad00f4287374573c6020208b21d3b556 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 12 Nov 2020 22:50:01 +0300 Subject: [PATCH 0214/1088] fix optimization with 'optimize_aggregators_of_group_by_keys' and joins --- .../AggregateFunctionOfGroupByKeysVisitor.h | 26 +++--- src/Interpreters/TreeOptimizer.cpp | 81 +++++-------------- ...egate_functions_of_key_with_join.reference | 1 + ...1_aggregate_functions_of_key_with_join.sql | 5 ++ 4 files changed, 36 insertions(+), 77 deletions(-) create mode 100644 tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.reference create mode 100644 tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.sql diff --git a/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h b/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h index 6b903ec45cf..587baa660cb 100644 --- a/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h +++ b/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h @@ -20,8 +20,8 @@ struct KeepAggregateFunctionMatcher { struct Data { - std::unordered_set & group_by_keys; - bool & keep_aggregator; + const NameSet & group_by_keys; + bool keep_aggregator; }; using Visitor = InDepthNodeVisitor; @@ -33,7 +33,7 @@ struct KeepAggregateFunctionMatcher static void visit(ASTFunction & function_node, Data & data) { - if ((function_node.arguments->children).empty()) + if (function_node.arguments->children.empty()) { data.keep_aggregator = true; return; @@ -47,12 +47,9 @@ struct KeepAggregateFunctionMatcher static void visit(ASTIdentifier & ident, Data & data) { - if (!data.group_by_keys.count(ident.shortName())) - { - /// if variable of a function is not in GROUP BY keys, this function should not be deleted + /// if variable of a function is not in GROUP BY keys, this function should not be deleted + if (!data.group_by_keys.count(ident.getColumnName())) data.keep_aggregator = true; - return; - } } static void visit(const ASTPtr & ast, Data & data) @@ -75,21 +72,21 @@ struct KeepAggregateFunctionMatcher } }; -using KeepAggregateFunctionVisitor = InDepthNodeVisitor; +using KeepAggregateFunctionVisitor = KeepAggregateFunctionMatcher::Visitor; class SelectAggregateFunctionOfGroupByKeysMatcher { public: struct Data { - std::unordered_set & group_by_keys; + const NameSet & group_by_keys; }; static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { /// Don't descent into table functions and subqueries and special case for ArrayJoin. - return !node->as() && - !(node->as() || node->as() || node->as()); + return !node->as() && !node->as() + && !node->as() && !node->as(); } static void visit(ASTPtr & ast, Data & data) @@ -99,12 +96,11 @@ public: if (function_node && (function_node->name == "min" || function_node->name == "max" || function_node->name == "any" || function_node->name == "anyLast")) { - bool keep_aggregator = false; - KeepAggregateFunctionVisitor::Data keep_data{data.group_by_keys, keep_aggregator}; + KeepAggregateFunctionVisitor::Data keep_data{data.group_by_keys, false}; KeepAggregateFunctionVisitor(keep_data).visit(function_node->arguments); /// Place argument of an aggregate function instead of function - if (!keep_aggregator) + if (!keep_data.keep_aggregator) { String alias = function_node->alias; ast = (function_node->arguments->children[0])->clone(); diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 61ca933dd53..b0f9ef187f1 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -177,43 +177,21 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum struct GroupByKeysInfo { - std::unordered_set key_names; ///set of keys' short names - bool has_identifier = false; + NameSet key_names; ///set of keys' short names bool has_function = false; - bool has_possible_collision = false; }; -GroupByKeysInfo getGroupByKeysInfo(ASTs & group_keys) +GroupByKeysInfo getGroupByKeysInfo(const ASTs & group_by_keys) { GroupByKeysInfo data; - ///filling set with short names of keys - for (auto & group_key : group_keys) + /// filling set with short names of keys + for (auto & group_key : group_by_keys) { if (group_key->as()) data.has_function = true; - if (auto * group_key_ident = group_key->as()) - { - data.has_identifier = true; - if (data.key_names.count(group_key_ident->shortName())) - { - ///There may be a collision between different tables having similar variables. - ///Due to the fact that we can't track these conflicts yet, - ///it's better to disable some optimizations to avoid elimination necessary keys. - data.has_possible_collision = true; - } - - data.key_names.insert(group_key_ident->shortName()); - } - else if (auto * group_key_func = group_key->as()) - { - data.key_names.insert(group_key_func->getColumnName()); - } - else - { - data.key_names.insert(group_key->getColumnName()); - } + data.key_names.insert(group_key->getColumnName()); } return data; @@ -225,47 +203,28 @@ void optimizeGroupByFunctionKeys(ASTSelectQuery * select_query) if (!select_query->groupBy()) return; - auto grp_by = select_query->groupBy(); - auto & group_keys = grp_by->children; + auto group_by = select_query->groupBy(); + const auto & group_by_keys = group_by->children; ASTs modified; ///result - GroupByKeysInfo group_by_keys_data = getGroupByKeysInfo(group_keys); + GroupByKeysInfo group_by_keys_data = getGroupByKeysInfo(group_by_keys); - if (!group_by_keys_data.has_function || group_by_keys_data.has_possible_collision) + if (!group_by_keys_data.has_function) return; GroupByFunctionKeysVisitor::Data visitor_data{group_by_keys_data.key_names}; - GroupByFunctionKeysVisitor(visitor_data).visit(grp_by); + GroupByFunctionKeysVisitor(visitor_data).visit(group_by); - modified.reserve(group_keys.size()); + modified.reserve(group_by_keys.size()); - ///filling the result - for (auto & group_key : group_keys) - { - if (auto * group_key_func = group_key->as()) - { - if (group_by_keys_data.key_names.count(group_key_func->getColumnName())) - modified.push_back(group_key); + /// filling the result + for (auto & group_key : group_by_keys) + if (group_by_keys_data.key_names.count(group_key->getColumnName())) + modified.push_back(group_key); - continue; - } - if (auto * group_key_ident = group_key->as()) - { - if (group_by_keys_data.key_names.count(group_key_ident->shortName())) - modified.push_back(group_key); - - continue; - } - else - { - if (group_by_keys_data.key_names.count(group_key->getColumnName())) - modified.push_back(group_key); - } - } - - ///modifying the input - grp_by->children = modified; + /// modifying the input + group_by->children = modified; } /// Eliminates min/max/any-aggregators of functions of GROUP BY keys @@ -274,10 +233,8 @@ void optimizeAggregateFunctionsOfGroupByKeys(ASTSelectQuery * select_query, ASTP if (!select_query->groupBy()) return; - auto grp_by = select_query->groupBy(); - auto & group_keys = grp_by->children; - - GroupByKeysInfo group_by_keys_data = getGroupByKeysInfo(group_keys); + auto & group_by_keys = select_query->groupBy()->children; + GroupByKeysInfo group_by_keys_data = getGroupByKeysInfo(group_by_keys); SelectAggregateFunctionOfGroupByKeysVisitor::Data visitor_data{group_by_keys_data.key_names}; SelectAggregateFunctionOfGroupByKeysVisitor(visitor_data).visit(node); diff --git a/tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.reference b/tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.reference new file mode 100644 index 00000000000..9874d6464ab --- /dev/null +++ b/tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.reference @@ -0,0 +1 @@ +1 2 diff --git a/tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.sql b/tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.sql new file mode 100644 index 00000000000..66047fcc1a6 --- /dev/null +++ b/tests/queries/0_stateless/01561_aggregate_functions_of_key_with_join.sql @@ -0,0 +1,5 @@ +SET optimize_aggregators_of_group_by_keys = 1; +SELECT source.key, max(target.key) FROM (SELECT 1 key, 'x' name) source +INNER JOIN (SELECT 2 key, 'x' name) target +ON source.name = target.name +GROUP BY source.key; From eb9604ecd005d8c076e001a04efae822d5567e03 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Thu, 12 Nov 2020 23:23:26 +0300 Subject: [PATCH 0215/1088] Fixed comparison of DateTime4 with different scales Made DateTime64 a stand-alone type rather than typedef to allow function resolution/deduction based on type. Added tests. --- src/Columns/ColumnDecimal.cpp | 1 + src/Core/DecimalComparison.h | 14 ++--- src/Core/DecimalFunctions.h | 60 ++++++++++++++++++- src/Core/Field.cpp | 4 ++ src/Core/Field.h | 3 + src/Core/Types.h | 17 +++++- src/Core/callOnTypeIndex.h | 2 +- src/DataTypes/DataTypeDateTime64.h | 1 + src/DataTypes/DataTypeDecimalBase.cpp | 1 + src/DataTypes/DataTypeDecimalBase.h | 42 +++++++------ src/DataTypes/IDataType.h | 8 +++ src/IO/ReadHelpers.h | 14 +++-- .../01553_datetime64_comparison.reference | 3 + .../01553_datetime64_comparison.sql | 20 +++++++ 14 files changed, 156 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/01553_datetime64_comparison.reference create mode 100644 tests/queries/0_stateless/01553_datetime64_comparison.sql diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 7c3af5fe095..196f77d0db8 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -421,4 +421,5 @@ template class ColumnDecimal; template class ColumnDecimal; template class ColumnDecimal; template class ColumnDecimal; +template class ColumnDecimal; } diff --git a/src/Core/DecimalComparison.h b/src/Core/DecimalComparison.h index 674ed31683b..600e876f326 100644 --- a/src/Core/DecimalComparison.h +++ b/src/Core/DecimalComparison.h @@ -113,15 +113,15 @@ private: static std::enable_if_t && IsDecimalNumber, Shift> getScales(const DataTypePtr & left_type, const DataTypePtr & right_type) { - const DataTypeDecimal * decimal0 = checkDecimal(*left_type); - const DataTypeDecimal * decimal1 = checkDecimal(*right_type); + const DataTypeDecimalBase * decimal0 = checkDecimalBase(*left_type); + const DataTypeDecimalBase * decimal1 = checkDecimalBase(*right_type); Shift shift; if (decimal0 && decimal1) { - auto result_type = decimalResultType(*decimal0, *decimal1); - shift.a = static_cast(result_type.scaleFactorFor(*decimal0, false).value); - shift.b = static_cast(result_type.scaleFactorFor(*decimal1, false).value); + auto result_type = DecimalUtils::binaryOpResult(*decimal0, *decimal1); + shift.a = static_cast(result_type.scaleFactorFor(decimal0->getTrait(), false).value); + shift.b = static_cast(result_type.scaleFactorFor(decimal1->getTrait(), false).value); } else if (decimal0) shift.b = static_cast(decimal0->getScaleMultiplier().value); @@ -136,7 +136,7 @@ private: getScales(const DataTypePtr & left_type, const DataTypePtr &) { Shift shift; - const DataTypeDecimal * decimal0 = checkDecimal(*left_type); + const DataTypeDecimalBase * decimal0 = checkDecimalBase(*left_type); if (decimal0) shift.b = static_cast(decimal0->getScaleMultiplier().value); return shift; @@ -147,7 +147,7 @@ private: getScales(const DataTypePtr &, const DataTypePtr & right_type) { Shift shift; - const DataTypeDecimal * decimal1 = checkDecimal(*right_type); + const DataTypeDecimalBase * decimal1 = checkDecimalBase(*right_type); if (decimal1) shift.a = static_cast(decimal1->getScaleMultiplier().value); return shift; diff --git a/src/Core/DecimalFunctions.h b/src/Core/DecimalFunctions.h index 52e8ebae322..b82cfd88e98 100644 --- a/src/Core/DecimalFunctions.h +++ b/src/Core/DecimalFunctions.h @@ -11,9 +11,13 @@ namespace DB { +template +class DataTypeNumber; + namespace ErrorCodes { extern const int DECIMAL_OVERFLOW; + extern const int ARGUMENT_OUT_OF_BOUND; } namespace DecimalUtils @@ -23,6 +27,7 @@ static constexpr size_t minPrecision() { return 1; } template static constexpr size_t maxPrecision() { return 0; } template <> constexpr size_t maxPrecision() { return 9; } template <> constexpr size_t maxPrecision() { return 18; } +template <> constexpr size_t maxPrecision() { return 18; } template <> constexpr size_t maxPrecision() { return 38; } template <> constexpr size_t maxPrecision() { return 76; } @@ -31,7 +36,7 @@ inline auto scaleMultiplier(UInt32 scale) { if constexpr (std::is_same_v || std::is_same_v) return common::exp10_i32(scale); - else if constexpr (std::is_same_v || std::is_same_v) + else if constexpr (std::is_same_v || std::is_same_v || std::is_same_v) return common::exp10_i64(scale); else if constexpr (std::is_same_v || std::is_same_v) return common::exp10_i128(scale); @@ -51,6 +56,30 @@ struct DecimalComponents T fractional; }; +/// Traits used for determining final Type/Precision/Scale for certain math operations on decimals. +template +struct DataTypeDecimalTrait +{ + using FieldType = T; + const UInt32 precision; + const UInt32 scale; + + DataTypeDecimalTrait(UInt32 precision_, UInt32 scale_) + : precision(precision_), + scale(scale_) + {} + + /// @returns multiplier for U to become T with correct scale + template + T scaleFactorFor(const DataTypeDecimalTrait & x, bool) const + { + if (scale < x.scale) + throw Exception("Decimal result's scale is less than argument's one", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + const UInt32 scale_delta = scale - x.scale; /// scale_delta >= 0 + return DecimalUtils::scaleMultiplier(scale_delta); + } +}; + /** Make a decimal value from whole and fractional components with given scale multiplier. * where scale_multiplier = scaleMultiplier(scale) * this is to reduce number of calls to scaleMultiplier when scale is known. @@ -211,6 +240,35 @@ To convertTo(const DecimalType & decimal, size_t scale) } } +template typename DecimalType> +inline auto binaryOpResult(const DecimalType & tx, const DecimalType & ty) +{ + UInt32 scale{}; + if constexpr (is_multiply) + scale = tx.getScale() + ty.getScale(); + else if constexpr (is_division) + scale = tx.getScale(); + else + scale = (tx.getScale() > ty.getScale() ? tx.getScale() : ty.getScale()); + + if constexpr (sizeof(T) < sizeof(U)) + return DataTypeDecimalTrait(DecimalUtils::maxPrecision(), scale); + else + return DataTypeDecimalTrait(DecimalUtils::maxPrecision(), scale); +} + +template typename DecimalType> +inline const DataTypeDecimalTrait binaryOpResult(const DecimalType & tx, const DataTypeNumber &) +{ + return DataTypeDecimalTrait(DecimalUtils::maxPrecision(), tx.getScale()); +} + +template typename DecimalType> +inline const DataTypeDecimalTrait binaryOpResult(const DataTypeNumber &, const DecimalType & ty) +{ + return DataTypeDecimalTrait(DecimalUtils::maxPrecision(), ty.getScale()); +} + } } diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 2309a36dc40..6fda3b39b3b 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -592,6 +592,10 @@ template <> bool decimalEqual(Decimal256 x, Decimal256 y, UInt32 x_scale, UInt32 template <> bool decimalLess(Decimal256 x, Decimal256 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); } template <> bool decimalLessOrEqual(Decimal256 x, Decimal256 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); } +template <> bool decimalEqual(DateTime64 x, DateTime64 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); } +template <> bool decimalLess(DateTime64 x, DateTime64 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); } +template <> bool decimalLessOrEqual(DateTime64 x, DateTime64 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); } + inline void writeText(const Null &, WriteBuffer & buf) { writeText(std::string("Null"), buf); diff --git a/src/Core/Field.h b/src/Core/Field.h index 66e4f0ac8db..433a1eaa5be 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -193,10 +193,12 @@ template <> struct NearestFieldTypeImpl { using Type = DecimalField struct NearestFieldTypeImpl { using Type = DecimalField; }; template <> struct NearestFieldTypeImpl { using Type = DecimalField; }; template <> struct NearestFieldTypeImpl { using Type = DecimalField; }; +template <> struct NearestFieldTypeImpl { using Type = DecimalField; }; template <> struct NearestFieldTypeImpl> { using Type = DecimalField; }; template <> struct NearestFieldTypeImpl> { using Type = DecimalField; }; template <> struct NearestFieldTypeImpl> { using Type = DecimalField; }; template <> struct NearestFieldTypeImpl> { using Type = DecimalField; }; +template <> struct NearestFieldTypeImpl> { using Type = DecimalField; }; template <> struct NearestFieldTypeImpl { using Type = Float64; }; template <> struct NearestFieldTypeImpl { using Type = Float64; }; template <> struct NearestFieldTypeImpl { using Type = String; }; @@ -731,6 +733,7 @@ template <> struct Field::TypeToEnum>{ static const Type template <> struct Field::TypeToEnum>{ static const Types::Which value = Types::Decimal64; }; template <> struct Field::TypeToEnum>{ static const Types::Which value = Types::Decimal128; }; template <> struct Field::TypeToEnum>{ static const Types::Which value = Types::Decimal256; }; +template <> struct Field::TypeToEnum>{ static const Types::Which value = Types::Decimal64; }; template <> struct Field::TypeToEnum{ static const Types::Which value = Types::AggregateFunctionState; }; template <> struct Field::TypeToEnum { static const Types::Which value = Types::UInt256; }; template <> struct Field::TypeToEnum { static const Types::Which value = Types::Int256; }; diff --git a/src/Core/Types.h b/src/Core/Types.h index 3157598adc0..aafe87e2f2a 100644 --- a/src/Core/Types.h +++ b/src/Core/Types.h @@ -184,29 +184,44 @@ using Decimal64 = Decimal; using Decimal128 = Decimal; using Decimal256 = Decimal; -using DateTime64 = Decimal64; +// Distinguishable type to allow function resultion/deduction based on value type, +// but also relatively easy to convert to/from Decimal64. +class DateTime64 : public Decimal64 +{ +public: + using Base = Decimal64; + using Base::Base; + + DateTime64(const Base & v) + : Base(v) + {} +}; template <> struct TypeName { static constexpr const char * get() { return "Decimal32"; } }; template <> struct TypeName { static constexpr const char * get() { return "Decimal64"; } }; template <> struct TypeName { static constexpr const char * get() { return "Decimal128"; } }; template <> struct TypeName { static constexpr const char * get() { return "Decimal256"; } }; +template <> struct TypeName { static constexpr const char * get() { return "DateTime64"; } }; template <> struct TypeId { static constexpr const TypeIndex value = TypeIndex::Decimal32; }; template <> struct TypeId { static constexpr const TypeIndex value = TypeIndex::Decimal64; }; template <> struct TypeId { static constexpr const TypeIndex value = TypeIndex::Decimal128; }; template <> struct TypeId { static constexpr const TypeIndex value = TypeIndex::Decimal256; }; +template <> struct TypeId { static constexpr const TypeIndex value = TypeIndex::DateTime64; }; template constexpr bool IsDecimalNumber = false; template <> inline constexpr bool IsDecimalNumber = true; template <> inline constexpr bool IsDecimalNumber = true; template <> inline constexpr bool IsDecimalNumber = true; template <> inline constexpr bool IsDecimalNumber = true; +template <> inline constexpr bool IsDecimalNumber = true; template struct NativeType { using Type = T; }; template <> struct NativeType { using Type = Int32; }; template <> struct NativeType { using Type = Int64; }; template <> struct NativeType { using Type = Int128; }; template <> struct NativeType { using Type = Int256; }; +template <> struct NativeType { using Type = Int64; }; template constexpr bool OverBigInt = false; template <> inline constexpr bool OverBigInt = true; diff --git a/src/Core/callOnTypeIndex.h b/src/Core/callOnTypeIndex.h index 62eec37c0cb..2ef049762f1 100644 --- a/src/Core/callOnTypeIndex.h +++ b/src/Core/callOnTypeIndex.h @@ -141,7 +141,7 @@ inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f) { case TypeIndex::Date: return callOnBasicType(type_num2, std::forward(f)); case TypeIndex::DateTime: return callOnBasicType(type_num2, std::forward(f)); - case TypeIndex::DateTime64: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::DateTime64: return callOnBasicType(type_num2, std::forward(f)); default: break; } diff --git a/src/DataTypes/DataTypeDateTime64.h b/src/DataTypes/DataTypeDateTime64.h index 8cbeedfb48c..003e83b7195 100644 --- a/src/DataTypes/DataTypeDateTime64.h +++ b/src/DataTypes/DataTypeDateTime64.h @@ -16,6 +16,7 @@ namespace DB class DataTypeDateTime64 final : public DataTypeDecimalBase, public TimezoneMixin { public: + using Base = DataTypeDecimalBase; static constexpr UInt8 default_scale = 3; static constexpr auto family_name = "DateTime64"; diff --git a/src/DataTypes/DataTypeDecimalBase.cpp b/src/DataTypes/DataTypeDecimalBase.cpp index 123f370cb91..9fb445ab00d 100644 --- a/src/DataTypes/DataTypeDecimalBase.cpp +++ b/src/DataTypes/DataTypeDecimalBase.cpp @@ -102,5 +102,6 @@ template class DataTypeDecimalBase; template class DataTypeDecimalBase; template class DataTypeDecimalBase; template class DataTypeDecimalBase; +template class DataTypeDecimalBase; } diff --git a/src/DataTypes/DataTypeDecimalBase.h b/src/DataTypes/DataTypeDecimalBase.h index c5669ab735a..4be98e4d22d 100644 --- a/src/DataTypes/DataTypeDecimalBase.h +++ b/src/DataTypes/DataTypeDecimalBase.h @@ -150,39 +150,45 @@ public: static T getScaleMultiplier(UInt32 scale); + inline DecimalUtils::DataTypeDecimalTrait getTrait() const + { + return {precision, scale}; + } + protected: const UInt32 precision; const UInt32 scale; }; +template +inline const DataTypeDecimalBase * checkDecimalBase(const IDataType & data_type) +{ + if (isColumnedAsDecimalT(data_type)) + return static_cast *>(&data_type); + + return nullptr; +} + template typename DecimalType> inline auto decimalResultType(const DecimalType & tx, const DecimalType & ty) { - UInt32 scale{}; - if constexpr (is_multiply) - scale = tx.getScale() + ty.getScale(); - else if constexpr (is_division) - scale = tx.getScale(); - else - scale = (tx.getScale() > ty.getScale() ? tx.getScale() : ty.getScale()); - - if constexpr (sizeof(T) < sizeof(U)) - return DecimalType(DecimalUtils::maxPrecision(), scale); - else - return DecimalType(DecimalUtils::maxPrecision(), scale); + const auto result_trait = DecimalUtils::binaryOpResult(tx, ty); + return DecimalType(result_trait.precision, result_trait.scale); } -template typename DecimalType> -inline const DecimalType decimalResultType(const DecimalType & tx, const DataTypeNumber &) +template typename DecimalType> +inline const DecimalType decimalResultType(const DecimalType & tx, const DataTypeNumber & ty) { - return DecimalType(DecimalUtils::maxPrecision(), tx.getScale()); + const auto result_trait = DecimalUtils::binaryOpResult(tx, ty); + return DecimalType(result_trait.precision, result_trait.scale); } -template typename DecimalType> -inline const DecimalType decimalResultType(const DataTypeNumber &, const DecimalType & ty) +template typename DecimalType> +inline const DecimalType decimalResultType(const DataTypeNumber & tx, const DecimalType & ty) { - return DecimalType(DecimalUtils::maxPrecision(), ty.getScale()); + const auto result_trait = DecimalUtils::binaryOpResult(tx, ty); + return DecimalType(result_trait.precision, result_trait.scale); } template