diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index eb807d2a4abaa0..ad26c7f3eb6307 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -642,9 +642,13 @@ Status Compaction::do_inverted_index_compaction() { // format: rowsetId_segmentId std::vector> inverted_index_file_writers( dest_segment_num); - for (int i = 0; i < dest_segment_num; ++i) { + + // Some columns have already been indexed + // key: seg_id, value: inverted index file size + std::unordered_map compacted_idx_file_size; + for (int seg_id = 0; seg_id < dest_segment_num; ++seg_id) { std::string index_path_prefix { - InvertedIndexDescriptor::get_index_file_path_prefix(ctx.segment_path(i))}; + InvertedIndexDescriptor::get_index_file_path_prefix(ctx.segment_path(seg_id))}; auto inverted_index_file_reader = std::make_unique( ctx.fs(), index_path_prefix, _cur_tablet_schema->get_inverted_index_storage_format()); @@ -654,16 +658,31 @@ Status Compaction::do_inverted_index_compaction() { if (st.ok()) { auto index_not_need_to_compact = DORIS_TRY(inverted_index_file_reader->get_all_directories()); + // V1: each index is a separate file + // V2: all indexes are in a single file + if (_cur_tablet_schema->get_inverted_index_storage_format() != + doris::InvertedIndexStorageFormatPB::V1) { + int64_t fsize = 0; + st = ctx.fs()->file_size( + InvertedIndexDescriptor::get_index_file_path_v2(index_path_prefix), &fsize); + if (!st.ok()) { + LOG(ERROR) << "file size error in index compaction, error:" << st.msg(); + return st; + } + compacted_idx_file_size[seg_id] = fsize; + } auto inverted_index_file_writer = std::make_unique( - ctx.fs(), index_path_prefix, ctx.rowset_id.to_string(), i, + ctx.fs(), index_path_prefix, ctx.rowset_id.to_string(), seg_id, _cur_tablet_schema->get_inverted_index_storage_format()); RETURN_IF_ERROR(inverted_index_file_writer->initialize(index_not_need_to_compact)); - inverted_index_file_writers[i] = std::move(inverted_index_file_writer); + inverted_index_file_writers[seg_id] = std::move(inverted_index_file_writer); } else if (st.is()) { auto inverted_index_file_writer = std::make_unique( - ctx.fs(), index_path_prefix, ctx.rowset_id.to_string(), i, + ctx.fs(), index_path_prefix, ctx.rowset_id.to_string(), seg_id, _cur_tablet_schema->get_inverted_index_storage_format()); - inverted_index_file_writers[i] = std::move(inverted_index_file_writer); + inverted_index_file_writers[seg_id] = std::move(inverted_index_file_writer); + // no index file + compacted_idx_file_size[seg_id] = 0; } else { LOG(ERROR) << "inverted_index_file_reader init failed in index compaction, error:" << st; @@ -744,11 +763,13 @@ Status Compaction::do_inverted_index_compaction() { } uint64_t inverted_index_file_size = 0; - for (auto& inverted_index_file_writer : inverted_index_file_writers) { + for (int seg_id = 0; seg_id < dest_segment_num; ++seg_id) { + auto inverted_index_file_writer = inverted_index_file_writers[seg_id].get(); if (Status st = inverted_index_file_writer->close(); !st.ok()) { status = Status::Error(st.msg()); } else { inverted_index_file_size += inverted_index_file_writer->get_index_file_size(); + inverted_index_file_size -= compacted_idx_file_size[seg_id]; } } // check index compaction status. If status is not ok, we should return error and end this compaction round. diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index ea5b5ae01b6bb9..6a66ae6fc146b3 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -1925,7 +1925,8 @@ Status SegmentIterator::_read_columns(const std::vector& column_ids, } Status SegmentIterator::_init_current_block( - vectorized::Block* block, std::vector& current_columns) { + vectorized::Block* block, std::vector& current_columns, + uint32_t nrows_read_limit) { block->clear_column_data(_schema->num_column_ids()); for (size_t i = 0; i < _schema->num_column_ids(); i++) { @@ -1945,7 +1946,7 @@ Status SegmentIterator::_init_current_block( column_desc->path() == nullptr ? "" : column_desc->path()->get_path()); // TODO reuse current_columns[cid] = file_column_type->create_column(); - current_columns[cid]->reserve(_opts.block_row_max); + current_columns[cid]->reserve(nrows_read_limit); } else { // the column in block must clear() here to insert new data if (_is_pred_column[cid] || @@ -1964,7 +1965,7 @@ Status SegmentIterator::_init_current_block( } else if (column_desc->type() == FieldType::OLAP_FIELD_TYPE_DATETIME) { current_columns[cid]->set_datetime_type(); } - current_columns[cid]->reserve(_opts.block_row_max); + current_columns[cid]->reserve(nrows_read_limit); } } } @@ -2378,14 +2379,16 @@ Status SegmentIterator::_next_batch_internal(vectorized::Block* block) { } } } - RETURN_IF_ERROR(_init_current_block(block, _current_return_columns)); - _converted_column_ids.assign(_schema->columns().size(), 0); - _current_batch_rows_read = 0; uint32_t nrows_read_limit = _opts.block_row_max; if (_can_opt_topn_reads()) { nrows_read_limit = std::min(static_cast(_opts.topn_limit), nrows_read_limit); } + + RETURN_IF_ERROR(_init_current_block(block, _current_return_columns, nrows_read_limit)); + _converted_column_ids.assign(_schema->columns().size(), 0); + + _current_batch_rows_read = 0; RETURN_IF_ERROR(_read_columns_by_index( nrows_read_limit, _current_batch_rows_read, _lazy_materialization_read || _opts.record_rowids || _is_need_expr_eval)); diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h b/be/src/olap/rowset/segment_v2/segment_iterator.h index ae865ddc456950..cb904f21c6ac20 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.h +++ b/be/src/olap/rowset/segment_v2/segment_iterator.h @@ -221,7 +221,8 @@ class SegmentIterator : public RowwiseIterator { bool set_block_rowid); void _replace_version_col(size_t num_rows); Status _init_current_block(vectorized::Block* block, - std::vector& non_pred_vector); + std::vector& non_pred_vector, + uint32_t nrows_read_limit); uint16_t _evaluate_vectorization_predicate(uint16_t* sel_rowid_idx, uint16_t selected_size); uint16_t _evaluate_short_circuit_predicate(uint16_t* sel_rowid_idx, uint16_t selected_size); void _output_non_pred_columns(vectorized::Block* block); diff --git a/be/src/pipeline/exec/analytic_source_operator.cpp b/be/src/pipeline/exec/analytic_source_operator.cpp index 93e87cbce5d822..a036481d727789 100644 --- a/be/src/pipeline/exec/analytic_source_operator.cpp +++ b/be/src/pipeline/exec/analytic_source_operator.cpp @@ -559,11 +559,6 @@ Status AnalyticLocalState::close(RuntimeState* state) { std::vector tmp_result_window_columns; _result_window_columns.swap(tmp_result_window_columns); - // Some kinds of source operators has a 1-1 relationship with a sink operator (such as AnalyticOperator). - // We must ensure AnalyticSinkOperator will not be blocked if AnalyticSourceOperator already closed. - if (_shared_state && _shared_state->sink_deps.size() == 1) { - _shared_state->sink_deps.front()->set_always_ready(); - } return PipelineXLocalState::close(state); } diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 7887628b7fa476..3a55fdd9b8698e 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -567,7 +567,11 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block* } else if (!local_state._should_build_hash_table) { DCHECK(_shared_hashtable_controller != nullptr); DCHECK(_shared_hash_table_context != nullptr); - CHECK(_shared_hash_table_context->signaled); + // the instance which is not build hash table, it's should wait the signal of hash table build finished. + // but if it's running and signaled == false, maybe the source operator have closed caused by some short circuit, + if (!_shared_hash_table_context->signaled) { + return Status::Error("source have closed"); + } if (!_shared_hash_table_context->status.ok()) { return _shared_hash_table_context->status; diff --git a/be/src/pipeline/exec/multi_cast_data_streamer.cpp b/be/src/pipeline/exec/multi_cast_data_streamer.cpp index deebf7d11bb2e2..d44cf3974a6275 100644 --- a/be/src/pipeline/exec/multi_cast_data_streamer.cpp +++ b/be/src/pipeline/exec/multi_cast_data_streamer.cpp @@ -23,63 +23,97 @@ namespace doris::pipeline { -MultiCastBlock::MultiCastBlock(vectorized::Block* block, int used_count, size_t mem_size) - : _used_count(used_count), _mem_size(mem_size) { +MultiCastBlock::MultiCastBlock(vectorized::Block* block, int used_count, int un_finish_copy, + size_t mem_size) + : _used_count(used_count), _un_finish_copy(un_finish_copy), _mem_size(mem_size) { _block = vectorized::Block::create_unique(block->get_columns_with_type_and_name()); block->clear(); } Status MultiCastDataStreamer::pull(int sender_idx, doris::vectorized::Block* block, bool* eos) { - std::lock_guard l(_mutex); - auto& pos_to_pull = _sender_pos_to_read[sender_idx]; - if (pos_to_pull != _multi_cast_blocks.end()) { - if (pos_to_pull->_used_count == 1) { - DCHECK(pos_to_pull == _multi_cast_blocks.begin()); - pos_to_pull->_block->swap(*block); - - _cumulative_mem_size -= pos_to_pull->_mem_size; - pos_to_pull++; - _multi_cast_blocks.pop_front(); - } else { - pos_to_pull->_block->create_same_struct_block(0)->swap(*block); - RETURN_IF_ERROR(vectorized::MutableBlock(block).merge(*pos_to_pull->_block)); - pos_to_pull->_used_count--; - pos_to_pull++; + int* un_finish_copy = nullptr; + int use_count = 0; + { + std::lock_guard l(_mutex); + auto& pos_to_pull = _sender_pos_to_read[sender_idx]; + const auto end = _multi_cast_blocks.end(); + DCHECK(pos_to_pull != end); + + *block = *pos_to_pull->_block; + + _cumulative_mem_size -= pos_to_pull->_mem_size; + + pos_to_pull->_used_count--; + use_count = pos_to_pull->_used_count; + un_finish_copy = &pos_to_pull->_un_finish_copy; + + pos_to_pull++; + + if (pos_to_pull == end) { + _block_reading(sender_idx); } + + *eos = _eos and pos_to_pull == end; } - *eos = _eos and pos_to_pull == _multi_cast_blocks.end(); - if (pos_to_pull == _multi_cast_blocks.end()) { - _block_reading(sender_idx); + + if (use_count == 0) { + // will clear _multi_cast_blocks + _wait_copy_block(block, *un_finish_copy); + } else { + _copy_block(block, *un_finish_copy); } + return Status::OK(); } +void MultiCastDataStreamer::_copy_block(vectorized::Block* block, int& un_finish_copy) { + const auto rows = block->rows(); + for (int i = 0; i < block->columns(); ++i) { + block->get_by_position(i).column = block->get_by_position(i).column->clone_resized(rows); + } + + std::unique_lock l(_mutex); + un_finish_copy--; + if (un_finish_copy == 0) { + l.unlock(); + _cv.notify_one(); + } +} + +void MultiCastDataStreamer::_wait_copy_block(vectorized::Block* block, int& un_finish_copy) { + std::unique_lock l(_mutex); + _cv.wait(l, [&]() { return un_finish_copy == 0; }); + _multi_cast_blocks.pop_front(); +} + Status MultiCastDataStreamer::push(RuntimeState* state, doris::vectorized::Block* block, bool eos) { auto rows = block->rows(); COUNTER_UPDATE(_process_rows, rows); - auto block_mem_size = block->allocated_bytes(); - std::lock_guard l(_mutex); - int need_process_count = _cast_sender_count - _closed_sender_count; - if (need_process_count == 0) { - return Status::EndOfFile("All data streamer is EOF"); - } - // TODO: if the [queue back block rows + block->rows()] < batch_size, better - // do merge block. but need check the need_process_count and used_count whether - // equal - _multi_cast_blocks.emplace_back(block, need_process_count, block_mem_size); + const auto block_mem_size = block->allocated_bytes(); _cumulative_mem_size += block_mem_size; COUNTER_SET(_peak_mem_usage, std::max(_cumulative_mem_size, _peak_mem_usage->value())); - auto end = _multi_cast_blocks.end(); - end--; - for (int i = 0; i < _sender_pos_to_read.size(); ++i) { - if (_sender_pos_to_read[i] == _multi_cast_blocks.end()) { - _sender_pos_to_read[i] = end; - _set_ready_for_read(i); + { + std::lock_guard l(_mutex); + _multi_cast_blocks.emplace_back(block, _cast_sender_count, _cast_sender_count - 1, + block_mem_size); + // last elem + auto end = std::prev(_multi_cast_blocks.end()); + for (int i = 0; i < _sender_pos_to_read.size(); ++i) { + if (_sender_pos_to_read[i] == _multi_cast_blocks.end()) { + _sender_pos_to_read[i] = end; + _set_ready_for_read(i); + } + } + _eos = eos; + } + + if (_eos) { + for (auto* read_dep : _dependencies) { + read_dep->set_always_ready(); } } - _eos = eos; return Status::OK(); } @@ -92,13 +126,6 @@ void MultiCastDataStreamer::_set_ready_for_read(int sender_idx) { dep->set_ready(); } -void MultiCastDataStreamer::_set_ready_for_read() { - for (auto* dep : _dependencies) { - DCHECK(dep); - dep->set_ready(); - } -} - void MultiCastDataStreamer::_block_reading(int sender_idx) { if (_dependencies.empty()) { return; diff --git a/be/src/pipeline/exec/multi_cast_data_streamer.h b/be/src/pipeline/exec/multi_cast_data_streamer.h index 2112ebaaf205b1..07e64016363f65 100644 --- a/be/src/pipeline/exec/multi_cast_data_streamer.h +++ b/be/src/pipeline/exec/multi_cast_data_streamer.h @@ -23,10 +23,11 @@ namespace doris::pipeline { class Dependency; struct MultiCastBlock { - MultiCastBlock(vectorized::Block* block, int used_count, size_t mem_size); + MultiCastBlock(vectorized::Block* block, int used_count, int need_copy, size_t mem_size); std::unique_ptr _block; int _used_count; + int _un_finish_copy; size_t _mem_size; }; @@ -58,12 +59,6 @@ class MultiCastDataStreamer { RuntimeProfile* profile() { return _profile; } - void set_eos() { - std::lock_guard l(_mutex); - _eos = true; - _set_ready_for_read(); - } - void set_dep_by_sender_idx(int sender_idx, Dependency* dep) { _dependencies[sender_idx] = dep; _block_reading(sender_idx); @@ -71,17 +66,20 @@ class MultiCastDataStreamer { private: void _set_ready_for_read(int sender_idx); - void _set_ready_for_read(); void _block_reading(int sender_idx); + void _copy_block(vectorized::Block* block, int& un_finish_copy); + + void _wait_copy_block(vectorized::Block* block, int& un_finish_copy); + const RowDescriptor& _row_desc; RuntimeProfile* _profile = nullptr; std::list _multi_cast_blocks; std::vector::iterator> _sender_pos_to_read; + std::condition_variable _cv; std::mutex _mutex; bool _eos = false; int _cast_sender_count = 0; - int _closed_sender_count = 0; int64_t _cumulative_mem_size = 0; RuntimeProfile::Counter* _process_rows = nullptr; diff --git a/be/src/pipeline/exec/operator.cpp b/be/src/pipeline/exec/operator.cpp index eba380f4386aa3..ba3602a91cb1ef 100644 --- a/be/src/pipeline/exec/operator.cpp +++ b/be/src/pipeline/exec/operator.cpp @@ -511,6 +511,11 @@ Status PipelineXLocalState::close(RuntimeState* state) { _peak_memory_usage_counter->set(_mem_tracker->peak_consumption()); } _closed = true; + // Some kinds of source operators has a 1-1 relationship with a sink operator (such as AnalyticOperator). + // We must ensure AnalyticSinkOperator will not be blocked if AnalyticSourceOperator already closed. + if (_shared_state && _shared_state->sink_deps.size() == 1) { + _shared_state->sink_deps.front()->set_always_ready(); + } return Status::OK(); } diff --git a/be/src/util/bitmap_value.h b/be/src/util/bitmap_value.h index ac5e826e1fee66..2d15ac99611274 100644 --- a/be/src/util/bitmap_value.h +++ b/be/src/util/bitmap_value.h @@ -1252,8 +1252,7 @@ class BitmapValue { std::vector bitmaps; std::vector single_values; std::vector*> sets; - for (int i = 0; i < values.size(); ++i) { - auto* value = values[i]; + for (const auto* value : values) { switch (value->_type) { case EMPTY: break; @@ -1280,7 +1279,9 @@ class BitmapValue { _bitmap->add(_sv); break; case BITMAP: - *_bitmap |= detail::Roaring64Map::fastunion(bitmaps.size(), bitmaps.data()); + for (const auto* bitmap : bitmaps) { + *_bitmap |= *bitmap; + } break; case SET: { *_bitmap = detail::Roaring64Map::fastunion(bitmaps.size(), bitmaps.data()); @@ -1315,6 +1316,7 @@ class BitmapValue { _bitmap->add(v); } _type = BITMAP; + _set.clear(); break; case SET: { break; diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp index d788c8dc518a79..64a4adfa6beeda 100644 --- a/be/src/vec/exprs/vexpr.cpp +++ b/be/src/vec/exprs/vexpr.cpp @@ -627,7 +627,7 @@ std::string VExpr::gen_predicate_result_sign(Block& block, const ColumnNumbers& std::string column_name = block.get_by_position(arguments[0]).name; pred_result_sign += BeConsts::BLOCK_TEMP_COLUMN_PREFIX + column_name + "_" + function_name + "_"; - if (function_name == "in") { + if (function_name == "in" || function_name == "not_in") { // Generating 'result_sign' from 'inlist' requires sorting the values. std::set values; for (size_t i = 1; i < arguments.size(); i++) { diff --git a/be/src/vec/functions/function.h b/be/src/vec/functions/function.h index f3af3870c2eb61..36558c11df0207 100644 --- a/be/src/vec/functions/function.h +++ b/be/src/vec/functions/function.h @@ -521,7 +521,7 @@ class DefaultFunction final : public IFunctionBase { auto function_name = function->get_name(); return function_name == "eq" || function_name == "ne" || function_name == "lt" || function_name == "gt" || function_name == "le" || function_name == "ge" || - function_name == "in"; + function_name == "in" || function_name == "not_in"; } Status eval_inverted_index(FunctionContext* context, diff --git a/docker/thirdparties/custom_settings.env b/docker/thirdparties/custom_settings.env index ac178c81726f67..17db5563d93b14 100644 --- a/docker/thirdparties/custom_settings.env +++ b/docker/thirdparties/custom_settings.env @@ -21,5 +21,5 @@ # eg: CONTAINER_UID="doris-jack-" # NOTICE: change this uid will modify the file in docker-compose. CONTAINER_UID="doris--" -s3Endpoint="cos.ap-hongkong.myqcloud.com" -s3BucketName="doris-build-hk-1308700295" +s3Endpoint="oss-cn-hongkong.aliyuncs.com" +s3BucketName="qa-build-hk" diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java index ceb6b037a39fd8..4041fa4873b5be 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java @@ -2447,7 +2447,6 @@ public int hashCode() { int result = super.hashCode(); result = 31 * result + Objects.hashCode(opcode); result = 31 * result + Objects.hashCode(fnName); - result = 31 * result + Objects.hashCode(fnParams); return result; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java index f3e106743d4ae9..727192b4e572aa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java @@ -714,6 +714,7 @@ private void getDeleteBitmapUpdateLock(Map> tableToParttions, lo } StopWatch stopWatch = new StopWatch(); stopWatch.start(); + int totalRetryTime = 0; for (Map.Entry> entry : tableToParttions.entrySet()) { GetDeleteBitmapUpdateLockRequest.Builder builder = GetDeleteBitmapUpdateLockRequest.newBuilder(); builder.setTableId(entry.getKey()) @@ -790,10 +791,15 @@ private void getDeleteBitmapUpdateLock(Map> tableToParttions, lo cumulativePoints.put(tabletId, respCumulativePoints.get(i)); } } + totalRetryTime += retryTime; } stopWatch.stop(); - LOG.info("get delete bitmap lock successfully. txns: {}. time cost: {} ms.", - transactionId, stopWatch.getTime()); + if (totalRetryTime > 0 || stopWatch.getTime() > 20) { + LOG.info( + "get delete bitmap lock successfully. txns: {}. totalRetryTime: {}. " + + "partitionSize: {}. time cost: {} ms.", + transactionId, totalRetryTime, tableToParttions.size(), stopWatch.getTime()); + } } private void sendCalcDeleteBitmaptask(long dbId, long transactionId, diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java index b643176aaf160a..4ba5a2ebd96281 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java @@ -600,6 +600,11 @@ public void replayInitCatalog(InitCatalogLog log) { // Should not return null. // Because replyInitCatalog can only be called when `use_meta_cache` is false. // And if `use_meta_cache` is false, getDbForReplay() will not return null + if (!db.isPresent()) { + LOG.warn("met invalid db id {} in replayInitCatalog, catalog: {}, ignore it to skip bug.", + log.getRefreshDbIds().get(i), name); + continue; + } Preconditions.checkNotNull(db.get()); tmpDbNameToId.put(db.get().getFullName(), db.get().getId()); tmpIdToDb.put(db.get().getId(), db.get()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java index 1106df5f7c37f5..d7627d698d6134 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java @@ -43,7 +43,6 @@ import org.apache.doris.nereids.trees.expressions.IsNull; import org.apache.doris.nereids.trees.expressions.LessThan; import org.apache.doris.nereids.trees.expressions.LessThanEqual; -import org.apache.doris.nereids.trees.expressions.Like; import org.apache.doris.nereids.trees.expressions.Not; import org.apache.doris.nereids.trees.expressions.NullSafeEqual; import org.apache.doris.nereids.trees.expressions.Or; @@ -403,11 +402,6 @@ public Expression visitOr(Or or, ExpressionRewriteContext context) { } } - @Override - public Expression visitLike(Like like, ExpressionRewriteContext context) { - return like; - } - @Override public Expression visitCast(Cast cast, ExpressionRewriteContext context) { cast = rewriteChildren(cast, context); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/MasterCatalogExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/MasterCatalogExecutor.java index bb0f5a88e14fdc..0fd074451087e3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/MasterCatalogExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/MasterCatalogExecutor.java @@ -64,11 +64,20 @@ public void forward(long catalogId, long dbId) throws Exception { boolean isReturnToPool = false; try { TInitExternalCtlMetaResult result = client.initExternalCtlMeta(request); - Env.getCurrentEnv().getJournalObservable().waitOn(result.maxJournalId, waitTimeoutMs); if (!result.getStatus().equalsIgnoreCase(STATUS_OK)) { throw new UserException(result.getStatus()); + } else { + // DO NOT wait on journal replayed, this may cause deadlock. + // 1. hold table read lock + // 2. wait on journal replayed + // 3. previous journal (eg, txn journal) replayed need to hold table write lock + // 4. deadlock + // But no waiting on journal replayed may cause some request on non-master FE failed for some time. + // There is no good solution for this. + // In feature version, this whole process is refactored, so we temporarily remove this waiting. + // Env.getCurrentEnv().getJournalObservable().waitOn(result.maxJournalId, timeoutMs); + isReturnToPool = true; } - isReturnToPool = true; } catch (Exception e) { LOG.warn("Failed to finish forward init operation, please try again. ", e); throw e; diff --git a/regression-test/data/inverted_index_p0/test_index_rqg_bug3.out b/regression-test/data/inverted_index_p0/test_index_rqg_bug3.out new file mode 100644 index 00000000000000..cd01bedc787d41 --- /dev/null +++ b/regression-test/data/inverted_index_p0/test_index_rqg_bug3.out @@ -0,0 +1,43 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_bug_1 -- +-10 2023-12-11 +-10 2023-12-12 +-10 2023-12-13 +-10 2023-12-15 +-10 2023-12-15 +-10 2023-12-19 +-10 2023-12-19 +-10 2024-01-17 +-10 2024-02-18 +-10 2024-02-18 +-10 2025-02-18 +-10 2026-01-18 +-10 2026-02-18 +-4 2023-12-10 +-4 2023-12-11 +-4 2023-12-16 +-4 2024-01-31 +0 2024-01-19 +1 2023-12-16 +1 2024-01-09 +2 2023-12-10 +2 2023-12-11 +2 2024-01-08 +2 2024-01-31 +3 2023-12-20 +3 2024-01-19 +3 2025-06-18 +3 2026-02-18 +3 2027-01-16 +4 2023-12-12 +4 2023-12-12 +4 2024-01-08 +5 2023-12-16 +6 2024-02-18 +7 2023-12-17 +7 2023-12-20 +7 2027-01-09 +8 2025-02-18 +9 2024-02-18 +9 2024-02-18 + diff --git a/regression-test/data/inverted_index_p1/documents-1000.json b/regression-test/data/inverted_index_p1/documents-1000.json new file mode 100644 index 00000000000000..a1b0aa8342edc8 --- /dev/null +++ b/regression-test/data/inverted_index_p1/documents-1000.json @@ -0,0 +1,1000 @@ +{"@timestamp": 893964617, "clientip":"40.135.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964653, "clientip":"232.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964672, "clientip":"26.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964679, "clientip":"247.37.0.0", "request": "GET /french/splash_inet.html HTTP/1.0", "status": 200, "size": 3781} +{"@timestamp": 893964682, "clientip":"247.37.0.0", "request": "GET /images/hm_nbg.jpg HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893964687, "clientip":"252.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964689, "clientip":"247.37.0.0", "request": "GET /images/hm_brdl.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893964689, "clientip":"247.37.0.0", "request": "GET /images/hm_arw.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893964692, "clientip":"247.37.0.0", "request": "GET /images/nav_bg_top.gif HTTP/1.0", "status": 200, "size": 929} +{"@timestamp": 893964703, "clientip":"247.37.0.0", "request": "GET /french/images/nav_venue_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893964704, "clientip":"247.37.0.0", "request": "GET /french/images/nav_hosts_off.gif HTTP/1.0", "status": 200, "size": 1139} +{"@timestamp": 893964712, "clientip":"2.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964712, "clientip":"247.37.0.0", "request": "GET /french/tickets/body.html HTTP/1.0", "status": 200, "size": 3029} +{"@timestamp": 893964726, "clientip":"120.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964736, "clientip":"247.37.0.0", "request": "GET /french/tickets/images/ticket_hm_header.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893964736, "clientip":"247.37.0.0", "request": "GET /french/tickets/images/ticket_hm_nav.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893964737, "clientip":"247.37.0.0", "request": "GET /images/arw_lk.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893964750, "clientip":"247.37.0.0", "request": "GET /french/tickets/tck_0804.htm HTTP/1.0", "status": 200, "size": 14521} +{"@timestamp": 893964753, "clientip":"247.37.0.0", "request": "GET /french/tickets/images/ticket_quest_bg2.jpg HTTP/1.0", "status": 200, "size": 11324} +{"@timestamp": 893964755, "clientip":"126.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964758, "clientip":"247.37.0.0", "request": "GET /french/tickets/images/ticket_header.gif HTTP/1.0", "status": 200, "size": 671} +{"@timestamp": 893964758, "clientip":"247.37.0.0", "request": "GET /french/tickets/images/ticket_bu_abroad2.gif HTTP/1.0", "status": 200, "size": 1512} +{"@timestamp": 893964758, "clientip":"247.37.0.0", "request": "GET /french/tickets/images/ticket_bu_infrance2.gif HTTP/1.0", "status": 200, "size": 1136} +{"@timestamp": 893964758, "clientip":"247.37.0.0", "request": "GET /french/tickets/images/hm_f98_top.gif HTTP/1.0", "status": 200, "size": 1647} +{"@timestamp": 893964758, "clientip":"247.37.0.0", "request": "GET /french/tickets/images/ticket_bu_quest2.gif HTTP/1.0", "status": 200, "size": 1271} +{"@timestamp": 893964772, "clientip":"247.37.0.0", "request": "GET /french/news/3004bres.htm HTTP/1.0", "status": 200, "size": 5933} +{"@timestamp": 893964778, "clientip":"247.37.0.0", "request": "GET /french/images/hm_f98_top.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893964779, "clientip":"247.37.0.0", "request": "GET /images/bord_d.gif HTTP/1.0", "status": 200, "size": 231} +{"@timestamp": 893964779, "clientip":"247.37.0.0", "request": "GET /images/bord_g.gif HTTP/1.0", "status": 200, "size": 231} +{"@timestamp": 893964785, "clientip":"13.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964789, "clientip":"138.2.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964800, "clientip":"28.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964809, "clientip":"31.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964812, "clientip":"55.0.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 200, "size": 985} +{"@timestamp": 893964815, "clientip":"29.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964825, "clientip":"92.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964841, "clientip":"134.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964844, "clientip":"55.0.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 200, "size": 985} +{"@timestamp": 893964864, "clientip":"137.2.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964932, "clientip":"167.2.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964936, "clientip":"174.2.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964961, "clientip":"97.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893964973, "clientip":"174.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965016, "clientip":"121.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965036, "clientip":"2.2.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965050, "clientip":"91.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965059, "clientip":"47.2.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965070, "clientip":"142.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965084, "clientip":"104.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965106, "clientip":"40.135.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965115, "clientip":"109.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965123, "clientip":"184.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965139, "clientip":"55.0.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 200, "size": 985} +{"@timestamp": 893965149, "clientip":"56.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965150, "clientip":"161.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965150, "clientip":"237.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965154, "clientip":"41.135.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965162, "clientip":"131.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965172, "clientip":"55.0.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 200, "size": 985} +{"@timestamp": 893965182, "clientip":"42.135.0.0", "request": "GET /fth.htm HTTP/1.1", "status": 200, "size": 190} +{"@timestamp": 893965193, "clientip":"180.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965198, "clientip":"176.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965200, "clientip":"192.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965278, "clientip":"4.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965293, "clientip":"76.6.0.0", "request": "GET /images/teams_hm_bg.jpg HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965300, "clientip":"235.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965310, "clientip":"183.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965318, "clientip":"248.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965322, "clientip":"249.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965352, "clientip":"218.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965382, "clientip":"128.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965405, "clientip":"139.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965412, "clientip":"55.0.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 200, "size": 985} +{"@timestamp": 893965424, "clientip":"249.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965449, "clientip":"55.0.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 200, "size": 985} +{"@timestamp": 893965464, "clientip":"221.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965467, "clientip":"175.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965479, "clientip":"43.135.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965504, "clientip":"201.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965518, "clientip":"17.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965550, "clientip":"138.2.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965554, "clientip":"232.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965578, "clientip":"76.6.0.0", "request": "GET /images/nantes.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965585, "clientip":"252.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965598, "clientip":"120.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965616, "clientip":"2.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965622, "clientip":"44.135.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965626, "clientip":"44.135.0.0", "request": "GET /french/nav_inet.html HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965626, "clientip":"44.135.0.0", "request": "GET /french/splash_inet.html HTTP/1.0", "status": 200, "size": 3781} +{"@timestamp": 893965626, "clientip":"44.135.0.0", "request": "GET /french/nav_top_inet.html HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965627, "clientip":"44.135.0.0", "request": "GET /images/nav_bg_top.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965628, "clientip":"44.135.0.0", "request": "GET /images/hm_nbg.jpg HTTP/1.0", "status": 200, "size": 33665} +{"@timestamp": 893965631, "clientip":"44.135.0.0", "request": "GET /images/logo_cfo.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965632, "clientip":"44.135.0.0", "request": "GET /french/ProScroll.class HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965634, "clientip":"44.135.0.0", "request": "GET /images/nav_bg_bottom.jpg HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965634, "clientip":"44.135.0.0", "request": "GET /french/images/nav_news_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965634, "clientip":"44.135.0.0", "request": "GET /french/images/nav_comp_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965638, "clientip":"44.135.0.0", "request": "GET /french/images/nav_venue_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965648, "clientip":"126.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965649, "clientip":"44.135.0.0", "request": "GET /french/images/nav_tickets_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965651, "clientip":"44.135.0.0", "request": "GET /french/news/3004bres.htm HTTP/1.0", "status": 200, "size": 5933} +{"@timestamp": 893965651, "clientip":"44.135.0.0", "request": "GET /french/images/nav_field_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965652, "clientip":"44.135.0.0", "request": "GET /french/images/nav_history_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965655, "clientip":"44.135.0.0", "request": "GET /images/backnews.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965658, "clientip":"44.135.0.0", "request": "GET /french/images/nav_team_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965658, "clientip":"44.135.0.0", "request": "GET /french/images/nav_store_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965658, "clientip":"44.135.0.0", "request": "GET /french/images/nav_home_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965659, "clientip":"44.135.0.0", "request": "GET /french/images/nav_sitemap_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965659, "clientip":"44.135.0.0", "request": "GET /french/images/fpnewstop.gif HTTP/1.0", "status": 200, "size": 1317} +{"@timestamp": 893965660, "clientip":"44.135.0.0", "request": "GET /french/images/hm_f98_top.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965661, "clientip":"44.135.0.0", "request": "GET /french/images/news_btn_letter_off.gif HTTP/1.0", "status": 200, "size": 871} +{"@timestamp": 893965661, "clientip":"44.135.0.0", "request": "GET /french/images/news_btn_press_off.gif HTTP/1.0", "status": 200, "size": 1795} +{"@timestamp": 893965662, "clientip":"44.135.0.0", "request": "GET /french/images/news_btn_kits_off.gif HTTP/1.0", "status": 200, "size": 965} +{"@timestamp": 893965662, "clientip":"44.135.0.0", "request": "GET /french/images/nav_hosts_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965662, "clientip":"44.135.0.0", "request": "GET /french/images/nav_logo_sponsors.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965663, "clientip":"44.135.0.0", "request": "GET /images/bord_d.gif HTTP/1.0", "status": 200, "size": 231} +{"@timestamp": 893965664, "clientip":"44.135.0.0", "request": "GET /images/bord_g.gif HTTP/1.0", "status": 200, "size": 231} +{"@timestamp": 893965668, "clientip":"26.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965684, "clientip":"13.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965700, "clientip":"28.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965700, "clientip":"44.135.0.0", "request": "GET /french/news/newsprr.htm HTTP/1.0", "status": 200, "size": 28486} +{"@timestamp": 893965705, "clientip":"44.135.0.0", "request": "GET /french/images/space.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965706, "clientip":"44.135.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965706, "clientip":"44.135.0.0", "request": "GET /french/images/news_btn_press_on.gif HTTP/1.0", "status": 200, "size": 1757} +{"@timestamp": 893965706, "clientip":"44.135.0.0", "request": "GET /images/hm_f98_top.gif HTTP/1.0", "status": 200, "size": 915} +{"@timestamp": 893965708, "clientip":"44.135.0.0", "request": "GET /french/images/news_hd_press.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965708, "clientip":"44.135.0.0", "request": "GET /images/news_arrow.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893965712, "clientip":"55.0.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 200, "size": 985} +{"@timestamp": 893965724, "clientip":"31.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965727, "clientip":"92.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965727, "clientip":"134.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965731, "clientip":"29.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965738, "clientip":"55.0.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 200, "size": 985} +{"@timestamp": 893965753, "clientip":"237.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965796, "clientip":"42.135.0.0", "request": "GET /fth.htm HTTP/1.1", "status": 200, "size": 190} +{"@timestamp": 893965849, "clientip":"174.2.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965856, "clientip":"167.2.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965865, "clientip":"137.2.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965882, "clientip":"40.135.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965883, "clientip":"97.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965896, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/argentina78.html HTTP/1.0", "status": 200, "size": 20916} +{"@timestamp": 893965899, "clientip":"45.135.0.0", "request": "GET /french/history/images/history_hm_header.gif HTTP/1.0", "status": 200, "size": 1034} +{"@timestamp": 893965899, "clientip":"45.135.0.0", "request": "GET /french/history/images/football.GIF HTTP/1.0", "status": 200, "size": 1452} +{"@timestamp": 893965899, "clientip":"45.135.0.0", "request": "GET /french/history/images/france98b.GIF HTTP/1.0", "status": 200, "size": 915} +{"@timestamp": 893965901, "clientip":"45.135.0.0", "request": "GET /french/history/images/infrance.GIF HTTP/1.0", "status": 200, "size": 1017} +{"@timestamp": 893965901, "clientip":"45.135.0.0", "request": "GET /french/history/images/reading.GIF HTTP/1.0", "status": 200, "size": 1094} +{"@timestamp": 893965901, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_30_off.gif HTTP/1.0", "status": 200, "size": 406} +{"@timestamp": 893965902, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_38_off.gif HTTP/1.0", "status": 200, "size": 436} +{"@timestamp": 893965903, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_54_off.gif HTTP/1.0", "status": 200, "size": 403} +{"@timestamp": 893965904, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_62_off.gif HTTP/1.0", "status": 200, "size": 437} +{"@timestamp": 893965905, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_70_off.gif HTTP/1.0", "status": 200, "size": 409} +{"@timestamp": 893965905, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_78_on.gif HTTP/1.0", "status": 200, "size": 501} +{"@timestamp": 893965906, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_94_off.gif HTTP/1.0", "status": 200, "size": 432} +{"@timestamp": 893965906, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_86_off.gif HTTP/1.0", "status": 200, "size": 409} +{"@timestamp": 893965906, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/posters/argentina78.gif HTTP/1.0", "status": 200, "size": 4033} +{"@timestamp": 893965907, "clientip":"45.135.0.0", "request": "GET /images/hist7802.jpg HTTP/1.0", "status": 200, "size": 10813} +{"@timestamp": 893965907, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_trophy2.gif HTTP/1.0", "status": 200, "size": 443} +{"@timestamp": 893965908, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_50_off.gif HTTP/1.0", "status": 200, "size": 392} +{"@timestamp": 893965908, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_header.gif HTTP/1.0", "status": 200, "size": 2077} +{"@timestamp": 893965910, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_66_off.gif HTTP/1.0", "status": 200, "size": 435} +{"@timestamp": 893965911, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_82_off.gif HTTP/1.0", "status": 200, "size": 433} +{"@timestamp": 893965912, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bracket_top.gif HTTP/1.0", "status": 200, "size": 289} +{"@timestamp": 893965913, "clientip":"45.135.0.0", "request": "GET /images/hist7801.jpg HTTP/1.0", "status": 200, "size": 12855} +{"@timestamp": 893965913, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_74_off.gif HTTP/1.0", "status": 200, "size": 423} +{"@timestamp": 893965913, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_34_off.gif HTTP/1.0", "status": 200, "size": 397} +{"@timestamp": 893965915, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bracket_bot.gif HTTP/1.0", "status": 200, "size": 631} +{"@timestamp": 893965917, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_58_off.gif HTTP/1.0", "status": 200, "size": 397} +{"@timestamp": 893965917, "clientip":"45.135.0.0", "request": "GET /french/history/past_cups/images/past_bu_90_off.gif HTTP/1.0", "status": 200, "size": 429} +{"@timestamp": 893965917, "clientip":"45.135.0.0", "request": "GET /french/history/images/thecup.GIF HTTP/1.0", "status": 200, "size": 1036} +{"@timestamp": 893965930, "clientip":"174.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965932, "clientip":"121.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965937, "clientip":"91.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965976, "clientip":"142.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965979, "clientip":"104.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965988, "clientip":"47.135.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 200, "size": 985} +{"@timestamp": 893965991, "clientip":"47.135.0.0", "request": "GET /french/nav_top_inet.html HTTP/1.0", "status": 200, "size": 374} +{"@timestamp": 893965991, "clientip":"47.135.0.0", "request": "GET /french/splash_inet.html HTTP/1.0", "status": 200, "size": 3781} +{"@timestamp": 893965991, "clientip":"47.135.0.0", "request": "GET /french/nav_inet.html HTTP/1.0", "status": 200, "size": 2739} +{"@timestamp": 893965992, "clientip":"109.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893965993, "clientip":"47.135.0.0", "request": "GET /images/nav_bg_bottom.jpg HTTP/1.0", "status": 200, "size": 8389} +{"@timestamp": 893965993, "clientip":"47.135.0.0", "request": "GET /images/logo_cfo.gif HTTP/1.0", "status": 200, "size": 1504} +{"@timestamp": 893965993, "clientip":"47.135.0.0", "request": "GET /french/images/nav_team_off.gif HTTP/1.0", "status": 200, "size": 870} +{"@timestamp": 893965993, "clientip":"47.135.0.0", "request": "GET /images/nav_bg_top.gif HTTP/1.0", "status": 200, "size": 929} +{"@timestamp": 893965994, "clientip":"47.135.0.0", "request": "GET /french/images/nav_news_off.gif HTTP/1.0", "status": 200, "size": 855} +{"@timestamp": 893965994, "clientip":"47.135.0.0", "request": "GET /french/images/nav_comp_off.gif HTTP/1.0", "status": 200, "size": 995} +{"@timestamp": 893965998, "clientip":"47.135.0.0", "request": "GET /french/images/nav_sitemap_off.gif HTTP/1.0", "status": 200, "size": 413} +{"@timestamp": 893966000, "clientip":"47.135.0.0", "request": "GET /french/images/nav_field_off.gif HTTP/1.0", "status": 200, "size": 982} +{"@timestamp": 893966001, "clientip":"47.135.0.0", "request": "GET /french/images/nav_hosts_off.gif HTTP/1.0", "status": 200, "size": 1139} +{"@timestamp": 893966002, "clientip":"47.135.0.0", "request": "GET /images/hm_nbg.jpg HTTP/1.0", "status": 200, "size": 33665} +{"@timestamp": 893966002, "clientip":"47.135.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 200, "size": 42} +{"@timestamp": 893966003, "clientip":"47.135.0.0", "request": "GET /french/images/nav_store_off.gif HTTP/1.0", "status": 200, "size": 976} +{"@timestamp": 893966004, "clientip":"47.135.0.0", "request": "GET /french/ProScroll.class HTTP/1.0", "status": 200, "size": 6507} +{"@timestamp": 893966004, "clientip":"47.135.0.0", "request": "GET /french/images/hm_official.gif HTTP/1.0", "status": 200, "size": 972} +{"@timestamp": 893966004, "clientip":"47.135.0.0", "request": "GET /french/images/nav_logo_sponsors.gif HTTP/1.0", "status": 200, "size": 1991} +{"@timestamp": 893966004, "clientip":"46.135.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966005, "clientip":"47.135.0.0", "request": "GET /french/images/nav_home_off.gif HTTP/1.0", "status": 200, "size": 915} +{"@timestamp": 893966006, "clientip":"47.135.0.0", "request": "GET /french/images/nav_history_off.gif HTTP/1.0", "status": 200, "size": 966} +{"@timestamp": 893966008, "clientip":"47.135.0.0", "request": "GET /french/images/nav_venue_off.gif HTTP/1.0", "status": 200, "size": 945} +{"@timestamp": 893966010, "clientip":"47.135.0.0", "request": "GET /french/images/nav_tickets_off.gif HTTP/1.0", "status": 200, "size": 965} +{"@timestamp": 893966011, "clientip":"55.0.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 200, "size": 985} +{"@timestamp": 893966017, "clientip":"47.135.0.0", "request": "GET /images/hm_anime_f.gif HTTP/1.0", "status": 200, "size": 15529} +{"@timestamp": 893966017, "clientip":"47.135.0.0", "request": "GET /images/hm_day_f.gif HTTP/1.0", "status": 200, "size": 574} +{"@timestamp": 893966019, "clientip":"47.135.0.0", "request": "GET /images/hm_brdl.gif HTTP/1.0", "status": 200, "size": 208} +{"@timestamp": 893966021, "clientip":"47.135.0.0", "request": "GET /images/hm_linkf.gif HTTP/1.0", "status": 200, "size": 123} +{"@timestamp": 893966024, "clientip":"47.135.0.0", "request": "GET /images/hm_arw.gif HTTP/1.0", "status": 200, "size": 1050} +{"@timestamp": 893966028, "clientip":"47.135.0.0", "request": "GET /images/dot.gif HTTP/1.0", "status": 200, "size": 43} +{"@timestamp": 893966028, "clientip":"47.135.0.0", "request": "GET /images/hm_brdr.gif HTTP/1.0", "status": 200, "size": 235} +{"@timestamp": 893966031, "clientip":"47.135.0.0", "request": "GET /images/info.gif HTTP/1.0", "status": 200, "size": 1251} +{"@timestamp": 893966048, "clientip":"131.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966055, "clientip":"41.135.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966068, "clientip":"161.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966068, "clientip":"56.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966084, "clientip":"176.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966085, "clientip":"47.135.0.0", "request": "GET /french/frntpage.htm HTTP/1.0", "status": 200, "size": 12824} +{"@timestamp": 893966088, "clientip":"55.0.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 200, "size": 985} +{"@timestamp": 893966090, "clientip":"47.2.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966091, "clientip":"47.135.0.0", "request": "GET /images/backnews.gif HTTP/1.0", "status": 200, "size": 4573} +{"@timestamp": 893966091, "clientip":"47.135.0.0", "request": "GET /french/images/fpnewstop.gif HTTP/1.0", "status": 200, "size": 1317} +{"@timestamp": 893966091, "clientip":"47.135.0.0", "request": "GET /french/images/space.gif HTTP/1.0", "status": 200, "size": 42} +{"@timestamp": 893966092, "clientip":"180.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966096, "clientip":"47.135.0.0", "request": "GET /images/ligne1_case5.gif HTTP/1.0", "status": 200, "size": 1018} +{"@timestamp": 893966097, "clientip":"47.135.0.0", "request": "GET /images/case5.gif HTTP/1.0", "status": 200, "size": 1362} +{"@timestamp": 893966097, "clientip":"47.135.0.0", "request": "GET /images/dburton.jpg HTTP/1.0", "status": 200, "size": 12009} +{"@timestamp": 893966098, "clientip":"47.135.0.0", "request": "GET /images/ligne.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966100, "clientip":"47.135.0.0", "request": "GET /images/ligneb.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966102, "clientip":"47.135.0.0", "request": "GET /images/ligneb01.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966102, "clientip":"47.135.0.0", "request": "GET /images/ligne01.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966104, "clientip":"47.135.0.0", "request": "GET /images/hm_f98_top.gif HTTP/1.0", "status": 200, "size": 915} +{"@timestamp": 893966104, "clientip":"47.135.0.0", "request": "GET /images/base.gif HTTP/1.0", "status": 200, "size": 366} +{"@timestamp": 893966108, "clientip":"47.135.0.0", "request": "GET /french/images/news_btn_press_off.gif HTTP/1.0", "status": 200, "size": 1795} +{"@timestamp": 893966109, "clientip":"47.135.0.0", "request": "GET /french/images/lateb_new.gif HTTP/1.0", "status": 200, "size": 1285} +{"@timestamp": 893966109, "clientip":"47.135.0.0", "request": "GET /french/images/today_new.gif HTTP/1.0", "status": 200, "size": 869} +{"@timestamp": 893966110, "clientip":"47.135.0.0", "request": "GET /images/bord_stories.gif HTTP/1.0", "status": 200, "size": 520} +{"@timestamp": 893966111, "clientip":"47.135.0.0", "request": "GET /images/ligne4_latebreak.gif HTTP/1.0", "status": 200, "size": 1056} +{"@timestamp": 893966112, "clientip":"47.135.0.0", "request": "GET /images/bord_stories01.gif HTTP/1.0", "status": 200, "size": 333} +{"@timestamp": 893966113, "clientip":"47.135.0.0", "request": "GET /french/images/archives.gif HTTP/1.0", "status": 200, "size": 569} +{"@timestamp": 893966113, "clientip":"47.135.0.0", "request": "GET /french/images/top_stories.gif HTTP/1.0", "status": 200, "size": 1078} +{"@timestamp": 893966113, "clientip":"47.135.0.0", "request": "GET /french/images/news_btn_letter_off.gif HTTP/1.0", "status": 200, "size": 871} +{"@timestamp": 893966115, "clientip":"47.135.0.0", "request": "GET /french/images/news_btn_kits_off.gif HTTP/1.0", "status": 200, "size": 965} +{"@timestamp": 893966135, "clientip":"184.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966178, "clientip":"4.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966210, "clientip":"183.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966218, "clientip":"249.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966220, "clientip":"235.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966221, "clientip":"47.135.0.0", "request": "GET /french/news/3004tick.htm HTTP/1.0", "status": 200, "size": 4234} +{"@timestamp": 893966223, "clientip":"47.135.0.0", "request": "GET /french/images/hm_f98_top.gif HTTP/1.0", "status": 200, "size": 915} +{"@timestamp": 893966224, "clientip":"47.135.0.0", "request": "GET /images/ps_bdr_r.gif HTTP/1.0", "status": 200, "size": 281} +{"@timestamp": 893966224, "clientip":"47.135.0.0", "request": "GET /images/ps_bdr_l.gif HTTP/1.0", "status": 200, "size": 346} +{"@timestamp": 893966232, "clientip":"248.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966259, "clientip":"2.2.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966273, "clientip":"218.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966280, "clientip":"48.4.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966296, "clientip":"128.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966306, "clientip":"139.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966310, "clientip":"141.78.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966310, "clientip":"141.78.0.0", "request": "GET /french/nav_top_inet.html HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966310, "clientip":"141.78.0.0", "request": "GET /french/nav_inet.html HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966311, "clientip":"141.78.0.0", "request": "GET /french/splash_inet.html HTTP/1.0", "status": 200, "size": 3781} +{"@timestamp": 893966322, "clientip":"249.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966327, "clientip":"55.0.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 200, "size": 985} +{"@timestamp": 893966342, "clientip":"47.135.0.0", "request": "GET /french/frntpage.htm HTTP/1.0", "status": 200, "size": 12824} +{"@timestamp": 893966343, "clientip":"141.78.0.0", "request": "GET /french/ProScroll.class HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966344, "clientip":"141.78.0.0", "request": "GET /french/images/nav_history_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966344, "clientip":"141.78.0.0", "request": "GET /french/images/nav_store_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966344, "clientip":"141.78.0.0", "request": "GET /images/nav_bg_top.gif HTTP/1.0", "status": 200, "size": 929} +{"@timestamp": 893966344, "clientip":"141.78.0.0", "request": "GET /french/images/nav_tickets_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966345, "clientip":"141.78.0.0", "request": "GET /images/hm_brdr.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966345, "clientip":"141.78.0.0", "request": "GET /images/hm_anime_f.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966345, "clientip":"141.78.0.0", "request": "GET /images/logo_cfo.gif HTTP/1.0", "status": 200, "size": 1504} +{"@timestamp": 893966345, "clientip":"141.78.0.0", "request": "GET /images/hm_day_f.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966346, "clientip":"141.78.0.0", "request": "GET /french/images/nav_team_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966346, "clientip":"141.78.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966346, "clientip":"141.78.0.0", "request": "GET /french/images/nav_history_off.gif HTTP/1.0", "status": 206, "size": 838} +{"@timestamp": 893966347, "clientip":"141.78.0.0", "request": "GET /images/hm_brdr.gif HTTP/1.0", "status": 206, "size": 107} +{"@timestamp": 893966348, "clientip":"141.78.0.0", "request": "GET /french/images/nav_field_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966348, "clientip":"141.78.0.0", "request": "GET /french/images/nav_sitemap_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966348, "clientip":"141.78.0.0", "request": "GET /french/images/nav_home_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966348, "clientip":"141.78.0.0", "request": "GET /images/hm_brdl.gif HTTP/1.0", "status": 200, "size": 208} +{"@timestamp": 893966348, "clientip":"141.78.0.0", "request": "GET /images/dot.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966349, "clientip":"141.78.0.0", "request": "GET /french/images/nav_venue_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966350, "clientip":"141.78.0.0", "request": "GET /french/images/nav_hosts_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966351, "clientip":"141.78.0.0", "request": "GET /images/hm_linkf.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966351, "clientip":"141.78.0.0", "request": "GET /french/images/nav_logo_sponsors.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966352, "clientip":"141.78.0.0", "request": "GET /images/hm_arw.gif HTTP/1.0", "status": 200, "size": 1050} +{"@timestamp": 893966352, "clientip":"141.78.0.0", "request": "GET /images/info.gif HTTP/1.0", "status": 200, "size": 1251} +{"@timestamp": 893966368, "clientip":"221.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966381, "clientip":"43.135.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966384, "clientip":"175.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966384, "clientip":"55.0.0.0", "request": "GET /french/index.html HTTP/1.0", "status": 200, "size": 985} +{"@timestamp": 893966386, "clientip":"141.78.0.0", "request": "GET /french/competition/maincomp.htm HTTP/1.0", "status": 200, "size": 3048} +{"@timestamp": 893966388, "clientip":"141.78.0.0", "request": "GET /images/comp_hm_brac.gif HTTP/1.0", "status": 200, "size": 254} +{"@timestamp": 893966388, "clientip":"141.78.0.0", "request": "GET /french/images/comp_bg2_hm.jpg HTTP/1.0", "status": 200, "size": 25676} +{"@timestamp": 893966388, "clientip":"141.78.0.0", "request": "GET /french/images/bar.jpg HTTP/1.0", "status": 200, "size": 686} +{"@timestamp": 893966388, "clientip":"141.78.0.0", "request": "GET /french/images/comp_hm_header_shad.gif HTTP/1.0", "status": 200, "size": 2207} +{"@timestamp": 893966388, "clientip":"141.78.0.0", "request": "GET /french/images/france98b.gif HTTP/1.0", "status": 200, "size": 2122} +{"@timestamp": 893966389, "clientip":"141.78.0.0", "request": "GET /french/images/comp_hm_nav.gif HTTP/1.0", "status": 200, "size": 11652} +{"@timestamp": 893966390, "clientip":"141.78.0.0", "request": "GET /images/comp_hm_archive.gif HTTP/1.0", "status": 200, "size": 1644} +{"@timestamp": 893966391, "clientip":"141.78.0.0", "request": "GET /french/frntpage.htm HTTP/1.0", "status": 200, "size": 12824} +{"@timestamp": 893966392, "clientip":"141.78.0.0", "request": "GET /images/backnews.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966392, "clientip":"141.78.0.0", "request": "GET /french/images/fpnewstop.gif HTTP/1.0", "status": 200, "size": 1317} +{"@timestamp": 893966392, "clientip":"141.78.0.0", "request": "GET /french/images/space.gif HTTP/1.0", "status": 200, "size": 42} +{"@timestamp": 893966393, "clientip":"141.78.0.0", "request": "GET /french/images/news_btn_letter_off.gif HTTP/1.0", "status": 200, "size": 871} +{"@timestamp": 893966393, "clientip":"141.78.0.0", "request": "GET /images/hm_f98_top.gif HTTP/1.0", "status": 200, "size": 915} +{"@timestamp": 893966393, "clientip":"141.78.0.0", "request": "GET /images/ligneb.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966393, "clientip":"141.78.0.0", "request": "GET /french/images/today_new.gif HTTP/1.0", "status": 200, "size": 869} +{"@timestamp": 893966394, "clientip":"141.78.0.0", "request": "GET /images/case5.gif HTTP/1.0", "status": 200, "size": 1362} +{"@timestamp": 893966394, "clientip":"141.78.0.0", "request": "GET /images/ligne.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966395, "clientip":"141.78.0.0", "request": "GET /images/ligneb.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966395, "clientip":"141.78.0.0", "request": "GET /images/ligne01.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966395, "clientip":"141.78.0.0", "request": "GET /images/ligneb.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966395, "clientip":"141.78.0.0", "request": "GET /french/images/news_btn_press_off.gif HTTP/1.0", "status": 200, "size": 1795} +{"@timestamp": 893966396, "clientip":"141.78.0.0", "request": "GET /images/ligne01.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966397, "clientip":"141.78.0.0", "request": "GET /images/bord_stories01.gif HTTP/1.0", "status": 200, "size": 333} +{"@timestamp": 893966397, "clientip":"141.78.0.0", "request": "GET /images/dburton.jpg HTTP/1.0", "status": 200, "size": 12009} +{"@timestamp": 893966397, "clientip":"141.78.0.0", "request": "GET /images/ligne1_case5.gif HTTP/1.0", "status": 200, "size": 1018} +{"@timestamp": 893966397, "clientip":"141.78.0.0", "request": "GET /images/ligne4_latebreak.gif HTTP/1.0", "status": 200, "size": 1056} +{"@timestamp": 893966397, "clientip":"42.135.0.0", "request": "GET /fth.htm HTTP/1.1", "status": 200, "size": 190} +{"@timestamp": 893966398, "clientip":"141.78.0.0", "request": "GET /french/images/news_btn_kits_off.gif HTTP/1.0", "status": 200, "size": 965} +{"@timestamp": 893966398, "clientip":"141.78.0.0", "request": "GET /images/ligneb01.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966398, "clientip":"141.78.0.0", "request": "GET /images/bord_stories.gif HTTP/1.0", "status": 200, "size": 520} +{"@timestamp": 893966399, "clientip":"141.78.0.0", "request": "GET /images/base.gif HTTP/1.0", "status": 200, "size": 366} +{"@timestamp": 893966399, "clientip":"141.78.0.0", "request": "GET /french/images/lateb_new.gif HTTP/1.0", "status": 200, "size": 1285} +{"@timestamp": 893966399, "clientip":"141.78.0.0", "request": "GET /images/ligneb01.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966399, "clientip":"141.78.0.0", "request": "GET /french/images/archives.gif HTTP/1.0", "status": 200, "size": 569} +{"@timestamp": 893966400, "clientip":"141.78.0.0", "request": "GET /french/images/top_stories.gif HTTP/1.0", "status": 200, "size": 1078} +{"@timestamp": 893966402, "clientip":"0.0.0.0", "request": "GET /images/home_intro.anim.gif HTTP/1.0", "status": 200, "size": 60349} +{"@timestamp": 893966403, "clientip":"1.0.0.0", "request": "GET /images/home_bg_stars.gif HTTP/1.0", "status": 200, "size": 2557} +{"@timestamp": 893966403, "clientip":"1.0.0.0", "request": "GET /images/home_fr_phrase.gif HTTP/1.0", "status": 200, "size": 2843} +{"@timestamp": 893966403, "clientip":"2.0.0.0", "request": "GET /images/nav_bg_top.gif HTTP/1.0", "status": 200, "size": 929} +{"@timestamp": 893966403, "clientip":"1.0.0.0", "request": "GET /images/home_logo.gif HTTP/1.0", "status": 200, "size": 3401} +{"@timestamp": 893966404, "clientip":"2.0.0.0", "request": "GET /images/logo_cfo.gif HTTP/1.0", "status": 200, "size": 1504} +{"@timestamp": 893966404, "clientip":"1.0.0.0", "request": "GET /images/home_eng_phrase.gif HTTP/1.0", "status": 200, "size": 2861} +{"@timestamp": 893966404, "clientip":"3.0.0.0", "request": "GET /english/index.html HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966404, "clientip":"4.0.0.0", "request": "GET /english/frntpage.htm HTTP/1.0", "status": 200, "size": 12800} +{"@timestamp": 893966404, "clientip":"5.0.0.0", "request": "GET /images/hm_f98_top.gif HTTP/1.1", "status": 200, "size": 915} +{"@timestamp": 893966404, "clientip":"6.0.0.0", "request": "GET /images/team_hm_concacaf.gif HTTP/1.0", "status": 200, "size": 764} +{"@timestamp": 893966404, "clientip":"6.0.0.0", "request": "GET /images/team_hm_afc.gif HTTP/1.0", "status": 200, "size": 475} +{"@timestamp": 893966404, "clientip":"6.0.0.0", "request": "GET /images/team_hm_caf.gif HTTP/1.0", "status": 200, "size": 473} +{"@timestamp": 893966405, "clientip":"7.0.0.0", "request": "GET /english/playing/mascot/mascot.html HTTP/1.0", "status": 200, "size": 5521} +{"@timestamp": 893966405, "clientip":"1.0.0.0", "request": "GET /images/home_tool.gif HTTP/1.0", "status": 200, "size": 327} +{"@timestamp": 893966405, "clientip":"8.0.0.0", "request": "GET /english/images/comp_bu_stage1n.gif HTTP/1.0", "status": 200, "size": 1548} +{"@timestamp": 893966405, "clientip":"8.0.0.0", "request": "GET /english/images/comp_bu_stage2n_on.gif HTTP/1.0", "status": 200, "size": 996} +{"@timestamp": 893966405, "clientip":"8.0.0.0", "request": "GET /images/comp_stage2_brc_top.gif HTTP/1.0", "status": 200, "size": 163} +{"@timestamp": 893966405, "clientip":"8.0.0.0", "request": "GET /images/comp_stage2_brc_topr.gif HTTP/1.0", "status": 200, "size": 163} +{"@timestamp": 893966405, "clientip":"9.0.0.0", "request": "GET /english/history/past_cups/images/posters/france38.gif HTTP/1.0", "status": 200, "size": 4649} +{"@timestamp": 893966405, "clientip":"3.0.0.0", "request": "GET /english/nav_top_inet.html HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966405, "clientip":"3.0.0.0", "request": "GET /english/nav_inet.html HTTP/1.0", "status": 200, "size": 2672} +{"@timestamp": 893966405, "clientip":"3.0.0.0", "request": "GET /english/splash_inet.html HTTP/1.0", "status": 200, "size": 3730} +{"@timestamp": 893966405, "clientip":"9.0.0.0", "request": "GET /english/history/past_cups/images/38-1.jpg HTTP/1.0", "status": 200, "size": 14315} +{"@timestamp": 893966405, "clientip":"10.0.0.0", "request": "GET /images/cal_nant.gif HTTP/1.0", "status": 200, "size": 359} +{"@timestamp": 893966405, "clientip":"9.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_38_on.gif HTTP/1.0", "status": 200, "size": 507} +{"@timestamp": 893966405, "clientip":"8.0.0.0", "request": "GET /images/comp_stage2_brc_botr.gif HTTP/1.0", "status": 200, "size": 158} +{"@timestamp": 893966405, "clientip":"8.0.0.0", "request": "GET /images/comp_stage2_brc_bot.gif HTTP/1.0", "status": 200, "size": 160} +{"@timestamp": 893966405, "clientip":"8.0.0.0", "request": "GET /images/comp_stage2_brc.gif HTTP/1.0", "status": 200, "size": 82} +{"@timestamp": 893966405, "clientip":"11.0.0.0", "request": "GET /images/comp_stage2_brc_botr.gif HTTP/1.0", "status": 200, "size": 158} +{"@timestamp": 893966405, "clientip":"12.0.0.0", "request": "GET /english/nav_top_inet.html HTTP/1.0", "status": 200, "size": 374} +{"@timestamp": 893966405, "clientip":"10.0.0.0", "request": "GET /images/cal_lyon.gif HTTP/1.0", "status": 200, "size": 286} +{"@timestamp": 893966405, "clientip":"12.0.0.0", "request": "GET /english/nav_inet.html HTTP/1.0", "status": 200, "size": 2672} +{"@timestamp": 893966405, "clientip":"1.0.0.0", "request": "GET /images/home_sponsor.gif HTTP/1.0", "status": 200, "size": 2491} +{"@timestamp": 893966405, "clientip":"12.0.0.0", "request": "GET /english/splash_inet.html HTTP/1.0", "status": 200, "size": 3730} +{"@timestamp": 893966405, "clientip":"10.0.0.0", "request": "GET /images/cal_mars.gif HTTP/1.0", "status": 200, "size": 377} +{"@timestamp": 893966405, "clientip":"10.0.0.0", "request": "GET /images/cal-lens.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966405, "clientip":"10.0.0.0", "request": "GET /images/cal_toul.gif HTTP/1.0", "status": 200, "size": 380} +{"@timestamp": 893966406, "clientip":"8.0.0.0", "request": "GET /english/competition/stage2.htm HTTP/1.0", "status": 200, "size": 16606} +{"@timestamp": 893966406, "clientip":"12.0.0.0", "request": "GET /images/hm_nbg.jpg HTTP/1.0", "status": 200, "size": 33665} +{"@timestamp": 893966406, "clientip":"10.0.0.0", "request": "GET /images/cal-lens.gif HTTP/1.0", "status": 206, "size": 124} +{"@timestamp": 893966406, "clientip":"8.0.0.0", "request": "GET /english/images/comp_bg2_hm.jpg HTTP/1.0", "status": 200, "size": 25676} +{"@timestamp": 893966406, "clientip":"8.0.0.0", "request": "GET /english/images/comp_hm_header.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966406, "clientip":"10.0.0.0", "request": "GET /images/cal_mont.gif HTTP/1.0", "status": 200, "size": 316} +{"@timestamp": 893966406, "clientip":"8.0.0.0", "request": "GET /english/images/space.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966406, "clientip":"2.0.0.0", "request": "GET /english/images/nav_store_off.gif HTTP/1.0", "status": 200, "size": 934} +{"@timestamp": 893966406, "clientip":"10.0.0.0", "request": "GET /images/cal_steti.gif HTTP/1.0", "status": 200, "size": 1125} +{"@timestamp": 893966406, "clientip":"1.0.0.0", "request": "GET /images/home_eng_button.gif HTTP/1.0", "status": 200, "size": 1927} +{"@timestamp": 893966406, "clientip":"5.0.0.0", "request": "GET /french/images/news_btn_press_off.gif HTTP/1.1", "status": 200, "size": 1795} +{"@timestamp": 893966406, "clientip":"1.0.0.0", "request": "GET /images/home_intro.anim.gif HTTP/1.0", "status": 200, "size": 60349} +{"@timestamp": 893966406, "clientip":"13.0.0.0", "request": "GET /english/images/comp_hm_header.gif HTTP/1.0", "status": 200, "size": 1189} +{"@timestamp": 893966406, "clientip":"3.0.0.0", "request": "GET /images/nav_bg_top.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966406, "clientip":"14.0.0.0", "request": "GET /images/logo_cfo.gif HTTP/1.1", "status": 200, "size": 1504} +{"@timestamp": 893966406, "clientip":"3.0.0.0", "request": "GET /images/logo_cfo.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966406, "clientip":"15.0.0.0", "request": "GET /cgi-bin/trivia/Trivia.pl?ENG HTTP/1.0", "status": 200, "size": 6213} +{"@timestamp": 893966406, "clientip":"13.0.0.0", "request": "GET /english/images/france98b.gif HTTP/1.0", "status": 200, "size": 2122} +{"@timestamp": 893966406, "clientip":"13.0.0.0", "request": "GET /english/images/comp_bu_stage1n.gif HTTP/1.0", "status": 200, "size": 1548} +{"@timestamp": 893966406, "clientip":"16.0.0.0", "request": "GET /images/s102336.gif HTTP/1.0", "status": 200, "size": 177} +{"@timestamp": 893966406, "clientip":"17.0.0.0", "request": "GET /english/history/images/france98b.GIF HTTP/1.0", "status": 200, "size": 915} +{"@timestamp": 893966406, "clientip":"17.0.0.0", "request": "GET /english/history/images/football.GIF HTTP/1.0", "status": 200, "size": 1170} +{"@timestamp": 893966406, "clientip":"18.0.0.0", "request": "GET /english/competition/headtohead78.htm HTTP/1.0", "status": 200, "size": 20126} +{"@timestamp": 893966406, "clientip":"3.0.0.0", "request": "GET /english/images/nav_tickets_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966406, "clientip":"12.0.0.0", "request": "GET /images/hm_anime_e.gif HTTP/1.0", "status": 200, "size": 15609} +{"@timestamp": 893966406, "clientip":"17.0.0.0", "request": "GET /english/history/images/infrance.GIF HTTP/1.0", "status": 200, "size": 990} +{"@timestamp": 893966406, "clientip":"19.0.0.0", "request": "GET /english/images/nav_news_off.gif HTTP/1.0", "status": 200, "size": 853} +{"@timestamp": 893966406, "clientip":"141.78.0.0", "request": "GET /french/images/hm_f98_top.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966406, "clientip":"141.78.0.0", "request": "GET /images/ps_bdr_r.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966406, "clientip":"141.78.0.0", "request": "GET /french/news/3004tick.htm HTTP/1.0", "status": 200, "size": 4234} +{"@timestamp": 893966406, "clientip":"141.78.0.0", "request": "GET /images/ps_bdr_l.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966407, "clientip":"12.0.0.0", "request": "GET /images/logo_cfo.gif HTTP/1.0", "status": 200, "size": 1504} +{"@timestamp": 893966407, "clientip":"19.0.0.0", "request": "GET /english/images/nav_comp_off.gif HTTP/1.0", "status": 200, "size": 994} +{"@timestamp": 893966407, "clientip":"3.0.0.0", "request": "GET /images/nav_bg_bottom.jpg HTTP/1.0", "status": 200, "size": 8389} +{"@timestamp": 893966407, "clientip":"19.0.0.0", "request": "GET /images/nav_bg_bottom.jpg HTTP/1.0", "status": 200, "size": 8389} +{"@timestamp": 893966407, "clientip":"8.0.0.0", "request": "GET /english/images/france98b.gif HTTP/1.0", "status": 200, "size": 2122} +{"@timestamp": 893966407, "clientip":"3.0.0.0", "request": "GET /english/images/nav_news_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966407, "clientip":"3.0.0.0", "request": "GET /english/images/nav_comp_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966407, "clientip":"1.0.0.0", "request": "GET /images/home_fr_button.gif HTTP/1.0", "status": 200, "size": 2140} +{"@timestamp": 893966407, "clientip":"12.0.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 200, "size": 42} +{"@timestamp": 893966407, "clientip":"3.0.0.0", "request": "GET /english/images/nav_team_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966407, "clientip":"3.0.0.0", "request": "GET /english/images/nav_venue_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966407, "clientip":"12.0.0.0", "request": "GET /english/images/hm_official.gif HTTP/1.0", "status": 200, "size": 1807} +{"@timestamp": 893966407, "clientip":"12.0.0.0", "request": "GET /english/ProScroll.class HTTP/1.0", "status": 200, "size": 6507} +{"@timestamp": 893966407, "clientip":"20.0.0.0", "request": "GET /images/10982.gif HTTP/1.0", "status": 200, "size": 183} +{"@timestamp": 893966407, "clientip":"20.0.0.0", "request": "GET /images/11101.gif HTTP/1.0", "status": 200, "size": 415} +{"@timestamp": 893966407, "clientip":"0.0.0.0", "request": "GET /images/home_sponsor.gif HTTP/1.0", "status": 200, "size": 2491} +{"@timestamp": 893966407, "clientip":"5.0.0.0", "request": "GET /french/images/space.gif HTTP/1.1", "status": 200, "size": 42} +{"@timestamp": 893966407, "clientip":"5.0.0.0", "request": "GET /french/images/fpnewstop.gif HTTP/1.1", "status": 200, "size": 1317} +{"@timestamp": 893966407, "clientip":"13.0.0.0", "request": "GET /english/images/comp_bu_stage2n.gif HTTP/1.0", "status": 200, "size": 984} +{"@timestamp": 893966407, "clientip":"2.0.0.0", "request": "GET /english/images/nav_sitemap_off.gif HTTP/1.0", "status": 200, "size": 416} +{"@timestamp": 893966407, "clientip":"13.0.0.0", "request": "GET /english/images/comp_bu_groupsn_on.gif HTTP/1.0", "status": 200, "size": 963} +{"@timestamp": 893966407, "clientip":"20.0.0.0", "request": "GET /images/bordeaux.gif HTTP/1.0", "status": 200, "size": 723} +{"@timestamp": 893966407, "clientip":"10.0.0.0", "request": "GET /images/cal_bord.gif HTTP/1.0", "status": 200, "size": 416} +{"@timestamp": 893966407, "clientip":"2.0.0.0", "request": "GET /english/images/nav_venue_off.gif HTTP/1.0", "status": 200, "size": 870} +{"@timestamp": 893966407, "clientip":"15.0.0.0", "request": "GET /english/playing/images/trivia/banner.jpg HTTP/1.0", "status": 200, "size": 19967} +{"@timestamp": 893966407, "clientip":"15.0.0.0", "request": "GET /english/playing/images/trivia/01test.gif HTTP/1.0", "status": 200, "size": 1664} +{"@timestamp": 893966407, "clientip":"8.0.0.0", "request": "GET /english/images/comp_bu_stage2n_on.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966408, "clientip":"8.0.0.0", "request": "GET /english/images/comp_bu_refsn.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966408, "clientip":"15.0.0.0", "request": "GET /english/playing/images/trivia/quizbg.gif HTTP/1.0", "status": 200, "size": 1460} +{"@timestamp": 893966408, "clientip":"20.0.0.0", "request": "GET /images/lyon.gif HTTP/1.0", "status": 200, "size": 599} +{"@timestamp": 893966408, "clientip":"12.0.0.0", "request": "GET /images/hm_day_e.gif HTTP/1.0", "status": 200, "size": 499} +{"@timestamp": 893966408, "clientip":"12.0.0.0", "request": "GET /images/info.gif HTTP/1.0", "status": 200, "size": 1251} +{"@timestamp": 893966408, "clientip":"12.0.0.0", "request": "GET /english/images/nav_team_off.gif HTTP/1.0", "status": 200, "size": 776} +{"@timestamp": 893966408, "clientip":"2.0.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 200, "size": 42} +{"@timestamp": 893966408, "clientip":"12.0.0.0", "request": "GET /images/nav_bg_bottom.jpg HTTP/1.0", "status": 200, "size": 8389} +{"@timestamp": 893966408, "clientip":"2.0.0.0", "request": "GET /english/images/hm_official.gif HTTP/1.0", "status": 200, "size": 1807} +{"@timestamp": 893966408, "clientip":"12.0.0.0", "request": "GET /english/images/nav_comp_off.gif HTTP/1.0", "status": 200, "size": 994} +{"@timestamp": 893966408, "clientip":"21.0.0.0", "request": "GET /english/news/11415.htm HTTP/1.1", "status": 200, "size": 21300} +{"@timestamp": 893966408, "clientip":"8.0.0.0", "request": "GET /images/comp_stage2_brc_top.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966408, "clientip":"12.0.0.0", "request": "GET /english/images/nav_field_off.gif HTTP/1.0", "status": 200, "size": 1005} +{"@timestamp": 893966408, "clientip":"17.0.0.0", "request": "GET /english/history/images/reading.GIF HTTP/1.0", "status": 200, "size": 1171} +{"@timestamp": 893966408, "clientip":"22.0.0.0", "request": "GET /english/teams/teamqualify124.htm HTTP/1.0", "status": 200, "size": 3866} +{"@timestamp": 893966408, "clientip":"23.0.0.0", "request": "GET / HTTP/1.0", "status": 200, "size": 8712} +{"@timestamp": 893966408, "clientip":"24.0.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966409, "clientip":"24.0.0.0", "request": "GET /english/playing/images/anim/trivia_on.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966409, "clientip":"5.0.0.0", "request": "GET /french/images/today_new.gif HTTP/1.1", "status": 200, "size": 869} +{"@timestamp": 893966409, "clientip":"20.0.0.0", "request": "GET /images/saintdenis.gif HTTP/1.0", "status": 200, "size": 702} +{"@timestamp": 893966409, "clientip":"24.0.0.0", "request": "GET /english/playing/images/play_hm_mascot.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966409, "clientip":"5.0.0.0", "request": "GET /french/images/top_stories.gif HTTP/1.1", "status": 200, "size": 1078} +{"@timestamp": 893966409, "clientip":"5.0.0.0", "request": "GET /images/case5.gif HTTP/1.1", "status": 200, "size": 1362} +{"@timestamp": 893966409, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_30_off.gif HTTP/1.0", "status": 200, "size": 406} +{"@timestamp": 893966409, "clientip":"20.0.0.0", "request": "GET /images/toulouse.gif HTTP/1.0", "status": 200, "size": 704} +{"@timestamp": 893966409, "clientip":"25.0.0.0", "request": "GET /images/home_bg_stars.gif HTTP/1.1", "status": 200, "size": 2557} +{"@timestamp": 893966409, "clientip":"26.0.0.0", "request": "GET /english/images/nav_field_off.gif HTTP/1.0", "status": 200, "size": 1005} +{"@timestamp": 893966409, "clientip":"27.0.0.0", "request": "GET /english/images/teams_bu_group_on.gif HTTP/1.0", "status": 200, "size": 668} +{"@timestamp": 893966409, "clientip":"13.0.0.0", "request": "GET /english/images/comp_bu_refsn.gif HTTP/1.0", "status": 200, "size": 933} +{"@timestamp": 893966409, "clientip":"28.0.0.0", "request": "GET /english/individuals/player13893.htm HTTP/1.0", "status": 200, "size": 6472} +{"@timestamp": 893966409, "clientip":"13.0.0.0", "request": "GET /english/images/comp_bu_calendar.gif HTTP/1.0", "status": 200, "size": 1197} +{"@timestamp": 893966410, "clientip":"23.0.0.0", "request": "GET /images/home_fr_phrase.gif HTTP/1.0", "status": 200, "size": 2843} +{"@timestamp": 893966410, "clientip":"12.0.0.0", "request": "GET /images/nav_bg_top.gif HTTP/1.0", "status": 200, "size": 929} +{"@timestamp": 893966410, "clientip":"23.0.0.0", "request": "GET /images/home_logo.gif HTTP/1.0", "status": 200, "size": 3401} +{"@timestamp": 893966410, "clientip":"15.0.0.0", "request": "GET /english/playing/images/trivia/02how.gif HTTP/1.0", "status": 200, "size": 754} +{"@timestamp": 893966410, "clientip":"23.0.0.0", "request": "GET /images/home_eng_phrase.gif HTTP/1.0", "status": 200, "size": 2861} +{"@timestamp": 893966410, "clientip":"23.0.0.0", "request": "GET /images/home_tool.gif HTTP/1.0", "status": 200, "size": 327} +{"@timestamp": 893966410, "clientip":"29.0.0.0", "request": "GET /images/10511.jpg HTTP/1.0", "status": 200, "size": 15543} +{"@timestamp": 893966410, "clientip":"20.0.0.0", "request": "GET /images/11289.jpg HTTP/1.0", "status": 200, "size": 6444} +{"@timestamp": 893966410, "clientip":"12.0.0.0", "request": "GET /english/images/nav_logo_sponsors.gif HTTP/1.0", "status": 200, "size": 1991} +{"@timestamp": 893966410, "clientip":"12.0.0.0", "request": "GET /images/hm_brdl.gif HTTP/1.0", "status": 200, "size": 208} +{"@timestamp": 893966410, "clientip":"23.0.0.0", "request": "GET /images/home_sponsor.gif HTTP/1.0", "status": 200, "size": 2491} +{"@timestamp": 893966410, "clientip":"20.0.0.0", "request": "GET /images/11288.jpg HTTP/1.0", "status": 200, "size": 5874} +{"@timestamp": 893966410, "clientip":"23.0.0.0", "request": "GET /images/home_eng_button.gif HTTP/1.0", "status": 200, "size": 1927} +{"@timestamp": 893966410, "clientip":"23.0.0.0", "request": "GET /images/home_fr_button.gif HTTP/1.0", "status": 200, "size": 2140} +{"@timestamp": 893966410, "clientip":"8.0.0.0", "request": "GET /images/comp_stage2_brc.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966410, "clientip":"5.0.0.0", "request": "GET /images/dburton.jpg HTTP/1.1", "status": 200, "size": 12009} +{"@timestamp": 893966410, "clientip":"2.0.0.0", "request": "GET /images/hm_anime_e.gif HTTP/1.0", "status": 200, "size": 15609} +{"@timestamp": 893966410, "clientip":"12.0.0.0", "request": "GET /images/hm_brdr.gif HTTP/1.0", "status": 200, "size": 235} +{"@timestamp": 893966410, "clientip":"12.0.0.0", "request": "GET /english/images/nav_news_off.gif HTTP/1.0", "status": 200, "size": 853} +{"@timestamp": 893966410, "clientip":"15.0.0.0", "request": "GET /english/playing/images/trivia/04luck.gif HTTP/1.0", "status": 200, "size": 744} +{"@timestamp": 893966410, "clientip":"23.0.0.0", "request": "GET /images/home_intro.anim.gif HTTP/1.0", "status": 200, "size": 60349} +{"@timestamp": 893966410, "clientip":"2.0.0.0", "request": "GET /english/ProScroll.class HTTP/1.0", "status": 200, "size": 6507} +{"@timestamp": 893966410, "clientip":"5.0.0.0", "request": "GET /images/bord_stories01.gif HTTP/1.1", "status": 200, "size": 333} +{"@timestamp": 893966410, "clientip":"12.0.0.0", "request": "GET /english/images/nav_venue_off.gif HTTP/1.0", "status": 200, "size": 870} +{"@timestamp": 893966411, "clientip":"30.0.0.0", "request": "GET /english/playing/download/download.html HTTP/1.0", "status": 200, "size": 13898} +{"@timestamp": 893966411, "clientip":"2.0.0.0", "request": "GET /english/images/nav_hosts_off.gif HTTP/1.0", "status": 200, "size": 914} +{"@timestamp": 893966411, "clientip":"16.0.0.0", "request": "GET /images/s102373.gif HTTP/1.0", "status": 200, "size": 142} +{"@timestamp": 893966411, "clientip":"3.0.0.0", "request": "GET /images/hm_day_e.gif HTTP/1.0", "status": 200, "size": 499} +{"@timestamp": 893966411, "clientip":"3.0.0.0", "request": "GET /images/hm_nbg.jpg HTTP/1.0", "status": 200, "size": 33665} +{"@timestamp": 893966411, "clientip":"12.0.0.0", "request": "GET /english/images/nav_tickets_off.gif HTTP/1.0", "status": 200, "size": 937} +{"@timestamp": 893966411, "clientip":"3.0.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966411, "clientip":"3.0.0.0", "request": "GET /english/images/hm_official.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966411, "clientip":"5.0.0.0", "request": "GET /images/bord_stories.gif HTTP/1.1", "status": 200, "size": 520} +{"@timestamp": 893966411, "clientip":"3.0.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966411, "clientip":"31.0.0.0", "request": "GET /english/images/nav_store_off.gif HTTP/1.1", "status": 200, "size": 934} +{"@timestamp": 893966411, "clientip":"15.0.0.0", "request": "GET /english/playing/images/trivia/03score.gif HTTP/1.0", "status": 200, "size": 572} +{"@timestamp": 893966411, "clientip":"3.0.0.0", "request": "GET /english/images/nav_store_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966411, "clientip":"31.0.0.0", "request": "GET /english/images/nav_hosts_off.gif HTTP/1.1", "status": 200, "size": 914} +{"@timestamp": 893966411, "clientip":"32.0.0.0", "request": "GET /images/home_intro.anim.gif HTTP/1.1", "status": 200, "size": 60349} +{"@timestamp": 893966412, "clientip":"12.0.0.0", "request": "GET /english/images/nav_sitemap_off.gif HTTP/1.0", "status": 200, "size": 416} +{"@timestamp": 893966412, "clientip":"12.0.0.0", "request": "GET /images/hm_linkf.gif HTTP/1.0", "status": 200, "size": 123} +{"@timestamp": 893966412, "clientip":"12.0.0.0", "request": "GET /english/images/nav_history_off.gif HTTP/1.0", "status": 200, "size": 914} +{"@timestamp": 893966412, "clientip":"12.0.0.0", "request": "GET /images/hm_arw.gif HTTP/1.0", "status": 200, "size": 1050} +{"@timestamp": 893966412, "clientip":"12.0.0.0", "request": "GET /english/images/nav_hosts_off.gif HTTP/1.0", "status": 200, "size": 914} +{"@timestamp": 893966412, "clientip":"15.0.0.0", "request": "GET /english/playing/images/trivia/submit.gif HTTP/1.0", "status": 200, "size": 1808} +{"@timestamp": 893966412, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_header.gif HTTP/1.0", "status": 200, "size": 1989} +{"@timestamp": 893966412, "clientip":"12.0.0.0", "request": "GET /english/images/nav_store_off.gif HTTP/1.0", "status": 200, "size": 934} +{"@timestamp": 893966412, "clientip":"23.0.0.0", "request": "GET /english/index.html HTTP/1.0", "status": 200, "size": 892} +{"@timestamp": 893966412, "clientip":"12.0.0.0", "request": "GET /images/dot.gif HTTP/1.0", "status": 200, "size": 43} +{"@timestamp": 893966412, "clientip":"5.0.0.0", "request": "GET /french/images/lateb_new.gif HTTP/1.1", "status": 200, "size": 1285} +{"@timestamp": 893966412, "clientip":"19.0.0.0", "request": "GET /english/images/nav_field_off.gif HTTP/1.0", "status": 200, "size": 1005} +{"@timestamp": 893966412, "clientip":"25.0.0.0", "request": "GET /images/home_eng_phrase.gif HTTP/1.1", "status": 200, "size": 2861} +{"@timestamp": 893966412, "clientip":"19.0.0.0", "request": "GET /english/images/nav_venue_off.gif HTTP/1.0", "status": 200, "size": 870} +{"@timestamp": 893966412, "clientip":"23.0.0.0", "request": "GET /english/nav_top_inet.html HTTP/1.0", "status": 200, "size": 374} +{"@timestamp": 893966412, "clientip":"23.0.0.0", "request": "GET /english/nav_inet.html HTTP/1.0", "status": 200, "size": 2672} +{"@timestamp": 893966412, "clientip":"33.0.0.0", "request": "GET /images/home_fr_button.gif HTTP/1.1", "status": 200, "size": 2140} +{"@timestamp": 893966413, "clientip":"25.0.0.0", "request": "GET /images/home_fr_button.gif HTTP/1.1", "status": 200, "size": 2140} +{"@timestamp": 893966413, "clientip":"23.0.0.0", "request": "GET /images/nav_bg_bottom.jpg HTTP/1.0", "status": 200, "size": 8389} +{"@timestamp": 893966413, "clientip":"17.0.0.0", "request": "GET /english/history/images/thecup.GIF HTTP/1.0", "status": 200, "size": 1293} +{"@timestamp": 893966413, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_34_off.gif HTTP/1.0", "status": 200, "size": 397} +{"@timestamp": 893966413, "clientip":"23.0.0.0", "request": "GET /english/images/nav_team_off.gif HTTP/1.0", "status": 200, "size": 776} +{"@timestamp": 893966413, "clientip":"12.0.0.0", "request": "GET /english/images/nav_home_off.gif HTTP/1.0", "status": 200, "size": 828} +{"@timestamp": 893966413, "clientip":"16.0.0.0", "request": "GET /images/s102338.gif HTTP/1.0", "status": 200, "size": 138} +{"@timestamp": 893966413, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_38_off.gif HTTP/1.0", "status": 200, "size": 436} +{"@timestamp": 893966413, "clientip":"3.0.0.0", "request": "GET /english/images/nav_sitemap_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966413, "clientip":"23.0.0.0", "request": "GET /english/images/nav_comp_off.gif HTTP/1.0", "status": 200, "size": 994} +{"@timestamp": 893966413, "clientip":"20.0.0.0", "request": "GET /images/11103.gif HTTP/1.0", "status": 200, "size": 513} +{"@timestamp": 893966413, "clientip":"3.0.0.0", "request": "GET /english/images/nav_field_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966413, "clientip":"23.0.0.0", "request": "GET /english/ProScroll.class HTTP/1.0", "status": 200, "size": 6507} +{"@timestamp": 893966413, "clientip":"3.0.0.0", "request": "GET /english/images/nav_hosts_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966413, "clientip":"23.0.0.0", "request": "GET /english/images/nav_store_off.gif HTTP/1.0", "status": 200, "size": 934} +{"@timestamp": 893966413, "clientip":"3.0.0.0", "request": "GET /english/ProScroll.class HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966413, "clientip":"34.0.0.0", "request": "GET /english/playing/body.html HTTP/1.0", "status": 200, "size": 5033} +{"@timestamp": 893966413, "clientip":"19.0.0.0", "request": "GET /english/images/nav_hosts_off.gif HTTP/1.0", "status": 200, "size": 914} +{"@timestamp": 893966413, "clientip":"3.0.0.0", "request": "GET /images/hm_brdl.gif HTTP/1.0", "status": 200, "size": 208} +{"@timestamp": 893966414, "clientip":"35.0.0.0", "request": "GET / HTTP/1.0", "status": 200, "size": 8712} +{"@timestamp": 893966414, "clientip":"36.0.0.0", "request": "GET /english/frntpage.htm HTTP/1.0", "status": 200, "size": 12800} +{"@timestamp": 893966414, "clientip":"23.0.0.0", "request": "GET /english/images/nav_venue_off.gif HTTP/1.0", "status": 200, "size": 870} +{"@timestamp": 893966414, "clientip":"23.0.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 200, "size": 42} +{"@timestamp": 893966414, "clientip":"23.0.0.0", "request": "GET /english/images/nav_sitemap_off.gif HTTP/1.0", "status": 200, "size": 416} +{"@timestamp": 893966414, "clientip":"20.0.0.0", "request": "GET /images/marseille.gif HTTP/1.0", "status": 200, "size": 722} +{"@timestamp": 893966414, "clientip":"5.0.0.0", "request": "GET /images/ligne4_latebreak.gif HTTP/1.1", "status": 200, "size": 1056} +{"@timestamp": 893966414, "clientip":"23.0.0.0", "request": "GET /images/hm_linkf.gif HTTP/1.0", "status": 200, "size": 123} +{"@timestamp": 893966414, "clientip":"23.0.0.0", "request": "GET /english/images/nav_field_off.gif HTTP/1.0", "status": 200, "size": 1005} +{"@timestamp": 893966414, "clientip":"23.0.0.0", "request": "GET /english/images/nav_tickets_off.gif HTTP/1.0", "status": 200, "size": 937} +{"@timestamp": 893966414, "clientip":"23.0.0.0", "request": "GET /english/images/nav_history_off.gif HTTP/1.0", "status": 200, "size": 914} +{"@timestamp": 893966414, "clientip":"24.0.0.0", "request": "GET /cgi-bin/trivia/Trivia.pl?ENG HTTP/1.0", "status": 200, "size": 6228} +{"@timestamp": 893966414, "clientip":"5.0.0.0", "request": "GET /images/ligneb01.gif HTTP/1.1", "status": 200, "size": 169} +{"@timestamp": 893966414, "clientip":"12.0.0.0", "request": "GET /english/tickets/body.html HTTP/1.0", "status": 200, "size": 2925} +{"@timestamp": 893966414, "clientip":"23.0.0.0", "request": "GET /english/images/nav_hosts_off.gif HTTP/1.0", "status": 200, "size": 914} +{"@timestamp": 893966414, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_50_on.gif HTTP/1.0", "status": 200, "size": 455} +{"@timestamp": 893966414, "clientip":"23.0.0.0", "request": "GET /images/hm_anime_e.gif HTTP/1.0", "status": 200, "size": 15609} +{"@timestamp": 893966414, "clientip":"3.0.0.0", "request": "GET /english/images/nav_history_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966414, "clientip":"37.0.0.0", "request": "GET /french/competition/schedule.htm HTTP/1.0", "status": 200, "size": 49256} +{"@timestamp": 893966414, "clientip":"3.0.0.0", "request": "GET /images/hm_brdr.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966414, "clientip":"26.0.0.0", "request": "GET /english/images/nav_history_off.gif HTTP/1.0", "status": 200, "size": 914} +{"@timestamp": 893966414, "clientip":"232.41.0.0", "request": "GET / HTTP/1.0", "status": 200, "size": 8749} +{"@timestamp": 893966415, "clientip":"232.41.0.0", "request": "GET /images/home_tool.gif HTTP/1.0", "status": 200, "size": 327} +{"@timestamp": 893966415, "clientip":"19.0.0.0", "request": "GET /english/images/nav_home_off.gif HTTP/1.0", "status": 200, "size": 828} +{"@timestamp": 893966415, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_58_off.gif HTTP/1.0", "status": 200, "size": 397} +{"@timestamp": 893966415, "clientip":"12.0.0.0", "request": "GET /english/tickets/images/ticket_hm_bg.jpg HTTP/1.0", "status": 200, "size": 20929} +{"@timestamp": 893966415, "clientip":"12.0.0.0", "request": "GET /english/tickets/images/ticket_hm_header.gif HTTP/1.0", "status": 200, "size": 1226} +{"@timestamp": 893966415, "clientip":"12.0.0.0", "request": "GET /english/tickets/images/ticket_hm_nav.gif HTTP/1.0", "status": 200, "size": 11253} +{"@timestamp": 893966415, "clientip":"12.0.0.0", "request": "GET /images/arw_red_lk.gif HTTP/1.0", "status": 200, "size": 1068} +{"@timestamp": 893966415, "clientip":"23.0.0.0", "request": "GET /english/images/nav_home_off.gif HTTP/1.0", "status": 200, "size": 828} +{"@timestamp": 893966415, "clientip":"23.0.0.0", "request": "GET /english/images/nav_logo_sponsors.gif HTTP/1.0", "status": 200, "size": 1991} +{"@timestamp": 893966415, "clientip":"21.0.0.0", "request": "GET /english/images/news_hm_header.gif HTTP/1.1", "status": 200, "size": 527} +{"@timestamp": 893966415, "clientip":"21.0.0.0", "request": "GET /images/bg_generic.jpg HTTP/1.1", "status": 200, "size": 21127} +{"@timestamp": 893966415, "clientip":"38.0.0.0", "request": "GET /english/teams/teamgroup.htm HTTP/1.0", "status": 200, "size": 11971} +{"@timestamp": 893966415, "clientip":"21.0.0.0", "request": "GET /english/images/space.gif HTTP/1.1", "status": 200, "size": 42} +{"@timestamp": 893966415, "clientip":"23.0.0.0", "request": "GET /images/info.gif HTTP/1.0", "status": 200, "size": 1251} +{"@timestamp": 893966415, "clientip":"34.0.0.0", "request": "GET /english/playing/images/play_hm_bg_opt1.jpg HTTP/1.0", "status": 200, "size": 44502} +{"@timestamp": 893966415, "clientip":"5.0.0.0", "request": "GET /images/ligne.gif HTTP/1.1", "status": 200, "size": 169} +{"@timestamp": 893966415, "clientip":"3.0.0.0", "request": "GET /english/images/nav_logo_sponsors.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966415, "clientip":"12.0.0.0", "request": "GET /images/arw_lk.gif HTTP/1.0", "status": 200, "size": 669} +{"@timestamp": 893966415, "clientip":"19.0.0.0", "request": "GET /english/images/nav_history_off.gif HTTP/1.0", "status": 200, "size": 914} +{"@timestamp": 893966415, "clientip":"35.0.0.0", "request": "GET /images/home_bg_stars.gif HTTP/1.0", "status": 200, "size": 2557} +{"@timestamp": 893966416, "clientip":"23.0.0.0", "request": "GET /images/hm_brdl.gif HTTP/1.0", "status": 200, "size": 208} +{"@timestamp": 893966416, "clientip":"5.0.0.0", "request": "GET /french/images/news_btn_kits_off.gif HTTP/1.1", "status": 200, "size": 965} +{"@timestamp": 893966416, "clientip":"23.0.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 200, "size": 42} +{"@timestamp": 893966416, "clientip":"23.0.0.0", "request": "GET /english/images/hm_official.gif HTTP/1.0", "status": 200, "size": 1807} +{"@timestamp": 893966416, "clientip":"23.0.0.0", "request": "GET /images/hm_arw.gif HTTP/1.0", "status": 200, "size": 1050} +{"@timestamp": 893966416, "clientip":"19.0.0.0", "request": "GET /english/images/nav_sitemap_off.gif HTTP/1.0", "status": 200, "size": 416} +{"@timestamp": 893966416, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_62_off.gif HTTP/1.0", "status": 200, "size": 437} +{"@timestamp": 893966416, "clientip":"23.0.0.0", "request": "GET /images/hm_day_e.gif HTTP/1.0", "status": 200, "size": 499} +{"@timestamp": 893966416, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_66_off.gif HTTP/1.0", "status": 200, "size": 435} +{"@timestamp": 893966416, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_70_off.gif HTTP/1.0", "status": 200, "size": 409} +{"@timestamp": 893966416, "clientip":"39.0.0.0", "request": "GET /english/history/images/history_hm_header.gif HTTP/1.1", "status": 200, "size": 688} +{"@timestamp": 893966416, "clientip":"39.0.0.0", "request": "GET /english/history/images/history_hm_posters.jpg HTTP/1.1", "status": 200, "size": 33296} +{"@timestamp": 893966416, "clientip":"201.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966417, "clientip":"25.0.0.0", "request": "GET /images/home_eng_button.gif HTTP/1.1", "status": 200, "size": 1927} +{"@timestamp": 893966417, "clientip":"23.0.0.0", "request": "GET /images/dot.gif HTTP/1.0", "status": 200, "size": 43} +{"@timestamp": 893966417, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_74_off.gif HTTP/1.0", "status": 200, "size": 423} +{"@timestamp": 893966417, "clientip":"1.0.0.0", "request": "GET /english/index.html HTTP/1.0", "status": 200, "size": 892} +{"@timestamp": 893966417, "clientip":"5.0.0.0", "request": "GET /images/ligneb.gif HTTP/1.1", "status": 200, "size": 169} +{"@timestamp": 893966417, "clientip":"35.0.0.0", "request": "GET /images/home_eng_phrase.gif HTTP/1.0", "status": 200, "size": 2861} +{"@timestamp": 893966417, "clientip":"35.0.0.0", "request": "GET /images/home_fr_button.gif HTTP/1.0", "status": 200, "size": 2140} +{"@timestamp": 893966417, "clientip":"35.0.0.0", "request": "GET /images/home_tool.gif HTTP/1.0", "status": 200, "size": 327} +{"@timestamp": 893966417, "clientip":"35.0.0.0", "request": "GET /images/home_sponsor.gif HTTP/1.0", "status": 200, "size": 2491} +{"@timestamp": 893966417, "clientip":"35.0.0.0", "request": "GET /images/home_eng_button.gif HTTP/1.0", "status": 200, "size": 1927} +{"@timestamp": 893966417, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_78_off.gif HTTP/1.0", "status": 200, "size": 427} +{"@timestamp": 893966417, "clientip":"16.0.0.0", "request": "GET /english/images/team_group_header_d.gif HTTP/1.0", "status": 200, "size": 646} +{"@timestamp": 893966418, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_82_off.gif HTTP/1.0", "status": 200, "size": 433} +{"@timestamp": 893966418, "clientip":"40.0.0.0", "request": "GET /english/history/past_cups/italy34.html HTTP/1.1", "status": 200, "size": 13022} +{"@timestamp": 893966418, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_86_off.gif HTTP/1.0", "status": 200, "size": 409} +{"@timestamp": 893966418, "clientip":"19.0.0.0", "request": "GET /english/images/nav_store_off.gif HTTP/1.0", "status": 200, "size": 934} +{"@timestamp": 893966418, "clientip":"1.0.0.0", "request": "GET /english/index.html HTTP/1.0", "status": 200, "size": 892} +{"@timestamp": 893966418, "clientip":"37.0.0.0", "request": "GET /french/images/space.gif HTTP/1.0", "status": 200, "size": 42} +{"@timestamp": 893966418, "clientip":"19.0.0.0", "request": "GET /english/images/nav_tickets_off.gif HTTP/1.0", "status": 200, "size": 937} +{"@timestamp": 893966418, "clientip":"19.0.0.0", "request": "GET /english/images/nav_logo_sponsors.gif HTTP/1.0", "status": 200, "size": 1991} +{"@timestamp": 893966418, "clientip":"8.0.0.0", "request": "GET /english/images/comp_bu_groupsn_on.gif HTTP/1.0", "status": 200, "size": 963} +{"@timestamp": 893966418, "clientip":"22.0.0.0", "request": "GET /english/images/team_bu_detail2.gif HTTP/1.0", "status": 200, "size": 1438} +{"@timestamp": 893966418, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_90_off.gif HTTP/1.0", "status": 200, "size": 429} +{"@timestamp": 893966418, "clientip":"38.0.0.0", "request": "GET /english/images/teams_bu_group_on.gif HTTP/1.0", "status": 200, "size": 668} +{"@timestamp": 893966419, "clientip":"20.0.0.0", "request": "GET /images/nantes.gif HTTP/1.0", "status": 200, "size": 677} +{"@timestamp": 893966419, "clientip":"16.0.0.0", "request": "GET /images/s102326.gif HTTP/1.0", "status": 200, "size": 248} +{"@timestamp": 893966419, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_94_off.gif HTTP/1.0", "status": 200, "size": 432} +{"@timestamp": 893966419, "clientip":"1.0.0.0", "request": "GET /english/nav_top_inet.html HTTP/1.0", "status": 200, "size": 374} +{"@timestamp": 893966419, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bracket_top.gif HTTP/1.0", "status": 200, "size": 289} +{"@timestamp": 893966419, "clientip":"20.0.0.0", "request": "GET /images/saintetienne.gif HTTP/1.0", "status": 200, "size": 761} +{"@timestamp": 893966419, "clientip":"20.0.0.0", "request": "GET /images/11287.jpg HTTP/1.0", "status": 200, "size": 6431} +{"@timestamp": 893966420, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/posters/brasil50.gif HTTP/1.0", "status": 200, "size": 5003} +{"@timestamp": 893966420, "clientip":"20.0.0.0", "request": "GET /images/lens.gif HTTP/1.0", "status": 200, "size": 582} +{"@timestamp": 893966420, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_bracket_bot.gif HTTP/1.0", "status": 200, "size": 631} +{"@timestamp": 893966420, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/past_trophy.gif HTTP/1.0", "status": 200, "size": 800} +{"@timestamp": 893966420, "clientip":"41.0.0.0", "request": "GET /images/col.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966421, "clientip":"17.0.0.0", "request": "GET /english/history/past_cups/images/50-1.jpg HTTP/1.0", "status": 200, "size": 15314} +{"@timestamp": 893966421, "clientip":"1.0.0.0", "request": "GET /images/hm_nbg.jpg HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966421, "clientip":"40.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_30_off.gif HTTP/1.1", "status": 200, "size": 406} +{"@timestamp": 893966421, "clientip":"40.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_34_on.gif HTTP/1.1", "status": 200, "size": 474} +{"@timestamp": 893966421, "clientip":"16.0.0.0", "request": "GET /images/s102477.gif HTTP/1.0", "status": 200, "size": 214} +{"@timestamp": 893966421, "clientip":"26.0.0.0", "request": "GET /english/images/nav_hosts_off.gif HTTP/1.0", "status": 200, "size": 914} +{"@timestamp": 893966421, "clientip":"26.0.0.0", "request": "GET /english/images/nav_store_off.gif HTTP/1.0", "status": 200, "size": 934} +{"@timestamp": 893966421, "clientip":"8.0.0.0", "request": "GET /english/teams/teamgroup.htm HTTP/1.0", "status": 200, "size": 11971} +{"@timestamp": 893966421, "clientip":"38.0.0.0", "request": "GET /images/s102325.gif HTTP/1.0", "status": 200, "size": 187} +{"@timestamp": 893966422, "clientip":"15.0.0.0", "request": "GET /english/playing/links.html HTTP/1.0", "status": 200, "size": 18694} +{"@timestamp": 893966422, "clientip":"40.0.0.0", "request": "GET /english/history/past_cups/images/34-1.jpg HTTP/1.1", "status": 200, "size": 14275} +{"@timestamp": 893966422, "clientip":"8.0.0.0", "request": "GET /images/teams_hm_bg.jpg HTTP/1.0", "status": 200, "size": 18794} +{"@timestamp": 893966422, "clientip":"42.0.0.0", "request": "GET /english/frntpage.htm HTTP/1.0", "status": 200, "size": 12800} +{"@timestamp": 893966422, "clientip":"25.0.0.0", "request": "GET /images/home_sponsor.gif HTTP/1.1", "status": 200, "size": 2491} +{"@timestamp": 893966422, "clientip":"8.0.0.0", "request": "GET /english/images/team_group_header_a.gif HTTP/1.0", "status": 200, "size": 639} +{"@timestamp": 893966422, "clientip":"8.0.0.0", "request": "GET /english/images/teams_bu_group_on.gif HTTP/1.0", "status": 200, "size": 668} +{"@timestamp": 893966422, "clientip":"8.0.0.0", "request": "GET /english/images/teams_bu_confed_off.gif HTTP/1.0", "status": 200, "size": 1105} +{"@timestamp": 893966422, "clientip":"43.0.0.0", "request": "GET /english/individuals/player111722.htm HTTP/1.0", "status": 200, "size": 6523} +{"@timestamp": 893966423, "clientip":"20.0.0.0", "request": "GET /images/montpellier.gif HTTP/1.0", "status": 200, "size": 728} +{"@timestamp": 893966423, "clientip":"12.0.0.0", "request": "GET /english/tickets/out_france.html HTTP/1.0", "status": 200, "size": 6143} +{"@timestamp": 893966423, "clientip":"20.0.0.0", "request": "GET /images/11105.gif HTTP/1.0", "status": 200, "size": 114} +{"@timestamp": 893966423, "clientip":"12.0.0.0", "request": "GET /english/tickets/images/ticket_quest_bg2.jpg HTTP/1.0", "status": 200, "size": 11324} +{"@timestamp": 893966423, "clientip":"12.0.0.0", "request": "GET /english/tickets/images/ticket_header.gif HTTP/1.0", "status": 200, "size": 453} +{"@timestamp": 893966423, "clientip":"12.0.0.0", "request": "GET /english/tickets/images/hm_f98_top.gif HTTP/1.0", "status": 200, "size": 1647} +{"@timestamp": 893966423, "clientip":"21.0.0.0", "request": "GET /english/images/news_bu_press_off.gif HTTP/1.1", "status": 200, "size": 847} +{"@timestamp": 893966423, "clientip":"44.0.0.0", "request": "GET /english/teams/teambio169.htm HTTP/1.1", "status": 200, "size": 11157} +{"@timestamp": 893966423, "clientip":"38.0.0.0", "request": "GET /images/s102327.gif HTTP/1.0", "status": 200, "size": 97} +{"@timestamp": 893966423, "clientip":"21.0.0.0", "request": "GET /english/images/news_bu_letter_off.gif HTTP/1.1", "status": 200, "size": 940} +{"@timestamp": 893966423, "clientip":"45.0.0.0", "request": "GET /english/frntpage.htm HTTP/1.1", "status": 200, "size": 12800} +{"@timestamp": 893966423, "clientip":"32.0.0.0", "request": "GET /english/nav_inet.html HTTP/1.1", "status": 200, "size": 2672} +{"@timestamp": 893966423, "clientip":"15.0.0.0", "request": "GET /english/playing/images/backg.gif HTTP/1.0", "status": 200, "size": 1462} +{"@timestamp": 893966423, "clientip":"15.0.0.0", "request": "GET /english/playing/images/play_header.gif HTTP/1.0", "status": 200, "size": 2262} +{"@timestamp": 893966423, "clientip":"20.0.0.0", "request": "GET /images/paris_off.gif HTTP/1.0", "status": 200, "size": 658} +{"@timestamp": 893966423, "clientip":"8.0.0.0", "request": "GET /images/teams_hm_bracket.gif HTTP/1.0", "status": 200, "size": 655} +{"@timestamp": 893966424, "clientip":"8.0.0.0", "request": "GET /english/images/team_group_header_c.gif HTTP/1.0", "status": 200, "size": 670} +{"@timestamp": 893966424, "clientip":"29.0.0.0", "request": "GET /images/32t49811.jpg HTTP/1.0", "status": 200, "size": 4753} +{"@timestamp": 893966424, "clientip":"37.0.0.0", "request": "GET /images/cal-lens.gif HTTP/1.0", "status": 200, "size": 284} +{"@timestamp": 893966424, "clientip":"40.0.0.0", "request": "GET /english/history/past_cups/images/posters/italy34.gif HTTP/1.1", "status": 200, "size": 4474} +{"@timestamp": 893966424, "clientip":"12.0.0.0", "request": "GET /english/tickets/images/ticket_bu_quest2.gif HTTP/1.0", "status": 200, "size": 1201} +{"@timestamp": 893966424, "clientip":"38.0.0.0", "request": "GET /images/s102424.gif HTTP/1.0", "status": 200, "size": 164} +{"@timestamp": 893966424, "clientip":"20.0.0.0", "request": "GET /images/11102.gif HTTP/1.0", "status": 200, "size": 417} +{"@timestamp": 893966424, "clientip":"15.0.0.0", "request": "GET /english/playing/images/france98b.GIF HTTP/1.0", "status": 200, "size": 597} +{"@timestamp": 893966424, "clientip":"39.0.0.0", "request": "GET /english/history/images/france98b.GIF HTTP/1.1", "status": 200, "size": 915} +{"@timestamp": 893966424, "clientip":"39.0.0.0", "request": "GET /english/history/images/football.GIF HTTP/1.1", "status": 200, "size": 1170} +{"@timestamp": 893966424, "clientip":"46.0.0.0", "request": "GET /english/venues/venues/saint-denis.html HTTP/1.0", "status": 200, "size": 22760} +{"@timestamp": 893966425, "clientip":"12.0.0.0", "request": "GET /english/tickets/images/ticket_bu_abroad2_on.gif HTTP/1.0", "status": 200, "size": 2020} +{"@timestamp": 893966425, "clientip":"47.0.0.0", "request": "GET / HTTP/1.0", "status": 200, "size": 8712} +{"@timestamp": 893966425, "clientip":"12.0.0.0", "request": "GET /english/tickets/images/ticket_abroad_header.gif HTTP/1.0", "status": 200, "size": 3606} +{"@timestamp": 893966425, "clientip":"15.0.0.0", "request": "GET /english/playing/images/mascots.GIF HTTP/1.0", "status": 200, "size": 1275} +{"@timestamp": 893966425, "clientip":"44.0.0.0", "request": "GET /images/10512.jpg HTTP/1.1", "status": 200, "size": 17227} +{"@timestamp": 893966425, "clientip":"44.0.0.0", "request": "GET /images/32t49809.jpg HTTP/1.1", "status": 200, "size": 5623} +{"@timestamp": 893966425, "clientip":"32.0.0.0", "request": "GET /images/nav_bg_bottom.jpg HTTP/1.1", "status": 200, "size": 8389} +{"@timestamp": 893966425, "clientip":"12.0.0.0", "request": "GET /english/tickets/images/ticket_bu_infrance2.gif HTTP/1.0", "status": 200, "size": 1134} +{"@timestamp": 893966425, "clientip":"43.0.0.0", "request": "GET /images/32p49815.jpg HTTP/1.0", "status": 200, "size": 11735} +{"@timestamp": 893966425, "clientip":"43.0.0.0", "request": "GET /english/images/team_bu_detail_off.gif HTTP/1.0", "status": 200, "size": 918} +{"@timestamp": 893966425, "clientip":"44.0.0.0", "request": "GET /images/esp.gif HTTP/1.1", "status": 200, "size": 1892} +{"@timestamp": 893966425, "clientip":"38.0.0.0", "request": "GET /english/images/team_group_header_a.gif HTTP/1.0", "status": 200, "size": 639} +{"@timestamp": 893966425, "clientip":"48.0.0.0", "request": "GET /english/help/site.html HTTP/1.0", "status": 200, "size": 7697} +{"@timestamp": 893966425, "clientip":"15.0.0.0", "request": "GET /english/playing/images/downloads.GIF HTTP/1.0", "status": 200, "size": 1294} +{"@timestamp": 893966426, "clientip":"37.0.0.0", "request": "GET /french/images/comp_bu_stage1n.gif HTTP/1.0", "status": 200, "size": 1547} +{"@timestamp": 893966426, "clientip":"25.0.0.0", "request": "GET /french/index.html HTTP/1.1", "status": 200, "size": 954} +{"@timestamp": 893966426, "clientip":"21.0.0.0", "request": "GET /english/images/news_bu_kits_off.gif HTTP/1.1", "status": 200, "size": 1027} +{"@timestamp": 893966426, "clientip":"15.0.0.0", "request": "GET /english/playing/images/links_on.GIF HTTP/1.0", "status": 200, "size": 1393} +{"@timestamp": 893966426, "clientip":"15.0.0.0", "request": "GET /english/playing/images/trivia.GIF HTTP/1.0", "status": 200, "size": 999} +{"@timestamp": 893966426, "clientip":"0.0.0.0", "request": "GET /english/index.html HTTP/1.0", "status": 200, "size": 892} +{"@timestamp": 893966426, "clientip":"32.0.0.0", "request": "GET /english/nav_top_inet.html HTTP/1.1", "status": 200, "size": 374} +{"@timestamp": 893966427, "clientip":"46.0.0.0", "request": "GET /english/venues/cities/images/denis/venue_denn_bg.jpg HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966427, "clientip":"8.0.0.0", "request": "GET /english/images/team_hm_header_shad.gif HTTP/1.0", "status": 200, "size": 1379} +{"@timestamp": 893966427, "clientip":"46.0.0.0", "request": "GET /images/11295.jpg HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966427, "clientip":"46.0.0.0", "request": "GET /images/11294.jpg HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966427, "clientip":"8.0.0.0", "request": "GET /english/images/team_group_header_b.gif HTTP/1.0", "status": 200, "size": 665} +{"@timestamp": 893966427, "clientip":"38.0.0.0", "request": "GET /images/s102330.gif HTTP/1.0", "status": 200, "size": 259} +{"@timestamp": 893966427, "clientip":"15.0.0.0", "request": "GET /english/playing/images/banner2.gif HTTP/1.0", "status": 200, "size": 15328} +{"@timestamp": 893966427, "clientip":"49.0.0.0", "request": "GET /english/news/2704nevi.htm HTTP/1.0", "status": 200, "size": 3132} +{"@timestamp": 893966427, "clientip":"38.0.0.0", "request": "GET /english/images/team_group_header_b.gif HTTP/1.0", "status": 200, "size": 665} +{"@timestamp": 893966427, "clientip":"15.0.0.0", "request": "GET /english/playing/images/fifa_logo_sm.gif HTTP/1.0", "status": 200, "size": 2900} +{"@timestamp": 893966427, "clientip":"39.0.0.0", "request": "GET /english/history/images/infrance.GIF HTTP/1.1", "status": 200, "size": 990} +{"@timestamp": 893966427, "clientip":"47.135.0.0", "request": "GET /french/news/3004bres.htm HTTP/1.0", "status": 200, "size": 5933} +{"@timestamp": 893966428, "clientip":"50.0.0.0", "request": "GET /english/playing/download/images/big.bird.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966428, "clientip":"8.0.0.0", "request": "GET /english/images/team_group_header_e.gif HTTP/1.0", "status": 200, "size": 643} +{"@timestamp": 893966429, "clientip":"41.0.0.0", "request": "GET /english/images/team_bu_detail_on.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966429, "clientip":"30.0.0.0", "request": "GET /english/playing/mascot/images/misc.gif HTTP/1.0", "status": 200, "size": 2997} +{"@timestamp": 893966429, "clientip":"8.0.0.0", "request": "GET /english/images/team_group_header_g.gif HTTP/1.0", "status": 200, "size": 641} +{"@timestamp": 893966429, "clientip":"38.0.0.0", "request": "GET /images/s102382.gif HTTP/1.0", "status": 200, "size": 102} +{"@timestamp": 893966429, "clientip":"12.0.0.0", "request": "GET /english/tickets/10484.htm HTTP/1.0", "status": 200, "size": 16722} +{"@timestamp": 893966429, "clientip":"8.0.0.0", "request": "GET /english/images/team_group_header_h.gif HTTP/1.0", "status": 200, "size": 628} +{"@timestamp": 893966429, "clientip":"12.0.0.0", "request": "GET /english/tickets/images/ticket_bu_abroad2.gif HTTP/1.0", "status": 200, "size": 1486} +{"@timestamp": 893966430, "clientip":"12.0.0.0", "request": "GET /images/11103.gif HTTP/1.0", "status": 200, "size": 513} +{"@timestamp": 893966430, "clientip":"8.0.0.0", "request": "GET /english/images/team_group_header_d.gif HTTP/1.0", "status": 200, "size": 646} +{"@timestamp": 893966430, "clientip":"5.0.0.0", "request": "GET /french/images/archives.gif HTTP/1.1", "status": 200, "size": 569} +{"@timestamp": 893966430, "clientip":"39.0.0.0", "request": "GET /english/history/images/reading.GIF HTTP/1.1", "status": 200, "size": 1171} +{"@timestamp": 893966430, "clientip":"32.0.0.0", "request": "GET /english/ProScroll.class HTTP/1.1", "status": 200, "size": 6507} +{"@timestamp": 893966430, "clientip":"8.0.0.0", "request": "GET /english/images/team_group_header_f.gif HTTP/1.0", "status": 200, "size": 631} +{"@timestamp": 893966430, "clientip":"26.0.0.0", "request": "GET /english/images/nav_sitemap_off.gif HTTP/1.0", "status": 200, "size": 416} +{"@timestamp": 893966430, "clientip":"47.135.0.0", "request": "GET /images/bord_g.gif HTTP/1.0", "status": 200, "size": 231} +{"@timestamp": 893966430, "clientip":"47.135.0.0", "request": "GET /images/bord_d.gif HTTP/1.0", "status": 200, "size": 231} +{"@timestamp": 893966431, "clientip":"12.0.0.0", "request": "GET /images/10982.gif HTTP/1.0", "status": 200, "size": 183} +{"@timestamp": 893966431, "clientip":"38.0.0.0", "request": "GET /images/s102373.gif HTTP/1.0", "status": 200, "size": 142} +{"@timestamp": 893966431, "clientip":"38.0.0.0", "request": "GET /images/s102328.gif HTTP/1.0", "status": 200, "size": 236} +{"@timestamp": 893966431, "clientip":"39.0.0.0", "request": "GET /english/history/past_cups/images/past_header.gif HTTP/1.1", "status": 200, "size": 1989} +{"@timestamp": 893966431, "clientip":"51.0.0.0", "request": "GET /english/teams/teambio160.htm HTTP/1.1", "status": 200, "size": 11097} +{"@timestamp": 893966431, "clientip":"38.0.0.0", "request": "GET /english/images/team_group_header_f.gif HTTP/1.0", "status": 200, "size": 631} +{"@timestamp": 893966431, "clientip":"52.0.0.0", "request": "GET /english/member/body.html HTTP/1.1", "status": 200, "size": 5097} +{"@timestamp": 893966432, "clientip":"10.0.0.0", "request": "GET /english/competition/stage2.htm HTTP/1.0", "status": 200, "size": 16606} +{"@timestamp": 893966432, "clientip":"0.0.0.0", "request": "GET /english/nav_top_inet.html HTTP/1.0", "status": 200, "size": 374} +{"@timestamp": 893966432, "clientip":"53.0.0.0", "request": "GET /english/competition/headtohead75.htm HTTP/1.1", "status": 200, "size": 5152} +{"@timestamp": 893966432, "clientip":"25.0.0.0", "request": "GET /french/nav_top_inet.html HTTP/1.1", "status": 200, "size": 374} +{"@timestamp": 893966432, "clientip":"0.0.0.0", "request": "GET /english/nav_inet.html HTTP/1.0", "status": 200, "size": 2672} +{"@timestamp": 893966432, "clientip":"0.0.0.0", "request": "GET /english/splash_inet.html HTTP/1.0", "status": 200, "size": 3730} +{"@timestamp": 893966432, "clientip":"39.0.0.0", "request": "GET /english/history/images/history_hm_header.gif HTTP/1.1", "status": 200, "size": 688} +{"@timestamp": 893966432, "clientip":"47.0.0.0", "request": "GET /english/index.html HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966432, "clientip":"37.0.0.0", "request": "GET /images/cal_nant.gif HTTP/1.0", "status": 200, "size": 359} +{"@timestamp": 893966432, "clientip":"37.0.0.0", "request": "GET /images/cal_stdenis.gif HTTP/1.0", "status": 200, "size": 402} +{"@timestamp": 893966432, "clientip":"41.0.0.0", "request": "GET /images/32t49807.jpg HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966433, "clientip":"32.0.0.0", "request": "GET /english/images/nav_news_off.gif HTTP/1.1", "status": 200, "size": 853} +{"@timestamp": 893966433, "clientip":"38.0.0.0", "request": "GET /images/s102320.gif HTTP/1.0", "status": 200, "size": 259} +{"@timestamp": 893966433, "clientip":"42.0.0.0", "request": "GET /images/backnews.gif HTTP/1.0", "status": 200, "size": 4573} +{"@timestamp": 893966433, "clientip":"38.0.0.0", "request": "GET /images/s102438.gif HTTP/1.0", "status": 200, "size": 297} +{"@timestamp": 893966433, "clientip":"54.0.0.0", "request": "GET /english/playing/download/images/big.bird.gif HTTP/1.0", "status": 200, "size": 4870} +{"@timestamp": 893966433, "clientip":"0.0.0.0", "request": "GET /english/images/nav_news_off.gif HTTP/1.0", "status": 200, "size": 853} +{"@timestamp": 893966433, "clientip":"38.0.0.0", "request": "GET /images/s102329.gif HTTP/1.0", "status": 200, "size": 159} +{"@timestamp": 893966433, "clientip":"10.0.0.0", "request": "GET /images/comp_stage2_brc_bot.gif HTTP/1.0", "status": 200, "size": 160} +{"@timestamp": 893966434, "clientip":"26.0.0.0", "request": "GET /english/images/nav_home_off.gif HTTP/1.0", "status": 200, "size": 828} +{"@timestamp": 893966434, "clientip":"0.0.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 200, "size": 42} +{"@timestamp": 893966434, "clientip":"41.0.0.0", "request": "GET /english/images/team_hm_header.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966434, "clientip":"32.0.0.0", "request": "GET /english/images/nav_team_off.gif HTTP/1.1", "status": 200, "size": 776} +{"@timestamp": 893966434, "clientip":"38.0.0.0", "request": "GET /english/images/team_group_header_c.gif HTTP/1.0", "status": 200, "size": 670} +{"@timestamp": 893966434, "clientip":"52.0.0.0", "request": "GET /english/member/images/submit.gif HTTP/1.1", "status": 200, "size": 447} +{"@timestamp": 893966434, "clientip":"12.0.0.0", "request": "GET /images/10981.gif HTTP/1.0", "status": 200, "size": 173} +{"@timestamp": 893966434, "clientip":"39.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_34_off.gif HTTP/1.1", "status": 200, "size": 397} +{"@timestamp": 893966434, "clientip":"38.0.0.0", "request": "GET /images/s102377.gif HTTP/1.0", "status": 200, "size": 173} +{"@timestamp": 893966434, "clientip":"12.0.0.0", "request": "GET /images/11116.gif HTTP/1.0", "status": 200, "size": 667} +{"@timestamp": 893966435, "clientip":"26.0.0.0", "request": "GET /english/images/nav_logo_sponsors.gif HTTP/1.0", "status": 200, "size": 1991} +{"@timestamp": 893966435, "clientip":"12.0.0.0", "request": "GET /images/11106.gif HTTP/1.0", "status": 200, "size": 114} +{"@timestamp": 893966435, "clientip":"12.0.0.0", "request": "GET /images/11105.gif HTTP/1.0", "status": 200, "size": 114} +{"@timestamp": 893966435, "clientip":"12.0.0.0", "request": "GET /images/11102.gif HTTP/1.0", "status": 200, "size": 417} +{"@timestamp": 893966435, "clientip":"39.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_38_off.gif HTTP/1.1", "status": 200, "size": 436} +{"@timestamp": 893966435, "clientip":"38.0.0.0", "request": "GET /english/images/team_group_header_h.gif HTTP/1.0", "status": 200, "size": 628} +{"@timestamp": 893966435, "clientip":"25.0.0.0", "request": "GET /images/nav_bg_top.gif HTTP/1.1", "status": 200, "size": 929} +{"@timestamp": 893966435, "clientip":"52.0.0.0", "request": "GET / HTTP/1.1", "status": 200, "size": 8712} +{"@timestamp": 893966435, "clientip":"12.0.0.0", "request": "GET /images/11101.gif HTTP/1.0", "status": 200, "size": 415} +{"@timestamp": 893966435, "clientip":"32.0.0.0", "request": "GET /english/images/nav_venue_off.gif HTTP/1.1", "status": 200, "size": 870} +{"@timestamp": 893966436, "clientip":"55.0.0.0", "request": "GET / HTTP/1.0", "status": 200, "size": 8712} +{"@timestamp": 893966436, "clientip":"56.0.0.0", "request": "GET /images/home_intro.anim.gif HTTP/1.0", "status": 200, "size": 60349} +{"@timestamp": 893966436, "clientip":"38.0.0.0", "request": "GET /images/s102443.gif HTTP/1.0", "status": 200, "size": 165} +{"@timestamp": 893966436, "clientip":"46.0.0.0", "request": "GET /images/paris.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966436, "clientip":"38.0.0.0", "request": "GET /images/s140875.gif HTTP/1.0", "status": 200, "size": 184} +{"@timestamp": 893966436, "clientip":"37.0.0.0", "request": "GET /images/cal_bord.gif HTTP/1.0", "status": 200, "size": 416} +{"@timestamp": 893966436, "clientip":"37.0.0.0", "request": "GET /images/cal_steti.gif HTTP/1.0", "status": 200, "size": 1125} +{"@timestamp": 893966436, "clientip":"37.0.0.0", "request": "GET /images/cal_mont.gif HTTP/1.0", "status": 200, "size": 316} +{"@timestamp": 893966436, "clientip":"51.0.0.0", "request": "GET /images/32t49813.jpg HTTP/1.1", "status": 200, "size": 4714} +{"@timestamp": 893966436, "clientip":"2.0.0.0", "request": "GET /english/images/hm_f98_top.gif HTTP/1.0", "status": 200, "size": 915} +{"@timestamp": 893966437, "clientip":"23.0.0.0", "request": "GET /english/tickets/body.html HTTP/1.0", "status": 200, "size": 2925} +{"@timestamp": 893966437, "clientip":"52.0.0.0", "request": "GET /english/member/images/cfologo.gif HTTP/1.1", "status": 200, "size": 2820} +{"@timestamp": 893966437, "clientip":"52.0.0.0", "request": "GET /english/member/images/member_header.jpg HTTP/1.1", "status": 200, "size": 10457} +{"@timestamp": 893966437, "clientip":"32.0.0.0", "request": "GET /english/news/3004bres.htm HTTP/1.1", "status": 200, "size": 5765} +{"@timestamp": 893966437, "clientip":"38.0.0.0", "request": "GET /images/s102386.gif HTTP/1.0", "status": 200, "size": 279} +{"@timestamp": 893966437, "clientip":"41.0.0.0", "request": "GET /images/news_hm_arw.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966437, "clientip":"39.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_54_off.gif HTTP/1.1", "status": 200, "size": 403} +{"@timestamp": 893966437, "clientip":"38.0.0.0", "request": "GET /images/s102326.gif HTTP/1.0", "status": 200, "size": 248} +{"@timestamp": 893966437, "clientip":"23.0.0.0", "request": "GET /english/tickets/images/ticket_hm_bg.jpg HTTP/1.0", "status": 200, "size": 20929} +{"@timestamp": 893966437, "clientip":"23.0.0.0", "request": "GET /english/tickets/images/ticket_hm_header.gif HTTP/1.0", "status": 200, "size": 1226} +{"@timestamp": 893966437, "clientip":"23.0.0.0", "request": "GET /english/tickets/images/ticket_hm_nav.gif HTTP/1.0", "status": 200, "size": 11253} +{"@timestamp": 893966438, "clientip":"33.0.0.0", "request": "GET /images/home_tool.gif HTTP/1.1", "status": 200, "size": 327} +{"@timestamp": 893966438, "clientip":"38.0.0.0", "request": "GET /english/competition/maincomp.htm HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966438, "clientip":"46.0.0.0", "request": "GET /images/11291.jpg HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966438, "clientip":"47.0.0.0", "request": "GET /english/ProScroll.class HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966438, "clientip":"39.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_58_off.gif HTTP/1.1", "status": 200, "size": 397} +{"@timestamp": 893966439, "clientip":"25.0.0.0", "request": "GET /french/images/nav_news_off.gif HTTP/1.1", "status": 200, "size": 855} +{"@timestamp": 893966439, "clientip":"57.0.0.0", "request": "GET /images/hm_f98_top.gif HTTP/1.0", "status": 200, "size": 915} +{"@timestamp": 893966439, "clientip":"47.0.0.0", "request": "GET /english/images/nav_field_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966439, "clientip":"58.0.0.0", "request": "GET /images/logo_cfo.gif HTTP/1.0", "status": 200, "size": 1504} +{"@timestamp": 893966439, "clientip":"57.0.0.0", "request": "GET /images/ligne1_case5.gif HTTP/1.0", "status": 200, "size": 1018} +{"@timestamp": 893966439, "clientip":"57.0.0.0", "request": "GET /english/images/top_stories.gif HTTP/1.0", "status": 200, "size": 1210} +{"@timestamp": 893966440, "clientip":"47.0.0.0", "request": "GET /images/hm_arw.gif HTTP/1.0", "status": 200, "size": 1050} +{"@timestamp": 893966440, "clientip":"47.0.0.0", "request": "GET /images/hm_linkf.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966440, "clientip":"38.0.0.0", "request": "GET /english/images/team_hm_header_shad.gif HTTP/1.0", "status": 200, "size": 1379} +{"@timestamp": 893966440, "clientip":"38.0.0.0", "request": "GET /english/images/comp_hm_nav.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966441, "clientip":"25.0.0.0", "request": "GET /french/images/nav_venue_off.gif HTTP/1.1", "status": 200, "size": 945} +{"@timestamp": 893966441, "clientip":"47.0.0.0", "request": "GET /images/info.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966441, "clientip":"59.0.0.0", "request": "GET /english/news/2704leag.htm HTTP/1.0", "status": 200, "size": 4801} +{"@timestamp": 893966441, "clientip":"60.0.0.0", "request": "GET /images/10531.jpg HTTP/1.0", "status": 200, "size": 16554} +{"@timestamp": 893966441, "clientip":"47.0.0.0", "request": "GET /english/images/nav_hosts_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966441, "clientip":"47.0.0.0", "request": "GET /images/dot.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966441, "clientip":"47.0.0.0", "request": "GET /english/images/nav_home_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966441, "clientip":"47.0.0.0", "request": "GET /images/hm_day_e.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966441, "clientip":"47.0.0.0", "request": "GET /english/images/nav_store_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966442, "clientip":"61.0.0.0", "request": "GET /english/venues/cities/montpellier.html HTTP/1.0", "status": 200, "size": 9686} +{"@timestamp": 893966442, "clientip":"62.0.0.0", "request": "GET /english/playing/links.html HTTP/1.0", "status": 200, "size": 18694} +{"@timestamp": 893966442, "clientip":"23.0.0.0", "request": "GET /images/arw_lk.gif HTTP/1.0", "status": 200, "size": 669} +{"@timestamp": 893966442, "clientip":"63.0.0.0", "request": "GET /images/home_bg_stars.gif HTTP/1.0", "status": 200, "size": 2557} +{"@timestamp": 893966442, "clientip":"63.0.0.0", "request": "GET /images/home_fr_phrase.gif HTTP/1.0", "status": 200, "size": 2843} +{"@timestamp": 893966442, "clientip":"63.0.0.0", "request": "GET /images/home_logo.gif HTTP/1.0", "status": 200, "size": 3401} +{"@timestamp": 893966442, "clientip":"47.0.0.0", "request": "GET /english/images/nav_tickets_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966442, "clientip":"47.0.0.0", "request": "GET /english/images/nav_team_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966442, "clientip":"46.0.0.0", "request": "GET /images/sdffr.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966442, "clientip":"47.0.0.0", "request": "GET /english/images/nav_history_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966442, "clientip":"39.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_62_off.gif HTTP/1.1", "status": 200, "size": 437} +{"@timestamp": 893966442, "clientip":"39.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_66_off.gif HTTP/1.1", "status": 200, "size": 435} +{"@timestamp": 893966443, "clientip":"64.0.0.0", "request": "GET /images/home_tool.gif HTTP/1.0", "status": 200, "size": 327} +{"@timestamp": 893966443, "clientip":"38.0.0.0", "request": "GET /english/images/comp_hm_nav.gif HTTP/1.0", "status": 206, "size": 10902} +{"@timestamp": 893966443, "clientip":"25.0.0.0", "request": "GET /french/images/nav_tickets_off.gif HTTP/1.1", "status": 200, "size": 965} +{"@timestamp": 893966443, "clientip":"57.0.0.0", "request": "GET /images/case5.gif HTTP/1.0", "status": 200, "size": 1362} +{"@timestamp": 893966443, "clientip":"65.0.0.0", "request": "GET /english/index.html HTTP/1.0", "status": 200, "size": 892} +{"@timestamp": 893966443, "clientip":"30.0.0.0", "request": "GET /english/playing/mascot/images/button.03.gif HTTP/1.0", "status": 200, "size": 893} +{"@timestamp": 893966443, "clientip":"57.0.0.0", "request": "GET /images/bord_stories.gif HTTP/1.0", "status": 200, "size": 520} +{"@timestamp": 893966443, "clientip":"57.0.0.0", "request": "GET /images/bord_stories01.gif HTTP/1.0", "status": 200, "size": 333} +{"@timestamp": 893966444, "clientip":"65.0.0.0", "request": "GET /english/nav_inet.html HTTP/1.0", "status": 200, "size": 2672} +{"@timestamp": 893966444, "clientip":"65.0.0.0", "request": "GET /english/splash_inet.html HTTP/1.0", "status": 200, "size": 3730} +{"@timestamp": 893966444, "clientip":"39.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_70_off.gif HTTP/1.1", "status": 200, "size": 409} +{"@timestamp": 893966444, "clientip":"65.0.0.0", "request": "GET /images/nav_bg_bottom.jpg HTTP/1.0", "status": 200, "size": 8389} +{"@timestamp": 893966444, "clientip":"65.0.0.0", "request": "GET /english/images/nav_team_off.gif HTTP/1.0", "status": 200, "size": 776} +{"@timestamp": 893966444, "clientip":"6.0.0.0", "request": "GET /french/venues/body.html HTTP/1.0", "status": 200, "size": 2042} +{"@timestamp": 893966444, "clientip":"65.0.0.0", "request": "GET /english/images/nav_news_off.gif HTTP/1.0", "status": 200, "size": 853} +{"@timestamp": 893966444, "clientip":"65.0.0.0", "request": "GET /images/nav_bg_top.gif HTTP/1.0", "status": 200, "size": 929} +{"@timestamp": 893966444, "clientip":"64.0.0.0", "request": "GET /images/home_sponsor.gif HTTP/1.0", "status": 200, "size": 2491} +{"@timestamp": 893966444, "clientip":"65.0.0.0", "request": "GET /images/logo_cfo.gif HTTP/1.0", "status": 200, "size": 1504} +{"@timestamp": 893966444, "clientip":"65.0.0.0", "request": "GET /images/hm_nbg.jpg HTTP/1.0", "status": 200, "size": 33665} +{"@timestamp": 893966444, "clientip":"65.0.0.0", "request": "GET /english/images/nav_comp_off.gif HTTP/1.0", "status": 200, "size": 994} +{"@timestamp": 893966444, "clientip":"66.0.0.0", "request": "GET / HTTP/1.0", "status": 200, "size": 8712} +{"@timestamp": 893966444, "clientip":"62.0.0.0", "request": "GET /english/playing/images/backg.gif HTTP/1.0", "status": 200, "size": 1462} +{"@timestamp": 893966444, "clientip":"67.0.0.0", "request": "GET /english/individuals/player389.htm HTTP/1.0", "status": 200, "size": 7001} +{"@timestamp": 893966444, "clientip":"47.0.0.0", "request": "GET /english/teams/teamgroup.htm HTTP/1.0", "status": 200, "size": 11971} +{"@timestamp": 893966445, "clientip":"7.0.0.0", "request": "GET /english/playing/images/play_header.gif HTTP/1.0", "status": 200, "size": 2262} +{"@timestamp": 893966445, "clientip":"30.0.0.0", "request": "GET /english/playing/mascot/images/footix.test.gif HTTP/1.0", "status": 200, "size": 20385} +{"@timestamp": 893966445, "clientip":"6.0.0.0", "request": "GET /french/venues/images/Venue_map_mid_off.gif HTTP/1.0", "status": 200, "size": 9911} +{"@timestamp": 893966445, "clientip":"25.0.0.0", "request": "GET /french/images/nav_field_off.gif HTTP/1.1", "status": 200, "size": 982} +{"@timestamp": 893966445, "clientip":"64.0.0.0", "request": "GET /images/home_intro.anim.gif HTTP/1.0", "status": 200, "size": 60349} +{"@timestamp": 893966445, "clientip":"6.0.0.0", "request": "GET /french/venues/images/Venue_map_top_off.gif HTTP/1.0", "status": 200, "size": 8369} +{"@timestamp": 893966445, "clientip":"6.0.0.0", "request": "GET /french/venues/images/venue_hm_nav.gif HTTP/1.0", "status": 200, "size": 7644} +{"@timestamp": 893966445, "clientip":"68.0.0.0", "request": "GET /english/venues/cities/images/montpellier/venue_mont_header.jpg HTTP/1.0", "status": 200, "size": 11562} +{"@timestamp": 893966445, "clientip":"69.0.0.0", "request": "GET /english/history/history_of/images/cup/cup.gif HTTP/1.0", "status": 200, "size": 5469} +{"@timestamp": 893966445, "clientip":"34.0.0.0", "request": "GET /english/history/body.html HTTP/1.0", "status": 200, "size": 2909} +{"@timestamp": 893966445, "clientip":"70.0.0.0", "request": "GET /english/venues/cities/images/marseille/mars_c.gif HTTP/1.0", "status": 200, "size": 369} +{"@timestamp": 893966446, "clientip":"64.0.0.0", "request": "GET /images/home_fr_button.gif HTTP/1.0", "status": 200, "size": 2140} +{"@timestamp": 893966446, "clientip":"66.0.0.0", "request": "GET / HTTP/1.0", "status": 200, "size": 8712} +{"@timestamp": 893966446, "clientip":"6.0.0.0", "request": "GET /french/venues/images/Venue_map_bot_off.gif HTTP/1.0", "status": 200, "size": 7397} +{"@timestamp": 893966446, "clientip":"71.0.0.0", "request": "GET /english/teams/teambio148.htm HTTP/1.0", "status": 200, "size": 10857} +{"@timestamp": 893966446, "clientip":"72.0.0.0", "request": "GET / HTTP/1.0", "status": 200, "size": 8712} +{"@timestamp": 893966446, "clientip":"39.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_78_off.gif HTTP/1.1", "status": 200, "size": 427} +{"@timestamp": 893966446, "clientip":"67.0.0.0", "request": "GET /images/102373.gif HTTP/1.0", "status": 200, "size": 1703} +{"@timestamp": 893966446, "clientip":"47.0.0.0", "request": "GET /english/images/teams_bu_group_on.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966446, "clientip":"67.0.0.0", "request": "GET /images/32p49804.jpg HTTP/1.0", "status": 200, "size": 11023} +{"@timestamp": 893966446, "clientip":"47.0.0.0", "request": "GET /english/images/teams_bu_confed_off.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966447, "clientip":"71.0.0.0", "request": "GET /images/32t49812.jpg HTTP/1.0", "status": 200, "size": 4132} +{"@timestamp": 893966447, "clientip":"71.0.0.0", "request": "GET /images/ger.gif HTTP/1.0", "status": 200, "size": 1603} +{"@timestamp": 893966447, "clientip":"39.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_82_off.gif HTTP/1.1", "status": 200, "size": 433} +{"@timestamp": 893966447, "clientip":"25.0.0.0", "request": "GET /french/images/nav_store_off.gif HTTP/1.1", "status": 200, "size": 976} +{"@timestamp": 893966447, "clientip":"65.0.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 200, "size": 42} +{"@timestamp": 893966447, "clientip":"65.0.0.0", "request": "GET /english/images/hm_official.gif HTTP/1.0", "status": 200, "size": 1807} +{"@timestamp": 893966447, "clientip":"47.0.0.0", "request": "GET /images/s102325.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966447, "clientip":"65.0.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 200, "size": 42} +{"@timestamp": 893966447, "clientip":"47.0.0.0", "request": "GET /english/images/team_group_header_a.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966447, "clientip":"65.0.0.0", "request": "GET /english/images/nav_field_off.gif HTTP/1.0", "status": 200, "size": 1005} +{"@timestamp": 893966447, "clientip":"65.0.0.0", "request": "GET /english/images/nav_history_off.gif HTTP/1.0", "status": 200, "size": 914} +{"@timestamp": 893966448, "clientip":"65.0.0.0", "request": "GET /english/ProScroll.class HTTP/1.0", "status": 200, "size": 6507} +{"@timestamp": 893966448, "clientip":"47.0.0.0", "request": "GET /images/s102424.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966448, "clientip":"39.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_86_off.gif HTTP/1.1", "status": 200, "size": 409} +{"@timestamp": 893966448, "clientip":"47.0.0.0", "request": "GET /images/s102487.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966448, "clientip":"65.0.0.0", "request": "GET /english/images/nav_hosts_off.gif HTTP/1.0", "status": 200, "size": 914} +{"@timestamp": 893966448, "clientip":"65.0.0.0", "request": "GET /images/hm_brdl.gif HTTP/1.0", "status": 200, "size": 208} +{"@timestamp": 893966448, "clientip":"73.0.0.0", "request": "GET / HTTP/1.1", "status": 304, "size": 0} +{"@timestamp": 893966448, "clientip":"62.0.0.0", "request": "GET /english/playing/images/links_on.GIF HTTP/1.0", "status": 200, "size": 1393} +{"@timestamp": 893966448, "clientip":"62.0.0.0", "request": "GET /english/playing/images/banner2.gif HTTP/1.0", "status": 200, "size": 15328} +{"@timestamp": 893966448, "clientip":"66.0.0.0", "request": "GET /images/home_bg_stars.gif HTTP/1.0", "status": 200, "size": 2557} +{"@timestamp": 893966448, "clientip":"57.0.0.0", "request": "GET /english/images/lateb_new.gif HTTP/1.0", "status": 200, "size": 1431} +{"@timestamp": 893966448, "clientip":"47.0.0.0", "request": "GET /images/s102477.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966448, "clientip":"65.0.0.0", "request": "GET /english/images/nav_sitemap_off.gif HTTP/1.0", "status": 200, "size": 416} +{"@timestamp": 893966448, "clientip":"65.0.0.0", "request": "GET /english/images/nav_logo_sponsors.gif HTTP/1.0", "status": 200, "size": 1991} +{"@timestamp": 893966448, "clientip":"47.0.0.0", "request": "GET /english/images/team_group_header_e.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966448, "clientip":"65.0.0.0", "request": "GET /images/hm_linkf.gif HTTP/1.0", "status": 200, "size": 123} +{"@timestamp": 893966449, "clientip":"65.0.0.0", "request": "GET /images/info.gif HTTP/1.0", "status": 200, "size": 1251} +{"@timestamp": 893966449, "clientip":"74.0.0.0", "request": "GET / HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966449, "clientip":"75.0.0.0", "request": "GET /english/competition/stage2.htm HTTP/1.0", "status": 200, "size": 16606} +{"@timestamp": 893966449, "clientip":"47.0.0.0", "request": "GET /images/s102324.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966449, "clientip":"25.0.0.0", "request": "GET /french/images/nav_sitemap_off.gif HTTP/1.1", "status": 200, "size": 413} +{"@timestamp": 893966449, "clientip":"65.0.0.0", "request": "GET /images/hm_day_e.gif HTTP/1.0", "status": 200, "size": 499} +{"@timestamp": 893966449, "clientip":"14.0.0.0", "request": "GET /images/home_bg_stars.gif HTTP/1.1", "status": 200, "size": 2557} +{"@timestamp": 893966449, "clientip":"47.0.0.0", "request": "GET /images/s102376.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966449, "clientip":"40.0.0.0", "request": "GET /english/history/past_cups/france38.html HTTP/1.1", "status": 200, "size": 12669} +{"@timestamp": 893966449, "clientip":"73.0.0.0", "request": "GET /images/home_fr_phrase.gif HTTP/1.1", "status": 200, "size": 2843} +{"@timestamp": 893966449, "clientip":"73.0.0.0", "request": "GET /images/home_intro.anim.gif HTTP/1.1", "status": 200, "size": 60349} +{"@timestamp": 893966449, "clientip":"65.0.0.0", "request": "GET /images/hm_brdr.gif HTTP/1.0", "status": 200, "size": 235} +{"@timestamp": 893966449, "clientip":"65.0.0.0", "request": "GET /english/images/nav_store_off.gif HTTP/1.0", "status": 200, "size": 934} +{"@timestamp": 893966449, "clientip":"76.0.0.0", "request": "GET /english/competition/stage1.htm HTTP/1.1", "status": 200, "size": 36783} +{"@timestamp": 893966449, "clientip":"65.0.0.0", "request": "GET /images/hm_arw.gif HTTP/1.0", "status": 200, "size": 1050} +{"@timestamp": 893966449, "clientip":"53.0.0.0", "request": "GET /english/competition/headtohead76.htm HTTP/1.1", "status": 200, "size": 16909} +{"@timestamp": 893966449, "clientip":"65.0.0.0", "request": "GET /images/dot.gif HTTP/1.0", "status": 200, "size": 43} +{"@timestamp": 893966449, "clientip":"2.0.0.0", "request": "GET /english/images/nav_history_off.gif HTTP/1.0", "status": 200, "size": 914} +{"@timestamp": 893966449, "clientip":"73.0.0.0", "request": "GET /images/home_eng_phrase.gif HTTP/1.1", "status": 200, "size": 2861} +{"@timestamp": 893966450, "clientip":"2.0.0.0", "request": "GET /english/images/nav_field_off.gif HTTP/1.0", "status": 200, "size": 1005} +{"@timestamp": 893966450, "clientip":"66.0.0.0", "request": "GET /images/home_fr_phrase.gif HTTP/1.0", "status": 200, "size": 2843} +{"@timestamp": 893966450, "clientip":"1.0.0.0", "request": "GET /english/images/hm_official.gif HTTP/1.0", "status": 200, "size": 1807} +{"@timestamp": 893966450, "clientip":"1.0.0.0", "request": "GET /images/nav_bg_top.gif HTTP/1.0", "status": 200, "size": 929} +{"@timestamp": 893966450, "clientip":"66.0.0.0", "request": "GET /images/home_intro.anim.gif HTTP/1.0", "status": 200, "size": 60349} +{"@timestamp": 893966450, "clientip":"1.0.0.0", "request": "GET /images/logo_cfo.gif HTTP/1.0", "status": 200, "size": 1504} +{"@timestamp": 893966450, "clientip":"1.0.0.0", "request": "GET /images/space.gif HTTP/1.0", "status": 200, "size": 42} +{"@timestamp": 893966450, "clientip":"77.0.0.0", "request": "GET /english/images/france98b.gif HTTP/1.0", "status": 200, "size": 2122} +{"@timestamp": 893966450, "clientip":"1.0.0.0", "request": "GET /english/ProScroll.class HTTP/1.0", "status": 200, "size": 6507} +{"@timestamp": 893966450, "clientip":"77.0.0.0", "request": "GET /english/images/space.gif HTTP/1.0", "status": 200, "size": 42} +{"@timestamp": 893966450, "clientip":"57.0.0.0", "request": "GET /images/ligne4_latebreak.gif HTTP/1.0", "status": 200, "size": 1056} +{"@timestamp": 893966450, "clientip":"47.0.0.0", "request": "GET /images/s102373.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966450, "clientip":"47.0.0.0", "request": "GET /images/s102443.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966450, "clientip":"232.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"@timestamp": 893966451, "clientip":"66.0.0.0", "request": "GET /images/home_eng_phrase.gif HTTP/1.0", "status": 200, "size": 2861} +{"@timestamp": 893966451, "clientip":"75.0.0.0", "request": "GET /english/images/comp_bu_stage2n_on.gif HTTP/1.0", "status": 200, "size": 996} +{"@timestamp": 893966451, "clientip":"57.0.0.0", "request": "GET /images/ligneb.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966451, "clientip":"39.0.0.0", "request": "GET /english/history/past_cups/images/past_bu_94_off.gif HTTP/1.1", "status": 200, "size": 432} +{"@timestamp": 893966451, "clientip":"77.0.0.0", "request": "GET /english/images/comp_bu_stage1n.gif HTTP/1.0", "status": 200, "size": 1548} +{"@timestamp": 893966451, "clientip":"47.0.0.0", "request": "GET /images/s140875.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966451, "clientip":"47.0.0.0", "request": "GET /images/s102321.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966451, "clientip":"6.0.0.0", "request": "GET /french/venues/cities/images/denis/venue_denn_bg.jpg HTTP/1.0", "status": 200, "size": 21003} +{"@timestamp": 893966451, "clientip":"6.0.0.0", "request": "GET /french/venues/images/venue_header.gif HTTP/1.0", "status": 200, "size": 740} +{"@timestamp": 893966451, "clientip":"75.0.0.0", "request": "GET /images/comp_stage2_brc_top.gif HTTP/1.0", "status": 200, "size": 163} +{"@timestamp": 893966451, "clientip":"75.0.0.0", "request": "GET /images/comp_stage2_brc_topr.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966451, "clientip":"25.0.0.0", "request": "GET /images/hm_nbg.jpg HTTP/1.1", "status": 200, "size": 33665} +{"@timestamp": 893966451, "clientip":"6.0.0.0", "request": "GET /french/venues/images/venue_bu_city_on.gif HTTP/1.0", "status": 200, "size": 1061} +{"@timestamp": 893966451, "clientip":"66.0.0.0", "request": "GET /images/home_tool.gif HTTP/1.0", "status": 200, "size": 327} +{"@timestamp": 893966451, "clientip":"47.0.0.0", "request": "GET /english/images/team_group_header_g.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966451, "clientip":"47.0.0.0", "request": "GET /images/s102357.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966451, "clientip":"75.0.0.0", "request": "GET /images/comp_stage2_brc_bot.gif HTTP/1.0", "status": 200, "size": 160} +{"@timestamp": 893966451, "clientip":"69.0.0.0", "request": "GET /english/history/history_of/images/cup/coup_du_mondtxt.gif HTTP/1.0", "status": 200, "size": 5053} +{"@timestamp": 893966451, "clientip":"73.0.0.0", "request": "GET /images/home_fr_button.gif HTTP/1.1", "status": 200, "size": 2140} +{"@timestamp": 893966451, "clientip":"1.0.0.0", "request": "GET /images/nav_bg_bottom.jpg HTTP/1.0", "status": 200, "size": 8389} +{"@timestamp": 893966451, "clientip":"1.0.0.0", "request": "GET /english/images/nav_news_off.gif HTTP/1.0", "status": 200, "size": 853} +{"@timestamp": 893966451, "clientip":"1.0.0.0", "request": "GET /english/images/nav_comp_off.gif HTTP/1.0", "status": 200, "size": 994} +{"@timestamp": 893966451, "clientip":"47.0.0.0", "request": "GET /english/images/team_hm_header_shad.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966452, "clientip":"47.0.0.0", "request": "GET /english/images/team_group_header_c.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966452, "clientip":"78.0.0.0", "request": "GET /english/venues/cities/images/montpellier/12eglise.jpg HTTP/1.0", "status": 200, "size": 3035} +{"@timestamp": 893966452, "clientip":"47.0.0.0", "request": "GET /images/s102383.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966452, "clientip":"6.0.0.0", "request": "GET /french/venues/cities/images/denis/12maisons.jpg HTTP/1.0", "status": 200, "size": 2559} +{"@timestamp": 893966452, "clientip":"75.0.0.0", "request": "GET /images/comp_stage2_brc_botr.gif HTTP/1.0", "status": 200, "size": 158} +{"@timestamp": 893966452, "clientip":"6.0.0.0", "request": "GET /french/venues/cities/images/denis/denis_a.gif HTTP/1.0", "status": 200, "size": 520} +{"@timestamp": 893966452, "clientip":"70.0.0.0", "request": "GET /english/venues/cities/images/montpellier/17mystery.gif HTTP/1.0", "status": 200, "size": 542} +{"@timestamp": 893966452, "clientip":"79.0.0.0", "request": "GET /english/history/images/history_hm_bg2.jpg HTTP/1.0", "status": 200, "size": 22515} +{"@timestamp": 893966452, "clientip":"79.0.0.0", "request": "GET /english/history/images/history_hm_header.gif HTTP/1.0", "status": 200, "size": 688} +{"@timestamp": 893966452, "clientip":"2.0.0.0", "request": "GET /english/images/nav_home_off.gif HTTP/1.0", "status": 200, "size": 828} +{"@timestamp": 893966452, "clientip":"1.0.0.0", "request": "GET /images/hm_day_e.gif HTTP/1.0", "status": 200, "size": 499} +{"@timestamp": 893966452, "clientip":"70.0.0.0", "request": "GET /english/venues/cities/images/montpellier/18corum.gif HTTP/1.0", "status": 200, "size": 650} +{"@timestamp": 893966452, "clientip":"66.0.0.0", "request": "GET /images/home_eng_button.gif HTTP/1.0", "status": 200, "size": 1927} +{"@timestamp": 893966452, "clientip":"80.0.0.0", "request": "GET /english/news/player01.htm HTTP/1.0", "status": 200, "size": 42591} +{"@timestamp": 893966452, "clientip":"79.0.0.0", "request": "GET /english/history/images/history_hm_3094.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966452, "clientip":"79.0.0.0", "request": "GET /english/history/images/history_hm_posters.jpg HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966452, "clientip":"39.0.0.0", "request": "GET /english/history/past_cups/images/past_bracket_top.gif HTTP/1.1", "status": 200, "size": 289} +{"@timestamp": 893966452, "clientip":"2.0.0.0", "request": "GET /english/images/nav_tickets_off.gif HTTP/1.0", "status": 200, "size": 937} +{"@timestamp": 893966452, "clientip":"47.0.0.0", "request": "GET /images/s102380.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966452, "clientip":"6.0.0.0", "request": "GET /french/venues/cities/images/denis/16tradition.gif HTTP/1.0", "status": 200, "size": 387} +{"@timestamp": 893966452, "clientip":"66.0.0.0", "request": "GET /images/home_sponsor.gif HTTP/1.0", "status": 200, "size": 2491} +{"@timestamp": 893966453, "clientip":"70.0.0.0", "request": "GET /english/venues/cities/images/marseille/mars_t.gif HTTP/1.0", "status": 200, "size": 353} +{"@timestamp": 893966453, "clientip":"73.0.0.0", "request": "GET /images/home_tool.gif HTTP/1.1", "status": 200, "size": 327} +{"@timestamp": 893966453, "clientip":"81.0.0.0", "request": "GET / HTTP/1.0", "status": 200, "size": 8712} +{"@timestamp": 893966453, "clientip":"5.0.0.0", "request": "GET /french/news/3004arge.htm HTTP/1.1", "status": 200, "size": 4869} +{"@timestamp": 893966453, "clientip":"78.0.0.0", "request": "GET /english/venues/cities/images/marseille/mars_i.gif HTTP/1.0", "status": 200, "size": 314} +{"@timestamp": 893966453, "clientip":"1.0.0.0", "request": "GET /images/hm_arw.gif HTTP/1.0", "status": 200, "size": 1050} +{"@timestamp": 893966453, "clientip":"77.0.0.0", "request": "GET /english/images/comp_bu_stage2n.gif HTTP/1.0", "status": 200, "size": 984} +{"@timestamp": 893966453, "clientip":"2.0.0.0", "request": "GET /english/images/nav_logo_sponsors.gif HTTP/1.0", "status": 200, "size": 1991} +{"@timestamp": 893966453, "clientip":"62.0.0.0", "request": "GET /english/playing/images/fifa_logo_sm.gif HTTP/1.0", "status": 200, "size": 2900} +{"@timestamp": 893966453, "clientip":"68.0.0.0", "request": "GET /english/venues/cities/images/marseille/mars_a.gif HTTP/1.0", "status": 200, "size": 434} +{"@timestamp": 893966453, "clientip":"47.0.0.0", "request": "GET /images/s102320.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966453, "clientip":"3.0.0.0", "request": "GET /english/images/fpnewstop.gif HTTP/1.0", "status": 200, "size": 568} +{"@timestamp": 893966453, "clientip":"1.0.0.0", "request": "GET /english/images/nav_venue_off.gif HTTP/1.0", "status": 200, "size": 870} +{"@timestamp": 893966453, "clientip":"3.0.0.0", "request": "GET /english/images/news_btn_letter_off.gif HTTP/1.0", "status": 200, "size": 852} +{"@timestamp": 893966453, "clientip":"47.0.0.0", "request": "GET /images/s102353.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966453, "clientip":"6.0.0.0", "request": "GET /french/venues/cities/images/denis/17commercial.gif HTTP/1.0", "status": 200, "size": 614} +{"@timestamp": 893966453, "clientip":"57.0.0.0", "request": "GET /images/ligne.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966453, "clientip":"47.0.0.0", "request": "GET /images/s102329.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966453, "clientip":"3.0.0.0", "request": "GET /english/images/news_btn_kits_off.gif HTTP/1.0", "status": 200, "size": 933} +{"@timestamp": 893966453, "clientip":"57.0.0.0", "request": "GET /images/ligne01.gif HTTP/1.0", "status": 200, "size": 169} +{"@timestamp": 893966453, "clientip":"7.0.0.0", "request": "GET /english/playing/images/links.GIF HTTP/1.0", "status": 200, "size": 1394} +{"@timestamp": 893966453, "clientip":"47.0.0.0", "request": "GET /images/s102377.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966454, "clientip":"47.0.0.0", "request": "GET /images/s102328.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966454, "clientip":"66.0.0.0", "request": "GET /images/home_fr_button.gif HTTP/1.0", "status": 200, "size": 2140} +{"@timestamp": 893966454, "clientip":"47.0.0.0", "request": "GET /images/s102323.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966454, "clientip":"34.0.0.0", "request": "GET /english/history/images/history_hm_3094.gif HTTP/1.0", "status": 200, "size": 1031} +{"@timestamp": 893966454, "clientip":"69.0.0.0", "request": "GET /english/history/history_of/images/cup/trophytxt.gif HTTP/1.0", "status": 200, "size": 1425} +{"@timestamp": 893966454, "clientip":"6.0.0.0", "request": "GET /french/venues/cities/images/denis/19historical.gif HTTP/1.0", "status": 200, "size": 616} +{"@timestamp": 893966454, "clientip":"57.0.0.0", "request": "GET /images/base.gif HTTP/1.0", "status": 200, "size": 366} +{"@timestamp": 893966454, "clientip":"79.0.0.0", "request": "GET /english/history/images/history_hm_nav.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966454, "clientip":"47.0.0.0", "request": "GET /english/images/team_group_header_f.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966454, "clientip":"6.0.0.0", "request": "GET /french/venues/cities/images/denis/18collect.gif HTTP/1.0", "status": 200, "size": 714} +{"@timestamp": 893966454, "clientip":"47.0.0.0", "request": "GET /images/s102442.gif HTTP/1.0", "status": 304, "size": 0} +{"@timestamp": 893966455, "clientip":"34.0.0.0", "request": "GET /english/history/images/history_hm_nav.gif HTTP/1.0", "status": 200, "size": 18328} diff --git a/regression-test/suites/external_table_p0/paimon/paimon_base_filesystem.groovy b/regression-test/suites/external_table_p0/paimon/paimon_base_filesystem.groovy index 7be15f94243e7b..0e00cd8fb7a8bc 100644 --- a/regression-test/suites/external_table_p0/paimon/paimon_base_filesystem.groovy +++ b/regression-test/suites/external_table_p0/paimon/paimon_base_filesystem.groovy @@ -29,6 +29,7 @@ suite("paimon_base_filesystem", "p0,external,doris,external_docker,external_dock String s3ak = getS3AK() String s3sk = getS3SK() + def s3Endpoint = getS3Endpoint() def cos = """select c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c18 from ${catalog_cos}.zd.all_table order by c18""" def oss = """select * from ${catalog_oss}.paimonossdb1.test_tableoss order by a""" @@ -48,9 +49,9 @@ suite("paimon_base_filesystem", "p0,external,doris,external_docker,external_dock create catalog if not exists ${catalog_oss} properties ( "type" = "paimon", "warehouse" = "oss://paimon-zd/paimonoss", - "oss.endpoint"="oss-cn-beijing.aliyuncs.com", "oss.access_key"="${ak}", - "oss.secret_key"="${sk}" + "oss.secret_key"="${sk}", + "oss.endpoint"="oss-cn-beijing.aliyuncs.com" ); """ logger.info("catalog " + catalog_cos + " created") diff --git a/regression-test/suites/external_table_p2/hive/test_hive_write_insert_s3.groovy b/regression-test/suites/external_table_p2/hive/test_hive_write_insert_s3.groovy index 87633ba1b09a5f..cf9fea67cbd685 100644 --- a/regression-test/suites/external_table_p2/hive/test_hive_write_insert_s3.groovy +++ b/regression-test/suites/external_table_p2/hive/test_hive_write_insert_s3.groovy @@ -17,6 +17,7 @@ suite("test_hive_write_insert_s3", "p2,external,hive,external_remote,external_remote_hive") { def format_compressions = ["parquet_snappy"] + def s3BucketName = getS3BucketName() def q01 = { String format_compression, String catalog_name -> logger.info("hive sql: " + """ truncate table all_types_${format_compression}_s3; """) @@ -76,8 +77,8 @@ suite("test_hive_write_insert_s3", "p2,external,hive,external_remote,external_re hive_remote """ DROP TABLE IF EXISTS all_types_par_${format_compression}_s3_${catalog_name}_q02; """ logger.info("hive sql: " + """ CREATE TABLE IF NOT EXISTS all_types_par_${format_compression}_s3_${catalog_name}_q02 like all_types_par_${format_compression}_s3; """) hive_remote """ CREATE TABLE IF NOT EXISTS all_types_par_${format_compression}_s3_${catalog_name}_q02 like all_types_par_${format_compression}_s3; """ - logger.info("hive sql: " + """ ALTER TABLE all_types_par_${format_compression}_s3_${catalog_name}_q02 SET LOCATION 'cosn://doris-build-1308700295/regression/write/data/all_types_par_${format_compression}_s3_${catalog_name}_q02'; """) - hive_remote """ ALTER TABLE all_types_par_${format_compression}_s3_${catalog_name}_q02 SET LOCATION 'cosn://doris-build-1308700295/regression/write/data/all_types_par_${format_compression}_s3_${catalog_name}_q02'; """ + logger.info("hive sql: " + """ ALTER TABLE all_types_par_${format_compression}_s3_${catalog_name}_q02 SET LOCATION 'cosn://${s3BucketName}/regression/write/data/all_types_par_${format_compression}_s3_${catalog_name}_q02'; """) + hive_remote """ ALTER TABLE all_types_par_${format_compression}_s3_${catalog_name}_q02 SET LOCATION 'cosn://${s3BucketName}/regression/write/data/all_types_par_${format_compression}_s3_${catalog_name}_q02'; """ sql """refresh catalog ${catalog_name};""" sql """ diff --git a/regression-test/suites/github_events_p2/load.groovy b/regression-test/suites/github_events_p2/load.groovy index dc2e0dbb97505c..92a588a2214b29 100644 --- a/regression-test/suites/github_events_p2/load.groovy +++ b/regression-test/suites/github_events_p2/load.groovy @@ -31,7 +31,7 @@ suite("load") { ak "${getS3AK()}" sk "${getS3SK()}" endpoint "http://${getS3Endpoint()}" - region "ap-beijing" + region "${getS3Region()}" repository "regression_test_github_events" snapshot "github_events" timestamp "2022-03-23-12-19-51" diff --git a/regression-test/suites/inverted_index_p0/test_index_rqg_bug3.groovy b/regression-test/suites/inverted_index_p0/test_index_rqg_bug3.groovy new file mode 100644 index 00000000000000..eb49100effd1ee --- /dev/null +++ b/regression-test/suites/inverted_index_p0/test_index_rqg_bug3.groovy @@ -0,0 +1,81 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +suite("test_index_rqg_bug3", "test_index_rqg_bug3"){ + def table1 = "test_index_rqg_bug3" + + sql "drop table if exists ${table1}" + + sql """ + CREATE TABLE ${table1} ( + `col_int_undef_signed_not_null` INT NOT NULL, + `col_date_undef_signed_not_null` DATE NOT NULL, + `col_bigint_undef_signed_not_null_index_inverted` BIGINT NOT NULL, + `col_bigint_undef_signed_not_null` BIGINT NOT NULL, + `col_int_undef_signed` INT NULL, + `col_int_undef_signed_index_inverted` INT NULL, + `col_int_undef_signed_not_null_index_inverted` INT NOT NULL, + `col_bigint_undef_signed` BIGINT NULL, + `col_bigint_undef_signed_index_inverted` BIGINT NULL, + `col_date_undef_signed` DATE NULL, + `col_date_undef_signed_index_inverted` DATE NULL, + `col_date_undef_signed_not_null_index_inverted` DATE NOT NULL, + `col_varchar_10__undef_signed` VARCHAR(10) NULL, + `col_varchar_10__undef_signed_index_inverted` VARCHAR(10) NULL, + `col_varchar_10__undef_signed_not_null` VARCHAR(10) NOT NULL, + `col_varchar_10__undef_signed_not_null_index_inverted` VARCHAR(10) NOT NULL, + `col_varchar_1024__undef_signed` VARCHAR(1024) NULL, + `col_varchar_1024__undef_signed_index_inverted` VARCHAR(1024) NULL, + `col_varchar_1024__undef_signed_not_null` VARCHAR(1024) NOT NULL, + `col_varchar_1024__undef_signed_not_null_index_inverted` VARCHAR(1024) NOT NULL, + `pk` INT NULL, + INDEX col_int_undef_signed_index_inverted_idx (`col_int_undef_signed_index_inverted`) USING INVERTED, + INDEX col_int_undef_signed_not_null_index_inverted_idx (`col_int_undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_bigint_undef_signed_index_inverted_idx (`col_bigint_undef_signed_index_inverted`) USING INVERTED, + INDEX col_bigint_undef_signed_not_null_index_inverted_idx (`col_bigint_undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_date_undef_signed_index_inverted_idx (`col_date_undef_signed_index_inverted`) USING INVERTED, + INDEX col_date_undef_signed_not_null_index_inverted_idx (`col_date_undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_varchar_10__undef_signed_index_inverted_idx (`col_varchar_10__undef_signed_index_inverted`) USING INVERTED, + INDEX col_varchar_10__undef_signed_not_null_index_inverted_idx (`col_varchar_10__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_varchar_1024__undef_signed_index_inverted_idx (`col_varchar_1024__undef_signed_index_inverted`) USING INVERTED, + INDEX col_varchar_1024__undef_signed_not_null_index_inverted_idx (`col_varchar_1024__undef_signed_not_null_index_inverted`) USING INVERTED + ) ENGINE=OLAP + UNIQUE KEY(`col_int_undef_signed_not_null`, `col_date_undef_signed_not_null`, `col_bigint_undef_signed_not_null_index_inverted`, `col_bigint_undef_signed_not_null`) + DISTRIBUTED BY HASH(`col_bigint_undef_signed_not_null`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ + insert into ${table1}(pk,col_int_undef_signed,col_int_undef_signed_index_inverted,col_int_undef_signed_not_null,col_int_undef_signed_not_null_index_inverted,col_bigint_undef_signed,col_bigint_undef_signed_index_inverted,col_bigint_undef_signed_not_null,col_bigint_undef_signed_not_null_index_inverted,col_date_undef_signed,col_date_undef_signed_index_inverted,col_date_undef_signed_not_null,col_date_undef_signed_not_null_index_inverted,col_varchar_10__undef_signed,col_varchar_10__undef_signed_index_inverted,col_varchar_10__undef_signed_not_null,col_varchar_10__undef_signed_not_null_index_inverted,col_varchar_1024__undef_signed,col_varchar_1024__undef_signed_index_inverted,col_varchar_1024__undef_signed_not_null,col_varchar_1024__undef_signed_not_null_index_inverted) values (0,null,null,6,-10,686522353588051391,null,-4348126053228614825,-6032791270525051561,'2025-06-18','2023-12-20','2024-02-18','2024-02-18',null,'o','o','x',null,null,'q','o'),(1,-4,-4,2,-4,-2326501514208488669,-8144130583099259882,6094773265621719520,-4389392644235126205,'2024-01-09','2023-12-19','2024-01-31','2024-02-18',null,null,'c','n',null,'k','p','y'),(2,-10,-4,-4,-10,-5761419801766661970,null,1575077084160592390,-6748295140918895470,'2023-12-09','2024-01-31','2023-12-11','2024-01-08','j',null,'m','k','p','s','s','u'),(3,2,null,3,6,7058901979703960583,-8791856880716210018,9136811854525385821,-584135196107353733,'2024-02-18','2023-12-14','2024-01-19','2024-02-18','e','h','i','b','x','a','h','d'),(4,6,-4,4,0,2965210993977402598,null,1623398766611349339,-719530465607559880,'2024-02-18','2024-01-31','2023-12-12','2027-01-16','c','p','h','o','t','l','t','k'),(5,-4,-10,7,-4,null,-4312428052005262901,1664741259066823538,-6520957605791451399,'2024-02-18','2024-01-19','2027-01-09','2024-01-31','f','d','q','k',null,'u','n','x'),(6,-10,0,4,-4,-4719935591389099252,-8933690584562378263,1354434296669241202,2684538782485650790,'2023-12-16','2026-01-18','2024-01-08','2025-06-18',null,null,'f','x','x','v','i','m'),(7,0,-10,-10,-10,-5480618991908294867,null,5334008298577672775,7173424974650158839,'2024-01-09',null,'2023-12-09','2024-02-18','y','k','x','u',null,null,'a','b'),(8,6,-4,5,8,-7188890154699493125,-1925845279956226794,-5657889190097714482,1640041513228273840,'2027-01-16',null,'2025-02-18','2023-12-20','y','x','p','w','d','j','k','d'),(9,null,-4,1,3,-2080159247468648985,-1306911382131817506,1219720796746600015,-978348523441835274,'2024-02-18','2025-06-18','2025-06-18','2025-02-18',null,'i','y','s','c',null,'m','o'),(10,2,1,-10,0,-7569151440408756108,null,1393963672572074196,-3822268135049166532,'2024-01-08','2027-01-16','2023-12-13','2024-02-18','s',null,'q','z',null,'u','b','v'),(11,9,8,-10,7,-4419333711868488787,5670069993430301249,-5101280938951897282,7291919080934920934,'2027-01-09','2025-02-18','2024-01-17','2026-01-18','u','c','d','r',null,'m','r','p'),(12,9,8,7,9,-697217673128595873,-2415793798160514654,-1909943068043737865,5844073448689265407,'2024-01-17','2025-02-17','2023-12-17','2023-12-14','a','z','v','o','q','u','z','h'),(13,-10,6,-10,1,null,null,-6933219731543784592,-4745778751501231119,'2023-12-20',null,'2026-01-18','2026-01-18',null,'r','w','c','k',null,'t','e'),(14,6,4,-10,-10,null,377907223978572611,-7447442174599113505,4949011330273779695,'2023-12-17','2024-02-18','2026-01-18','2024-01-08','g','w','u','k',null,'m','g','d'),(15,null,-4,-10,3,-5441857898369120483,-2001300041828347883,4385022502994073333,6762545521805735020,'2024-01-17','2023-12-11','2023-12-15','2023-12-10','p','u','p','i','i','c','j','v'),(16,8,1,-10,3,7499177078109776887,8002215544264694167,-4914597203639379766,7611185654075676834,null,null,'2024-01-08','2023-12-17','e','h','q','t',null,null,'p','d'),(17,-4,-10,3,6,null,null,4596273190394276006,-3248366019937329149,'2024-01-09','2023-12-19','2023-12-20','2023-12-11',null,null,'i','f',null,'t','q','a'),(18,0,-4,-10,1,null,-2000849949571150330,7208571222986345744,2598345170237057509,'2024-01-09','2023-12-09','2024-02-18','2024-01-19','a','f','w','o','b','y','q','f'),(19,7,7,-4,0,null,5717592572856392823,-8128856226419623044,-7534868883394863810,'2023-12-20','2023-12-09','2023-12-10','2023-12-12','r','t','y','d','l','a','y','v'),(20,2,null,0,2,null,-6905288165492491017,1934258578152616096,-1388806210542225140,null,null,'2024-01-19','2026-01-18','p',null,'p','g','u','b','i','c'),(21,null,null,-10,-4,1698759627767041241,null,-6613269394014189122,1915677852069340594,'2023-12-18','2024-01-31','2025-02-18','2024-01-08','e','g','l','h',null,null,'v','n'),(22,-10,-10,-4,5,-3720952595350369266,1539673860923570193,5089313038468606351,262016952853919148,'2023-12-15','2025-02-18','2024-01-19','2024-01-08','d','g','d','e','l',null,'m','g'),(23,-10,1,-10,7,null,-4884323809040291936,-4428424779275301738,-3325468851678420401,'2023-12-19','2027-01-16','2026-02-18','2024-01-09','z','v','v','v','r','d','j','y'),(24,4,8,3,-10,1026316126533561197,-8966784351064986909,496857885215447340,-6148636280121789215,'2024-02-18','2025-06-18','2026-02-18','2025-02-18','f',null,'j','k',null,'s','i','q'),(25,5,5,4,6,8574091287090543865,null,-773937635554104337,6026917236758217609,'2026-02-18','2027-01-09','2023-12-12','2024-01-19','s','p','t','d','t','l','u','m'),(26,0,-10,-10,9,-2429694321063869458,null,8908961259233183763,6894623222255264210,'2024-01-17','2023-12-14','2023-12-11','2023-12-09','w','o','l','g','m','r','h','i'),(27,5,0,2,-10,7748161344545453064,null,3244053576839674045,-7948008233666340932,'2024-02-18','2023-12-20','2024-01-08','2023-12-14','n','a','r','q','c','y','q','u'),(28,4,-4,5,4,null,2204997326584988589,7997961660331289189,8763906081360257030,'2025-06-18','2023-12-18','2023-12-16','2023-12-16','k','i','d','t','y','c','o','a'),(29,0,5,-4,4,null,null,7562098503580472041,929168144988769048,'2026-01-18','2023-12-11','2023-12-10','2024-01-31','p','d','j','j','j','h','f','p'),(30,5,-10,-4,4,3945007524910600918,null,-8466778503120120353,-9169615505661358433,'2023-12-13','2024-01-19','2023-12-16','2023-12-10',null,'p','g','d','e','e','r','u'),(31,5,4,8,6,-7544567449016691208,-7026985677386241453,-2698203866546802012,-8383194363413620107,'2024-01-09','2027-01-09','2025-02-18','2025-02-18','f','t','g','n','r','i','p','o'),(32,-4,null,7,-10,-5468978947503379654,-5676001133436456624,-5328902013300281884,2338117992866157501,'2023-12-20','2023-12-15','2023-12-20','2024-01-08','f','z','y','t','j','c','e','x'),(33,-10,6,-10,-10,6715916167220457165,-3864032264700366706,7115861918737510196,-937991761308321600,'2025-02-18','2024-02-18','2023-12-19','2024-02-18','x',null,'t','x','h','o','p','v'),(34,-4,8,9,-10,-4718823602932239136,-3633212199616285968,-5190227402771860745,5545611345261203982,'2024-01-08','2026-01-18','2024-02-18','2023-12-16','f','t','n','h',null,'y','e','u'),(35,-4,3,-10,8,null,7722389449895645140,-4965839022248115530,6494405496923526511,'2023-12-10','2024-02-18','2026-02-18','2024-01-09','u','t','a','t','w',null,'h','w'),(36,5,6,-10,0,null,null,84960662585385706,2611830596442233539,null,'2026-01-18','2023-12-15','2026-01-18','b','t','p','b','g','g','z','k'),(37,-10,-10,3,-10,null,-5462312886211329186,-2793882411087244594,7564921654378090880,'2025-06-18','2027-01-09','2027-01-16','2023-12-09','n','k','l','z','y','i','o','c'),(38,4,null,-10,3,null,2065313248357950073,2398686393322288278,-5793325226082177083,'2023-12-14','2024-01-17','2023-12-12','2024-01-31','m',null,'n','c','g','f','r','m'),(39,5,1,9,0,-2901110266746457515,-7419676417711330947,5568223068212783910,-8443206843568552423,'2023-12-20','2023-12-15','2024-02-18','2024-01-17','j',null,'x','m','c','u','j','a'),(40,-4,5,-4,-4,3686987810771014559,4528672918224579415,-531153650185309112,-4795413900154192584,'2023-12-12','2024-01-19','2024-01-31','2024-01-19','m','o','k','p','v','s','f','c'),(41,null,6,-10,-10,1371451390800312645,-945321182848207621,-8418988114521301883,-8987180461079691062,'2024-01-09','2023-12-10','2023-12-19','2023-12-12','s','i','x','u','h','e','q','y'),(42,-10,null,2,1,null,-2863490765313461654,3110048825870954129,-2547950560699735251,'2025-06-18','2024-01-08','2023-12-10','2023-12-10','d','y','d','h','t','o','t','w'),(43,3,0,4,1,-7282895869404970499,5532011705197458854,-4502369753730677912,-3032934141238479600,'2023-12-18','2024-02-18','2023-12-19','2026-02-18','w',null,'m','n','g',null,'j','q'),(44,null,null,2,9,5430716729257430348,null,-8208477558683957238,-7953995265596299120,'2023-12-18','2023-12-18','2023-12-11','2025-06-18','w','w','a','u','k','k','j','q'),(45,6,-4,-10,1,-8903356732633014005,null,2532821444113211740,-2346292639048659545,'2024-01-08','2023-12-12','2025-02-18','2023-12-19','v','b','k','e','i','q','h','l'),(46,7,-10,-10,-10,null,-6646527298990960109,-7898216427196445987,-1558528416630681469,null,'2027-01-09','2024-02-18','2025-02-17','k','d','o','n','h','g','x','p'),(47,3,6,3,7,-6864291355591117572,4024432796468900765,-6272917113481022986,-1984131539617763529,'2024-01-17','2024-01-17','2025-06-18','2025-06-18','c','p','t','y','i','c','y','i'),(48,9,null,1,-4,null,-7244007199905240117,8657019614874868097,-492287318340969091,'2024-02-18','2023-12-11','2024-01-09','2027-01-09',null,null,'m','i','t','k','r','x'),(49,1,null,1,5,8407263822602373073,-3275760834800206047,-2117832965174816037,5807219087033669504,'2023-12-13','2024-01-19','2023-12-16','2024-02-18','r','n','n','o','r','g','j','q'); + """ + + qt_select_bug_1 """ + SELECT col_int_undef_signed_not_null, col_date_undef_signed_not_null + FROM ${table1} AS table1 + WHERE ( + NOT ( + ( + table1.`col_int_undef_signed_not_null` is NULL + ) + OR table1.col_varchar_1024__undef_signed_index_inverted IN ('h', 'j') + ) + OR table1.`col_date_undef_signed_not_null` IN ('2024-02-18') + ) + ORDER BY col_int_undef_signed_not_null, col_date_undef_signed_not_null; """ + +} diff --git a/regression-test/suites/inverted_index_p0/test_show_data.groovy b/regression-test/suites/inverted_index_p1/test_show_data.groovy similarity index 88% rename from regression-test/suites/inverted_index_p0/test_show_data.groovy rename to regression-test/suites/inverted_index_p1/test_show_data.groovy index 6622e2bfd6cb0d..294a3b8d695be0 100644 --- a/regression-test/suites/inverted_index_p0/test_show_data.groovy +++ b/regression-test/suites/inverted_index_p1/test_show_data.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_show_data", "p0") { +suite("test_show_data", "p1") { // define a sql table def testTableWithoutIndex = "test_show_data_httplogs_without_index" def testTableWithIndex = "test_show_data_httplogs_with_index" @@ -212,7 +212,7 @@ suite("test_show_data", "p0") { } } -suite("test_show_data_for_bkd", "p0") { +suite("test_show_data_for_bkd", "p1") { // define a sql table def testTableWithoutBKDIndex = "test_show_data_httplogs_without_bkd_index" def testTableWithBKDIndex = "test_show_data_httplogs_with_bkd_index" @@ -409,7 +409,7 @@ suite("test_show_data_for_bkd", "p0") { } } -suite("test_show_data_multi_add", "p0") { +suite("test_show_data_multi_add", "p1") { // define a sql table def testTableWithoutIndex = "test_show_data_httplogs_multi_add_without_index" def testTableWithIndex = "test_show_data_httplogs_multi_add_with_index" @@ -612,7 +612,7 @@ suite("test_show_data_multi_add", "p0") { } } -suite("test_show_data_with_compaction", "p0, nonConcurrent") { +suite("test_show_data_with_compaction", "p1") { // define a sql table def tableWithIndexCompaction = "test_with_index_compaction" def tableWithOutIndexCompaction = "test_without_index_compaction" @@ -742,46 +742,77 @@ suite("test_show_data_with_compaction", "p0, nonConcurrent") { return "wait_timeout" } + def run_compaction_and_wait = { tableName -> + //TabletId,ReplicaId,BackendId,SchemaHash,Version,LstSuccessVersion,LstFailedVersion,LstFailedTime,LocalDataSize,RemoteDataSize,RowCount,State,LstConsistencyCheckTime,CheckVersion,VersionCount,QueryHits,PathHash,MetaUrl,CompactionStatus + def tablets = sql_return_maparray """ show tablets from ${tableName}; """ + + // trigger compactions for all tablets in ${tableName} + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_run_full_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactJson = parseJson(out.trim()) + if (compactJson.status.toLowerCase() == "fail") { + logger.info("Compaction was done automatically!") + } else { + assertEquals("success", compactJson.status.toLowerCase()) + } + } - try { - - def run_compaction_and_wait = { tableName -> - //TabletId,ReplicaId,BackendId,SchemaHash,Version,LstSuccessVersion,LstFailedVersion,LstFailedTime,LocalDataSize,RemoteDataSize,RowCount,State,LstConsistencyCheckTime,CheckVersion,VersionCount,QueryHits,PathHash,MetaUrl,CompactionStatus - def tablets = sql_return_maparray """ show tablets from ${tableName}; """ - - // trigger compactions for all tablets in ${tableName} - for (def tablet in tablets) { + // wait for all compactions done + for (def tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) String tablet_id = tablet.TabletId backend_id = tablet.BackendId - (code, out, err) = be_run_full_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) - logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) assertEquals(code, 0) - def compactJson = parseJson(out.trim()) - if (compactJson.status.toLowerCase() == "fail") { - logger.info("Compaction was done automatically!") - } else { - assertEquals("success", compactJson.status.toLowerCase()) - } - } - - // wait for all compactions done - for (def tablet in tablets) { - boolean running = true - do { - Thread.sleep(1000) - String tablet_id = tablet.TabletId - backend_id = tablet.BackendId - (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) - logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) - assertEquals(code, 0) - def compactionStatus = parseJson(out.trim()) - assertEquals("success", compactionStatus.status.toLowerCase()) - running = compactionStatus.run_status - } while (running) - } + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) } + } - set_be_config.call("inverted_index_compaction_enable", "false") + def create_table_run_compaction_and_wait = { test_name -> + sql """ DROP TABLE IF EXISTS ${test_name}; """ + sql """ + CREATE TABLE ${test_name} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( "replication_num" = "1", "disable_auto_compaction" = "true"); + """ + + sql """ INSERT INTO ${test_name} VALUES (1, "andy", "andy love apple", 100); """ + sql """ INSERT INTO ${test_name} VALUES (1, "bason", "bason hate pear", 99); """ + sql """ INSERT INTO ${test_name} VALUES (2, "andy", "andy love apple", 100); """ + sql """ INSERT INTO ${test_name} VALUES (2, "bason", "bason hate pear", 99); """ + sql """ INSERT INTO ${test_name} VALUES (3, "andy", "andy love apple", 100); """ + sql """ INSERT INTO ${test_name} VALUES (3, "bason", "bason hate pear", 99); """ + def data_size = wait_for_show_data_finish(test_name, 60000, 0) + assertTrue(data_size != "wait_timeout") + run_compaction_and_wait(test_name) + data_size = wait_for_show_data_finish(test_name, 60000, data_size) + assertTrue(data_size != "wait_timeout") + return data_size + } + + try { + + set_be_config.call("inverted_index_compaction_enable", "true") sql "DROP TABLE IF EXISTS ${tableWithIndexCompaction}" create_table_with_index.call(tableWithIndexCompaction) @@ -793,11 +824,13 @@ suite("test_show_data_with_compaction", "p0, nonConcurrent") { sql "sync" - run_compaction_and_wait(tableWithIndexCompaction) def with_index_size = wait_for_show_data_finish(tableWithIndexCompaction, 60000, 0) assertTrue(with_index_size != "wait_timeout") + run_compaction_and_wait(tableWithIndexCompaction) + with_index_size = wait_for_show_data_finish(tableWithIndexCompaction, 60000, with_index_size) + assertTrue(with_index_size != "wait_timeout") - set_be_config.call("inverted_index_compaction_enable", "true") + set_be_config.call("inverted_index_compaction_enable", "false") sql "DROP TABLE IF EXISTS ${tableWithOutIndexCompaction}" create_table_with_index.call(tableWithOutIndexCompaction) @@ -807,13 +840,26 @@ suite("test_show_data_with_compaction", "p0, nonConcurrent") { load_httplogs_data.call(tableWithOutIndexCompaction, '9', 'true', 'json', 'documents-1000.json') load_httplogs_data.call(tableWithOutIndexCompaction, '10', 'true', 'json', 'documents-1000.json') - run_compaction_and_wait(tableWithOutIndexCompaction) def another_with_index_size = wait_for_show_data_finish(tableWithOutIndexCompaction, 60000, 0) assertTrue(another_with_index_size != "wait_timeout") + run_compaction_and_wait(tableWithOutIndexCompaction) + another_with_index_size = wait_for_show_data_finish(tableWithOutIndexCompaction, 60000, another_with_index_size) + assertTrue(another_with_index_size != "wait_timeout") + + logger.info("with_index_size is {}, another_with_index_size is {}", with_index_size, another_with_index_size) + assertEquals(another_with_index_size, with_index_size) + + set_be_config.call("inverted_index_compaction_enable", "true") + + def tableName = "test_inverted_index_compaction" + def data_size_1 = create_table_run_compaction_and_wait(tableName) + + set_be_config.call("inverted_index_compaction_enable", "false") + def data_size_2 = create_table_run_compaction_and_wait(tableName) + + logger.info("data_size_1 is {}, data_size_2 is {}", data_size_1, data_size_2) + assertEquals(data_size_1, data_size_2) - if (!isCloudMode()) { - assertEquals(another_with_index_size, with_index_size) - } } finally { // sql "DROP TABLE IF EXISTS ${tableWithIndexCompaction}" // sql "DROP TABLE IF EXISTS ${tableWithOutIndexCompaction}" diff --git a/regression-test/suites/load_p0/broker_load/test_compress_type.groovy b/regression-test/suites/load_p0/broker_load/test_compress_type.groovy index 693e533fa5086f..723a07d5296b37 100644 --- a/regression-test/suites/load_p0/broker_load/test_compress_type.groovy +++ b/regression-test/suites/load_p0/broker_load/test_compress_type.groovy @@ -17,6 +17,7 @@ suite("test_compress_type", "load_p0") { def tableName = "basic_data" + def s3BucketName = getS3BucketName() // GZ/LZO/BZ2/LZ4FRAME/DEFLATE/LZOP def compressTypes = [ @@ -62,24 +63,24 @@ suite("test_compress_type", "load_p0") { ] def paths = [ - "s3://doris-build-1308700295/regression/load/data/basic_data.csv.gz", - "s3://doris-build-1308700295/regression/load/data/basic_data.csv.bz2", - "s3://doris-build-1308700295/regression/load/data/basic_data.csv.lz4", - "s3://doris-build-1308700295/regression/load/data/basic_data.csv.gz", - "s3://doris-build-1308700295/regression/load/data/basic_data.csv.bz2", - "s3://doris-build-1308700295/regression/load/data/basic_data.csv.lz4", - "s3://doris-build-1308700295/regression/load/data/basic_data.csv.gz", - "s3://doris-build-1308700295/regression/load/data/basic_data.csv.bz2", - "s3://doris-build-1308700295/regression/load/data/basic_data.csv.lz4", - "s3://doris-build-1308700295/regression/load/data/basic_data.csv.gz", - "s3://doris-build-1308700295/regression/load/data/basic_data.csv.bz2", - "s3://doris-build-1308700295/regression/load/data/basic_data.csv.lz4", - "s3://doris-build-1308700295/regression/load/data/basic_data_by_line.json.gz", - "s3://doris-build-1308700295/regression/load/data/basic_data_by_line.json.bz2", - "s3://doris-build-1308700295/regression/load/data/basic_data_by_line.json.lz4", - "s3://doris-build-1308700295/regression/load/data/basic_data_by_line.json.gz", - "s3://doris-build-1308700295/regression/load/data/basic_data_by_line.json.bz2", - "s3://doris-build-1308700295/regression/load/data/basic_data_by_line.json.lz4", + "s3://${s3BucketName}/regression/load/data/basic_data.csv.gz", + "s3://${s3BucketName}/regression/load/data/basic_data.csv.bz2", + "s3://${s3BucketName}/regression/load/data/basic_data.csv.lz4", + "s3://${s3BucketName}/regression/load/data/basic_data.csv.gz", + "s3://${s3BucketName}/regression/load/data/basic_data.csv.bz2", + "s3://${s3BucketName}/regression/load/data/basic_data.csv.lz4", + "s3://${s3BucketName}/regression/load/data/basic_data.csv.gz", + "s3://${s3BucketName}/regression/load/data/basic_data.csv.bz2", + "s3://${s3BucketName}/regression/load/data/basic_data.csv.lz4", + "s3://${s3BucketName}/regression/load/data/basic_data.csv.gz", + "s3://${s3BucketName}/regression/load/data/basic_data.csv.bz2", + "s3://${s3BucketName}/regression/load/data/basic_data.csv.lz4", + "s3://${s3BucketName}/regression/load/data/basic_data_by_line.json.gz", + "s3://${s3BucketName}/regression/load/data/basic_data_by_line.json.bz2", + "s3://${s3BucketName}/regression/load/data/basic_data_by_line.json.lz4", + "s3://${s3BucketName}/regression/load/data/basic_data_by_line.json.gz", + "s3://${s3BucketName}/regression/load/data/basic_data_by_line.json.bz2", + "s3://${s3BucketName}/regression/load/data/basic_data_by_line.json.lz4", ] def labels = [] @@ -137,8 +138,8 @@ suite("test_compress_type", "load_p0") { WITH S3 ( "AWS_ACCESS_KEY" = "$ak", "AWS_SECRET_KEY" = "$sk", - "AWS_ENDPOINT" = "cos.ap-beijing.myqcloud.com", - "AWS_REGION" = "ap-beijing", + "AWS_ENDPOINT" = "${getS3Endpoint()}", + "AWS_REGION" = "${getS3Region()}", "provider" = "${getS3Provider()}" ) """ diff --git a/regression-test/suites/load_p0/broker_load/test_csv_with_enclose_and_escapeS3_load.groovy b/regression-test/suites/load_p0/broker_load/test_csv_with_enclose_and_escapeS3_load.groovy index eea25fb453495f..291f623a512eac 100644 --- a/regression-test/suites/load_p0/broker_load/test_csv_with_enclose_and_escapeS3_load.groovy +++ b/regression-test/suites/load_p0/broker_load/test_csv_with_enclose_and_escapeS3_load.groovy @@ -19,6 +19,7 @@ suite("test_csv_with_enclose_and_escapeS3_load", "load_p0") { def tableName = "test_csv_with_enclose_and_escape" + def s3BucketName = getS3BucketName() sql """ DROP TABLE IF EXISTS ${tableName} """ sql """ @@ -48,24 +49,24 @@ suite("test_csv_with_enclose_and_escapeS3_load", "load_p0") { ] for (i in 0.. 50", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data.csv") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data.csv") .addProperty("format", "csv") .addProperty("column_separator", "|") .addProperty("force_parsing_by_standard_uri", "true")) @@ -720,14 +722,14 @@ suite("test_s3_tvf", "p2") { attributeList.add(new TvfAttribute("agg_tbl_basic_tvf", "c1 as k00,c2 as k01,c3 as k02,c4 as k03,c5 as k04,c6 as k05,c7 as k06,c8 as k07,c9 as k08,c10 as k09,c11 as k10,c12 as k11,c13 as k12,c14 as k13,c15 as k14,c16 as k15,c17 as k16,c18 as k17,c19 as k18, to_bitmap(c6) as k19, HLL_HASH(c6) as k20, TO_QUANTILE_STATE(c5, 1.0) as k21, to_bitmap(c6) as kd19, HLL_HASH(c6) as kd20, TO_QUANTILE_STATE(c5, 1.0) as kd21", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18,k19,k20,k21,kd19,kd20,kd21" ,"WHERE c1 > 50", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data.csv") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data.csv") .addProperty("format", "csv") .addProperty("column_separator", "|") .addProperty("force_parsing_by_standard_uri", "true")) for(String table : arrayTables) { attributeList.add(new TvfAttribute(table, ["k00", "k01", "k02", "k03", "k04", "k05", "k06", "k07", "k08", "k09", "k10", "k11", "k12", "k13", "k14", "k15", "k16", "k17"], "WHERE c1 > 50", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_array_data.csv") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_array_data.csv") .addProperty("format", "csv") .addProperty("column_separator", "|") .addProperty("force_parsing_by_standard_uri", "true")) @@ -735,7 +737,7 @@ suite("test_s3_tvf", "p2") { for(String table : uniqTable) { attributeList.add(new TvfAttribute(table, ["k00", "k01", "k02", "k03", "k04", "k05", "k06", "k07", "k08", "k09", "k10", "k11", "k12", "k13", "k14", "k15", "k16", "k17", "k18"], "", "ORDER BY c1") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data.csv") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data.csv") .addProperty("format", "csv") .addProperty("column_separator", "|") .addProperty("force_parsing_by_standard_uri", "true")) @@ -743,7 +745,7 @@ suite("test_s3_tvf", "p2") { for(String table : basicTables) { attributeList.add(new TvfAttribute(table, "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18","k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18", "", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data.parq") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data.parq") .addProperty("format", "parquet") .addProperty("column_separator", "|") .addProperty("force_parsing_by_standard_uri", "true")) @@ -751,14 +753,14 @@ suite("test_s3_tvf", "p2") { attributeList.add(new TvfAttribute("agg_tbl_basic_tvf", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18, to_bitmap(k05) as k19, HLL_HASH(k05) as k20, TO_QUANTILE_STATE(k04, 1.0) as k21, to_bitmap(k05) as kd19, HLL_HASH(k05) as kd20, TO_QUANTILE_STATE(k04, 1.0) as kd21", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18,k19,k20,k21,kd19,kd20,kd21" ,"", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data.parq") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data.parq") .addProperty("format", "parquet") .addProperty("column_separator", "|") .addProperty("force_parsing_by_standard_uri", "true")) for(String table : arrayTables) { attributeList.add(new TvfAttribute(table, "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", "", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_array_data.parq") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_array_data.parq") .addProperty("format", "parquet") .addProperty("column_separator", "|") .addProperty("force_parsing_by_standard_uri", "true")) @@ -766,7 +768,7 @@ suite("test_s3_tvf", "p2") { for(String table : basicTables) { attributeList.add(new TvfAttribute(table, "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18","k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18", "", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data.orc") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data.orc") .addProperty("format", "orc") .addProperty("column_separator", "|") .addProperty("force_parsing_by_standard_uri", "true")) @@ -774,14 +776,14 @@ suite("test_s3_tvf", "p2") { attributeList.add(new TvfAttribute("agg_tbl_basic_tvf", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18, to_bitmap(k05) as k19, HLL_HASH(k05) as k20, TO_QUANTILE_STATE(k04, 1.0) as k21, to_bitmap(k05) as kd19, HLL_HASH(k05) as kd20, TO_QUANTILE_STATE(k04, 1.0) as kd21", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18,k19,k20,k21,kd19,kd20,kd21" ,"", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data.orc") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data.orc") .addProperty("format", "orc") .addProperty("column_separator", "|") .addProperty("force_parsing_by_standard_uri", "true")) for(String table : arrayTables) { attributeList.add(new TvfAttribute(table, "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", "", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_array_data.orc") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_array_data.orc") .addProperty("format", "orc") .addProperty("column_separator", "|") .addProperty("force_parsing_by_standard_uri", "true")) @@ -789,7 +791,7 @@ suite("test_s3_tvf", "p2") { for(String table : basicTables) { attributeList.add(new TvfAttribute(table, "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18","k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18", "", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data.json") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data.json") .addProperty("format", "json") .addProperty("read_json_by_line", "false") .addProperty("strip_outer_array", "true") @@ -799,7 +801,7 @@ suite("test_s3_tvf", "p2") { attributeList.add(new TvfAttribute("agg_tbl_basic_tvf", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18, to_bitmap(k05) as k19, HLL_HASH(k05) as k20, TO_QUANTILE_STATE(k04, 1.0) as k21, to_bitmap(k05) as kd19, HLL_HASH(k05) as kd20, TO_QUANTILE_STATE(k04, 1.0) as kd21", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18,k19,k20,k21,kd19,kd20,kd21" ,"", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data.json") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data.json") .addProperty("format", "json") .addProperty("read_json_by_line", "false") .addProperty("strip_outer_array", "true") @@ -808,7 +810,7 @@ suite("test_s3_tvf", "p2") { for(String table : arrayTables) { attributeList.add(new TvfAttribute(table, "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", "", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_array_data.json") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_array_data.json") .addProperty("format", "json") .addProperty("read_json_by_line", "false") .addProperty("strip_outer_array", "true") @@ -818,7 +820,7 @@ suite("test_s3_tvf", "p2") { for(String table : basicTables) { attributeList.add(new TvfAttribute(table, "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18","k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18", "", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data_by_line.json") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data_by_line.json") .addProperty("format", "json") .addProperty("read_json_by_line", "true") .addProperty("strip_outer_array", "false") @@ -828,7 +830,7 @@ suite("test_s3_tvf", "p2") { attributeList.add(new TvfAttribute("agg_tbl_basic_tvf", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18, to_bitmap(k05) as k19, HLL_HASH(k05) as k20, TO_QUANTILE_STATE(k04, 1.0) as k21, to_bitmap(k05) as kd19, HLL_HASH(k05) as kd20, TO_QUANTILE_STATE(k04, 1.0) as kd21", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18,k19,k20,k21,kd19,kd20,kd21" ,"", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data_by_line.json") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data_by_line.json") .addProperty("format", "json") .addProperty("read_json_by_line", "true") .addProperty("strip_outer_array", "false") @@ -837,7 +839,7 @@ suite("test_s3_tvf", "p2") { for(String table : arrayTables) { attributeList.add(new TvfAttribute(table, "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", "", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_array_data_by_line.json") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_array_data_by_line.json") .addProperty("format", "json") .addProperty("read_json_by_line", "true") .addProperty("strip_outer_array", "false") @@ -850,7 +852,7 @@ suite("test_s3_tvf", "p2") { // line_delimiter: \t for(String table : basicTables) { attributeList.add(new TvfAttribute(table, ["K00", "K01", "K02", "K03", "K04", "K05", "K06", "K07", "K08", "K09", "K10", "K11", "K12", "K13", "K14", "K15", "K16", "K17", "K18"], "", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data_by_line_delimiter.csv") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data_by_line_delimiter.csv") .addProperty("format", "csv") .addProperty("column_separator", "|") .addProperty("line_delimiter", "\t") @@ -859,7 +861,7 @@ suite("test_s3_tvf", "p2") { attributeList.add(new TvfAttribute("agg_tbl_basic_tvf", "c1 as k00,c2 as k01,c3 as k02,c4 as k03,c5 as k04,c6 as k05,c7 as k06,c8 as k07,c9 as k08,c10 as k09,c11 as k10,c12 as k11,c13 as k12,c14 as k13,c15 as k14,c16 as k15,c17 as k16,c18 as k17,c19 as k18, to_bitmap(c6) as k19, HLL_HASH(c6) as k20, TO_QUANTILE_STATE(c5, 1.0) as k21, to_bitmap(c6) as kd19, HLL_HASH(c6) as kd20, TO_QUANTILE_STATE(c5, 1.0) as kd21", "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18,k19,k20,k21,kd19,kd20,kd21" ,"", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data_by_line_delimiter.csv") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data_by_line_delimiter.csv") .addProperty("format", "csv") .addProperty("column_separator", "|") .addProperty("line_delimiter", "\t") @@ -867,7 +869,7 @@ suite("test_s3_tvf", "p2") { for(String table : arrayTables) { attributeList.add(new TvfAttribute(table, ["K00", "K01", "K02", "K03", "K04", "K05", "K06", "K07", "K08", "K09", "K10", "K11", "K12", "K13", "K14", "K15", "K16", "K17"], "", "") - .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_array_data_by_tab_line_delimiter.csv") + .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_array_data_by_tab_line_delimiter.csv") .addProperty("format", "csv") .addProperty("column_separator", "|") .addProperty("line_delimiter", "\t") @@ -877,7 +879,7 @@ suite("test_s3_tvf", "p2") { // invalid line delimiter, this will case error // for(String table : basicTables) { // attributeList.add(new TvfAttribute(table, ["K00", "K01", "K02", "K03", "K04", "K05", "K06", "K07", "K08", "K09", "K10", "K11", "K12", "K13", "K14", "K15", "K16", "K17", "K18"], "", "") - // .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data.csv") + // .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data.csv") // .addProperty("format", "csv") // .addProperty("column_separator", "|") // .addProperty("line_delimiter", ",") @@ -886,7 +888,7 @@ suite("test_s3_tvf", "p2") { // attributeList.add(new TvfAttribute("agg_tbl_basic_tvf", "c1 as k00,c2 as k01,c3 as k02,c4 as k03,c5 as k04,c6 as k05,c7 as k06,c8 as k07,c9 as k08,c10 as k09,c11 as k10,c12 as k11,c13 as k12,c14 as k13,c15 as k14,c16 as k15,c17 as k16,c18 as k17,c19 as k18, to_bitmap(c6) as k19, HLL_HASH(c6) as k20, TO_QUANTILE_STATE(c5, 1.0) as k21, to_bitmap(c6) as kd19, HLL_HASH(c6) as kd20, TO_QUANTILE_STATE(c5, 1.0) as kd21", // "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18,k19,k20,k21,kd19,kd20,kd21" ,"", "") - // .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_data.csv") + // .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_data.csv") // .addProperty("format", "csv") // .addProperty("column_separator", "|") // .addProperty("line_delimiter", ",") @@ -894,7 +896,7 @@ suite("test_s3_tvf", "p2") { // for(String table : arrayTables) { // attributeList.add(new TvfAttribute(table, ["K00", "K01", "K02", "K03", "K04", "K05", "K06", "K07", "K08", "K09", "K10", "K11", "K12", "K13", "K14", "K15", "K16", "K17"], "", "") - // .addProperty("uri", "s3://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/load/data/basic_array_data.csv") + // .addProperty("uri", "s3://${s3BucketName}.${s3Endpoint}/regression/load/data/basic_array_data.csv") // .addProperty("format", "csv") // .addProperty("column_separator", "|") // .addProperty("line_delimiter", ",") @@ -923,7 +925,7 @@ suite("test_s3_tvf", "p2") { FROM S3 ( "s3.access_key" = "$ak", "s3.secret_key" = "$sk", - "s3.region" = "ap-beijing", + "s3.region" = "${s3Region}", ${prop} ) ${attribute.whereClause} ${attribute.orderByClause} diff --git a/regression-test/suites/nereids_p0/expression/fold_constant/fold_constant_by_fe.groovy b/regression-test/suites/nereids_p0/expression/fold_constant/fold_constant_by_fe.groovy index a706ee9abf9114..cc63662657c37c 100644 --- a/regression-test/suites/nereids_p0/expression/fold_constant/fold_constant_by_fe.groovy +++ b/regression-test/suites/nereids_p0/expression/fold_constant/fold_constant_by_fe.groovy @@ -19,6 +19,7 @@ suite("test_fold_constant_by_fe") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' sql 'set enable_fold_nondeterministic_fn=true' + sql 'set enable_fold_constant_by_be=false' def results = sql 'select uuid(), uuid()' assertFalse(Objects.equals(results[0][0], results[0][1])) @@ -154,4 +155,12 @@ suite("test_fold_constant_by_fe") { res = res.split('VUNION')[1] assertFalse(res.contains("unix")) } -} \ No newline at end of file + + // test null like string cause of fe need to fold constant like that to enable not null derive + res = sql """explain select null like '%123%'""" + assertFalse(res.contains("like")) + // now fe fold constant still can not deal with this case + res = sql """explain select "12" like '%123%'""" + assertTrue(res.contains("like")) + +} diff --git a/regression-test/suites/partition_p2/auto_partition/diff_data/stress_test_diff_date_list.groovy b/regression-test/suites/partition_p2/auto_partition/diff_data/stress_test_diff_date_list.groovy index 2167c40f1d2a38..df7ce235e85afa 100644 --- a/regression-test/suites/partition_p2/auto_partition/diff_data/stress_test_diff_date_list.groovy +++ b/regression-test/suites/partition_p2/auto_partition/diff_data/stress_test_diff_date_list.groovy @@ -28,7 +28,7 @@ suite("stress_test_diff_date_list", "p2,nonConcurrent") { // get doris-db from s3 def dirPath = context.file.parent def fileName = "doris-dbgen" - def fileUrl = "http://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/doris-dbgen-23-10-18/doris-dbgen-23-10-20/doris-dbgen" + def fileUrl = "http://${getS3BucketName()}.${getS3Endpoint()}/regression/doris-dbgen-23-10-18/doris-dbgen-23-10-20/doris-dbgen" def filePath = Paths.get(dirPath, fileName) if (!Files.exists(filePath)) { new URL(fileUrl).withInputStream { inputStream -> diff --git a/regression-test/suites/partition_p2/auto_partition/same_data/stress_test_same_date_range.groovy b/regression-test/suites/partition_p2/auto_partition/same_data/stress_test_same_date_range.groovy index fb400105758601..c8bbdfbffc434f 100644 --- a/regression-test/suites/partition_p2/auto_partition/same_data/stress_test_same_date_range.groovy +++ b/regression-test/suites/partition_p2/auto_partition/same_data/stress_test_same_date_range.groovy @@ -28,7 +28,7 @@ suite("stress_test_same_date_range", "p2,nonConcurrent") { // get doris-db from s3 def dirPath = context.file.parent def fileName = "doris-dbgen" - def fileUrl = "http://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/doris-dbgen-23-10-18/doris-dbgen-23-10-20/doris-dbgen" + def fileUrl = "http://${getS3BucketName()}.${getS3Endpoint()}/regression/doris-dbgen-23-10-18/doris-dbgen-23-10-20/doris-dbgen" def filePath = Paths.get(dirPath, fileName) if (!Files.exists(filePath)) { new URL(fileUrl).withInputStream { inputStream -> diff --git a/regression-test/suites/partition_p2/auto_partition/two_stream_load/stress_test_two_stream_load.groovy b/regression-test/suites/partition_p2/auto_partition/two_stream_load/stress_test_two_stream_load.groovy index 478ee0d71240e5..ecbb277c5fd90b 100644 --- a/regression-test/suites/partition_p2/auto_partition/two_stream_load/stress_test_two_stream_load.groovy +++ b/regression-test/suites/partition_p2/auto_partition/two_stream_load/stress_test_two_stream_load.groovy @@ -26,7 +26,7 @@ suite("stress_test_two_stream_load", "p2,nonConcurrent") { // get doris-db from s3 def dirPath = context.file.parent def fileName = "doris-dbgen" - def fileUrl = "http://doris-build-1308700295.cos.ap-beijing.myqcloud.com/regression/doris-dbgen-23-10-18/doris-dbgen-23-10-20/doris-dbgen" + def fileUrl = "http://${getS3BucketName()}.${getS3Endpoint()}/regression/doris-dbgen-23-10-18/doris-dbgen-23-10-20/doris-dbgen" def filePath = Paths.get(dirPath, fileName) if (!Files.exists(filePath)) { new URL(fileUrl).withInputStream { inputStream -> diff --git a/regression-test/suites/query_profile/s3_load_profile_test.groovy b/regression-test/suites/query_profile/s3_load_profile_test.groovy index 33bdf48895a0d4..686e357d4e1395 100644 --- a/regression-test/suites/query_profile/s3_load_profile_test.groovy +++ b/regression-test/suites/query_profile/s3_load_profile_test.groovy @@ -29,6 +29,8 @@ def getProfile = { id -> // ref https://github.com/apache/doris/blob/3525a03815814f66ec78aa2ad6bbd9225b0e7a6b/regression-test/suites/load_p0/broker_load/test_s3_load.groovy suite('s3_load_profile_test') { + def s3Endpoint = getS3Endpoint() + def s3Region = getS3Region() sql "drop table if exists dup_tbl_basic;" sql """ CREATE TABLE dup_tbl_basic @@ -97,7 +99,7 @@ PROPERTIES ( "replication_num" = "1" ); """ - def loadAttribute =new LoadAttributes("s3://doris-build-1308700295/regression/load/data/basic_data.csv", + def loadAttribute =new LoadAttributes("s3://${getS3BucketName()}/regression/load/data/basic_data.csv", "dup_tbl_basic", "LINES TERMINATED BY \"\n\"", "COLUMNS TERMINATED BY \"|\"", "FORMAT AS \"CSV\"", "(k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18)", "", "", "", "", "") @@ -128,8 +130,8 @@ PROPERTIES ( WITH S3 ( "AWS_ACCESS_KEY" = "$ak", "AWS_SECRET_KEY" = "$sk", - "AWS_ENDPOINT" = "cos.ap-beijing.myqcloud.com", - "AWS_REGION" = "ap-beijing", + "AWS_ENDPOINT" = "${s3Endpoint}", + "AWS_REGION" = "${s3Region}", "use_path_style" = "$loadAttribute.usePathStyle", "provider" = "${getS3Provider()}" ) diff --git a/regression-test/suites/statistics/test_update_rows_and_partition_first_load.groovy b/regression-test/suites/statistics/test_update_rows_and_partition_first_load.groovy index a589a484d5ed15..ef927ddaf4329b 100644 --- a/regression-test/suites/statistics/test_update_rows_and_partition_first_load.groovy +++ b/regression-test/suites/statistics/test_update_rows_and_partition_first_load.groovy @@ -16,7 +16,9 @@ // under the License. suite("test_update_rows_and_partition_first_load", "p2") { - + def s3BucketName = getS3BucketName() + def s3Endpoint = getS3Endpoint() + def s3Region = getS3Region() String ak = getS3AK() String sk = getS3SK() String enabled = context.config.otherConfigs.get("enableBrokerLoad") @@ -88,24 +90,24 @@ suite("test_update_rows_and_partition_first_load", "p2") { def label = "part_" + UUID.randomUUID().toString().replace("-", "0") sql """ LOAD LABEL ${label} ( - DATA INFILE("s3://doris-build-1308700295/regression/load/data/update_rows_1.csv") + DATA INFILE("s3://${s3BucketName}/regression/load/data/update_rows_1.csv") INTO TABLE update_rows_test1 COLUMNS TERMINATED BY ",", - DATA INFILE("s3://doris-build-1308700295/regression/load/data/update_rows_2.csv") + DATA INFILE("s3://${s3BucketName}/regression/load/data/update_rows_2.csv") INTO TABLE update_rows_test2 COLUMNS TERMINATED BY ",", - DATA INFILE("s3://doris-build-1308700295/regression/load/data/update_rows_1.csv") + DATA INFILE("s3://${s3BucketName}/regression/load/data/update_rows_1.csv") INTO TABLE partition_test1 COLUMNS TERMINATED BY ",", - DATA INFILE("s3://doris-build-1308700295/regression/load/data/update_rows_2.csv") + DATA INFILE("s3://${s3BucketName}/regression/load/data/update_rows_2.csv") INTO TABLE partition_test2 COLUMNS TERMINATED BY "," ) WITH S3 ( "AWS_ACCESS_KEY" = "$ak", "AWS_SECRET_KEY" = "$sk", - "AWS_ENDPOINT" = "cos.ap-beijing.myqcloud.com", - "AWS_REGION" = "ap-beijing", + "AWS_ENDPOINT" = "${s3Endpoint}", + "AWS_REGION" = "${s3Region}", "provider" = "${getS3Provider()}" ); """ diff --git a/regression-test/suites/tpcds_sf1000_p2/load.groovy b/regression-test/suites/tpcds_sf1000_p2/load.groovy index aaf4fd54d71466..9bf888e93b0d7b 100644 --- a/regression-test/suites/tpcds_sf1000_p2/load.groovy +++ b/regression-test/suites/tpcds_sf1000_p2/load.groovy @@ -21,12 +21,13 @@ * */ suite("load") { + def s3Region = getS3Region() restore { location "s3://${getS3BucketName()}/regression/tpcds/sf1000" ak "${getS3AK()}" sk "${getS3SK()}" endpoint "http://${getS3Endpoint()}" - region "ap-beijing" + region "${s3Region}" repository "tpcds_backup" snapshot "tpcds_customer" timestamp "2022-03-31-10-16-46" @@ -40,7 +41,7 @@ suite("load") { ak "${getS3AK()}" sk "${getS3SK()}" endpoint "http://${getS3Endpoint()}" - region "ap-beijing" + region "${s3Region}" repository "tpcds_backup" snapshot "tpcds" timestamp "2022-03-30-12-22-31"