diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt index 116c4fd96e057fc..d59ee18f8ed93b2 100644 --- a/be/CMakeLists.txt +++ b/be/CMakeLists.txt @@ -739,7 +739,7 @@ set(BUILD_SHARED_LIBS OFF) option(ENABLE_CLANG_COVERAGE "coverage option" OFF) if (ENABLE_CLANG_COVERAGE AND ENABLE_CLANG_COVERAGE STREQUAL ON AND COMPILER_CLANG) - SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fprofile-instr-generate -fcoverage-mapping") + SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fprofile-instr-generate -fcoverage-mapping -DLLVM_PROFILE") endif () if (${MAKE_TEST} STREQUAL "ON") diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index 86ee162bad95a5c..05486be044b7021 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -1565,7 +1565,6 @@ void PublishVersionTaskPool::_publish_version_worker_thread_callback() { .tag("retry_time", retry_time) .error(status); ++retry_time; - std::this_thread::sleep_for(std::chrono::seconds(1)); } } if (status.is() && !is_task_timeout) { diff --git a/be/src/clucene b/be/src/clucene index 70c1a692bbb1277..6f8a21ffe15bd78 160000 --- a/be/src/clucene +++ b/be/src/clucene @@ -1 +1 @@ -Subproject commit 70c1a692bbb1277f107ff2ddedda41b3a223c632 +Subproject commit 6f8a21ffe15bd78a1cd3e685067ee5c9ed071827 diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 40f5a2b6b66e9b2..6acb373928b7ed4 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1104,8 +1104,12 @@ DEFINE_Int32(ingest_binlog_work_pool_size, "-1"); // Download binlog rate limit, unit is KB/s, 0 means no limit DEFINE_Int32(download_binlog_rate_limit_kbs, "0"); +DEFINE_mInt32(buffered_reader_read_timeout_ms, "20000"); + DEFINE_Bool(enable_snapshot_action, "false"); +DEFINE_mInt32(s3_writer_buffer_allocation_timeout_second, "60"); + // clang-format off #ifdef BE_TEST // test s3 diff --git a/be/src/common/config.h b/be/src/common/config.h index 7d7119ecfc11ef3..24a7340063dde25 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1160,9 +1160,14 @@ DECLARE_Int32(ingest_binlog_work_pool_size); // Download binlog rate limit, unit is KB/s DECLARE_Int32(download_binlog_rate_limit_kbs); +DECLARE_mInt32(buffered_reader_read_timeout_ms); + // whether to enable /api/snapshot api DECLARE_Bool(enable_snapshot_action); +// The timeout config for S3 write buffer allocation +DECLARE_mInt32(s3_writer_buffer_allocation_timeout_second); + #ifdef BE_TEST // test s3 DECLARE_String(test_s3_resource); diff --git a/be/src/exec/es/es_scroll_parser.cpp b/be/src/exec/es/es_scroll_parser.cpp index 4c52f869c56b910..74e8a43b3cec71d 100644 --- a/be/src/exec/es/es_scroll_parser.cpp +++ b/be/src/exec/es/es_scroll_parser.cpp @@ -222,7 +222,8 @@ Status get_date_value_int(const rapidjson::Value& col, PrimitiveType type, bool if (ok) { // The local time zone can change by session variable `time_zone` // We should use the user specified time zone, not the actual system local time zone. - success = dt_val.from_unixtime(std::chrono::system_clock::to_time_t(tp), time_zone); + success = true; + dt_val.from_unixtime(std::chrono::system_clock::to_time_t(tp), time_zone); } } else if (str_length == 19) { // YYYY-MM-DDTHH:MM:SS @@ -231,8 +232,8 @@ Status get_date_value_int(const rapidjson::Value& col, PrimitiveType type, bool const bool ok = cctz::parse("%Y-%m-%dT%H:%M:%S", str_date, cctz::utc_time_zone(), &tp); if (ok) { - success = dt_val.from_unixtime(std::chrono::system_clock::to_time_t(tp), - time_zone); + success = true; + dt_val.from_unixtime(std::chrono::system_clock::to_time_t(tp), time_zone); } } else { // YYYY-MM-DD HH:MM:SS @@ -243,7 +244,8 @@ Status get_date_value_int(const rapidjson::Value& col, PrimitiveType type, bool // string long like "1677895728000" int64_t time_long = std::atol(str_date.c_str()); if (time_long > 0) { - success = dt_val.from_unixtime(time_long / 1000, time_zone); + success = true; + dt_val.from_unixtime(time_long / 1000, time_zone); } } else { // YYYY-MM-DD or others @@ -255,9 +257,7 @@ Status get_date_value_int(const rapidjson::Value& col, PrimitiveType type, bool } } else { - if (!dt_val.from_unixtime(col.GetInt64() / 1000, time_zone)) { - RETURN_ERROR_IF_CAST_FORMAT_ERROR(col, type); - } + dt_val.from_unixtime(col.GetInt64() / 1000, time_zone); } if constexpr (is_datetime_v1) { if (type == TYPE_DATE) { diff --git a/be/src/exec/olap_common.h b/be/src/exec/olap_common.h index 5bd06d8d54030eb..7133f40e7adc8f1 100644 --- a/be/src/exec/olap_common.h +++ b/be/src/exec/olap_common.h @@ -304,6 +304,10 @@ class ColumnValueRange { condition.__set_condition_op("match_all"); } else if (value.first == MatchType::MATCH_PHRASE) { condition.__set_condition_op("match_phrase"); + } else if (value.first == MatchType::MATCH_PHRASE_PREFIX) { + condition.__set_condition_op("match_phrase_prefix"); + } else if (value.first == MatchType::MATCH_REGEXP) { + condition.__set_condition_op("match_regexp"); } else if (value.first == MatchType::MATCH_ELEMENT_EQ) { condition.__set_condition_op("match_element_eq"); } else if (value.first == MatchType::MATCH_ELEMENT_LT) { diff --git a/be/src/exec/olap_utils.h b/be/src/exec/olap_utils.h index 1d6bdf959302b08..1b8525dc1b99488 100644 --- a/be/src/exec/olap_utils.h +++ b/be/src/exec/olap_utils.h @@ -170,6 +170,8 @@ enum class MatchType { MATCH_ELEMENT_GT = 5, MATCH_ELEMENT_LE = 6, MATCH_ELEMENT_GE = 7, + MATCH_PHRASE_PREFIX = 8, + MATCH_REGEXP = 9, }; inline MatchType to_match_type(TExprOpcode::type type) { @@ -183,6 +185,12 @@ inline MatchType to_match_type(TExprOpcode::type type) { case TExprOpcode::type::MATCH_PHRASE: return MatchType::MATCH_PHRASE; break; + case TExprOpcode::type::MATCH_PHRASE_PREFIX: + return MatchType::MATCH_PHRASE_PREFIX; + break; + case TExprOpcode::type::MATCH_REGEXP: + return MatchType::MATCH_REGEXP; + break; case TExprOpcode::type::MATCH_ELEMENT_EQ: return MatchType::MATCH_ELEMENT_EQ; break; @@ -212,6 +220,10 @@ inline MatchType to_match_type(const std::string& condition_op) { return MatchType::MATCH_ALL; } else if (condition_op.compare("match_phrase") == 0) { return MatchType::MATCH_PHRASE; + } else if (condition_op.compare("match_phrase_prefix") == 0) { + return MatchType::MATCH_PHRASE_PREFIX; + } else if (condition_op.compare("match_regexp") == 0) { + return MatchType::MATCH_REGEXP; } else if (condition_op.compare("match_element_eq") == 0) { return MatchType::MATCH_ELEMENT_EQ; } else if (condition_op.compare("match_element_lt") == 0) { @@ -229,6 +241,8 @@ inline MatchType to_match_type(const std::string& condition_op) { inline bool is_match_condition(const std::string& op) { if (0 == strcasecmp(op.c_str(), "match_any") || 0 == strcasecmp(op.c_str(), "match_all") || 0 == strcasecmp(op.c_str(), "match_phrase") || + 0 == strcasecmp(op.c_str(), "match_phrase_prefix") || + 0 == strcasecmp(op.c_str(), "match_regexp") || 0 == strcasecmp(op.c_str(), "match_element_eq") || 0 == strcasecmp(op.c_str(), "match_element_lt") || 0 == strcasecmp(op.c_str(), "match_element_gt") || @@ -241,7 +255,8 @@ inline bool is_match_condition(const std::string& op) { inline bool is_match_operator(const TExprOpcode::type& op_type) { return TExprOpcode::MATCH_ANY == op_type || TExprOpcode::MATCH_ALL == op_type || - TExprOpcode::MATCH_PHRASE == op_type || TExprOpcode::MATCH_ELEMENT_EQ == op_type || + TExprOpcode::MATCH_PHRASE == op_type || TExprOpcode::MATCH_PHRASE_PREFIX == op_type || + TExprOpcode::MATCH_REGEXP == op_type || TExprOpcode::MATCH_ELEMENT_EQ == op_type || TExprOpcode::MATCH_ELEMENT_LT == op_type || TExprOpcode::MATCH_ELEMENT_GT == op_type || TExprOpcode::MATCH_ELEMENT_LE == op_type || TExprOpcode::MATCH_ELEMENT_GE == op_type; } diff --git a/be/src/io/fs/buffered_reader.cpp b/be/src/io/fs/buffered_reader.cpp index 18e638d6d75ceac..339ee9a8b11f225 100644 --- a/be/src/io/fs/buffered_reader.cpp +++ b/be/src/io/fs/buffered_reader.cpp @@ -387,20 +387,15 @@ Status MergeRangeFileReader::_fill_box(int range_index, size_t start_offset, siz return Status::OK(); } -// the condition variable would wait at most 10 seconds -// otherwise it would quit the procedure and treat it -// as one time out error status and would make the load -// task failed -constexpr static int WAIT_TIME_OUT_MS = 10000; - // there exists occasions where the buffer is already closed but // some prior tasks are still queued in thread pool, so we have to check whether // the buffer is closed each time the condition variable is notified. void PrefetchBuffer::reset_offset(size_t offset) { { std::unique_lock lck {_lock}; - if (!_prefetched.wait_for(lck, std::chrono::milliseconds(WAIT_TIME_OUT_MS), - [this]() { return _buffer_status != BufferStatus::PENDING; })) { + if (!_prefetched.wait_for( + lck, std::chrono::milliseconds(config::buffered_reader_read_timeout_ms), + [this]() { return _buffer_status != BufferStatus::PENDING; })) { _prefetch_status = Status::TimedOut("time out when reset prefetch buffer"); return; } @@ -427,10 +422,12 @@ void PrefetchBuffer::reset_offset(size_t offset) { void PrefetchBuffer::prefetch_buffer() { { std::unique_lock lck {_lock}; - if (!_prefetched.wait_for(lck, std::chrono::milliseconds(WAIT_TIME_OUT_MS), [this]() { - return _buffer_status == BufferStatus::RESET || - _buffer_status == BufferStatus::CLOSED; - })) { + if (!_prefetched.wait_for( + lck, std::chrono::milliseconds(config::buffered_reader_read_timeout_ms), + [this]() { + return _buffer_status == BufferStatus::RESET || + _buffer_status == BufferStatus::CLOSED; + })) { _prefetch_status = Status::TimedOut("time out when invoking prefetch buffer"); return; } @@ -470,7 +467,8 @@ void PrefetchBuffer::prefetch_buffer() { _statis.prefetch_request_io += 1; _statis.prefetch_request_bytes += _len; std::unique_lock lck {_lock}; - if (!_prefetched.wait_for(lck, std::chrono::milliseconds(WAIT_TIME_OUT_MS), + if (!_prefetched.wait_for(lck, + std::chrono::milliseconds(config::buffered_reader_read_timeout_ms), [this]() { return _buffer_status == BufferStatus::PENDING; })) { _prefetch_status = Status::TimedOut("time out when invoking prefetch buffer"); return; @@ -555,10 +553,12 @@ Status PrefetchBuffer::read_buffer(size_t off, const char* out, size_t buf_len, { std::unique_lock lck {_lock}; // buffer must be prefetched or it's closed - if (!_prefetched.wait_for(lck, std::chrono::milliseconds(WAIT_TIME_OUT_MS), [this]() { - return _buffer_status == BufferStatus::PREFETCHED || - _buffer_status == BufferStatus::CLOSED; - })) { + if (!_prefetched.wait_for( + lck, std::chrono::milliseconds(config::buffered_reader_read_timeout_ms), + [this]() { + return _buffer_status == BufferStatus::PREFETCHED || + _buffer_status == BufferStatus::CLOSED; + })) { _prefetch_status = Status::TimedOut("time out when read prefetch buffer"); return _prefetch_status; } @@ -594,7 +594,8 @@ Status PrefetchBuffer::read_buffer(size_t off, const char* out, size_t buf_len, void PrefetchBuffer::close() { std::unique_lock lck {_lock}; // in case _reader still tries to write to the buf after we close the buffer - if (!_prefetched.wait_for(lck, std::chrono::milliseconds(WAIT_TIME_OUT_MS), + if (!_prefetched.wait_for(lck, + std::chrono::milliseconds(config::buffered_reader_read_timeout_ms), [this]() { return _buffer_status != BufferStatus::PENDING; })) { _prefetch_status = Status::TimedOut("time out when close prefetch buffer"); return; diff --git a/be/src/io/fs/s3_file_write_bufferpool.cpp b/be/src/io/fs/s3_file_write_bufferpool.cpp index 48887f9c6ea6a75..5c5aa662316778d 100644 --- a/be/src/io/fs/s3_file_write_bufferpool.cpp +++ b/be/src/io/fs/s3_file_write_bufferpool.cpp @@ -17,6 +17,7 @@ #include "s3_file_write_bufferpool.h" +#include #include #include "common/config.h" @@ -40,7 +41,7 @@ void S3FileBuffer::on_finished() { // when there is memory preserved, directly write data to buf // TODO:(AlexYue): write to file cache otherwise, then we'll wait for free buffer // and to rob it -void S3FileBuffer::append_data(const Slice& data) { +Status S3FileBuffer::append_data(const Slice& data) { Defer defer {[&] { _size += data.get_size(); }}; while (true) { // if buf is not empty, it means there is memory preserved for this buf @@ -50,9 +51,14 @@ void S3FileBuffer::append_data(const Slice& data) { } else { // wait allocate buffer pool auto tmp = S3FileBufferPool::GetInstance()->allocate(true); + if (tmp->get_size() == 0) { + return Status::InternalError("Failed to allocate s3 writer buffer for {} seconds", + config::s3_writer_buffer_allocation_timeout_second); + } rob_buffer(tmp); } } + return Status::OK(); } void S3FileBuffer::submit() { @@ -81,13 +87,17 @@ void S3FileBufferPool::init(int32_t s3_write_buffer_whole_size, int32_t s3_write std::shared_ptr S3FileBufferPool::allocate(bool reserve) { std::shared_ptr buf = std::make_shared(_thread_pool); + int64_t timeout = config::s3_writer_buffer_allocation_timeout_second; // if need reserve then we must ensure return buf with memory preserved if (reserve) { { std::unique_lock lck {_lock}; - _cv.wait(lck, [this]() { return !_free_raw_buffers.empty(); }); - buf->reserve_buffer(_free_raw_buffers.front()); - _free_raw_buffers.pop_front(); + _cv.wait_for(lck, std::chrono::seconds(timeout), + [this]() { return !_free_raw_buffers.empty(); }); + if (!_free_raw_buffers.empty()) { + buf->reserve_buffer(_free_raw_buffers.front()); + _free_raw_buffers.pop_front(); + } } return buf; } diff --git a/be/src/io/fs/s3_file_write_bufferpool.h b/be/src/io/fs/s3_file_write_bufferpool.h index ad5f698f983d9d2..b4d3f322904e11c 100644 --- a/be/src/io/fs/s3_file_write_bufferpool.h +++ b/be/src/io/fs/s3_file_write_bufferpool.h @@ -52,7 +52,7 @@ struct S3FileBuffer : public std::enable_shared_from_this { // append data into the memory buffer inside // or into the file cache if the buffer has no memory buffer - void append_data(const Slice& data); + Status append_data(const Slice& data); // upload to S3 and file cache in async threadpool void submit(); // set the callback to upload to S3 file diff --git a/be/src/io/fs/s3_file_writer.cpp b/be/src/io/fs/s3_file_writer.cpp index 4a937f52057a321..18de6ed038983dc 100644 --- a/be/src/io/fs/s3_file_writer.cpp +++ b/be/src/io/fs/s3_file_writer.cpp @@ -240,7 +240,8 @@ Status S3FileWriter::appendv(const Slice* data, size_t data_cnt) { // if the buffer has memory buf inside, the data would be written into memory first then S3 then file cache // it would be written to cache then S3 if the buffer doesn't have memory preserved - _pending_buf->append_data(Slice {data[i].get_data() + pos, data_size_to_append}); + RETURN_IF_ERROR(_pending_buf->append_data( + Slice {data[i].get_data() + pos, data_size_to_append})); // if it's the last part, it could be less than 5MB, or it must // satisfy that the size is larger than or euqal to 5MB diff --git a/be/src/olap/match_predicate.cpp b/be/src/olap/match_predicate.cpp index 61d257231553bb4..8ffd6d9993609a4 100644 --- a/be/src/olap/match_predicate.cpp +++ b/be/src/olap/match_predicate.cpp @@ -95,6 +95,12 @@ InvertedIndexQueryType MatchPredicate::_to_inverted_index_query_type(MatchType m case MatchType::MATCH_PHRASE: ret = InvertedIndexQueryType::MATCH_PHRASE_QUERY; break; + case MatchType::MATCH_PHRASE_PREFIX: + ret = InvertedIndexQueryType::MATCH_PHRASE_PREFIX_QUERY; + break; + case MatchType::MATCH_REGEXP: + ret = InvertedIndexQueryType::MATCH_REGEXP_QUERY; + break; case MatchType::MATCH_ELEMENT_EQ: ret = InvertedIndexQueryType::EQUAL_QUERY; break; @@ -117,7 +123,7 @@ InvertedIndexQueryType MatchPredicate::_to_inverted_index_query_type(MatchType m } bool MatchPredicate::_skip_evaluate(InvertedIndexIterator* iterator) const { - if (_match_type == MatchType::MATCH_PHRASE && + if ((_match_type == MatchType::MATCH_PHRASE || _match_type == MatchType::MATCH_PHRASE_PREFIX) && iterator->get_inverted_index_reader_type() == InvertedIndexReaderType::FULLTEXT && get_parser_phrase_support_string_from_properties(iterator->get_index_properties()) == INVERTED_INDEX_PARSER_PHRASE_SUPPORT_NO) { diff --git a/be/src/olap/reader.cpp b/be/src/olap/reader.cpp index 43cdf4e637dc936..0d4e59e852d5785 100644 --- a/be/src/olap/reader.cpp +++ b/be/src/olap/reader.cpp @@ -256,6 +256,7 @@ Status TabletReader::_capture_rs_readers(const ReaderParams& read_params) { _reader_context.remaining_conjunct_roots = read_params.remaining_conjunct_roots; _reader_context.common_expr_ctxs_push_down = read_params.common_expr_ctxs_push_down; _reader_context.output_columns = &read_params.output_columns; + _reader_context.push_down_agg_type_opt = read_params.push_down_agg_type_opt; return Status::OK(); } diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/conjunction_query.cpp b/be/src/olap/rowset/segment_v2/inverted_index/query/conjunction_query.cpp index b77edc79ade905a..b2448a8fa8e233f 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/conjunction_query.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/conjunction_query.cpp @@ -38,12 +38,12 @@ ConjunctionQuery::~ConjunctionQuery() { } void ConjunctionQuery::add(const std::wstring& field_name, const std::vector& terms) { - if (terms.size() < 1) { - _CLTHROWA(CL_ERR_IllegalArgument, "ConjunctionQuery::add: terms.size() < 1"); + if (terms.empty()) { + _CLTHROWA(CL_ERR_IllegalArgument, "ConjunctionQuery::add: terms empty"); } std::vector iterators; - for (auto& term : terms) { + for (const auto& term : terms) { std::wstring ws_term = StringUtil::string_to_wstring(term); Term* t = _CLNEW Term(field_name.c_str(), ws_term.c_str()); _terms.push_back(t); diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.cpp b/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.cpp index 07a159b32224088..7b797d7b54a91e9 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.cpp @@ -22,26 +22,25 @@ namespace doris { DisjunctionQuery::DisjunctionQuery(IndexReader* reader) : _reader(reader) {} DisjunctionQuery::~DisjunctionQuery() { - for (auto& term : _terms) { - if (term) { - _CLDELETE(term); - } - } for (auto& term_doc : _term_docs) { if (term_doc) { _CLDELETE(term_doc); } } + for (auto& term : _terms) { + if (term) { + _CLDELETE(term); + } + } } void DisjunctionQuery::add(const std::wstring& field_name, const std::vector& terms) { - if (terms.size() < 1) { - _CLTHROWA(CL_ERR_IllegalArgument, "ConjunctionQuery::add: terms.size() < 1"); + if (terms.empty()) { + _CLTHROWA(CL_ERR_IllegalArgument, "DisjunctionQuery::add: terms empty"); } - for (auto& term : terms) { + for (const auto& term : terms) { std::wstring ws_term = StringUtil::string_to_wstring(term); - _wsterms.emplace_back(&ws_term); Term* t = _CLNEW Term(field_name.c_str(), ws_term.c_str()); _terms.push_back(t); TermDocs* term_doc = _reader->termDocs(t); diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.h b/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.h index f42fd69dabc2efd..bb0a837f42a3139 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.h +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.h @@ -39,7 +39,6 @@ class DisjunctionQuery { private: IndexReader* _reader = nullptr; - std::vector _wsterms; std::vector _terms; std::vector _term_docs; std::vector _term_iterators; diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.cpp b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.cpp new file mode 100644 index 000000000000000..4b0340cda4a0115 --- /dev/null +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.cpp @@ -0,0 +1,63 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "phrase_prefix_query.h" + +#include "olap/rowset//segment_v2/inverted_index/query/prefix_query.h" + +namespace doris { + +namespace segment_v2 { + +PhrasePrefixQuery::PhrasePrefixQuery(const std::shared_ptr& searcher) + : _searcher(searcher) {} + +void PhrasePrefixQuery::add(const std::wstring& field_name, const std::vector& terms) { + if (terms.empty()) { + return; + } + + for (size_t i = 0; i < terms.size(); i++) { + if (i < terms.size() - 1) { + std::wstring ws = StringUtil::string_to_wstring(terms[i]); + Term* t = _CLNEW Term(field_name.c_str(), ws.c_str()); + _query.add(t); + _CLDECDELETE(t); + } else { + std::vector prefix_terms; + PrefixQuery::get_prefix_terms(_searcher->getReader(), field_name, terms[i], + prefix_terms, _max_expansions); + if (prefix_terms.empty()) { + continue; + } + _query.add(prefix_terms); + for (auto& t : prefix_terms) { + _CLDECDELETE(t); + } + } + } +} + +void PhrasePrefixQuery::search(roaring::Roaring& roaring) { + _searcher->_search(&_query, [&roaring](const int32_t docid, const float_t /*score*/) { + roaring.add(docid); + }); +} + +} // namespace segment_v2 + +} // namespace doris \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.h b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.h new file mode 100644 index 000000000000000..28007620ce581ee --- /dev/null +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.h @@ -0,0 +1,54 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include + +#include + +#include "CLucene/search/MultiPhraseQuery.h" +#include "roaring/roaring.hh" + +CL_NS_USE(index) +CL_NS_USE(search) + +namespace doris { + +namespace segment_v2 { + +class PhrasePrefixQuery { +public: + PhrasePrefixQuery(const std::shared_ptr& searcher); + ~PhrasePrefixQuery() = default; + + void set_max_expansions(int32_t max_expansions) { _max_expansions = max_expansions; } + + void add(const std::wstring& field_name, const std::vector& terms); + void search(roaring::Roaring& roaring); + +private: + std::shared_ptr _searcher; + MultiPhraseQuery _query; + + int32_t _max_expansions = 50; +}; + +} // namespace segment_v2 + +} // namespace doris \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.cpp b/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.cpp new file mode 100644 index 000000000000000..7d23d6eb60f3486 --- /dev/null +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.cpp @@ -0,0 +1,80 @@ +// 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. + +#include "prefix_query.h" + +namespace doris { + +void PrefixQuery::get_prefix_terms(IndexReader* reader, const std::wstring& field_name, + const std::string& prefix, + std::vector& prefix_terms, + int32_t max_expansions) { + std::wstring ws_prefix = StringUtil::string_to_wstring(prefix); + + Term* prefix_term = _CLNEW Term(field_name.c_str(), ws_prefix.c_str()); + TermEnum* enumerator = reader->terms(prefix_term); + + int32_t count = 0; + Term* lastTerm = nullptr; + try { + const TCHAR* prefixText = prefix_term->text(); + const TCHAR* prefixField = prefix_term->field(); + const TCHAR* tmp = nullptr; + size_t i = 0; + size_t prefixLen = prefix_term->textLength(); + do { + lastTerm = enumerator->term(); + if (lastTerm != nullptr && lastTerm->field() == prefixField) { + size_t termLen = lastTerm->textLength(); + if (prefixLen > termLen) { + break; + } + + tmp = lastTerm->text(); + + for (i = prefixLen - 1; i != -1; --i) { + if (tmp[i] != prefixText[i]) { + tmp = nullptr; + break; + } + } + if (tmp == nullptr) { + break; + } + + if (max_expansions > 0 && count >= max_expansions) { + break; + } + + Term* t = _CLNEW Term(field_name.c_str(), tmp); + prefix_terms.push_back(t); + count++; + } else { + break; + } + _CLDECDELETE(lastTerm); + } while (enumerator->next()); + } + _CLFINALLY({ + enumerator->close(); + _CLDELETE(enumerator); + _CLDECDELETE(lastTerm); + _CLDECDELETE(prefix_term); + }); +} + +} // namespace doris \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.h b/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.h new file mode 100644 index 000000000000000..5deb0c1a3628ad2 --- /dev/null +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.h @@ -0,0 +1,40 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include + +#include + +CL_NS_USE(index) + +namespace doris { + +class PrefixQuery { +public: + PrefixQuery() = default; + ~PrefixQuery() = default; + + static void get_prefix_terms(IndexReader* reader, const std::wstring& field_name, + const std::string& prefix, + std::vector& prefix_terms, + int32_t max_expansions = 50); +}; + +} // namespace doris \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.cpp b/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.cpp new file mode 100644 index 000000000000000..83c5401bac0e5b8 --- /dev/null +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.cpp @@ -0,0 +1,98 @@ +// 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. + +#include "regexp_query.h" + +#include +#include + +#include "common/logging.h" + +namespace doris::segment_v2 { + +RegexpQuery::RegexpQuery(const std::shared_ptr& searcher) + : _searcher(searcher), query(searcher->getReader()) {} + +void RegexpQuery::add(const std::wstring& field_name, const std::string& pattern) { + hs_database_t* database = nullptr; + hs_compile_error_t* compile_err = nullptr; + hs_scratch_t* scratch = nullptr; + + if (hs_compile(pattern.data(), HS_FLAG_DOTALL | HS_FLAG_ALLOWEMPTY | HS_FLAG_UTF8, + HS_MODE_BLOCK, nullptr, &database, &compile_err) != HS_SUCCESS) { + LOG(ERROR) << "hyperscan compilation failed: " << compile_err->message; + hs_free_compile_error(compile_err); + return; + } + + if (hs_alloc_scratch(database, &scratch) != HS_SUCCESS) { + LOG(ERROR) << "hyperscan could not allocate scratch space."; + hs_free_database(database); + return; + } + + auto on_match = [](unsigned int id, unsigned long long from, unsigned long long to, + unsigned int flags, void* context) -> int { + *((bool*)context) = true; + return 0; + }; + + Term* term = nullptr; + TermEnum* enumerator = nullptr; + std::vector terms; + int32_t count = 0; + + try { + enumerator = _searcher->getReader()->terms(); + while (enumerator->next()) { + term = enumerator->term(); + std::string input = lucene_wcstoutf8string(term->text(), term->textLength()); + + bool is_match = false; + if (hs_scan(database, input.data(), input.size(), 0, scratch, on_match, + (void*)&is_match) != HS_SUCCESS) { + LOG(ERROR) << "hyperscan match failed: " << input; + break; + } + + if (is_match) { + terms.emplace_back(std::move(input)); + if (++count >= _max_expansions) { + break; + } + } + + _CLDECDELETE(term); + } + } + _CLFINALLY({ + _CLDECDELETE(term); + enumerator->close(); + _CLDELETE(enumerator); + + hs_free_scratch(scratch); + hs_free_database(database); + }) + + query.add(field_name, terms); +} + +void RegexpQuery::search(roaring::Roaring& roaring) { + query.search(roaring); +} + +} // namespace doris::segment_v2 diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.h b/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.h new file mode 100644 index 000000000000000..3791ad50d8f78fa --- /dev/null +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.h @@ -0,0 +1,46 @@ +// 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. + +#pragma once + +#include + +#include "olap/rowset/segment_v2/inverted_index/query/disjunction_query.h" + +CL_NS_USE(index) +CL_NS_USE(search) + +namespace doris::segment_v2 { + +class RegexpQuery { +public: + RegexpQuery(const std::shared_ptr& searcher); + ~RegexpQuery() = default; + + void set_max_expansions(int32_t max_expansions) { _max_expansions = max_expansions; } + + void add(const std::wstring& field_name, const std::string& pattern); + void search(roaring::Roaring& roaring); + +private: + std::shared_ptr _searcher; + + int32_t _max_expansions = 50; + DisjunctionQuery query; +}; + +} // namespace doris::segment_v2 diff --git a/be/src/olap/rowset/segment_v2/inverted_index_query_type.h b/be/src/olap/rowset/segment_v2/inverted_index_query_type.h index 1ebfe6359181e99..3037f979f6edf70 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_query_type.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_query_type.h @@ -32,6 +32,8 @@ enum class InvertedIndexQueryType { MATCH_ANY_QUERY = 5, MATCH_ALL_QUERY = 6, MATCH_PHRASE_QUERY = 7, + MATCH_PHRASE_PREFIX_QUERY = 8, + MATCH_REGEXP_QUERY = 9, }; inline std::string InvertedIndexQueryType_toString(InvertedIndexQueryType query_type) { @@ -63,6 +65,12 @@ inline std::string InvertedIndexQueryType_toString(InvertedIndexQueryType query_ case InvertedIndexQueryType::MATCH_PHRASE_QUERY: { return "MPHRASE"; } + case InvertedIndexQueryType::MATCH_PHRASE_PREFIX_QUERY: { + return "MPHRASEPREFIX"; + } + case InvertedIndexQueryType::MATCH_REGEXP_QUERY: { + return "MREGEXP"; + } default: return ""; } diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp index 7d710d72c382ab7..292884e631b939a 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp @@ -50,12 +50,15 @@ #include "CLucene/analysis/standard95/StandardAnalyzer.h" #include "common/config.h" #include "common/logging.h" +#include "inverted_index_query_type.h" #include "io/fs/file_system.h" #include "olap/inverted_index_parser.h" #include "olap/key_coder.h" #include "olap/olap_common.h" #include "olap/rowset/segment_v2/inverted_index/char_filter/char_filter_factory.h" #include "olap/rowset/segment_v2/inverted_index/query/conjunction_query.h" +#include "olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.h" +#include "olap/rowset/segment_v2/inverted_index/query/regexp_query.h" #include "olap/rowset/segment_v2/inverted_index_cache.h" #include "olap/rowset/segment_v2/inverted_index_compound_directory.h" #include "olap/rowset/segment_v2/inverted_index_desc.h" @@ -90,7 +93,9 @@ bool InvertedIndexReader::_is_range_query(InvertedIndexQueryType query_type) { bool InvertedIndexReader::_is_match_query(InvertedIndexQueryType query_type) { return (query_type == InvertedIndexQueryType::MATCH_ANY_QUERY || query_type == InvertedIndexQueryType::MATCH_ALL_QUERY || - query_type == InvertedIndexQueryType::MATCH_PHRASE_QUERY); + query_type == InvertedIndexQueryType::MATCH_PHRASE_QUERY || + query_type == InvertedIndexQueryType::MATCH_PHRASE_PREFIX_QUERY || + query_type == InvertedIndexQueryType::MATCH_REGEXP_QUERY); } bool InvertedIndexReader::indexExists(io::Path& index_file_path) { @@ -141,10 +146,13 @@ std::unique_ptr InvertedIndexReader::create_reader( return reader; } -std::vector InvertedIndexReader::get_analyse_result( - lucene::util::Reader* reader, lucene::analysis::Analyzer* analyzer, - const std::string& field_name, InvertedIndexQueryType query_type, bool drop_duplicates) { - std::vector analyse_result; +void InvertedIndexReader::get_analyse_result(std::vector& analyse_result, + lucene::util::Reader* reader, + lucene::analysis::Analyzer* analyzer, + const std::string& field_name, + InvertedIndexQueryType query_type, + bool drop_duplicates) { + analyse_result.clear(); std::wstring field_ws = std::wstring(field_name.begin(), field_name.end()); std::unique_ptr token_stream( @@ -168,8 +176,6 @@ std::vector InvertedIndexReader::get_analyse_result( std::set unrepeated_result(analyse_result.begin(), analyse_result.end()); analyse_result.assign(unrepeated_result.begin(), unrepeated_result.end()); } - - return analyse_result; } Status InvertedIndexReader::read_null_bitmap(InvertedIndexQueryCacheHandle* cache_handle, @@ -246,19 +252,25 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run auto index_file_name = InvertedIndexDescriptor::get_index_file_name(path.filename(), _index_meta.index_id()); auto index_file_path = index_dir / index_file_name; - InvertedIndexCtxSPtr inverted_index_ctx = std::make_shared(); - inverted_index_ctx->parser_type = get_inverted_index_parser_type_from_string( - get_parser_string_from_properties(_index_meta.properties())); - inverted_index_ctx->parser_mode = - get_parser_mode_string_from_properties(_index_meta.properties()); - inverted_index_ctx->char_filter_map = - get_parser_char_filter_map_from_properties(_index_meta.properties()); + try { - auto analyzer = create_analyzer(inverted_index_ctx.get()); - auto reader = create_reader(inverted_index_ctx.get(), search_str); - inverted_index_ctx->analyzer = analyzer.get(); - std::vector analyse_result = - get_analyse_result(reader.get(), analyzer.get(), column_name, query_type); + std::vector analyse_result; + if (query_type == InvertedIndexQueryType::MATCH_REGEXP_QUERY) { + analyse_result.emplace_back(search_str); + } else { + InvertedIndexCtxSPtr inverted_index_ctx = std::make_shared(); + inverted_index_ctx->parser_type = get_inverted_index_parser_type_from_string( + get_parser_string_from_properties(_index_meta.properties())); + inverted_index_ctx->parser_mode = + get_parser_mode_string_from_properties(_index_meta.properties()); + inverted_index_ctx->char_filter_map = + get_parser_char_filter_map_from_properties(_index_meta.properties()); + auto analyzer = create_analyzer(inverted_index_ctx.get()); + auto reader = create_reader(inverted_index_ctx.get(), search_str); + inverted_index_ctx->analyzer = analyzer.get(); + get_analyse_result(analyse_result, reader.get(), analyzer.get(), column_name, + query_type); + } if (analyse_result.empty()) { auto msg = fmt::format( @@ -267,7 +279,9 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run search_str, get_parser_string_from_properties(_index_meta.properties())); if (query_type == InvertedIndexQueryType::MATCH_ALL_QUERY || query_type == InvertedIndexQueryType::MATCH_ANY_QUERY || - query_type == InvertedIndexQueryType::MATCH_PHRASE_QUERY) { + query_type == InvertedIndexQueryType::MATCH_PHRASE_QUERY || + query_type == InvertedIndexQueryType::MATCH_PHRASE_PREFIX_QUERY || + query_type == InvertedIndexQueryType::MATCH_REGEXP_QUERY) { LOG(WARNING) << msg; return Status::OK(); } else { @@ -294,6 +308,7 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run roaring::Roaring query_match_bitmap; bool null_bitmap_already_read = false; if (query_type == InvertedIndexQueryType::MATCH_PHRASE_QUERY || + query_type == InvertedIndexQueryType::MATCH_PHRASE_PREFIX_QUERY || query_type == InvertedIndexQueryType::MATCH_ALL_QUERY || query_type == InvertedIndexQueryType::EQUAL_QUERY) { std::string str_tokens; @@ -302,7 +317,7 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run str_tokens += " "; } - auto cache = InvertedIndexQueryCache::instance(); + auto* cache = InvertedIndexQueryCache::instance(); InvertedIndexQueryCache::CacheKey cache_key; cache_key.index_path = index_file_path; cache_key.column_name = column_name; @@ -333,6 +348,10 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run query.reset(phrase_query); res = normal_index_search(stats, query_type, index_searcher, null_bitmap_already_read, query, term_match_bitmap); + } else if (query_type == InvertedIndexQueryType::MATCH_PHRASE_PREFIX_QUERY) { + res = match_phrase_prefix_index_search(stats, runtime_state, field_ws, + analyse_result, index_searcher, + term_match_bitmap); } else { res = match_all_index_search(stats, runtime_state, field_ws, analyse_result, index_searcher, term_match_bitmap); @@ -346,13 +365,45 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run cache->insert(cache_key, term_match_bitmap, &cache_handle); } query_match_bitmap = *term_match_bitmap; + } else if (query_type == InvertedIndexQueryType::MATCH_REGEXP_QUERY) { + const std::string& pattern = analyse_result[0]; + + std::shared_ptr term_match_bitmap = nullptr; + auto* cache = InvertedIndexQueryCache::instance(); + + InvertedIndexQueryCache::CacheKey cache_key; + cache_key.index_path = index_file_path; + cache_key.column_name = column_name; + cache_key.query_type = query_type; + cache_key.value = pattern; + InvertedIndexQueryCacheHandle cache_handle; + if (cache->lookup(cache_key, &cache_handle)) { + stats->inverted_index_query_cache_hit++; + term_match_bitmap = cache_handle.get_bitmap(); + } else { + stats->inverted_index_query_cache_miss++; + + auto index_searcher = get_index_search(); + + term_match_bitmap = std::make_shared(); + + Status res = match_regexp_index_search(stats, runtime_state, field_ws, pattern, + index_searcher, term_match_bitmap); + if (!res.ok()) { + return res; + } + + term_match_bitmap->runOptimize(); + cache->insert(cache_key, term_match_bitmap, &cache_handle); + } + query_match_bitmap = *term_match_bitmap; } else { bool first = true; for (auto token : analyse_result) { std::shared_ptr term_match_bitmap = nullptr; // try to get term bitmap match result from cache to avoid query index on cache hit - auto cache = InvertedIndexQueryCache::instance(); + auto* cache = InvertedIndexQueryCache::instance(); // use EQUAL_QUERY type here since cache is for each term/token //auto token = lucene_wcstoutf8string(token_ws.c_str(), token_ws.length()); std::wstring token_ws = StringUtil::string_to_wstring(token); @@ -471,6 +522,42 @@ Status FullTextIndexReader::match_all_index_search( return Status::OK(); } +Status FullTextIndexReader::match_phrase_prefix_index_search( + OlapReaderStatistics* stats, RuntimeState* runtime_state, const std::wstring& field_ws, + const std::vector& analyse_result, const IndexSearcherPtr& index_searcher, + const std::shared_ptr& term_match_bitmap) { + TQueryOptions queryOptions = runtime_state->query_options(); + try { + SCOPED_RAW_TIMER(&stats->inverted_index_searcher_search_timer); + PhrasePrefixQuery query(index_searcher); + query.set_max_expansions(queryOptions.inverted_index_max_expansions); + query.add(field_ws, analyse_result); + query.search(*term_match_bitmap); + } catch (const CLuceneError& e) { + return Status::Error("CLuceneError occured: {}", + e.what()); + } + return Status::OK(); +} + +Status FullTextIndexReader::match_regexp_index_search( + OlapReaderStatistics* stats, RuntimeState* runtime_state, const std::wstring& field_ws, + const std::string& pattern, const IndexSearcherPtr& index_searcher, + const std::shared_ptr& term_match_bitmap) { + TQueryOptions queryOptions = runtime_state->query_options(); + try { + SCOPED_RAW_TIMER(&stats->inverted_index_searcher_search_timer); + RegexpQuery query(index_searcher); + query.set_max_expansions(queryOptions.inverted_index_max_expansions); + query.add(field_ws, pattern); + query.search(*term_match_bitmap); + } catch (const CLuceneError& e) { + return Status::Error("CLuceneError occured: {}", + e.what()); + } + return Status::OK(); +} + void FullTextIndexReader::check_null_bitmap(const IndexSearcherPtr& index_searcher, bool& null_bitmap_already_read) { // try to reuse index_searcher's directory to read null_bitmap to cache diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.h b/be/src/olap/rowset/segment_v2/inverted_index_reader.h index 20c5c731f9eca8b..8b5c786f36f9098 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_reader.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.h @@ -97,11 +97,12 @@ class InvertedIndexReader : public std::enable_shared_from_this get_analyse_result(lucene::util::Reader* reader, - lucene::analysis::Analyzer* analyzer, - const std::string& field_name, - InvertedIndexQueryType query_type, - bool drop_duplicates = true); + static void get_analyse_result(std::vector& analyse_result, + lucene::util::Reader* reader, + lucene::analysis::Analyzer* analyzer, + const std::string& field_name, InvertedIndexQueryType query_type, + bool drop_duplicates = true); + static std::unique_ptr create_reader(InvertedIndexCtx* inverted_index_ctx, const std::string& value); static std::unique_ptr create_analyzer( @@ -153,6 +154,16 @@ class FullTextIndexReader : public InvertedIndexReader { const std::shared_ptr& term_match_bitmap); void check_null_bitmap(const IndexSearcherPtr& index_searcher, bool& null_bitmap_already_read); + + Status match_phrase_prefix_index_search( + OlapReaderStatistics* stats, RuntimeState* runtime_state, const std::wstring& field_ws, + const std::vector& analyse_result, const IndexSearcherPtr& index_searcher, + const std::shared_ptr& term_match_bitmap); + + Status match_regexp_index_search(OlapReaderStatistics* stats, RuntimeState* runtime_state, + const std::wstring& field_ws, const std::string& pattern, + const IndexSearcherPtr& index_searcher, + const std::shared_ptr& term_match_bitmap); }; class StringTypeInvertedIndexReader : public InvertedIndexReader { diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp b/be/src/olap/rowset/segment_v2/segment_writer.cpp index 02ea943818d812f..29ec7e02bc56525 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp @@ -444,13 +444,6 @@ Status SegmentWriter::append_block_with_partial_content(const vectorized::Block* // mark key with delete sign as deleted. bool have_delete_sign = (delete_sign_column_data != nullptr && delete_sign_column_data[block_pos] != 0); - if (have_delete_sign && !_tablet_schema->has_sequence_col() && !have_input_seq_column) { - // we can directly use delete bitmap to mark the rows with delete sign as deleted - // if sequence column doesn't exist to eliminate reading delete sign columns in later reads - _mow_context->delete_bitmap->add({_opts.rowset_ctx->rowset_id, _segment_id, - DeleteBitmap::TEMP_VERSION_FOR_DELETE_SIGN}, - segment_pos); - } RowLocation loc; // save rowset shared ptr so this rowset wouldn't delete @@ -699,29 +692,6 @@ Status SegmentWriter::append_block(const vectorized::Block* block, size_t row_po _serialize_block_to_row_column(*const_cast(block)); } - if (_opts.write_type == DataWriteType::TYPE_DIRECT && _opts.enable_unique_key_merge_on_write && - !_tablet_schema->has_sequence_col() && _tablet_schema->delete_sign_idx() != -1) { - const vectorized::ColumnWithTypeAndName& delete_sign_column = - block->get_by_position(_tablet_schema->delete_sign_idx()); - auto& delete_sign_col = - reinterpret_cast(*(delete_sign_column.column)); - if (delete_sign_col.size() >= row_pos + num_rows) { - const vectorized::Int8* delete_sign_column_data = delete_sign_col.get_data().data(); - uint32_t segment_start_pos = - _column_writers[_tablet_schema->delete_sign_idx()]->get_next_rowid(); - for (size_t block_pos = row_pos, seg_pos = segment_start_pos; - seg_pos < segment_start_pos + num_rows; block_pos++, seg_pos++) { - // we can directly use delete bitmap to mark the rows with delete sign as deleted - // if sequence column doesn't exist to eliminate reading delete sign columns in later reads - if (delete_sign_column_data[block_pos]) { - _mow_context->delete_bitmap->add({_opts.rowset_ctx->rowset_id, _segment_id, - DeleteBitmap::TEMP_VERSION_FOR_DELETE_SIGN}, - seg_pos); - } - } - } - } - _olap_data_convertor->set_source_content(block, row_pos, num_rows); // find all row pos for short key indexes diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 4c1ebd721de8269..611e32828bba25c 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -3075,14 +3075,6 @@ Status Tablet::calc_segment_delete_bitmap(RowsetSharedPtr rowset, continue; } if (is_partial_update && rowset_writer != nullptr) { - if (delete_bitmap->contains( - {rowset_id, seg->id(), DeleteBitmap::TEMP_VERSION_FOR_DELETE_SIGN}, - row_id)) { - LOG(INFO) - << "DEBUG: skip a delete sign column while calc_segment_delete_bitmap " - << "processing confict for partial update"; - continue; - } // In publish version, record rows to be deleted for concurrent update // For example, if version 5 and 6 update a row, but version 6 only see // version 4 when write, and when publish version, version 5's value will diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index 11dc3532514c8bd..0ef058760eae436 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -343,7 +343,6 @@ class DeleteBitmap { // tablet's delete bitmap we can use arbitary version number in BitmapKey. Here we define some version numbers // for specific usage during this periods to avoid conflicts constexpr static inline uint64_t TEMP_VERSION_COMMON = 0; - constexpr static inline uint64_t TEMP_VERSION_FOR_DELETE_SIGN = 1; /** * diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 7e7330a43ecd21c..4bde38365f380cd 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -279,6 +279,9 @@ Status FragmentExecState::execute() { Status FragmentExecState::cancel(const PPlanFragmentCancelReason& reason, const std::string& msg) { if (!_cancelled) { std::lock_guard l(_status_lock); + if (_cancelled) { // double check. may re-enter cuz MemLimiter + return Status::OK(); + } if (reason == PPlanFragmentCancelReason::LIMIT_REACH) { _executor.set_is_report_on_cancel(false); } diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index 3c4ebb9feed6f79..6db53e383c5d968 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -112,6 +112,7 @@ class RuntimeState { TQueryType::type query_type() const { return _query_options.query_type; } int64_t timestamp_ms() const { return _timestamp_ms; } int32_t nano_seconds() const { return _nano_seconds; } + // if possible, use timezone_obj() rather than timezone() const std::string& timezone() const { return _timezone; } const cctz::time_zone& timezone_obj() const { return _timezone_obj; } const std::string& user() const { return _user; } diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp index d0c68996b381898..a350f4438aedb14 100644 --- a/be/src/service/doris_main.cpp +++ b/be/src/service/doris_main.cpp @@ -77,6 +77,7 @@ static void help(const char*); extern "C" { void __lsan_do_leak_check(); +int __llvm_profile_write_file(); } namespace doris { @@ -549,7 +550,11 @@ int main(int argc, char** argv) { #endif sleep(10); } - + LOG(INFO) << "Doris main exiting."; +#if defined(LLVM_PROFILE) + __llvm_profile_write_file(); + LOG(INFO) << "Flush profile file."; +#endif doris::TabletSchemaCache::stop_and_join(); http_service.stop(); brpc_service.join(); diff --git a/be/src/vec/core/field.h b/be/src/vec/core/field.h index 941ca3fe3dc974c..d9bb6ce66240152 100644 --- a/be/src/vec/core/field.h +++ b/be/src/vec/core/field.h @@ -367,6 +367,7 @@ class Field { return "QuantileState"; default: LOG(FATAL) << "type not supported, type=" << Types::to_string(which); + __builtin_unreachable(); break; } } diff --git a/be/src/vec/exec/join/vhash_join_node.cpp b/be/src/vec/exec/join/vhash_join_node.cpp index d0ac9c1ca5eb846..df5c4faaa17bfbd 100644 --- a/be/src/vec/exec/join/vhash_join_node.cpp +++ b/be/src/vec/exec/join/vhash_join_node.cpp @@ -551,9 +551,7 @@ Status HashJoinNode::pull(doris::RuntimeState* state, vectorized::Block* output_ Block temp_block; //get probe side output column for (int i = 0; i < _left_output_slot_flags.size(); ++i) { - if (_left_output_slot_flags[i]) { - temp_block.insert(_probe_block.get_by_position(i)); - } + temp_block.insert(_probe_block.get_by_position(i)); } auto mark_column = ColumnNullable::create(ColumnUInt8::create(block_rows, 0), ColumnUInt8::create(block_rows, 1)); diff --git a/be/src/vec/functions/date_time_transforms.h b/be/src/vec/functions/date_time_transforms.h index 877a66e002acff3..a513f9f154dd14e 100644 --- a/be/src/vec/functions/date_time_transforms.h +++ b/be/src/vec/functions/date_time_transforms.h @@ -222,10 +222,10 @@ struct FromUnixTimeImpl { static inline auto execute(FromType val, StringRef format, ColumnString::Chars& res_data, size_t& offset, const cctz::time_zone& time_zone) { DateType dt; - if (format.size > 128 || val < 0 || val > TIMESTAMP_VALID_MAX || - !dt.from_unixtime(val, time_zone)) { + if (format.size > 128 || val < 0 || val > TIMESTAMP_VALID_MAX) { return std::pair {offset, true}; } + dt.from_unixtime(val, time_zone); char buf[128]; if (!dt.to_format_string(format.data, format.size, buf)) { diff --git a/be/src/vec/functions/function_convert_tz.h b/be/src/vec/functions/function_convert_tz.h index 9d226b706462444..378788b538f4470 100644 --- a/be/src/vec/functions/function_convert_tz.h +++ b/be/src/vec/functions/function_convert_tz.h @@ -131,12 +131,7 @@ struct ConvertTZImpl { result_column->insert_default(); return; } - - if (!ts_value2.from_unixtime(timestamp, to_tz)) { - result_null_map[index_now] = true; - result_column->insert_default(); - return; - } + ts_value2.from_unixtime(timestamp, to_tz); } else { int64_t timestamp; if (!ts_value.unix_timestamp(×tamp, from_tz)) { @@ -145,11 +140,7 @@ struct ConvertTZImpl { return; } - if (!ts_value2.from_unixtime(timestamp, to_tz)) { - result_null_map[index_now] = true; - result_column->insert_default(); - return; - } + ts_value2.from_unixtime(timestamp, to_tz); } result_column->insert(binary_cast(ts_value2)); diff --git a/be/src/vec/functions/function_date_or_datetime_computation.h b/be/src/vec/functions/function_date_or_datetime_computation.h index f3459ae42b8bf2f..bca886df2adc74a 100644 --- a/be/src/vec/functions/function_date_or_datetime_computation.h +++ b/be/src/vec/functions/function_date_or_datetime_computation.h @@ -923,34 +923,30 @@ struct CurrentDateTimeImpl { DateValueType dtv; bool use_const; if constexpr (WithPrecision) { - if (const ColumnConst* const_column = check_and_get_column( + if (const auto* const_column = check_and_get_column( block.get_by_position(arguments[0]).column)) { int scale = const_column->get_int(0); - if (dtv.from_unixtime(context->state()->timestamp_ms() / 1000, - context->state()->nano_seconds(), - context->state()->timezone_obj(), scale)) { - if constexpr (std::is_same_v) { - reinterpret_cast(&dtv)->set_type(TIME_DATETIME); - } - auto date_packed_int = binary_cast(dtv); - col_to->insert_data( - const_cast(reinterpret_cast(&date_packed_int)), 0); - - } else { - auto invalid_val = 0; - col_to->insert_data( - const_cast(reinterpret_cast(&invalid_val)), 0); + dtv.from_unixtime(context->state()->timestamp_ms() / 1000, + context->state()->nano_seconds(), + context->state()->timezone_obj(), scale); + if constexpr (std::is_same_v) { + reinterpret_cast(&dtv)->set_type(TIME_DATETIME); } + auto date_packed_int = binary_cast(dtv); + col_to->insert_data( + const_cast(reinterpret_cast(&date_packed_int)), 0); + use_const = true; - } else if (const ColumnNullable* nullable_column = check_and_get_column( + } else if (const auto* nullable_column = check_and_get_column( block.get_by_position(arguments[0]).column)) { const auto& null_map = nullable_column->get_null_map_data(); const auto& nested_column = nullable_column->get_nested_column_ptr(); for (int i = 0; i < input_rows_count; i++) { - if (!null_map[i] && dtv.from_unixtime(context->state()->timestamp_ms() / 1000, - context->state()->nano_seconds(), - context->state()->timezone_obj(), - nested_column->get64(i))) { + if (!null_map[i]) { + dtv.from_unixtime(context->state()->timestamp_ms() / 1000, + context->state()->nano_seconds(), + context->state()->timezone_obj(), + nested_column->get64(i)); if constexpr (std::is_same_v) { reinterpret_cast(&dtv)->set_type(TIME_DATETIME); } @@ -968,38 +964,27 @@ struct CurrentDateTimeImpl { } else { auto& int_column = block.get_by_position(arguments[0]).column; for (int i = 0; i < input_rows_count; i++) { - if (dtv.from_unixtime(context->state()->timestamp_ms() / 1000, - context->state()->nano_seconds(), - context->state()->timezone_obj(), int_column->get64(i))) { - if constexpr (std::is_same_v) { - reinterpret_cast(&dtv)->set_type(TIME_DATETIME); - } - auto date_packed_int = binary_cast(dtv); - col_to->insert_data( - const_cast(reinterpret_cast(&date_packed_int)), - 0); - } else { - auto invalid_val = 0; - col_to->insert_data( - const_cast(reinterpret_cast(&invalid_val)), 0); + dtv.from_unixtime(context->state()->timestamp_ms() / 1000, + context->state()->nano_seconds(), + context->state()->timezone_obj(), int_column->get64(i)); + if constexpr (std::is_same_v) { + reinterpret_cast(&dtv)->set_type(TIME_DATETIME); } + auto date_packed_int = binary_cast(dtv); + col_to->insert_data( + const_cast(reinterpret_cast(&date_packed_int)), 0); } use_const = false; } } else { - if (dtv.from_unixtime(context->state()->timestamp_ms() / 1000, - context->state()->timezone_obj())) { - if constexpr (std::is_same_v) { - reinterpret_cast(&dtv)->set_type(TIME_DATETIME); - } - auto date_packed_int = binary_cast(dtv); - col_to->insert_data( - const_cast(reinterpret_cast(&date_packed_int)), 0); - } else { - auto invalid_val = 0; - col_to->insert_data(const_cast(reinterpret_cast(&invalid_val)), - 0); + dtv.from_unixtime(context->state()->timestamp_ms() / 1000, + context->state()->timezone_obj()); + if constexpr (std::is_same_v) { + reinterpret_cast(&dtv)->set_type(TIME_DATETIME); } + auto date_packed_int = binary_cast(dtv); + col_to->insert_data(const_cast(reinterpret_cast(&date_packed_int)), + 0); use_const = true; } @@ -1022,31 +1007,21 @@ struct CurrentDateImpl { auto col_to = ColumnVector::create(); if constexpr (std::is_same_v) { DateV2Value dtv; - if (dtv.from_unixtime(context->state()->timestamp_ms() / 1000, - context->state()->timezone_obj())) { - auto date_packed_int = binary_cast, uint32_t>( - *reinterpret_cast*>(&dtv)); - col_to->insert_data( - const_cast(reinterpret_cast(&date_packed_int)), 0); - } else { - auto invalid_val = 0; - col_to->insert_data(const_cast(reinterpret_cast(&invalid_val)), - 0); - } + dtv.from_unixtime(context->state()->timestamp_ms() / 1000, + context->state()->timezone_obj()); + auto date_packed_int = binary_cast, uint32_t>( + *reinterpret_cast*>(&dtv)); + col_to->insert_data(const_cast(reinterpret_cast(&date_packed_int)), + 0); } else { VecDateTimeValue dtv; - if (dtv.from_unixtime(context->state()->timestamp_ms() / 1000, - context->state()->timezone_obj())) { - reinterpret_cast(&dtv)->set_type(TIME_DATE); - auto date_packed_int = binary_cast( - *reinterpret_cast(&dtv)); - col_to->insert_data( - const_cast(reinterpret_cast(&date_packed_int)), 0); - } else { - auto invalid_val = 0; - col_to->insert_data(const_cast(reinterpret_cast(&invalid_val)), - 0); - } + dtv.from_unixtime(context->state()->timestamp_ms() / 1000, + context->state()->timezone_obj()); + reinterpret_cast(&dtv)->set_type(TIME_DATE); + auto date_packed_int = binary_cast( + *reinterpret_cast(&dtv)); + col_to->insert_data(const_cast(reinterpret_cast(&date_packed_int)), + 0); } block.get_by_position(result).column = ColumnConst::create(std::move(col_to), input_rows_count); @@ -1062,15 +1037,11 @@ struct CurrentTimeImpl { size_t result, size_t input_rows_count) { auto col_to = ColumnVector::create(); VecDateTimeValue dtv; - if (dtv.from_unixtime(context->state()->timestamp_ms() / 1000, - context->state()->timezone_obj())) { - double time = dtv.hour() * 3600l + dtv.minute() * 60l + dtv.second(); - time *= (1000 * 1000); - col_to->insert_data(const_cast(reinterpret_cast(&time)), 0); - } else { - auto invalid_val = 0; - col_to->insert_data(const_cast(reinterpret_cast(&invalid_val)), 0); - } + dtv.from_unixtime(context->state()->timestamp_ms() / 1000, + context->state()->timezone_obj()); + double time = dtv.hour() * 3600l + dtv.minute() * 60l + dtv.second(); + time *= (1000 * 1000); + col_to->insert_data(const_cast(reinterpret_cast(&time)), 0); block.get_by_position(result).column = ColumnConst::create(std::move(col_to), input_rows_count); return Status::OK(); @@ -1149,15 +1120,28 @@ struct TimestampToDateTime : IFunction { auto null_vector = ColumnVector::create(); res_col->get_data().resize_fill(input_rows_count, 0); null_vector->get_data().resize_fill(input_rows_count, false); + NullMap& null_map = null_vector->get_data(); auto& res_data = res_col->get_data(); const cctz::time_zone& time_zone = context->state()->timezone_obj(); + for (int i = 0; i < input_rows_count; ++i) { Int64 value = column_data.get_element(i); + if (value < 0) { + null_map[i] = true; + continue; + } + auto& dt = reinterpret_cast&>(res_data[i]); - null_map[i] = !dt.from_unixtime(value / Impl::ratio, time_zone); - dt.set_microsecond((value % Impl::ratio) * ratio_to_micro); + dt.from_unixtime(value / Impl::ratio, time_zone); + + if (dt.is_valid_date()) [[likely]] { + dt.set_microsecond((value % Impl::ratio) * ratio_to_micro); + } else { + null_map[i] = true; + } } + block.get_by_position(result).column = ColumnNullable::create(std::move(res_col), std::move(null_vector)); return Status::OK(); diff --git a/be/src/vec/functions/function_tokenize.cpp b/be/src/vec/functions/function_tokenize.cpp index 11760a30f5025f4..54d9bee4ae9edc6 100644 --- a/be/src/vec/functions/function_tokenize.cpp +++ b/be/src/vec/functions/function_tokenize.cpp @@ -79,10 +79,10 @@ void FunctionTokenize::_do_tokenize(const ColumnString& src_column_string, auto reader = doris::segment_v2::InvertedIndexReader::create_reader( &inverted_index_ctx, tokenize_str.to_string()); - std::vector query_tokens = - doris::segment_v2::InvertedIndexReader::get_analyse_result( - reader.get(), inverted_index_ctx.analyzer, "tokenize", - doris::segment_v2::InvertedIndexQueryType::MATCH_PHRASE_QUERY); + std::vector query_tokens; + doris::segment_v2::InvertedIndexReader::get_analyse_result( + query_tokens, reader.get(), inverted_index_ctx.analyzer, "tokenize", + doris::segment_v2::InvertedIndexQueryType::MATCH_PHRASE_QUERY); for (auto token : query_tokens) { const size_t old_size = column_string_chars.size(); const size_t split_part_size = token.length(); diff --git a/be/src/vec/functions/match.cpp b/be/src/vec/functions/match.cpp index 3497b2ef7a96a50..c81c1617ca61d4e 100644 --- a/be/src/vec/functions/match.cpp +++ b/be/src/vec/functions/match.cpp @@ -129,10 +129,10 @@ inline std::vector FunctionMatchBase::analyse_data_token( auto reader = doris::segment_v2::InvertedIndexReader::create_reader( inverted_index_ctx, str_ref.to_string()); - std::vector element_tokens = - doris::segment_v2::InvertedIndexReader::get_analyse_result( - reader.get(), inverted_index_ctx->analyzer, column_name, query_type, - false); + std::vector element_tokens; + doris::segment_v2::InvertedIndexReader::get_analyse_result( + element_tokens, reader.get(), inverted_index_ctx->analyzer, column_name, + query_type, false); data_tokens.insert(data_tokens.end(), element_tokens.begin(), element_tokens.end()); } } else { @@ -140,8 +140,9 @@ inline std::vector FunctionMatchBase::analyse_data_token( auto reader = doris::segment_v2::InvertedIndexReader::create_reader(inverted_index_ctx, str_ref.to_string()); - data_tokens = doris::segment_v2::InvertedIndexReader::get_analyse_result( - reader.get(), inverted_index_ctx->analyzer, column_name, query_type, false); + doris::segment_v2::InvertedIndexReader::get_analyse_result(data_tokens, reader.get(), + inverted_index_ctx->analyzer, + column_name, query_type, false); } return data_tokens; } @@ -160,10 +161,10 @@ Status FunctionMatchAny::execute_match(const std::string& column_name, << inverted_index_parser_type_to_string(parser_type); auto reader = doris::segment_v2::InvertedIndexReader::create_reader(inverted_index_ctx, match_query_str); - std::vector query_tokens = - doris::segment_v2::InvertedIndexReader::get_analyse_result( - reader.get(), inverted_index_ctx->analyzer, column_name, - doris::segment_v2::InvertedIndexQueryType::MATCH_ANY_QUERY); + std::vector query_tokens; + doris::segment_v2::InvertedIndexReader::get_analyse_result( + query_tokens, reader.get(), inverted_index_ctx->analyzer, column_name, + doris::segment_v2::InvertedIndexQueryType::MATCH_ANY_QUERY); if (query_tokens.empty()) { LOG(WARNING) << fmt::format( "token parser result is empty for query, " @@ -205,10 +206,10 @@ Status FunctionMatchAll::execute_match(const std::string& column_name, << inverted_index_parser_type_to_string(parser_type); auto reader = doris::segment_v2::InvertedIndexReader::create_reader(inverted_index_ctx, match_query_str); - std::vector query_tokens = - doris::segment_v2::InvertedIndexReader::get_analyse_result( - reader.get(), inverted_index_ctx->analyzer, column_name, - doris::segment_v2::InvertedIndexQueryType::MATCH_ALL_QUERY); + std::vector query_tokens; + doris::segment_v2::InvertedIndexReader::get_analyse_result( + query_tokens, reader.get(), inverted_index_ctx->analyzer, column_name, + doris::segment_v2::InvertedIndexQueryType::MATCH_ALL_QUERY); if (query_tokens.empty()) { LOG(WARNING) << fmt::format( "token parser result is empty for query, " @@ -256,10 +257,10 @@ Status FunctionMatchPhrase::execute_match(const std::string& column_name, << inverted_index_parser_type_to_string(parser_type); auto reader = doris::segment_v2::InvertedIndexReader::create_reader(inverted_index_ctx, match_query_str); - std::vector query_tokens = - doris::segment_v2::InvertedIndexReader::get_analyse_result( - reader.get(), inverted_index_ctx->analyzer, column_name, - doris::segment_v2::InvertedIndexQueryType::MATCH_PHRASE_QUERY); + std::vector query_tokens; + doris::segment_v2::InvertedIndexReader::get_analyse_result( + query_tokens, reader.get(), inverted_index_ctx->analyzer, column_name, + doris::segment_v2::InvertedIndexQueryType::MATCH_PHRASE_QUERY); if (query_tokens.empty()) { LOG(WARNING) << fmt::format( "token parser result is empty for query, " @@ -313,6 +314,8 @@ void register_function_match(SimpleFunctionFactory& factory) { factory.register_function(); factory.register_function(); factory.register_function(); + factory.register_function(); + factory.register_function(); factory.register_function(); factory.register_function(); factory.register_function(); diff --git a/be/src/vec/functions/match.h b/be/src/vec/functions/match.h index b8e7f91cb019b8a..13701bd2d6000aa 100644 --- a/be/src/vec/functions/match.h +++ b/be/src/vec/functions/match.h @@ -128,6 +128,40 @@ class FunctionMatchPhrase : public FunctionMatchBase { ColumnUInt8::Container& result) override; }; +class FunctionMatchPhrasePrefix : public FunctionMatchBase { +public: + static constexpr auto name = "match_phrase_prefix"; + static FunctionPtr create() { return std::make_shared(); } + + String get_name() const override { return name; } + + Status execute_match(const std::string& column_name, const std::string& match_query_str, + size_t input_rows_count, const ColumnString* string_col, + InvertedIndexCtx* inverted_index_ctx, + const ColumnArray::Offsets64* array_offsets, + ColumnUInt8::Container& result) override { + return Status::Error( + "FunctionMatchPhrasePrefix not support execute_match"); + } +}; + +class FunctionMatchRegexp : public FunctionMatchBase { +public: + static constexpr auto name = "match_regexp"; + static FunctionPtr create() { return std::make_shared(); } + + String get_name() const override { return name; } + + Status execute_match(const std::string& column_name, const std::string& match_query_str, + size_t input_rows_count, const ColumnString* string_col, + InvertedIndexCtx* inverted_index_ctx, + const ColumnArray::Offsets64* array_offsets, + ColumnUInt8::Container& result) override { + return Status::Error( + "FunctionMatchRegexp not support execute_match"); + } +}; + class FunctionMatchElementEQ : public FunctionMatchBase { public: static constexpr auto name = "match_element_eq"; diff --git a/be/src/vec/olap/block_reader.cpp b/be/src/vec/olap/block_reader.cpp index fe670b0bb9c0029..9a0635f694d7ac6 100644 --- a/be/src/vec/olap/block_reader.cpp +++ b/be/src/vec/olap/block_reader.cpp @@ -123,7 +123,6 @@ Status BlockReader::_init_collect_iter(const ReaderParams& read_params) { _vcollect_iter.init(this, _is_rowsets_overlapping, read_params.read_orderby_key, read_params.read_orderby_key_reverse); - _reader_context.push_down_agg_type_opt = read_params.push_down_agg_type_opt; std::vector valid_rs_readers; for (int i = 0; i < read_params.rs_splits.size(); ++i) { diff --git a/be/src/vec/runtime/vdatetime_value.cpp b/be/src/vec/runtime/vdatetime_value.cpp index fbfa28560e793d8..d4aa060af309fa1 100644 --- a/be/src/vec/runtime/vdatetime_value.cpp +++ b/be/src/vec/runtime/vdatetime_value.cpp @@ -1729,10 +1729,11 @@ bool VecDateTimeValue::from_unixtime(int64_t timestamp, const std::string& timez if (!TimezoneUtils::find_cctz_time_zone(timezone, ctz)) { return false; } - return from_unixtime(timestamp, ctz); + from_unixtime(timestamp, ctz); + return true; } -bool VecDateTimeValue::from_unixtime(int64_t timestamp, const cctz::time_zone& ctz) { +void VecDateTimeValue::from_unixtime(int64_t timestamp, const cctz::time_zone& ctz) { static const cctz::time_point epoch = std::chrono::time_point_cast( std::chrono::system_clock::from_time_t(0)); @@ -1740,6 +1741,8 @@ bool VecDateTimeValue::from_unixtime(int64_t timestamp, const cctz::time_zone& c const auto tp = cctz::convert(t, ctz); + // there's no overflow check since it's hot path + _neg = 0; _type = TIME_DATETIME; _year = tp.year(); @@ -1748,8 +1751,6 @@ bool VecDateTimeValue::from_unixtime(int64_t timestamp, const cctz::time_zone& c _hour = tp.hour(); _minute = tp.minute(); _second = tp.second(); - - return true; } const char* VecDateTimeValue::month_name() const { @@ -3159,20 +3160,21 @@ bool DateV2Value::from_unixtime(int64_t timestamp, const std::string& timezon if (!TimezoneUtils::find_cctz_time_zone(timezone, ctz)) { return false; } - return from_unixtime(timestamp, ctz); + from_unixtime(timestamp, ctz); + return true; } template -bool DateV2Value::from_unixtime(int64_t timestamp, const cctz::time_zone& ctz) { +void DateV2Value::from_unixtime(int64_t timestamp, const cctz::time_zone& ctz) { static const cctz::time_point epoch = std::chrono::time_point_cast( std::chrono::system_clock::from_time_t(0)); cctz::time_point t = epoch + cctz::seconds(timestamp); - const auto tp = cctz::convert(t, ctz); + // there's no overflow check since it's hot path + set_time(tp.year(), tp.month(), tp.day(), tp.hour(), tp.minute(), tp.second(), 0); - return true; } template @@ -3182,11 +3184,12 @@ bool DateV2Value::from_unixtime(std::pair timestamp, if (!TimezoneUtils::find_cctz_time_zone(timezone, ctz)) { return false; } - return from_unixtime(timestamp, ctz); + from_unixtime(timestamp, ctz); + return true; } template -bool DateV2Value::from_unixtime(std::pair timestamp, +void DateV2Value::from_unixtime(std::pair timestamp, const cctz::time_zone& ctz) { static const cctz::time_point epoch = std::chrono::time_point_cast( @@ -3197,7 +3200,6 @@ bool DateV2Value::from_unixtime(std::pair timestamp, set_time(tp.year(), tp.month(), tp.day(), tp.hour(), tp.minute(), tp.second(), timestamp.second); - return true; } template @@ -3207,11 +3209,12 @@ bool DateV2Value::from_unixtime(int64_t timestamp, int32_t nano_seconds, if (!TimezoneUtils::find_cctz_time_zone(timezone, ctz)) { return false; } - return from_unixtime(timestamp, nano_seconds, ctz, scale); + from_unixtime(timestamp, nano_seconds, ctz, scale); + return true; } template -bool DateV2Value::from_unixtime(int64_t timestamp, int32_t nano_seconds, +void DateV2Value::from_unixtime(int64_t timestamp, int32_t nano_seconds, const cctz::time_zone& ctz, int scale) { static const cctz::time_point epoch = std::chrono::time_point_cast( @@ -3226,7 +3229,6 @@ bool DateV2Value::from_unixtime(int64_t timestamp, int32_t nano_seconds, set_time(tp.year(), tp.month(), tp.day(), tp.hour(), tp.minute(), tp.second(), nano_seconds / int_exp10(9 - scale) * int_exp10(6 - scale)); - return true; } template diff --git a/be/src/vec/runtime/vdatetime_value.h b/be/src/vec/runtime/vdatetime_value.h index 4a5387e5a704670..e043aa9028d689e 100644 --- a/be/src/vec/runtime/vdatetime_value.h +++ b/be/src/vec/runtime/vdatetime_value.h @@ -503,9 +503,10 @@ class VecDateTimeValue { // Now this type is a temp solution with little changes bool unix_timestamp(int64_t* timestamp, const cctz::time_zone& ctz) const; //construct datetime_value from timestamp and timezone - //timestamp is an internal timestamp value representing seconds since '1970-01-01 00:00:00' UTC + //timestamp is an internal timestamp value representing seconds since '1970-01-01 00:00:00' UTC. negative avaliable. + //we don't do any check in it because it's hot path. any usage want ensure the time legality should check itself. bool from_unixtime(int64_t, const std::string& timezone); - bool from_unixtime(int64_t, const cctz::time_zone& ctz); + void from_unixtime(int64_t, const cctz::time_zone& ctz); bool operator==(const VecDateTimeValue& other) const { // NOTE: This is not same with MySQL. @@ -964,14 +965,14 @@ class DateV2Value { bool unix_timestamp(std::pair* timestamp, const cctz::time_zone& ctz) const; //construct datetime_value from timestamp and timezone - //timestamp is an internal timestamp value representing seconds since '1970-01-01 00:00:00' UTC + //timestamp is an internal timestamp value representing seconds since '1970-01-01 00:00:00' UTC. negative avaliable. + //we don't do any check in it because it's hot path. any usage want ensure the time legality should check itself. bool from_unixtime(int64_t, const std::string& timezone); - bool from_unixtime(int64_t, const cctz::time_zone& ctz); + void from_unixtime(int64_t, const cctz::time_zone& ctz); bool from_unixtime(std::pair, const std::string& timezone); - bool from_unixtime(std::pair, const cctz::time_zone& ctz); - - bool from_unixtime(int64_t, int32_t, const std::string& timezone, const int scale); - bool from_unixtime(int64_t, int32_t, const cctz::time_zone& ctz, int scale); + void from_unixtime(std::pair, const cctz::time_zone& ctz); + bool from_unixtime(int64_t, int32_t, const std::string& timezone, int scale); + void from_unixtime(int64_t, int32_t, const cctz::time_zone& ctz, int scale); bool operator==(const DateV2Value& other) const { // NOTE: This is not same with MySQL. diff --git a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql index 4e80d7466d2f913..e798ecd7f2bb9c8 100644 --- a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql +++ b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql @@ -1798,3 +1798,11 @@ create table stats_test2 (id INT, value STRING) STORED AS PARQUET; insert into stats_test1 values (1, 'name1'), (2, 'name2'), (3, 'name3'); INSERT INTO stats_test2 VALUES (1, ';'), (2, '\*'); + +create table employee_gz(name string,salary string) +row format serde 'org.apache.hadoop.hive.serde2.OpenCSVSerde' +with serdeproperties +('quoteChar'='\"' +,'seperatorChar'=','); + +insert into employee_gz values ('a', '1.1'), ('b', '2.2'); diff --git a/docs/en/docs/sql-manual/sql-functions/date-time-functions/from-second.md b/docs/en/docs/sql-manual/sql-functions/date-time-functions/from-second.md new file mode 100644 index 000000000000000..bcbf92521ad4b81 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/date-time-functions/from-second.md @@ -0,0 +1,73 @@ +--- +{ + "title": "FROM_SECOND", + "language": "en" +} +--- + + + +## from_second +### description +#### syntax + +`DATETIME FROM_SECOND(BIGINT unix_timestamp)` +`DATETIME FROM_MILLISECOND(BIGINT unix_timestamp)` +`DATETIME FROM_MICROSECOND(BIGINT unix_timestamp)` + +Converts a timestamp to its DATETIME represent, with argument as an integer and returned as a DATETIME type. Returns `NULL` if `unix_timestamp < 0` or if the function result is greater than `9999-12-31 23:59:59.999999`. + +### example + +``` +mysql> set time_zone='Asia/Shanghai'; + +mysql> select from_second(-1); ++---------------------------+ +| from_second(-1) | ++---------------------------+ +| NULL | ++---------------------------+ + +mysql> select from_millisecond(12345678); ++----------------------------+ +| from_millisecond(12345678) | ++----------------------------+ +| 1970-01-01 11:25:45.678 | ++----------------------------+ + +mysql> select from_microsecond(253402271999999999); ++--------------------------------------+ +| from_microsecond(253402271999999999) | ++--------------------------------------+ +| 9999-12-31 23:59:59.999999 | ++--------------------------------------+ + +mysql> select from_microsecond(253402272000000000); ++--------------------------------------+ +| from_microsecond(253402272000000000) | ++--------------------------------------+ +| NULL | ++--------------------------------------+ +``` + +### keywords + + FROM_SECOND,FROM,SECOND,MILLISECOND,MICROSECOND diff --git a/docs/sidebars.json b/docs/sidebars.json index 35dc65bf34ac54f..01adcc1052fb3cb 100644 --- a/docs/sidebars.json +++ b/docs/sidebars.json @@ -347,6 +347,7 @@ "sql-manual/sql-functions/date-time-functions/from-days", "sql-manual/sql-functions/date-time-functions/last-day", "sql-manual/sql-functions/date-time-functions/to-monday", + "sql-manual/sql-functions/date-time-functions/from-second", "sql-manual/sql-functions/date-time-functions/from-unixtime", "sql-manual/sql-functions/date-time-functions/unix-timestamp", "sql-manual/sql-functions/date-time-functions/utc-timestamp", diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/date-time-functions/from-second.md b/docs/zh-CN/docs/sql-manual/sql-functions/date-time-functions/from-second.md index 601cb4a70711e3f..df3c031966ba9af 100644 --- a/docs/zh-CN/docs/sql-manual/sql-functions/date-time-functions/from-second.md +++ b/docs/zh-CN/docs/sql-manual/sql-functions/date-time-functions/from-second.md @@ -32,34 +32,19 @@ under the License. `DATETIME FROM_MILLISECOND(BIGINT unix_timestamp)` `DATETIME FROM_MICROSECOND(BIGINT unix_timestamp)` -将时间戳转化为对应的 DATETIME - -传入的是整型,返回的是DATETIME类型 - +将时间戳转化为对应的 DATETIME,传入的是整型,返回的是DATETIME类型。若`unix_timestamp < 0` 或函数结果大于 `9999-12-31 23:59:59.999999`,则返回`NULL`。 ### example ``` -mysql> select from_microsecond(0); -+----------------------------+ -| from_microsecond(0) | -+----------------------------+ -| 1970-01-01 08:00:00.000000 | -+----------------------------+ - -mysql> select from_microsecond(12345678); -+----------------------------+ -| from_microsecond(12345678) | -+----------------------------+ -| 1970-01-01 08:00:12.345678 | -+----------------------------+ +mysql> set time_zone='Asia/Shanghai'; -mysql> select from_millisecond(0); -+-------------------------+ -| from_millisecond(0) | -+-------------------------+ -| 1970-01-01 08:00:00.000 | -+-------------------------+ +mysql> select from_second(-1); ++---------------------------+ +| from_second(-1) | ++---------------------------+ +| NULL | ++---------------------------+ mysql> select from_millisecond(12345678); +----------------------------+ @@ -68,14 +53,21 @@ mysql> select from_millisecond(12345678); | 1970-01-01 11:25:45.678 | +----------------------------+ -mysql> select from_second(21474836470); -+--------------------------+ -| from_second(21474836470) | -+--------------------------+ -| 2650-07-06 16:21:10 | -+--------------------------+ +mysql> select from_microsecond(253402271999999999); ++--------------------------------------+ +| from_microsecond(253402271999999999) | ++--------------------------------------+ +| 9999-12-31 23:59:59.999999 | ++--------------------------------------+ + +mysql> select from_microsecond(253402272000000000); ++--------------------------------------+ +| from_microsecond(253402272000000000) | ++--------------------------------------+ +| NULL | ++--------------------------------------+ ``` ### keywords - FROM_SECOND + FROM_SECOND,FROM,SECOND,MILLISECOND,MICROSECOND diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 index abf6e4bb27f00e9..378a34f3da0bfdd 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 @@ -338,6 +338,8 @@ MATCH_ELEMENT_GT: 'ELEMENT_GT'; MATCH_ELEMENT_LE: 'ELEMENT_LE'; MATCH_ELEMENT_LT: 'ELEMENT_LT'; MATCH_PHRASE: 'MATCH_PHRASE'; +MATCH_PHRASE_PREFIX: 'MATCH_PHRASE_PREFIX'; +MATCH_REGEXP: 'MATCH_REGEXP'; MATERIALIZED: 'MATERIALIZED'; MAX: 'MAX'; MAXVALUE: 'MAXVALUE'; diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 8c9ece2799f7e02..ba44219e7e047f2 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -312,7 +312,7 @@ booleanExpression predicate : NOT? kind=BETWEEN lower=valueExpression AND upper=valueExpression | NOT? kind=(LIKE | REGEXP | RLIKE) pattern=valueExpression - | NOT? kind=(MATCH | MATCH_ANY | MATCH_ALL | MATCH_PHRASE) pattern=valueExpression + | NOT? kind=(MATCH | MATCH_ANY | MATCH_ALL | MATCH_PHRASE | MATCH_PHRASE_PREFIX | MATCH_REGEXP) pattern=valueExpression | NOT? kind=IN LEFT_PAREN query RIGHT_PAREN | NOT? kind=IN LEFT_PAREN expression (COMMA expression)* RIGHT_PAREN | IS NOT? kind=NULL @@ -440,7 +440,7 @@ specifiedPartition constant : NULL #nullLiteral | interval #intervalLiteral - | type=(DATE | DATEV1 | DATEV2 | TIMESTAMP) STRING_LITERAL #typeConstructor + | type=(DATE | DATEV1 | DATEV2 | TIMESTAMP) STRING_LITERAL #typeConstructor | number #numericLiteral | booleanValue #booleanLiteral | STRING_LITERAL #stringLiteral diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 07345f6b6b9c5ea..08ffc389a928145 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -468,6 +468,8 @@ terminal String KW_MATCH_ANY, KW_MATCH_ALL, KW_MATCH_PHRASE, + KW_MATCH_PHRASE_PREFIX, + KW_MATCH_REGEXP, KW_MATCH_ELEMENT_EQ, KW_MATCH_ELEMENT_LT, KW_MATCH_ELEMENT_GT, @@ -959,7 +961,7 @@ precedence left KW_AND; precedence left KW_NOT, NOT; precedence left KW_BETWEEN, KW_IN, KW_IS, KW_EXISTS; precedence left KW_LIKE, KW_REGEXP; -precedence left KW_MATCH_ANY, KW_MATCH_ALL, KW_MATCH_PHRASE, KW_MATCH, KW_MATCH_ELEMENT_EQ, KW_MATCH_ELEMENT_LT, KW_MATCH_ELEMENT_GT, KW_MATCH_ELEMENT_LE, KW_MATCH_ELEMENT_GE; +precedence left KW_MATCH_ANY, KW_MATCH_ALL, KW_MATCH_PHRASE, KW_MATCH_PHRASE_PREFIX, KW_MATCH_REGEXP, KW_MATCH, KW_MATCH_ELEMENT_EQ, KW_MATCH_ELEMENT_LT, KW_MATCH_ELEMENT_GT, KW_MATCH_ELEMENT_LE, KW_MATCH_ELEMENT_GE; precedence left EQUAL, LESSTHAN, GREATERTHAN; precedence left ADD, SUBTRACT; precedence left AT, STAR, DIVIDE, MOD, KW_DIV; @@ -6985,6 +6987,10 @@ match_predicate ::= {: RESULT = new MatchPredicate(MatchPredicate.Operator.MATCH_ALL, e1, e2); :} | expr:e1 KW_MATCH_PHRASE expr:e2 {: RESULT = new MatchPredicate(MatchPredicate.Operator.MATCH_PHRASE, e1, e2); :} + | expr:e1 KW_MATCH_PHRASE_PREFIX expr:e2 + {: RESULT = new MatchPredicate(MatchPredicate.Operator.MATCH_PHRASE_PREFIX, e1, e2); :} + | expr:e1 KW_MATCH_REGEXP expr:e2 + {: RESULT = new MatchPredicate(MatchPredicate.Operator.MATCH_REGEXP, e1, e2); :} | expr:e1 KW_MATCH_ELEMENT_EQ expr:e2 {: RESULT = new MatchPredicate(MatchPredicate.Operator.MATCH_ELEMENT_EQ, e1, e2); :} | expr:e1 KW_MATCH_ELEMENT_LT expr:e2 diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java index 38f02c49cc41e1e..6eaa45cdfddba7d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java @@ -39,6 +39,7 @@ import com.google.common.base.Joiner; import com.google.common.base.Preconditions; +import com.google.common.base.Splitter; import com.google.common.base.Strings; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; @@ -49,6 +50,7 @@ import java.util.Map; import java.util.Optional; import java.util.UUID; +import java.util.stream.Collectors; // EXPORT statement, export data to dirs by broker. // @@ -64,7 +66,6 @@ public class ExportStmt extends StatementBase { private static final String DEFAULT_COLUMN_SEPARATOR = "\t"; private static final String DEFAULT_LINE_DELIMITER = "\n"; - private static final String DEFAULT_COLUMNS = ""; private static final String DEFAULT_PARALLELISM = "1"; private static final Integer DEFAULT_TIMEOUT = 7200; @@ -121,7 +122,6 @@ public ExportStmt(TableRef tableRef, Expr whereExpr, String path, this.columnSeparator = DEFAULT_COLUMN_SEPARATOR; this.lineDelimiter = DEFAULT_LINE_DELIMITER; this.timeout = DEFAULT_TIMEOUT; - this.columns = DEFAULT_COLUMNS; // The ExportStmt may be created in replay thread, there is no ConnectionContext // in replay thread, so we need to clone session variable from default session variable. @@ -352,7 +352,14 @@ private void checkProperties(Map properties) throws UserExceptio properties, ExportStmt.DEFAULT_COLUMN_SEPARATOR)); this.lineDelimiter = Separator.convertSeparator(PropertyAnalyzer.analyzeLineDelimiter( properties, ExportStmt.DEFAULT_LINE_DELIMITER)); - this.columns = properties.getOrDefault(LoadStmt.KEY_IN_PARAM_COLUMNS, DEFAULT_COLUMNS); + + // null means not specified + // "" means user specified zero columns + this.columns = properties.getOrDefault(LoadStmt.KEY_IN_PARAM_COLUMNS, null); + // check columns are exits + if (this.columns != null) { + checkColumns(); + } // format this.format = properties.getOrDefault(LoadStmt.KEY_IN_PARAM_FORMAT_TYPE, "csv").toLowerCase(); @@ -384,6 +391,24 @@ private void checkProperties(Map properties) throws UserExceptio label = properties.get(LABEL); } + private void checkColumns() throws DdlException { + if (this.columns.isEmpty()) { + throw new DdlException("columns can not be empty"); + } + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(this.tblName.getDb()); + Table table = db.getTableOrDdlException(this.tblName.getTbl()); + List tableColumns = table.getBaseSchema().stream().map(column -> column.getName()) + .collect(Collectors.toList()); + Splitter split = Splitter.on(',').trimResults().omitEmptyStrings(); + + List columnsSpecified = split.splitToList(this.columns.toLowerCase()); + for (String columnName : columnsSpecified) { + if (!tableColumns.contains(columnName)) { + throw new DdlException("unknown column [" + columnName + "] in table [" + this.tblName.getTbl() + "]"); + } + } + } + @Override public String toSql() { StringBuilder sb = new StringBuilder(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/MatchPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/MatchPredicate.java index 10579614524e1d1..f106aec956c72c0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/MatchPredicate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/MatchPredicate.java @@ -50,6 +50,8 @@ public enum Operator { MATCH_ANY("MATCH_ANY", "match_any", TExprOpcode.MATCH_ANY), MATCH_ALL("MATCH_ALL", "match_all", TExprOpcode.MATCH_ALL), MATCH_PHRASE("MATCH_PHRASE", "match_phrase", TExprOpcode.MATCH_PHRASE), + MATCH_PHRASE_PREFIX("MATCH_PHRASE_PREFIX", "match_phrase_prefix", TExprOpcode.MATCH_PHRASE_PREFIX), + MATCH_REGEXP("MATCH_REGEXP", "match_regexp", TExprOpcode.MATCH_REGEXP), MATCH_ELEMENT_EQ("MATCH_ELEMENT_EQ", "match_element_eq", TExprOpcode.MATCH_ELEMENT_EQ), MATCH_ELEMENT_LT("MATCH_ELEMENT_LT", "match_element_lt", TExprOpcode.MATCH_ELEMENT_LT), MATCH_ELEMENT_GT("MATCH_ELEMENT_GT", "match_element_gt", TExprOpcode.MATCH_ELEMENT_GT), @@ -147,6 +149,26 @@ public static void initBuiltins(FunctionSet functionSet) { symbolNotUsed, Lists.newArrayList(new ArrayType(t), t), Type.BOOLEAN)); + functionSet.addBuiltinBothScalaAndVectorized(ScalarFunction.createBuiltinOperator( + Operator.MATCH_PHRASE_PREFIX.getName(), + symbolNotUsed, + Lists.newArrayList(t, t), + Type.BOOLEAN)); + functionSet.addBuiltinBothScalaAndVectorized(ScalarFunction.createBuiltinOperator( + Operator.MATCH_PHRASE_PREFIX.getName(), + symbolNotUsed, + Lists.newArrayList(new ArrayType(t), t), + Type.BOOLEAN)); + functionSet.addBuiltinBothScalaAndVectorized(ScalarFunction.createBuiltinOperator( + Operator.MATCH_REGEXP.getName(), + symbolNotUsed, + Lists.newArrayList(t, t), + Type.BOOLEAN)); + functionSet.addBuiltinBothScalaAndVectorized(ScalarFunction.createBuiltinOperator( + Operator.MATCH_REGEXP.getName(), + symbolNotUsed, + Lists.newArrayList(new ArrayType(t), t), + Type.BOOLEAN)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java index b3ced6d92cdc6dd..967ceb4b272994c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java @@ -503,6 +503,24 @@ public void analyze(Analyzer analyzer) throws UserException { } fromClause.setNeedToSql(needToSql); fromClause.analyze(analyzer); + + if (!isForbiddenMVRewrite()) { + Boolean haveMv = false; + for (TableRef tbl : fromClause) { + if (!tbl.haveDesc() || !(tbl.getTable() instanceof OlapTable)) { + continue; + } + OlapTable olapTable = (OlapTable) tbl.getTable(); + if (olapTable.getIndexIds().size() != 1) { + haveMv = true; + } + } + + if (!haveMv) { + forbiddenMVRewrite(); + } + } + // Generate !empty() predicates to filter out empty collections. // Skip this step when analyzing a WITH-clause because CollectionTableRefs // do not register collection slots in their parent in that context diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java index b6c47e1cbfc3663..e4b5caa44689311 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java @@ -329,6 +329,10 @@ public TableSnapshot getTableSnapshot() { return tableSnapshot; } + public Boolean haveDesc() { + return desc != null; + } + /** * This method should only be called after the TableRef has been analyzed. */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java index 97052caccd6e2f5..11969cb86543c32 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java @@ -70,7 +70,7 @@ public abstract class JdbcClient { protected final ConcurrentHashMap lowerTableToRealTable = new ConcurrentHashMap<>(); private final AtomicBoolean dbNamesLoaded = new AtomicBoolean(false); - private final AtomicBoolean tableNamesLoaded = new AtomicBoolean(false); + private final ConcurrentHashMap tableNamesLoadedMap = new ConcurrentHashMap<>(); public static JdbcClient createJdbcClient(JdbcClientConfig jdbcClientConfig) { String dbType = parseDbType(jdbcClientConfig.getJdbcUrl()); @@ -382,7 +382,8 @@ private void loadDatabaseNamesIfNeeded() { } private void loadTableNamesIfNeeded(String dbName) { - if (tableNamesLoaded.compareAndSet(false, true)) { + AtomicBoolean isLoaded = tableNamesLoadedMap.computeIfAbsent(dbName, k -> new AtomicBoolean(false)); + if (isLoaded.compareAndSet(false, true)) { getTablesNameList(dbName); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 23cb6c572e2a262..447f0c28442354c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -158,6 +158,8 @@ import org.apache.doris.nereids.trees.expressions.MatchAll; import org.apache.doris.nereids.trees.expressions.MatchAny; import org.apache.doris.nereids.trees.expressions.MatchPhrase; +import org.apache.doris.nereids.trees.expressions.MatchPhrasePrefix; +import org.apache.doris.nereids.trees.expressions.MatchRegexp; import org.apache.doris.nereids.trees.expressions.Mod; import org.apache.doris.nereids.trees.expressions.Multiply; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -1927,6 +1929,18 @@ private Expression withPredicate(Expression valueExpression, PredicateContext ct getExpression(ctx.pattern) ); break; + case DorisParser.MATCH_PHRASE_PREFIX: + outExpression = new MatchPhrasePrefix( + valueExpression, + getExpression(ctx.pattern) + ); + break; + case DorisParser.MATCH_REGEXP: + outExpression = new MatchRegexp( + valueExpression, + getExpression(ctx.pattern) + ); + break; default: throw new ParseException("Unsupported predicate type: " + ctx.kind.getText(), ctx); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java index dd7f1197b119f23..a7eb7c7e5cc42da 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java @@ -41,9 +41,9 @@ import com.google.common.collect.ImmutableList.Builder; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -101,23 +101,94 @@ public class NormalizeAggregate extends OneRewriteRuleFactory implements Normali @Override public Rule build() { return logicalAggregate().whenNot(LogicalAggregate::isNormalized).then(aggregate -> { + // The LogicalAggregate node may contain window agg functions and usual agg functions + // we call window agg functions as window-agg and usual agg functions as trival-agg for short + // This rule simplify LogicalAggregate node by: + // 1. Push down some exprs from old LogicalAggregate node to a new child LogicalProject Node, + // 2. create a new LogicalAggregate with normalized group by exprs and trival-aggs + // 3. Pull up normalized old LogicalAggregate's output exprs to a new parent LogicalProject Node + // Push down exprs: + // 1. all group by exprs + // 2. child contains subquery expr in trival-agg + // 3. child contains window expr in trival-agg + // 4. all input slots of trival-agg + // 5. expr(including subquery) in distinct trival-agg + // Normalize LogicalAggregate's output. + // 1. normalize group by exprs by outputs of bottom LogicalProject + // 2. normalize trival-aggs by outputs of bottom LogicalProject + // 3. build normalized agg outputs + // Pull up exprs: + // normalize all output exprs in old LogicalAggregate to build a parent project node, typically includes: + // 1. simple slots + // 2. aliases + // a. alias with no aggs child + // b. alias with trival-agg child + // c. alias with window-agg - List aggregateOutput = aggregate.getOutputExpressions(); - Set existsAlias = ExpressionUtils.mutableCollect(aggregateOutput, Alias.class::isInstance); + // Push down exprs: + // collect group by exprs + Set groupingByExprs = + ImmutableSet.copyOf(aggregate.getGroupByExpressions()); + // collect all trival-agg + List aggregateOutput = aggregate.getOutputExpressions(); List aggFuncs = Lists.newArrayList(); aggregateOutput.forEach(o -> o.accept(CollectNonWindowedAggFuncs.INSTANCE, aggFuncs)); - // we need push down subquery exprs inside non-window and non-distinct agg functions - // because the distinct agg's children would be pushed down in later process - Set subqueryExprs = ExpressionUtils.mutableCollect(aggFuncs.stream() - .filter(aggFunc -> !aggFunc.isDistinct()).collect(Collectors.toList()), - SubqueryExpr.class::isInstance); - Set groupingByExprs = ImmutableSet.copyOf(aggregate.getGroupByExpressions()); + // split non-distinct agg child as two part + // TRUE part 1: need push down itself, if it contains subqury or window expression + // FALSE part 2: need push down its input slots, if it DOES NOT contain subqury or window expression + Map> categorizedNoDistinctAggsChildren = aggFuncs.stream() + .filter(aggFunc -> !aggFunc.isDistinct()) + .flatMap(agg -> agg.children().stream()) + .collect(Collectors.groupingBy( + child -> child.containsType(SubqueryExpr.class, WindowExpression.class), + Collectors.toSet())); + + // split distinct agg child as two parts + // TRUE part 1: need push down itself, if it is NOT SlotReference or Literal + // FALSE part 2: need push down its input slots, if it is SlotReference or Literal + Map> categorizedDistinctAggsChildren = aggFuncs.stream() + .filter(aggFunc -> aggFunc.isDistinct()).flatMap(agg -> agg.children().stream()) + .collect(Collectors.groupingBy( + child -> !(child instanceof SlotReference || child instanceof Literal), + Collectors.toSet())); + + Set needPushSelf = Sets.union( + categorizedNoDistinctAggsChildren.getOrDefault(true, new HashSet<>()), + categorizedDistinctAggsChildren.getOrDefault(true, new HashSet<>())); + Set needPushInputSlots = ExpressionUtils.getInputSlotSet(Sets.union( + categorizedNoDistinctAggsChildren.getOrDefault(false, new HashSet<>()), + categorizedDistinctAggsChildren.getOrDefault(false, new HashSet<>()))); + + Set existsAlias = + ExpressionUtils.mutableCollect(aggregateOutput, Alias.class::isInstance); + + // push down 3 kinds of exprs, these pushed exprs will be used to normalize agg output later + // 1. group by exprs + // 2. trivalAgg children + // 3. trivalAgg input slots + Set allPushDownExprs = + Sets.union(groupingByExprs, Sets.union(needPushSelf, needPushInputSlots)); NormalizeToSlotContext bottomSlotContext = - NormalizeToSlotContext.buildContext(existsAlias, Sets.union(groupingByExprs, subqueryExprs)); - Set bottomOutputs = - bottomSlotContext.pushDownToNamedExpression(Sets.union(groupingByExprs, subqueryExprs)); + NormalizeToSlotContext.buildContext(existsAlias, allPushDownExprs); + Set pushedGroupByExprs = + bottomSlotContext.pushDownToNamedExpression(groupingByExprs); + Set pushedTrivalAggChildren = + bottomSlotContext.pushDownToNamedExpression(needPushSelf); + Set pushedTrivalAggInputSlots = + bottomSlotContext.pushDownToNamedExpression(needPushInputSlots); + Set bottomProjects = Sets.union(pushedGroupByExprs, + Sets.union(pushedTrivalAggChildren, pushedTrivalAggInputSlots)); + + // create bottom project + Plan bottomPlan; + if (!bottomProjects.isEmpty()) { + bottomPlan = new LogicalProject<>(ImmutableList.copyOf(bottomProjects), + aggregate.child()); + } else { + bottomPlan = aggregate.child(); + } // use group by context to normalize agg functions to process // sql like: select sum(a + 1) from t group by a + 1 @@ -129,89 +200,37 @@ public Rule build() { // after normalize: // agg(output: sum(alias(a + 1)[#1])[#2], group_by: alias(a + 1)[#1]) // +-- project((a[#0] + 1)[#1]) - List normalizedAggFuncs = bottomSlotContext.normalizeToUseSlotRef(aggFuncs); - Set bottomProjects = Sets.newHashSet(bottomOutputs); - // TODO: if we have distinct agg, we must push down its children, - // because need use it to generate distribution enforce - // step 1: split agg functions into 2 parts: distinct and not distinct - List distinctAggFuncs = Lists.newArrayList(); - List nonDistinctAggFuncs = Lists.newArrayList(); - for (AggregateFunction aggregateFunction : normalizedAggFuncs) { - if (aggregateFunction.isDistinct()) { - distinctAggFuncs.add(aggregateFunction); - } else { - nonDistinctAggFuncs.add(aggregateFunction); - } - } - // step 2: if we only have one distinct agg function, we do push down for it - if (!distinctAggFuncs.isEmpty()) { - // process distinct normalize and put it back to normalizedAggFuncs - List newDistinctAggFuncs = Lists.newArrayList(); - Map replaceMap = Maps.newHashMap(); - Map aliasCache = Maps.newHashMap(); - for (AggregateFunction distinctAggFunc : distinctAggFuncs) { - List newChildren = Lists.newArrayList(); - for (Expression child : distinctAggFunc.children()) { - if (child instanceof SlotReference || child instanceof Literal) { - newChildren.add(child); - } else { - NamedExpression alias; - if (aliasCache.containsKey(child)) { - alias = aliasCache.get(child); - } else { - alias = new Alias(child); - aliasCache.put(child, alias); - } - bottomProjects.add(alias); - newChildren.add(alias.toSlot()); - } - } - AggregateFunction newDistinctAggFunc = distinctAggFunc.withChildren(newChildren); - replaceMap.put(distinctAggFunc, newDistinctAggFunc); - newDistinctAggFuncs.add(newDistinctAggFunc); - } - aggregateOutput = aggregateOutput.stream() - .map(e -> ExpressionUtils.replace(e, replaceMap)) - .map(NamedExpression.class::cast) - .collect(Collectors.toList()); - distinctAggFuncs = newDistinctAggFuncs; - } - normalizedAggFuncs = Lists.newArrayList(nonDistinctAggFuncs); - normalizedAggFuncs.addAll(distinctAggFuncs); - // TODO: process redundant expressions in aggregate functions children + + // normalize group by exprs by bottomProjects + List normalizedGroupExprs = + bottomSlotContext.normalizeToUseSlotRef(groupingByExprs); + + // normalize trival-aggs by bottomProjects + List normalizedAggFuncs = + bottomSlotContext.normalizeToUseSlotRef(aggFuncs); + // build normalized agg output NormalizeToSlotContext normalizedAggFuncsToSlotContext = NormalizeToSlotContext.buildContext(existsAlias, normalizedAggFuncs); - // agg output include 2 part, normalized group by slots and normalized agg functions + + // agg output include 2 parts + // pushedGroupByExprs and normalized agg functions List normalizedAggOutput = ImmutableList.builder() - .addAll(bottomOutputs.stream().map(NamedExpression::toSlot).iterator()) - .addAll(normalizedAggFuncsToSlotContext.pushDownToNamedExpression(normalizedAggFuncs)) + .addAll(pushedGroupByExprs.stream().map(NamedExpression::toSlot).iterator()) + .addAll(normalizedAggFuncsToSlotContext + .pushDownToNamedExpression(normalizedAggFuncs)) .build(); - // add normalized agg's input slots to bottom projects - Set bottomProjectSlots = bottomProjects.stream() - .map(NamedExpression::toSlot) - .collect(Collectors.toSet()); - Set aggInputSlots = normalizedAggFuncs.stream() - .map(Expression::getInputSlots) - .flatMap(Set::stream) - .filter(e -> !bottomProjectSlots.contains(e)) - .collect(Collectors.toSet()); - bottomProjects.addAll(aggInputSlots); - // build group by exprs - List normalizedGroupExprs = bottomSlotContext.normalizeToUseSlotRef(groupingByExprs); - Plan bottomPlan; - if (!bottomProjects.isEmpty()) { - bottomPlan = new LogicalProject<>(ImmutableList.copyOf(bottomProjects), aggregate.child()); - } else { - bottomPlan = aggregate.child(); - } + // create new agg node + LogicalAggregate newAggregate = + aggregate.withNormalized(normalizedGroupExprs, normalizedAggOutput, bottomPlan); + // create upper projects by normalize all output exprs in old LogicalAggregate List upperProjects = normalizeOutput(aggregateOutput, bottomSlotContext, normalizedAggFuncsToSlotContext); - return new LogicalProject<>(upperProjects, - aggregate.withNormalized(normalizedGroupExprs, normalizedAggOutput, bottomPlan)); + // create a parent project node + return new LogicalProject<>(upperProjects, newAggregate); }).toRule(RuleType.NORMALIZE_AGGREGATE); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java index 9c04b59ce6df6b1..d5c9ded22e360d7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java @@ -512,6 +512,10 @@ private Statistics estimateColumnLessThanColumn(Expression leftExpr, ColumnStati context.addKeyIfSlot(rightExpr); return statistics; } + if (leftRange.isInfinite() || rightRange.isInfinite()) { + return context.statistics.withSel(DEFAULT_INEQUALITY_COEFFICIENT); + } + double leftOverlapPercent = leftRange.overlapPercentWith(rightRange); // Left always greater than right if (leftOverlapPercent == 0) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Match.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Match.java index bc9837eafec5bc6..5b3027365a8d91c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Match.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Match.java @@ -50,6 +50,10 @@ public Operator op() throws AnalysisException { return Operator.MATCH_ALL; case "MATCH_PHRASE": return Operator.MATCH_PHRASE; + case "MATCH_PHRASE_PREFIX": + return Operator.MATCH_PHRASE_PREFIX; + case "MATCH_REGEXP": + return Operator.MATCH_REGEXP; default: throw new AnalysisException("UnSupported type: " + symbol); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/MatchPhrasePrefix.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/MatchPhrasePrefix.java new file mode 100644 index 000000000000000..748da21ce30c685 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/MatchPhrasePrefix.java @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions; + +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * like expression: a MATCH_PHRASE_PREFIX 'hello w'. + */ +public class MatchPhrasePrefix extends Match { + public MatchPhrasePrefix(Expression left, Expression right) { + super(ImmutableList.of(left, right), "MATCH_PHRASE_PREFIX"); + } + + private MatchPhrasePrefix(List children) { + super(children, "MATCH_PHRASE_PREFIX"); + } + + @Override + public MatchPhrasePrefix withChildren(List children) { + Preconditions.checkArgument(children.size() == 2); + return new MatchPhrasePrefix(children); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitMatchPhrasePrefix(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/MatchRegexp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/MatchRegexp.java new file mode 100644 index 000000000000000..6bb55aeb8978a5e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/MatchRegexp.java @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions; + +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * like expression: a MATCH_REGEXP '^h\\w*'. + */ +public class MatchRegexp extends Match { + public MatchRegexp(Expression left, Expression right) { + super(ImmutableList.of(left, right), "MATCH_REGEXP"); + } + + private MatchRegexp(List children) { + super(children, "MATCH_REGEXP"); + } + + @Override + public MatchRegexp withChildren(List children) { + Preconditions.checkArgument(children.size() == 2); + return new MatchRegexp(children); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitMatchRegexp(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Numbers.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Numbers.java index c5febcf9749f447..9b24fcfd6239bff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Numbers.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Numbers.java @@ -22,7 +22,6 @@ import org.apache.doris.catalog.Type; import org.apache.doris.common.NereidsException; import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Properties; import org.apache.doris.nereids.trees.expressions.Slot; @@ -88,16 +87,6 @@ public R accept(ExpressionVisitor visitor, C context) { return visitor.visitNumbers(this, context); } - @Override - public PhysicalProperties getPhysicalProperties() { - // TODO: use gather after coordinator support plan gather scan - // String backendNum = getTVFProperties().getMap().getOrDefault(NumbersTableValuedFunction.BACKEND_NUM, "1"); - // if (backendNum.trim().equals("1")) { - // return PhysicalProperties.GATHER; - // } - return PhysicalProperties.ANY; - } - @Override public Numbers withChildren(List children) { Preconditions.checkArgument(children().size() == 1 diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java index 5acc73eb75a9f83..c0969362ed6c2db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java @@ -109,7 +109,7 @@ public boolean nullable() { } public PhysicalProperties getPhysicalProperties() { - return PhysicalProperties.ANY; + return PhysicalProperties.STORAGE_ANY; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java index 4d89227ee9fcf8b..179570b824e21c5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java @@ -56,6 +56,8 @@ import org.apache.doris.nereids.trees.expressions.MatchAll; import org.apache.doris.nereids.trees.expressions.MatchAny; import org.apache.doris.nereids.trees.expressions.MatchPhrase; +import org.apache.doris.nereids.trees.expressions.MatchPhrasePrefix; +import org.apache.doris.nereids.trees.expressions.MatchRegexp; import org.apache.doris.nereids.trees.expressions.Mod; import org.apache.doris.nereids.trees.expressions.Multiply; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -454,6 +456,14 @@ public R visitMatchPhrase(MatchPhrase matchPhrase, C context) { return visitMatch(matchPhrase, context); } + public R visitMatchPhrasePrefix(MatchPhrasePrefix matchPhrasePrefix, C context) { + return visitMatch(matchPhrasePrefix, context); + } + + public R visitMatchRegexp(MatchRegexp matchRegexp, C context) { + return visitMatch(matchRegexp, context); + } + /* ******************************************************************************************** * Unbound expressions * ********************************************************************************************/ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java index fa6fd6290083896..ccaadd7cd9ccd3e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java @@ -130,6 +130,14 @@ public LogicalPlan getLogicalQuery() { @Override public Plan getExplainPlan(ConnectContext ctx) { + if (!ctx.getSessionVariable().isEnableNereidsDML()) { + try { + ctx.getSessionVariable().enableFallbackToOriginalPlannerOnce(); + } catch (Exception e) { + throw new AnalysisException("failed to set fallback to original planner to true", e); + } + throw new AnalysisException("Nereids DML is disabled, will try to fall back to the original planner"); + } return completeQueryPlan(ctx, logicalQuery); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java index 33bbfe8d546aa4a..0ff6099163c69ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java @@ -160,6 +160,14 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { @Override public Plan getExplainPlan(ConnectContext ctx) { + if (!ctx.getSessionVariable().isEnableNereidsDML()) { + try { + ctx.getSessionVariable().enableFallbackToOriginalPlannerOnce(); + } catch (Exception e) { + throw new AnalysisException("failed to set fallback to original planner to true", e); + } + throw new AnalysisException("Nereids DML is disabled, will try to fall back to the original planner"); + } return this.logicalQuery; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java index 6236ba019b5f9b4..92f3fb21ee54148 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java @@ -148,6 +148,14 @@ private void checkTable(ConnectContext ctx) throws AnalysisException { @Override public Plan getExplainPlan(ConnectContext ctx) throws AnalysisException { + if (!ctx.getSessionVariable().isEnableNereidsDML()) { + try { + ctx.getSessionVariable().enableFallbackToOriginalPlannerOnce(); + } catch (Exception e) { + throw new AnalysisException("failed to set fallback to original planner to true", e); + } + throw new AnalysisException("Nereids DML is disabled, will try to fall back to the original planner"); + } return completeQueryPlan(ctx, logicalQuery); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java index 9967e5cf0b6a59e..7baa65d264783b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java @@ -207,6 +207,8 @@ public static DataType convertPrimitiveFromStrings(List types, boolean t case "date": return Config.enable_date_conversion && tryConvert ? DateV2Type.INSTANCE : DateType.INSTANCE; + case "datev1": + return DateType.INSTANCE; case "datev2": return DateV2Type.INSTANCE; case "time": @@ -222,6 +224,15 @@ public static DataType convertPrimitiveFromStrings(List types, boolean t default: throw new AnalysisException("Nereids do not support type: " + type); } + case "datetimev1": + switch (types.size()) { + case 1: + return DateTimeType.INSTANCE; + case 2: + throw new AnalysisException("Nereids do not support datetimev1 type with precision"); + default: + throw new AnalysisException("Nereids do not support type: " + type); + } case "datetimev2": switch (types.size()) { case 1: diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/HiveScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/HiveScanNode.java index 943d30017e7c2aa..58b93112477b407 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/HiveScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/HiveScanNode.java @@ -76,6 +76,9 @@ public class HiveScanNode extends FileQueryScanNode { public static final String DEFAULT_FIELD_DELIMITER = "\1"; // "\x01" public static final String PROP_LINE_DELIMITER = "line.delim"; public static final String DEFAULT_LINE_DELIMITER = "\n"; + public static final String PROP_SEPERATOR_CHAR = "seperatorChar"; + public static final String PROP_QUOTA_CHAR = "quoteChar"; + public static final String PROP_COLLECTION_DELIMITER_HIVE2 = "colelction.delim"; public static final String PROP_COLLECTION_DELIMITER_HIVE3 = "collection.delim"; @@ -362,7 +365,16 @@ protected Map getLocationProperties() throws UserException { protected TFileAttributes getFileAttributes() throws UserException { TFileTextScanRangeParams textParams = new TFileTextScanRangeParams(); java.util.Map delimiter = hmsTable.getRemoteTable().getSd().getSerdeInfo().getParameters(); - textParams.setColumnSeparator(delimiter.getOrDefault(PROP_FIELD_DELIMITER, DEFAULT_FIELD_DELIMITER)); + if (delimiter.containsKey(PROP_FIELD_DELIMITER)) { + textParams.setColumnSeparator(delimiter.get(PROP_FIELD_DELIMITER)); + } else if (delimiter.containsKey(PROP_SEPERATOR_CHAR)) { + textParams.setColumnSeparator(delimiter.get(PROP_SEPERATOR_CHAR)); + } else { + textParams.setColumnSeparator(DEFAULT_FIELD_DELIMITER); + } + if (delimiter.containsKey(PROP_QUOTA_CHAR)) { + textParams.setEnclose(delimiter.get(PROP_QUOTA_CHAR).getBytes()[0]); + } textParams.setLineDelimiter(delimiter.getOrDefault(PROP_LINE_DELIMITER, DEFAULT_LINE_DELIMITER)); textParams.setMapkvDelimiter(delimiter.getOrDefault(PROP_MAP_KV_DELIMITER, DEFAULT_MAP_KV_DELIMITER)); @@ -377,6 +389,9 @@ protected TFileAttributes getFileAttributes() throws UserException { TFileAttributes fileAttributes = new TFileAttributes(); fileAttributes.setTextParams(textParams); fileAttributes.setHeaderType(""); + if (textParams.isSet(TFileTextScanRangeParams._Fields.ENCLOSE)) { + fileAttributes.setTrimDoubleQuotes(true); + } return fileAttributes; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 2fd5dd7adea7b77..da4212f04185e4a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -402,6 +402,7 @@ public class SessionVariable implements Serializable, Writable { public static final String ENABLE_UNIQUE_KEY_PARTIAL_UPDATE = "enable_unique_key_partial_update"; public static final String INVERTED_INDEX_CONJUNCTION_OPT_THRESHOLD = "inverted_index_conjunction_opt_threshold"; + public static final String INVERTED_INDEX_MAX_EXPANSIONS = "inverted_index_max_expansions"; public static final String AUTO_ANALYZE_START_TIME = "auto_analyze_start_time"; @@ -1192,6 +1193,12 @@ public void setMaxJoinNumberOfReorder(int maxJoinNumberOfReorder) { flag = VariableMgr.GLOBAL) public String autoAnalyzeEndTime = "23:59:59"; + @VariableMgr.VarAttr(name = INVERTED_INDEX_MAX_EXPANSIONS, + description = {"这个参数用来限制查询时扩展的词项(terms)的数量,以此来控制查询的性能", + "This parameter is used to limit the number of term expansions during a query," + + " thereby controlling query performance"}) + public int invertedIndexMaxExpansions = 50; + @VariableMgr.VarAttr(name = ENABLE_UNIQUE_KEY_PARTIAL_UPDATE, needForward = true) public boolean enableUniqueKeyPartialUpdate = false; @@ -2435,6 +2442,7 @@ public TQueryOptions toThrift() { tResult.setTruncateCharOrVarcharColumns(truncateCharOrVarcharColumns); tResult.setInvertedIndexConjunctionOptThreshold(invertedIndexConjunctionOptThreshold); + tResult.setInvertedIndexMaxExpansions(invertedIndexMaxExpansions); tResult.setFasterFloatConvert(fasterFloatConvert); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticRange.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticRange.java index 74b77c2ee7c91ff..7b7b08ab24669de 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticRange.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticRange.java @@ -239,4 +239,8 @@ public double getDistinctValues() { return distinctValues; } + @Override + public String toString() { + return "(" + lowExpr + "," + highExpr + ")"; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java index 6e429f033f84611..8cbb9df23716e0a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java @@ -39,6 +39,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; @@ -194,10 +195,34 @@ public static void dropStatistics(long tblId, Set colNames) throws DdlEx public static void dropStatisticsByColName(long tblId, Set colNames, String statsTblName) throws DdlException { Map params = new HashMap<>(); - String right = colNames.stream().map(s -> "'" + s + "'").collect(Collectors.joining(",")); - String inPredicate = String.format("tbl_id = %s AND %s IN (%s)", tblId, "col_id", right); params.put("tblName", statsTblName); - params.put("condition", inPredicate); + Iterator iterator = colNames.iterator(); + int columnCount = 0; + StringBuilder inPredicate = new StringBuilder(); + while (iterator.hasNext()) { + inPredicate.append("'"); + inPredicate.append(iterator.next()); + inPredicate.append("'"); + inPredicate.append(","); + columnCount++; + if (columnCount == Config.max_allowed_in_element_num_of_delete) { + executeDropSql(inPredicate, tblId, params); + columnCount = 0; + inPredicate.setLength(0); + } + } + if (inPredicate.length() > 0) { + executeDropSql(inPredicate, tblId, params); + } + } + + public static void executeDropSql(StringBuilder inPredicate, long tblId, Map params) + throws DdlException { + if (inPredicate.length() > 0) { + inPredicate.delete(inPredicate.length() - 1, inPredicate.length()); + } + String predicate = String.format("tbl_id = '%s' AND %s IN (%s)", tblId, "col_id", inPredicate); + params.put("condition", predicate); try { StatisticsUtil.execUpdate(new StringSubstitutor(params).replace(DROP_TABLE_STATISTICS_TEMPLATE)); } catch (Exception e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java index a9429a35578eed1..9c350e9049cb5dc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java @@ -149,7 +149,11 @@ public static QueryState execUpdate(String sql) throws Exception { StmtExecutor stmtExecutor = new StmtExecutor(r.connectContext, sql); r.connectContext.setExecutor(stmtExecutor); stmtExecutor.execute(); - return r.connectContext.getState(); + QueryState state = r.connectContext.getState(); + if (state.getStateType().equals(QueryState.MysqlStateType.ERR)) { + throw new Exception(state.getErrorMessage()); + } + return state; } } diff --git a/fe/fe-core/src/main/jflex/sql_scanner.flex b/fe/fe-core/src/main/jflex/sql_scanner.flex index 4a19494ce80c992..86a3dc7482de609 100644 --- a/fe/fe-core/src/main/jflex/sql_scanner.flex +++ b/fe/fe-core/src/main/jflex/sql_scanner.flex @@ -309,6 +309,8 @@ import org.apache.doris.qe.SqlModeHelper; keywordMap.put("match_any", new Integer(SqlParserSymbols.KW_MATCH_ANY)); keywordMap.put("match_all", new Integer(SqlParserSymbols.KW_MATCH_ALL)); keywordMap.put("match_phrase", new Integer(SqlParserSymbols.KW_MATCH_PHRASE)); + keywordMap.put("match_phrase_prefix", new Integer(SqlParserSymbols.KW_MATCH_PHRASE_PREFIX)); + keywordMap.put("match_regexp", new Integer(SqlParserSymbols.KW_MATCH_REGEXP)); keywordMap.put("element_eq", new Integer(SqlParserSymbols.KW_MATCH_ELEMENT_EQ)); keywordMap.put("element_lt", new Integer(SqlParserSymbols.KW_MATCH_ELEMENT_LT)); keywordMap.put("element_gt", new Integer(SqlParserSymbols.KW_MATCH_ELEMENT_GT)); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java index d3064bde01224a0..128569af31b795a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java @@ -21,6 +21,7 @@ import org.apache.doris.common.Config; import org.apache.doris.common.Pair; import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.exceptions.ParseException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.trees.expressions.Cast; @@ -36,6 +37,8 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.types.DateTimeType; +import org.apache.doris.nereids.types.DateType; import org.apache.doris.nereids.types.DecimalV2Type; import org.apache.doris.nereids.types.DecimalV3Type; @@ -261,6 +264,26 @@ public void testParseDecimal() { Assertions.assertEquals(doubleCount, Config.enable_decimal_conversion ? 0 : 1); } + @Test + public void testDatev1() { + String dv1 = "SELECT CAST('2023-12-18' AS DATEV1) FROM t"; + NereidsParser nereidsParser = new NereidsParser(); + LogicalPlan logicalPlan = (LogicalPlan) nereidsParser.parseSingle(dv1).child(0); + Assertions.assertEquals(DateType.INSTANCE, logicalPlan.getExpressions().get(0).getDataType()); + } + + @Test + public void testDatetimev1() { + String dtv1 = "SELECT CAST('2023-12-18' AS DATETIMEV1) FROM t"; + NereidsParser nereidsParser = new NereidsParser(); + LogicalPlan logicalPlan = (LogicalPlan) nereidsParser.parseSingle(dtv1).child(0); + Assertions.assertEquals(DateTimeType.INSTANCE, logicalPlan.getExpressions().get(0).getDataType()); + + String wrongDtv1 = "SELECT CAST('2023-12-18' AS DATETIMEV1(2))"; + Assertions.assertThrows(AnalysisException.class, () -> nereidsParser.parseSingle(wrongDtv1).child(0)); + + } + @Test public void parseSetOperation() { String union = "select * from t1 union select * from t2 union all select * from t3"; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ReadLockTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ReadLockTest.java index 58212c2d3ba5d24..c24a846a1bb2453 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ReadLockTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ReadLockTest.java @@ -108,20 +108,26 @@ public void testScalarSubQuery() { } @Test - public void testInserInto() { + public void testInsertInto() { String sql = "INSERT INTO supplier(s_suppkey) SELECT lo_orderkey FROM lineorder"; StatementContext statementContext = MemoTestUtils.createStatementContext(connectContext, sql); - InsertIntoTableCommand insertIntoTableCommand = (InsertIntoTableCommand) parser.parseSingle(sql); - NereidsPlanner planner = new NereidsPlanner(statementContext); - planner.plan( - (LogicalPlan) insertIntoTableCommand.getExplainPlan(connectContext), - PhysicalProperties.ANY - ); - CascadesContext cascadesContext = planner.getCascadesContext(); - List f = cascadesContext.getTables(); - Assertions.assertEquals(2, f.size()); - Set tableNames = f.stream().map(TableIf::getName).collect(Collectors.toSet()); - Assertions.assertTrue(tableNames.contains("supplier")); - Assertions.assertTrue(tableNames.contains("lineorder")); + boolean originalDML = connectContext.getSessionVariable().enableNereidsDML; + connectContext.getSessionVariable().enableNereidsDML = true; + try { + InsertIntoTableCommand insertIntoTableCommand = (InsertIntoTableCommand) parser.parseSingle(sql); + NereidsPlanner planner = new NereidsPlanner(statementContext); + planner.plan( + (LogicalPlan) insertIntoTableCommand.getExplainPlan(connectContext), + PhysicalProperties.ANY + ); + CascadesContext cascadesContext = planner.getCascadesContext(); + List f = cascadesContext.getTables(); + Assertions.assertEquals(2, f.size()); + Set tableNames = f.stream().map(TableIf::getName).collect(Collectors.toSet()); + Assertions.assertTrue(tableNames.contains("supplier")); + Assertions.assertTrue(tableNames.contains("lineorder")); + } finally { + connectContext.getSessionVariable().enableNereidsDML = originalDML; + } } } diff --git a/gensrc/thrift/Opcodes.thrift b/gensrc/thrift/Opcodes.thrift index f6444ebe218fd3e..72a1d80e0362224 100644 --- a/gensrc/thrift/Opcodes.thrift +++ b/gensrc/thrift/Opcodes.thrift @@ -93,4 +93,6 @@ enum TExprOpcode { MATCH_ELEMENT_GT, MATCH_ELEMENT_LE, MATCH_ELEMENT_GE, + MATCH_PHRASE_PREFIX, + MATCH_REGEXP, } diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 62eb5a0827b0905..9ff6a589d69dd2c 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -249,6 +249,8 @@ struct TQueryOptions { 86: optional i32 analyze_timeout = 43200; 87: optional bool faster_float_convert = false; + + 88: optional i32 inverted_index_max_expansions = 50; } diff --git a/regression-test/data/correctness/test_from_millisecond_microsecond.out b/regression-test/data/correctness/test_from_millisecond_microsecond.out index cd9084bc134379f..414b2a88cfd4e0e 100644 --- a/regression-test/data/correctness/test_from_millisecond_microsecond.out +++ b/regression-test/data/correctness/test_from_millisecond_microsecond.out @@ -15,9 +15,9 @@ 3001-01-19 07:59:59 3001-01-19T07:59:59 \N 3001-01-19T08:00 2650-07-06 16:21:10 2650-07-06T16:21:10 -- !select4 -- -1919810114514 1919810114514 -89417891234789 488885820389 -1235817896941 1235817896941 +1919810114514 \N \N +89417891234789 \N \N +1235817896941 \N \N \N \N \N -- !select5 -- @@ -48,9 +48,9 @@ 2038-01-19 11:14:07 2038-01-19T11:14:07 2038-01-19 11:14:08 2038-01-19T11:14:08 2650-07-06 16:21:10 2650-07-06T16:21:10 -- !select10 -- -1919810114514 1919810114514 -89417891234789 488885820389 -1235817896941 1235817896941 +1919810114514 \N \N +89417891234789 \N \N +1235817896941 \N \N \N \N \N -- !select11 -- @@ -75,3 +75,12 @@ 2 1672502400 1672502400123 1672502400123000 3 1672502400 1672502400123 1672502400123456 +-- !sql -- +\N + +-- !sql -- +9999-12-31T23:59:59.999999 + +-- !sql -- +\N + diff --git a/regression-test/data/correctness_p0/test_order_by.out b/regression-test/data/correctness_p0/test_order_by.out new file mode 100644 index 000000000000000..23d447cd520e276 --- /dev/null +++ b/regression-test/data/correctness_p0/test_order_by.out @@ -0,0 +1,22 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +2023-12-19T00:01:12 2023-12-19 00:06:08.618 2023-12-18T23:56:12 +2023-12-19T00:01:12 2023-12-19 00:05:58.513 2023-12-18T23:56:12 +2023-12-18T23:56:12 2023-12-19 00:01:08.799 2023-12-18T23:56:12 +2023-12-18T23:56:12 2023-12-19 00:01:08.797 2023-12-18T23:56:12 +2023-12-18T23:56:12 2023-12-19 00:01:08.796 2023-12-18T23:56:12 + +-- !select -- +2023-12-18T23:56:12 2023-12-19 00:01:08.799 2023-12-18T23:56:12 +2023-12-18T23:56:12 2023-12-19 00:01:08.797 2023-12-18T23:56:12 +2023-12-18T23:56:12 2023-12-19 00:01:08.796 2023-12-18T23:56:12 +2023-12-19T00:01:12 2023-12-19 00:06:08.618 2023-12-18T23:56:12 +2023-12-19T00:01:12 2023-12-19 00:05:58.513 2023-12-18T23:56:12 + +-- !select -- +2023-12-19T00:01:12 2023-12-19 00:06:08.618 2023-12-18T23:56:12 +2023-12-19T00:01:12 2023-12-19 00:05:58.513 2023-12-18T23:56:12 +2023-12-18T23:56:12 2023-12-19 00:01:08.799 2023-12-18T23:56:12 +2023-12-18T23:56:12 2023-12-19 00:01:08.797 2023-12-18T23:56:12 +2023-12-18T23:56:12 2023-12-19 00:01:08.796 2023-12-18T23:56:12 + diff --git a/regression-test/data/external_table_p0/hive/test_hive_serde_prop.out b/regression-test/data/external_table_p0/hive/test_hive_serde_prop.out new file mode 100644 index 000000000000000..1cde2baec279243 --- /dev/null +++ b/regression-test/data/external_table_p0/hive/test_hive_serde_prop.out @@ -0,0 +1,4 @@ +test_hive_serde_prop.out -- This file is automatically generated. You should know what you did if you want to edit this +-- !1 -- +a 1.1 +b 2.2 diff --git a/regression-test/data/inverted_index_p0/test_index_match_phrase_prefix.out b/regression-test/data/inverted_index_p0/test_index_match_phrase_prefix.out new file mode 100644 index 000000000000000..140fd5ee9379922 --- /dev/null +++ b/regression-test/data/inverted_index_p0/test_index_match_phrase_prefix.out @@ -0,0 +1,31 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +863 + +-- !sql -- +863 + +-- !sql -- +235 + +-- !sql -- +235 + +-- !sql -- +166 + +-- !sql -- +166 + +-- !sql -- +56 + +-- !sql -- +56 + +-- !sql -- +7 + +-- !sql -- +7 + diff --git a/regression-test/data/inverted_index_p0/test_index_match_regexp.out b/regression-test/data/inverted_index_p0/test_index_match_regexp.out new file mode 100644 index 000000000000000..eab27de65ee45fc --- /dev/null +++ b/regression-test/data/inverted_index_p0/test_index_match_regexp.out @@ -0,0 +1,16 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1000 + +-- !sql -- +54 + +-- !sql -- +910 + +-- !sql -- +60 + +-- !sql -- +38 + diff --git a/regression-test/data/query_p0/join/jira8317.out b/regression-test/data/query_p0/join/jira8317.out new file mode 100644 index 000000000000000..795b1ba3421802b --- /dev/null +++ b/regression-test/data/query_p0/join/jira8317.out @@ -0,0 +1,815 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +17 +19 +21 +22 +24 +25 +27 +28 +29 +30 +31 +32 +33 +34 +36 +37 +39 +40 +41 +42 +45 +47 +48 +49 +50 +51 +52 +53 +54 +55 +57 +58 +59 +61 +62 +63 +64 +65 +66 +67 +68 +69 +70 +71 +72 +73 +74 +75 +76 +77 +78 +79 +80 +82 +83 +85 +86 +87 +89 +90 +91 +92 +93 +94 +95 +96 +98 +99 +100 +103 +104 +105 +106 +107 +108 +109 +110 +111 +112 +114 +115 +116 +117 +118 +119 +120 +121 +122 +123 +124 +125 +126 +127 +128 +130 +131 +132 +133 +134 +136 +137 +138 +139 +140 +141 +142 +144 +145 +148 +149 +150 +151 +152 +153 +154 +155 +156 +157 +158 +159 +162 +163 +164 +165 +166 +167 +168 +169 +170 +171 +173 +174 +175 +176 +177 +178 +179 +180 +181 +184 +185 +186 +187 +188 +191 +192 +193 +194 +196 +197 +198 +199 +200 +201 +202 +203 +204 +205 +206 +207 +209 +210 +211 +215 +216 +219 +221 +222 +223 +224 +227 +228 +229 +230 +231 +232 +234 +235 +236 +237 +238 +239 +240 +241 +242 +243 +244 +246 +247 +248 +249 +250 +251 +253 +254 +255 +257 +258 +259 +260 +261 +263 +264 +265 +266 +267 +268 +269 +270 +271 +272 +273 +274 +275 +276 +277 +278 +279 +280 +282 +283 +284 +285 +287 +288 +289 +290 +291 +293 +294 +295 +296 +297 +298 +299 +301 +302 +305 +306 +307 +308 +309 +310 +311 +312 +313 +314 +315 +317 +318 +319 +320 +321 +322 +323 +325 +327 +328 +329 +330 +331 +334 +335 +336 +337 +338 +339 +340 +341 +346 +347 +348 +350 +351 +352 +353 +354 +355 +356 +357 +358 +359 +360 +361 +362 +363 +364 +365 +366 +368 +369 +370 +371 +372 +373 +375 +376 +377 +378 +379 +380 +381 +382 +383 +385 +387 +388 +389 +390 +392 +394 +395 +397 +398 +399 +400 +401 +402 +403 +404 +405 +407 +408 +410 +412 +414 +415 +416 +418 +419 +420 +421 +422 +423 +424 +425 +426 +427 +428 +429 +430 +431 +433 +434 +435 +436 +437 +439 +440 +441 +442 +443 +444 +445 +446 +448 +449 +450 +452 +454 +455 +456 +458 +459 +460 +461 +463 +464 +465 +466 +467 +468 +469 +470 +471 +472 +473 +474 +475 +476 +477 +478 +479 +480 +481 +483 +484 +485 +487 +488 +489 +490 +491 +493 +494 +496 +497 +498 +499 +501 +502 +504 +505 +506 +507 +508 +509 +511 +512 +514 +516 +517 +518 +519 +521 +522 +523 +524 +525 +526 +527 +528 +530 +533 +534 +535 +536 +537 +538 +539 +540 +541 +542 +544 +546 +548 +549 +550 +551 +552 +553 +554 +557 +558 +559 +560 +561 +562 +563 +564 +565 +566 +567 +568 +571 +572 +574 +575 +576 +578 +579 +581 +582 +583 +585 +586 +587 +588 +589 +590 +591 +592 +593 +594 +595 +597 +598 +599 +600 +601 +602 +603 +604 +605 +606 +608 +609 +610 +611 +613 +614 +615 +616 +618 +619 +620 +622 +623 +624 +625 +627 +630 +631 +633 +634 +635 +636 +637 +638 +639 +640 +641 +643 +644 +645 +650 +652 +653 +654 +655 +657 +658 +659 +660 +661 +662 +663 +666 +668 +670 +671 +672 +673 +674 +676 +677 +678 +679 +681 +682 +683 +684 +685 +686 +688 +689 +690 +691 +692 +694 +695 +696 +697 +698 +699 +700 +702 +703 +704 +706 +707 +708 +709 +710 +711 +712 +713 +715 +716 +717 +718 +719 +720 +721 +722 +723 +724 +725 +726 +727 +728 +729 +730 +731 +733 +734 +735 +736 +737 +738 +740 +741 +742 +745 +746 +747 +749 +750 +751 +752 +753 +754 +755 +756 +757 +758 +759 +761 +762 +763 +764 +765 +766 +767 +768 +770 +771 +772 +773 +774 +775 +776 +777 +778 +779 +780 +781 +782 +783 +784 +785 +786 +787 +788 +789 +790 +791 +792 +794 +795 +797 +798 +799 +800 +803 +804 +805 +806 +809 +811 +812 +815 +817 +818 +820 +822 +823 +824 +826 +827 +828 +829 +831 +832 +834 +835 +837 +838 +839 +840 +842 +843 +844 +847 +848 +849 +850 +852 +853 +854 +855 +856 +857 +858 +859 +860 +863 +864 +865 +866 +867 +868 +871 +872 +873 +874 +875 +876 +878 +879 +880 +881 +882 +883 +884 +886 +888 +889 +890 +891 +893 +894 +895 +896 +897 +898 +899 +900 +901 +902 +903 +904 +905 +906 +907 +908 +909 +911 +913 +914 +916 +918 +919 +920 +921 +922 +923 +924 +925 +926 +927 +928 +929 +930 +932 +933 +934 +935 +936 +937 +938 +939 +940 +942 +943 +944 +946 +947 +948 +949 +950 +951 +952 +954 +955 +956 +957 +958 +959 +961 +962 +963 +966 +969 +970 +971 +972 +974 +975 +978 +980 +981 +982 +983 +984 +986 +987 +988 +989 +990 +991 +992 +993 +994 +996 +999 + diff --git a/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete_sign.out b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete_sign.out index f14434b2f9f88c3..8d3e69bbe26ac9d 100644 --- a/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete_sign.out +++ b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete_sign.out @@ -27,7 +27,6 @@ -- !2 -- -- !3 -- -1 2 \N -- !1 -- 1 1 1 1 @@ -65,7 +64,6 @@ -- !2 -- -- !3 -- -1 2 \N -- !1 -- 1 1 1 1 diff --git a/regression-test/data/unique_with_mow_p0/test_delete_sign_delete_bitmap.out b/regression-test/data/unique_with_mow_p0/test_delete_sign_delete_bitmap.out deleted file mode 100644 index 687aeab54a10095..000000000000000 --- a/regression-test/data/unique_with_mow_p0/test_delete_sign_delete_bitmap.out +++ /dev/null @@ -1,54 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !sql -- -1 1 1 1 1 -2 2 2 2 2 -3 3 3 3 3 -4 4 4 4 4 -5 5 5 5 5 - --- !after_delete -- -2 2 2 2 2 -4 4 4 4 4 - --- !1 -- -1 1 1 1 1 0 -1 1 1 1 1 1 -2 2 2 2 2 0 -3 3 3 3 3 0 -3 3 3 3 3 1 -4 4 4 4 4 0 -5 5 5 5 5 0 -5 5 5 5 5 1 - --- !2 -- -2 2 2 2 2 0 -4 4 4 4 4 0 - --- !sql -- -1 1 1 1 1 -2 2 2 2 2 -3 3 3 3 3 -4 4 4 4 4 -5 5 5 5 5 - --- !after_delete -- -2 2 2 2 2 -4 4 4 4 4 - --- !1 -- -1 1 1 1 1 0 -1 1 1 1 1 1 -2 2 2 2 2 0 -3 3 3 3 3 0 -3 3 3 3 3 1 -4 4 4 4 4 0 -5 5 5 5 5 0 -5 5 5 5 5 1 - --- !2 -- -1 1 1 1 1 1 -2 2 2 2 2 0 -3 3 3 3 3 1 -4 4 4 4 4 0 -5 5 5 5 5 1 - diff --git a/regression-test/suites/correctness/test_from_millisecond_microsecond.groovy b/regression-test/suites/correctness/test_from_millisecond_microsecond.groovy index fe7202772ac7709..d03325e0d7c28d9 100644 --- a/regression-test/suites/correctness/test_from_millisecond_microsecond.groovy +++ b/regression-test/suites/correctness/test_from_millisecond_microsecond.groovy @@ -199,4 +199,9 @@ suite("test_from_millisecond_microsecond") { from millimicro order by id; """ + + sql " set time_zone='Asia/Shanghai' " + qt_sql " select from_second(-1) " + qt_sql " select from_microsecond(253402271999999999) " + qt_sql " select from_microsecond(253402272000000000) " } \ No newline at end of file diff --git a/regression-test/suites/correctness_p0/test_order_by.groovy b/regression-test/suites/correctness_p0/test_order_by.groovy new file mode 100644 index 000000000000000..3bcb253f6d89b82 --- /dev/null +++ b/regression-test/suites/correctness_p0/test_order_by.groovy @@ -0,0 +1,63 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_order_by") { + sql """ + drop table if exists test_order_by; + """ + + sql """ + create table if not exists test_order_by( + create_time datetime null default current_timestamp, + run_time varchar(200) null comment '时间戳', + create_time2 datetime null + ) + duplicate key(create_time,run_time) + distributed by hash(create_time) buckets 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ + insert into test_order_by values ('2023-12-18 23:56:12','2023-12-19 00:01:08.799','2023-12-18 23:56:12'); + """ + sql """ + insert into test_order_by values ('2023-12-18 23:56:12','2023-12-19 00:01:08.797','2023-12-18 23:56:12'); + """ + sql """ + insert into test_order_by values ('2023-12-18 23:56:12','2023-12-19 00:01:08.796','2023-12-18 23:56:12'); + """ + sql """ + insert into test_order_by values ('2023-12-19 00:01:12','2023-12-19 00:06:08.618','2023-12-18 23:56:12'); + """ + sql """ + insert into test_order_by values ('2023-12-19 00:01:12','2023-12-19 00:05:58.513','2023-12-18 23:56:12'); + """ + + qt_select """ + select * from test_order_by order by create_time desc; + """ + + qt_select """ + select * from test_order_by order by create_time, run_time desc; + """ + + qt_select """ + select * from test_order_by order by create_time desc, run_time desc; + """ +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p0/hive/test_hive_serde_prop.groovy b/regression-test/suites/external_table_p0/hive/test_hive_serde_prop.groovy new file mode 100644 index 000000000000000..41d7056d20832b7 --- /dev/null +++ b/regression-test/suites/external_table_p0/hive/test_hive_serde_prop.groovy @@ -0,0 +1,36 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_hive_serde_prop", "external_docker,hive,external_docker_hive,p0,external") { + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String catalog_name = "test_hive_serde_prop" + String ex_db_name = "`stats_test`" + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String hms_port = context.config.otherConfigs.get("hms_port") + + sql """drop catalog if exists ${catalog_name} """ + + sql """CREATE CATALOG ${catalog_name} PROPERTIES ( + 'type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', + 'hadoop.username' = 'hive' + );""" + + qt_1 """select * from ${catalog_name}.${ex_db_name}.employee_gz order by name;""" + } +} diff --git a/regression-test/suites/inverted_index_p0/test_index_match_phrase_prefix.groovy b/regression-test/suites/inverted_index_p0/test_index_match_phrase_prefix.groovy new file mode 100644 index 000000000000000..b23bc1b5a8b82af --- /dev/null +++ b/regression-test/suites/inverted_index_p0/test_index_match_phrase_prefix.groovy @@ -0,0 +1,98 @@ +// 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_match_phrase_prefix", "p0"){ + def indexTbName1 = "test_index_match_phrase_prefix" + + sql "DROP TABLE IF EXISTS ${indexTbName1}" + + sql """ + CREATE TABLE ${indexTbName1} ( + `@timestamp` int(11) NULL COMMENT "", + `clientip` varchar(20) NULL COMMENT "", + `request` text NULL COMMENT "", + `status` int(11) NULL COMMENT "", + `size` int(11) NULL COMMENT "", + INDEX request_idx (`request`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "true") COMMENT '' + ) ENGINE=OLAP + DUPLICATE KEY(`@timestamp`) + COMMENT "OLAP" + DISTRIBUTED BY RANDOM BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + def load_httplogs_data = {table_name, label, read_flag, format_flag, file_name, ignore_failure=false, + expected_succ_rows = -1, load_to_single_tablet = 'true' -> + + // load the json data + streamLoad { + table "${table_name}" + + // set http request header params + set 'label', label + "_" + UUID.randomUUID().toString() + set 'read_json_by_line', read_flag + set 'format', format_flag + file file_name // import json file + time 10000 // limit inflight 10s + if (expected_succ_rows >= 0) { + set 'max_filter_ratio', '1' + } + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (ignore_failure && expected_succ_rows < 0) { return } + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + if (expected_succ_rows >= 0) { + assertEquals(json.NumberLoadedRows, expected_succ_rows) + } else { + assertEquals(json.NumberTotalRows, json.NumberLoadedRows + json.NumberUnselectedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + } + } + + try { + load_httplogs_data.call(indexTbName1, 'test_index_match_phrase_prefix', 'true', 'json', 'documents-1000.json') + + qt_sql """ select count() from test_index_match_phrase_prefix where request match_phrase_prefix 'ima'; """ + qt_sql """ select count() from test_index_match_phrase_prefix where request like '%ima%'; """ + + qt_sql """ select count() from test_index_match_phrase_prefix where request match_phrase_prefix 'images/h'; """ + qt_sql """ select count() from test_index_match_phrase_prefix where request like '%images/h%'; """ + + qt_sql """ select count() from test_index_match_phrase_prefix where request match_phrase_prefix 'images/hm'; """ + qt_sql """ select count() from test_index_match_phrase_prefix where request like '%images/hm%'; """ + + qt_sql """ select count() from test_index_match_phrase_prefix where request match_phrase_prefix '/french/images/n'; """ + qt_sql """ select count() from test_index_match_phrase_prefix where request like '%/french/images/n%'; """ + + qt_sql """ select count() from test_index_match_phrase_prefix where request match_phrase_prefix '/french/tickets/images/ti'; """ + qt_sql """ select count() from test_index_match_phrase_prefix where request like '%/french/tickets/images/ti%'; """ + } finally { + //try_sql("DROP TABLE IF EXISTS ${testTable}") + } +} \ No newline at end of file diff --git a/regression-test/suites/inverted_index_p0/test_index_match_regexp.groovy b/regression-test/suites/inverted_index_p0/test_index_match_regexp.groovy new file mode 100644 index 000000000000000..4c1ee1a5b0b4847 --- /dev/null +++ b/regression-test/suites/inverted_index_p0/test_index_match_regexp.groovy @@ -0,0 +1,89 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + + +suite("test_index_match_regexp", "p0"){ + def indexTbName1 = "test_index_match_regexp" + + sql "DROP TABLE IF EXISTS ${indexTbName1}" + + sql """ + CREATE TABLE ${indexTbName1} ( + `@timestamp` int(11) NULL COMMENT "", + `clientip` varchar(20) NULL COMMENT "", + `request` text NULL COMMENT "", + `status` int(11) NULL COMMENT "", + `size` int(11) NULL COMMENT "", + INDEX request_idx (`request`) USING INVERTED PROPERTIES("parser" = "english") COMMENT '' + ) ENGINE=OLAP + DUPLICATE KEY(`@timestamp`) + COMMENT "OLAP" + DISTRIBUTED BY RANDOM BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + def load_httplogs_data = {table_name, label, read_flag, format_flag, file_name, ignore_failure=false, + expected_succ_rows = -1, load_to_single_tablet = 'true' -> + + // load the json data + streamLoad { + table "${table_name}" + + // set http request header params + set 'label', label + "_" + UUID.randomUUID().toString() + set 'read_json_by_line', read_flag + set 'format', format_flag + file file_name // import json file + time 10000 // limit inflight 10s + if (expected_succ_rows >= 0) { + set 'max_filter_ratio', '1' + } + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (ignore_failure && expected_succ_rows < 0) { return } + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + if (expected_succ_rows >= 0) { + assertEquals(json.NumberLoadedRows, expected_succ_rows) + } else { + assertEquals(json.NumberTotalRows, json.NumberLoadedRows + json.NumberUnselectedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + } + } + + try { + load_httplogs_data.call(indexTbName1, 'test_index_match_regexp', 'true', 'json', 'documents-1000.json') + + qt_sql """ select count() from test_index_match_regexp where request match_regexp '^h'; """ + qt_sql """ select count() from test_index_match_regexp where request match_regexp '^team'; """ + qt_sql """ select count() from test_index_match_regexp where request match_regexp 's\$'; """ + qt_sql """ select count() from test_index_match_regexp where request match_regexp 'er\$'; """ + qt_sql """ select count() from test_index_match_regexp where request match_regexp '.*tickets.*'; """ + } finally { + //try_sql("DROP TABLE IF EXISTS ${testTable}") + } +} \ No newline at end of file diff --git a/regression-test/suites/nereids_function_p0/tvf/tvf.groovy b/regression-test/suites/nereids_function_p0/tvf/tvf.groovy index 488decd32c23668..9fd4a2538ce0461 100644 --- a/regression-test/suites/nereids_function_p0/tvf/tvf.groovy +++ b/regression-test/suites/nereids_function_p0/tvf/tvf.groovy @@ -28,4 +28,8 @@ suite("nereids_tvf") { sql """ select * from numbers("number"="10") a right join numbers("number"="10") b on true; """ + + sql """ + select * from numbers("number" = "1") union all select * from numbers("number" = "1"); + """ } \ No newline at end of file diff --git a/regression-test/suites/nereids_p0/aggregate/agg_distinct_case_when.groovy b/regression-test/suites/nereids_p0/aggregate/agg_distinct_case_when.groovy new file mode 100644 index 000000000000000..74caa459c154eee --- /dev/null +++ b/regression-test/suites/nereids_p0/aggregate/agg_distinct_case_when.groovy @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +suite("agg_distinct_case_when") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + sql "DROP TABLE IF EXISTS agg_test_table_t;" + sql """ + CREATE TABLE `agg_test_table_t` ( + `k1` varchar(65533) NULL, + `k2` text NULL, + `k3` text null, + `k4` text null + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "is_being_synced" = "false", + "storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false" + ); + """ + + sql """insert into agg_test_table_t(`k1`,`k2`,`k3`) values('20231026221524','PA','adigu1bububud');""" + sql """ + select + count(distinct case when t.k2='PA' and loan_date=to_date(substr(t.k1,1,8)) then t.k2 end ) + from ( + select substr(k1,1,8) loan_date,k3,k2,k1 from agg_test_table_t) t + group by + substr(t.k1,1,8);""" + + sql "DROP TABLE IF EXISTS agg_test_table_t;" +} \ No newline at end of file diff --git a/regression-test/suites/nereids_rules_p0/partition_prune/test_multi_range_partition.groovy b/regression-test/suites/nereids_rules_p0/partition_prune/test_multi_range_partition.groovy index 49570682ea2e92a..1e65e73a376af79 100644 --- a/regression-test/suites/nereids_rules_p0/partition_prune/test_multi_range_partition.groovy +++ b/regression-test/suites/nereids_rules_p0/partition_prune/test_multi_range_partition.groovy @@ -169,12 +169,12 @@ suite("test_multi_range_partition") { explain { sql "select * from pt where k1=7 and k2 in (null);" - contains "VEMPTYSET" + contains "partitions=0/3" } explain { sql "select * from pt where k1=7 and k2 not in (null);" - contains "VEMPTYSET" + contains "partitions=0/3" } explain { @@ -189,13 +189,13 @@ suite("test_multi_range_partition") { explain { sql "select * from pt where k2 in (null);" - contains "VEMPTYSET" + contains "partitions=0/3" } // p1/p2/p3 NOT pruned explain { sql "select * from pt where k2 not in (null)" - contains "VEMPTYSET" + contains "partitions=0/3" } explain { diff --git a/regression-test/suites/query_p0/join/jira8317.groovy b/regression-test/suites/query_p0/join/jira8317.groovy new file mode 100644 index 000000000000000..a2b2d9ced175009 --- /dev/null +++ b/regression-test/suites/query_p0/join/jira8317.groovy @@ -0,0 +1,46 @@ +// 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("jira8317") { + sql """ + DROP TABLE IF EXISTS `table_1000_undef_undef2`; + """ + sql """ + CREATE TABLE `table_1000_undef_undef2` ( + `pk` int(11) NULL, + `col_bigint_undef_signed` bigint(20) NULL, + `col_bigint_undef_signed2` bigint(20) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`pk`, `col_bigint_undef_signed`, `col_bigint_undef_signed2`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`pk`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "is_being_synced" = "false", + "storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false" + ); + """ + sql """ + INSERT INTO `table_1000_undef_undef2` VALUES (1,-19003,-120),(7,-7365641,-8081123887699099479),(39,NULL,8157469758106312693),(45,-27,-8123020834936742524),(58,4999,3743594151072369933),(69,655619458377523526,7519664894086114892),(87,8262093739053643222,1766397486317050982),(91,4704,123),(93,-7587996,18142),(102,-417417511588815387,NULL),(105,6230407775549921021,8109),(111,-9885,1417473),(123,-2555577,-5337152),(125,-59,-1),(126,19,-16894),(134,24543,76),(157,5395653578239812058,-1764615),(166,3757370,115714765999722154),(175,-2899615982387187770,6986202),(196,NULL,1447),(199,5068178230655872542,-10674),(205,22,6431560),(218,-15874,NULL),(246,-8735,-5775020777947744263),(265,NULL,13482),(298,9216134360394789591,5679),(308,6845580,-3004454),(322,-6816415254937360604,29758),(341,-4767381996628937761,8171354672951628997),(371,-627487131429034642,251),(374,-16,NULL),(378,-4114549148709048844,-2570924511164827202),(391,29515,NULL),(398,3229973449457755169,9186583667918079554),(403,32,946998133221629324),(406,NULL,NULL),(412,4527414356688238927,-12),(435,-124,5876613814197233863),(438,-30856,NULL),(472,14434,-17744),(477,-5877513137857099418,-3310242399155057576),(492,-1648965159783569768,NULL),(497,5891759987264692585,28119),(504,-3009353251680330192,-3385585),(509,-6742623742434484156,-6746617),(510,2658442546697658903,NULL),(514,-8234,63),(533,-13,2573219),(537,-7946380441594310393,-11702),(543,-91,NULL),(558,6972989139639064837,8925213840422996445),(572,NULL,-4262993),(579,-5897753,-26000),(582,NULL,-1106448149334542023),(585,NULL,-9110552557766495430),(586,19018,-8696199199951370044),(603,-2905465,1065078903442796472),(611,-29913,4000),(613,-25,1334372101623564589),(625,7922,6923419),(631,-5273350,-59),(646,6848743253557587590,NULL),(660,147308,21376),(690,85,2768344),(701,-17302,NULL),(702,NULL,-79),(706,4214846,13065),(715,13968,4374384568269861716),(737,-5211216,-1613482),(743,NULL,NULL),(749,-11,-25175),(764,-5187474377935208440,5528350657610937595),(774,11254,-4067970),(795,-3941881,16531),(798,-7415236,10847),(823,-8442816819368076641,82),(858,99,-6114372),(878,NULL,-2995508),(883,-12837,-5229344807687917760),(889,-31,9331),(903,579841,66183395754462121),(930,-5462323,5620223),(935,-2161998,-5457916537252374932),(941,4455127172594657873,NULL),(959,11161,-27604),(963,-6823471941991501348,247878),(983,28739,813222649038017676),(995,-6880546792162904257,NULL),(12,NULL,-6244283584821915660),(23,-55,NULL),(27,-837602972884855962,32),(32,28665,-2886572565035405541),(50,2735653,-61),(90,5041,-4293116302471282150),(103,5509801503377205293,123),(110,3835650522971313342,6534645771525321721),(121,101,5449358),(136,NULL,-2846700542533112846),(141,-5412553832315383215,4),(142,-1416930957245477234,17),(149,12615,32427),(150,10340,-4956030870704664676),(156,-3280843,7750834006460826023),(159,7577082264394679086,-90),(164,3980,-6133128),(167,-14,-947978666917274289),(168,6871907822690179662,-3859458),(173,-92,14),(176,8137848140098659898,2810201290537783567),(179,-405144302754595661,-7896018842801290229),(201,-786268,23113),(209,-5281,5551649),(229,4144,25571),(257,-7010271,-4838803),(277,959701,3554446),(314,58,44),(343,7743,NULL),(352,2084635494667751228,-10691),(358,842043435432900253,-7193426924333597012),(362,3364256032068110028,-7474997),(375,-339,8057614),(379,NULL,1804970089654852765),(382,3478857192154069111,2127),(389,-624910,88),(390,2940413006456803386,61),(394,0,-6186),(404,4704782582378089789,48568957078751827),(414,-4365630394963277377,6075188749901339503),(431,NULL,-19429),(433,NULL,-2976069229306407708),(434,NULL,1758157016624770386),(439,-83,1319438),(445,-445678981456339247,6764170805906335296),(455,-8805832866437583211,-107),(456,-861505,3001103443915789354),(467,NULL,82),(470,16,-11217),(484,8847033568544232196,-7245108982741225054),(494,-1414,97),(496,29231,-4088969),(505,-429024549165417607,-3329),(506,NULL,2728150847869367661),(524,-4079451130938998056,-1457),(527,-250460,-1713739740231516514),(529,-1168162,NULL),(547,-6833274923109161689,NULL),(553,-3203931376566131833,-6193646836234948352),(561,1325130049773122078,10867),(562,15581,3019322609487292475),(567,NULL,-39),(573,69,NULL),(574,3949615548534239825,-100),(584,NULL,NULL),(587,2261044,7674764774128981957),(598,8204503,880502),(621,-7615727908514562682,NULL),(641,-42,-8123418684499871829),(647,141760341012454652,NULL),(653,-2149175478618130107,3096451179703625471),(673,NULL,6585330350640852279),(686,-3706480083217016813,1943182905989046798),(709,372863230934331886,7699766326928220154),(713,5780453,1497),(714,3828854,NULL),(721,7995142928132780707,4877213368395338842),(728,NULL,624334),(736,4199323193775611798,7421028117980622640),(739,4078105932476725481,NULL),(741,NULL,-8030356),(746,-1691668988061150727,4953),(763,14228,72),(769,5875503408282616544,NULL),(770,916090495168036187,9039593784456337022),(794,-4348345074306189013,16322),(801,-7945002,NULL),(811,96,-5979751868393457717),(816,531860,NULL),(819,7804946845868268544,NULL),(828,100,2595300248585806407),(831,-50,-332),(832,-6678277,-8304006),(835,NULL,-5708517391660480378),(837,4896762,-6425523),(838,NULL,92),(847,NULL,-4790981),(849,4484657,23),(855,76,-34),(862,-6066951284946440145,NULL),(881,-6384500018181769145,-4995),(896,6345,17888),(905,NULL,-2152246231596184366),(931,3453489499313622510,NULL),(937,-3685552750649905429,5663331292536785292),(982,7658554534507473930,8122974),(996,NULL,303808834288504749),(10,7584726391219605514,2468167924208642541),(17,6509530535377649821,-5328319169040493739),(18,49,NULL),(20,-6527598895639492262,NULL),(29,-5866415,21646),(42,-44519,-8638763694774975489),(47,NULL,6086031836504758079),(55,13461,8998339185115890014),(65,NULL,76),(80,-83,-4493687052648700102),(97,16824,NULL),(98,2978027,396260940423748282),(130,-4464201716261474000,-102),(132,NULL,1966425726719050511),(147,-8478602112084000819,NULL),(153,3041423,13299),(154,NULL,-91),(181,998037,67),(188,NULL,-3946295),(198,-69,9016921369494035032),(207,1889,-3955184002801981483),(221,7864152,122),(224,-17071,2316671),(236,-1025113713003683545,-1226340),(241,-8009722132928344996,26),(242,NULL,-1471491612676690021),(254,29,912740859434241591),(282,8643837030944926247,707775277638919669),(287,NULL,-6821544590025584605),(299,-8985253589781319746,-3874834507563132194),(301,7668472,7104930320909341814),(302,748935885669691964,5067902107495476756),(313,-9140802912354990700,5441555),(319,-7124590116659760463,-5125176200435102165),(329,-10340,78),(332,-3339673176542551819,NULL),(335,2419411,-1901056538424971252),(337,-8291,-12769),(340,424408,-8162190112821529673),(347,-2239338666097551522,-7462339164763394452),(355,5399779,-5309554096122267472),(361,NULL,-4515410),(367,-2170359324698210109,NULL),(387,700620298677679667,84),(396,2551,NULL),(413,5794128052296395518,NULL),(419,-47,6834335),(459,-22787,7789993),(488,-2217634051081839681,3281819763197679891),(499,-29542,-2029883),(501,7973466,44),(508,NULL,-1749107),(515,6925179417946177837,NULL),(536,-70,90),(549,8339263,7239652582614798447),(550,NULL,7039126),(571,-6150562,5516156392943835657),(589,-123,-1616145454488370839),(590,-3463397376201407469,8930),(592,-4978661873397300045,122),(601,3333746916756457066,-3483445),(607,93,NULL),(610,NULL,15),(629,NULL,NULL),(630,NULL,30962),(636,4023466830320315597,111),(648,2332209,NULL),(656,NULL,NULL),(676,8197104,-18),(693,1435,NULL),(697,-2584090,12695),(698,-28555,-5392951363422528831),(704,-7846786,-124),(716,103,-6481510795639224384),(724,-3960364795137656128,16074),(727,72,2939342),(733,-21593,4780608459126151856),(751,-126,-56),(759,-30,22415),(765,NULL,1071158728575403719),(766,-18452,-61),(776,6587217,392373),(779,NULL,4641444979137222268),(781,48,-3787054345370776396),(789,7034962070260565702,2706922),(808,-15917,NULL),(814,-7298406028401184882,NULL),(826,-8346979819853596875,1415739),(844,NULL,980128),(856,NULL,8227658889254292598),(859,NULL,1171863007622977054),(864,NULL,-65),(867,NULL,7815111009804780969),(869,4695916024573961629,NULL),(870,7144,NULL),(879,-27552,123),(888,-4028748,4204207056170546662),(891,-19,3286942628751182271),(894,-5402682406854330395,-784055),(908,NULL,8950571864606229113),(911,-3745966675626169780,13395),(913,-1170923455330784971,85),(914,4136454162034896486,-4070673719932479581),(926,24632,24318),(943,-8375091309258618653,-2936126218440019409),(948,-123,1610601693923998472),(951,1475910,-7297111122578779437),(961,NULL,-81),(967,-843647332522456781,NULL),(974,6851253506720470225,-8288146),(988,NULL,-7390268336797388107),(993,29270,-2797538284629322623),(5,-9,-8247024),(35,1909929,NULL),(37,-2627996488191996852,-124),(38,2349628,NULL),(41,2978379213057630453,24905),(44,3451914,NULL),(59,4642,16640),(66,6926033115934836462,-2458619),(85,11034,19875),(95,-21,-2388),(107,-69,11796),(109,8491274892812862101,2299429),(161,-3646169,NULL),(174,-2279485,213914168875414933),(182,-5436555,NULL),(195,NULL,NULL),(197,-6466256,31770),(239,1308755,5650765516391102953),(244,398668,8785367598720568954),(267,-766976,-1254807649554272221),(270,110,5889557294528025294),(281,5601878,NULL),(290,-15750,-2600442523073963023),(295,-5306785,-7840350067006497141),(296,NULL,-48),(307,-10668,-4373),(309,2235531,-1909211319961976604),(316,-8196365,NULL),(357,29620,26),(372,-15055,-3341467),(376,1621,37),(401,-112,-117),(407,1311935412221730318,385248169862200655),(416,6685056,17255),(443,38,-7131330),(446,NULL,-2383532543279797781),(449,182177155427310301,5282939133421584539),(450,-27901,8947682171473971908),(473,-6553066099932132358,121),(476,-31384,-1649731641486982129),(482,5085310725940875242,NULL),(491,3390182466930480807,-125),(522,-3347055818402203912,17951),(530,-6692831,-30823179153490681),(532,2767909414662046486,NULL),(554,4408183511012281375,-82),(559,1197815,3363082),(577,NULL,NULL),(595,-7755127,25508),(602,-7577439,-1989239408092377511),(627,-8408964045275489321,21826),(639,2286283,-117),(659,4014625,5934219403338702239),(665,941521124005766042,NULL),(666,-2213573778515267265,4609962876929807378),(668,8039422972721333548,-31799),(671,-6,-5255403509696490557),(674,4603035140417755525,618893),(679,814974268575462359,8553895561364066384),(707,4957487757813438825,-97),(719,NULL,108),(722,NULL,-31038),(775,1457566,2908605),(784,-678762,-24168),(790,91,5394162204294271583),(793,-2457,NULL),(802,13,NULL),(821,NULL,NULL),(874,NULL,-22485),(876,NULL,-11498),(882,1256028894004327657,-7286),(916,NULL,1058),(919,NULL,831),(920,-64,5952583986530345076),(928,-120,8347056),(940,NULL,-107),(946,NULL,-83),(957,7526555572834980057,102),(968,-838669,NULL),(973,52,NULL),(979,-25004,NULL),(981,546252829378776336,-35),(991,-2892979981845178245,-3940552871761534102),(994,-11489,-1991206822210877686),(999,22348,2405614249297288825),(2,NULL,28171),(25,-3656190094982092092,30409),(40,-3748199547839907485,5503890541345078597),(46,84,NULL),(53,-128,-61),(54,-8269168781827786430,-5102),(82,4184,-50),(96,-6434483,-31),(119,110,47),(133,2155665551155817753,22253),(140,25166,6343052151579052647),(145,-2286557251729634997,6297902659931381317),(155,6393561532387656406,4726408),(158,NULL,6661470),(169,NULL,3727251999414877243),(178,-7299845225536788492,21190),(200,-6929922433097756798,8358884),(203,22526,14517),(231,7588,-1262011),(237,NULL,2832751343332662641),(245,-527,NULL),(256,5618873835904861978,NULL),(261,1773,-7646808),(279,-4560938074136143129,-1190688719641091658),(288,-94,-2657490),(297,22708,-1241),(306,-17789,-971308743185799464),(315,837918066574711347,63),(318,6386604,-27872),(327,NULL,17748212270502268),(328,16125,124),(339,78,4752),(353,61,5645014),(359,3972305887111991089,-6361675409207159253),(386,24216,NULL),(410,3374701,41),(441,4279451380111775243,12188),(442,57,-97),(444,-4327137168635641650,-8970007877007216112),(448,-22,-26262),(453,112,NULL),(454,NULL,-7126072071915141261),(457,12132,NULL),(458,933316,-2164104),(463,5158519217548931603,30258),(478,-421102480438674427,92),(486,5174048,NULL),(489,NULL,-6825750909146378675),(490,6134673,-28949),(495,2879165,NULL),(507,3961344,-8114401),(528,-14795,-5887313510192644320),(531,65,NULL),(548,-8736,24541),(555,93,NULL),(565,-1151975420820005234,26809),(581,57,-30335),(591,-2126352,-95),(608,-373415,-45),(628,NULL,NULL),(635,-7861273,6414378702125981026),(643,2578260142717909518,-2112977388786723057),(649,-17,NULL),(652,-93,4696023757590112756),(655,NULL,6090130961852809314),(663,-101,4266023271879213191),(667,2246060166515030806,NULL),(677,-72,9009112869992696950),(687,-1958481392504462618,NULL),(692,-9164780234680340708,1818044309241700768),(720,NULL,3239412815429093077),(726,-5671180041146395313,21747),(732,NULL,NULL),(735,5558417,6170),(747,22312,1256),(757,-6878661753962454976,4734744),(762,NULL,-7435992397167971603),(786,5260231462483745125,-6589740),(788,11531,-19860),(791,44,8323394),(792,38,1198556203747026211),(809,NULL,-5442565824474992791),(818,NULL,24259),(820,1105463832750467847,-28874),(830,-20537,NULL),(840,-2970386359172329133,-4768150),(843,18368,-8069632),(871,27020,43),(877,-6191297629078889893,NULL),(892,-3609,NULL),(904,NULL,16650),(910,-14615,NULL),(921,-487712520551325843,128225658329108826),(922,5283532537730025607,-8646942128987133587),(929,NULL,7256235176355115076),(954,-807436,-4862882),(966,-48,-6642718189949321798),(970,-3304,-102),(990,-7549790081492622316,-2459063),(0,4111815540043927035,8085129),(9,3693292453600931913,-7426979266376293294),(52,NULL,-11145),(62,-88,-2767242),(68,-5639,-5524433459173788420),(71,NULL,687603),(75,464511926649421669,-62),(77,-12654,2836401432842654844),(83,-7351405255002120977,-90),(86,NULL,21),(89,-6132322,-7957101103606180165),(100,-24324,-1477125498852936266),(104,NULL,1426487),(112,3771373101009182262,1540198),(115,33,31406),(122,-2192908,-4795289696312410622),(127,NULL,-7984346073659404814),(129,52,NULL),(135,6871757,NULL),(171,14361,702),(191,-3521649179954079196,6526012170845080698),(215,-303071030315791859,-6723),(216,103,-6684663),(219,-7094035970782337573,28862),(222,NULL,-8000855088777995780),(227,2897786541049681539,-12679),(233,30489,NULL),(248,13,16426),(263,-5858187280427613017,-17423),(264,2730425,-76),(272,5398964081755208111,6044705681410172266),(275,-8942018056968670103,17363),(278,-1518409288582470005,-30925),(310,-252074049178343128,9349),(320,NULL,-1592322),(323,-181989186606245880,1059339339137376353),(326,-1577358560474870181,NULL),(344,-7171659679753465749,NULL),(349,NULL,NULL),(369,-7972874,-6211326),(370,66,79),(381,4858,-7590430),(384,-2680331345002790991,NULL),(402,-4883753472510552397,3481679298690027286),(408,-6885173383833547854,-8208893090021618106),(426,39,5924446947522301132),(428,-5411961,872810384978979206),(440,7629036500770650480,20084),(452,-906844,-6209893189280643105),(479,-6074359000588882854,-94),(481,NULL,3405553967983271947),(487,-5709599122578883108,11245),(502,-16522,-5279651),(511,8128593733912771104,-19063),(535,-1364995919787189202,-4785760193186387976),(541,NULL,-5189444),(542,49,51),(544,31,-8633665699397962694),(556,23552,NULL),(564,-90,-20145),(568,-20613,6065051852821863500),(583,NULL,87),(612,124,NULL),(615,-2941252221325060670,64),(616,NULL,-353417),(619,-1423854028539795071,5134303457879623225),(624,-34,-8591711174510447295),(633,-30557,-3749108),(634,-6573892,-7097524521375559702),(644,1966,12),(662,-7507281163587916613,-3966416),(685,-4615,71),(688,-5949214781242049125,14035),(691,-5209382,99),(700,-1476865436961273798,8277735),(703,8533268123187836105,8194580057868554306),(710,-3849102395457533547,31447),(731,-79,-4192361969424990412),(742,6984059,-21),(745,2813025,-7766186139988493349),(748,285017,NULL),(753,15794,874771667139909604),(756,6155168821955677687,-59),(772,-17618,63),(796,-1610039,NULL),(804,-14753,7915819),(841,-821512547235601029,NULL),(852,-60,5309),(861,-67,NULL),(884,-456837439169958159,-5932727383604993033),(887,NULL,NULL),(893,-2896160680485931234,-7263986),(899,10965,-47),(901,18508,2103677822818945230),(906,-3126000,69),(933,-5158152299362784403,2219668354459551316),(934,29833,-4871405361094207703),(952,-6247675,-515855442808708507),(964,NULL,NULL),(971,5,-30032),(976,-4800430,NULL),(4,-3931887103089601675,-5812000168930348278),(19,-384485024054923633,1753078931045546584),(26,23794,NULL),(57,NULL,-5016222164692095479),(63,-7339295091776859963,-7938110999951928377),(64,8591602329888756160,6),(67,NULL,-2588029),(81,31886,NULL),(88,2888063,NULL),(99,3482512,-51),(101,-9100517790851301511,NULL),(113,4904865,NULL),(116,-640270380985407803,-6276420),(131,-753091619392669232,-5483459729580460905),(137,-2863391965642466819,-46),(146,NULL,NULL),(148,-15385,1134004502222994811),(151,-5173026,-8005709905089683810),(152,NULL,-4856),(165,-11895,108),(172,65,NULL),(184,124,-26917),(190,8052751817808907826,NULL),(211,4661996449074067775,2842013773797377298),(223,-1090761,7102687439113490311),(226,-3289784,NULL),(240,83,5843),(243,66506976084747580,-9347),(255,-549965379048371586,-1498708452930729977),(259,6796906250774503034,8307988),(271,8635889924323584913,6892249318430708438),(291,5173719,358228050546343499),(317,3932505505624009752,-3863151155613135444),(324,32,NULL),(331,3584,4059412094801260828),(334,17584,-4578113528369028198),(342,14435,NULL),(345,-4645750154548480832,NULL),(351,-3566749,89),(365,-26,-6411),(368,-2807688,3767610),(373,44,-7190591715827938648),(388,NULL,7275),(420,4287564,3404399046005142924),(430,6850874,4),(437,-3576511654891528168,-5839888925440092245),(460,-6942021,3117725495314230835),(466,-19788,-1903446577535696513),(485,3,116),(498,7945541,-12434),(513,-2279,NULL),(520,-78,NULL),(525,1330443,-15346),(526,874001368005184751,7194282),(538,NULL,-8737019690509553036),(551,NULL,3374897402739486263),(552,-4077690737150866138,49),(560,-8377517817149662783,704061339418835366),(563,NULL,83),(569,NULL,NULL),(575,-6072838,1323651332771913044),(576,24,1247596521788677414),(593,20262,-1523934208783695934),(596,19827,NULL),(606,-1503,-7093768777424189068),(620,NULL,2994813974863066652),(645,-6108654463761732268,41),(650,859102966614210074,2419418321794102856),(657,5342059439170500130,-109),(669,4433580851321889731,NULL),(672,61,-3045363),(678,-76,14866),(689,-8440807050962240371,-119),(695,7469359,117),(718,1803589052733522633,-7239697),(723,-22800,-2202171),(738,-59,-5402492267089758391),(740,-3496556,-1975552),(758,6633229662328973423,-8915742411819657406),(771,-28092,-501689519072062584),(773,4,-64),(778,-49,-8111292327362428982),(803,1523140,-92),(806,8420,22570),(810,NULL,NULL),(812,5449679474300647136,1327674),(815,4213516,2569536),(817,-2156427761069226553,1057668384468729898),(839,NULL,-7029137857768150330),(868,NULL,9081510617318852344),(872,5494946067283635941,7787764831438909235),(890,-17144,114),(915,NULL,NULL),(939,3326096054977737078,-4903728),(942,7835703329494457090,7990597485356659900),(944,NULL,853743),(950,NULL,24),(956,1371429113901830499,2736763932039229471),(960,-154865,NULL),(969,7134752816183335915,-977189201074403522),(977,95113,NULL),(984,5823111,4946350),(997,7916220,NULL),(8,5101318286519021398,6106775),(14,-5493389,-217814),(16,-3222893433467889260,NULL),(21,5359986120838631677,-16019),(22,-5691,-10607),(31,21,7582518432817006423),(33,-2615373,2675655157761445762),(34,1762479079353341529,-12340),(48,-3513302,-5439284087794055294),(73,-6023279181224294374,22975),(76,-3743398159856895673,-117),(84,-7734606586197398923,NULL),(94,7288905,6817607),(106,4115950009674914438,-1667751889288961743),(108,-2129251568876445290,-61),(120,-8085625,-2891654),(143,-10931,NULL),(160,-44,NULL),(177,17140,-32),(180,28393,-782623965324753119),(183,7298413,NULL),(193,3048057016133781712,7509352837011696690),(206,-18290,-4758166139239598072),(208,31036,NULL),(214,-65,NULL),(220,4540899,NULL),(225,-19805,NULL),(228,NULL,4110125877452766131),(232,-3364648,1631677345239857533),(249,-5087709,-6538490537746530818),(250,-18147,-48),(262,NULL,NULL),(268,-116,23421),(274,41,123),(285,-7129603271824146514,6431375),(294,NULL,-6342163705907165881),(300,-2145658827575088107,NULL),(303,NULL,NULL),(312,7791766,58),(346,NULL,-4117960514664592199),(348,-43,21695),(363,-5360331,-4430471200094294884),(366,-469901,3826248794611497693),(385,65,-554056),(392,6040301106917674637,2342310499943339164),(397,43,-7941200788302328539),(429,-5636027303656445893,6991093),(432,-5956276,NULL),(447,-1849906,NULL),(451,6355855541589389487,NULL),(471,14,-3925548188353229853),(475,-7443770,-7983411),(480,NULL,-121),(483,3301493867582981702,517174994880179659),(516,-7551633,7739639090468795435),(519,3925980781863456300,5896270),(523,4092505972360504411,-41),(540,3541640,-4254187),(546,-8629310699690472819,7690272214148349542),(557,-2404912802398426683,22193),(588,NULL,-3055089333756836395),(594,3266539074159303742,2452498),(617,-1301326614920685131,NULL),(618,2949974086785997693,48),(623,-2,8831392486900658904),(626,-2099950707561625221,NULL),(664,-5870753,NULL),(670,6107,-8188395),(682,NULL,8238593),(699,4162422,-14684),(725,-27277,-2508),(730,7430,8517850305265278456),(744,-2263138478370096128,NULL),(750,NULL,7839119),(755,-8498248735492430388,5710673502616107521),(761,-8060253782878893995,-2835730775288417257),(767,NULL,1733217535285280781),(768,115,-3),(800,2726916,735551229436305501),(805,NULL,2145865776339616215),(827,-7262,-90),(833,3962613,NULL),(834,-5539803088705391836,-3312649576750410759),(845,21,NULL),(848,-6113085,7761907218999018126),(851,2271644,NULL),(853,3862221010123412166,-50),(854,NULL,1530305003311176873),(857,13,305667466429405666),(860,-3688466260434187982,6018303),(865,626311431408092433,-91),(880,7835670,-11634),(885,3361,NULL),(886,3196,-341001939843852296),(898,4969696225135338155,-5511256225152745746),(917,NULL,NULL),(927,75,125),(932,-617785904669739739,95),(947,7729,-3124572004468002537),(949,95,9117118440965085428),(953,-6175124363808799396,NULL),(965,NULL,NULL),(972,-2105922759977938281,21895),(978,-4247346,30155),(980,4135302856517213901,9109572558070260701),(987,NULL,-11841),(989,61,6819461),(998,-6220737771190384788,NULL),(11,6593886403872375119,-16511),(13,3109167239027650377,5390321295285305810),(28,-24840,100),(36,NULL,-6408980843302191920),(43,-126,NULL),(51,4265083,8852142350798772903),(60,-3058683020495786354,NULL),(70,-55,2397),(74,-75,156030454139134730),(79,2004999601618816405,-583932755116454381),(114,-5668457783037393310,1636864),(128,-6151461,30868),(138,NULL,-7335917688405253750),(163,95,-5936953849760482136),(170,NULL,-21762),(187,60,-85),(189,-121,NULL),(194,2649086955813120290,-452674900134009639),(213,5431543,NULL),(217,8328645474847533949,NULL),(235,-94,1701112),(238,7994897363707629441,2455193),(252,-99,NULL),(266,-636338947658804332,-6574370972353623087),(273,3399841331481191930,1187728914747827866),(276,2448579,5182583850274274462),(280,NULL,-88),(283,6242010,6276414414749547409),(286,64688392076899328,NULL),(293,NULL,-6392053),(305,6466493300255959331,29),(311,NULL,4614926),(321,-19,6394402832353128048),(333,-4940027135186387555,NULL),(336,-31723,-3107114),(354,1417256969460282874,-27583),(356,73,17116),(360,-7100427797971946112,6420910521568845647),(377,1212643,1001929),(380,6221504,-64),(383,12866,-12564),(395,-8072245041343644390,-4005003078678267710),(400,29826,191792257504749093),(405,1812308824776809386,111),(409,8620771597253240666,NULL),(415,NULL,-5286899337188580178),(417,NULL,NULL),(418,-22,-2735429332768127055),(423,-1232699246386428688,-17346),(424,-26616,84),(427,1906989094204981041,538708664883812526),(465,-7510487689170731662,-93),(468,-3361074,-3022257106871145202),(500,-3774683245205225016,NULL),(503,5814915496907384294,NULL),(534,NULL,-110),(545,29,NULL),(566,-5126425334831146422,48),(570,-8989,NULL),(599,84,119),(600,-6646754,4226801443964164083),(605,993416606925324547,-7629165),(614,NULL,-8422993161921369978),(632,26,NULL),(637,6340059197405004080,37),(638,2219025844682861975,-9181477340794568463),(640,8703206720597762646,-2467944),(658,4564753,-5711632628628133377),(675,7200328330890720300,NULL),(681,NULL,-2138727),(684,24109,10781),(696,-83,104),(705,-507277,NULL),(708,-422,-20919),(711,6420852753839600494,-253548),(712,-11386,-8748934422715143027),(717,-1244366,-2622781283346094733),(729,-2785524705743908777,946),(752,4226304450905757021,-89),(777,-34,3949395314502958079),(780,-6183873771127917812,6),(783,NULL,-27038),(785,NULL,18084),(797,7165227569808679512,9086183973837245793),(824,NULL,7837168),(829,-4019,8693244640059699436),(836,-4326053640378667417,NULL),(846,-20,NULL),(863,-6971126357652283746,1608289151182385306),(866,-7272848223721510499,-3734731507169483271),(875,NULL,-86),(895,3575,8979114980730519384),(897,1066324201229535054,-8881870179956783400),(900,-9137449359837863640,59),(907,-127,-26),(909,NULL,7307),(912,NULL,NULL),(918,-31299,8123726555850609990),(924,-7473599114166575834,8509366320847652317),(936,1091299238574024073,-8659602183742129639),(975,NULL,-1376426860463547942),(992,6920936140426763341,125),(3,-17502,-95),(6,NULL,-5683413993623134550),(15,8363285885828694100,2082650),(24,16827,4),(30,-1099629164736334542,5627737),(49,5468397,8230743511360080890),(56,1,NULL),(61,-35,-4087),(72,6236410,-18721589225916857),(78,109,-6462502474718655676),(92,NULL,2737271036229952551),(117,75,79),(118,-5547888431972866290,-60),(124,NULL,-3502233152353710671),(139,-318858,-6145754),(144,NULL,-8081314284157635734),(162,NULL,-4007796119574596450),(185,24,6092784220403498333),(186,NULL,-1925856382609852356),(192,NULL,1793501792072506064),(202,NULL,6584),(204,8801039723827585466,1),(210,-27601,-7479594981745120479),(212,-19929,NULL),(230,-124,3081827),(234,-5638401233682557873,-2972958),(247,NULL,-17384),(251,-17963,-17493),(253,NULL,431994),(258,3811692031303816498,-8980399418514755466),(260,-2162298,3111495764270491971),(269,1240606550222638489,-2025089919332743348),(284,NULL,-18028),(289,7840072496645669499,94),(292,-24068,NULL),(304,-2509251131080275613,NULL),(325,-5307406235636027683,-2000830512981277932),(330,4693318918861754777,-28052),(338,-47,-940286411604413543),(350,NULL,-2),(364,2764581187917452461,-2936323506157834634),(393,NULL,NULL),(399,-6380694531413692838,83),(411,NULL,NULL),(421,8067341,4237809),(422,4406443631050203521,-2897),(425,NULL,8502),(436,-6647982904353318197,-67),(461,-15865,-2475591754437600283),(462,43,NULL),(464,NULL,100),(469,-35,7531),(474,1729632703944492122,-414181),(493,-8868224922750828162,8275),(512,-6626760,12915),(517,5128,-87),(518,NULL,13155),(521,82,-4710064576991418898),(539,-1375268,-4843085683459073436),(578,-831682,6763301871997841994),(580,-3688788363461668115,NULL),(597,2980694444701422663,-9174131947691445728),(604,7727609030398481542,71),(609,-117,-31169),(622,234855394259543210,-8929516928448387753),(642,-3136863043231674436,NULL),(651,-2992469382270864975,NULL),(654,-1040676426731775778,-52),(661,-13123,-858),(680,-8459283889233837633,NULL),(683,-21,4748),(694,NULL,4182044146860417311),(734,-704713319638119722,8724468042514057496),(754,-6749223,-9),(760,5822749402259818364,NULL),(782,-8327928,8712801357888227090),(787,NULL,-8782409205836577879),(799,-52,7489470),(807,-11721,NULL),(813,NULL,NULL),(822,2917776,-5540580),(825,NULL,NULL),(842,-8663028079322341263,8529054945093469750),(850,-59,27),(873,-7420016877487330561,3063312),(902,24249,-242451),(923,-54,-2013158759380653631),(925,-8,-5838798365711545535),(938,25829,-6116),(945,-19,NULL),(955,-7043,-2666830601791651688),(958,-7699885015045149322,7342680),(962,7810427825411411021,-3301282690172279938),(985,NULL,NULL),(986,449502,-16); + """ + qt_sql """ + SELECT pk AS C1 FROM table_1000_undef_undef2 AS T1 WHERE `col_bigint_undef_signed` NOT IN (SELECT `col_bigint_undef_signed2` AS C2 FROM table_1000_undef_undef2 AS T2 ) OR `col_bigint_undef_signed2` <> 9 order by 1; + """ +} \ No newline at end of file diff --git a/regression-test/suites/query_p0/sql_functions/window_functions/test_window_fn.groovy b/regression-test/suites/query_p0/sql_functions/window_functions/test_window_fn.groovy index 22a9e798f0abe4e..e8604debe45cec7 100644 --- a/regression-test/suites/query_p0/sql_functions/window_functions/test_window_fn.groovy +++ b/regression-test/suites/query_p0/sql_functions/window_functions/test_window_fn.groovy @@ -383,10 +383,8 @@ suite("test_window_fn") { "storage_format" = "V2" ); """ - test { - sql """SELECT SUM(MAX(c1) OVER (PARTITION BY c2, c3)) FROM test_window_in_agg;""" - exception "errCode = 2, detailMessage = AGGREGATE clause must not contain analytic expressions" - } + sql """set enable_nereids_planner=true;""" + sql """SELECT SUM(MAX(c1) OVER (PARTITION BY c2, c3)) FROM test_window_in_agg;""" sql "DROP TABLE IF EXISTS test_window_in_agg;" } diff --git a/regression-test/suites/statistics/analyze_stats.groovy b/regression-test/suites/statistics/analyze_stats.groovy index 73d3263487d6f99..3adfe2be2082011 100644 --- a/regression-test/suites/statistics/analyze_stats.groovy +++ b/regression-test/suites/statistics/analyze_stats.groovy @@ -1464,6 +1464,1120 @@ PARTITION `p599` VALUES IN (599) assertEquals("N/A", result[0][6]) assertEquals("N/A", result[0][7]) + // Test drop stats with more than 1024 columns. + sql """ + CREATE TABLE if not exists col1100 ( + c1 int, + c2 int, + c3 int, + c4 int, + c5 int, + c6 int, + c7 int, + c8 int, + c9 int, + c10 int, + c11 int, + c12 int, + c13 int, + c14 int, + c15 int, + c16 int, + c17 int, + c18 int, + c19 int, + c20 int, + c21 int, + c22 int, + c23 int, + c24 int, + c25 int, + c26 int, + c27 int, + c28 int, + c29 int, + c30 int, + c31 int, + c32 int, + c33 int, + c34 int, + c35 int, + c36 int, + c37 int, + c38 int, + c39 int, + c40 int, + c41 int, + c42 int, + c43 int, + c44 int, + c45 int, + c46 int, + c47 int, + c48 int, + c49 int, + c50 int, + c51 int, + c52 int, + c53 int, + c54 int, + c55 int, + c56 int, + c57 int, + c58 int, + c59 int, + c60 int, + c61 int, + c62 int, + c63 int, + c64 int, + c65 int, + c66 int, + c67 int, + c68 int, + c69 int, + c70 int, + c71 int, + c72 int, + c73 int, + c74 int, + c75 int, + c76 int, + c77 int, + c78 int, + c79 int, + c80 int, + c81 int, + c82 int, + c83 int, + c84 int, + c85 int, + c86 int, + c87 int, + c88 int, + c89 int, + c90 int, + c91 int, + c92 int, + c93 int, + c94 int, + c95 int, + c96 int, + c97 int, + c98 int, + c99 int, + c100 int, + c101 int, + c102 int, + c103 int, + c104 int, + c105 int, + c106 int, + c107 int, + c108 int, + c109 int, + c110 int, + c111 int, + c112 int, + c113 int, + c114 int, + c115 int, + c116 int, + c117 int, + c118 int, + c119 int, + c120 int, + c121 int, + c122 int, + c123 int, + c124 int, + c125 int, + c126 int, + c127 int, + c128 int, + c129 int, + c130 int, + c131 int, + c132 int, + c133 int, + c134 int, + c135 int, + c136 int, + c137 int, + c138 int, + c139 int, + c140 int, + c141 int, + c142 int, + c143 int, + c144 int, + c145 int, + c146 int, + c147 int, + c148 int, + c149 int, + c150 int, + c151 int, + c152 int, + c153 int, + c154 int, + c155 int, + c156 int, + c157 int, + c158 int, + c159 int, + c160 int, + c161 int, + c162 int, + c163 int, + c164 int, + c165 int, + c166 int, + c167 int, + c168 int, + c169 int, + c170 int, + c171 int, + c172 int, + c173 int, + c174 int, + c175 int, + c176 int, + c177 int, + c178 int, + c179 int, + c180 int, + c181 int, + c182 int, + c183 int, + c184 int, + c185 int, + c186 int, + c187 int, + c188 int, + c189 int, + c190 int, + c191 int, + c192 int, + c193 int, + c194 int, + c195 int, + c196 int, + c197 int, + c198 int, + c199 int, + c200 int, + c201 int, + c202 int, + c203 int, + c204 int, + c205 int, + c206 int, + c207 int, + c208 int, + c209 int, + c210 int, + c211 int, + c212 int, + c213 int, + c214 int, + c215 int, + c216 int, + c217 int, + c218 int, + c219 int, + c220 int, + c221 int, + c222 int, + c223 int, + c224 int, + c225 int, + c226 int, + c227 int, + c228 int, + c229 int, + c230 int, + c231 int, + c232 int, + c233 int, + c234 int, + c235 int, + c236 int, + c237 int, + c238 int, + c239 int, + c240 int, + c241 int, + c242 int, + c243 int, + c244 int, + c245 int, + c246 int, + c247 int, + c248 int, + c249 int, + c250 int, + c251 int, + c252 int, + c253 int, + c254 int, + c255 int, + c256 int, + c257 int, + c258 int, + c259 int, + c260 int, + c261 int, + c262 int, + c263 int, + c264 int, + c265 int, + c266 int, + c267 int, + c268 int, + c269 int, + c270 int, + c271 int, + c272 int, + c273 int, + c274 int, + c275 int, + c276 int, + c277 int, + c278 int, + c279 int, + c280 int, + c281 int, + c282 int, + c283 int, + c284 int, + c285 int, + c286 int, + c287 int, + c288 int, + c289 int, + c290 int, + c291 int, + c292 int, + c293 int, + c294 int, + c295 int, + c296 int, + c297 int, + c298 int, + c299 int, + c300 int, + c301 int, + c302 int, + c303 int, + c304 int, + c305 int, + c306 int, + c307 int, + c308 int, + c309 int, + c310 int, + c311 int, + c312 int, + c313 int, + c314 int, + c315 int, + c316 int, + c317 int, + c318 int, + c319 int, + c320 int, + c321 int, + c322 int, + c323 int, + c324 int, + c325 int, + c326 int, + c327 int, + c328 int, + c329 int, + c330 int, + c331 int, + c332 int, + c333 int, + c334 int, + c335 int, + c336 int, + c337 int, + c338 int, + c339 int, + c340 int, + c341 int, + c342 int, + c343 int, + c344 int, + c345 int, + c346 int, + c347 int, + c348 int, + c349 int, + c350 int, + c351 int, + c352 int, + c353 int, + c354 int, + c355 int, + c356 int, + c357 int, + c358 int, + c359 int, + c360 int, + c361 int, + c362 int, + c363 int, + c364 int, + c365 int, + c366 int, + c367 int, + c368 int, + c369 int, + c370 int, + c371 int, + c372 int, + c373 int, + c374 int, + c375 int, + c376 int, + c377 int, + c378 int, + c379 int, + c380 int, + c381 int, + c382 int, + c383 int, + c384 int, + c385 int, + c386 int, + c387 int, + c388 int, + c389 int, + c390 int, + c391 int, + c392 int, + c393 int, + c394 int, + c395 int, + c396 int, + c397 int, + c398 int, + c399 int, + c400 int, + c401 int, + c402 int, + c403 int, + c404 int, + c405 int, + c406 int, + c407 int, + c408 int, + c409 int, + c410 int, + c411 int, + c412 int, + c413 int, + c414 int, + c415 int, + c416 int, + c417 int, + c418 int, + c419 int, + c420 int, + c421 int, + c422 int, + c423 int, + c424 int, + c425 int, + c426 int, + c427 int, + c428 int, + c429 int, + c430 int, + c431 int, + c432 int, + c433 int, + c434 int, + c435 int, + c436 int, + c437 int, + c438 int, + c439 int, + c440 int, + c441 int, + c442 int, + c443 int, + c444 int, + c445 int, + c446 int, + c447 int, + c448 int, + c449 int, + c450 int, + c451 int, + c452 int, + c453 int, + c454 int, + c455 int, + c456 int, + c457 int, + c458 int, + c459 int, + c460 int, + c461 int, + c462 int, + c463 int, + c464 int, + c465 int, + c466 int, + c467 int, + c468 int, + c469 int, + c470 int, + c471 int, + c472 int, + c473 int, + c474 int, + c475 int, + c476 int, + c477 int, + c478 int, + c479 int, + c480 int, + c481 int, + c482 int, + c483 int, + c484 int, + c485 int, + c486 int, + c487 int, + c488 int, + c489 int, + c490 int, + c491 int, + c492 int, + c493 int, + c494 int, + c495 int, + c496 int, + c497 int, + c498 int, + c499 int, + c500 int, + c501 int, + c502 int, + c503 int, + c504 int, + c505 int, + c506 int, + c507 int, + c508 int, + c509 int, + c510 int, + c511 int, + c512 int, + c513 int, + c514 int, + c515 int, + c516 int, + c517 int, + c518 int, + c519 int, + c520 int, + c521 int, + c522 int, + c523 int, + c524 int, + c525 int, + c526 int, + c527 int, + c528 int, + c529 int, + c530 int, + c531 int, + c532 int, + c533 int, + c534 int, + c535 int, + c536 int, + c537 int, + c538 int, + c539 int, + c540 int, + c541 int, + c542 int, + c543 int, + c544 int, + c545 int, + c546 int, + c547 int, + c548 int, + c549 int, + c550 int, + c551 int, + c552 int, + c553 int, + c554 int, + c555 int, + c556 int, + c557 int, + c558 int, + c559 int, + c560 int, + c561 int, + c562 int, + c563 int, + c564 int, + c565 int, + c566 int, + c567 int, + c568 int, + c569 int, + c570 int, + c571 int, + c572 int, + c573 int, + c574 int, + c575 int, + c576 int, + c577 int, + c578 int, + c579 int, + c580 int, + c581 int, + c582 int, + c583 int, + c584 int, + c585 int, + c586 int, + c587 int, + c588 int, + c589 int, + c590 int, + c591 int, + c592 int, + c593 int, + c594 int, + c595 int, + c596 int, + c597 int, + c598 int, + c599 int, + c600 int, + c601 int, + c602 int, + c603 int, + c604 int, + c605 int, + c606 int, + c607 int, + c608 int, + c609 int, + c610 int, + c611 int, + c612 int, + c613 int, + c614 int, + c615 int, + c616 int, + c617 int, + c618 int, + c619 int, + c620 int, + c621 int, + c622 int, + c623 int, + c624 int, + c625 int, + c626 int, + c627 int, + c628 int, + c629 int, + c630 int, + c631 int, + c632 int, + c633 int, + c634 int, + c635 int, + c636 int, + c637 int, + c638 int, + c639 int, + c640 int, + c641 int, + c642 int, + c643 int, + c644 int, + c645 int, + c646 int, + c647 int, + c648 int, + c649 int, + c650 int, + c651 int, + c652 int, + c653 int, + c654 int, + c655 int, + c656 int, + c657 int, + c658 int, + c659 int, + c660 int, + c661 int, + c662 int, + c663 int, + c664 int, + c665 int, + c666 int, + c667 int, + c668 int, + c669 int, + c670 int, + c671 int, + c672 int, + c673 int, + c674 int, + c675 int, + c676 int, + c677 int, + c678 int, + c679 int, + c680 int, + c681 int, + c682 int, + c683 int, + c684 int, + c685 int, + c686 int, + c687 int, + c688 int, + c689 int, + c690 int, + c691 int, + c692 int, + c693 int, + c694 int, + c695 int, + c696 int, + c697 int, + c698 int, + c699 int, + c700 int, + c701 int, + c702 int, + c703 int, + c704 int, + c705 int, + c706 int, + c707 int, + c708 int, + c709 int, + c710 int, + c711 int, + c712 int, + c713 int, + c714 int, + c715 int, + c716 int, + c717 int, + c718 int, + c719 int, + c720 int, + c721 int, + c722 int, + c723 int, + c724 int, + c725 int, + c726 int, + c727 int, + c728 int, + c729 int, + c730 int, + c731 int, + c732 int, + c733 int, + c734 int, + c735 int, + c736 int, + c737 int, + c738 int, + c739 int, + c740 int, + c741 int, + c742 int, + c743 int, + c744 int, + c745 int, + c746 int, + c747 int, + c748 int, + c749 int, + c750 int, + c751 int, + c752 int, + c753 int, + c754 int, + c755 int, + c756 int, + c757 int, + c758 int, + c759 int, + c760 int, + c761 int, + c762 int, + c763 int, + c764 int, + c765 int, + c766 int, + c767 int, + c768 int, + c769 int, + c770 int, + c771 int, + c772 int, + c773 int, + c774 int, + c775 int, + c776 int, + c777 int, + c778 int, + c779 int, + c780 int, + c781 int, + c782 int, + c783 int, + c784 int, + c785 int, + c786 int, + c787 int, + c788 int, + c789 int, + c790 int, + c791 int, + c792 int, + c793 int, + c794 int, + c795 int, + c796 int, + c797 int, + c798 int, + c799 int, + c800 int, + c801 int, + c802 int, + c803 int, + c804 int, + c805 int, + c806 int, + c807 int, + c808 int, + c809 int, + c810 int, + c811 int, + c812 int, + c813 int, + c814 int, + c815 int, + c816 int, + c817 int, + c818 int, + c819 int, + c820 int, + c821 int, + c822 int, + c823 int, + c824 int, + c825 int, + c826 int, + c827 int, + c828 int, + c829 int, + c830 int, + c831 int, + c832 int, + c833 int, + c834 int, + c835 int, + c836 int, + c837 int, + c838 int, + c839 int, + c840 int, + c841 int, + c842 int, + c843 int, + c844 int, + c845 int, + c846 int, + c847 int, + c848 int, + c849 int, + c850 int, + c851 int, + c852 int, + c853 int, + c854 int, + c855 int, + c856 int, + c857 int, + c858 int, + c859 int, + c860 int, + c861 int, + c862 int, + c863 int, + c864 int, + c865 int, + c866 int, + c867 int, + c868 int, + c869 int, + c870 int, + c871 int, + c872 int, + c873 int, + c874 int, + c875 int, + c876 int, + c877 int, + c878 int, + c879 int, + c880 int, + c881 int, + c882 int, + c883 int, + c884 int, + c885 int, + c886 int, + c887 int, + c888 int, + c889 int, + c890 int, + c891 int, + c892 int, + c893 int, + c894 int, + c895 int, + c896 int, + c897 int, + c898 int, + c899 int, + c900 int, + c901 int, + c902 int, + c903 int, + c904 int, + c905 int, + c906 int, + c907 int, + c908 int, + c909 int, + c910 int, + c911 int, + c912 int, + c913 int, + c914 int, + c915 int, + c916 int, + c917 int, + c918 int, + c919 int, + c920 int, + c921 int, + c922 int, + c923 int, + c924 int, + c925 int, + c926 int, + c927 int, + c928 int, + c929 int, + c930 int, + c931 int, + c932 int, + c933 int, + c934 int, + c935 int, + c936 int, + c937 int, + c938 int, + c939 int, + c940 int, + c941 int, + c942 int, + c943 int, + c944 int, + c945 int, + c946 int, + c947 int, + c948 int, + c949 int, + c950 int, + c951 int, + c952 int, + c953 int, + c954 int, + c955 int, + c956 int, + c957 int, + c958 int, + c959 int, + c960 int, + c961 int, + c962 int, + c963 int, + c964 int, + c965 int, + c966 int, + c967 int, + c968 int, + c969 int, + c970 int, + c971 int, + c972 int, + c973 int, + c974 int, + c975 int, + c976 int, + c977 int, + c978 int, + c979 int, + c980 int, + c981 int, + c982 int, + c983 int, + c984 int, + c985 int, + c986 int, + c987 int, + c988 int, + c989 int, + c990 int, + c991 int, + c992 int, + c993 int, + c994 int, + c995 int, + c996 int, + c997 int, + c998 int, + c999 int, + c1000 int, + c1001 int, + c1002 int, + c1003 int, + c1004 int, + c1005 int, + c1006 int, + c1007 int, + c1008 int, + c1009 int, + c1010 int, + c1011 int, + c1012 int, + c1013 int, + c1014 int, + c1015 int, + c1016 int, + c1017 int, + c1018 int, + c1019 int, + c1020 int, + c1021 int, + c1022 int, + c1023 int, + c1024 int, + c1025 int, + c1026 int, + c1027 int, + c1028 int, + c1029 int, + c1030 int, + c1031 int, + c1032 int, + c1033 int, + c1034 int, + c1035 int, + c1036 int, + c1037 int, + c1038 int, + c1039 int, + c1040 int, + c1041 int, + c1042 int, + c1043 int, + c1044 int, + c1045 int, + c1046 int, + c1047 int, + c1048 int, + c1049 int, + c1050 int, + c1051 int, + c1052 int, + c1053 int, + c1054 int, + c1055 int, + c1056 int, + c1057 int, + c1058 int, + c1059 int, + c1060 int, + c1061 int, + c1062 int, + c1063 int, + c1064 int, + c1065 int, + c1066 int, + c1067 int, + c1068 int, + c1069 int, + c1070 int, + c1071 int, + c1072 int, + c1073 int, + c1074 int, + c1075 int, + c1076 int, + c1077 int, + c1078 int, + c1079 int, + c1080 int, + c1081 int, + c1082 int, + c1083 int, + c1084 int, + c1085 int, + c1086 int, + c1087 int, + c1088 int, + c1089 int, + c1090 int, + c1091 int, + c1092 int, + c1093 int, + c1094 int, + c1095 int, + c1096 int, + c1097 int, + c1098 int, + c1099 int, + c1100 int + )ENGINE=OLAP + DUPLICATE KEY(`c1`) + COMMENT "OLAP" + DISTRIBUTED BY HASH(`c1`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """drop stats col1100 """ + sql """DROP TABLE IF EXISTS col1100""" + // Test trigger type. sql """DROP DATABASE IF EXISTS trigger""" sql """CREATE DATABASE IF NOT EXISTS trigger""" @@ -1501,4 +2615,6 @@ PARTITION `p599` VALUES IN (599) assertEquals(result[0][10], "MANUAL") assertEquals(result[1][10], "MANUAL") } + sql """DROP DATABASE IF EXISTS trigger""" + } diff --git a/regression-test/suites/unique_with_mow_p0/test_delete_sign_delete_bitmap.groovy b/regression-test/suites/unique_with_mow_p0/test_delete_sign_delete_bitmap.groovy index f9b89c1eea7a277..be6324d2ec76a64 100644 --- a/regression-test/suites/unique_with_mow_p0/test_delete_sign_delete_bitmap.groovy +++ b/regression-test/suites/unique_with_mow_p0/test_delete_sign_delete_bitmap.groovy @@ -17,80 +17,80 @@ suite('test_delete_sign_delete_bitmap') { - def tableName1 = "test_delete_sign_delete_bitmap1" - sql "DROP TABLE IF EXISTS ${tableName1};" - sql """ CREATE TABLE IF NOT EXISTS ${tableName1} ( - `k1` int NOT NULL, - `c1` int, - `c2` int, - `c3` int, - `c4` int - )UNIQUE KEY(k1) - DISTRIBUTED BY HASH(k1) BUCKETS 1 - PROPERTIES ( - "enable_unique_key_merge_on_write" = "true", - "disable_auto_compaction" = "true", - "replication_num" = "1" - );""" + // def tableName1 = "test_delete_sign_delete_bitmap1" + // sql "DROP TABLE IF EXISTS ${tableName1};" + // sql """ CREATE TABLE IF NOT EXISTS ${tableName1} ( + // `k1` int NOT NULL, + // `c1` int, + // `c2` int, + // `c3` int, + // `c4` int + // )UNIQUE KEY(k1) + // DISTRIBUTED BY HASH(k1) BUCKETS 1 + // PROPERTIES ( + // "enable_unique_key_merge_on_write" = "true", + // "disable_auto_compaction" = "true", + // "replication_num" = "1" + // );""" - sql "insert into ${tableName1} values(1,1,1,1,1),(2,2,2,2,2),(3,3,3,3,3),(4,4,4,4,4),(5,5,5,5,5);" - qt_sql "select * from ${tableName1} order by k1,c1,c2,c3,c4;" - // sql "insert into ${tableName1}(k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__) select k1,c1,c2,c3,c4,1 from ${tableName1} where k1 in (1,3,5);" - sql """insert into ${tableName1}(k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__) values(1,1,1,1,1,1),(3,3,3,3,3,1),(5,5,5,5,5,1);""" - sql "sync" - qt_after_delete "select * from ${tableName1} order by k1,c1,c2,c3,c4;" - sql "set skip_delete_sign=true;" - sql "set skip_storage_engine_merge=true;" - sql "set skip_delete_bitmap=true;" - sql "sync" - // skip_delete_bitmap=true, skip_delete_sign=true - qt_1 "select k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__ from ${tableName1} order by k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__;" + // sql "insert into ${tableName1} values(1,1,1,1,1),(2,2,2,2,2),(3,3,3,3,3),(4,4,4,4,4),(5,5,5,5,5);" + // qt_sql "select * from ${tableName1} order by k1,c1,c2,c3,c4;" + // // sql "insert into ${tableName1}(k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__) select k1,c1,c2,c3,c4,1 from ${tableName1} where k1 in (1,3,5);" + // sql """insert into ${tableName1}(k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__) values(1,1,1,1,1,1),(3,3,3,3,3,1),(5,5,5,5,5,1);""" + // sql "sync" + // qt_after_delete "select * from ${tableName1} order by k1,c1,c2,c3,c4;" + // sql "set skip_delete_sign=true;" + // sql "set skip_storage_engine_merge=true;" + // sql "set skip_delete_bitmap=true;" + // sql "sync" + // // skip_delete_bitmap=true, skip_delete_sign=true + // qt_1 "select k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__ from ${tableName1} order by k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__;" - sql "set skip_delete_sign=true;" - sql "set skip_delete_bitmap=false;" - sql "sync" - // skip_delete_bitmap=false, skip_delete_sign=true - qt_2 "select k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__ from ${tableName1} order by k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__;" - sql "drop table if exists ${tableName1};" + // sql "set skip_delete_sign=true;" + // sql "set skip_delete_bitmap=false;" + // sql "sync" + // // skip_delete_bitmap=false, skip_delete_sign=true + // qt_2 "select k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__ from ${tableName1} order by k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__;" + // sql "drop table if exists ${tableName1};" - sql "set skip_delete_sign=false;" - sql "set skip_storage_engine_merge=false;" - sql "set skip_delete_bitmap=false;" - sql "sync" - def tableName2 = "test_delete_sign_delete_bitmap2" - sql "DROP TABLE IF EXISTS ${tableName2};" - sql """ CREATE TABLE IF NOT EXISTS ${tableName2} ( - `k1` int NOT NULL, - `c1` int, - `c2` int, - `c3` int, - `c4` int - )UNIQUE KEY(k1) - DISTRIBUTED BY HASH(k1) BUCKETS 1 - PROPERTIES ( - "enable_unique_key_merge_on_write" = "true", - "disable_auto_compaction" = "true", - "replication_num" = "1", - "function_column.sequence_col" = 'c4' - );""" + // sql "set skip_delete_sign=false;" + // sql "set skip_storage_engine_merge=false;" + // sql "set skip_delete_bitmap=false;" + // sql "sync" + // def tableName2 = "test_delete_sign_delete_bitmap2" + // sql "DROP TABLE IF EXISTS ${tableName2};" + // sql """ CREATE TABLE IF NOT EXISTS ${tableName2} ( + // `k1` int NOT NULL, + // `c1` int, + // `c2` int, + // `c3` int, + // `c4` int + // )UNIQUE KEY(k1) + // DISTRIBUTED BY HASH(k1) BUCKETS 1 + // PROPERTIES ( + // "enable_unique_key_merge_on_write" = "true", + // "disable_auto_compaction" = "true", + // "replication_num" = "1", + // "function_column.sequence_col" = 'c4' + // );""" - sql "insert into ${tableName2} values(1,1,1,1,1),(2,2,2,2,2),(3,3,3,3,3),(4,4,4,4,4),(5,5,5,5,5);" - qt_sql "select * from ${tableName2} order by k1,c1,c2,c3,c4;" - sql """insert into ${tableName2}(k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__) values(1,1,1,1,1,1),(3,3,3,3,3,1),(5,5,5,5,5,1);""" - sql "sync" - qt_after_delete "select * from ${tableName2} order by k1,c1,c2,c3,c4;" - sql "set skip_delete_sign=true;" - sql "set skip_storage_engine_merge=true;" - sql "set skip_delete_bitmap=true;" - sql "sync" - // skip_delete_bitmap=true, skip_delete_sign=true - qt_1 "select k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__ from ${tableName2} order by k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__;" + // sql "insert into ${tableName2} values(1,1,1,1,1),(2,2,2,2,2),(3,3,3,3,3),(4,4,4,4,4),(5,5,5,5,5);" + // qt_sql "select * from ${tableName2} order by k1,c1,c2,c3,c4;" + // sql """insert into ${tableName2}(k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__) values(1,1,1,1,1,1),(3,3,3,3,3,1),(5,5,5,5,5,1);""" + // sql "sync" + // qt_after_delete "select * from ${tableName2} order by k1,c1,c2,c3,c4;" + // sql "set skip_delete_sign=true;" + // sql "set skip_storage_engine_merge=true;" + // sql "set skip_delete_bitmap=true;" + // sql "sync" + // // skip_delete_bitmap=true, skip_delete_sign=true + // qt_1 "select k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__ from ${tableName2} order by k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__;" - sql "set skip_delete_sign=true;" - sql "set skip_delete_bitmap=false;" - sql "sync" - // skip_delete_bitmap=false, skip_delete_sign=true - qt_2 "select k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__ from ${tableName2} order by k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__;" - sql "drop table if exists ${tableName2};" + // sql "set skip_delete_sign=true;" + // sql "set skip_delete_bitmap=false;" + // sql "sync" + // // skip_delete_bitmap=false, skip_delete_sign=true + // qt_2 "select k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__ from ${tableName2} order by k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__;" + // sql "drop table if exists ${tableName2};" } diff --git a/tools/tpcds-tools/bin/create-tpcds-tables.sh b/tools/tpcds-tools/bin/create-tpcds-tables.sh index 496f71fee7b4ea8..c9268fef7e53572 100755 --- a/tools/tpcds-tools/bin/create-tpcds-tables.sh +++ b/tools/tpcds-tools/bin/create-tpcds-tables.sh @@ -42,6 +42,7 @@ Usage: $0 OPTS=$(getopt \ -n "$0" \ -o '' \ + -o 'hs:' \ -- "$@") eval set -- "${OPTS}" @@ -58,6 +59,10 @@ while true; do HELP=1 shift ;; + -s) + SCALE_FACTOR=$2 + shift 2 + ;; --) shift break @@ -73,6 +78,11 @@ if [[ "${HELP}" -eq 1 ]]; then usage fi +if [[ ${SCALE_FACTOR} -ne 1 ]] && [[ ${SCALE_FACTOR} -ne 100 ]] && [[ ${SCALE_FACTOR} -ne 1000 ]] && [[ ${SCALE_FACTOR} -ne 10000 ]]; then + echo "${SCALE_FACTOR} scale is not supported" + exit 1 +fi + check_prerequest() { local CMD=$1 local NAME=$2 diff --git a/tools/tpcds-tools/bin/load-tpcds-data.sh b/tools/tpcds-tools/bin/load-tpcds-data.sh index a5fe926f652ec65..5bd4f1b34818c06 100755 --- a/tools/tpcds-tools/bin/load-tpcds-data.sh +++ b/tools/tpcds-tools/bin/load-tpcds-data.sh @@ -197,3 +197,15 @@ end_time=$(date +%s) echo "End time: $(date)" echo "Finish load tpcds data, Time taken: $((end_time - start_time)) seconds" +echo '============================================' +echo "analyze database ${DB}" +run_sql() { + echo "$*" + mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" -e "$*" +} +start=$(date +%s) +run_sql "analyze database ${DB} with sync;" +end=$(date +%s) +analyzeTime=$((end - start)) +echo "analyze database ${DB} with sync total time: ${analyzeTime} s" +echo '============================================' diff --git a/tools/tpcds-tools/bin/run-tpcds-queries.sh b/tools/tpcds-tools/bin/run-tpcds-queries.sh index b404eaf0794808e..2fe80afe04ed695 100755 --- a/tools/tpcds-tools/bin/run-tpcds-queries.sh +++ b/tools/tpcds-tools/bin/run-tpcds-queries.sh @@ -29,11 +29,10 @@ ROOT=$( ) CURDIR="${ROOT}" -TPCDS_QUERIES_DIR="${CURDIR}/../queries" usage() { echo " -This script is used to run TPC-DS 103 queries, +This script is used to run TPC-DS 99 queries, will use mysql client to connect Doris server which parameter is specified in doris-cluster.conf file. Usage: $0 " @@ -43,10 +42,12 @@ Usage: $0 OPTS=$(getopt \ -n "$0" \ -o '' \ + -o 'hs:' \ -- "$@") eval set -- "${OPTS}" HELP=0 +SCALE_FACTOR=1 if [[ $# == 0 ]]; then usage @@ -58,6 +59,10 @@ while true; do HELP=1 shift ;; + -s) + SCALE_FACTOR=$2 + shift 2 + ;; --) shift break @@ -73,6 +78,27 @@ if [[ "${HELP}" -eq 1 ]]; then usage fi +if [[ ${SCALE_FACTOR} -eq 1 ]]; then + echo "Running tpcds sf 1 queries" + TPCDS_QUERIES_DIR="${CURDIR}/../queries/sf1" + TPCDS_OPT_CONF="${CURDIR}/../conf/opt/opt_sf1.sql" +elif [[ ${SCALE_FACTOR} -eq 100 ]]; then + echo "Running tpcds sf 100 queries" + TPCDS_QUERIES_DIR="${CURDIR}/../queries/sf100" + TPCDS_OPT_CONF="${CURDIR}/../conf/opt/opt_sf100.sql" +elif [[ ${SCALE_FACTOR} -eq 1000 ]]; then + echo "Running tpcds sf 1000 queries" + TPCDS_QUERIES_DIR="${CURDIR}/../queries/sf1000" + TPCDS_OPT_CONF="${CURDIR}/../conf/opt/opt_sf1000.sql" +elif [[ ${SCALE_FACTOR} -eq 10000 ]]; then + echo "Running tpcds sf 10000 queries" + TPCDS_QUERIES_DIR="${CURDIR}/../queries/sf10000" + TPCDS_OPT_CONF="${CURDIR}/../conf/opt/opt_sf10000.sql" +else + echo "${SCALE_FACTOR} scale is NOT support currently." + exit 1 +fi + check_prerequest() { local CMD=$1 local NAME=$2 @@ -84,7 +110,6 @@ check_prerequest() { check_prerequest "mysql --version" "mysql" -#shellcheck source=/dev/null source "${CURDIR}/../conf/doris-cluster.conf" export MYSQL_PWD=${PASSWORD:-} @@ -99,21 +124,18 @@ run_sql() { mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" -e "$*" } +echo '============================================' +run_sql "source ${TPCDS_OPT_CONF};" echo '============================================' run_sql "show variables;" echo '============================================' run_sql "show table status;" echo '============================================' -start=$(date +%s) -run_sql "analyze database ${DB} with full with sync;" -end=$(date +%s) -totalTime=$((end - start)) -echo "analyze database ${DB} with full with sync total time: ${totalTime} s" -echo '============================================' -echo "Time Unit: ms" RESULT_DIR="${CURDIR}/result" -rm "${RESULT_DIR}" +if [[ -d "${RESULT_DIR}" ]]; then + rm -r "${RESULT_DIR}" +fi mkdir -p "${RESULT_DIR}" touch result.csv cold_run_sum=0 @@ -153,4 +175,6 @@ for i in {1..99}; do fi done +echo "Total cold run time: ${cold_run_sum} ms" +echo "Total hot run time: ${best_hot_run_sum} ms" echo 'Finish tpcds queries.' diff --git a/tools/tpcds-tools/conf/opt/opt_sf1.sql b/tools/tpcds-tools/conf/opt/opt_sf1.sql new file mode 100644 index 000000000000000..e4a4d26dec59d60 --- /dev/null +++ b/tools/tpcds-tools/conf/opt/opt_sf1.sql @@ -0,0 +1,3 @@ +set global experimental_enable_nereids_planner=true; +set global experimental_enable_pipeline_engine=true; +set global runtime_filter_wait_time_ms=1000; diff --git a/tools/tpcds-tools/conf/opt/opt_sf100.sql b/tools/tpcds-tools/conf/opt/opt_sf100.sql new file mode 100644 index 000000000000000..231906fe506bd70 --- /dev/null +++ b/tools/tpcds-tools/conf/opt/opt_sf100.sql @@ -0,0 +1,3 @@ +set global experimental_enable_nereids_planner=true; +set global experimental_enable_pipeline_engine=true; +set global runtime_filter_wait_time_ms=10000; diff --git a/tools/tpcds-tools/conf/opt/opt_sf1000.sql b/tools/tpcds-tools/conf/opt/opt_sf1000.sql new file mode 100644 index 000000000000000..6a42b7071b1c210 --- /dev/null +++ b/tools/tpcds-tools/conf/opt/opt_sf1000.sql @@ -0,0 +1,6 @@ +set global experimental_enable_nereids_planner=true; +set global experimental_enable_pipeline_engine=true; +set global runtime_filter_wait_time_ms=10000; +set global broadcast_right_table_scale_factor=0.0; +set global max_join_number_bushy_tree=10; +set global memo_max_group_expression_size=15000; diff --git a/tools/tpcds-tools/conf/opt/opt_sf10000.sql b/tools/tpcds-tools/conf/opt/opt_sf10000.sql new file mode 100644 index 000000000000000..ce361d4e3f3afed --- /dev/null +++ b/tools/tpcds-tools/conf/opt/opt_sf10000.sql @@ -0,0 +1,6 @@ +set global experimental_enable_nereids_planner=true; +set global experimental_enable_pipeline_engine=true; +set global runtime_filter_wait_time_ms=100000; +set global broadcast_right_table_scale_factor=5.0; +set global max_join_number_bushy_tree=10; +set global memo_max_group_expression_size=15000; diff --git a/tools/tpcds-tools/ddl/create-tpcds-tables-sf100.sql b/tools/tpcds-tools/ddl/create-tpcds-tables-sf100.sql new file mode 100644 index 000000000000000..6ed0f820eeb2e26 --- /dev/null +++ b/tools/tpcds-tools/ddl/create-tpcds-tables-sf100.sql @@ -0,0 +1,751 @@ +-- 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. +drop table if exists customer_demographics; +CREATE TABLE IF NOT EXISTS customer_demographics ( + cd_demo_sk bigint not null, + cd_gender char(1), + cd_marital_status char(1), + cd_education_status char(20), + cd_purchase_estimate integer, + cd_credit_rating char(10), + cd_dep_count integer, + cd_dep_employed_count integer, + cd_dep_college_count integer +) +DUPLICATE KEY(cd_demo_sk) +DISTRIBUTED BY HASH(cd_gender) BUCKETS 12 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists reason; +CREATE TABLE IF NOT EXISTS reason ( + r_reason_sk bigint not null, + r_reason_id char(16) not null, + r_reason_desc char(100) + ) +DUPLICATE KEY(r_reason_sk) +DISTRIBUTED BY HASH(r_reason_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists date_dim; +CREATE TABLE IF NOT EXISTS date_dim ( + d_date_sk bigint not null, + d_date_id char(16) not null, + d_date datev2, + d_month_seq integer, + d_week_seq integer, + d_quarter_seq integer, + d_year integer, + d_dow integer, + d_moy integer, + d_dom integer, + d_qoy integer, + d_fy_year integer, + d_fy_quarter_seq integer, + d_fy_week_seq integer, + d_day_name char(9), + d_quarter_name char(6), + d_holiday char(1), + d_weekend char(1), + d_following_holiday char(1), + d_first_dom integer, + d_last_dom integer, + d_same_day_ly integer, + d_same_day_lq integer, + d_current_day char(1), + d_current_week char(1), + d_current_month char(1), + d_current_quarter char(1), + d_current_year char(1) +) +DUPLICATE KEY(d_date_sk) +DISTRIBUTED BY HASH(d_date_sk) BUCKETS 12 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists warehouse; +CREATE TABLE IF NOT EXISTS warehouse ( + w_warehouse_sk bigint not null, + w_warehouse_id char(16) not null, + w_warehouse_name varchar(20), + w_warehouse_sq_ft integer, + w_street_number char(10), + w_street_name varchar(60), + w_street_type char(15), + w_suite_number char(10), + w_city varchar(60), + w_county varchar(30), + w_state char(2), + w_zip char(10), + w_country varchar(20), + w_gmt_offset decimalv3(5,2) +) +DUPLICATE KEY(w_warehouse_sk) +DISTRIBUTED BY HASH(w_warehouse_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists catalog_sales; +CREATE TABLE IF NOT EXISTS catalog_sales ( + cs_item_sk bigint not null, + cs_order_number bigint not null, + cs_sold_date_sk bigint, + cs_sold_time_sk bigint, + cs_ship_date_sk bigint, + cs_bill_customer_sk bigint, + cs_bill_cdemo_sk bigint, + cs_bill_hdemo_sk bigint, + cs_bill_addr_sk bigint, + cs_ship_customer_sk bigint, + cs_ship_cdemo_sk bigint, + cs_ship_hdemo_sk bigint, + cs_ship_addr_sk bigint, + cs_call_center_sk bigint, + cs_catalog_page_sk bigint, + cs_ship_mode_sk bigint, + cs_warehouse_sk bigint, + cs_promo_sk bigint, + cs_quantity integer, + cs_wholesale_cost decimalv3(7,2), + cs_list_price decimalv3(7,2), + cs_sales_price decimalv3(7,2), + cs_ext_discount_amt decimalv3(7,2), + cs_ext_sales_price decimalv3(7,2), + cs_ext_wholesale_cost decimalv3(7,2), + cs_ext_list_price decimalv3(7,2), + cs_ext_tax decimalv3(7,2), + cs_coupon_amt decimalv3(7,2), + cs_ext_ship_cost decimalv3(7,2), + cs_net_paid decimalv3(7,2), + cs_net_paid_inc_tax decimalv3(7,2), + cs_net_paid_inc_ship decimalv3(7,2), + cs_net_paid_inc_ship_tax decimalv3(7,2), + cs_net_profit decimalv3(7,2) +) +DUPLICATE KEY(cs_item_sk, cs_order_number) +PARTITION BY RANGE(cs_sold_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450878"), +PARTITION `p2` VALUES LESS THAN ("2450939"), +PARTITION `p3` VALUES LESS THAN ("2451000"), +PARTITION `p4` VALUES LESS THAN ("2451061"), +PARTITION `p5` VALUES LESS THAN ("2451122"), +PARTITION `p6` VALUES LESS THAN ("2451183"), +PARTITION `p7` VALUES LESS THAN ("2451244"), +PARTITION `p8` VALUES LESS THAN ("2451305"), +PARTITION `p9` VALUES LESS THAN ("2451366"), +PARTITION `p10` VALUES LESS THAN ("2451427"), +PARTITION `p11` VALUES LESS THAN ("2451488"), +PARTITION `p12` VALUES LESS THAN ("2451549"), +PARTITION `p13` VALUES LESS THAN ("2451610"), +PARTITION `p14` VALUES LESS THAN ("2451671"), +PARTITION `p15` VALUES LESS THAN ("2451732"), +PARTITION `p16` VALUES LESS THAN ("2451793"), +PARTITION `p17` VALUES LESS THAN ("2451854"), +PARTITION `p18` VALUES LESS THAN ("2451915"), +PARTITION `p19` VALUES LESS THAN ("2451976"), +PARTITION `p20` VALUES LESS THAN ("2452037"), +PARTITION `p21` VALUES LESS THAN ("2452098"), +PARTITION `p22` VALUES LESS THAN ("2452159"), +PARTITION `p23` VALUES LESS THAN ("2452220"), +PARTITION `p24` VALUES LESS THAN ("2452281"), +PARTITION `p25` VALUES LESS THAN ("2452342"), +PARTITION `p26` VALUES LESS THAN ("2452403"), +PARTITION `p27` VALUES LESS THAN ("2452464"), +PARTITION `p28` VALUES LESS THAN ("2452525"), +PARTITION `p29` VALUES LESS THAN ("2452586"), +PARTITION `p30` VALUES LESS THAN ("2452657") +) +DISTRIBUTED BY HASH(cs_item_sk, cs_order_number) BUCKETS 32 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "catalog" +); +drop table if exists call_center; +CREATE TABLE IF NOT EXISTS call_center ( + cc_call_center_sk bigint not null, + cc_call_center_id char(16) not null, + cc_rec_start_date datev2, + cc_rec_end_date datev2, + cc_closed_date_sk integer, + cc_open_date_sk integer, + cc_name varchar(50), + cc_class varchar(50), + cc_employees integer, + cc_sq_ft integer, + cc_hours char(20), + cc_manager varchar(40), + cc_mkt_id integer, + cc_mkt_class char(50), + cc_mkt_desc varchar(100), + cc_market_manager varchar(40), + cc_division integer, + cc_division_name varchar(50), + cc_company integer, + cc_company_name char(50), + cc_street_number char(10), + cc_street_name varchar(60), + cc_street_type char(15), + cc_suite_number char(10), + cc_city varchar(60), + cc_county varchar(30), + cc_state char(2), + cc_zip char(10), + cc_country varchar(20), + cc_gmt_offset decimalv3(5,2), + cc_tax_percentage decimalv3(5,2) +) +DUPLICATE KEY(cc_call_center_sk) +DISTRIBUTED BY HASH(cc_call_center_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists inventory; +CREATE TABLE IF NOT EXISTS inventory ( + inv_date_sk bigint not null, + inv_item_sk bigint not null, + inv_warehouse_sk bigint, + inv_quantity_on_hand integer +) +DUPLICATE KEY(inv_date_sk, inv_item_sk, inv_warehouse_sk) +DISTRIBUTED BY HASH(inv_date_sk, inv_item_sk, inv_warehouse_sk) BUCKETS 32 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists catalog_returns; +CREATE TABLE IF NOT EXISTS catalog_returns ( + cr_item_sk bigint not null, + cr_order_number bigint not null, + cr_returned_date_sk bigint, + cr_returned_time_sk bigint, + cr_refunded_customer_sk bigint, + cr_refunded_cdemo_sk bigint, + cr_refunded_hdemo_sk bigint, + cr_refunded_addr_sk bigint, + cr_returning_customer_sk bigint, + cr_returning_cdemo_sk bigint, + cr_returning_hdemo_sk bigint, + cr_returning_addr_sk bigint, + cr_call_center_sk bigint, + cr_catalog_page_sk bigint, + cr_ship_mode_sk bigint, + cr_warehouse_sk bigint, + cr_reason_sk bigint, + cr_return_quantity integer, + cr_return_amount decimalv3(7,2), + cr_return_tax decimalv3(7,2), + cr_return_amt_inc_tax decimalv3(7,2), + cr_fee decimalv3(7,2), + cr_return_ship_cost decimalv3(7,2), + cr_refunded_cash decimalv3(7,2), + cr_reversed_charge decimalv3(7,2), + cr_store_credit decimalv3(7,2), + cr_net_loss decimalv3(7,2) +) +DUPLICATE KEY(cr_item_sk, cr_order_number) +DISTRIBUTED BY HASH(cr_item_sk, cr_order_number) BUCKETS 32 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "catalog" +); +drop table if exists household_demographics; +CREATE TABLE IF NOT EXISTS household_demographics ( + hd_demo_sk bigint not null, + hd_income_band_sk bigint, + hd_buy_potential char(15), + hd_dep_count integer, + hd_vehicle_count integer +) +DUPLICATE KEY(hd_demo_sk) +DISTRIBUTED BY HASH(hd_demo_sk) BUCKETS 3 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists customer_address; +CREATE TABLE IF NOT EXISTS customer_address ( + ca_address_sk bigint not null, + ca_address_id char(16) not null, + ca_street_number char(10), + ca_street_name varchar(60), + ca_street_type char(15), + ca_suite_number char(10), + ca_city varchar(60), + ca_county varchar(30), + ca_state char(2), + ca_zip char(10), + ca_country varchar(20), + ca_gmt_offset decimalv3(5,2), + ca_location_type char(20) +) +DUPLICATE KEY(ca_address_sk) +DISTRIBUTED BY HASH(ca_address_sk) BUCKETS 12 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists income_band; +CREATE TABLE IF NOT EXISTS income_band ( + ib_income_band_sk bigint not null, + ib_lower_bound integer, + ib_upper_bound integer +) +DUPLICATE KEY(ib_income_band_sk) +DISTRIBUTED BY HASH(ib_income_band_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists catalog_page; +CREATE TABLE IF NOT EXISTS catalog_page ( + cp_catalog_page_sk bigint not null, + cp_catalog_page_id char(16) not null, + cp_start_date_sk integer, + cp_end_date_sk integer, + cp_department varchar(50), + cp_catalog_number integer, + cp_catalog_page_number integer, + cp_description varchar(100), + cp_type varchar(100) +) +DUPLICATE KEY(cp_catalog_page_sk) +DISTRIBUTED BY HASH(cp_catalog_page_sk) BUCKETS 3 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists item; +CREATE TABLE IF NOT EXISTS item ( + i_item_sk bigint not null, + i_item_id char(16) not null, + i_rec_start_date datev2, + i_rec_end_date datev2, + i_item_desc varchar(200), + i_current_price decimalv3(7,2), + i_wholesale_cost decimalv3(7,2), + i_brand_id integer, + i_brand char(50), + i_class_id integer, + i_class char(50), + i_category_id integer, + i_category char(50), + i_manufact_id integer, + i_manufact char(50), + i_size char(20), + i_formulation char(20), + i_color char(20), + i_units char(10), + i_container char(10), + i_manager_id integer, + i_product_name char(50) +) +DUPLICATE KEY(i_item_sk) +DISTRIBUTED BY HASH(i_item_sk) BUCKETS 12 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists web_returns; +CREATE TABLE IF NOT EXISTS web_returns ( + wr_item_sk bigint not null, + wr_order_number bigint not null, + wr_returned_date_sk bigint, + wr_returned_time_sk bigint, + wr_refunded_customer_sk bigint, + wr_refunded_cdemo_sk bigint, + wr_refunded_hdemo_sk bigint, + wr_refunded_addr_sk bigint, + wr_returning_customer_sk bigint, + wr_returning_cdemo_sk bigint, + wr_returning_hdemo_sk bigint, + wr_returning_addr_sk bigint, + wr_web_page_sk bigint, + wr_reason_sk bigint, + wr_return_quantity integer, + wr_return_amt decimalv3(7,2), + wr_return_tax decimalv3(7,2), + wr_return_amt_inc_tax decimalv3(7,2), + wr_fee decimalv3(7,2), + wr_return_ship_cost decimalv3(7,2), + wr_refunded_cash decimalv3(7,2), + wr_reversed_charge decimalv3(7,2), + wr_account_credit decimalv3(7,2), + wr_net_loss decimalv3(7,2) +) +DUPLICATE KEY(wr_item_sk, wr_order_number) +DISTRIBUTED BY HASH(wr_item_sk, wr_order_number) BUCKETS 32 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "web" +); +drop table if exists web_site; +CREATE TABLE IF NOT EXISTS web_site ( + web_site_sk bigint not null, + web_site_id char(16) not null, + web_rec_start_date datev2, + web_rec_end_date datev2, + web_name varchar(50), + web_open_date_sk bigint, + web_close_date_sk bigint, + web_class varchar(50), + web_manager varchar(40), + web_mkt_id integer, + web_mkt_class varchar(50), + web_mkt_desc varchar(100), + web_market_manager varchar(40), + web_company_id integer, + web_company_name char(50), + web_street_number char(10), + web_street_name varchar(60), + web_street_type char(15), + web_suite_number char(10), + web_city varchar(60), + web_county varchar(30), + web_state char(2), + web_zip char(10), + web_country varchar(20), + web_gmt_offset decimalv3(5,2), + web_tax_percentage decimalv3(5,2) +) +DUPLICATE KEY(web_site_sk) +DISTRIBUTED BY HASH(web_site_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists promotion; +CREATE TABLE IF NOT EXISTS promotion ( + p_promo_sk bigint not null, + p_promo_id char(16) not null, + p_start_date_sk bigint, + p_end_date_sk bigint, + p_item_sk bigint, + p_cost decimalv3(15,2), + p_response_targe integer, + p_promo_name char(50), + p_channel_dmail char(1), + p_channel_email char(1), + p_channel_catalog char(1), + p_channel_tv char(1), + p_channel_radio char(1), + p_channel_press char(1), + p_channel_event char(1), + p_channel_demo char(1), + p_channel_details varchar(100), + p_purpose char(15), + p_discount_active char(1) +) +DUPLICATE KEY(p_promo_sk) +DISTRIBUTED BY HASH(p_promo_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists web_sales; +CREATE TABLE IF NOT EXISTS web_sales ( + ws_item_sk bigint not null, + ws_order_number bigint not null, + ws_sold_date_sk bigint, + ws_sold_time_sk bigint, + ws_ship_date_sk bigint, + ws_bill_customer_sk bigint, + ws_bill_cdemo_sk bigint, + ws_bill_hdemo_sk bigint, + ws_bill_addr_sk bigint, + ws_ship_customer_sk bigint, + ws_ship_cdemo_sk bigint, + ws_ship_hdemo_sk bigint, + ws_ship_addr_sk bigint, + ws_web_page_sk bigint, + ws_web_site_sk bigint, + ws_ship_mode_sk bigint, + ws_warehouse_sk bigint, + ws_promo_sk bigint, + ws_quantity integer, + ws_wholesale_cost decimalv3(7,2), + ws_list_price decimalv3(7,2), + ws_sales_price decimalv3(7,2), + ws_ext_discount_amt decimalv3(7,2), + ws_ext_sales_price decimalv3(7,2), + ws_ext_wholesale_cost decimalv3(7,2), + ws_ext_list_price decimalv3(7,2), + ws_ext_tax decimalv3(7,2), + ws_coupon_amt decimalv3(7,2), + ws_ext_ship_cost decimalv3(7,2), + ws_net_paid decimalv3(7,2), + ws_net_paid_inc_tax decimalv3(7,2), + ws_net_paid_inc_ship decimalv3(7,2), + ws_net_paid_inc_ship_tax decimalv3(7,2), + ws_net_profit decimalv3(7,2) +) +DUPLICATE KEY(ws_item_sk, ws_order_number) +PARTITION BY RANGE(ws_sold_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450878"), +PARTITION `p2` VALUES LESS THAN ("2450939"), +PARTITION `p3` VALUES LESS THAN ("2451000"), +PARTITION `p4` VALUES LESS THAN ("2451061"), +PARTITION `p5` VALUES LESS THAN ("2451122"), +PARTITION `p6` VALUES LESS THAN ("2451183"), +PARTITION `p7` VALUES LESS THAN ("2451244"), +PARTITION `p8` VALUES LESS THAN ("2451305"), +PARTITION `p9` VALUES LESS THAN ("2451366"), +PARTITION `p10` VALUES LESS THAN ("2451427"), +PARTITION `p11` VALUES LESS THAN ("2451488"), +PARTITION `p12` VALUES LESS THAN ("2451549"), +PARTITION `p13` VALUES LESS THAN ("2451610"), +PARTITION `p14` VALUES LESS THAN ("2451671"), +PARTITION `p15` VALUES LESS THAN ("2451732"), +PARTITION `p16` VALUES LESS THAN ("2451793"), +PARTITION `p17` VALUES LESS THAN ("2451854"), +PARTITION `p18` VALUES LESS THAN ("2451915"), +PARTITION `p19` VALUES LESS THAN ("2451976"), +PARTITION `p20` VALUES LESS THAN ("2452037"), +PARTITION `p21` VALUES LESS THAN ("2452098"), +PARTITION `p22` VALUES LESS THAN ("2452159"), +PARTITION `p23` VALUES LESS THAN ("2452220"), +PARTITION `p24` VALUES LESS THAN ("2452281"), +PARTITION `p25` VALUES LESS THAN ("2452342"), +PARTITION `p26` VALUES LESS THAN ("2452403"), +PARTITION `p27` VALUES LESS THAN ("2452464"), +PARTITION `p28` VALUES LESS THAN ("2452525"), +PARTITION `p29` VALUES LESS THAN ("2452586"), +PARTITION `p30` VALUES LESS THAN ("2452657") +) +DISTRIBUTED BY HASH(ws_item_sk, ws_order_number) BUCKETS 32 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "web" +); +drop table if exists store; +CREATE TABLE IF NOT EXISTS store ( + s_store_sk bigint not null, + s_store_id char(16) not null, + s_rec_start_date datev2, + s_rec_end_date datev2, + s_closed_date_sk bigint, + s_store_name varchar(50), + s_number_employees integer, + s_floor_space integer, + s_hours char(20), + s_manager varchar(40), + s_market_id integer, + s_geography_class varchar(100), + s_market_desc varchar(100), + s_market_manager varchar(40), + s_division_id integer, + s_division_name varchar(50), + s_company_id integer, + s_company_name varchar(50), + s_street_number varchar(10), + s_street_name varchar(60), + s_street_type char(15), + s_suite_number char(10), + s_city varchar(60), + s_county varchar(30), + s_state char(2), + s_zip char(10), + s_country varchar(20), + s_gmt_offset decimalv3(5,2), + s_tax_precentage decimalv3(5,2) +) +DUPLICATE KEY(s_store_sk) +DISTRIBUTED BY HASH(s_store_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists time_dim; +CREATE TABLE IF NOT EXISTS time_dim ( + t_time_sk bigint not null, + t_time_id char(16) not null, + t_time integer, + t_hour integer, + t_minute integer, + t_second integer, + t_am_pm char(2), + t_shift char(20), + t_sub_shift char(20), + t_meal_time char(20) +) +DUPLICATE KEY(t_time_sk) +DISTRIBUTED BY HASH(t_time_sk) BUCKETS 12 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists web_page; +CREATE TABLE IF NOT EXISTS web_page ( + wp_web_page_sk bigint not null, + wp_web_page_id char(16) not null, + wp_rec_start_date datev2, + wp_rec_end_date datev2, + wp_creation_date_sk bigint, + wp_access_date_sk bigint, + wp_autogen_flag char(1), + wp_customer_sk bigint, + wp_url varchar(100), + wp_type char(50), + wp_char_count integer, + wp_link_count integer, + wp_image_count integer, + wp_max_ad_count integer +) +DUPLICATE KEY(wp_web_page_sk) +DISTRIBUTED BY HASH(wp_web_page_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists store_returns; +CREATE TABLE IF NOT EXISTS store_returns ( + sr_item_sk bigint not null, + sr_ticket_number bigint not null, + sr_returned_date_sk bigint, + sr_return_time_sk bigint, + sr_customer_sk bigint, + sr_cdemo_sk bigint, + sr_hdemo_sk bigint, + sr_addr_sk bigint, + sr_store_sk bigint, + sr_reason_sk bigint, + sr_return_quantity integer, + sr_return_amt decimalv3(7,2), + sr_return_tax decimalv3(7,2), + sr_return_amt_inc_tax decimalv3(7,2), + sr_fee decimalv3(7,2), + sr_return_ship_cost decimalv3(7,2), + sr_refunded_cash decimalv3(7,2), + sr_reversed_charge decimalv3(7,2), + sr_store_credit decimalv3(7,2), + sr_net_loss decimalv3(7,2) +) +duplicate key(sr_item_sk, sr_ticket_number) +distributed by hash (sr_item_sk, sr_ticket_number) buckets 32 +properties ( + "replication_num" = "1", + "colocate_with" = "store" +); +drop table if exists store_sales; +CREATE TABLE IF NOT EXISTS store_sales ( + ss_item_sk bigint not null, + ss_ticket_number bigint not null, + ss_sold_date_sk bigint, + ss_sold_time_sk bigint, + ss_customer_sk bigint, + ss_cdemo_sk bigint, + ss_hdemo_sk bigint, + ss_addr_sk bigint, + ss_store_sk bigint, + ss_promo_sk bigint, + ss_quantity integer, + ss_wholesale_cost decimalv3(7,2), + ss_list_price decimalv3(7,2), + ss_sales_price decimalv3(7,2), + ss_ext_discount_amt decimalv3(7,2), + ss_ext_sales_price decimalv3(7,2), + ss_ext_wholesale_cost decimalv3(7,2), + ss_ext_list_price decimalv3(7,2), + ss_ext_tax decimalv3(7,2), + ss_coupon_amt decimalv3(7,2), + ss_net_paid decimalv3(7,2), + ss_net_paid_inc_tax decimalv3(7,2), + ss_net_profit decimalv3(7,2) +) +DUPLICATE KEY(ss_item_sk, ss_ticket_number) +PARTITION BY RANGE(ss_sold_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450878"), +PARTITION `p2` VALUES LESS THAN ("2450939"), +PARTITION `p3` VALUES LESS THAN ("2451000"), +PARTITION `p4` VALUES LESS THAN ("2451061"), +PARTITION `p5` VALUES LESS THAN ("2451122"), +PARTITION `p6` VALUES LESS THAN ("2451183"), +PARTITION `p7` VALUES LESS THAN ("2451244"), +PARTITION `p8` VALUES LESS THAN ("2451305"), +PARTITION `p9` VALUES LESS THAN ("2451366"), +PARTITION `p10` VALUES LESS THAN ("2451427"), +PARTITION `p11` VALUES LESS THAN ("2451488"), +PARTITION `p12` VALUES LESS THAN ("2451549"), +PARTITION `p13` VALUES LESS THAN ("2451610"), +PARTITION `p14` VALUES LESS THAN ("2451671"), +PARTITION `p15` VALUES LESS THAN ("2451732"), +PARTITION `p16` VALUES LESS THAN ("2451793"), +PARTITION `p17` VALUES LESS THAN ("2451854"), +PARTITION `p18` VALUES LESS THAN ("2451915"), +PARTITION `p19` VALUES LESS THAN ("2451976"), +PARTITION `p20` VALUES LESS THAN ("2452037"), +PARTITION `p21` VALUES LESS THAN ("2452098"), +PARTITION `p22` VALUES LESS THAN ("2452159"), +PARTITION `p23` VALUES LESS THAN ("2452220"), +PARTITION `p24` VALUES LESS THAN ("2452281"), +PARTITION `p25` VALUES LESS THAN ("2452342"), +PARTITION `p26` VALUES LESS THAN ("2452403"), +PARTITION `p27` VALUES LESS THAN ("2452464"), +PARTITION `p28` VALUES LESS THAN ("2452525"), +PARTITION `p29` VALUES LESS THAN ("2452586"), +PARTITION `p30` VALUES LESS THAN ("2452657") +) +DISTRIBUTED BY HASH(ss_item_sk, ss_ticket_number) BUCKETS 32 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "store" +); +drop table if exists ship_mode; +CREATE TABLE IF NOT EXISTS ship_mode ( + sm_ship_mode_sk bigint not null, + sm_ship_mode_id char(16) not null, + sm_type char(30), + sm_code char(10), + sm_carrier char(20), + sm_contract char(20) +) +DUPLICATE KEY(sm_ship_mode_sk) +DISTRIBUTED BY HASH(sm_ship_mode_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists customer; +CREATE TABLE IF NOT EXISTS customer ( + c_customer_sk bigint not null, + c_customer_id char(16) not null, + c_current_cdemo_sk bigint, + c_current_hdemo_sk bigint, + c_current_addr_sk bigint, + c_first_shipto_date_sk bigint, + c_first_sales_date_sk bigint, + c_salutation char(10), + c_first_name char(20), + c_last_name char(30), + c_preferred_cust_flag char(1), + c_birth_day integer, + c_birth_month integer, + c_birth_year integer, + c_birth_country varchar(20), + c_login char(13), + c_email_address char(50), + c_last_review_date_sk bigint +) +DUPLICATE KEY(c_customer_sk) +DISTRIBUTED BY HASH(c_customer_id) BUCKETS 12 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists dbgen_version; +CREATE TABLE IF NOT EXISTS dbgen_version +( + dv_version varchar(16) , + dv_create_date datev2 , + dv_create_time datetime , + dv_cmdline_args varchar(200) +) +DUPLICATE KEY(dv_version) +DISTRIBUTED BY HASH(dv_version) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); diff --git a/tools/tpcds-tools/ddl/create-tpcds-tables-sf1000.sql b/tools/tpcds-tools/ddl/create-tpcds-tables-sf1000.sql new file mode 100644 index 000000000000000..fa7d1f8768a8fc6 --- /dev/null +++ b/tools/tpcds-tools/ddl/create-tpcds-tables-sf1000.sql @@ -0,0 +1,1177 @@ +-- 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. +drop table if exists customer_demographics; +CREATE TABLE IF NOT EXISTS customer_demographics ( + cd_demo_sk bigint not null, + cd_gender char(1), + cd_marital_status char(1), + cd_education_status char(20), + cd_purchase_estimate integer, + cd_credit_rating char(10), + cd_dep_count integer, + cd_dep_employed_count integer, + cd_dep_college_count integer +) +DUPLICATE KEY(cd_demo_sk) +DISTRIBUTED BY HASH(cd_gender) BUCKETS 12 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists reason; +CREATE TABLE IF NOT EXISTS reason ( + r_reason_sk bigint not null, + r_reason_id char(16) not null, + r_reason_desc char(100) + ) +DUPLICATE KEY(r_reason_sk) +DISTRIBUTED BY HASH(r_reason_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists date_dim; +CREATE TABLE IF NOT EXISTS date_dim ( + d_date_sk bigint not null, + d_date_id char(16) not null, + d_date datev2, + d_month_seq integer, + d_week_seq integer, + d_quarter_seq integer, + d_year integer, + d_dow integer, + d_moy integer, + d_dom integer, + d_qoy integer, + d_fy_year integer, + d_fy_quarter_seq integer, + d_fy_week_seq integer, + d_day_name char(9), + d_quarter_name char(6), + d_holiday char(1), + d_weekend char(1), + d_following_holiday char(1), + d_first_dom integer, + d_last_dom integer, + d_same_day_ly integer, + d_same_day_lq integer, + d_current_day char(1), + d_current_week char(1), + d_current_month char(1), + d_current_quarter char(1), + d_current_year char(1) +) +DUPLICATE KEY(d_date_sk) +DISTRIBUTED BY HASH(d_date_sk) BUCKETS 12 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists warehouse; +CREATE TABLE IF NOT EXISTS warehouse ( + w_warehouse_sk bigint not null, + w_warehouse_id char(16) not null, + w_warehouse_name varchar(20), + w_warehouse_sq_ft integer, + w_street_number char(10), + w_street_name varchar(60), + w_street_type char(15), + w_suite_number char(10), + w_city varchar(60), + w_county varchar(30), + w_state char(2), + w_zip char(10), + w_country varchar(20), + w_gmt_offset decimalv3(5,2) +) +DUPLICATE KEY(w_warehouse_sk) +DISTRIBUTED BY HASH(w_warehouse_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists catalog_sales; +CREATE TABLE IF NOT EXISTS catalog_sales ( + cs_item_sk bigint not null, + cs_order_number bigint not null, + cs_sold_date_sk bigint, + cs_sold_time_sk bigint, + cs_ship_date_sk bigint, + cs_bill_customer_sk bigint, + cs_bill_cdemo_sk bigint, + cs_bill_hdemo_sk bigint, + cs_bill_addr_sk bigint, + cs_ship_customer_sk bigint, + cs_ship_cdemo_sk bigint, + cs_ship_hdemo_sk bigint, + cs_ship_addr_sk bigint, + cs_call_center_sk bigint, + cs_catalog_page_sk bigint, + cs_ship_mode_sk bigint, + cs_warehouse_sk bigint, + cs_promo_sk bigint, + cs_quantity integer, + cs_wholesale_cost decimalv3(7,2), + cs_list_price decimalv3(7,2), + cs_sales_price decimalv3(7,2), + cs_ext_discount_amt decimalv3(7,2), + cs_ext_sales_price decimalv3(7,2), + cs_ext_wholesale_cost decimalv3(7,2), + cs_ext_list_price decimalv3(7,2), + cs_ext_tax decimalv3(7,2), + cs_coupon_amt decimalv3(7,2), + cs_ext_ship_cost decimalv3(7,2), + cs_net_paid decimalv3(7,2), + cs_net_paid_inc_tax decimalv3(7,2), + cs_net_paid_inc_ship decimalv3(7,2), + cs_net_paid_inc_ship_tax decimalv3(7,2), + cs_net_profit decimalv3(7,2) +) +DUPLICATE KEY(cs_item_sk, cs_order_number) +PARTITION BY RANGE(cs_sold_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(cs_item_sk, cs_order_number) BUCKETS 64 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "catalog" +); +drop table if exists call_center; +CREATE TABLE IF NOT EXISTS call_center ( + cc_call_center_sk bigint not null, + cc_call_center_id char(16) not null, + cc_rec_start_date datev2, + cc_rec_end_date datev2, + cc_closed_date_sk integer, + cc_open_date_sk integer, + cc_name varchar(50), + cc_class varchar(50), + cc_employees integer, + cc_sq_ft integer, + cc_hours char(20), + cc_manager varchar(40), + cc_mkt_id integer, + cc_mkt_class char(50), + cc_mkt_desc varchar(100), + cc_market_manager varchar(40), + cc_division integer, + cc_division_name varchar(50), + cc_company integer, + cc_company_name char(50), + cc_street_number char(10), + cc_street_name varchar(60), + cc_street_type char(15), + cc_suite_number char(10), + cc_city varchar(60), + cc_county varchar(30), + cc_state char(2), + cc_zip char(10), + cc_country varchar(20), + cc_gmt_offset decimalv3(5,2), + cc_tax_percentage decimalv3(5,2) +) +DUPLICATE KEY(cc_call_center_sk) +DISTRIBUTED BY HASH(cc_call_center_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists inventory; +CREATE TABLE IF NOT EXISTS inventory ( + inv_date_sk bigint not null, + inv_item_sk bigint not null, + inv_warehouse_sk bigint, + inv_quantity_on_hand integer +) +DUPLICATE KEY(inv_date_sk, inv_item_sk, inv_warehouse_sk) +PARTITION BY RANGE(inv_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(inv_item_sk, inv_warehouse_sk) BUCKETS 64 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists catalog_returns; +CREATE TABLE IF NOT EXISTS catalog_returns ( + cr_item_sk bigint not null, + cr_order_number bigint not null, + cr_returned_date_sk bigint, + cr_returned_time_sk bigint, + cr_refunded_customer_sk bigint, + cr_refunded_cdemo_sk bigint, + cr_refunded_hdemo_sk bigint, + cr_refunded_addr_sk bigint, + cr_returning_customer_sk bigint, + cr_returning_cdemo_sk bigint, + cr_returning_hdemo_sk bigint, + cr_returning_addr_sk bigint, + cr_call_center_sk bigint, + cr_catalog_page_sk bigint, + cr_ship_mode_sk bigint, + cr_warehouse_sk bigint, + cr_reason_sk bigint, + cr_return_quantity integer, + cr_return_amount decimalv3(7,2), + cr_return_tax decimalv3(7,2), + cr_return_amt_inc_tax decimalv3(7,2), + cr_fee decimalv3(7,2), + cr_return_ship_cost decimalv3(7,2), + cr_refunded_cash decimalv3(7,2), + cr_reversed_charge decimalv3(7,2), + cr_store_credit decimalv3(7,2), + cr_net_loss decimalv3(7,2) +) +DUPLICATE KEY(cr_item_sk, cr_order_number) +PARTITION BY RANGE(cr_returned_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(cr_item_sk, cr_order_number) BUCKETS 64 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "catalog" +); +drop table if exists household_demographics; +CREATE TABLE IF NOT EXISTS household_demographics ( + hd_demo_sk bigint not null, + hd_income_band_sk bigint, + hd_buy_potential char(15), + hd_dep_count integer, + hd_vehicle_count integer +) +DUPLICATE KEY(hd_demo_sk) +DISTRIBUTED BY HASH(hd_demo_sk) BUCKETS 3 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists customer_address; +CREATE TABLE IF NOT EXISTS customer_address ( + ca_address_sk bigint not null, + ca_address_id char(16) not null, + ca_street_number char(10), + ca_street_name varchar(60), + ca_street_type char(15), + ca_suite_number char(10), + ca_city varchar(60), + ca_county varchar(30), + ca_state char(2), + ca_zip char(10), + ca_country varchar(20), + ca_gmt_offset decimalv3(5,2), + ca_location_type char(20) +) +DUPLICATE KEY(ca_address_sk) +DISTRIBUTED BY HASH(ca_address_sk) BUCKETS 64 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists income_band; +CREATE TABLE IF NOT EXISTS income_band ( + ib_income_band_sk bigint not null, + ib_lower_bound integer, + ib_upper_bound integer +) +DUPLICATE KEY(ib_income_band_sk) +DISTRIBUTED BY HASH(ib_income_band_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists catalog_page; +CREATE TABLE IF NOT EXISTS catalog_page ( + cp_catalog_page_sk bigint not null, + cp_catalog_page_id char(16) not null, + cp_start_date_sk integer, + cp_end_date_sk integer, + cp_department varchar(50), + cp_catalog_number integer, + cp_catalog_page_number integer, + cp_description varchar(100), + cp_type varchar(100) +) +DUPLICATE KEY(cp_catalog_page_sk) +DISTRIBUTED BY HASH(cp_catalog_page_sk) BUCKETS 3 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists item; +CREATE TABLE IF NOT EXISTS item ( + i_item_sk bigint not null, + i_item_id char(16) not null, + i_rec_start_date datev2, + i_rec_end_date datev2, + i_item_desc varchar(200), + i_current_price decimalv3(7,2), + i_wholesale_cost decimalv3(7,2), + i_brand_id integer, + i_brand char(50), + i_class_id integer, + i_class char(50), + i_category_id integer, + i_category char(50), + i_manufact_id integer, + i_manufact char(50), + i_size char(20), + i_formulation char(20), + i_color char(20), + i_units char(10), + i_container char(10), + i_manager_id integer, + i_product_name char(50) +) +DUPLICATE KEY(i_item_sk) +DISTRIBUTED BY HASH(i_item_sk) BUCKETS 24 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists web_returns; +CREATE TABLE IF NOT EXISTS web_returns ( + wr_item_sk bigint not null, + wr_order_number bigint not null, + wr_returned_date_sk bigint, + wr_returned_time_sk bigint, + wr_refunded_customer_sk bigint, + wr_refunded_cdemo_sk bigint, + wr_refunded_hdemo_sk bigint, + wr_refunded_addr_sk bigint, + wr_returning_customer_sk bigint, + wr_returning_cdemo_sk bigint, + wr_returning_hdemo_sk bigint, + wr_returning_addr_sk bigint, + wr_web_page_sk bigint, + wr_reason_sk bigint, + wr_return_quantity integer, + wr_return_amt decimalv3(7,2), + wr_return_tax decimalv3(7,2), + wr_return_amt_inc_tax decimalv3(7,2), + wr_fee decimalv3(7,2), + wr_return_ship_cost decimalv3(7,2), + wr_refunded_cash decimalv3(7,2), + wr_reversed_charge decimalv3(7,2), + wr_account_credit decimalv3(7,2), + wr_net_loss decimalv3(7,2) +) +DUPLICATE KEY(wr_item_sk, wr_order_number) +PARTITION BY RANGE(wr_returned_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(wr_item_sk, wr_order_number) BUCKETS 64 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "web" +); +drop table if exists web_site; +CREATE TABLE IF NOT EXISTS web_site ( + web_site_sk bigint not null, + web_site_id char(16) not null, + web_rec_start_date datev2, + web_rec_end_date datev2, + web_name varchar(50), + web_open_date_sk bigint, + web_close_date_sk bigint, + web_class varchar(50), + web_manager varchar(40), + web_mkt_id integer, + web_mkt_class varchar(50), + web_mkt_desc varchar(100), + web_market_manager varchar(40), + web_company_id integer, + web_company_name char(50), + web_street_number char(10), + web_street_name varchar(60), + web_street_type char(15), + web_suite_number char(10), + web_city varchar(60), + web_county varchar(30), + web_state char(2), + web_zip char(10), + web_country varchar(20), + web_gmt_offset decimalv3(5,2), + web_tax_percentage decimalv3(5,2) +) +DUPLICATE KEY(web_site_sk) +DISTRIBUTED BY HASH(web_site_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists promotion; +CREATE TABLE IF NOT EXISTS promotion ( + p_promo_sk bigint not null, + p_promo_id char(16) not null, + p_start_date_sk bigint, + p_end_date_sk bigint, + p_item_sk bigint, + p_cost decimalv3(15,2), + p_response_targe integer, + p_promo_name char(50), + p_channel_dmail char(1), + p_channel_email char(1), + p_channel_catalog char(1), + p_channel_tv char(1), + p_channel_radio char(1), + p_channel_press char(1), + p_channel_event char(1), + p_channel_demo char(1), + p_channel_details varchar(100), + p_purpose char(15), + p_discount_active char(1) +) +DUPLICATE KEY(p_promo_sk) +DISTRIBUTED BY HASH(p_promo_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists web_sales; +CREATE TABLE IF NOT EXISTS web_sales ( + ws_item_sk bigint not null, + ws_order_number bigint not null, + ws_sold_date_sk bigint, + ws_sold_time_sk bigint, + ws_ship_date_sk bigint, + ws_bill_customer_sk bigint, + ws_bill_cdemo_sk bigint, + ws_bill_hdemo_sk bigint, + ws_bill_addr_sk bigint, + ws_ship_customer_sk bigint, + ws_ship_cdemo_sk bigint, + ws_ship_hdemo_sk bigint, + ws_ship_addr_sk bigint, + ws_web_page_sk bigint, + ws_web_site_sk bigint, + ws_ship_mode_sk bigint, + ws_warehouse_sk bigint, + ws_promo_sk bigint, + ws_quantity integer, + ws_wholesale_cost decimalv3(7,2), + ws_list_price decimalv3(7,2), + ws_sales_price decimalv3(7,2), + ws_ext_discount_amt decimalv3(7,2), + ws_ext_sales_price decimalv3(7,2), + ws_ext_wholesale_cost decimalv3(7,2), + ws_ext_list_price decimalv3(7,2), + ws_ext_tax decimalv3(7,2), + ws_coupon_amt decimalv3(7,2), + ws_ext_ship_cost decimalv3(7,2), + ws_net_paid decimalv3(7,2), + ws_net_paid_inc_tax decimalv3(7,2), + ws_net_paid_inc_ship decimalv3(7,2), + ws_net_paid_inc_ship_tax decimalv3(7,2), + ws_net_profit decimalv3(7,2) +) +DUPLICATE KEY(ws_item_sk, ws_order_number) +PARTITION BY RANGE(ws_sold_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(ws_item_sk, ws_order_number) BUCKETS 64 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "web" +); +drop table if exists store; +CREATE TABLE IF NOT EXISTS store ( + s_store_sk bigint not null, + s_store_id char(16) not null, + s_rec_start_date datev2, + s_rec_end_date datev2, + s_closed_date_sk bigint, + s_store_name varchar(50), + s_number_employees integer, + s_floor_space integer, + s_hours char(20), + s_manager varchar(40), + s_market_id integer, + s_geography_class varchar(100), + s_market_desc varchar(100), + s_market_manager varchar(40), + s_division_id integer, + s_division_name varchar(50), + s_company_id integer, + s_company_name varchar(50), + s_street_number varchar(10), + s_street_name varchar(60), + s_street_type char(15), + s_suite_number char(10), + s_city varchar(60), + s_county varchar(30), + s_state char(2), + s_zip char(10), + s_country varchar(20), + s_gmt_offset decimalv3(5,2), + s_tax_precentage decimalv3(5,2) +) +DUPLICATE KEY(s_store_sk) +DISTRIBUTED BY HASH(s_store_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists time_dim; +CREATE TABLE IF NOT EXISTS time_dim ( + t_time_sk bigint not null, + t_time_id char(16) not null, + t_time integer, + t_hour integer, + t_minute integer, + t_second integer, + t_am_pm char(2), + t_shift char(20), + t_sub_shift char(20), + t_meal_time char(20) +) +DUPLICATE KEY(t_time_sk) +DISTRIBUTED BY HASH(t_time_sk) BUCKETS 12 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists web_page; +CREATE TABLE IF NOT EXISTS web_page ( + wp_web_page_sk bigint not null, + wp_web_page_id char(16) not null, + wp_rec_start_date datev2, + wp_rec_end_date datev2, + wp_creation_date_sk bigint, + wp_access_date_sk bigint, + wp_autogen_flag char(1), + wp_customer_sk bigint, + wp_url varchar(100), + wp_type char(50), + wp_char_count integer, + wp_link_count integer, + wp_image_count integer, + wp_max_ad_count integer +) +DUPLICATE KEY(wp_web_page_sk) +DISTRIBUTED BY HASH(wp_web_page_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists store_returns; +CREATE TABLE IF NOT EXISTS store_returns ( + sr_item_sk bigint not null, + sr_ticket_number bigint not null, + sr_returned_date_sk bigint, + sr_return_time_sk bigint, + sr_customer_sk bigint, + sr_cdemo_sk bigint, + sr_hdemo_sk bigint, + sr_addr_sk bigint, + sr_store_sk bigint, + sr_reason_sk bigint, + sr_return_quantity integer, + sr_return_amt decimalv3(7,2), + sr_return_tax decimalv3(7,2), + sr_return_amt_inc_tax decimalv3(7,2), + sr_fee decimalv3(7,2), + sr_return_ship_cost decimalv3(7,2), + sr_refunded_cash decimalv3(7,2), + sr_reversed_charge decimalv3(7,2), + sr_store_credit decimalv3(7,2), + sr_net_loss decimalv3(7,2) +) +DUPLICATE KEY(sr_item_sk, sr_ticket_number) +PARTITION BY RANGE(sr_returned_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(sr_item_sk, sr_ticket_number) BUCKETS 64 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "store" +); +drop table if exists store_sales; +CREATE TABLE IF NOT EXISTS store_sales ( + ss_item_sk bigint not null, + ss_ticket_number bigint not null, + ss_sold_date_sk bigint, + ss_sold_time_sk bigint, + ss_customer_sk bigint, + ss_cdemo_sk bigint, + ss_hdemo_sk bigint, + ss_addr_sk bigint, + ss_store_sk bigint, + ss_promo_sk bigint, + ss_quantity integer, + ss_wholesale_cost decimalv3(7,2), + ss_list_price decimalv3(7,2), + ss_sales_price decimalv3(7,2), + ss_ext_discount_amt decimalv3(7,2), + ss_ext_sales_price decimalv3(7,2), + ss_ext_wholesale_cost decimalv3(7,2), + ss_ext_list_price decimalv3(7,2), + ss_ext_tax decimalv3(7,2), + ss_coupon_amt decimalv3(7,2), + ss_net_paid decimalv3(7,2), + ss_net_paid_inc_tax decimalv3(7,2), + ss_net_profit decimalv3(7,2) +) +DUPLICATE KEY(ss_item_sk, ss_ticket_number) +PARTITION BY RANGE(ss_sold_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(ss_item_sk, ss_ticket_number) BUCKETS 64 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "store" +); +drop table if exists ship_mode; +CREATE TABLE IF NOT EXISTS ship_mode ( + sm_ship_mode_sk bigint not null, + sm_ship_mode_id char(16) not null, + sm_type char(30), + sm_code char(10), + sm_carrier char(20), + sm_contract char(20) +) +DUPLICATE KEY(sm_ship_mode_sk) +DISTRIBUTED BY HASH(sm_ship_mode_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists customer; +CREATE TABLE IF NOT EXISTS customer ( + c_customer_sk bigint not null, + c_customer_id char(16) not null, + c_current_cdemo_sk bigint, + c_current_hdemo_sk bigint, + c_current_addr_sk bigint, + c_first_shipto_date_sk bigint, + c_first_sales_date_sk bigint, + c_salutation char(10), + c_first_name char(20), + c_last_name char(30), + c_preferred_cust_flag char(1), + c_birth_day integer, + c_birth_month integer, + c_birth_year integer, + c_birth_country varchar(20), + c_login char(13), + c_email_address char(50), + c_last_review_date_sk bigint +) +DUPLICATE KEY(c_customer_sk) +DISTRIBUTED BY HASH(c_customer_id) BUCKETS 24 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists dbgen_version; +CREATE TABLE IF NOT EXISTS dbgen_version +( + dv_version varchar(16) , + dv_create_date datev2 , + dv_create_time datetime , + dv_cmdline_args varchar(200) +) +DUPLICATE KEY(dv_version) +DISTRIBUTED BY HASH(dv_version) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); diff --git a/tools/tpcds-tools/ddl/create-tpcds-tables-sf10000.sql b/tools/tpcds-tools/ddl/create-tpcds-tables-sf10000.sql new file mode 100644 index 000000000000000..760672922c4d48b --- /dev/null +++ b/tools/tpcds-tools/ddl/create-tpcds-tables-sf10000.sql @@ -0,0 +1,1177 @@ +-- 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. +drop table if exists customer_demographics; +CREATE TABLE IF NOT EXISTS customer_demographics ( + cd_demo_sk bigint not null, + cd_gender char(1), + cd_marital_status char(1), + cd_education_status char(20), + cd_purchase_estimate integer, + cd_credit_rating char(10), + cd_dep_count integer, + cd_dep_employed_count integer, + cd_dep_college_count integer +) +DUPLICATE KEY(cd_demo_sk) +DISTRIBUTED BY HASH(cd_gender) BUCKETS 12 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists reason; +CREATE TABLE IF NOT EXISTS reason ( + r_reason_sk bigint not null, + r_reason_id char(16) not null, + r_reason_desc char(100) + ) +DUPLICATE KEY(r_reason_sk) +DISTRIBUTED BY HASH(r_reason_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists date_dim; +CREATE TABLE IF NOT EXISTS date_dim ( + d_date_sk bigint not null, + d_date_id char(16) not null, + d_date datev2, + d_month_seq integer, + d_week_seq integer, + d_quarter_seq integer, + d_year integer, + d_dow integer, + d_moy integer, + d_dom integer, + d_qoy integer, + d_fy_year integer, + d_fy_quarter_seq integer, + d_fy_week_seq integer, + d_day_name char(9), + d_quarter_name char(6), + d_holiday char(1), + d_weekend char(1), + d_following_holiday char(1), + d_first_dom integer, + d_last_dom integer, + d_same_day_ly integer, + d_same_day_lq integer, + d_current_day char(1), + d_current_week char(1), + d_current_month char(1), + d_current_quarter char(1), + d_current_year char(1) +) +DUPLICATE KEY(d_date_sk) +DISTRIBUTED BY HASH(d_date_sk) BUCKETS 12 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists warehouse; +CREATE TABLE IF NOT EXISTS warehouse ( + w_warehouse_sk bigint not null, + w_warehouse_id char(16) not null, + w_warehouse_name varchar(20), + w_warehouse_sq_ft integer, + w_street_number char(10), + w_street_name varchar(60), + w_street_type char(15), + w_suite_number char(10), + w_city varchar(60), + w_county varchar(30), + w_state char(2), + w_zip char(10), + w_country varchar(20), + w_gmt_offset decimalv3(5,2) +) +DUPLICATE KEY(w_warehouse_sk) +DISTRIBUTED BY HASH(w_warehouse_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists catalog_sales; +CREATE TABLE IF NOT EXISTS catalog_sales ( + cs_item_sk bigint not null, + cs_order_number bigint not null, + cs_sold_date_sk bigint, + cs_sold_time_sk bigint, + cs_ship_date_sk bigint, + cs_bill_customer_sk bigint, + cs_bill_cdemo_sk bigint, + cs_bill_hdemo_sk bigint, + cs_bill_addr_sk bigint, + cs_ship_customer_sk bigint, + cs_ship_cdemo_sk bigint, + cs_ship_hdemo_sk bigint, + cs_ship_addr_sk bigint, + cs_call_center_sk bigint, + cs_catalog_page_sk bigint, + cs_ship_mode_sk bigint, + cs_warehouse_sk bigint, + cs_promo_sk bigint, + cs_quantity integer, + cs_wholesale_cost decimalv3(7,2), + cs_list_price decimalv3(7,2), + cs_sales_price decimalv3(7,2), + cs_ext_discount_amt decimalv3(7,2), + cs_ext_sales_price decimalv3(7,2), + cs_ext_wholesale_cost decimalv3(7,2), + cs_ext_list_price decimalv3(7,2), + cs_ext_tax decimalv3(7,2), + cs_coupon_amt decimalv3(7,2), + cs_ext_ship_cost decimalv3(7,2), + cs_net_paid decimalv3(7,2), + cs_net_paid_inc_tax decimalv3(7,2), + cs_net_paid_inc_ship decimalv3(7,2), + cs_net_paid_inc_ship_tax decimalv3(7,2), + cs_net_profit decimalv3(7,2) +) +DUPLICATE KEY(cs_item_sk, cs_order_number) +PARTITION BY RANGE(cs_sold_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(cs_item_sk, cs_order_number) BUCKETS 256 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "catalog" +); +drop table if exists call_center; +CREATE TABLE IF NOT EXISTS call_center ( + cc_call_center_sk bigint not null, + cc_call_center_id char(16) not null, + cc_rec_start_date datev2, + cc_rec_end_date datev2, + cc_closed_date_sk integer, + cc_open_date_sk integer, + cc_name varchar(50), + cc_class varchar(50), + cc_employees integer, + cc_sq_ft integer, + cc_hours char(20), + cc_manager varchar(40), + cc_mkt_id integer, + cc_mkt_class char(50), + cc_mkt_desc varchar(100), + cc_market_manager varchar(40), + cc_division integer, + cc_division_name varchar(50), + cc_company integer, + cc_company_name char(50), + cc_street_number char(10), + cc_street_name varchar(60), + cc_street_type char(15), + cc_suite_number char(10), + cc_city varchar(60), + cc_county varchar(30), + cc_state char(2), + cc_zip char(10), + cc_country varchar(20), + cc_gmt_offset decimalv3(5,2), + cc_tax_percentage decimalv3(5,2) +) +DUPLICATE KEY(cc_call_center_sk) +DISTRIBUTED BY HASH(cc_call_center_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists inventory; +CREATE TABLE IF NOT EXISTS inventory ( + inv_date_sk bigint not null, + inv_item_sk bigint not null, + inv_warehouse_sk bigint, + inv_quantity_on_hand integer +) +DUPLICATE KEY(inv_date_sk, inv_item_sk, inv_warehouse_sk) +PARTITION BY RANGE(inv_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(inv_item_sk, inv_warehouse_sk) BUCKETS 256 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists catalog_returns; +CREATE TABLE IF NOT EXISTS catalog_returns ( + cr_item_sk bigint not null, + cr_order_number bigint not null, + cr_returned_date_sk bigint, + cr_returned_time_sk bigint, + cr_refunded_customer_sk bigint, + cr_refunded_cdemo_sk bigint, + cr_refunded_hdemo_sk bigint, + cr_refunded_addr_sk bigint, + cr_returning_customer_sk bigint, + cr_returning_cdemo_sk bigint, + cr_returning_hdemo_sk bigint, + cr_returning_addr_sk bigint, + cr_call_center_sk bigint, + cr_catalog_page_sk bigint, + cr_ship_mode_sk bigint, + cr_warehouse_sk bigint, + cr_reason_sk bigint, + cr_return_quantity integer, + cr_return_amount decimalv3(7,2), + cr_return_tax decimalv3(7,2), + cr_return_amt_inc_tax decimalv3(7,2), + cr_fee decimalv3(7,2), + cr_return_ship_cost decimalv3(7,2), + cr_refunded_cash decimalv3(7,2), + cr_reversed_charge decimalv3(7,2), + cr_store_credit decimalv3(7,2), + cr_net_loss decimalv3(7,2) +) +DUPLICATE KEY(cr_item_sk, cr_order_number) +PARTITION BY RANGE(cr_returned_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(cr_item_sk, cr_order_number) BUCKETS 256 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "catalog" +); +drop table if exists household_demographics; +CREATE TABLE IF NOT EXISTS household_demographics ( + hd_demo_sk bigint not null, + hd_income_band_sk bigint, + hd_buy_potential char(15), + hd_dep_count integer, + hd_vehicle_count integer +) +DUPLICATE KEY(hd_demo_sk) +DISTRIBUTED BY HASH(hd_demo_sk) BUCKETS 3 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists customer_address; +CREATE TABLE IF NOT EXISTS customer_address ( + ca_address_sk bigint not null, + ca_address_id char(16) not null, + ca_street_number char(10), + ca_street_name varchar(60), + ca_street_type char(15), + ca_suite_number char(10), + ca_city varchar(60), + ca_county varchar(30), + ca_state char(2), + ca_zip char(10), + ca_country varchar(20), + ca_gmt_offset decimalv3(5,2), + ca_location_type char(20) +) +DUPLICATE KEY(ca_address_sk) +DISTRIBUTED BY HASH(ca_address_sk) BUCKETS 256 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists income_band; +CREATE TABLE IF NOT EXISTS income_band ( + ib_income_band_sk bigint not null, + ib_lower_bound integer, + ib_upper_bound integer +) +DUPLICATE KEY(ib_income_band_sk) +DISTRIBUTED BY HASH(ib_income_band_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists catalog_page; +CREATE TABLE IF NOT EXISTS catalog_page ( + cp_catalog_page_sk bigint not null, + cp_catalog_page_id char(16) not null, + cp_start_date_sk integer, + cp_end_date_sk integer, + cp_department varchar(50), + cp_catalog_number integer, + cp_catalog_page_number integer, + cp_description varchar(100), + cp_type varchar(100) +) +DUPLICATE KEY(cp_catalog_page_sk) +DISTRIBUTED BY HASH(cp_catalog_page_sk) BUCKETS 3 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists item; +CREATE TABLE IF NOT EXISTS item ( + i_item_sk bigint not null, + i_item_id char(16) not null, + i_rec_start_date datev2, + i_rec_end_date datev2, + i_item_desc varchar(200), + i_current_price decimalv3(7,2), + i_wholesale_cost decimalv3(7,2), + i_brand_id integer, + i_brand char(50), + i_class_id integer, + i_class char(50), + i_category_id integer, + i_category char(50), + i_manufact_id integer, + i_manufact char(50), + i_size char(20), + i_formulation char(20), + i_color char(20), + i_units char(10), + i_container char(10), + i_manager_id integer, + i_product_name char(50) +) +DUPLICATE KEY(i_item_sk) +DISTRIBUTED BY HASH(i_item_sk) BUCKETS 32 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists web_returns; +CREATE TABLE IF NOT EXISTS web_returns ( + wr_item_sk bigint not null, + wr_order_number bigint not null, + wr_returned_date_sk bigint, + wr_returned_time_sk bigint, + wr_refunded_customer_sk bigint, + wr_refunded_cdemo_sk bigint, + wr_refunded_hdemo_sk bigint, + wr_refunded_addr_sk bigint, + wr_returning_customer_sk bigint, + wr_returning_cdemo_sk bigint, + wr_returning_hdemo_sk bigint, + wr_returning_addr_sk bigint, + wr_web_page_sk bigint, + wr_reason_sk bigint, + wr_return_quantity integer, + wr_return_amt decimalv3(7,2), + wr_return_tax decimalv3(7,2), + wr_return_amt_inc_tax decimalv3(7,2), + wr_fee decimalv3(7,2), + wr_return_ship_cost decimalv3(7,2), + wr_refunded_cash decimalv3(7,2), + wr_reversed_charge decimalv3(7,2), + wr_account_credit decimalv3(7,2), + wr_net_loss decimalv3(7,2) +) +DUPLICATE KEY(wr_item_sk, wr_order_number) +PARTITION BY RANGE(wr_returned_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(wr_item_sk, wr_order_number) BUCKETS 256 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "web" +); +drop table if exists web_site; +CREATE TABLE IF NOT EXISTS web_site ( + web_site_sk bigint not null, + web_site_id char(16) not null, + web_rec_start_date datev2, + web_rec_end_date datev2, + web_name varchar(50), + web_open_date_sk bigint, + web_close_date_sk bigint, + web_class varchar(50), + web_manager varchar(40), + web_mkt_id integer, + web_mkt_class varchar(50), + web_mkt_desc varchar(100), + web_market_manager varchar(40), + web_company_id integer, + web_company_name char(50), + web_street_number char(10), + web_street_name varchar(60), + web_street_type char(15), + web_suite_number char(10), + web_city varchar(60), + web_county varchar(30), + web_state char(2), + web_zip char(10), + web_country varchar(20), + web_gmt_offset decimalv3(5,2), + web_tax_percentage decimalv3(5,2) +) +DUPLICATE KEY(web_site_sk) +DISTRIBUTED BY HASH(web_site_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists promotion; +CREATE TABLE IF NOT EXISTS promotion ( + p_promo_sk bigint not null, + p_promo_id char(16) not null, + p_start_date_sk bigint, + p_end_date_sk bigint, + p_item_sk bigint, + p_cost decimalv3(15,2), + p_response_targe integer, + p_promo_name char(50), + p_channel_dmail char(1), + p_channel_email char(1), + p_channel_catalog char(1), + p_channel_tv char(1), + p_channel_radio char(1), + p_channel_press char(1), + p_channel_event char(1), + p_channel_demo char(1), + p_channel_details varchar(100), + p_purpose char(15), + p_discount_active char(1) +) +DUPLICATE KEY(p_promo_sk) +DISTRIBUTED BY HASH(p_promo_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists web_sales; +CREATE TABLE IF NOT EXISTS web_sales ( + ws_item_sk bigint not null, + ws_order_number bigint not null, + ws_sold_date_sk bigint, + ws_sold_time_sk bigint, + ws_ship_date_sk bigint, + ws_bill_customer_sk bigint, + ws_bill_cdemo_sk bigint, + ws_bill_hdemo_sk bigint, + ws_bill_addr_sk bigint, + ws_ship_customer_sk bigint, + ws_ship_cdemo_sk bigint, + ws_ship_hdemo_sk bigint, + ws_ship_addr_sk bigint, + ws_web_page_sk bigint, + ws_web_site_sk bigint, + ws_ship_mode_sk bigint, + ws_warehouse_sk bigint, + ws_promo_sk bigint, + ws_quantity integer, + ws_wholesale_cost decimalv3(7,2), + ws_list_price decimalv3(7,2), + ws_sales_price decimalv3(7,2), + ws_ext_discount_amt decimalv3(7,2), + ws_ext_sales_price decimalv3(7,2), + ws_ext_wholesale_cost decimalv3(7,2), + ws_ext_list_price decimalv3(7,2), + ws_ext_tax decimalv3(7,2), + ws_coupon_amt decimalv3(7,2), + ws_ext_ship_cost decimalv3(7,2), + ws_net_paid decimalv3(7,2), + ws_net_paid_inc_tax decimalv3(7,2), + ws_net_paid_inc_ship decimalv3(7,2), + ws_net_paid_inc_ship_tax decimalv3(7,2), + ws_net_profit decimalv3(7,2) +) +DUPLICATE KEY(ws_item_sk, ws_order_number) +PARTITION BY RANGE(ws_sold_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(ws_item_sk, ws_order_number) BUCKETS 256 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "web" +); +drop table if exists store; +CREATE TABLE IF NOT EXISTS store ( + s_store_sk bigint not null, + s_store_id char(16) not null, + s_rec_start_date datev2, + s_rec_end_date datev2, + s_closed_date_sk bigint, + s_store_name varchar(50), + s_number_employees integer, + s_floor_space integer, + s_hours char(20), + s_manager varchar(40), + s_market_id integer, + s_geography_class varchar(100), + s_market_desc varchar(100), + s_market_manager varchar(40), + s_division_id integer, + s_division_name varchar(50), + s_company_id integer, + s_company_name varchar(50), + s_street_number varchar(10), + s_street_name varchar(60), + s_street_type char(15), + s_suite_number char(10), + s_city varchar(60), + s_county varchar(30), + s_state char(2), + s_zip char(10), + s_country varchar(20), + s_gmt_offset decimalv3(5,2), + s_tax_precentage decimalv3(5,2) +) +DUPLICATE KEY(s_store_sk) +DISTRIBUTED BY HASH(s_store_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists time_dim; +CREATE TABLE IF NOT EXISTS time_dim ( + t_time_sk bigint not null, + t_time_id char(16) not null, + t_time integer, + t_hour integer, + t_minute integer, + t_second integer, + t_am_pm char(2), + t_shift char(20), + t_sub_shift char(20), + t_meal_time char(20) +) +DUPLICATE KEY(t_time_sk) +DISTRIBUTED BY HASH(t_time_sk) BUCKETS 12 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists web_page; +CREATE TABLE IF NOT EXISTS web_page ( + wp_web_page_sk bigint not null, + wp_web_page_id char(16) not null, + wp_rec_start_date datev2, + wp_rec_end_date datev2, + wp_creation_date_sk bigint, + wp_access_date_sk bigint, + wp_autogen_flag char(1), + wp_customer_sk bigint, + wp_url varchar(100), + wp_type char(50), + wp_char_count integer, + wp_link_count integer, + wp_image_count integer, + wp_max_ad_count integer +) +DUPLICATE KEY(wp_web_page_sk) +DISTRIBUTED BY HASH(wp_web_page_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists store_returns; +CREATE TABLE IF NOT EXISTS store_returns ( + sr_item_sk bigint not null, + sr_ticket_number bigint not null, + sr_returned_date_sk bigint, + sr_return_time_sk bigint, + sr_customer_sk bigint, + sr_cdemo_sk bigint, + sr_hdemo_sk bigint, + sr_addr_sk bigint, + sr_store_sk bigint, + sr_reason_sk bigint, + sr_return_quantity integer, + sr_return_amt decimalv3(7,2), + sr_return_tax decimalv3(7,2), + sr_return_amt_inc_tax decimalv3(7,2), + sr_fee decimalv3(7,2), + sr_return_ship_cost decimalv3(7,2), + sr_refunded_cash decimalv3(7,2), + sr_reversed_charge decimalv3(7,2), + sr_store_credit decimalv3(7,2), + sr_net_loss decimalv3(7,2) +) +DUPLICATE KEY(sr_item_sk, sr_ticket_number) +PARTITION BY RANGE(sr_returned_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(sr_item_sk, sr_ticket_number) BUCKETS 256 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "store" +); +drop table if exists store_sales; +CREATE TABLE IF NOT EXISTS store_sales ( + ss_item_sk bigint not null, + ss_ticket_number bigint not null, + ss_sold_date_sk bigint, + ss_sold_time_sk bigint, + ss_customer_sk bigint, + ss_cdemo_sk bigint, + ss_hdemo_sk bigint, + ss_addr_sk bigint, + ss_store_sk bigint, + ss_promo_sk bigint, + ss_quantity integer, + ss_wholesale_cost decimalv3(7,2), + ss_list_price decimalv3(7,2), + ss_sales_price decimalv3(7,2), + ss_ext_discount_amt decimalv3(7,2), + ss_ext_sales_price decimalv3(7,2), + ss_ext_wholesale_cost decimalv3(7,2), + ss_ext_list_price decimalv3(7,2), + ss_ext_tax decimalv3(7,2), + ss_coupon_amt decimalv3(7,2), + ss_net_paid decimalv3(7,2), + ss_net_paid_inc_tax decimalv3(7,2), + ss_net_profit decimalv3(7,2) +) +DUPLICATE KEY(ss_item_sk, ss_ticket_number) +PARTITION BY RANGE(ss_sold_date_sk) +( +PARTITION `p1` VALUES LESS THAN ("2450846"), +PARTITION `p2` VALUES LESS THAN ("2450874"), +PARTITION `p3` VALUES LESS THAN ("2450905"), +PARTITION `p4` VALUES LESS THAN ("2450935"), +PARTITION `p5` VALUES LESS THAN ("2450966"), +PARTITION `p6` VALUES LESS THAN ("2450996"), +PARTITION `p7` VALUES LESS THAN ("2451027"), +PARTITION `p8` VALUES LESS THAN ("2451058"), +PARTITION `p9` VALUES LESS THAN ("2451088"), +PARTITION `p10` VALUES LESS THAN ("2451119"), +PARTITION `p11` VALUES LESS THAN ("2451149"), +PARTITION `p12` VALUES LESS THAN ("2451180"), +PARTITION `p13` VALUES LESS THAN ("2451211"), +PARTITION `p14` VALUES LESS THAN ("2451239"), +PARTITION `p15` VALUES LESS THAN ("2451270"), +PARTITION `p16` VALUES LESS THAN ("2451300"), +PARTITION `p17` VALUES LESS THAN ("2451331"), +PARTITION `p18` VALUES LESS THAN ("2451361"), +PARTITION `p19` VALUES LESS THAN ("2451392"), +PARTITION `p20` VALUES LESS THAN ("2451423"), +PARTITION `p21` VALUES LESS THAN ("2451453"), +PARTITION `p22` VALUES LESS THAN ("2451484"), +PARTITION `p23` VALUES LESS THAN ("2451514"), +PARTITION `p24` VALUES LESS THAN ("2451545"), +PARTITION `p25` VALUES LESS THAN ("2451576"), +PARTITION `p26` VALUES LESS THAN ("2451605"), +PARTITION `p27` VALUES LESS THAN ("2451635"), +PARTITION `p28` VALUES LESS THAN ("2451666"), +PARTITION `p29` VALUES LESS THAN ("2451696"), +PARTITION `p30` VALUES LESS THAN ("2451726"), +PARTITION `p31` VALUES LESS THAN ("2451756"), +PARTITION `p32` VALUES LESS THAN ("2451787"), +PARTITION `p33` VALUES LESS THAN ("2451817"), +PARTITION `p34` VALUES LESS THAN ("2451848"), +PARTITION `p35` VALUES LESS THAN ("2451877"), +PARTITION `p36` VALUES LESS THAN ("2451906"), +PARTITION `p37` VALUES LESS THAN ("2451937"), +PARTITION `p38` VALUES LESS THAN ("2451968"), +PARTITION `p39` VALUES LESS THAN ("2451999"), +PARTITION `p40` VALUES LESS THAN ("2452031"), +PARTITION `p41` VALUES LESS THAN ("2452062"), +PARTITION `p42` VALUES LESS THAN ("2452092"), +PARTITION `p43` VALUES LESS THAN ("2452123"), +PARTITION `p44` VALUES LESS THAN ("2452154"), +PARTITION `p45` VALUES LESS THAN ("2452184"), +PARTITION `p46` VALUES LESS THAN ("2452215"), +PARTITION `p47` VALUES LESS THAN ("2452245"), +PARTITION `p48` VALUES LESS THAN ("2452276"), +PARTITION `p49` VALUES LESS THAN ("2452307"), +PARTITION `p50` VALUES LESS THAN ("2452335"), +PARTITION `p51` VALUES LESS THAN ("2452366"), +PARTITION `p52` VALUES LESS THAN ("2452396"), +PARTITION `p53` VALUES LESS THAN ("2452427"), +PARTITION `p54` VALUES LESS THAN ("2452457"), +PARTITION `p55` VALUES LESS THAN ("2452488"), +PARTITION `p56` VALUES LESS THAN ("2452519"), +PARTITION `p57` VALUES LESS THAN ("2452549"), +PARTITION `p58` VALUES LESS THAN ("2452580"), +PARTITION `p59` VALUES LESS THAN ("2452610"), +PARTITION `p60` VALUES LESS THAN ("2452641"), +PARTITION `p61` VALUES LESS THAN ("2452672"), +PARTITION `p62` VALUES LESS THAN ("2452700"), +PARTITION `p63` VALUES LESS THAN ("2452731"), +PARTITION `p64` VALUES LESS THAN ("2452761"), +PARTITION `p65` VALUES LESS THAN ("2452792"), +PARTITION `p66` VALUES LESS THAN ("2452822"), +PARTITION `p67` VALUES LESS THAN ("2452853"), +PARTITION `p68` VALUES LESS THAN ("2452884"), +PARTITION `p69` VALUES LESS THAN ("2452914"), +PARTITION `p70` VALUES LESS THAN ("2452945"), +PARTITION `p71` VALUES LESS THAN ("2452975"), +PARTITION `p72` VALUES LESS THAN (MAXVALUE) +) +DISTRIBUTED BY HASH(ss_item_sk, ss_ticket_number) BUCKETS 256 +PROPERTIES ( + "replication_num" = "1", + "colocate_with" = "store" +); +drop table if exists ship_mode; +CREATE TABLE IF NOT EXISTS ship_mode ( + sm_ship_mode_sk bigint not null, + sm_ship_mode_id char(16) not null, + sm_type char(30), + sm_code char(10), + sm_carrier char(20), + sm_contract char(20) +) +DUPLICATE KEY(sm_ship_mode_sk) +DISTRIBUTED BY HASH(sm_ship_mode_sk) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists customer; +CREATE TABLE IF NOT EXISTS customer ( + c_customer_sk bigint not null, + c_customer_id char(16) not null, + c_current_cdemo_sk bigint, + c_current_hdemo_sk bigint, + c_current_addr_sk bigint, + c_first_shipto_date_sk bigint, + c_first_sales_date_sk bigint, + c_salutation char(10), + c_first_name char(20), + c_last_name char(30), + c_preferred_cust_flag char(1), + c_birth_day integer, + c_birth_month integer, + c_birth_year integer, + c_birth_country varchar(20), + c_login char(13), + c_email_address char(50), + c_last_review_date_sk bigint +) +DUPLICATE KEY(c_customer_sk) +DISTRIBUTED BY HASH(c_customer_id) BUCKETS 256 +PROPERTIES ( + "replication_num" = "1" +); +drop table if exists dbgen_version; +CREATE TABLE IF NOT EXISTS dbgen_version +( + dv_version varchar(16) , + dv_create_date datev2 , + dv_create_time datetime , + dv_cmdline_args varchar(200) +) +DUPLICATE KEY(dv_version) +DISTRIBUTED BY HASH(dv_version) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); diff --git a/tools/tpcds-tools/queries/sf1/query1.sql b/tools/tpcds-tools/queries/sf1/query1.sql new file mode 100644 index 000000000000000..c2f6725361ecfe3 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query1.sql @@ -0,0 +1,22 @@ +with customer_total_return as +(select sr_customer_sk as ctr_customer_sk +,sr_store_sk as ctr_store_sk +,sum(SR_FEE) as ctr_total_return +from store_returns +,date_dim +where sr_returned_date_sk = d_date_sk +and d_year =2000 +group by sr_customer_sk +,sr_store_sk) + select c_customer_id +from customer_total_return ctr1 +,store +,customer +where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 +from customer_total_return ctr2 +where ctr1.ctr_store_sk = ctr2.ctr_store_sk) +and s_store_sk = ctr1.ctr_store_sk +and s_state = 'TN' +and ctr1.ctr_customer_sk = c_customer_sk +order by c_customer_id +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query10.sql b/tools/tpcds-tools/queries/sf1/query10.sql new file mode 100644 index 000000000000000..077f80ad92a66d3 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query10.sql @@ -0,0 +1,56 @@ +select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3, + cd_dep_count, + count(*) cnt4, + cd_dep_employed_count, + count(*) cnt5, + cd_dep_college_count, + count(*) cnt6 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_county in ('Fairfield County','Campbell County','Washtenaw County','Escambia County','Cleburne County') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2001 and + d_moy between 3 and 3+3) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2001 and + d_moy between 3 ANd 3+3) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2001 and + d_moy between 3 and 3+3)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query11.sql b/tools/tpcds-tools/queries/sf1/query11.sql new file mode 100644 index 000000000000000..f139721ec2c8c78 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query11.sql @@ -0,0 +1,78 @@ +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(ss_ext_list_price-ss_ext_discount_amt) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(ws_ext_list_price-ws_ext_discount_amt) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + ) + select + t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.dyear = 1998 + and t_s_secyear.dyear = 1998+1 + and t_w_firstyear.dyear = 1998 + and t_w_secyear.dyear = 1998+1 + and t_s_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else 0.0 end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else 0.0 end + order by t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query12.sql b/tools/tpcds-tools/queries/sf1/query12.sql new file mode 100644 index 000000000000000..253ddb640a994f0 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query12.sql @@ -0,0 +1,31 @@ +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ws_ext_sales_price) as itemrevenue + ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over + (partition by i_class) as revenueratio +from + web_sales + ,item + ,date_dim +where + ws_item_sk = i_item_sk + and i_category in ('Men', 'Books', 'Electronics') + and ws_sold_date_sk = d_date_sk + and d_date between cast('2001-06-15' as date) + and (cast('2001-06-15' as date) + interval 30 day) +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query13.sql b/tools/tpcds-tools/queries/sf1/query13.sql new file mode 100644 index 000000000000000..456bdf47b400305 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query13.sql @@ -0,0 +1,49 @@ +select avg(ss_quantity) + ,avg(ss_ext_sales_price) + ,avg(ss_ext_wholesale_cost) + ,sum(ss_ext_wholesale_cost) + from store_sales + ,store + ,customer_demographics + ,household_demographics + ,customer_address + ,date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 2001 + and((ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'M' + and cd_education_status = 'College' + and ss_sales_price between 100.00 and 150.00 + and hd_dep_count = 3 + )or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'D' + and cd_education_status = 'Primary' + and ss_sales_price between 50.00 and 100.00 + and hd_dep_count = 1 + ) or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'W' + and cd_education_status = '2 yr Degree' + and ss_sales_price between 150.00 and 200.00 + and hd_dep_count = 1 + )) + and((ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('IL', 'TN', 'TX') + and ss_net_profit between 100 and 200 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('WY', 'OH', 'ID') + and ss_net_profit between 150 and 300 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('MS', 'SC', 'IA') + and ss_net_profit between 50 and 250 + )) +; diff --git a/tools/tpcds-tools/queries/sf1/query14.sql b/tools/tpcds-tools/queries/sf1/query14.sql new file mode 100644 index 000000000000000..e4cc31f9c863f8e --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query14.sql @@ -0,0 +1,207 @@ +with cross_items as + (select i_item_sk ss_item_sk + from item, + (select iss.i_brand_id brand_id + ,iss.i_class_id class_id + ,iss.i_category_id category_id + from store_sales + ,item iss + ,date_dim d1 + where ss_item_sk = iss.i_item_sk + and ss_sold_date_sk = d1.d_date_sk + and d1.d_year between 1999 AND 1999 + 2 + intersect + select ics.i_brand_id + ,ics.i_class_id + ,ics.i_category_id + from catalog_sales + ,item ics + ,date_dim d2 + where cs_item_sk = ics.i_item_sk + and cs_sold_date_sk = d2.d_date_sk + and d2.d_year between 1999 AND 1999 + 2 + intersect + select iws.i_brand_id + ,iws.i_class_id + ,iws.i_category_id + from web_sales + ,item iws + ,date_dim d3 + where ws_item_sk = iws.i_item_sk + and ws_sold_date_sk = d3.d_date_sk + and d3.d_year between 1999 AND 1999 + 2) + t where i_brand_id = brand_id + and i_class_id = class_id + and i_category_id = category_id +), + avg_sales as + (select avg(quantity*list_price) average_sales + from (select ss_quantity quantity + ,ss_list_price list_price + from store_sales + ,date_dim + where ss_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2 + union all + select cs_quantity quantity + ,cs_list_price list_price + from catalog_sales + ,date_dim + where cs_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2 + union all + select ws_quantity quantity + ,ws_list_price list_price + from web_sales + ,date_dim + where ws_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2) x) + select channel, i_brand_id,i_class_id,i_category_id,sum(sales), sum(number_sales) + from( + select 'store' channel, i_brand_id,i_class_id + ,i_category_id,sum(ss_quantity*ss_list_price) sales + , count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1999+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales) + union all + select 'catalog' channel, i_brand_id,i_class_id,i_category_id, sum(cs_quantity*cs_list_price) sales, count(*) number_sales + from catalog_sales + ,item + ,date_dim + where cs_item_sk in (select ss_item_sk from cross_items) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1999+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(cs_quantity*cs_list_price) > (select average_sales from avg_sales) + union all + select 'web' channel, i_brand_id,i_class_id,i_category_id, sum(ws_quantity*ws_list_price) sales , count(*) number_sales + from web_sales + ,item + ,date_dim + where ws_item_sk in (select ss_item_sk from cross_items) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1999+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(ws_quantity*ws_list_price) > (select average_sales from avg_sales) + ) y + group by rollup (channel, i_brand_id,i_class_id,i_category_id) + order by channel,i_brand_id,i_class_id,i_category_id + limit 100; +with cross_items as + (select i_item_sk ss_item_sk + from item, + (select iss.i_brand_id brand_id + ,iss.i_class_id class_id + ,iss.i_category_id category_id + from store_sales + ,item iss + ,date_dim d1 + where ss_item_sk = iss.i_item_sk + and ss_sold_date_sk = d1.d_date_sk + and d1.d_year between 1999 AND 1999 + 2 + intersect + select ics.i_brand_id + ,ics.i_class_id + ,ics.i_category_id + from catalog_sales + ,item ics + ,date_dim d2 + where cs_item_sk = ics.i_item_sk + and cs_sold_date_sk = d2.d_date_sk + and d2.d_year between 1999 AND 1999 + 2 + intersect + select iws.i_brand_id + ,iws.i_class_id + ,iws.i_category_id + from web_sales + ,item iws + ,date_dim d3 + where ws_item_sk = iws.i_item_sk + and ws_sold_date_sk = d3.d_date_sk + and d3.d_year between 1999 AND 1999 + 2) x + where i_brand_id = brand_id + and i_class_id = class_id + and i_category_id = category_id +), + avg_sales as +(select avg(quantity*list_price) average_sales + from (select ss_quantity quantity + ,ss_list_price list_price + from store_sales + ,date_dim + where ss_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2 + union all + select cs_quantity quantity + ,cs_list_price list_price + from catalog_sales + ,date_dim + where cs_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2 + union all + select ws_quantity quantity + ,ws_list_price list_price + from web_sales + ,date_dim + where ws_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2) x) + select this_year.channel ty_channel + ,this_year.i_brand_id ty_brand + ,this_year.i_class_id ty_class + ,this_year.i_category_id ty_category + ,this_year.sales ty_sales + ,this_year.number_sales ty_number_sales + ,last_year.channel ly_channel + ,last_year.i_brand_id ly_brand + ,last_year.i_class_id ly_class + ,last_year.i_category_id ly_category + ,last_year.sales ly_sales + ,last_year.number_sales ly_number_sales + from + (select 'store' channel, i_brand_id,i_class_id,i_category_id + ,sum(ss_quantity*ss_list_price) sales, count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_week_seq = (select d_week_seq + from date_dim + where d_year = 1999 + 1 + and d_moy = 12 + and d_dom = 3) + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) this_year, + (select 'store' channel, i_brand_id,i_class_id + ,i_category_id, sum(ss_quantity*ss_list_price) sales, count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_week_seq = (select d_week_seq + from date_dim + where d_year = 1999 + and d_moy = 12 + and d_dom = 3) + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) last_year + where this_year.i_brand_id= last_year.i_brand_id + and this_year.i_class_id = last_year.i_class_id + and this_year.i_category_id = last_year.i_category_id + order by this_year.channel, this_year.i_brand_id, this_year.i_class_id, this_year.i_category_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query15.sql b/tools/tpcds-tools/queries/sf1/query15.sql new file mode 100644 index 000000000000000..a6bb8f07028eb64 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query15.sql @@ -0,0 +1,17 @@ +select ca_zip + ,sum(cs_sales_price) + from catalog_sales + ,customer + ,customer_address + ,date_dim + where cs_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', + '85392', '85460', '80348', '81792') + or ca_state in ('CA','WA','GA') + or cs_sales_price > 500) + and cs_sold_date_sk = d_date_sk + and d_qoy = 2 and d_year = 2001 + group by ca_zip + order by ca_zip + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query16.sql b/tools/tpcds-tools/queries/sf1/query16.sql new file mode 100644 index 000000000000000..168d2b4f9d875a4 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query16.sql @@ -0,0 +1,28 @@ +select + count(distinct cs_order_number) as "order count" + ,sum(cs_ext_ship_cost) as "total shipping cost" + ,sum(cs_net_profit) as "total net profit" +from + catalog_sales cs1 + ,date_dim + ,customer_address + ,call_center +where + d_date between '2002-4-01' and + (cast('2002-4-01' as date) + interval 60 day) +and cs1.cs_ship_date_sk = d_date_sk +and cs1.cs_ship_addr_sk = ca_address_sk +and ca_state = 'PA' +and cs1.cs_call_center_sk = cc_call_center_sk +and cc_county in ('Williamson County','Williamson County','Williamson County','Williamson County', + 'Williamson County' +) +and exists (select * + from catalog_sales cs2 + where cs1.cs_order_number = cs2.cs_order_number + and cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) +and not exists(select * + from catalog_returns cr1 + where cs1.cs_order_number = cr1.cr_order_number) +order by count(distinct cs_order_number) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query17.sql b/tools/tpcds-tools/queries/sf1/query17.sql new file mode 100644 index 000000000000000..02b9c4761c5bb3c --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query17.sql @@ -0,0 +1,42 @@ +select i_item_id + ,i_item_desc + ,s_state + ,count(ss_quantity) as store_sales_quantitycount + ,avg(ss_quantity) as store_sales_quantityave + ,stddev_samp(ss_quantity) as store_sales_quantitystdev + ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov + ,count(sr_return_quantity) as store_returns_quantitycount + ,avg(sr_return_quantity) as store_returns_quantityave + ,stddev_samp(sr_return_quantity) as store_returns_quantitystdev + ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov + ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave + ,stddev_samp(cs_quantity) as catalog_sales_quantitystdev + ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov + from store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where d1.d_quarter_name = '2001Q1' + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_quarter_name in ('2001Q1','2001Q2','2001Q3') + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_quarter_name in ('2001Q1','2001Q2','2001Q3') + group by i_item_id + ,i_item_desc + ,s_state + order by i_item_id + ,i_item_desc + ,s_state +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query18.sql b/tools/tpcds-tools/queries/sf1/query18.sql new file mode 100644 index 000000000000000..1f36ffc8374ab70 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query18.sql @@ -0,0 +1,31 @@ +select i_item_id, + ca_country, + ca_state, + ca_county, + avg( cast(cs_quantity as decimal(12,2))) agg1, + avg( cast(cs_list_price as decimal(12,2))) agg2, + avg( cast(cs_coupon_amt as decimal(12,2))) agg3, + avg( cast(cs_sales_price as decimal(12,2))) agg4, + avg( cast(cs_net_profit as decimal(12,2))) agg5, + avg( cast(c_birth_year as decimal(12,2))) agg6, + avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7 + from catalog_sales, customer_demographics cd1, + customer_demographics cd2, customer, customer_address, date_dim, item + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd1.cd_demo_sk and + cs_bill_customer_sk = c_customer_sk and + cd1.cd_gender = 'F' and + cd1.cd_education_status = 'Primary' and + c_current_cdemo_sk = cd2.cd_demo_sk and + c_current_addr_sk = ca_address_sk and + c_birth_month in (1,3,7,11,10,4) and + d_year = 2001 and + ca_state in ('AL','MO','TN' + ,'GA','MT','IN','CA') + group by rollup (i_item_id, ca_country, ca_state, ca_county) + order by ca_country, + ca_state, + ca_county, + i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query19.sql b/tools/tpcds-tools/queries/sf1/query19.sql new file mode 100644 index 000000000000000..0f44101ac589c70 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query19.sql @@ -0,0 +1,22 @@ +select i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item,customer,customer_address,store + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=14 + and d_moy=11 + and d_year=2002 + and ss_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and substr(ca_zip,1,5) <> substr(s_zip,1,5) + and ss_store_sk = s_store_sk + group by i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact + order by ext_price desc + ,i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact +limit 100 ; diff --git a/tools/tpcds-tools/queries/sf1/query2.sql b/tools/tpcds-tools/queries/sf1/query2.sql new file mode 100644 index 000000000000000..5dc50b1156c022e --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query2.sql @@ -0,0 +1,57 @@ +with wscs as + (select sold_date_sk + ,sales_price + from (select ws_sold_date_sk sold_date_sk + ,ws_ext_sales_price sales_price + from web_sales + union all + select cs_sold_date_sk sold_date_sk + ,cs_ext_sales_price sales_price + from catalog_sales) t), + wswscs as + (select d_week_seq, + sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales + from wscs + ,date_dim + where d_date_sk = sold_date_sk + group by d_week_seq) + select d_week_seq1 + ,round(sun_sales1/sun_sales2,2) + ,round(mon_sales1/mon_sales2,2) + ,round(tue_sales1/tue_sales2,2) + ,round(wed_sales1/wed_sales2,2) + ,round(thu_sales1/thu_sales2,2) + ,round(fri_sales1/fri_sales2,2) + ,round(sat_sales1/sat_sales2,2) + from + (select wswscs.d_week_seq d_week_seq1 + ,sun_sales sun_sales1 + ,mon_sales mon_sales1 + ,tue_sales tue_sales1 + ,wed_sales wed_sales1 + ,thu_sales thu_sales1 + ,fri_sales fri_sales1 + ,sat_sales sat_sales1 + from wswscs,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 1998) y, + (select wswscs.d_week_seq d_week_seq2 + ,sun_sales sun_sales2 + ,mon_sales mon_sales2 + ,tue_sales tue_sales2 + ,wed_sales wed_sales2 + ,thu_sales thu_sales2 + ,fri_sales fri_sales2 + ,sat_sales sat_sales2 + from wswscs + ,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 1998+1) z + where d_week_seq1=d_week_seq2-53 + order by d_week_seq1; diff --git a/tools/tpcds-tools/queries/sf1/query20.sql b/tools/tpcds-tools/queries/sf1/query20.sql new file mode 100644 index 000000000000000..1e0b1e1c523bfad --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query20.sql @@ -0,0 +1,27 @@ +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(cs_ext_sales_price) as itemrevenue + ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over + (partition by i_class) as revenueratio + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and i_category in ('Books', 'Music', 'Sports') + and cs_sold_date_sk = d_date_sk + and d_date between cast('2002-06-18' as date) + and (cast('2002-06-18' as date) + interval 30 day) + group by i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + order by i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query21.sql b/tools/tpcds-tools/queries/sf1/query21.sql new file mode 100644 index 000000000000000..7bced24450465a6 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query21.sql @@ -0,0 +1,27 @@ +select * + from(select w_warehouse_name + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('1999-06-22' as date)) + then inv_quantity_on_hand + else 0 end) as inv_before + ,sum(case when (cast(d_date as date) >= cast ('1999-06-22' as date)) + then inv_quantity_on_hand + else 0 end) as inv_after + from inventory + ,warehouse + ,item + ,date_dim + where i_current_price between 0.99 and 1.49 + and i_item_sk = inv_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_date between (cast ('1999-06-22' as date) - interval 30 day) + and (cast ('1999-06-22' as date) + interval 30 day) + group by w_warehouse_name, i_item_id) x + where (case when inv_before > 0 + then inv_after / inv_before + else null + end) between 2.0/3.0 and 3.0/2.0 + order by w_warehouse_name + ,i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query22.sql b/tools/tpcds-tools/queries/sf1/query22.sql new file mode 100644 index 000000000000000..dc0349840a33342 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query22.sql @@ -0,0 +1,17 @@ +select i_product_name + ,i_brand + ,i_class + ,i_category + ,avg(inv_quantity_on_hand) qoh + from inventory + ,date_dim + ,item + where inv_date_sk=d_date_sk + and inv_item_sk=i_item_sk + and d_month_seq between 1200 and 1200 + 11 + group by rollup(i_product_name + ,i_brand + ,i_class + ,i_category) +order by qoh, i_product_name, i_brand, i_class, i_category +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query23.sql b/tools/tpcds-tools/queries/sf1/query23.sql new file mode 100644 index 000000000000000..aff598b44dd0af8 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query23.sql @@ -0,0 +1,104 @@ +with frequent_ss_items as + (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt + from store_sales + ,date_dim + ,item + where ss_sold_date_sk = d_date_sk + and ss_item_sk = i_item_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by substr(i_item_desc,1,30),i_item_sk,d_date + having count(*) >4), + max_store_sales as + (select max(csales) tpcds_cmax + from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales + from store_sales + ,customer + ,date_dim + where ss_customer_sk = c_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by c_customer_sk) t), + best_ss_customer as + (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales + from store_sales + ,customer + where ss_customer_sk = c_customer_sk + group by c_customer_sk + having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select + * +from + max_store_sales)) + select sum(sales) + from (select cs_quantity*cs_list_price sales + from catalog_sales + ,date_dim + where d_year = 2000 + and d_moy = 7 + and cs_sold_date_sk = d_date_sk + and cs_item_sk in (select item_sk from frequent_ss_items) + and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) + union all + select ws_quantity*ws_list_price sales + from web_sales + ,date_dim + where d_year = 2000 + and d_moy = 7 + and ws_sold_date_sk = d_date_sk + and ws_item_sk in (select item_sk from frequent_ss_items) + and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)) t2 + limit 100; +with frequent_ss_items as + (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt + from store_sales + ,date_dim + ,item + where ss_sold_date_sk = d_date_sk + and ss_item_sk = i_item_sk + and d_year in (2000,2000 + 1,2000 + 2,2000 + 3) + group by substr(i_item_desc,1,30),i_item_sk,d_date + having count(*) >4), + max_store_sales as + (select max(csales) tpcds_cmax + from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales + from store_sales + ,customer + ,date_dim + where ss_customer_sk = c_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by c_customer_sk) t), + best_ss_customer as + (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales + from store_sales + ,customer + where ss_customer_sk = c_customer_sk + group by c_customer_sk + having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select + * + from max_store_sales)) + select c_last_name,c_first_name,sales + from (select c_last_name,c_first_name,sum(cs_quantity*cs_list_price) sales + from catalog_sales + ,customer + ,date_dim + where d_year = 2000 + and d_moy = 7 + and cs_sold_date_sk = d_date_sk + and cs_item_sk in (select item_sk from frequent_ss_items) + and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) + and cs_bill_customer_sk = c_customer_sk + group by c_last_name,c_first_name + union all + select c_last_name,c_first_name,sum(ws_quantity*ws_list_price) sales + from web_sales + ,customer + ,date_dim + where d_year = 2000 + and d_moy = 7 + and ws_sold_date_sk = d_date_sk + and ws_item_sk in (select item_sk from frequent_ss_items) + and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer) + and ws_bill_customer_sk = c_customer_sk + group by c_last_name,c_first_name) t2 + order by c_last_name,c_first_name,sales + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query24.sql b/tools/tpcds-tools/queries/sf1/query24.sql new file mode 100644 index 000000000000000..f29cf674646846d --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query24.sql @@ -0,0 +1,104 @@ +with ssales as +(select c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size + ,sum(ss_net_paid) netpaid +from store_sales + ,store_returns + ,store + ,item + ,customer + ,customer_address +where ss_ticket_number = sr_ticket_number + and ss_item_sk = sr_item_sk + and ss_customer_sk = c_customer_sk + and ss_item_sk = i_item_sk + and ss_store_sk = s_store_sk + and c_current_addr_sk = ca_address_sk + and c_birth_country <> upper(ca_country) + and s_zip = ca_zip +and s_market_id=5 +group by c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size) +select c_last_name + ,c_first_name + ,s_store_name + ,sum(netpaid) paid +from ssales +where i_color = 'aquamarine' +group by c_last_name + ,c_first_name + ,s_store_name +having sum(netpaid) > (select 0.05*avg(netpaid) + from ssales) +order by c_last_name + ,c_first_name + ,s_store_name +; +with ssales as +(select c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size + ,sum(ss_net_paid) netpaid +from store_sales + ,store_returns + ,store + ,item + ,customer + ,customer_address +where ss_ticket_number = sr_ticket_number + and ss_item_sk = sr_item_sk + and ss_customer_sk = c_customer_sk + and ss_item_sk = i_item_sk + and ss_store_sk = s_store_sk + and c_current_addr_sk = ca_address_sk + and c_birth_country <> upper(ca_country) + and s_zip = ca_zip + and s_market_id = 5 +group by c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size) +select c_last_name + ,c_first_name + ,s_store_name + ,sum(netpaid) paid +from ssales +where i_color = 'seashell' +group by c_last_name + ,c_first_name + ,s_store_name +having sum(netpaid) > (select 0.05*avg(netpaid) + from ssales) +order by c_last_name + ,c_first_name + ,s_store_name +; diff --git a/tools/tpcds-tools/queries/sf1/query25.sql b/tools/tpcds-tools/queries/sf1/query25.sql new file mode 100644 index 000000000000000..c927ff6bb06c969 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query25.sql @@ -0,0 +1,45 @@ +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,max(ss_net_profit) as store_sales_profit + ,max(sr_net_loss) as store_returns_loss + ,max(cs_net_profit) as catalog_sales_profit + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 1999 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 10 + and d2.d_year = 1999 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_moy between 4 and 10 + and d3.d_year = 1999 + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query26.sql b/tools/tpcds-tools/queries/sf1/query26.sql new file mode 100644 index 000000000000000..e601c2cdd81eb43 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query26.sql @@ -0,0 +1,18 @@ +select i_item_id, + avg(cs_quantity) agg1, + avg(cs_list_price) agg2, + avg(cs_coupon_amt) agg3, + avg(cs_sales_price) agg4 + from catalog_sales, customer_demographics, date_dim, item, promotion + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd_demo_sk and + cs_promo_sk = p_promo_sk and + cd_gender = 'M' and + cd_marital_status = 'W' and + cd_education_status = 'Unknown' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 2002 + group by i_item_id + order by i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query27.sql b/tools/tpcds-tools/queries/sf1/query27.sql new file mode 100644 index 000000000000000..3b6159be2bb2271 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query27.sql @@ -0,0 +1,20 @@ +select i_item_id, + s_state, grouping(s_state) g_state, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, store, item + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_store_sk = s_store_sk and + ss_cdemo_sk = cd_demo_sk and + cd_gender = 'M' and + cd_marital_status = 'W' and + cd_education_status = 'Secondary' and + d_year = 1999 and + s_state in ('TN','TN', 'TN', 'TN', 'TN', 'TN') + group by rollup (i_item_id, s_state) + order by i_item_id + ,s_state + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query28.sql b/tools/tpcds-tools/queries/sf1/query28.sql new file mode 100644 index 000000000000000..b34cbcc1778d502 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query28.sql @@ -0,0 +1,50 @@ +select * +from (select avg(ss_list_price) B1_LP + ,count(ss_list_price) B1_CNT + ,count(distinct ss_list_price) B1_CNTD + from store_sales + where ss_quantity between 0 and 5 + and (ss_list_price between 107 and 107+10 + or ss_coupon_amt between 1319 and 1319+1000 + or ss_wholesale_cost between 60 and 60+20)) B1, + (select avg(ss_list_price) B2_LP + ,count(ss_list_price) B2_CNT + ,count(distinct ss_list_price) B2_CNTD + from store_sales + where ss_quantity between 6 and 10 + and (ss_list_price between 23 and 23+10 + or ss_coupon_amt between 825 and 825+1000 + or ss_wholesale_cost between 43 and 43+20)) B2, + (select avg(ss_list_price) B3_LP + ,count(ss_list_price) B3_CNT + ,count(distinct ss_list_price) B3_CNTD + from store_sales + where ss_quantity between 11 and 15 + and (ss_list_price between 74 and 74+10 + or ss_coupon_amt between 4381 and 4381+1000 + or ss_wholesale_cost between 57 and 57+20)) B3, + (select avg(ss_list_price) B4_LP + ,count(ss_list_price) B4_CNT + ,count(distinct ss_list_price) B4_CNTD + from store_sales + where ss_quantity between 16 and 20 + and (ss_list_price between 89 and 89+10 + or ss_coupon_amt between 3117 and 3117+1000 + or ss_wholesale_cost between 68 and 68+20)) B4, + (select avg(ss_list_price) B5_LP + ,count(ss_list_price) B5_CNT + ,count(distinct ss_list_price) B5_CNTD + from store_sales + where ss_quantity between 21 and 25 + and (ss_list_price between 58 and 58+10 + or ss_coupon_amt between 9402 and 9402+1000 + or ss_wholesale_cost between 38 and 38+20)) B5, + (select avg(ss_list_price) B6_LP + ,count(ss_list_price) B6_CNT + ,count(distinct ss_list_price) B6_CNTD + from store_sales + where ss_quantity between 26 and 30 + and (ss_list_price between 64 and 64+10 + or ss_coupon_amt between 5792 and 5792+1000 + or ss_wholesale_cost between 73 and 73+20)) B6 +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query29.sql b/tools/tpcds-tools/queries/sf1/query29.sql new file mode 100644 index 000000000000000..49b31ce112f7b0f --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query29.sql @@ -0,0 +1,44 @@ +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,max(ss_quantity) as store_sales_quantity + ,max(sr_return_quantity) as store_returns_quantity + ,max(cs_quantity) as catalog_sales_quantity + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 1998 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 4 + 3 + and d2.d_year = 1998 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_year in (1998,1998+1,1998+2) + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query3.sql b/tools/tpcds-tools/queries/sf1/query3.sql new file mode 100644 index 000000000000000..6f368b3a2b52800 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query3.sql @@ -0,0 +1,18 @@ +select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_sales_price) sum_agg + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 816 + and dt.d_moy=11 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,sum_agg desc + ,brand_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query30.sql b/tools/tpcds-tools/queries/sf1/query30.sql new file mode 100644 index 000000000000000..f5c7b4ee3c9a6da --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query30.sql @@ -0,0 +1,28 @@ +with customer_total_return as + (select wr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(wr_return_amt) as ctr_total_return + from web_returns + ,date_dim + ,customer_address + where wr_returned_date_sk = d_date_sk + and d_year =2000 + and wr_returning_addr_sk = ca_address_sk + group by wr_returning_customer_sk + ,ca_state) + select c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date_sk,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'AR' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date_sk,ctr_total_return +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query31.sql b/tools/tpcds-tools/queries/sf1/query31.sql new file mode 100644 index 000000000000000..d0c361d5e777de4 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query31.sql @@ -0,0 +1,49 @@ +with ss as + (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales + from store_sales,date_dim,customer_address + where ss_sold_date_sk = d_date_sk + and ss_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year), + ws as + (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales + from web_sales,date_dim,customer_address + where ws_sold_date_sk = d_date_sk + and ws_bill_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year) + select + ss1.ca_county + ,ss1.d_year + ,ws2.web_sales/ws1.web_sales web_q1_q2_increase + ,ss2.store_sales/ss1.store_sales store_q1_q2_increase + ,ws3.web_sales/ws2.web_sales web_q2_q3_increase + ,ss3.store_sales/ss2.store_sales store_q2_q3_increase + from + ss ss1 + ,ss ss2 + ,ss ss3 + ,ws ws1 + ,ws ws2 + ,ws ws3 + where + ss1.d_qoy = 1 + and ss1.d_year = 1999 + and ss1.ca_county = ss2.ca_county + and ss2.d_qoy = 2 + and ss2.d_year = 1999 + and ss2.ca_county = ss3.ca_county + and ss3.d_qoy = 3 + and ss3.d_year = 1999 + and ss1.ca_county = ws1.ca_county + and ws1.d_qoy = 1 + and ws1.d_year = 1999 + and ws1.ca_county = ws2.ca_county + and ws2.d_qoy = 2 + and ws2.d_year = 1999 + and ws1.ca_county = ws3.ca_county + and ws3.d_qoy = 3 + and ws3.d_year =1999 + and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end + > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end + and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end + > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end + order by store_q2_q3_increase; diff --git a/tools/tpcds-tools/queries/sf1/query32.sql b/tools/tpcds-tools/queries/sf1/query32.sql new file mode 100644 index 000000000000000..2d74830b3fa41c0 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query32.sql @@ -0,0 +1,25 @@ +select sum(cs_ext_discount_amt) as "excess discount amount" +from + catalog_sales + ,item + ,date_dim +where +i_manufact_id = 722 +and i_item_sk = cs_item_sk +and d_date between '2001-03-09' and + (cast('2001-03-09' as date) + interval 90 day) +and d_date_sk = cs_sold_date_sk +and cs_ext_discount_amt + > ( + select + 1.3 * avg(cs_ext_discount_amt) + from + catalog_sales + ,date_dim + where + cs_item_sk = i_item_sk + and d_date between '2001-03-09' and + (cast('2001-03-09' as date) + interval 90 day) + and d_date_sk = cs_sold_date_sk + ) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query33.sql b/tools/tpcds-tools/queries/sf1/query33.sql new file mode 100644 index 000000000000000..35e52280e4cc359 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query33.sql @@ -0,0 +1,72 @@ +with ss as ( + select + i_manufact_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 3 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id), + cs as ( + select + i_manufact_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 3 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id), + ws as ( + select + i_manufact_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 3 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id) + select i_manufact_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_manufact_id + order by total_sales +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query34.sql b/tools/tpcds-tools/queries/sf1/query34.sql new file mode 100644 index 000000000000000..cbe0fb1e432f090 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query34.sql @@ -0,0 +1,28 @@ +select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28) + and (household_demographics.hd_buy_potential = '1001-5000' or + household_demographics.hd_buy_potential = '0-500') + and household_demographics.hd_vehicle_count > 0 + and (case when household_demographics.hd_vehicle_count > 0 + then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count + else null + end) > 1.2 + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County', + 'Williamson County','Williamson County','Williamson County','Williamson County') + group by ss_ticket_number,ss_customer_sk) dn,customer + where ss_customer_sk = c_customer_sk + and cnt between 15 and 20 + order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc, ss_ticket_number; diff --git a/tools/tpcds-tools/queries/sf1/query35.sql b/tools/tpcds-tools/queries/sf1/query35.sql new file mode 100644 index 000000000000000..94db12fc554a3e5 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query35.sql @@ -0,0 +1,55 @@ +select + ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + count(*) cnt1, + avg(cd_dep_count), + stddev_samp(cd_dep_count), + sum(cd_dep_count), + cd_dep_employed_count, + count(*) cnt2, + avg(cd_dep_employed_count), + stddev_samp(cd_dep_employed_count), + sum(cd_dep_employed_count), + cd_dep_college_count, + count(*) cnt3, + avg(cd_dep_college_count), + stddev_samp(cd_dep_college_count), + sum(cd_dep_college_count) + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 1999 and + d_qoy < 4) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 1999 and + d_qoy < 4) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 1999 and + d_qoy < 4)) + group by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query36.sql b/tools/tpcds-tools/queries/sf1/query36.sql new file mode 100644 index 000000000000000..becddb653e414d5 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query36.sql @@ -0,0 +1,27 @@ +select + sum(ss_net_profit)/sum(ss_ext_sales_price) as gross_margin + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,item + ,store + where + d1.d_year = 2000 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and s_state in ('TN','TN','TN','TN', + 'TN','TN','TN','TN') + group by rollup(i_category,i_class) + order by + lochierarchy desc + ,case when lochierarchy = 0 then i_category end + ,rank_within_parent + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query37.sql b/tools/tpcds-tools/queries/sf1/query37.sql new file mode 100644 index 000000000000000..1b57326035777ca --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query37.sql @@ -0,0 +1,14 @@ +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, catalog_sales + where i_current_price between 29 and 29 + 30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('2002-03-29' as date) and (cast('2002-03-29' as date) + interval 60 day) + and i_manufact_id in (705,742,777,944) + and inv_quantity_on_hand between 100 and 500 + and cs_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query38.sql b/tools/tpcds-tools/queries/sf1/query38.sql new file mode 100644 index 000000000000000..e7e5373e70dc61e --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query38.sql @@ -0,0 +1,20 @@ +select count(*) from ( + select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1189 and 1189 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1189 and 1189 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1189 and 1189 + 11 +) hot_cust +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query39.sql b/tools/tpcds-tools/queries/sf1/query39.sql new file mode 100644 index 000000000000000..9ffe92d6792472b --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query39.sql @@ -0,0 +1,51 @@ +with inv as +(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stdev,mean, case mean when 0 then null else stdev/mean end cov + from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean + from inventory + ,item + ,warehouse + ,date_dim + where inv_item_sk = i_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_year =2000 + group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo + where case mean when 0 then 0 else stdev/mean end > 1) +select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov + ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov +from inv inv1,inv inv2 +where inv1.i_item_sk = inv2.i_item_sk + and inv1.w_warehouse_sk = inv2.w_warehouse_sk + and inv1.d_moy=1 + and inv2.d_moy=1+1 +order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov + ,inv2.d_moy,inv2.mean, inv2.cov +; +with inv as +(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stdev,mean, case mean when 0 then null else stdev/mean end cov + from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean + from inventory + ,item + ,warehouse + ,date_dim + where inv_item_sk = i_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_year =2000 + group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo + where case mean when 0 then 0 else stdev/mean end > 1) +select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov + ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov +from inv inv1,inv inv2 +where inv1.i_item_sk = inv2.i_item_sk + and inv1.w_warehouse_sk = inv2.w_warehouse_sk + and inv1.d_moy=1 + and inv2.d_moy=1+1 + and inv1.cov > 1.5 +order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov + ,inv2.d_moy,inv2.mean, inv2.cov +; diff --git a/tools/tpcds-tools/queries/sf1/query4.sql b/tools/tpcds-tools/queries/sf1/query4.sql new file mode 100644 index 000000000000000..99d6806f73d77a8 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query4.sql @@ -0,0 +1,113 @@ +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(((ss_ext_list_price-ss_ext_wholesale_cost-ss_ext_discount_amt)+ss_ext_sales_price)/2) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum((((cs_ext_list_price-cs_ext_wholesale_cost-cs_ext_discount_amt)+cs_ext_sales_price)/2) ) year_total + ,'c' sale_type + from customer + ,catalog_sales + ,date_dim + where c_customer_sk = cs_bill_customer_sk + and cs_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year +union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum((((ws_ext_list_price-ws_ext_wholesale_cost-ws_ext_discount_amt)+ws_ext_sales_price)/2) ) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + ) + select + t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_birth_country + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_c_firstyear + ,year_total t_c_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_c_secyear.customer_id + and t_s_firstyear.customer_id = t_c_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_c_firstyear.sale_type = 'c' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_c_secyear.sale_type = 'c' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.dyear = 1999 + and t_s_secyear.dyear = 1999+1 + and t_c_firstyear.dyear = 1999 + and t_c_secyear.dyear = 1999+1 + and t_w_firstyear.dyear = 1999 + and t_w_secyear.dyear = 1999+1 + and t_s_firstyear.year_total > 0 + and t_c_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end + and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end + > case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end + order by t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_birth_country +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query40.sql b/tools/tpcds-tools/queries/sf1/query40.sql new file mode 100644 index 000000000000000..5e971da2a8597b3 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query40.sql @@ -0,0 +1,25 @@ +select + w_state + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('2001-05-02' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_before + ,sum(case when (cast(d_date as date) >= cast ('2001-05-02' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_after + from + catalog_sales left outer join catalog_returns on + (cs_order_number = cr_order_number + and cs_item_sk = cr_item_sk) + ,warehouse + ,item + ,date_dim + where + i_current_price between 0.99 and 1.49 + and i_item_sk = cs_item_sk + and cs_warehouse_sk = w_warehouse_sk + and cs_sold_date_sk = d_date_sk + and d_date between (cast ('2001-05-02' as date) - interval 30 day) + and (cast ('2001-05-02' as date) + interval 30 day) + group by + w_state,i_item_id + order by w_state,i_item_id +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query41.sql b/tools/tpcds-tools/queries/sf1/query41.sql new file mode 100644 index 000000000000000..65753b637718846 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query41.sql @@ -0,0 +1,49 @@ +select distinct(i_product_name) + from item i1 + where i_manufact_id between 704 and 704+40 + and (select count(*) as item_cnt + from item + where (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'forest' or i_color = 'lime') and + (i_units = 'Pallet' or i_units = 'Pound') and + (i_size = 'economy' or i_size = 'small') + ) or + (i_category = 'Women' and + (i_color = 'navy' or i_color = 'slate') and + (i_units = 'Gross' or i_units = 'Bunch') and + (i_size = 'extra large' or i_size = 'petite') + ) or + (i_category = 'Men' and + (i_color = 'powder' or i_color = 'sky') and + (i_units = 'Dozen' or i_units = 'Lb') and + (i_size = 'N/A' or i_size = 'large') + ) or + (i_category = 'Men' and + (i_color = 'maroon' or i_color = 'smoke') and + (i_units = 'Ounce' or i_units = 'Case') and + (i_size = 'economy' or i_size = 'small') + ))) or + (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'dark' or i_color = 'aquamarine') and + (i_units = 'Ton' or i_units = 'Tbl') and + (i_size = 'economy' or i_size = 'small') + ) or + (i_category = 'Women' and + (i_color = 'frosted' or i_color = 'plum') and + (i_units = 'Dram' or i_units = 'Box') and + (i_size = 'extra large' or i_size = 'petite') + ) or + (i_category = 'Men' and + (i_color = 'papaya' or i_color = 'peach') and + (i_units = 'Bundle' or i_units = 'Carton') and + (i_size = 'N/A' or i_size = 'large') + ) or + (i_category = 'Men' and + (i_color = 'firebrick' or i_color = 'sienna') and + (i_units = 'Cup' or i_units = 'Each') and + (i_size = 'economy' or i_size = 'small') + )))) > 0 + order by i_product_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query42.sql b/tools/tpcds-tools/queries/sf1/query42.sql new file mode 100644 index 000000000000000..ae388efd37e5738 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query42.sql @@ -0,0 +1,19 @@ +select dt.d_year + ,item.i_category_id + ,item.i_category + ,sum(ss_ext_sales_price) + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=11 + and dt.d_year=1998 + group by dt.d_year + ,item.i_category_id + ,item.i_category + order by sum(ss_ext_sales_price) desc,dt.d_year + ,item.i_category_id + ,item.i_category +limit 100 ; diff --git a/tools/tpcds-tools/queries/sf1/query43.sql b/tools/tpcds-tools/queries/sf1/query43.sql new file mode 100644 index 000000000000000..b1caee291694661 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query43.sql @@ -0,0 +1,16 @@ +select s_store_name, s_store_id, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from date_dim, store_sales, store + where d_date_sk = ss_sold_date_sk and + s_store_sk = ss_store_sk and + s_gmt_offset = -5 and + d_year = 2000 + group by s_store_name, s_store_id + order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query44.sql b/tools/tpcds-tools/queries/sf1/query44.sql new file mode 100644 index 000000000000000..294fca870fb09c5 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query44.sql @@ -0,0 +1,32 @@ +select asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing +from(select * + from (select item_sk,rank() over (order by rank_col asc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 4 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 4 + and ss_hdemo_sk is null + group by ss_store_sk))V1)V11 + where rnk < 11) asceding, + (select * + from (select item_sk,rank() over (order by rank_col desc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 4 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 4 + and ss_hdemo_sk is null + group by ss_store_sk))V2)V21 + where rnk < 11) descending, +item i1, +item i2 +where asceding.rnk = descending.rnk + and i1.i_item_sk=asceding.item_sk + and i2.i_item_sk=descending.item_sk +order by asceding.rnk +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query45.sql b/tools/tpcds-tools/queries/sf1/query45.sql new file mode 100644 index 000000000000000..32e490332a76396 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query45.sql @@ -0,0 +1,17 @@ +select ca_zip, ca_city, sum(ws_sales_price) + from web_sales, customer, customer_address, date_dim, item + where ws_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ws_item_sk = i_item_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792') + or + i_item_id in (select i_item_id + from item + where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) + ) + ) + and ws_sold_date_sk = d_date_sk + and d_qoy = 1 and d_year = 2000 + group by ca_zip, ca_city + order by ca_zip, ca_city + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query46.sql b/tools/tpcds-tools/queries/sf1/query46.sql new file mode 100644 index 000000000000000..b9390efcb107115 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query46.sql @@ -0,0 +1,32 @@ +select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and (household_demographics.hd_dep_count = 8 or + household_demographics.hd_vehicle_count= 0) + and date_dim.d_dow in (6,0) + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_city in ('Midway','Fairview','Fairview','Midway','Fairview') + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query47.sql b/tools/tpcds-tools/queries/sf1/query47.sql new file mode 100644 index 000000000000000..2b056c61502f7b7 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query47.sql @@ -0,0 +1,48 @@ +with v1 as( + select i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, + s_store_name, s_company_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + s_store_name, s_company_name + order by d_year, d_moy) rn + from item, store_sales, date_dim, store + where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + ( + d_year = 2000 or + ( d_year = 2000-1 and d_moy =12) or + ( d_year = 2000+1 and d_moy =1) + ) + group by i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy), + v2 as( + select v1.s_store_name, v1.s_company_name + ,v1.d_year + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1.s_store_name = v1_lag.s_store_name and + v1.s_store_name = v1_lead.s_store_name and + v1.s_company_name = v1_lag.s_company_name and + v1.s_company_name = v1_lead.s_company_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 2000 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, nsum + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query48.sql b/tools/tpcds-tools/queries/sf1/query48.sql new file mode 100644 index 000000000000000..e454f7780096f5f --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query48.sql @@ -0,0 +1,64 @@ +select sum (ss_quantity) + from store_sales, store, customer_demographics, customer_address, date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 2001 + and + ( + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'S' + and + cd_education_status = 'Secondary' + and + ss_sales_price between 100.00 and 150.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'M' + and + cd_education_status = '2 yr Degree' + and + ss_sales_price between 50.00 and 100.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'D' + and + cd_education_status = 'Advanced Degree' + and + ss_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('ND', 'NY', 'SD') + and ss_net_profit between 0 and 2000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('MD', 'GA', 'KS') + and ss_net_profit between 150 and 3000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('CO', 'MN', 'NC') + and ss_net_profit between 50 and 25000 + ) + ) +; diff --git a/tools/tpcds-tools/queries/sf1/query49.sql b/tools/tpcds-tools/queries/sf1/query49.sql new file mode 100644 index 000000000000000..5fdadcb1b62f713 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query49.sql @@ -0,0 +1,126 @@ +select channel, item, return_ratio, return_rank, currency_rank from + (select + 'web' as channel + ,web.item + ,web.return_ratio + ,web.return_rank + ,web.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select ws.ws_item_sk as item + ,(cast(sum(coalesce(wr.wr_return_quantity,0)) as decimal(15,4))/ + cast(sum(coalesce(ws.ws_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(wr.wr_return_amt,0)) as decimal(15,4))/ + cast(sum(coalesce(ws.ws_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + web_sales ws left outer join web_returns wr + on (ws.ws_order_number = wr.wr_order_number and + ws.ws_item_sk = wr.wr_item_sk) + ,date_dim + where + wr.wr_return_amt > 10000 + and ws.ws_net_profit > 1 + and ws.ws_net_paid > 0 + and ws.ws_quantity > 0 + and ws_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 11 + group by ws.ws_item_sk + ) in_web + ) web + where + ( + web.return_rank <= 10 + or + web.currency_rank <= 10 + ) + union + select + 'catalog' as channel + ,catalog.item + ,catalog.return_ratio + ,catalog.return_rank + ,catalog.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select + cs.cs_item_sk as item + ,(cast(sum(coalesce(cr.cr_return_quantity,0)) as decimal(15,4))/ + cast(sum(coalesce(cs.cs_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(cr.cr_return_amount,0)) as decimal(15,4))/ + cast(sum(coalesce(cs.cs_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + catalog_sales cs left outer join catalog_returns cr + on (cs.cs_order_number = cr.cr_order_number and + cs.cs_item_sk = cr.cr_item_sk) + ,date_dim + where + cr.cr_return_amount > 10000 + and cs.cs_net_profit > 1 + and cs.cs_net_paid > 0 + and cs.cs_quantity > 0 + and cs_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 11 + group by cs.cs_item_sk + ) in_cat + ) catalog + where + ( + catalog.return_rank <= 10 + or + catalog.currency_rank <=10 + ) + union + select + 'store' as channel + ,store.item + ,store.return_ratio + ,store.return_rank + ,store.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select sts.ss_item_sk as item + ,(cast(sum(coalesce(sr.sr_return_quantity,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(sr.sr_return_amt,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + store_sales sts left outer join store_returns sr + on (sts.ss_ticket_number = sr.sr_ticket_number and sts.ss_item_sk = sr.sr_item_sk) + ,date_dim + where + sr.sr_return_amt > 10000 + and sts.ss_net_profit > 1 + and sts.ss_net_paid > 0 + and sts.ss_quantity > 0 + and ss_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 11 + group by sts.ss_item_sk + ) in_store + ) store + where ( + store.return_rank <= 10 + or + store.currency_rank <= 10 + ) + ) + t order by 1,4,5,2 + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query5.sql b/tools/tpcds-tools/queries/sf1/query5.sql new file mode 100644 index 000000000000000..96c97b7c8a1636a --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query5.sql @@ -0,0 +1,125 @@ +with ssr as + (select s_store_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( select ss_store_sk as store_sk, + ss_sold_date_sk as date_sk, + ss_ext_sales_price as sales_price, + ss_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from store_sales + union all + select sr_store_sk as store_sk, + sr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + sr_return_amt as return_amt, + sr_net_loss as net_loss + from store_returns + ) salesreturns, + date_dim, + store + where date_sk = d_date_sk + and d_date between cast('2000-08-19' as date) + and (cast('2000-08-19' as date) + interval 14 day) + and store_sk = s_store_sk + group by s_store_id) + , + csr as + (select cp_catalog_page_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( select cs_catalog_page_sk as page_sk, + cs_sold_date_sk as date_sk, + cs_ext_sales_price as sales_price, + cs_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from catalog_sales + union all + select cr_catalog_page_sk as page_sk, + cr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + cr_return_amount as return_amt, + cr_net_loss as net_loss + from catalog_returns + ) salesreturns, + date_dim, + catalog_page + where date_sk = d_date_sk + and d_date between cast('2000-08-19' as date) + and (cast('2000-08-19' as date) + interval 14 day) + and page_sk = cp_catalog_page_sk + group by cp_catalog_page_id) + , + wsr as + (select web_site_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( select ws_web_site_sk as wsr_web_site_sk, + ws_sold_date_sk as date_sk, + ws_ext_sales_price as sales_price, + ws_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from web_sales + union all + select ws_web_site_sk as wsr_web_site_sk, + wr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + wr_return_amt as return_amt, + wr_net_loss as net_loss + from web_returns left outer join web_sales on + ( wr_item_sk = ws_item_sk + and wr_order_number = ws_order_number) + ) salesreturns, + date_dim, + web_site + where date_sk = d_date_sk + and d_date between cast('2000-08-19' as date) + and (cast('2000-08-19' as date) + interval 14 day) + and wsr_web_site_sk = web_site_sk + group by web_site_id) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , concat('store', s_store_id) id + , sales + , returns + , (profit - profit_loss) as profit + from ssr + union all + select 'catalog channel' as channel + , concat('catalog_page', cp_catalog_page_id) id + , sales + , returns + , (profit - profit_loss) as profit + from csr + union all + select 'web channel' as channel + , concat('web_site', web_site_id) id + , sales + , returns + , (profit - profit_loss) as profit + from wsr + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query50.sql b/tools/tpcds-tools/queries/sf1/query50.sql new file mode 100644 index 000000000000000..2a45d211cc3628f --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query50.sql @@ -0,0 +1,56 @@ +select + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 30) and + (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 60) and + (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 90) and + (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 120) then 1 else 0 end) as ">120 days" +from + store_sales + ,store_returns + ,store + ,date_dim d1 + ,date_dim d2 +where + d2.d_year = 2001 +and d2.d_moy = 8 +and ss_ticket_number = sr_ticket_number +and ss_item_sk = sr_item_sk +and ss_sold_date_sk = d1.d_date_sk +and sr_returned_date_sk = d2.d_date_sk +and ss_customer_sk = sr_customer_sk +and ss_store_sk = s_store_sk +group by + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip +order by s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query51.sql b/tools/tpcds-tools/queries/sf1/query51.sql new file mode 100644 index 000000000000000..fe90be989f8a416 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query51.sql @@ -0,0 +1,42 @@ +WITH web_v1 as ( +select + ws_item_sk item_sk, d_date, + sum(sum(ws_sales_price)) + over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from web_sales + ,date_dim +where ws_sold_date_sk=d_date_sk + and d_month_seq between 1212 and 1212+11 + and ws_item_sk is not NULL +group by ws_item_sk, d_date), +store_v1 as ( +select + ss_item_sk item_sk, d_date, + sum(sum(ss_sales_price)) + over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from store_sales + ,date_dim +where ss_sold_date_sk=d_date_sk + and d_month_seq between 1212 and 1212+11 + and ss_item_sk is not NULL +group by ss_item_sk, d_date) + select * +from (select item_sk + ,d_date + ,web_sales + ,store_sales + ,max(web_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) web_cumulative + ,max(store_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) store_cumulative + from (select case when web.item_sk is not null then web.item_sk else store.item_sk end item_sk + ,case when web.d_date is not null then web.d_date else store.d_date end d_date + ,web.cume_sales web_sales + ,store.cume_sales store_sales + from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk + and web.d_date = store.d_date) + )x )y +where web_cumulative > store_cumulative +order by item_sk + ,d_date +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query52.sql b/tools/tpcds-tools/queries/sf1/query52.sql new file mode 100644 index 000000000000000..2fce4c3fdf1c278 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query52.sql @@ -0,0 +1,19 @@ +select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_ext_sales_price) ext_price + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=12 + and dt.d_year=2000 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,ext_price desc + ,brand_id +limit 100 ; diff --git a/tools/tpcds-tools/queries/sf1/query53.sql b/tools/tpcds-tools/queries/sf1/query53.sql new file mode 100644 index 000000000000000..68feef7b4bae1fa --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query53.sql @@ -0,0 +1,25 @@ +select * from +(select i_manufact_id, +sum(ss_sales_price) sum_sales, +avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and +ss_sold_date_sk = d_date_sk and +ss_store_sk = s_store_sk and +d_month_seq in (1186,1186+1,1186+2,1186+3,1186+4,1186+5,1186+6,1186+7,1186+8,1186+9,1186+10,1186+11) and +((i_category in ('Books','Children','Electronics') and +i_class in ('personal','portable','reference','self-help') and +i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7', + 'exportiunivamalg #9','scholaramalgamalg #9')) +or(i_category in ('Women','Music','Men') and +i_class in ('accessories','classical','fragrances','pants') and +i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1', + 'importoamalg #1'))) +group by i_manufact_id, d_qoy ) tmp1 +where case when avg_quarterly_sales > 0 + then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales + else null end > 0.1 +order by avg_quarterly_sales, + sum_sales, + i_manufact_id +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query54.sql b/tools/tpcds-tools/queries/sf1/query54.sql new file mode 100644 index 000000000000000..f195c56ce66cf14 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query54.sql @@ -0,0 +1,53 @@ +with my_customers as ( + select distinct c_customer_sk + , c_current_addr_sk + from + ( select cs_sold_date_sk sold_date_sk, + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + from catalog_sales + union all + select ws_sold_date_sk sold_date_sk, + ws_bill_customer_sk customer_sk, + ws_item_sk item_sk + from web_sales + ) cs_or_ws_sales, + item, + date_dim, + customer + where sold_date_sk = d_date_sk + and item_sk = i_item_sk + and i_category = 'Music' + and i_class = 'country' + and c_customer_sk = cs_or_ws_sales.customer_sk + and d_moy = 1 + and d_year = 1999 + ) + , my_revenue as ( + select c_customer_sk, + sum(ss_ext_sales_price) as revenue + from my_customers, + store_sales, + customer_address, + store, + date_dim + where c_current_addr_sk = ca_address_sk + and ca_county = s_county + and ca_state = s_state + and ss_sold_date_sk = d_date_sk + and c_customer_sk = ss_customer_sk + and d_month_seq between (select distinct d_month_seq+1 + from date_dim where d_year = 1999 and d_moy = 1) + and (select distinct d_month_seq+3 + from date_dim where d_year = 1999 and d_moy = 1) + group by c_customer_sk + ) + , segments as + (select cast((revenue/50) as int) as segment + from my_revenue + ) + select segment, count(*) as num_customers, segment*50 as segment_base + from segments + group by segment + order by segment, num_customers + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query55.sql b/tools/tpcds-tools/queries/sf1/query55.sql new file mode 100644 index 000000000000000..7248468b83c3546 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query55.sql @@ -0,0 +1,11 @@ +select i_brand_id brand_id, i_brand brand, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=52 + and d_moy=11 + and d_year=2000 + group by i_brand, i_brand_id + order by ext_price desc, i_brand_id +limit 100 ; diff --git a/tools/tpcds-tools/queries/sf1/query56.sql b/tools/tpcds-tools/queries/sf1/query56.sql new file mode 100644 index 000000000000000..8b8bfa96ce4ed8d --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query56.sql @@ -0,0 +1,66 @@ +with ss as ( + select i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where i_item_id in (select + i_item_id +from item +where i_color in ('powder','orchid','pink')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 3 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + cs as ( + select i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('powder','orchid','pink')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 3 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + ws as ( + select i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('powder','orchid','pink')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 3 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id) + select i_item_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by total_sales, + i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query57.sql b/tools/tpcds-tools/queries/sf1/query57.sql new file mode 100644 index 000000000000000..6c08d2623a16536 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query57.sql @@ -0,0 +1,45 @@ +with v1 as( + select i_category, i_brand, + cc_name, + d_year, d_moy, + sum(cs_sales_price) sum_sales, + avg(sum(cs_sales_price)) over + (partition by i_category, i_brand, + cc_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + cc_name + order by d_year, d_moy) rn + from item, catalog_sales, date_dim, call_center + where cs_item_sk = i_item_sk and + cs_sold_date_sk = d_date_sk and + cc_call_center_sk= cs_call_center_sk and + ( + d_year = 2001 or + ( d_year = 2001-1 and d_moy =12) or + ( d_year = 2001+1 and d_moy =1) + ) + group by i_category, i_brand, + cc_name , d_year, d_moy), + v2 as( + select v1.i_category, v1.i_brand, v1.cc_name + ,v1.d_year + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1. cc_name = v1_lag. cc_name and + v1. cc_name = v1_lead. cc_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 2001 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, avg_monthly_sales + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query58.sql b/tools/tpcds-tools/queries/sf1/query58.sql new file mode 100644 index 000000000000000..f45b29749f0a4d5 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query58.sql @@ -0,0 +1,62 @@ +with ss_items as + (select i_item_id item_id + ,sum(ss_ext_sales_price) ss_item_rev + from store_sales + ,item + ,date_dim + where ss_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '2001-06-16')) + and ss_sold_date_sk = d_date_sk + group by i_item_id), + cs_items as + (select i_item_id item_id + ,sum(cs_ext_sales_price) cs_item_rev + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '2001-06-16')) + and cs_sold_date_sk = d_date_sk + group by i_item_id), + ws_items as + (select i_item_id item_id + ,sum(ws_ext_sales_price) ws_item_rev + from web_sales + ,item + ,date_dim + where ws_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq =(select d_week_seq + from date_dim + where d_date = '2001-06-16')) + and ws_sold_date_sk = d_date_sk + group by i_item_id) + select ss_items.item_id + ,ss_item_rev + ,ss_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ss_dev + ,cs_item_rev + ,cs_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 cs_dev + ,ws_item_rev + ,ws_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ws_dev + ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average + from ss_items,cs_items,ws_items + where ss_items.item_id=cs_items.item_id + and ss_items.item_id=ws_items.item_id + and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + order by item_id + ,ss_item_rev + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query59.sql b/tools/tpcds-tools/queries/sf1/query59.sql new file mode 100644 index 000000000000000..cb5e36ae4e3dc2a --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query59.sql @@ -0,0 +1,41 @@ +with wss as + (select d_week_seq, + ss_store_sk, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + group by d_week_seq,ss_store_sk + ) + select s_store_name1,s_store_id1,d_week_seq1 + ,sun_sales1/sun_sales2,mon_sales1/mon_sales2 + ,tue_sales1/tue_sales2,wed_sales1/wed_sales2,thu_sales1/thu_sales2 + ,fri_sales1/fri_sales2,sat_sales1/sat_sales2 + from + (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1 + ,s_store_id s_store_id1,sun_sales sun_sales1 + ,mon_sales mon_sales1,tue_sales tue_sales1 + ,wed_sales wed_sales1,thu_sales thu_sales1 + ,fri_sales fri_sales1,sat_sales sat_sales1 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + ss_store_sk = s_store_sk and + d_month_seq between 1195 and 1195 + 11) y, + (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2 + ,s_store_id s_store_id2,sun_sales sun_sales2 + ,mon_sales mon_sales2,tue_sales tue_sales2 + ,wed_sales wed_sales2,thu_sales thu_sales2 + ,fri_sales fri_sales2,sat_sales sat_sales2 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + ss_store_sk = s_store_sk and + d_month_seq between 1195+ 12 and 1195 + 23) x + where s_store_id1=s_store_id2 + and d_week_seq1=d_week_seq2-52 + order by s_store_name1,s_store_id1,d_week_seq1 +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query6.sql b/tools/tpcds-tools/queries/sf1/query6.sql new file mode 100644 index 000000000000000..00decd2cdc48a55 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query6.sql @@ -0,0 +1,23 @@ +select a.ca_state state, count(*) cnt + from customer_address a + ,customer c + ,store_sales s + ,date_dim d + ,item i + where a.ca_address_sk = c.c_current_addr_sk + and c.c_customer_sk = s.ss_customer_sk + and s.ss_sold_date_sk = d.d_date_sk + and s.ss_item_sk = i.i_item_sk + and d.d_month_seq = + (select distinct (d_month_seq) + from date_dim + where d_year = 2002 + and d_moy = 3 ) + and i.i_current_price > 1.2 * + (select avg(j.i_current_price) + from item j + where j.i_category = i.i_category) + group by a.ca_state + having count(*) >= 10 + order by cnt, a.ca_state + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query60.sql b/tools/tpcds-tools/queries/sf1/query60.sql new file mode 100644 index 000000000000000..9efaa98622e94ad --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query60.sql @@ -0,0 +1,75 @@ +with ss as ( + select + i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Jewelry')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 10 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_item_id), + cs as ( + select + i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Jewelry')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 10 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_item_id), + ws as ( + select + i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Jewelry')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 10 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_item_id) + select + i_item_id +,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by i_item_id + ,total_sales + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query61.sql b/tools/tpcds-tools/queries/sf1/query61.sql new file mode 100644 index 000000000000000..c9310e3c0650c36 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query61.sql @@ -0,0 +1,41 @@ +select promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100 +from + (select sum(ss_ext_sales_price) promotions + from store_sales + ,store + ,promotion + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_promo_sk = p_promo_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -7 + and i_category = 'Home' + and (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y') + and s_gmt_offset = -7 + and d_year = 2000 + and d_moy = 12) promotional_sales, + (select sum(ss_ext_sales_price) total + from store_sales + ,store + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -7 + and i_category = 'Home' + and s_gmt_offset = -7 + and d_year = 2000 + and d_moy = 12) all_sales +order by promotions, total +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query62.sql b/tools/tpcds-tools/queries/sf1/query62.sql new file mode 100644 index 000000000000000..4475549eaca2348 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query62.sql @@ -0,0 +1,32 @@ +select + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 30) and + (ws_ship_date_sk - ws_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 60) and + (ws_ship_date_sk - ws_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 90) and + (ws_ship_date_sk - ws_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 120) then 1 else 0 end) as ">120 days" +from + web_sales + ,warehouse + ,ship_mode + ,web_site + ,date_dim +where + d_month_seq between 1223 and 1223 + 11 +and ws_ship_date_sk = d_date_sk +and ws_warehouse_sk = w_warehouse_sk +and ws_ship_mode_sk = sm_ship_mode_sk +and ws_web_site_sk = web_site_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,web_name +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query63.sql b/tools/tpcds-tools/queries/sf1/query63.sql new file mode 100644 index 000000000000000..8772871eb6eb3aa --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query63.sql @@ -0,0 +1,26 @@ +select * +from (select i_manager_id + ,sum(ss_sales_price) sum_sales + ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales + from item + ,store_sales + ,date_dim + ,store + where ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and d_month_seq in (1222,1222+1,1222+2,1222+3,1222+4,1222+5,1222+6,1222+7,1222+8,1222+9,1222+10,1222+11) + and (( i_category in ('Books','Children','Electronics') + and i_class in ('personal','portable','reference','self-help') + and i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7', + 'exportiunivamalg #9','scholaramalgamalg #9')) + or( i_category in ('Women','Music','Men') + and i_class in ('accessories','classical','fragrances','pants') + and i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1', + 'importoamalg #1'))) +group by i_manager_id, d_moy) tmp1 +where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 +order by i_manager_id + ,avg_monthly_sales + ,sum_sales +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query64.sql b/tools/tpcds-tools/queries/sf1/query64.sql new file mode 100644 index 000000000000000..21e6c28590f532d --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query64.sql @@ -0,0 +1,118 @@ +with cs_ui as + (select cs_item_sk + ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund + from catalog_sales + ,catalog_returns + where cs_item_sk = cr_item_sk + and cs_order_number = cr_order_number + group by cs_item_sk + having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)), +cross_sales as + (select i_product_name product_name + ,i_item_sk item_sk + ,s_store_name store_name + ,s_zip store_zip + ,ad1.ca_street_number b_street_number + ,ad1.ca_street_name b_street_name + ,ad1.ca_city b_city + ,ad1.ca_zip b_zip + ,ad2.ca_street_number c_street_number + ,ad2.ca_street_name c_street_name + ,ad2.ca_city c_city + ,ad2.ca_zip c_zip + ,d1.d_year as syear + ,d2.d_year as fsyear + ,d3.d_year s2year + ,count(*) cnt + ,sum(ss_wholesale_cost) s1 + ,sum(ss_list_price) s2 + ,sum(ss_coupon_amt) s3 + FROM store_sales + ,store_returns + ,cs_ui + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,customer + ,customer_demographics cd1 + ,customer_demographics cd2 + ,promotion + ,household_demographics hd1 + ,household_demographics hd2 + ,customer_address ad1 + ,customer_address ad2 + ,income_band ib1 + ,income_band ib2 + ,item + WHERE ss_store_sk = s_store_sk AND + ss_sold_date_sk = d1.d_date_sk AND + ss_customer_sk = c_customer_sk AND + ss_cdemo_sk= cd1.cd_demo_sk AND + ss_hdemo_sk = hd1.hd_demo_sk AND + ss_addr_sk = ad1.ca_address_sk and + ss_item_sk = i_item_sk and + ss_item_sk = sr_item_sk and + ss_ticket_number = sr_ticket_number and + ss_item_sk = cs_ui.cs_item_sk and + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_hdemo_sk = hd2.hd_demo_sk AND + c_current_addr_sk = ad2.ca_address_sk and + c_first_sales_date_sk = d2.d_date_sk and + c_first_shipto_date_sk = d3.d_date_sk and + ss_promo_sk = p_promo_sk and + hd1.hd_income_band_sk = ib1.ib_income_band_sk and + hd2.hd_income_band_sk = ib2.ib_income_band_sk and + cd1.cd_marital_status <> cd2.cd_marital_status and + i_color in ('orange','lace','lawn','misty','blush','pink') and + i_current_price between 48 and 48 + 10 and + i_current_price between 48 + 1 and 48 + 15 +group by i_product_name + ,i_item_sk + ,s_store_name + ,s_zip + ,ad1.ca_street_number + ,ad1.ca_street_name + ,ad1.ca_city + ,ad1.ca_zip + ,ad2.ca_street_number + ,ad2.ca_street_name + ,ad2.ca_city + ,ad2.ca_zip + ,d1.d_year + ,d2.d_year + ,d3.d_year +) +select cs1.product_name + ,cs1.store_name + ,cs1.store_zip + ,cs1.b_street_number + ,cs1.b_street_name + ,cs1.b_city + ,cs1.b_zip + ,cs1.c_street_number + ,cs1.c_street_name + ,cs1.c_city + ,cs1.c_zip + ,cs1.syear + ,cs1.cnt + ,cs1.s1 as s11 + ,cs1.s2 as s21 + ,cs1.s3 as s31 + ,cs2.s1 as s12 + ,cs2.s2 as s22 + ,cs2.s3 as s32 + ,cs2.syear + ,cs2.cnt +from cross_sales cs1,cross_sales cs2 +where cs1.item_sk=cs2.item_sk and + cs1.syear = 1999 and + cs2.syear = 1999 + 1 and + cs2.cnt <= cs1.cnt and + cs1.store_name = cs2.store_name and + cs1.store_zip = cs2.store_zip +order by cs1.product_name + ,cs1.store_name + ,cs2.cnt + ,cs1.s1 + ,cs2.s1; diff --git a/tools/tpcds-tools/queries/sf1/query65.sql b/tools/tpcds-tools/queries/sf1/query65.sql new file mode 100644 index 000000000000000..62fab6d16eacb82 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query65.sql @@ -0,0 +1,26 @@ +select + s_store_name, + i_item_desc, + sc.revenue, + i_current_price, + i_wholesale_cost, + i_brand + from store, item, + (select ss_store_sk, avg(revenue) as ave + from + (select ss_store_sk, ss_item_sk, + sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1176 and 1176+11 + group by ss_store_sk, ss_item_sk) sa + group by ss_store_sk) sb, + (select ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1176 and 1176+11 + group by ss_store_sk, ss_item_sk) sc + where sb.ss_store_sk = sc.ss_store_sk and + sc.revenue <= 0.1 * sb.ave and + s_store_sk = sc.ss_store_sk and + i_item_sk = sc.ss_item_sk + order by s_store_name, i_item_desc +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query66.sql b/tools/tpcds-tools/queries/sf1/query66.sql new file mode 100644 index 000000000000000..732608eba51e289 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query66.sql @@ -0,0 +1,217 @@ +select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,ship_carriers + ,year + ,sum(jan_sales) as jan_sales + ,sum(feb_sales) as feb_sales + ,sum(mar_sales) as mar_sales + ,sum(apr_sales) as apr_sales + ,sum(may_sales) as may_sales + ,sum(jun_sales) as jun_sales + ,sum(jul_sales) as jul_sales + ,sum(aug_sales) as aug_sales + ,sum(sep_sales) as sep_sales + ,sum(oct_sales) as oct_sales + ,sum(nov_sales) as nov_sales + ,sum(dec_sales) as dec_sales + ,sum(jan_sales/w_warehouse_sq_ft) as jan_sales_per_sq_foot + ,sum(feb_sales/w_warehouse_sq_ft) as feb_sales_per_sq_foot + ,sum(mar_sales/w_warehouse_sq_ft) as mar_sales_per_sq_foot + ,sum(apr_sales/w_warehouse_sq_ft) as apr_sales_per_sq_foot + ,sum(may_sales/w_warehouse_sq_ft) as may_sales_per_sq_foot + ,sum(jun_sales/w_warehouse_sq_ft) as jun_sales_per_sq_foot + ,sum(jul_sales/w_warehouse_sq_ft) as jul_sales_per_sq_foot + ,sum(aug_sales/w_warehouse_sq_ft) as aug_sales_per_sq_foot + ,sum(sep_sales/w_warehouse_sq_ft) as sep_sales_per_sq_foot + ,sum(oct_sales/w_warehouse_sq_ft) as oct_sales_per_sq_foot + ,sum(nov_sales/w_warehouse_sq_ft) as nov_sales_per_sq_foot + ,sum(dec_sales/w_warehouse_sq_ft) as dec_sales_per_sq_foot + ,sum(jan_net) as jan_net + ,sum(feb_net) as feb_net + ,sum(mar_net) as mar_net + ,sum(apr_net) as apr_net + ,sum(may_net) as may_net + ,sum(jun_net) as jun_net + ,sum(jul_net) as jul_net + ,sum(aug_net) as aug_net + ,sum(sep_net) as sep_net + ,sum(oct_net) as oct_net + ,sum(nov_net) as nov_net + ,sum(dec_net) as dec_net + from ( + select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,concat(concat('ORIENTAL ', ','), ' BOXBUNDLES') as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then ws_ext_sales_price* ws_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then ws_ext_sales_price* ws_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then ws_ext_sales_price* ws_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then ws_ext_sales_price* ws_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then ws_ext_sales_price* ws_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then ws_ext_sales_price* ws_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then ws_ext_sales_price* ws_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then ws_ext_sales_price* ws_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then ws_ext_sales_price* ws_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then ws_ext_sales_price* ws_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then ws_ext_sales_price* ws_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then ws_ext_sales_price* ws_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as dec_net + from + web_sales + ,warehouse + ,date_dim + ,time_dim + ,ship_mode + where + ws_warehouse_sk = w_warehouse_sk + and ws_sold_date_sk = d_date_sk + and ws_sold_time_sk = t_time_sk + and ws_ship_mode_sk = sm_ship_mode_sk + and d_year = 2001 + and t_time between 42970 and 42970+28800 + and sm_carrier in ('ORIENTAL','BOXBUNDLES') + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,d_year + union all + select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,concat(concat('ORIENTAL ', ','), ' BOXBUNDLES') as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then cs_ext_list_price* cs_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then cs_ext_list_price* cs_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then cs_ext_list_price* cs_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then cs_ext_list_price* cs_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then cs_ext_list_price* cs_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then cs_ext_list_price* cs_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then cs_ext_list_price* cs_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then cs_ext_list_price* cs_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then cs_ext_list_price* cs_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then cs_ext_list_price* cs_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then cs_ext_list_price* cs_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then cs_ext_list_price* cs_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then cs_net_paid * cs_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then cs_net_paid * cs_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then cs_net_paid * cs_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then cs_net_paid * cs_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then cs_net_paid * cs_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then cs_net_paid * cs_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then cs_net_paid * cs_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then cs_net_paid * cs_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then cs_net_paid * cs_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then cs_net_paid * cs_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then cs_net_paid * cs_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then cs_net_paid * cs_quantity else 0 end) as dec_net + from + catalog_sales + ,warehouse + ,date_dim + ,time_dim + ,ship_mode + where + cs_warehouse_sk = w_warehouse_sk + and cs_sold_date_sk = d_date_sk + and cs_sold_time_sk = t_time_sk + and cs_ship_mode_sk = sm_ship_mode_sk + and d_year = 2001 + and t_time between 42970 AND 42970+28800 + and sm_carrier in ('ORIENTAL','BOXBUNDLES') + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,d_year + ) x + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,ship_carriers + ,year + order by w_warehouse_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query67.sql b/tools/tpcds-tools/queries/sf1/query67.sql new file mode 100644 index 000000000000000..af49b1e94f02d37 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query67.sql @@ -0,0 +1,41 @@ +select * +from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rank() over (partition by i_category order by sumsales desc) rk + from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sum(coalesce(ss_sales_price*ss_quantity,0)) sumsales + from store_sales + ,date_dim + ,store + ,item + where ss_sold_date_sk=d_date_sk + and ss_item_sk=i_item_sk + and ss_store_sk = s_store_sk + and d_month_seq between 1217 and 1217+11 + group by rollup(i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy,s_store_id))dw1) dw2 +where rk <= 100 +order by i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rk +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query68.sql b/tools/tpcds-tools/queries/sf1/query68.sql new file mode 100644 index 000000000000000..1f742baef74d8da --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query68.sql @@ -0,0 +1,39 @@ +select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,extended_price + ,extended_tax + ,list_price + from (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_ext_sales_price) extended_price + ,sum(ss_ext_list_price) list_price + ,sum(ss_ext_tax) extended_tax + from store_sales + ,date_dim + ,store + ,household_demographics + ,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_dep_count = 3 or + household_demographics.hd_vehicle_count= 4) + and date_dim.d_year in (1998,1998+1,1998+2) + and store.s_city in ('Fairview','Midway') + group by ss_ticket_number + ,ss_customer_sk + ,ss_addr_sk,ca_city) dn + ,customer + ,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,ss_ticket_number + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query69.sql b/tools/tpcds-tools/queries/sf1/query69.sql new file mode 100644 index 000000000000000..f5473324694bf6f --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query69.sql @@ -0,0 +1,44 @@ +select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_state in ('IL','TX','ME') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 1 and 1+2) and + (not exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 1 and 1+2) and + not exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 1 and 1+2)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query7.sql b/tools/tpcds-tools/queries/sf1/query7.sql new file mode 100644 index 000000000000000..c4b33b79d18b8a8 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query7.sql @@ -0,0 +1,18 @@ +select i_item_id, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, item, promotion + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_cdemo_sk = cd_demo_sk and + ss_promo_sk = p_promo_sk and + cd_gender = 'F' and + cd_marital_status = 'W' and + cd_education_status = 'College' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 2001 + group by i_item_id + order by i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query70.sql b/tools/tpcds-tools/queries/sf1/query70.sql new file mode 100644 index 000000000000000..21b90f20ad5355d --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query70.sql @@ -0,0 +1,35 @@ +select + sum(ss_net_profit) as total_sum + ,s_state + ,s_county + ,grouping(s_state)+grouping(s_county) as lochierarchy + ,rank() over ( + partition by grouping(s_state)+grouping(s_county), + case when grouping(s_county) = 0 then s_state end + order by sum(ss_net_profit) desc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,store + where + d1.d_month_seq between 1220 and 1220+11 + and d1.d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + and s_state in + ( select s_state + from (select s_state as s_state, + rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking + from store_sales, store, date_dim + where d_month_seq between 1220 and 1220+11 + and d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + group by s_state + ) tmp1 + where ranking <= 5 + ) + group by rollup(s_state,s_county) + order by + lochierarchy desc + ,case when lochierarchy = 0 then s_state end + ,rank_within_parent + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query71.sql b/tools/tpcds-tools/queries/sf1/query71.sql new file mode 100644 index 000000000000000..234ec201a194dab --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query71.sql @@ -0,0 +1,37 @@ +select i_brand_id brand_id, i_brand brand,t_hour,t_minute, + sum(ext_price) ext_price + from item, (select ws_ext_sales_price as ext_price, + ws_sold_date_sk as sold_date_sk, + ws_item_sk as sold_item_sk, + ws_sold_time_sk as time_sk + from web_sales,date_dim + where d_date_sk = ws_sold_date_sk + and d_moy=12 + and d_year=2002 + union all + select cs_ext_sales_price as ext_price, + cs_sold_date_sk as sold_date_sk, + cs_item_sk as sold_item_sk, + cs_sold_time_sk as time_sk + from catalog_sales,date_dim + where d_date_sk = cs_sold_date_sk + and d_moy=12 + and d_year=2002 + union all + select ss_ext_sales_price as ext_price, + ss_sold_date_sk as sold_date_sk, + ss_item_sk as sold_item_sk, + ss_sold_time_sk as time_sk + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + and d_moy=12 + and d_year=2002 + ) tmp,time_dim + where + sold_item_sk = i_item_sk + and i_manager_id=1 + and time_sk = t_time_sk + and (t_meal_time = 'breakfast' or t_meal_time = 'dinner') + group by i_brand, i_brand_id,t_hour,t_minute + order by ext_price desc, i_brand_id + ; diff --git a/tools/tpcds-tools/queries/sf1/query72.sql b/tools/tpcds-tools/queries/sf1/query72.sql new file mode 100644 index 000000000000000..6252e897e3ee1db --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query72.sql @@ -0,0 +1,26 @@ +select i_item_desc + ,w_warehouse_name + ,d1.d_week_seq + ,sum(case when p_promo_sk is null then 1 else 0 end) no_promo + ,sum(case when p_promo_sk is not null then 1 else 0 end) promo + ,count(*) total_cnt +from catalog_sales +join inventory on (cs_item_sk = inv_item_sk) +join warehouse on (w_warehouse_sk=inv_warehouse_sk) +join item on (i_item_sk = cs_item_sk) +join customer_demographics on (cs_bill_cdemo_sk = cd_demo_sk) +join household_demographics on (cs_bill_hdemo_sk = hd_demo_sk) +join date_dim d1 on (cs_sold_date_sk = d1.d_date_sk) +join date_dim d2 on (inv_date_sk = d2.d_date_sk) +join date_dim d3 on (cs_ship_date_sk = d3.d_date_sk) +left outer join promotion on (cs_promo_sk=p_promo_sk) +left outer join catalog_returns on (cr_item_sk = cs_item_sk and cr_order_number = cs_order_number) +where d1.d_week_seq = d2.d_week_seq + and inv_quantity_on_hand < cs_quantity + and (d3.d_date > (d1.d_date + INTERVAL '5' DAY)) + and hd_buy_potential = '1001-5000' + and d1.d_year = 1998 + and cd_marital_status = 'S' +group by i_item_desc,w_warehouse_name,d1.d_week_seq +order by total_cnt desc, i_item_desc, w_warehouse_name, d_week_seq +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query73.sql b/tools/tpcds-tools/queries/sf1/query73.sql new file mode 100644 index 000000000000000..85282bd0867c3ec --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query73.sql @@ -0,0 +1,25 @@ +select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_buy_potential = '1001-5000' or + household_demographics.hd_buy_potential = '5001-10000') + and household_demographics.hd_vehicle_count > 0 + and case when household_demographics.hd_vehicle_count > 0 then + household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1 + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County') + group by ss_ticket_number,ss_customer_sk) dj,customer + where ss_customer_sk = c_customer_sk + and cnt between 1 and 5 + order by cnt desc, c_last_name asc; diff --git a/tools/tpcds-tools/queries/sf1/query74.sql b/tools/tpcds-tools/queries/sf1/query74.sql new file mode 100644 index 000000000000000..741fa29be48919c --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query74.sql @@ -0,0 +1,58 @@ +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,max(ss_net_paid) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (1999,1999+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,max(ws_net_paid) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + and d_year in (1999,1999+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + ) + select + t_s_secyear.customer_id, t_s_secyear.customer_first_name, t_s_secyear.customer_last_name + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.year = 1999 + and t_s_secyear.year = 1999+1 + and t_w_firstyear.year = 1999 + and t_w_secyear.year = 1999+1 + and t_s_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end + order by 1,3,2 +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query75.sql b/tools/tpcds-tools/queries/sf1/query75.sql new file mode 100644 index 000000000000000..cce4cf2c68394fa --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query75.sql @@ -0,0 +1,67 @@ +WITH all_sales AS ( + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,SUM(sales_cnt) AS sales_cnt + ,SUM(sales_amt) AS sales_amt + FROM (SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt + ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt + FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk + JOIN date_dim ON d_date_sk=cs_sold_date_sk + LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number + AND cs_item_sk=cr_item_sk) + WHERE i_category='Sports' + UNION + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt + ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt + FROM store_sales JOIN item ON i_item_sk=ss_item_sk + JOIN date_dim ON d_date_sk=ss_sold_date_sk + LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number + AND ss_item_sk=sr_item_sk) + WHERE i_category='Sports' + UNION + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt + ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt + FROM web_sales JOIN item ON i_item_sk=ws_item_sk + JOIN date_dim ON d_date_sk=ws_sold_date_sk + LEFT JOIN web_returns ON (ws_order_number=wr_order_number + AND ws_item_sk=wr_item_sk) + WHERE i_category='Sports') sales_detail + GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) + SELECT prev_yr.d_year AS prev_year + ,curr_yr.d_year AS year + ,curr_yr.i_brand_id + ,curr_yr.i_class_id + ,curr_yr.i_category_id + ,curr_yr.i_manufact_id + ,prev_yr.sales_cnt AS prev_yr_cnt + ,curr_yr.sales_cnt AS curr_yr_cnt + ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff + ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff + FROM all_sales curr_yr, all_sales prev_yr + WHERE curr_yr.i_brand_id=prev_yr.i_brand_id + AND curr_yr.i_class_id=prev_yr.i_class_id + AND curr_yr.i_category_id=prev_yr.i_category_id + AND curr_yr.i_manufact_id=prev_yr.i_manufact_id + AND curr_yr.d_year=2002 + AND prev_yr.d_year=2002-1 + AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9 + ORDER BY sales_cnt_diff,sales_amt_diff + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query76.sql b/tools/tpcds-tools/queries/sf1/query76.sql new file mode 100644 index 000000000000000..837387fec6d8692 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query76.sql @@ -0,0 +1,21 @@ +select channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM ( + SELECT 'store' as channel, 'ss_customer_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price + FROM store_sales, item, date_dim + WHERE ss_customer_sk IS NULL + AND ss_sold_date_sk=d_date_sk + AND ss_item_sk=i_item_sk + UNION ALL + SELECT 'web' as channel, 'ws_promo_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price + FROM web_sales, item, date_dim + WHERE ws_promo_sk IS NULL + AND ws_sold_date_sk=d_date_sk + AND ws_item_sk=i_item_sk + UNION ALL + SELECT 'catalog' as channel, 'cs_bill_customer_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price + FROM catalog_sales, item, date_dim + WHERE cs_bill_customer_sk IS NULL + AND cs_sold_date_sk=d_date_sk + AND cs_item_sk=i_item_sk) foo +GROUP BY channel, col_name, d_year, d_qoy, i_category +ORDER BY channel, col_name, d_year, d_qoy, i_category +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query77.sql b/tools/tpcds-tools/queries/sf1/query77.sql new file mode 100644 index 000000000000000..cd4805724ff2f8e --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query77.sql @@ -0,0 +1,105 @@ +with ss as + (select s_store_sk, + sum(ss_ext_sales_price) as sales, + sum(ss_net_profit) as profit + from store_sales, + date_dim, + store + where ss_sold_date_sk = d_date_sk + and d_date between cast('2000-08-10' as date) + and (cast('2000-08-10' as date) + interval 30 day) + and ss_store_sk = s_store_sk + group by s_store_sk) + , + sr as + (select s_store_sk, + sum(sr_return_amt) as returns, + sum(sr_net_loss) as profit_loss + from store_returns, + date_dim, + store + where sr_returned_date_sk = d_date_sk + and d_date between cast('2000-08-10' as date) + and (cast('2000-08-10' as date) + interval 30 day) + and sr_store_sk = s_store_sk + group by s_store_sk), + cs as + (select cs_call_center_sk, + sum(cs_ext_sales_price) as sales, + sum(cs_net_profit) as profit + from catalog_sales, + date_dim + where cs_sold_date_sk = d_date_sk + and d_date between cast('2000-08-10' as date) + and (cast('2000-08-10' as date) + interval 30 day) + group by cs_call_center_sk + ), + cr as + (select cr_call_center_sk, + sum(cr_return_amount) as returns, + sum(cr_net_loss) as profit_loss + from catalog_returns, + date_dim + where cr_returned_date_sk = d_date_sk + and d_date between cast('2000-08-10' as date) + and (cast('2000-08-10' as date) + interval 30 day) + group by cr_call_center_sk + ), + ws as + ( select wp_web_page_sk, + sum(ws_ext_sales_price) as sales, + sum(ws_net_profit) as profit + from web_sales, + date_dim, + web_page + where ws_sold_date_sk = d_date_sk + and d_date between cast('2000-08-10' as date) + and (cast('2000-08-10' as date) + interval 30 day) + and ws_web_page_sk = wp_web_page_sk + group by wp_web_page_sk), + wr as + (select wp_web_page_sk, + sum(wr_return_amt) as returns, + sum(wr_net_loss) as profit_loss + from web_returns, + date_dim, + web_page + where wr_returned_date_sk = d_date_sk + and d_date between cast('2000-08-10' as date) + and (cast('2000-08-10' as date) + interval 30 day) + and wr_web_page_sk = wp_web_page_sk + group by wp_web_page_sk) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , ss.s_store_sk as id + , sales + , coalesce(returns, 0) as returns + , (profit - coalesce(profit_loss,0)) as profit + from ss left join sr + on ss.s_store_sk = sr.s_store_sk + union all + select 'catalog channel' as channel + , cs_call_center_sk as id + , sales + , returns + , (profit - profit_loss) as profit + from cs + , cr + union all + select 'web channel' as channel + , ws.wp_web_page_sk as id + , sales + , coalesce(returns, 0) returns + , (profit - coalesce(profit_loss,0)) as profit + from ws left join wr + on ws.wp_web_page_sk = wr.wp_web_page_sk + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query78.sql b/tools/tpcds-tools/queries/sf1/query78.sql new file mode 100644 index 000000000000000..9a827172b0f8686 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query78.sql @@ -0,0 +1,55 @@ +with ws as + (select d_year AS ws_sold_year, ws_item_sk, + ws_bill_customer_sk ws_customer_sk, + sum(ws_quantity) ws_qty, + sum(ws_wholesale_cost) ws_wc, + sum(ws_sales_price) ws_sp + from web_sales + left join web_returns on wr_order_number=ws_order_number and ws_item_sk=wr_item_sk + join date_dim on ws_sold_date_sk = d_date_sk + where wr_order_number is null + group by d_year, ws_item_sk, ws_bill_customer_sk + ), +cs as + (select d_year AS cs_sold_year, cs_item_sk, + cs_bill_customer_sk cs_customer_sk, + sum(cs_quantity) cs_qty, + sum(cs_wholesale_cost) cs_wc, + sum(cs_sales_price) cs_sp + from catalog_sales + left join catalog_returns on cr_order_number=cs_order_number and cs_item_sk=cr_item_sk + join date_dim on cs_sold_date_sk = d_date_sk + where cr_order_number is null + group by d_year, cs_item_sk, cs_bill_customer_sk + ), +ss as + (select d_year AS ss_sold_year, ss_item_sk, + ss_customer_sk, + sum(ss_quantity) ss_qty, + sum(ss_wholesale_cost) ss_wc, + sum(ss_sales_price) ss_sp + from store_sales + left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk + join date_dim on ss_sold_date_sk = d_date_sk + where sr_ticket_number is null + group by d_year, ss_item_sk, ss_customer_sk + ) + select +ss_customer_sk, +round(ss_qty/(coalesce(ws_qty,0)+coalesce(cs_qty,0)),2) ratio, +ss_qty store_qty, ss_wc store_wholesale_cost, ss_sp store_sales_price, +coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty, +coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost, +coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price +from ss +left join ws on (ws_sold_year=ss_sold_year and ws_item_sk=ss_item_sk and ws_customer_sk=ss_customer_sk) +left join cs on (cs_sold_year=ss_sold_year and cs_item_sk=ss_item_sk and cs_customer_sk=ss_customer_sk) +where (coalesce(ws_qty,0)>0 or coalesce(cs_qty, 0)>0) and ss_sold_year=1998 +order by + ss_customer_sk, + ss_qty desc, ss_wc desc, ss_sp desc, + other_chan_qty, + other_chan_wholesale_cost, + other_chan_sales_price, + ratio +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query79.sql b/tools/tpcds-tools/queries/sf1/query79.sql new file mode 100644 index 000000000000000..81edd1f4f20a2de --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query79.sql @@ -0,0 +1,20 @@ +select + c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,store.s_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (household_demographics.hd_dep_count = 7 or household_demographics.hd_vehicle_count > -1) + and date_dim.d_dow = 1 + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_number_employees between 200 and 295 + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer + where ss_customer_sk = c_customer_sk + order by c_last_name,c_first_name,substr(s_city,1,30), profit +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query8.sql b/tools/tpcds-tools/queries/sf1/query8.sql new file mode 100644 index 000000000000000..8909d5ba1d8b17b --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query8.sql @@ -0,0 +1,105 @@ +select s_store_name + ,sum(ss_net_profit) + from store_sales + ,date_dim + ,store, + (select ca_zip + from ( + SELECT substr(ca_zip,1,5) ca_zip + FROM customer_address + WHERE substr(ca_zip,1,5) IN ( + '47602','16704','35863','28577','83910','36201', + '58412','48162','28055','41419','80332', + '38607','77817','24891','16226','18410', + '21231','59345','13918','51089','20317', + '17167','54585','67881','78366','47770', + '18360','51717','73108','14440','21800', + '89338','45859','65501','34948','25973', + '73219','25333','17291','10374','18829', + '60736','82620','41351','52094','19326', + '25214','54207','40936','21814','79077', + '25178','75742','77454','30621','89193', + '27369','41232','48567','83041','71948', + '37119','68341','14073','16891','62878', + '49130','19833','24286','27700','40979', + '50412','81504','94835','84844','71954', + '39503','57649','18434','24987','12350', + '86379','27413','44529','98569','16515', + '27287','24255','21094','16005','56436', + '91110','68293','56455','54558','10298', + '83647','32754','27052','51766','19444', + '13869','45645','94791','57631','20712', + '37788','41807','46507','21727','71836', + '81070','50632','88086','63991','20244', + '31655','51782','29818','63792','68605', + '94898','36430','57025','20601','82080', + '33869','22728','35834','29086','92645', + '98584','98072','11652','78093','57553', + '43830','71144','53565','18700','90209', + '71256','38353','54364','28571','96560', + '57839','56355','50679','45266','84680', + '34306','34972','48530','30106','15371', + '92380','84247','92292','68852','13338', + '34594','82602','70073','98069','85066', + '47289','11686','98862','26217','47529', + '63294','51793','35926','24227','14196', + '24594','32489','99060','49472','43432', + '49211','14312','88137','47369','56877', + '20534','81755','15794','12318','21060', + '73134','41255','63073','81003','73873', + '66057','51184','51195','45676','92696', + '70450','90669','98338','25264','38919', + '59226','58581','60298','17895','19489', + '52301','80846','95464','68770','51634', + '19988','18367','18421','11618','67975', + '25494','41352','95430','15734','62585', + '97173','33773','10425','75675','53535', + '17879','41967','12197','67998','79658', + '59130','72592','14851','43933','68101', + '50636','25717','71286','24660','58058', + '72991','95042','15543','33122','69280', + '11912','59386','27642','65177','17672', + '33467','64592','36335','54010','18767', + '63193','42361','49254','33113','33159', + '36479','59080','11855','81963','31016', + '49140','29392','41836','32958','53163', + '13844','73146','23952','65148','93498', + '14530','46131','58454','13376','13378', + '83986','12320','17193','59852','46081', + '98533','52389','13086','68843','31013', + '13261','60560','13443','45533','83583', + '11489','58218','19753','22911','25115', + '86709','27156','32669','13123','51933', + '39214','41331','66943','14155','69998', + '49101','70070','35076','14242','73021', + '59494','15782','29752','37914','74686', + '83086','34473','15751','81084','49230', + '91894','60624','17819','28810','63180', + '56224','39459','55233','75752','43639', + '55349','86057','62361','50788','31830', + '58062','18218','85761','60083','45484', + '21204','90229','70041','41162','35390', + '16364','39500','68908','26689','52868', + '81335','40146','11340','61527','61794', + '71997','30415','59004','29450','58117', + '69952','33562','83833','27385','61860', + '96435','48333','23065','32961','84919', + '61997','99132','22815','56600','68730', + '48017','95694','32919','88217','27116', + '28239','58032','18884','16791','21343', + '97462','18569','75660','15475') + intersect + select ca_zip + from (SELECT substr(ca_zip,1,5) ca_zip,count(*) cnt + FROM customer_address, customer + WHERE ca_address_sk = c_current_addr_sk and + c_preferred_cust_flag='Y' + group by ca_zip + having count(*) > 10)A1)A2) V1 + where ss_store_sk = s_store_sk + and ss_sold_date_sk = d_date_sk + and d_qoy = 2 and d_year = 1998 + and (substr(s_zip,1,2) = substr(V1.ca_zip,1,2)) + group by s_store_name + order by s_store_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query80.sql b/tools/tpcds-tools/queries/sf1/query80.sql new file mode 100644 index 000000000000000..82f57c72361d7fb --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query80.sql @@ -0,0 +1,93 @@ +with ssr as + (select s_store_id as store_id, + sum(ss_ext_sales_price) as sales, + sum(coalesce(sr_return_amt, 0)) as returns, + sum(ss_net_profit - coalesce(sr_net_loss, 0)) as profit + from store_sales left outer join store_returns on + (ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number), + date_dim, + store, + item, + promotion + where ss_sold_date_sk = d_date_sk + and d_date between cast('2002-08-14' as date) + and (cast('2002-08-14' as date) + interval 30 day) + and ss_store_sk = s_store_sk + and ss_item_sk = i_item_sk + and i_current_price > 50 + and ss_promo_sk = p_promo_sk + and p_channel_tv = 'N' + group by s_store_id) + , + csr as + (select cp_catalog_page_id as catalog_page_id, + sum(cs_ext_sales_price) as sales, + sum(coalesce(cr_return_amount, 0)) as returns, + sum(cs_net_profit - coalesce(cr_net_loss, 0)) as profit + from catalog_sales left outer join catalog_returns on + (cs_item_sk = cr_item_sk and cs_order_number = cr_order_number), + date_dim, + catalog_page, + item, + promotion + where cs_sold_date_sk = d_date_sk + and d_date between cast('2002-08-14' as date) + and (cast('2002-08-14' as date) + interval 30 day) + and cs_catalog_page_sk = cp_catalog_page_sk + and cs_item_sk = i_item_sk + and i_current_price > 50 + and cs_promo_sk = p_promo_sk + and p_channel_tv = 'N' +group by cp_catalog_page_id) + , + wsr as + (select web_site_id, + sum(ws_ext_sales_price) as sales, + sum(coalesce(wr_return_amt, 0)) as returns, + sum(ws_net_profit - coalesce(wr_net_loss, 0)) as profit + from web_sales left outer join web_returns on + (ws_item_sk = wr_item_sk and ws_order_number = wr_order_number), + date_dim, + web_site, + item, + promotion + where ws_sold_date_sk = d_date_sk + and d_date between cast('2002-08-14' as date) + and (cast('2002-08-14' as date) + interval 30 day) + and ws_web_site_sk = web_site_sk + and ws_item_sk = i_item_sk + and i_current_price > 50 + and ws_promo_sk = p_promo_sk + and p_channel_tv = 'N' +group by web_site_id) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , concat('store', store_id) as id + , sales + , returns + , profit + from ssr + union all + select 'catalog channel' as channel + , concat('catalog_page', catalog_page_id) as id + , sales + , returns + , profit + from csr + union all + select 'web channel' as channel + , concat('web_site', web_site_id) as id + , sales + , returns + , profit + from wsr + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query81.sql b/tools/tpcds-tools/queries/sf1/query81.sql new file mode 100644 index 000000000000000..127d49bdd3c90f9 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query81.sql @@ -0,0 +1,28 @@ +with customer_total_return as + (select cr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(cr_return_amt_inc_tax) as ctr_total_return + from catalog_returns + ,date_dim + ,customer_address + where cr_returned_date_sk = d_date_sk + and d_year =2001 + and cr_returning_addr_sk = ca_address_sk + group by cr_returning_customer_sk + ,ca_state ) + select c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'TN' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query82.sql b/tools/tpcds-tools/queries/sf1/query82.sql new file mode 100644 index 000000000000000..c36ca8aecb84967 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query82.sql @@ -0,0 +1,14 @@ +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, store_sales + where i_current_price between 58 and 58+30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('2001-01-13' as date) and (cast('2001-01-13' as date) + interval 60 day) + and i_manufact_id in (259,559,580,485) + and inv_quantity_on_hand between 100 and 500 + and ss_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query83.sql b/tools/tpcds-tools/queries/sf1/query83.sql new file mode 100644 index 000000000000000..321cdc470751842 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query83.sql @@ -0,0 +1,64 @@ +with sr_items as + (select i_item_id item_id, + sum(sr_return_quantity) sr_item_qty + from store_returns, + item, + date_dim + where sr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2001-07-13','2001-09-10','2001-11-16'))) + and sr_returned_date_sk = d_date_sk + group by i_item_id), + cr_items as + (select i_item_id item_id, + sum(cr_return_quantity) cr_item_qty + from catalog_returns, + item, + date_dim + where cr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2001-07-13','2001-09-10','2001-11-16'))) + and cr_returned_date_sk = d_date_sk + group by i_item_id), + wr_items as + (select i_item_id item_id, + sum(wr_return_quantity) wr_item_qty + from web_returns, + item, + date_dim + where wr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2001-07-13','2001-09-10','2001-11-16'))) + and wr_returned_date_sk = d_date_sk + group by i_item_id) + select sr_items.item_id + ,sr_item_qty + ,sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev + ,cr_item_qty + ,cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev + ,wr_item_qty + ,wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev + ,(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 average + from sr_items + ,cr_items + ,wr_items + where sr_items.item_id=cr_items.item_id + and sr_items.item_id=wr_items.item_id + order by sr_items.item_id + ,sr_item_qty + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query84.sql b/tools/tpcds-tools/queries/sf1/query84.sql new file mode 100644 index 000000000000000..7d9fe1930a94541 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query84.sql @@ -0,0 +1,18 @@ +select c_customer_id as customer_id + , concat(concat(coalesce(c_last_name,''), ','), coalesce(c_first_name,'')) as customername + from customer + ,customer_address + ,customer_demographics + ,household_demographics + ,income_band + ,store_returns + where ca_city = 'Woodland' + and c_current_addr_sk = ca_address_sk + and ib_lower_bound >= 60306 + and ib_upper_bound <= 60306 + 50000 + and ib_income_band_sk = hd_income_band_sk + and cd_demo_sk = c_current_cdemo_sk + and hd_demo_sk = c_current_hdemo_sk + and sr_cdemo_sk = cd_demo_sk + order by c_customer_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query85.sql b/tools/tpcds-tools/queries/sf1/query85.sql new file mode 100644 index 000000000000000..7ff852ac02cd5bc --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query85.sql @@ -0,0 +1,81 @@ +select substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) + from web_sales, web_returns, web_page, customer_demographics cd1, + customer_demographics cd2, customer_address, date_dim, reason + where ws_web_page_sk = wp_web_page_sk + and ws_item_sk = wr_item_sk + and ws_order_number = wr_order_number + and ws_sold_date_sk = d_date_sk and d_year = 1998 + and cd1.cd_demo_sk = wr_refunded_cdemo_sk + and cd2.cd_demo_sk = wr_returning_cdemo_sk + and ca_address_sk = wr_refunded_addr_sk + and r_reason_sk = wr_reason_sk + and + ( + ( + cd1.cd_marital_status = 'D' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Primary' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 100.00 and 150.00 + ) + or + ( + cd1.cd_marital_status = 'S' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'College' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 50.00 and 100.00 + ) + or + ( + cd1.cd_marital_status = 'U' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Advanced Degree' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ca_country = 'United States' + and + ca_state in ('NC', 'TX', 'IA') + and ws_net_profit between 100 and 200 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('WI', 'WV', 'GA') + and ws_net_profit between 150 and 300 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('OK', 'VA', 'KY') + and ws_net_profit between 50 and 250 + ) + ) +group by r_reason_desc +order by substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query86.sql b/tools/tpcds-tools/queries/sf1/query86.sql new file mode 100644 index 000000000000000..173fff760f66dee --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query86.sql @@ -0,0 +1,23 @@ +select + sum(ws_net_paid) as total_sum + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ws_net_paid) desc) as rank_within_parent + from + web_sales + ,date_dim d1 + ,item + where + d1.d_month_seq between 1186 and 1186+11 + and d1.d_date_sk = ws_sold_date_sk + and i_item_sk = ws_item_sk + group by rollup(i_category,i_class) + order by + lochierarchy desc, + case when lochierarchy = 0 then i_category end, + rank_within_parent + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query87.sql b/tools/tpcds-tools/queries/sf1/query87.sql new file mode 100644 index 000000000000000..8d27131f21f4952 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query87.sql @@ -0,0 +1,20 @@ +select count(*) +from ((select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1202 and 1202+11) + except + (select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1202 and 1202+11) + except + (select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1202 and 1202+11) +) cool_cust +; diff --git a/tools/tpcds-tools/queries/sf1/query88.sql b/tools/tpcds-tools/queries/sf1/query88.sql new file mode 100644 index 000000000000000..0d8eea8d116dcf8 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query88.sql @@ -0,0 +1,91 @@ +select * +from + (select count(*) h8_30_to_9 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 8 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s1, + (select count(*) h9_to_9_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s2, + (select count(*) h9_30_to_10 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s3, + (select count(*) h10_to_10_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s4, + (select count(*) h10_30_to_11 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s5, + (select count(*) h11_to_11_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s6, + (select count(*) h11_30_to_12 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s7, + (select count(*) h12_to_12_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 12 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s8 +; diff --git a/tools/tpcds-tools/queries/sf1/query89.sql b/tools/tpcds-tools/queries/sf1/query89.sql new file mode 100644 index 000000000000000..d88f0530687b1fb --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query89.sql @@ -0,0 +1,25 @@ +select * +from( +select i_category, i_class, i_brand, + s_store_name, s_company_name, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, s_store_name, s_company_name) + avg_monthly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + d_year in (2001) and + ((i_category in ('Books','Children','Electronics') and + i_class in ('history','school-uniforms','audio') + ) + or (i_category in ('Men','Sports','Shoes') and + i_class in ('pants','tennis','womens') + )) +group by i_category, i_class, i_brand, + s_store_name, s_company_name, d_moy) tmp1 +where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1 +order by sum_sales - avg_monthly_sales, s_store_name +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query9.sql b/tools/tpcds-tools/queries/sf1/query9.sql new file mode 100644 index 000000000000000..df99d02dd2e1d2b --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query9.sql @@ -0,0 +1,48 @@ +select case when (select count(*) + from store_sales + where ss_quantity between 1 and 20) > 1071 + then (select avg(ss_ext_tax) + from store_sales + where ss_quantity between 1 and 20) + else (select avg(ss_net_paid_inc_tax) + from store_sales + where ss_quantity between 1 and 20) end bucket1 , + case when (select count(*) + from store_sales + where ss_quantity between 21 and 40) > 39161 + then (select avg(ss_ext_tax) + from store_sales + where ss_quantity between 21 and 40) + else (select avg(ss_net_paid_inc_tax) + from store_sales + where ss_quantity between 21 and 40) end bucket2, + case when (select count(*) + from store_sales + where ss_quantity between 41 and 60) > 29434 + then (select avg(ss_ext_tax) + from store_sales + where ss_quantity between 41 and 60) + else (select avg(ss_net_paid_inc_tax) + from store_sales + where ss_quantity between 41 and 60) end bucket3, + case when (select count(*) + from store_sales + where ss_quantity between 61 and 80) > 6568 + then (select avg(ss_ext_tax) + from store_sales + where ss_quantity between 61 and 80) + else (select avg(ss_net_paid_inc_tax) + from store_sales + where ss_quantity between 61 and 80) end bucket4, + case when (select count(*) + from store_sales + where ss_quantity between 81 and 100) > 21216 + then (select avg(ss_ext_tax) + from store_sales + where ss_quantity between 81 and 100) + else (select avg(ss_net_paid_inc_tax) + from store_sales + where ss_quantity between 81 and 100) end bucket5 +from reason +where r_reason_sk = 1 +; diff --git a/tools/tpcds-tools/queries/sf1/query90.sql b/tools/tpcds-tools/queries/sf1/query90.sql new file mode 100644 index 000000000000000..45dce0ab06a282a --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query90.sql @@ -0,0 +1,19 @@ +select cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio + from ( select count(*) amc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 12 and 12+1 + and household_demographics.hd_dep_count = 6 + and web_page.wp_char_count between 5000 and 5200) at, + ( select count(*) pmc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 14 and 14+1 + and household_demographics.hd_dep_count = 6 + and web_page.wp_char_count between 5000 and 5200) pt + order by am_pm_ratio + limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query91.sql b/tools/tpcds-tools/queries/sf1/query91.sql new file mode 100644 index 000000000000000..b8354f62c69e346 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query91.sql @@ -0,0 +1,28 @@ +select + cc_call_center_id Call_Center, + cc_name Call_Center_Name, + cc_manager Manager, + sum(cr_net_loss) Returns_Loss +from + call_center, + catalog_returns, + date_dim, + customer, + customer_address, + customer_demographics, + household_demographics +where + cr_call_center_sk = cc_call_center_sk +and cr_returned_date_sk = d_date_sk +and cr_returning_customer_sk= c_customer_sk +and cd_demo_sk = c_current_cdemo_sk +and hd_demo_sk = c_current_hdemo_sk +and ca_address_sk = c_current_addr_sk +and d_year = 2000 +and d_moy = 12 +and ( (cd_marital_status = 'M' and cd_education_status = 'Unknown') + or(cd_marital_status = 'W' and cd_education_status = 'Advanced Degree')) +and hd_buy_potential like 'Unknown%' +and ca_gmt_offset = -7 +group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status +order by sum(cr_net_loss) desc; diff --git a/tools/tpcds-tools/queries/sf1/query92.sql b/tools/tpcds-tools/queries/sf1/query92.sql new file mode 100644 index 000000000000000..fc79c8bdeba02ef --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query92.sql @@ -0,0 +1,27 @@ +select + sum(ws_ext_discount_amt) as "Excess Discount Amount" +from + web_sales + ,item + ,date_dim +where +i_manufact_id = 714 +and i_item_sk = ws_item_sk +and d_date between '2000-02-01' and + (cast('2000-02-01' as date) + interval 90 day) +and d_date_sk = ws_sold_date_sk +and ws_ext_discount_amt + > ( + SELECT + 1.3 * avg(ws_ext_discount_amt) + FROM + web_sales + ,date_dim + WHERE + ws_item_sk = i_item_sk + and d_date between '2000-02-01' and + (cast('2000-02-01' as date) + interval 90 day) + and d_date_sk = ws_sold_date_sk + ) +order by sum(ws_ext_discount_amt) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query93.sql b/tools/tpcds-tools/queries/sf1/query93.sql new file mode 100644 index 000000000000000..43cf18d6a16e87f --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query93.sql @@ -0,0 +1,15 @@ +select ss_customer_sk + ,sum(act_sales) sumsales + from (select ss_item_sk + ,ss_ticket_number + ,ss_customer_sk + ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price + else (ss_quantity*ss_sales_price) end act_sales + from store_sales left outer join store_returns on (sr_item_sk = ss_item_sk + and sr_ticket_number = ss_ticket_number) + ,reason + where sr_reason_sk = r_reason_sk + and r_reason_desc = 'reason 58') t + group by ss_customer_sk + order by sumsales, ss_customer_sk +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query94.sql b/tools/tpcds-tools/queries/sf1/query94.sql new file mode 100644 index 000000000000000..5d37954d82a672d --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query94.sql @@ -0,0 +1,26 @@ +select + count(distinct ws_order_number) as "order count" + ,sum(ws_ext_ship_cost) as "total shipping cost" + ,sum(ws_net_profit) as "total net profit" +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '2002-5-01' and + (cast('2002-5-01' as date) + interval 60 day) +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'OK' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and exists (select * + from web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) +and not exists(select * + from web_returns wr1 + where ws1.ws_order_number = wr1.wr_order_number) +order by count(distinct ws_order_number) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query95.sql b/tools/tpcds-tools/queries/sf1/query95.sql new file mode 100644 index 000000000000000..8927ea96b88b270 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query95.sql @@ -0,0 +1,29 @@ +with ws_wh as +(select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2 + from web_sales ws1,web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) + select + count(distinct ws_order_number) as "order count" + ,sum(ws_ext_ship_cost) as "total shipping cost" + ,sum(ws_net_profit) as "total net profit" +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '2001-4-01' and + (cast('2001-4-01' as date) + interval 60 day) +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'VA' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and ws1.ws_order_number in (select ws_order_number + from ws_wh) +and ws1.ws_order_number in (select wr_order_number + from web_returns,ws_wh + where wr_order_number = ws_wh.ws_order_number) +order by count(distinct ws_order_number) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query96.sql b/tools/tpcds-tools/queries/sf1/query96.sql new file mode 100644 index 000000000000000..84e2eaad2a62355 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query96.sql @@ -0,0 +1,13 @@ +select count(*) +from store_sales + ,household_demographics + ,time_dim, store +where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 8 + and time_dim.t_minute >= 30 + and household_demographics.hd_dep_count = 0 + and store.s_store_name = 'ese' +order by count(*) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query97.sql b/tools/tpcds-tools/queries/sf1/query97.sql new file mode 100644 index 000000000000000..9ef89c5ffb6c123 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query97.sql @@ -0,0 +1,22 @@ +with ssci as ( +select ss_customer_sk customer_sk + ,ss_item_sk item_sk +from store_sales,date_dim +where ss_sold_date_sk = d_date_sk + and d_month_seq between 1199 and 1199 + 11 +group by ss_customer_sk + ,ss_item_sk), +csci as( + select cs_bill_customer_sk customer_sk + ,cs_item_sk item_sk +from catalog_sales,date_dim +where cs_sold_date_sk = d_date_sk + and d_month_seq between 1199 and 1199 + 11 +group by cs_bill_customer_sk + ,cs_item_sk) + select sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only + ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only + ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog +from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk + and ssci.item_sk = csci.item_sk) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1/query98.sql b/tools/tpcds-tools/queries/sf1/query98.sql new file mode 100644 index 000000000000000..0d16696158915f7 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query98.sql @@ -0,0 +1,30 @@ +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ss_ext_sales_price) as itemrevenue + ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over + (partition by i_class) as revenueratio +from + store_sales + ,item + ,date_dim +where + ss_item_sk = i_item_sk + and i_category in ('Men', 'Sports', 'Jewelry') + and ss_sold_date_sk = d_date_sk + and d_date between cast('1999-02-05' as date) + and (cast('1999-02-05' as date) + interval 30 day) +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio; diff --git a/tools/tpcds-tools/queries/sf1/query99.sql b/tools/tpcds-tools/queries/sf1/query99.sql new file mode 100644 index 000000000000000..f144236d6b83dfc --- /dev/null +++ b/tools/tpcds-tools/queries/sf1/query99.sql @@ -0,0 +1,32 @@ +select + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and + (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and + (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and + (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 120) then 1 else 0 end) as ">120 days" +from + catalog_sales + ,warehouse + ,ship_mode + ,call_center + ,date_dim +where + d_month_seq between 1194 and 1194 + 11 +and cs_ship_date_sk = d_date_sk +and cs_warehouse_sk = w_warehouse_sk +and cs_ship_mode_sk = sm_ship_mode_sk +and cs_call_center_sk = cc_call_center_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query1.sql b/tools/tpcds-tools/queries/sf1000/query1.sql new file mode 100644 index 000000000000000..c2f6725361ecfe3 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query1.sql @@ -0,0 +1,22 @@ +with customer_total_return as +(select sr_customer_sk as ctr_customer_sk +,sr_store_sk as ctr_store_sk +,sum(SR_FEE) as ctr_total_return +from store_returns +,date_dim +where sr_returned_date_sk = d_date_sk +and d_year =2000 +group by sr_customer_sk +,sr_store_sk) + select c_customer_id +from customer_total_return ctr1 +,store +,customer +where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 +from customer_total_return ctr2 +where ctr1.ctr_store_sk = ctr2.ctr_store_sk) +and s_store_sk = ctr1.ctr_store_sk +and s_state = 'TN' +and ctr1.ctr_customer_sk = c_customer_sk +order by c_customer_id +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query10.sql b/tools/tpcds-tools/queries/sf1000/query10.sql new file mode 100644 index 000000000000000..077f80ad92a66d3 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query10.sql @@ -0,0 +1,56 @@ +select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3, + cd_dep_count, + count(*) cnt4, + cd_dep_employed_count, + count(*) cnt5, + cd_dep_college_count, + count(*) cnt6 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_county in ('Fairfield County','Campbell County','Washtenaw County','Escambia County','Cleburne County') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2001 and + d_moy between 3 and 3+3) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2001 and + d_moy between 3 ANd 3+3) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2001 and + d_moy between 3 and 3+3)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query11.sql b/tools/tpcds-tools/queries/sf1000/query11.sql new file mode 100644 index 000000000000000..f139721ec2c8c78 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query11.sql @@ -0,0 +1,78 @@ +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(ss_ext_list_price-ss_ext_discount_amt) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(ws_ext_list_price-ws_ext_discount_amt) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + ) + select + t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.dyear = 1998 + and t_s_secyear.dyear = 1998+1 + and t_w_firstyear.dyear = 1998 + and t_w_secyear.dyear = 1998+1 + and t_s_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else 0.0 end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else 0.0 end + order by t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query12.sql b/tools/tpcds-tools/queries/sf1000/query12.sql new file mode 100644 index 000000000000000..253ddb640a994f0 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query12.sql @@ -0,0 +1,31 @@ +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ws_ext_sales_price) as itemrevenue + ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over + (partition by i_class) as revenueratio +from + web_sales + ,item + ,date_dim +where + ws_item_sk = i_item_sk + and i_category in ('Men', 'Books', 'Electronics') + and ws_sold_date_sk = d_date_sk + and d_date between cast('2001-06-15' as date) + and (cast('2001-06-15' as date) + interval 30 day) +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query13.sql b/tools/tpcds-tools/queries/sf1000/query13.sql new file mode 100644 index 000000000000000..456bdf47b400305 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query13.sql @@ -0,0 +1,49 @@ +select avg(ss_quantity) + ,avg(ss_ext_sales_price) + ,avg(ss_ext_wholesale_cost) + ,sum(ss_ext_wholesale_cost) + from store_sales + ,store + ,customer_demographics + ,household_demographics + ,customer_address + ,date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 2001 + and((ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'M' + and cd_education_status = 'College' + and ss_sales_price between 100.00 and 150.00 + and hd_dep_count = 3 + )or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'D' + and cd_education_status = 'Primary' + and ss_sales_price between 50.00 and 100.00 + and hd_dep_count = 1 + ) or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'W' + and cd_education_status = '2 yr Degree' + and ss_sales_price between 150.00 and 200.00 + and hd_dep_count = 1 + )) + and((ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('IL', 'TN', 'TX') + and ss_net_profit between 100 and 200 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('WY', 'OH', 'ID') + and ss_net_profit between 150 and 300 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('MS', 'SC', 'IA') + and ss_net_profit between 50 and 250 + )) +; diff --git a/tools/tpcds-tools/queries/sf1000/query14.sql b/tools/tpcds-tools/queries/sf1000/query14.sql new file mode 100644 index 000000000000000..0ec2b10f1843b5f --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query14.sql @@ -0,0 +1,207 @@ +with cross_items as + (select i_item_sk ss_item_sk + from item, + (select iss.i_brand_id brand_id + ,iss.i_class_id class_id + ,iss.i_category_id category_id + from store_sales + ,item iss + ,date_dim d1 + where ss_item_sk = iss.i_item_sk + and ss_sold_date_sk = d1.d_date_sk + and d1.d_year between 1999 AND 1999 + 2 + intersect + select ics.i_brand_id + ,ics.i_class_id + ,ics.i_category_id + from catalog_sales + ,item ics + ,date_dim d2 + where cs_item_sk = ics.i_item_sk + and cs_sold_date_sk = d2.d_date_sk + and d2.d_year between 1999 AND 1999 + 2 + intersect + select iws.i_brand_id + ,iws.i_class_id + ,iws.i_category_id + from web_sales + ,item iws + ,date_dim d3 + where ws_item_sk = iws.i_item_sk + and ws_sold_date_sk = d3.d_date_sk + and d3.d_year between 1999 AND 1999 + 2) + t where i_brand_id = brand_id + and i_class_id = class_id + and i_category_id = category_id +), +avg_sales as + (select avg(quantity*list_price) average_sales + from (select ss_quantity quantity + ,ss_list_price list_price + from store_sales + ,date_dim + where ss_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2 + union all + select cs_quantity quantity + ,cs_list_price list_price + from catalog_sales + ,date_dim + where cs_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2 + union all + select ws_quantity quantity + ,ws_list_price list_price + from web_sales + ,date_dim + where ws_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2) x) + select channel, i_brand_id,i_class_id,i_category_id,sum(sales), sum(number_sales) + from( + select 'store' channel, i_brand_id,i_class_id + ,i_category_id,sum(ss_quantity*ss_list_price) sales + , count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1999+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales) + union all + select 'catalog' channel, i_brand_id,i_class_id,i_category_id, sum(cs_quantity*cs_list_price) sales, count(*) number_sales + from catalog_sales + ,item + ,date_dim + where cs_item_sk in (select ss_item_sk from cross_items) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1999+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(cs_quantity*cs_list_price) > (select average_sales from avg_sales) + union all + select 'web' channel, i_brand_id,i_class_id,i_category_id, sum(ws_quantity*ws_list_price) sales , count(*) number_sales + from web_sales + ,item + ,date_dim + where ws_item_sk in (select ss_item_sk from cross_items) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1999+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(ws_quantity*ws_list_price) > (select average_sales from avg_sales) + ) y + group by rollup (channel, i_brand_id,i_class_id,i_category_id) + order by channel,i_brand_id,i_class_id,i_category_id + limit 100; +with cross_items as + (select i_item_sk ss_item_sk + from item, + (select iss.i_brand_id brand_id + ,iss.i_class_id class_id + ,iss.i_category_id category_id + from store_sales + ,item iss + ,date_dim d1 + where ss_item_sk = iss.i_item_sk + and ss_sold_date_sk = d1.d_date_sk + and d1.d_year between 1999 AND 1999 + 2 + intersect + select ics.i_brand_id + ,ics.i_class_id + ,ics.i_category_id + from catalog_sales + ,item ics + ,date_dim d2 + where cs_item_sk = ics.i_item_sk + and cs_sold_date_sk = d2.d_date_sk + and d2.d_year between 1999 AND 1999 + 2 + intersect + select iws.i_brand_id + ,iws.i_class_id + ,iws.i_category_id + from web_sales + ,item iws + ,date_dim d3 + where ws_item_sk = iws.i_item_sk + and ws_sold_date_sk = d3.d_date_sk + and d3.d_year between 1999 AND 1999 + 2) x + where i_brand_id = brand_id + and i_class_id = class_id + and i_category_id = category_id +), + avg_sales as +(select avg(quantity*list_price) average_sales + from (select ss_quantity quantity + ,ss_list_price list_price + from store_sales + ,date_dim + where ss_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2 + union all + select cs_quantity quantity + ,cs_list_price list_price + from catalog_sales + ,date_dim + where cs_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2 + union all + select ws_quantity quantity + ,ws_list_price list_price + from web_sales + ,date_dim + where ws_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2) x) + select this_year.channel ty_channel + ,this_year.i_brand_id ty_brand + ,this_year.i_class_id ty_class + ,this_year.i_category_id ty_category + ,this_year.sales ty_sales + ,this_year.number_sales ty_number_sales + ,last_year.channel ly_channel + ,last_year.i_brand_id ly_brand + ,last_year.i_class_id ly_class + ,last_year.i_category_id ly_category + ,last_year.sales ly_sales + ,last_year.number_sales ly_number_sales + from + (select 'store' channel, i_brand_id,i_class_id,i_category_id + ,sum(ss_quantity*ss_list_price) sales, count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_week_seq = (select d_week_seq + from date_dim + where d_year = 1999 + 1 + and d_moy = 12 + and d_dom = 3) + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) this_year, + (select 'store' channel, i_brand_id,i_class_id + ,i_category_id, sum(ss_quantity*ss_list_price) sales, count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_week_seq = (select d_week_seq + from date_dim + where d_year = 1999 + and d_moy = 12 + and d_dom = 3) + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) last_year + where this_year.i_brand_id= last_year.i_brand_id + and this_year.i_class_id = last_year.i_class_id + and this_year.i_category_id = last_year.i_category_id + order by this_year.channel, this_year.i_brand_id, this_year.i_class_id, this_year.i_category_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query15.sql b/tools/tpcds-tools/queries/sf1000/query15.sql new file mode 100644 index 000000000000000..a6bb8f07028eb64 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query15.sql @@ -0,0 +1,17 @@ +select ca_zip + ,sum(cs_sales_price) + from catalog_sales + ,customer + ,customer_address + ,date_dim + where cs_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', + '85392', '85460', '80348', '81792') + or ca_state in ('CA','WA','GA') + or cs_sales_price > 500) + and cs_sold_date_sk = d_date_sk + and d_qoy = 2 and d_year = 2001 + group by ca_zip + order by ca_zip + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query16.sql b/tools/tpcds-tools/queries/sf1000/query16.sql new file mode 100644 index 000000000000000..168d2b4f9d875a4 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query16.sql @@ -0,0 +1,28 @@ +select + count(distinct cs_order_number) as "order count" + ,sum(cs_ext_ship_cost) as "total shipping cost" + ,sum(cs_net_profit) as "total net profit" +from + catalog_sales cs1 + ,date_dim + ,customer_address + ,call_center +where + d_date between '2002-4-01' and + (cast('2002-4-01' as date) + interval 60 day) +and cs1.cs_ship_date_sk = d_date_sk +and cs1.cs_ship_addr_sk = ca_address_sk +and ca_state = 'PA' +and cs1.cs_call_center_sk = cc_call_center_sk +and cc_county in ('Williamson County','Williamson County','Williamson County','Williamson County', + 'Williamson County' +) +and exists (select * + from catalog_sales cs2 + where cs1.cs_order_number = cs2.cs_order_number + and cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) +and not exists(select * + from catalog_returns cr1 + where cs1.cs_order_number = cr1.cr_order_number) +order by count(distinct cs_order_number) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query17.sql b/tools/tpcds-tools/queries/sf1000/query17.sql new file mode 100644 index 000000000000000..02b9c4761c5bb3c --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query17.sql @@ -0,0 +1,42 @@ +select i_item_id + ,i_item_desc + ,s_state + ,count(ss_quantity) as store_sales_quantitycount + ,avg(ss_quantity) as store_sales_quantityave + ,stddev_samp(ss_quantity) as store_sales_quantitystdev + ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov + ,count(sr_return_quantity) as store_returns_quantitycount + ,avg(sr_return_quantity) as store_returns_quantityave + ,stddev_samp(sr_return_quantity) as store_returns_quantitystdev + ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov + ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave + ,stddev_samp(cs_quantity) as catalog_sales_quantitystdev + ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov + from store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where d1.d_quarter_name = '2001Q1' + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_quarter_name in ('2001Q1','2001Q2','2001Q3') + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_quarter_name in ('2001Q1','2001Q2','2001Q3') + group by i_item_id + ,i_item_desc + ,s_state + order by i_item_id + ,i_item_desc + ,s_state +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query18.sql b/tools/tpcds-tools/queries/sf1000/query18.sql new file mode 100644 index 000000000000000..1f36ffc8374ab70 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query18.sql @@ -0,0 +1,31 @@ +select i_item_id, + ca_country, + ca_state, + ca_county, + avg( cast(cs_quantity as decimal(12,2))) agg1, + avg( cast(cs_list_price as decimal(12,2))) agg2, + avg( cast(cs_coupon_amt as decimal(12,2))) agg3, + avg( cast(cs_sales_price as decimal(12,2))) agg4, + avg( cast(cs_net_profit as decimal(12,2))) agg5, + avg( cast(c_birth_year as decimal(12,2))) agg6, + avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7 + from catalog_sales, customer_demographics cd1, + customer_demographics cd2, customer, customer_address, date_dim, item + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd1.cd_demo_sk and + cs_bill_customer_sk = c_customer_sk and + cd1.cd_gender = 'F' and + cd1.cd_education_status = 'Primary' and + c_current_cdemo_sk = cd2.cd_demo_sk and + c_current_addr_sk = ca_address_sk and + c_birth_month in (1,3,7,11,10,4) and + d_year = 2001 and + ca_state in ('AL','MO','TN' + ,'GA','MT','IN','CA') + group by rollup (i_item_id, ca_country, ca_state, ca_county) + order by ca_country, + ca_state, + ca_county, + i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query19.sql b/tools/tpcds-tools/queries/sf1000/query19.sql new file mode 100644 index 000000000000000..0f44101ac589c70 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query19.sql @@ -0,0 +1,22 @@ +select i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item,customer,customer_address,store + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=14 + and d_moy=11 + and d_year=2002 + and ss_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and substr(ca_zip,1,5) <> substr(s_zip,1,5) + and ss_store_sk = s_store_sk + group by i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact + order by ext_price desc + ,i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact +limit 100 ; diff --git a/tools/tpcds-tools/queries/sf1000/query2.sql b/tools/tpcds-tools/queries/sf1000/query2.sql new file mode 100644 index 000000000000000..5dc50b1156c022e --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query2.sql @@ -0,0 +1,57 @@ +with wscs as + (select sold_date_sk + ,sales_price + from (select ws_sold_date_sk sold_date_sk + ,ws_ext_sales_price sales_price + from web_sales + union all + select cs_sold_date_sk sold_date_sk + ,cs_ext_sales_price sales_price + from catalog_sales) t), + wswscs as + (select d_week_seq, + sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales + from wscs + ,date_dim + where d_date_sk = sold_date_sk + group by d_week_seq) + select d_week_seq1 + ,round(sun_sales1/sun_sales2,2) + ,round(mon_sales1/mon_sales2,2) + ,round(tue_sales1/tue_sales2,2) + ,round(wed_sales1/wed_sales2,2) + ,round(thu_sales1/thu_sales2,2) + ,round(fri_sales1/fri_sales2,2) + ,round(sat_sales1/sat_sales2,2) + from + (select wswscs.d_week_seq d_week_seq1 + ,sun_sales sun_sales1 + ,mon_sales mon_sales1 + ,tue_sales tue_sales1 + ,wed_sales wed_sales1 + ,thu_sales thu_sales1 + ,fri_sales fri_sales1 + ,sat_sales sat_sales1 + from wswscs,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 1998) y, + (select wswscs.d_week_seq d_week_seq2 + ,sun_sales sun_sales2 + ,mon_sales mon_sales2 + ,tue_sales tue_sales2 + ,wed_sales wed_sales2 + ,thu_sales thu_sales2 + ,fri_sales fri_sales2 + ,sat_sales sat_sales2 + from wswscs + ,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 1998+1) z + where d_week_seq1=d_week_seq2-53 + order by d_week_seq1; diff --git a/tools/tpcds-tools/queries/sf1000/query20.sql b/tools/tpcds-tools/queries/sf1000/query20.sql new file mode 100644 index 000000000000000..1e0b1e1c523bfad --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query20.sql @@ -0,0 +1,27 @@ +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(cs_ext_sales_price) as itemrevenue + ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over + (partition by i_class) as revenueratio + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and i_category in ('Books', 'Music', 'Sports') + and cs_sold_date_sk = d_date_sk + and d_date between cast('2002-06-18' as date) + and (cast('2002-06-18' as date) + interval 30 day) + group by i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + order by i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query21.sql b/tools/tpcds-tools/queries/sf1000/query21.sql new file mode 100644 index 000000000000000..7bced24450465a6 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query21.sql @@ -0,0 +1,27 @@ +select * + from(select w_warehouse_name + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('1999-06-22' as date)) + then inv_quantity_on_hand + else 0 end) as inv_before + ,sum(case when (cast(d_date as date) >= cast ('1999-06-22' as date)) + then inv_quantity_on_hand + else 0 end) as inv_after + from inventory + ,warehouse + ,item + ,date_dim + where i_current_price between 0.99 and 1.49 + and i_item_sk = inv_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_date between (cast ('1999-06-22' as date) - interval 30 day) + and (cast ('1999-06-22' as date) + interval 30 day) + group by w_warehouse_name, i_item_id) x + where (case when inv_before > 0 + then inv_after / inv_before + else null + end) between 2.0/3.0 and 3.0/2.0 + order by w_warehouse_name + ,i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query22.sql b/tools/tpcds-tools/queries/sf1000/query22.sql new file mode 100644 index 000000000000000..dc0349840a33342 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query22.sql @@ -0,0 +1,17 @@ +select i_product_name + ,i_brand + ,i_class + ,i_category + ,avg(inv_quantity_on_hand) qoh + from inventory + ,date_dim + ,item + where inv_date_sk=d_date_sk + and inv_item_sk=i_item_sk + and d_month_seq between 1200 and 1200 + 11 + group by rollup(i_product_name + ,i_brand + ,i_class + ,i_category) +order by qoh, i_product_name, i_brand, i_class, i_category +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query23.sql b/tools/tpcds-tools/queries/sf1000/query23.sql new file mode 100644 index 000000000000000..b9896edaf59249d --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query23.sql @@ -0,0 +1,104 @@ +with frequent_ss_items as + (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt + from store_sales + ,date_dim + ,item + where ss_sold_date_sk = d_date_sk + and ss_item_sk = i_item_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by substr(i_item_desc,1,30),i_item_sk,d_date + having count(*) >4), + max_store_sales as + (select max(csales) tpcds_cmax + from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales + from store_sales + ,customer + ,date_dim + where ss_customer_sk = c_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by c_customer_sk) t), +best_ss_customer as + (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales + from store_sales + ,customer + where ss_customer_sk = c_customer_sk + group by c_customer_sk + having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select + * +from + max_store_sales)) + select sum(sales) + from (select cs_quantity*cs_list_price sales + from catalog_sales + ,date_dim + where d_year = 2000 + and d_moy = 7 + and cs_sold_date_sk = d_date_sk + and cs_item_sk in (select item_sk from frequent_ss_items) + and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) + union all + select ws_quantity*ws_list_price sales + from web_sales + ,date_dim + where d_year = 2000 + and d_moy = 7 + and ws_sold_date_sk = d_date_sk + and ws_item_sk in (select item_sk from frequent_ss_items) + and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)) t2 + limit 100; +with frequent_ss_items as + (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt + from store_sales + ,date_dim + ,item + where ss_sold_date_sk = d_date_sk + and ss_item_sk = i_item_sk + and d_year in (2000,2000 + 1,2000 + 2,2000 + 3) + group by substr(i_item_desc,1,30),i_item_sk,d_date + having count(*) >4), + max_store_sales as + (select max(csales) tpcds_cmax + from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales + from store_sales + ,customer + ,date_dim + where ss_customer_sk = c_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by c_customer_sk) t), + best_ss_customer as + (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales + from store_sales + ,customer + where ss_customer_sk = c_customer_sk + group by c_customer_sk + having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select + * + from max_store_sales)) + select c_last_name,c_first_name,sales + from (select c_last_name,c_first_name,sum(cs_quantity*cs_list_price) sales + from catalog_sales + ,customer + ,date_dim + where d_year = 2000 + and d_moy = 7 + and cs_sold_date_sk = d_date_sk + and cs_item_sk in (select item_sk from frequent_ss_items) + and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) + and cs_bill_customer_sk = c_customer_sk + group by c_last_name,c_first_name + union all + select c_last_name,c_first_name,sum(ws_quantity*ws_list_price) sales + from web_sales + ,customer + ,date_dim + where d_year = 2000 + and d_moy = 7 + and ws_sold_date_sk = d_date_sk + and ws_item_sk in (select item_sk from frequent_ss_items) + and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer) + and ws_bill_customer_sk = c_customer_sk + group by c_last_name,c_first_name) t2 + order by c_last_name,c_first_name,sales + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query24.sql b/tools/tpcds-tools/queries/sf1000/query24.sql new file mode 100644 index 000000000000000..f29cf674646846d --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query24.sql @@ -0,0 +1,104 @@ +with ssales as +(select c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size + ,sum(ss_net_paid) netpaid +from store_sales + ,store_returns + ,store + ,item + ,customer + ,customer_address +where ss_ticket_number = sr_ticket_number + and ss_item_sk = sr_item_sk + and ss_customer_sk = c_customer_sk + and ss_item_sk = i_item_sk + and ss_store_sk = s_store_sk + and c_current_addr_sk = ca_address_sk + and c_birth_country <> upper(ca_country) + and s_zip = ca_zip +and s_market_id=5 +group by c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size) +select c_last_name + ,c_first_name + ,s_store_name + ,sum(netpaid) paid +from ssales +where i_color = 'aquamarine' +group by c_last_name + ,c_first_name + ,s_store_name +having sum(netpaid) > (select 0.05*avg(netpaid) + from ssales) +order by c_last_name + ,c_first_name + ,s_store_name +; +with ssales as +(select c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size + ,sum(ss_net_paid) netpaid +from store_sales + ,store_returns + ,store + ,item + ,customer + ,customer_address +where ss_ticket_number = sr_ticket_number + and ss_item_sk = sr_item_sk + and ss_customer_sk = c_customer_sk + and ss_item_sk = i_item_sk + and ss_store_sk = s_store_sk + and c_current_addr_sk = ca_address_sk + and c_birth_country <> upper(ca_country) + and s_zip = ca_zip + and s_market_id = 5 +group by c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size) +select c_last_name + ,c_first_name + ,s_store_name + ,sum(netpaid) paid +from ssales +where i_color = 'seashell' +group by c_last_name + ,c_first_name + ,s_store_name +having sum(netpaid) > (select 0.05*avg(netpaid) + from ssales) +order by c_last_name + ,c_first_name + ,s_store_name +; diff --git a/tools/tpcds-tools/queries/sf1000/query25.sql b/tools/tpcds-tools/queries/sf1000/query25.sql new file mode 100644 index 000000000000000..c927ff6bb06c969 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query25.sql @@ -0,0 +1,45 @@ +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,max(ss_net_profit) as store_sales_profit + ,max(sr_net_loss) as store_returns_loss + ,max(cs_net_profit) as catalog_sales_profit + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 1999 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 10 + and d2.d_year = 1999 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_moy between 4 and 10 + and d3.d_year = 1999 + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query26.sql b/tools/tpcds-tools/queries/sf1000/query26.sql new file mode 100644 index 000000000000000..e601c2cdd81eb43 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query26.sql @@ -0,0 +1,18 @@ +select i_item_id, + avg(cs_quantity) agg1, + avg(cs_list_price) agg2, + avg(cs_coupon_amt) agg3, + avg(cs_sales_price) agg4 + from catalog_sales, customer_demographics, date_dim, item, promotion + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd_demo_sk and + cs_promo_sk = p_promo_sk and + cd_gender = 'M' and + cd_marital_status = 'W' and + cd_education_status = 'Unknown' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 2002 + group by i_item_id + order by i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query27.sql b/tools/tpcds-tools/queries/sf1000/query27.sql new file mode 100644 index 000000000000000..3b6159be2bb2271 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query27.sql @@ -0,0 +1,20 @@ +select i_item_id, + s_state, grouping(s_state) g_state, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, store, item + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_store_sk = s_store_sk and + ss_cdemo_sk = cd_demo_sk and + cd_gender = 'M' and + cd_marital_status = 'W' and + cd_education_status = 'Secondary' and + d_year = 1999 and + s_state in ('TN','TN', 'TN', 'TN', 'TN', 'TN') + group by rollup (i_item_id, s_state) + order by i_item_id + ,s_state + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query28.sql b/tools/tpcds-tools/queries/sf1000/query28.sql new file mode 100644 index 000000000000000..b34cbcc1778d502 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query28.sql @@ -0,0 +1,50 @@ +select * +from (select avg(ss_list_price) B1_LP + ,count(ss_list_price) B1_CNT + ,count(distinct ss_list_price) B1_CNTD + from store_sales + where ss_quantity between 0 and 5 + and (ss_list_price between 107 and 107+10 + or ss_coupon_amt between 1319 and 1319+1000 + or ss_wholesale_cost between 60 and 60+20)) B1, + (select avg(ss_list_price) B2_LP + ,count(ss_list_price) B2_CNT + ,count(distinct ss_list_price) B2_CNTD + from store_sales + where ss_quantity between 6 and 10 + and (ss_list_price between 23 and 23+10 + or ss_coupon_amt between 825 and 825+1000 + or ss_wholesale_cost between 43 and 43+20)) B2, + (select avg(ss_list_price) B3_LP + ,count(ss_list_price) B3_CNT + ,count(distinct ss_list_price) B3_CNTD + from store_sales + where ss_quantity between 11 and 15 + and (ss_list_price between 74 and 74+10 + or ss_coupon_amt between 4381 and 4381+1000 + or ss_wholesale_cost between 57 and 57+20)) B3, + (select avg(ss_list_price) B4_LP + ,count(ss_list_price) B4_CNT + ,count(distinct ss_list_price) B4_CNTD + from store_sales + where ss_quantity between 16 and 20 + and (ss_list_price between 89 and 89+10 + or ss_coupon_amt between 3117 and 3117+1000 + or ss_wholesale_cost between 68 and 68+20)) B4, + (select avg(ss_list_price) B5_LP + ,count(ss_list_price) B5_CNT + ,count(distinct ss_list_price) B5_CNTD + from store_sales + where ss_quantity between 21 and 25 + and (ss_list_price between 58 and 58+10 + or ss_coupon_amt between 9402 and 9402+1000 + or ss_wholesale_cost between 38 and 38+20)) B5, + (select avg(ss_list_price) B6_LP + ,count(ss_list_price) B6_CNT + ,count(distinct ss_list_price) B6_CNTD + from store_sales + where ss_quantity between 26 and 30 + and (ss_list_price between 64 and 64+10 + or ss_coupon_amt between 5792 and 5792+1000 + or ss_wholesale_cost between 73 and 73+20)) B6 +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query29.sql b/tools/tpcds-tools/queries/sf1000/query29.sql new file mode 100644 index 000000000000000..49b31ce112f7b0f --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query29.sql @@ -0,0 +1,44 @@ +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,max(ss_quantity) as store_sales_quantity + ,max(sr_return_quantity) as store_returns_quantity + ,max(cs_quantity) as catalog_sales_quantity + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 1998 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 4 + 3 + and d2.d_year = 1998 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_year in (1998,1998+1,1998+2) + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query3.sql b/tools/tpcds-tools/queries/sf1000/query3.sql new file mode 100644 index 000000000000000..6f368b3a2b52800 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query3.sql @@ -0,0 +1,18 @@ +select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_sales_price) sum_agg + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 816 + and dt.d_moy=11 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,sum_agg desc + ,brand_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query30.sql b/tools/tpcds-tools/queries/sf1000/query30.sql new file mode 100644 index 000000000000000..f5c7b4ee3c9a6da --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query30.sql @@ -0,0 +1,28 @@ +with customer_total_return as + (select wr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(wr_return_amt) as ctr_total_return + from web_returns + ,date_dim + ,customer_address + where wr_returned_date_sk = d_date_sk + and d_year =2000 + and wr_returning_addr_sk = ca_address_sk + group by wr_returning_customer_sk + ,ca_state) + select c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date_sk,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'AR' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date_sk,ctr_total_return +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query31.sql b/tools/tpcds-tools/queries/sf1000/query31.sql new file mode 100644 index 000000000000000..d0c361d5e777de4 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query31.sql @@ -0,0 +1,49 @@ +with ss as + (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales + from store_sales,date_dim,customer_address + where ss_sold_date_sk = d_date_sk + and ss_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year), + ws as + (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales + from web_sales,date_dim,customer_address + where ws_sold_date_sk = d_date_sk + and ws_bill_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year) + select + ss1.ca_county + ,ss1.d_year + ,ws2.web_sales/ws1.web_sales web_q1_q2_increase + ,ss2.store_sales/ss1.store_sales store_q1_q2_increase + ,ws3.web_sales/ws2.web_sales web_q2_q3_increase + ,ss3.store_sales/ss2.store_sales store_q2_q3_increase + from + ss ss1 + ,ss ss2 + ,ss ss3 + ,ws ws1 + ,ws ws2 + ,ws ws3 + where + ss1.d_qoy = 1 + and ss1.d_year = 1999 + and ss1.ca_county = ss2.ca_county + and ss2.d_qoy = 2 + and ss2.d_year = 1999 + and ss2.ca_county = ss3.ca_county + and ss3.d_qoy = 3 + and ss3.d_year = 1999 + and ss1.ca_county = ws1.ca_county + and ws1.d_qoy = 1 + and ws1.d_year = 1999 + and ws1.ca_county = ws2.ca_county + and ws2.d_qoy = 2 + and ws2.d_year = 1999 + and ws1.ca_county = ws3.ca_county + and ws3.d_qoy = 3 + and ws3.d_year =1999 + and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end + > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end + and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end + > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end + order by store_q2_q3_increase; diff --git a/tools/tpcds-tools/queries/sf1000/query32.sql b/tools/tpcds-tools/queries/sf1000/query32.sql new file mode 100644 index 000000000000000..2d74830b3fa41c0 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query32.sql @@ -0,0 +1,25 @@ +select sum(cs_ext_discount_amt) as "excess discount amount" +from + catalog_sales + ,item + ,date_dim +where +i_manufact_id = 722 +and i_item_sk = cs_item_sk +and d_date between '2001-03-09' and + (cast('2001-03-09' as date) + interval 90 day) +and d_date_sk = cs_sold_date_sk +and cs_ext_discount_amt + > ( + select + 1.3 * avg(cs_ext_discount_amt) + from + catalog_sales + ,date_dim + where + cs_item_sk = i_item_sk + and d_date between '2001-03-09' and + (cast('2001-03-09' as date) + interval 90 day) + and d_date_sk = cs_sold_date_sk + ) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query33.sql b/tools/tpcds-tools/queries/sf1000/query33.sql new file mode 100644 index 000000000000000..35e52280e4cc359 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query33.sql @@ -0,0 +1,72 @@ +with ss as ( + select + i_manufact_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 3 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id), + cs as ( + select + i_manufact_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 3 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id), + ws as ( + select + i_manufact_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 3 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id) + select i_manufact_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_manufact_id + order by total_sales +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query34.sql b/tools/tpcds-tools/queries/sf1000/query34.sql new file mode 100644 index 000000000000000..cbe0fb1e432f090 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query34.sql @@ -0,0 +1,28 @@ +select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28) + and (household_demographics.hd_buy_potential = '1001-5000' or + household_demographics.hd_buy_potential = '0-500') + and household_demographics.hd_vehicle_count > 0 + and (case when household_demographics.hd_vehicle_count > 0 + then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count + else null + end) > 1.2 + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County', + 'Williamson County','Williamson County','Williamson County','Williamson County') + group by ss_ticket_number,ss_customer_sk) dn,customer + where ss_customer_sk = c_customer_sk + and cnt between 15 and 20 + order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc, ss_ticket_number; diff --git a/tools/tpcds-tools/queries/sf1000/query35.sql b/tools/tpcds-tools/queries/sf1000/query35.sql new file mode 100644 index 000000000000000..94db12fc554a3e5 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query35.sql @@ -0,0 +1,55 @@ +select + ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + count(*) cnt1, + avg(cd_dep_count), + stddev_samp(cd_dep_count), + sum(cd_dep_count), + cd_dep_employed_count, + count(*) cnt2, + avg(cd_dep_employed_count), + stddev_samp(cd_dep_employed_count), + sum(cd_dep_employed_count), + cd_dep_college_count, + count(*) cnt3, + avg(cd_dep_college_count), + stddev_samp(cd_dep_college_count), + sum(cd_dep_college_count) + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 1999 and + d_qoy < 4) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 1999 and + d_qoy < 4) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 1999 and + d_qoy < 4)) + group by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query36.sql b/tools/tpcds-tools/queries/sf1000/query36.sql new file mode 100644 index 000000000000000..becddb653e414d5 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query36.sql @@ -0,0 +1,27 @@ +select + sum(ss_net_profit)/sum(ss_ext_sales_price) as gross_margin + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,item + ,store + where + d1.d_year = 2000 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and s_state in ('TN','TN','TN','TN', + 'TN','TN','TN','TN') + group by rollup(i_category,i_class) + order by + lochierarchy desc + ,case when lochierarchy = 0 then i_category end + ,rank_within_parent + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query37.sql b/tools/tpcds-tools/queries/sf1000/query37.sql new file mode 100644 index 000000000000000..1b57326035777ca --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query37.sql @@ -0,0 +1,14 @@ +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, catalog_sales + where i_current_price between 29 and 29 + 30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('2002-03-29' as date) and (cast('2002-03-29' as date) + interval 60 day) + and i_manufact_id in (705,742,777,944) + and inv_quantity_on_hand between 100 and 500 + and cs_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query38.sql b/tools/tpcds-tools/queries/sf1000/query38.sql new file mode 100644 index 000000000000000..e7e5373e70dc61e --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query38.sql @@ -0,0 +1,20 @@ +select count(*) from ( + select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1189 and 1189 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1189 and 1189 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1189 and 1189 + 11 +) hot_cust +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query39.sql b/tools/tpcds-tools/queries/sf1000/query39.sql new file mode 100644 index 000000000000000..48e4e3a12a5cb7e --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query39.sql @@ -0,0 +1,49 @@ +with inv as +(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stdev,mean, case mean when 0 then null else stdev/mean end cov + from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean + from inventory + ,item + ,warehouse + ,date_dim + where inv_item_sk = i_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_year =2000 + group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo + where case mean when 0 then 0 else stdev/mean end > 1) +select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov + ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov +from inv inv1,inv inv2 +where inv1.i_item_sk = inv2.i_item_sk + and inv1.w_warehouse_sk = inv2.w_warehouse_sk + and inv1.d_moy=1 + and inv2.d_moy=1+1 +order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov + ,inv2.d_moy,inv2.mean, inv2.cov; +with inv as +(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stdev,mean, case mean when 0 then null else stdev/mean end cov + from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean + from inventory + ,item + ,warehouse + ,date_dim + where inv_item_sk = i_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_year =2000 + group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo + where case mean when 0 then 0 else stdev/mean end > 1) +select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov + ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov +from inv inv1,inv inv2 +where inv1.i_item_sk = inv2.i_item_sk + and inv1.w_warehouse_sk = inv2.w_warehouse_sk + and inv1.d_moy=1 + and inv2.d_moy=1+1 + and inv1.cov > 1.5 +order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov + ,inv2.d_moy,inv2.mean, inv2.cov; diff --git a/tools/tpcds-tools/queries/sf1000/query4.sql b/tools/tpcds-tools/queries/sf1000/query4.sql new file mode 100644 index 000000000000000..99d6806f73d77a8 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query4.sql @@ -0,0 +1,113 @@ +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(((ss_ext_list_price-ss_ext_wholesale_cost-ss_ext_discount_amt)+ss_ext_sales_price)/2) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum((((cs_ext_list_price-cs_ext_wholesale_cost-cs_ext_discount_amt)+cs_ext_sales_price)/2) ) year_total + ,'c' sale_type + from customer + ,catalog_sales + ,date_dim + where c_customer_sk = cs_bill_customer_sk + and cs_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year +union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum((((ws_ext_list_price-ws_ext_wholesale_cost-ws_ext_discount_amt)+ws_ext_sales_price)/2) ) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + ) + select + t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_birth_country + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_c_firstyear + ,year_total t_c_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_c_secyear.customer_id + and t_s_firstyear.customer_id = t_c_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_c_firstyear.sale_type = 'c' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_c_secyear.sale_type = 'c' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.dyear = 1999 + and t_s_secyear.dyear = 1999+1 + and t_c_firstyear.dyear = 1999 + and t_c_secyear.dyear = 1999+1 + and t_w_firstyear.dyear = 1999 + and t_w_secyear.dyear = 1999+1 + and t_s_firstyear.year_total > 0 + and t_c_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end + and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end + > case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end + order by t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_birth_country +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query40.sql b/tools/tpcds-tools/queries/sf1000/query40.sql new file mode 100644 index 000000000000000..5e971da2a8597b3 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query40.sql @@ -0,0 +1,25 @@ +select + w_state + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('2001-05-02' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_before + ,sum(case when (cast(d_date as date) >= cast ('2001-05-02' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_after + from + catalog_sales left outer join catalog_returns on + (cs_order_number = cr_order_number + and cs_item_sk = cr_item_sk) + ,warehouse + ,item + ,date_dim + where + i_current_price between 0.99 and 1.49 + and i_item_sk = cs_item_sk + and cs_warehouse_sk = w_warehouse_sk + and cs_sold_date_sk = d_date_sk + and d_date between (cast ('2001-05-02' as date) - interval 30 day) + and (cast ('2001-05-02' as date) + interval 30 day) + group by + w_state,i_item_id + order by w_state,i_item_id +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query41.sql b/tools/tpcds-tools/queries/sf1000/query41.sql new file mode 100644 index 000000000000000..65753b637718846 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query41.sql @@ -0,0 +1,49 @@ +select distinct(i_product_name) + from item i1 + where i_manufact_id between 704 and 704+40 + and (select count(*) as item_cnt + from item + where (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'forest' or i_color = 'lime') and + (i_units = 'Pallet' or i_units = 'Pound') and + (i_size = 'economy' or i_size = 'small') + ) or + (i_category = 'Women' and + (i_color = 'navy' or i_color = 'slate') and + (i_units = 'Gross' or i_units = 'Bunch') and + (i_size = 'extra large' or i_size = 'petite') + ) or + (i_category = 'Men' and + (i_color = 'powder' or i_color = 'sky') and + (i_units = 'Dozen' or i_units = 'Lb') and + (i_size = 'N/A' or i_size = 'large') + ) or + (i_category = 'Men' and + (i_color = 'maroon' or i_color = 'smoke') and + (i_units = 'Ounce' or i_units = 'Case') and + (i_size = 'economy' or i_size = 'small') + ))) or + (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'dark' or i_color = 'aquamarine') and + (i_units = 'Ton' or i_units = 'Tbl') and + (i_size = 'economy' or i_size = 'small') + ) or + (i_category = 'Women' and + (i_color = 'frosted' or i_color = 'plum') and + (i_units = 'Dram' or i_units = 'Box') and + (i_size = 'extra large' or i_size = 'petite') + ) or + (i_category = 'Men' and + (i_color = 'papaya' or i_color = 'peach') and + (i_units = 'Bundle' or i_units = 'Carton') and + (i_size = 'N/A' or i_size = 'large') + ) or + (i_category = 'Men' and + (i_color = 'firebrick' or i_color = 'sienna') and + (i_units = 'Cup' or i_units = 'Each') and + (i_size = 'economy' or i_size = 'small') + )))) > 0 + order by i_product_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query42.sql b/tools/tpcds-tools/queries/sf1000/query42.sql new file mode 100644 index 000000000000000..ae388efd37e5738 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query42.sql @@ -0,0 +1,19 @@ +select dt.d_year + ,item.i_category_id + ,item.i_category + ,sum(ss_ext_sales_price) + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=11 + and dt.d_year=1998 + group by dt.d_year + ,item.i_category_id + ,item.i_category + order by sum(ss_ext_sales_price) desc,dt.d_year + ,item.i_category_id + ,item.i_category +limit 100 ; diff --git a/tools/tpcds-tools/queries/sf1000/query43.sql b/tools/tpcds-tools/queries/sf1000/query43.sql new file mode 100644 index 000000000000000..b1caee291694661 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query43.sql @@ -0,0 +1,16 @@ +select s_store_name, s_store_id, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from date_dim, store_sales, store + where d_date_sk = ss_sold_date_sk and + s_store_sk = ss_store_sk and + s_gmt_offset = -5 and + d_year = 2000 + group by s_store_name, s_store_id + order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query44.sql b/tools/tpcds-tools/queries/sf1000/query44.sql new file mode 100644 index 000000000000000..294fca870fb09c5 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query44.sql @@ -0,0 +1,32 @@ +select asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing +from(select * + from (select item_sk,rank() over (order by rank_col asc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 4 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 4 + and ss_hdemo_sk is null + group by ss_store_sk))V1)V11 + where rnk < 11) asceding, + (select * + from (select item_sk,rank() over (order by rank_col desc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 4 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 4 + and ss_hdemo_sk is null + group by ss_store_sk))V2)V21 + where rnk < 11) descending, +item i1, +item i2 +where asceding.rnk = descending.rnk + and i1.i_item_sk=asceding.item_sk + and i2.i_item_sk=descending.item_sk +order by asceding.rnk +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query45.sql b/tools/tpcds-tools/queries/sf1000/query45.sql new file mode 100644 index 000000000000000..32e490332a76396 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query45.sql @@ -0,0 +1,17 @@ +select ca_zip, ca_city, sum(ws_sales_price) + from web_sales, customer, customer_address, date_dim, item + where ws_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ws_item_sk = i_item_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792') + or + i_item_id in (select i_item_id + from item + where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) + ) + ) + and ws_sold_date_sk = d_date_sk + and d_qoy = 1 and d_year = 2000 + group by ca_zip, ca_city + order by ca_zip, ca_city + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query46.sql b/tools/tpcds-tools/queries/sf1000/query46.sql new file mode 100644 index 000000000000000..b9390efcb107115 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query46.sql @@ -0,0 +1,32 @@ +select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and (household_demographics.hd_dep_count = 8 or + household_demographics.hd_vehicle_count= 0) + and date_dim.d_dow in (6,0) + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_city in ('Midway','Fairview','Fairview','Midway','Fairview') + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query47.sql b/tools/tpcds-tools/queries/sf1000/query47.sql new file mode 100644 index 000000000000000..2b056c61502f7b7 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query47.sql @@ -0,0 +1,48 @@ +with v1 as( + select i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, + s_store_name, s_company_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + s_store_name, s_company_name + order by d_year, d_moy) rn + from item, store_sales, date_dim, store + where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + ( + d_year = 2000 or + ( d_year = 2000-1 and d_moy =12) or + ( d_year = 2000+1 and d_moy =1) + ) + group by i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy), + v2 as( + select v1.s_store_name, v1.s_company_name + ,v1.d_year + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1.s_store_name = v1_lag.s_store_name and + v1.s_store_name = v1_lead.s_store_name and + v1.s_company_name = v1_lag.s_company_name and + v1.s_company_name = v1_lead.s_company_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 2000 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, nsum + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query48.sql b/tools/tpcds-tools/queries/sf1000/query48.sql new file mode 100644 index 000000000000000..e454f7780096f5f --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query48.sql @@ -0,0 +1,64 @@ +select sum (ss_quantity) + from store_sales, store, customer_demographics, customer_address, date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 2001 + and + ( + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'S' + and + cd_education_status = 'Secondary' + and + ss_sales_price between 100.00 and 150.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'M' + and + cd_education_status = '2 yr Degree' + and + ss_sales_price between 50.00 and 100.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'D' + and + cd_education_status = 'Advanced Degree' + and + ss_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('ND', 'NY', 'SD') + and ss_net_profit between 0 and 2000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('MD', 'GA', 'KS') + and ss_net_profit between 150 and 3000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('CO', 'MN', 'NC') + and ss_net_profit between 50 and 25000 + ) + ) +; diff --git a/tools/tpcds-tools/queries/sf1000/query49.sql b/tools/tpcds-tools/queries/sf1000/query49.sql new file mode 100644 index 000000000000000..5fdadcb1b62f713 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query49.sql @@ -0,0 +1,126 @@ +select channel, item, return_ratio, return_rank, currency_rank from + (select + 'web' as channel + ,web.item + ,web.return_ratio + ,web.return_rank + ,web.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select ws.ws_item_sk as item + ,(cast(sum(coalesce(wr.wr_return_quantity,0)) as decimal(15,4))/ + cast(sum(coalesce(ws.ws_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(wr.wr_return_amt,0)) as decimal(15,4))/ + cast(sum(coalesce(ws.ws_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + web_sales ws left outer join web_returns wr + on (ws.ws_order_number = wr.wr_order_number and + ws.ws_item_sk = wr.wr_item_sk) + ,date_dim + where + wr.wr_return_amt > 10000 + and ws.ws_net_profit > 1 + and ws.ws_net_paid > 0 + and ws.ws_quantity > 0 + and ws_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 11 + group by ws.ws_item_sk + ) in_web + ) web + where + ( + web.return_rank <= 10 + or + web.currency_rank <= 10 + ) + union + select + 'catalog' as channel + ,catalog.item + ,catalog.return_ratio + ,catalog.return_rank + ,catalog.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select + cs.cs_item_sk as item + ,(cast(sum(coalesce(cr.cr_return_quantity,0)) as decimal(15,4))/ + cast(sum(coalesce(cs.cs_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(cr.cr_return_amount,0)) as decimal(15,4))/ + cast(sum(coalesce(cs.cs_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + catalog_sales cs left outer join catalog_returns cr + on (cs.cs_order_number = cr.cr_order_number and + cs.cs_item_sk = cr.cr_item_sk) + ,date_dim + where + cr.cr_return_amount > 10000 + and cs.cs_net_profit > 1 + and cs.cs_net_paid > 0 + and cs.cs_quantity > 0 + and cs_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 11 + group by cs.cs_item_sk + ) in_cat + ) catalog + where + ( + catalog.return_rank <= 10 + or + catalog.currency_rank <=10 + ) + union + select + 'store' as channel + ,store.item + ,store.return_ratio + ,store.return_rank + ,store.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select sts.ss_item_sk as item + ,(cast(sum(coalesce(sr.sr_return_quantity,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(sr.sr_return_amt,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + store_sales sts left outer join store_returns sr + on (sts.ss_ticket_number = sr.sr_ticket_number and sts.ss_item_sk = sr.sr_item_sk) + ,date_dim + where + sr.sr_return_amt > 10000 + and sts.ss_net_profit > 1 + and sts.ss_net_paid > 0 + and sts.ss_quantity > 0 + and ss_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 11 + group by sts.ss_item_sk + ) in_store + ) store + where ( + store.return_rank <= 10 + or + store.currency_rank <= 10 + ) + ) + t order by 1,4,5,2 + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query5.sql b/tools/tpcds-tools/queries/sf1000/query5.sql new file mode 100644 index 000000000000000..96c97b7c8a1636a --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query5.sql @@ -0,0 +1,125 @@ +with ssr as + (select s_store_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( select ss_store_sk as store_sk, + ss_sold_date_sk as date_sk, + ss_ext_sales_price as sales_price, + ss_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from store_sales + union all + select sr_store_sk as store_sk, + sr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + sr_return_amt as return_amt, + sr_net_loss as net_loss + from store_returns + ) salesreturns, + date_dim, + store + where date_sk = d_date_sk + and d_date between cast('2000-08-19' as date) + and (cast('2000-08-19' as date) + interval 14 day) + and store_sk = s_store_sk + group by s_store_id) + , + csr as + (select cp_catalog_page_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( select cs_catalog_page_sk as page_sk, + cs_sold_date_sk as date_sk, + cs_ext_sales_price as sales_price, + cs_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from catalog_sales + union all + select cr_catalog_page_sk as page_sk, + cr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + cr_return_amount as return_amt, + cr_net_loss as net_loss + from catalog_returns + ) salesreturns, + date_dim, + catalog_page + where date_sk = d_date_sk + and d_date between cast('2000-08-19' as date) + and (cast('2000-08-19' as date) + interval 14 day) + and page_sk = cp_catalog_page_sk + group by cp_catalog_page_id) + , + wsr as + (select web_site_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( select ws_web_site_sk as wsr_web_site_sk, + ws_sold_date_sk as date_sk, + ws_ext_sales_price as sales_price, + ws_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from web_sales + union all + select ws_web_site_sk as wsr_web_site_sk, + wr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + wr_return_amt as return_amt, + wr_net_loss as net_loss + from web_returns left outer join web_sales on + ( wr_item_sk = ws_item_sk + and wr_order_number = ws_order_number) + ) salesreturns, + date_dim, + web_site + where date_sk = d_date_sk + and d_date between cast('2000-08-19' as date) + and (cast('2000-08-19' as date) + interval 14 day) + and wsr_web_site_sk = web_site_sk + group by web_site_id) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , concat('store', s_store_id) id + , sales + , returns + , (profit - profit_loss) as profit + from ssr + union all + select 'catalog channel' as channel + , concat('catalog_page', cp_catalog_page_id) id + , sales + , returns + , (profit - profit_loss) as profit + from csr + union all + select 'web channel' as channel + , concat('web_site', web_site_id) id + , sales + , returns + , (profit - profit_loss) as profit + from wsr + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query50.sql b/tools/tpcds-tools/queries/sf1000/query50.sql new file mode 100644 index 000000000000000..2a45d211cc3628f --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query50.sql @@ -0,0 +1,56 @@ +select + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 30) and + (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 60) and + (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 90) and + (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 120) then 1 else 0 end) as ">120 days" +from + store_sales + ,store_returns + ,store + ,date_dim d1 + ,date_dim d2 +where + d2.d_year = 2001 +and d2.d_moy = 8 +and ss_ticket_number = sr_ticket_number +and ss_item_sk = sr_item_sk +and ss_sold_date_sk = d1.d_date_sk +and sr_returned_date_sk = d2.d_date_sk +and ss_customer_sk = sr_customer_sk +and ss_store_sk = s_store_sk +group by + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip +order by s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query51.sql b/tools/tpcds-tools/queries/sf1000/query51.sql new file mode 100644 index 000000000000000..fe90be989f8a416 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query51.sql @@ -0,0 +1,42 @@ +WITH web_v1 as ( +select + ws_item_sk item_sk, d_date, + sum(sum(ws_sales_price)) + over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from web_sales + ,date_dim +where ws_sold_date_sk=d_date_sk + and d_month_seq between 1212 and 1212+11 + and ws_item_sk is not NULL +group by ws_item_sk, d_date), +store_v1 as ( +select + ss_item_sk item_sk, d_date, + sum(sum(ss_sales_price)) + over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from store_sales + ,date_dim +where ss_sold_date_sk=d_date_sk + and d_month_seq between 1212 and 1212+11 + and ss_item_sk is not NULL +group by ss_item_sk, d_date) + select * +from (select item_sk + ,d_date + ,web_sales + ,store_sales + ,max(web_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) web_cumulative + ,max(store_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) store_cumulative + from (select case when web.item_sk is not null then web.item_sk else store.item_sk end item_sk + ,case when web.d_date is not null then web.d_date else store.d_date end d_date + ,web.cume_sales web_sales + ,store.cume_sales store_sales + from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk + and web.d_date = store.d_date) + )x )y +where web_cumulative > store_cumulative +order by item_sk + ,d_date +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query52.sql b/tools/tpcds-tools/queries/sf1000/query52.sql new file mode 100644 index 000000000000000..2fce4c3fdf1c278 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query52.sql @@ -0,0 +1,19 @@ +select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_ext_sales_price) ext_price + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=12 + and dt.d_year=2000 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,ext_price desc + ,brand_id +limit 100 ; diff --git a/tools/tpcds-tools/queries/sf1000/query53.sql b/tools/tpcds-tools/queries/sf1000/query53.sql new file mode 100644 index 000000000000000..68feef7b4bae1fa --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query53.sql @@ -0,0 +1,25 @@ +select * from +(select i_manufact_id, +sum(ss_sales_price) sum_sales, +avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and +ss_sold_date_sk = d_date_sk and +ss_store_sk = s_store_sk and +d_month_seq in (1186,1186+1,1186+2,1186+3,1186+4,1186+5,1186+6,1186+7,1186+8,1186+9,1186+10,1186+11) and +((i_category in ('Books','Children','Electronics') and +i_class in ('personal','portable','reference','self-help') and +i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7', + 'exportiunivamalg #9','scholaramalgamalg #9')) +or(i_category in ('Women','Music','Men') and +i_class in ('accessories','classical','fragrances','pants') and +i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1', + 'importoamalg #1'))) +group by i_manufact_id, d_qoy ) tmp1 +where case when avg_quarterly_sales > 0 + then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales + else null end > 0.1 +order by avg_quarterly_sales, + sum_sales, + i_manufact_id +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query54.sql b/tools/tpcds-tools/queries/sf1000/query54.sql new file mode 100644 index 000000000000000..f195c56ce66cf14 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query54.sql @@ -0,0 +1,53 @@ +with my_customers as ( + select distinct c_customer_sk + , c_current_addr_sk + from + ( select cs_sold_date_sk sold_date_sk, + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + from catalog_sales + union all + select ws_sold_date_sk sold_date_sk, + ws_bill_customer_sk customer_sk, + ws_item_sk item_sk + from web_sales + ) cs_or_ws_sales, + item, + date_dim, + customer + where sold_date_sk = d_date_sk + and item_sk = i_item_sk + and i_category = 'Music' + and i_class = 'country' + and c_customer_sk = cs_or_ws_sales.customer_sk + and d_moy = 1 + and d_year = 1999 + ) + , my_revenue as ( + select c_customer_sk, + sum(ss_ext_sales_price) as revenue + from my_customers, + store_sales, + customer_address, + store, + date_dim + where c_current_addr_sk = ca_address_sk + and ca_county = s_county + and ca_state = s_state + and ss_sold_date_sk = d_date_sk + and c_customer_sk = ss_customer_sk + and d_month_seq between (select distinct d_month_seq+1 + from date_dim where d_year = 1999 and d_moy = 1) + and (select distinct d_month_seq+3 + from date_dim where d_year = 1999 and d_moy = 1) + group by c_customer_sk + ) + , segments as + (select cast((revenue/50) as int) as segment + from my_revenue + ) + select segment, count(*) as num_customers, segment*50 as segment_base + from segments + group by segment + order by segment, num_customers + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query55.sql b/tools/tpcds-tools/queries/sf1000/query55.sql new file mode 100644 index 000000000000000..7248468b83c3546 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query55.sql @@ -0,0 +1,11 @@ +select i_brand_id brand_id, i_brand brand, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=52 + and d_moy=11 + and d_year=2000 + group by i_brand, i_brand_id + order by ext_price desc, i_brand_id +limit 100 ; diff --git a/tools/tpcds-tools/queries/sf1000/query56.sql b/tools/tpcds-tools/queries/sf1000/query56.sql new file mode 100644 index 000000000000000..8b8bfa96ce4ed8d --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query56.sql @@ -0,0 +1,66 @@ +with ss as ( + select i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where i_item_id in (select + i_item_id +from item +where i_color in ('powder','orchid','pink')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 3 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + cs as ( + select i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('powder','orchid','pink')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 3 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + ws as ( + select i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('powder','orchid','pink')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 3 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id) + select i_item_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by total_sales, + i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query57.sql b/tools/tpcds-tools/queries/sf1000/query57.sql new file mode 100644 index 000000000000000..6c08d2623a16536 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query57.sql @@ -0,0 +1,45 @@ +with v1 as( + select i_category, i_brand, + cc_name, + d_year, d_moy, + sum(cs_sales_price) sum_sales, + avg(sum(cs_sales_price)) over + (partition by i_category, i_brand, + cc_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + cc_name + order by d_year, d_moy) rn + from item, catalog_sales, date_dim, call_center + where cs_item_sk = i_item_sk and + cs_sold_date_sk = d_date_sk and + cc_call_center_sk= cs_call_center_sk and + ( + d_year = 2001 or + ( d_year = 2001-1 and d_moy =12) or + ( d_year = 2001+1 and d_moy =1) + ) + group by i_category, i_brand, + cc_name , d_year, d_moy), + v2 as( + select v1.i_category, v1.i_brand, v1.cc_name + ,v1.d_year + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1. cc_name = v1_lag. cc_name and + v1. cc_name = v1_lead. cc_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 2001 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, avg_monthly_sales + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query58.sql b/tools/tpcds-tools/queries/sf1000/query58.sql new file mode 100644 index 000000000000000..f45b29749f0a4d5 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query58.sql @@ -0,0 +1,62 @@ +with ss_items as + (select i_item_id item_id + ,sum(ss_ext_sales_price) ss_item_rev + from store_sales + ,item + ,date_dim + where ss_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '2001-06-16')) + and ss_sold_date_sk = d_date_sk + group by i_item_id), + cs_items as + (select i_item_id item_id + ,sum(cs_ext_sales_price) cs_item_rev + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '2001-06-16')) + and cs_sold_date_sk = d_date_sk + group by i_item_id), + ws_items as + (select i_item_id item_id + ,sum(ws_ext_sales_price) ws_item_rev + from web_sales + ,item + ,date_dim + where ws_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq =(select d_week_seq + from date_dim + where d_date = '2001-06-16')) + and ws_sold_date_sk = d_date_sk + group by i_item_id) + select ss_items.item_id + ,ss_item_rev + ,ss_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ss_dev + ,cs_item_rev + ,cs_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 cs_dev + ,ws_item_rev + ,ws_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ws_dev + ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average + from ss_items,cs_items,ws_items + where ss_items.item_id=cs_items.item_id + and ss_items.item_id=ws_items.item_id + and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + order by item_id + ,ss_item_rev + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query59.sql b/tools/tpcds-tools/queries/sf1000/query59.sql new file mode 100644 index 000000000000000..ad596772673c3ae --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query59.sql @@ -0,0 +1,41 @@ +with wss as + (select d_week_seq, + ss_store_sk, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + group by d_week_seq,ss_store_sk + ) + select s_store_name1,s_store_id1,d_week_seq1 + ,sun_sales1/sun_sales2,mon_sales1/mon_sales2 + ,tue_sales1/tue_sales2,wed_sales1/wed_sales2,thu_sales1/thu_sales2 + ,fri_sales1/fri_sales2,sat_sales1/sat_sales2 + from + (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1 + ,s_store_id s_store_id1,sun_sales sun_sales1 + ,mon_sales mon_sales1,tue_sales tue_sales1 + ,wed_sales wed_sales1,thu_sales thu_sales1 + ,fri_sales fri_sales1,sat_sales sat_sales1 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + wss.ss_store_sk = s_store_sk and + d_month_seq between 1195 and 1195 + 11) y, + (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2 + ,s_store_id s_store_id2,sun_sales sun_sales2 + ,mon_sales mon_sales2,tue_sales tue_sales2 + ,wed_sales wed_sales2,thu_sales thu_sales2 + ,fri_sales fri_sales2,sat_sales sat_sales2 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + wss.ss_store_sk = s_store_sk and + d_month_seq between 1195+ 12 and 1195 + 23) x + where s_store_id1=s_store_id2 + and d_week_seq1=d_week_seq2-52 + order by s_store_name1,s_store_id1,d_week_seq1 +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query6.sql b/tools/tpcds-tools/queries/sf1000/query6.sql new file mode 100644 index 000000000000000..00decd2cdc48a55 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query6.sql @@ -0,0 +1,23 @@ +select a.ca_state state, count(*) cnt + from customer_address a + ,customer c + ,store_sales s + ,date_dim d + ,item i + where a.ca_address_sk = c.c_current_addr_sk + and c.c_customer_sk = s.ss_customer_sk + and s.ss_sold_date_sk = d.d_date_sk + and s.ss_item_sk = i.i_item_sk + and d.d_month_seq = + (select distinct (d_month_seq) + from date_dim + where d_year = 2002 + and d_moy = 3 ) + and i.i_current_price > 1.2 * + (select avg(j.i_current_price) + from item j + where j.i_category = i.i_category) + group by a.ca_state + having count(*) >= 10 + order by cnt, a.ca_state + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query60.sql b/tools/tpcds-tools/queries/sf1000/query60.sql new file mode 100644 index 000000000000000..9efaa98622e94ad --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query60.sql @@ -0,0 +1,75 @@ +with ss as ( + select + i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Jewelry')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 10 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_item_id), + cs as ( + select + i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Jewelry')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 10 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_item_id), + ws as ( + select + i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Jewelry')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 10 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_item_id) + select + i_item_id +,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by i_item_id + ,total_sales + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query61.sql b/tools/tpcds-tools/queries/sf1000/query61.sql new file mode 100644 index 000000000000000..c9310e3c0650c36 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query61.sql @@ -0,0 +1,41 @@ +select promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100 +from + (select sum(ss_ext_sales_price) promotions + from store_sales + ,store + ,promotion + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_promo_sk = p_promo_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -7 + and i_category = 'Home' + and (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y') + and s_gmt_offset = -7 + and d_year = 2000 + and d_moy = 12) promotional_sales, + (select sum(ss_ext_sales_price) total + from store_sales + ,store + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -7 + and i_category = 'Home' + and s_gmt_offset = -7 + and d_year = 2000 + and d_moy = 12) all_sales +order by promotions, total +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query62.sql b/tools/tpcds-tools/queries/sf1000/query62.sql new file mode 100644 index 000000000000000..4475549eaca2348 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query62.sql @@ -0,0 +1,32 @@ +select + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 30) and + (ws_ship_date_sk - ws_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 60) and + (ws_ship_date_sk - ws_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 90) and + (ws_ship_date_sk - ws_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 120) then 1 else 0 end) as ">120 days" +from + web_sales + ,warehouse + ,ship_mode + ,web_site + ,date_dim +where + d_month_seq between 1223 and 1223 + 11 +and ws_ship_date_sk = d_date_sk +and ws_warehouse_sk = w_warehouse_sk +and ws_ship_mode_sk = sm_ship_mode_sk +and ws_web_site_sk = web_site_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,web_name +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query63.sql b/tools/tpcds-tools/queries/sf1000/query63.sql new file mode 100644 index 000000000000000..8772871eb6eb3aa --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query63.sql @@ -0,0 +1,26 @@ +select * +from (select i_manager_id + ,sum(ss_sales_price) sum_sales + ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales + from item + ,store_sales + ,date_dim + ,store + where ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and d_month_seq in (1222,1222+1,1222+2,1222+3,1222+4,1222+5,1222+6,1222+7,1222+8,1222+9,1222+10,1222+11) + and (( i_category in ('Books','Children','Electronics') + and i_class in ('personal','portable','reference','self-help') + and i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7', + 'exportiunivamalg #9','scholaramalgamalg #9')) + or( i_category in ('Women','Music','Men') + and i_class in ('accessories','classical','fragrances','pants') + and i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1', + 'importoamalg #1'))) +group by i_manager_id, d_moy) tmp1 +where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 +order by i_manager_id + ,avg_monthly_sales + ,sum_sales +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query64.sql b/tools/tpcds-tools/queries/sf1000/query64.sql new file mode 100644 index 000000000000000..21e6c28590f532d --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query64.sql @@ -0,0 +1,118 @@ +with cs_ui as + (select cs_item_sk + ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund + from catalog_sales + ,catalog_returns + where cs_item_sk = cr_item_sk + and cs_order_number = cr_order_number + group by cs_item_sk + having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)), +cross_sales as + (select i_product_name product_name + ,i_item_sk item_sk + ,s_store_name store_name + ,s_zip store_zip + ,ad1.ca_street_number b_street_number + ,ad1.ca_street_name b_street_name + ,ad1.ca_city b_city + ,ad1.ca_zip b_zip + ,ad2.ca_street_number c_street_number + ,ad2.ca_street_name c_street_name + ,ad2.ca_city c_city + ,ad2.ca_zip c_zip + ,d1.d_year as syear + ,d2.d_year as fsyear + ,d3.d_year s2year + ,count(*) cnt + ,sum(ss_wholesale_cost) s1 + ,sum(ss_list_price) s2 + ,sum(ss_coupon_amt) s3 + FROM store_sales + ,store_returns + ,cs_ui + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,customer + ,customer_demographics cd1 + ,customer_demographics cd2 + ,promotion + ,household_demographics hd1 + ,household_demographics hd2 + ,customer_address ad1 + ,customer_address ad2 + ,income_band ib1 + ,income_band ib2 + ,item + WHERE ss_store_sk = s_store_sk AND + ss_sold_date_sk = d1.d_date_sk AND + ss_customer_sk = c_customer_sk AND + ss_cdemo_sk= cd1.cd_demo_sk AND + ss_hdemo_sk = hd1.hd_demo_sk AND + ss_addr_sk = ad1.ca_address_sk and + ss_item_sk = i_item_sk and + ss_item_sk = sr_item_sk and + ss_ticket_number = sr_ticket_number and + ss_item_sk = cs_ui.cs_item_sk and + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_hdemo_sk = hd2.hd_demo_sk AND + c_current_addr_sk = ad2.ca_address_sk and + c_first_sales_date_sk = d2.d_date_sk and + c_first_shipto_date_sk = d3.d_date_sk and + ss_promo_sk = p_promo_sk and + hd1.hd_income_band_sk = ib1.ib_income_band_sk and + hd2.hd_income_band_sk = ib2.ib_income_band_sk and + cd1.cd_marital_status <> cd2.cd_marital_status and + i_color in ('orange','lace','lawn','misty','blush','pink') and + i_current_price between 48 and 48 + 10 and + i_current_price between 48 + 1 and 48 + 15 +group by i_product_name + ,i_item_sk + ,s_store_name + ,s_zip + ,ad1.ca_street_number + ,ad1.ca_street_name + ,ad1.ca_city + ,ad1.ca_zip + ,ad2.ca_street_number + ,ad2.ca_street_name + ,ad2.ca_city + ,ad2.ca_zip + ,d1.d_year + ,d2.d_year + ,d3.d_year +) +select cs1.product_name + ,cs1.store_name + ,cs1.store_zip + ,cs1.b_street_number + ,cs1.b_street_name + ,cs1.b_city + ,cs1.b_zip + ,cs1.c_street_number + ,cs1.c_street_name + ,cs1.c_city + ,cs1.c_zip + ,cs1.syear + ,cs1.cnt + ,cs1.s1 as s11 + ,cs1.s2 as s21 + ,cs1.s3 as s31 + ,cs2.s1 as s12 + ,cs2.s2 as s22 + ,cs2.s3 as s32 + ,cs2.syear + ,cs2.cnt +from cross_sales cs1,cross_sales cs2 +where cs1.item_sk=cs2.item_sk and + cs1.syear = 1999 and + cs2.syear = 1999 + 1 and + cs2.cnt <= cs1.cnt and + cs1.store_name = cs2.store_name and + cs1.store_zip = cs2.store_zip +order by cs1.product_name + ,cs1.store_name + ,cs2.cnt + ,cs1.s1 + ,cs2.s1; diff --git a/tools/tpcds-tools/queries/sf1000/query65.sql b/tools/tpcds-tools/queries/sf1000/query65.sql new file mode 100644 index 000000000000000..62fab6d16eacb82 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query65.sql @@ -0,0 +1,26 @@ +select + s_store_name, + i_item_desc, + sc.revenue, + i_current_price, + i_wholesale_cost, + i_brand + from store, item, + (select ss_store_sk, avg(revenue) as ave + from + (select ss_store_sk, ss_item_sk, + sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1176 and 1176+11 + group by ss_store_sk, ss_item_sk) sa + group by ss_store_sk) sb, + (select ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1176 and 1176+11 + group by ss_store_sk, ss_item_sk) sc + where sb.ss_store_sk = sc.ss_store_sk and + sc.revenue <= 0.1 * sb.ave and + s_store_sk = sc.ss_store_sk and + i_item_sk = sc.ss_item_sk + order by s_store_name, i_item_desc +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query66.sql b/tools/tpcds-tools/queries/sf1000/query66.sql new file mode 100644 index 000000000000000..732608eba51e289 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query66.sql @@ -0,0 +1,217 @@ +select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,ship_carriers + ,year + ,sum(jan_sales) as jan_sales + ,sum(feb_sales) as feb_sales + ,sum(mar_sales) as mar_sales + ,sum(apr_sales) as apr_sales + ,sum(may_sales) as may_sales + ,sum(jun_sales) as jun_sales + ,sum(jul_sales) as jul_sales + ,sum(aug_sales) as aug_sales + ,sum(sep_sales) as sep_sales + ,sum(oct_sales) as oct_sales + ,sum(nov_sales) as nov_sales + ,sum(dec_sales) as dec_sales + ,sum(jan_sales/w_warehouse_sq_ft) as jan_sales_per_sq_foot + ,sum(feb_sales/w_warehouse_sq_ft) as feb_sales_per_sq_foot + ,sum(mar_sales/w_warehouse_sq_ft) as mar_sales_per_sq_foot + ,sum(apr_sales/w_warehouse_sq_ft) as apr_sales_per_sq_foot + ,sum(may_sales/w_warehouse_sq_ft) as may_sales_per_sq_foot + ,sum(jun_sales/w_warehouse_sq_ft) as jun_sales_per_sq_foot + ,sum(jul_sales/w_warehouse_sq_ft) as jul_sales_per_sq_foot + ,sum(aug_sales/w_warehouse_sq_ft) as aug_sales_per_sq_foot + ,sum(sep_sales/w_warehouse_sq_ft) as sep_sales_per_sq_foot + ,sum(oct_sales/w_warehouse_sq_ft) as oct_sales_per_sq_foot + ,sum(nov_sales/w_warehouse_sq_ft) as nov_sales_per_sq_foot + ,sum(dec_sales/w_warehouse_sq_ft) as dec_sales_per_sq_foot + ,sum(jan_net) as jan_net + ,sum(feb_net) as feb_net + ,sum(mar_net) as mar_net + ,sum(apr_net) as apr_net + ,sum(may_net) as may_net + ,sum(jun_net) as jun_net + ,sum(jul_net) as jul_net + ,sum(aug_net) as aug_net + ,sum(sep_net) as sep_net + ,sum(oct_net) as oct_net + ,sum(nov_net) as nov_net + ,sum(dec_net) as dec_net + from ( + select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,concat(concat('ORIENTAL ', ','), ' BOXBUNDLES') as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then ws_ext_sales_price* ws_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then ws_ext_sales_price* ws_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then ws_ext_sales_price* ws_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then ws_ext_sales_price* ws_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then ws_ext_sales_price* ws_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then ws_ext_sales_price* ws_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then ws_ext_sales_price* ws_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then ws_ext_sales_price* ws_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then ws_ext_sales_price* ws_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then ws_ext_sales_price* ws_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then ws_ext_sales_price* ws_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then ws_ext_sales_price* ws_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then ws_net_paid_inc_ship * ws_quantity else 0 end) as dec_net + from + web_sales + ,warehouse + ,date_dim + ,time_dim + ,ship_mode + where + ws_warehouse_sk = w_warehouse_sk + and ws_sold_date_sk = d_date_sk + and ws_sold_time_sk = t_time_sk + and ws_ship_mode_sk = sm_ship_mode_sk + and d_year = 2001 + and t_time between 42970 and 42970+28800 + and sm_carrier in ('ORIENTAL','BOXBUNDLES') + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,d_year + union all + select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,concat(concat('ORIENTAL ', ','), ' BOXBUNDLES') as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then cs_ext_list_price* cs_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then cs_ext_list_price* cs_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then cs_ext_list_price* cs_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then cs_ext_list_price* cs_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then cs_ext_list_price* cs_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then cs_ext_list_price* cs_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then cs_ext_list_price* cs_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then cs_ext_list_price* cs_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then cs_ext_list_price* cs_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then cs_ext_list_price* cs_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then cs_ext_list_price* cs_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then cs_ext_list_price* cs_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then cs_net_paid * cs_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then cs_net_paid * cs_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then cs_net_paid * cs_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then cs_net_paid * cs_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then cs_net_paid * cs_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then cs_net_paid * cs_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then cs_net_paid * cs_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then cs_net_paid * cs_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then cs_net_paid * cs_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then cs_net_paid * cs_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then cs_net_paid * cs_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then cs_net_paid * cs_quantity else 0 end) as dec_net + from + catalog_sales + ,warehouse + ,date_dim + ,time_dim + ,ship_mode + where + cs_warehouse_sk = w_warehouse_sk + and cs_sold_date_sk = d_date_sk + and cs_sold_time_sk = t_time_sk + and cs_ship_mode_sk = sm_ship_mode_sk + and d_year = 2001 + and t_time between 42970 AND 42970+28800 + and sm_carrier in ('ORIENTAL','BOXBUNDLES') + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,d_year + ) x + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,ship_carriers + ,year + order by w_warehouse_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query67.sql b/tools/tpcds-tools/queries/sf1000/query67.sql new file mode 100644 index 000000000000000..af49b1e94f02d37 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query67.sql @@ -0,0 +1,41 @@ +select * +from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rank() over (partition by i_category order by sumsales desc) rk + from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sum(coalesce(ss_sales_price*ss_quantity,0)) sumsales + from store_sales + ,date_dim + ,store + ,item + where ss_sold_date_sk=d_date_sk + and ss_item_sk=i_item_sk + and ss_store_sk = s_store_sk + and d_month_seq between 1217 and 1217+11 + group by rollup(i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy,s_store_id))dw1) dw2 +where rk <= 100 +order by i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rk +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query68.sql b/tools/tpcds-tools/queries/sf1000/query68.sql new file mode 100644 index 000000000000000..1f742baef74d8da --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query68.sql @@ -0,0 +1,39 @@ +select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,extended_price + ,extended_tax + ,list_price + from (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_ext_sales_price) extended_price + ,sum(ss_ext_list_price) list_price + ,sum(ss_ext_tax) extended_tax + from store_sales + ,date_dim + ,store + ,household_demographics + ,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_dep_count = 3 or + household_demographics.hd_vehicle_count= 4) + and date_dim.d_year in (1998,1998+1,1998+2) + and store.s_city in ('Fairview','Midway') + group by ss_ticket_number + ,ss_customer_sk + ,ss_addr_sk,ca_city) dn + ,customer + ,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,ss_ticket_number + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query69.sql b/tools/tpcds-tools/queries/sf1000/query69.sql new file mode 100644 index 000000000000000..f5473324694bf6f --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query69.sql @@ -0,0 +1,44 @@ +select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_state in ('IL','TX','ME') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 1 and 1+2) and + (not exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 1 and 1+2) and + not exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 1 and 1+2)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query7.sql b/tools/tpcds-tools/queries/sf1000/query7.sql new file mode 100644 index 000000000000000..c4b33b79d18b8a8 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query7.sql @@ -0,0 +1,18 @@ +select i_item_id, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, item, promotion + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_cdemo_sk = cd_demo_sk and + ss_promo_sk = p_promo_sk and + cd_gender = 'F' and + cd_marital_status = 'W' and + cd_education_status = 'College' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 2001 + group by i_item_id + order by i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query70.sql b/tools/tpcds-tools/queries/sf1000/query70.sql new file mode 100644 index 000000000000000..21b90f20ad5355d --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query70.sql @@ -0,0 +1,35 @@ +select + sum(ss_net_profit) as total_sum + ,s_state + ,s_county + ,grouping(s_state)+grouping(s_county) as lochierarchy + ,rank() over ( + partition by grouping(s_state)+grouping(s_county), + case when grouping(s_county) = 0 then s_state end + order by sum(ss_net_profit) desc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,store + where + d1.d_month_seq between 1220 and 1220+11 + and d1.d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + and s_state in + ( select s_state + from (select s_state as s_state, + rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking + from store_sales, store, date_dim + where d_month_seq between 1220 and 1220+11 + and d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + group by s_state + ) tmp1 + where ranking <= 5 + ) + group by rollup(s_state,s_county) + order by + lochierarchy desc + ,case when lochierarchy = 0 then s_state end + ,rank_within_parent + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query71.sql b/tools/tpcds-tools/queries/sf1000/query71.sql new file mode 100644 index 000000000000000..234ec201a194dab --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query71.sql @@ -0,0 +1,37 @@ +select i_brand_id brand_id, i_brand brand,t_hour,t_minute, + sum(ext_price) ext_price + from item, (select ws_ext_sales_price as ext_price, + ws_sold_date_sk as sold_date_sk, + ws_item_sk as sold_item_sk, + ws_sold_time_sk as time_sk + from web_sales,date_dim + where d_date_sk = ws_sold_date_sk + and d_moy=12 + and d_year=2002 + union all + select cs_ext_sales_price as ext_price, + cs_sold_date_sk as sold_date_sk, + cs_item_sk as sold_item_sk, + cs_sold_time_sk as time_sk + from catalog_sales,date_dim + where d_date_sk = cs_sold_date_sk + and d_moy=12 + and d_year=2002 + union all + select ss_ext_sales_price as ext_price, + ss_sold_date_sk as sold_date_sk, + ss_item_sk as sold_item_sk, + ss_sold_time_sk as time_sk + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + and d_moy=12 + and d_year=2002 + ) tmp,time_dim + where + sold_item_sk = i_item_sk + and i_manager_id=1 + and time_sk = t_time_sk + and (t_meal_time = 'breakfast' or t_meal_time = 'dinner') + group by i_brand, i_brand_id,t_hour,t_minute + order by ext_price desc, i_brand_id + ; diff --git a/tools/tpcds-tools/queries/sf1000/query72.sql b/tools/tpcds-tools/queries/sf1000/query72.sql new file mode 100644 index 000000000000000..6252e897e3ee1db --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query72.sql @@ -0,0 +1,26 @@ +select i_item_desc + ,w_warehouse_name + ,d1.d_week_seq + ,sum(case when p_promo_sk is null then 1 else 0 end) no_promo + ,sum(case when p_promo_sk is not null then 1 else 0 end) promo + ,count(*) total_cnt +from catalog_sales +join inventory on (cs_item_sk = inv_item_sk) +join warehouse on (w_warehouse_sk=inv_warehouse_sk) +join item on (i_item_sk = cs_item_sk) +join customer_demographics on (cs_bill_cdemo_sk = cd_demo_sk) +join household_demographics on (cs_bill_hdemo_sk = hd_demo_sk) +join date_dim d1 on (cs_sold_date_sk = d1.d_date_sk) +join date_dim d2 on (inv_date_sk = d2.d_date_sk) +join date_dim d3 on (cs_ship_date_sk = d3.d_date_sk) +left outer join promotion on (cs_promo_sk=p_promo_sk) +left outer join catalog_returns on (cr_item_sk = cs_item_sk and cr_order_number = cs_order_number) +where d1.d_week_seq = d2.d_week_seq + and inv_quantity_on_hand < cs_quantity + and (d3.d_date > (d1.d_date + INTERVAL '5' DAY)) + and hd_buy_potential = '1001-5000' + and d1.d_year = 1998 + and cd_marital_status = 'S' +group by i_item_desc,w_warehouse_name,d1.d_week_seq +order by total_cnt desc, i_item_desc, w_warehouse_name, d_week_seq +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query73.sql b/tools/tpcds-tools/queries/sf1000/query73.sql new file mode 100644 index 000000000000000..85282bd0867c3ec --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query73.sql @@ -0,0 +1,25 @@ +select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_buy_potential = '1001-5000' or + household_demographics.hd_buy_potential = '5001-10000') + and household_demographics.hd_vehicle_count > 0 + and case when household_demographics.hd_vehicle_count > 0 then + household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1 + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County') + group by ss_ticket_number,ss_customer_sk) dj,customer + where ss_customer_sk = c_customer_sk + and cnt between 1 and 5 + order by cnt desc, c_last_name asc; diff --git a/tools/tpcds-tools/queries/sf1000/query74.sql b/tools/tpcds-tools/queries/sf1000/query74.sql new file mode 100644 index 000000000000000..741fa29be48919c --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query74.sql @@ -0,0 +1,58 @@ +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,max(ss_net_paid) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (1999,1999+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,max(ws_net_paid) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + and d_year in (1999,1999+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + ) + select + t_s_secyear.customer_id, t_s_secyear.customer_first_name, t_s_secyear.customer_last_name + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.year = 1999 + and t_s_secyear.year = 1999+1 + and t_w_firstyear.year = 1999 + and t_w_secyear.year = 1999+1 + and t_s_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end + order by 1,3,2 +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query75.sql b/tools/tpcds-tools/queries/sf1000/query75.sql new file mode 100644 index 000000000000000..cce4cf2c68394fa --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query75.sql @@ -0,0 +1,67 @@ +WITH all_sales AS ( + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,SUM(sales_cnt) AS sales_cnt + ,SUM(sales_amt) AS sales_amt + FROM (SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt + ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt + FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk + JOIN date_dim ON d_date_sk=cs_sold_date_sk + LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number + AND cs_item_sk=cr_item_sk) + WHERE i_category='Sports' + UNION + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt + ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt + FROM store_sales JOIN item ON i_item_sk=ss_item_sk + JOIN date_dim ON d_date_sk=ss_sold_date_sk + LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number + AND ss_item_sk=sr_item_sk) + WHERE i_category='Sports' + UNION + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt + ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt + FROM web_sales JOIN item ON i_item_sk=ws_item_sk + JOIN date_dim ON d_date_sk=ws_sold_date_sk + LEFT JOIN web_returns ON (ws_order_number=wr_order_number + AND ws_item_sk=wr_item_sk) + WHERE i_category='Sports') sales_detail + GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) + SELECT prev_yr.d_year AS prev_year + ,curr_yr.d_year AS year + ,curr_yr.i_brand_id + ,curr_yr.i_class_id + ,curr_yr.i_category_id + ,curr_yr.i_manufact_id + ,prev_yr.sales_cnt AS prev_yr_cnt + ,curr_yr.sales_cnt AS curr_yr_cnt + ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff + ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff + FROM all_sales curr_yr, all_sales prev_yr + WHERE curr_yr.i_brand_id=prev_yr.i_brand_id + AND curr_yr.i_class_id=prev_yr.i_class_id + AND curr_yr.i_category_id=prev_yr.i_category_id + AND curr_yr.i_manufact_id=prev_yr.i_manufact_id + AND curr_yr.d_year=2002 + AND prev_yr.d_year=2002-1 + AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9 + ORDER BY sales_cnt_diff,sales_amt_diff + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query76.sql b/tools/tpcds-tools/queries/sf1000/query76.sql new file mode 100644 index 000000000000000..837387fec6d8692 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query76.sql @@ -0,0 +1,21 @@ +select channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM ( + SELECT 'store' as channel, 'ss_customer_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price + FROM store_sales, item, date_dim + WHERE ss_customer_sk IS NULL + AND ss_sold_date_sk=d_date_sk + AND ss_item_sk=i_item_sk + UNION ALL + SELECT 'web' as channel, 'ws_promo_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price + FROM web_sales, item, date_dim + WHERE ws_promo_sk IS NULL + AND ws_sold_date_sk=d_date_sk + AND ws_item_sk=i_item_sk + UNION ALL + SELECT 'catalog' as channel, 'cs_bill_customer_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price + FROM catalog_sales, item, date_dim + WHERE cs_bill_customer_sk IS NULL + AND cs_sold_date_sk=d_date_sk + AND cs_item_sk=i_item_sk) foo +GROUP BY channel, col_name, d_year, d_qoy, i_category +ORDER BY channel, col_name, d_year, d_qoy, i_category +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query77.sql b/tools/tpcds-tools/queries/sf1000/query77.sql new file mode 100644 index 000000000000000..cd4805724ff2f8e --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query77.sql @@ -0,0 +1,105 @@ +with ss as + (select s_store_sk, + sum(ss_ext_sales_price) as sales, + sum(ss_net_profit) as profit + from store_sales, + date_dim, + store + where ss_sold_date_sk = d_date_sk + and d_date between cast('2000-08-10' as date) + and (cast('2000-08-10' as date) + interval 30 day) + and ss_store_sk = s_store_sk + group by s_store_sk) + , + sr as + (select s_store_sk, + sum(sr_return_amt) as returns, + sum(sr_net_loss) as profit_loss + from store_returns, + date_dim, + store + where sr_returned_date_sk = d_date_sk + and d_date between cast('2000-08-10' as date) + and (cast('2000-08-10' as date) + interval 30 day) + and sr_store_sk = s_store_sk + group by s_store_sk), + cs as + (select cs_call_center_sk, + sum(cs_ext_sales_price) as sales, + sum(cs_net_profit) as profit + from catalog_sales, + date_dim + where cs_sold_date_sk = d_date_sk + and d_date between cast('2000-08-10' as date) + and (cast('2000-08-10' as date) + interval 30 day) + group by cs_call_center_sk + ), + cr as + (select cr_call_center_sk, + sum(cr_return_amount) as returns, + sum(cr_net_loss) as profit_loss + from catalog_returns, + date_dim + where cr_returned_date_sk = d_date_sk + and d_date between cast('2000-08-10' as date) + and (cast('2000-08-10' as date) + interval 30 day) + group by cr_call_center_sk + ), + ws as + ( select wp_web_page_sk, + sum(ws_ext_sales_price) as sales, + sum(ws_net_profit) as profit + from web_sales, + date_dim, + web_page + where ws_sold_date_sk = d_date_sk + and d_date between cast('2000-08-10' as date) + and (cast('2000-08-10' as date) + interval 30 day) + and ws_web_page_sk = wp_web_page_sk + group by wp_web_page_sk), + wr as + (select wp_web_page_sk, + sum(wr_return_amt) as returns, + sum(wr_net_loss) as profit_loss + from web_returns, + date_dim, + web_page + where wr_returned_date_sk = d_date_sk + and d_date between cast('2000-08-10' as date) + and (cast('2000-08-10' as date) + interval 30 day) + and wr_web_page_sk = wp_web_page_sk + group by wp_web_page_sk) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , ss.s_store_sk as id + , sales + , coalesce(returns, 0) as returns + , (profit - coalesce(profit_loss,0)) as profit + from ss left join sr + on ss.s_store_sk = sr.s_store_sk + union all + select 'catalog channel' as channel + , cs_call_center_sk as id + , sales + , returns + , (profit - profit_loss) as profit + from cs + , cr + union all + select 'web channel' as channel + , ws.wp_web_page_sk as id + , sales + , coalesce(returns, 0) returns + , (profit - coalesce(profit_loss,0)) as profit + from ws left join wr + on ws.wp_web_page_sk = wr.wp_web_page_sk + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query78.sql b/tools/tpcds-tools/queries/sf1000/query78.sql new file mode 100644 index 000000000000000..62dca42e44ebaa3 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query78.sql @@ -0,0 +1,55 @@ +with ws as + (select d_year AS ws_sold_year, ws_item_sk, + ws_bill_customer_sk ws_customer_sk, + sum(ws_quantity) ws_qty, + sum(ws_wholesale_cost) ws_wc, + sum(ws_sales_price) ws_sp + from web_sales + left join web_returns on wr_order_number=ws_order_number and ws_item_sk=wr_item_sk + join date_dim on ws_sold_date_sk = d_date_sk + where wr_order_number is null and d_year=1998 + group by d_year, ws_item_sk, ws_bill_customer_sk + ), +cs as + (select d_year AS cs_sold_year, cs_item_sk, + cs_bill_customer_sk cs_customer_sk, + sum(cs_quantity) cs_qty, + sum(cs_wholesale_cost) cs_wc, + sum(cs_sales_price) cs_sp + from catalog_sales + left join catalog_returns on cr_order_number=cs_order_number and cs_item_sk=cr_item_sk + join date_dim on cs_sold_date_sk = d_date_sk + where cr_order_number is null and d_year=1998 + group by d_year, cs_item_sk, cs_bill_customer_sk + ), +ss as + (select d_year AS ss_sold_year, ss_item_sk, + ss_customer_sk, + sum(ss_quantity) ss_qty, + sum(ss_wholesale_cost) ss_wc, + sum(ss_sales_price) ss_sp + from store_sales + left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk + join date_dim on ss_sold_date_sk = d_date_sk + where sr_ticket_number is null and d_year=1998 + group by d_year, ss_item_sk, ss_customer_sk + ) +select +ss_customer_sk, +round(ss_qty/(coalesce(ws_qty,0)+coalesce(cs_qty,0)),2) ratio, +ss_qty store_qty, ss_wc store_wholesale_cost, ss_sp store_sales_price, +coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty, +coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost, +coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price +from ss +left join ws on (ws_sold_year=ss_sold_year and ws_item_sk=ss_item_sk and ws_customer_sk=ss_customer_sk) +left join cs on (cs_sold_year=ss_sold_year and cs_item_sk=ss_item_sk and cs_customer_sk=ss_customer_sk) +where (coalesce(ws_qty,0)>0 or coalesce(cs_qty, 0)>0) and ss_sold_year=1998 +order by + ss_customer_sk, + ss_qty desc, ss_wc desc, ss_sp desc, + other_chan_qty, + other_chan_wholesale_cost, + other_chan_sales_price, + ratio +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query79.sql b/tools/tpcds-tools/queries/sf1000/query79.sql new file mode 100644 index 000000000000000..81edd1f4f20a2de --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query79.sql @@ -0,0 +1,20 @@ +select + c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,store.s_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (household_demographics.hd_dep_count = 7 or household_demographics.hd_vehicle_count > -1) + and date_dim.d_dow = 1 + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_number_employees between 200 and 295 + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer + where ss_customer_sk = c_customer_sk + order by c_last_name,c_first_name,substr(s_city,1,30), profit +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query8.sql b/tools/tpcds-tools/queries/sf1000/query8.sql new file mode 100644 index 000000000000000..8909d5ba1d8b17b --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query8.sql @@ -0,0 +1,105 @@ +select s_store_name + ,sum(ss_net_profit) + from store_sales + ,date_dim + ,store, + (select ca_zip + from ( + SELECT substr(ca_zip,1,5) ca_zip + FROM customer_address + WHERE substr(ca_zip,1,5) IN ( + '47602','16704','35863','28577','83910','36201', + '58412','48162','28055','41419','80332', + '38607','77817','24891','16226','18410', + '21231','59345','13918','51089','20317', + '17167','54585','67881','78366','47770', + '18360','51717','73108','14440','21800', + '89338','45859','65501','34948','25973', + '73219','25333','17291','10374','18829', + '60736','82620','41351','52094','19326', + '25214','54207','40936','21814','79077', + '25178','75742','77454','30621','89193', + '27369','41232','48567','83041','71948', + '37119','68341','14073','16891','62878', + '49130','19833','24286','27700','40979', + '50412','81504','94835','84844','71954', + '39503','57649','18434','24987','12350', + '86379','27413','44529','98569','16515', + '27287','24255','21094','16005','56436', + '91110','68293','56455','54558','10298', + '83647','32754','27052','51766','19444', + '13869','45645','94791','57631','20712', + '37788','41807','46507','21727','71836', + '81070','50632','88086','63991','20244', + '31655','51782','29818','63792','68605', + '94898','36430','57025','20601','82080', + '33869','22728','35834','29086','92645', + '98584','98072','11652','78093','57553', + '43830','71144','53565','18700','90209', + '71256','38353','54364','28571','96560', + '57839','56355','50679','45266','84680', + '34306','34972','48530','30106','15371', + '92380','84247','92292','68852','13338', + '34594','82602','70073','98069','85066', + '47289','11686','98862','26217','47529', + '63294','51793','35926','24227','14196', + '24594','32489','99060','49472','43432', + '49211','14312','88137','47369','56877', + '20534','81755','15794','12318','21060', + '73134','41255','63073','81003','73873', + '66057','51184','51195','45676','92696', + '70450','90669','98338','25264','38919', + '59226','58581','60298','17895','19489', + '52301','80846','95464','68770','51634', + '19988','18367','18421','11618','67975', + '25494','41352','95430','15734','62585', + '97173','33773','10425','75675','53535', + '17879','41967','12197','67998','79658', + '59130','72592','14851','43933','68101', + '50636','25717','71286','24660','58058', + '72991','95042','15543','33122','69280', + '11912','59386','27642','65177','17672', + '33467','64592','36335','54010','18767', + '63193','42361','49254','33113','33159', + '36479','59080','11855','81963','31016', + '49140','29392','41836','32958','53163', + '13844','73146','23952','65148','93498', + '14530','46131','58454','13376','13378', + '83986','12320','17193','59852','46081', + '98533','52389','13086','68843','31013', + '13261','60560','13443','45533','83583', + '11489','58218','19753','22911','25115', + '86709','27156','32669','13123','51933', + '39214','41331','66943','14155','69998', + '49101','70070','35076','14242','73021', + '59494','15782','29752','37914','74686', + '83086','34473','15751','81084','49230', + '91894','60624','17819','28810','63180', + '56224','39459','55233','75752','43639', + '55349','86057','62361','50788','31830', + '58062','18218','85761','60083','45484', + '21204','90229','70041','41162','35390', + '16364','39500','68908','26689','52868', + '81335','40146','11340','61527','61794', + '71997','30415','59004','29450','58117', + '69952','33562','83833','27385','61860', + '96435','48333','23065','32961','84919', + '61997','99132','22815','56600','68730', + '48017','95694','32919','88217','27116', + '28239','58032','18884','16791','21343', + '97462','18569','75660','15475') + intersect + select ca_zip + from (SELECT substr(ca_zip,1,5) ca_zip,count(*) cnt + FROM customer_address, customer + WHERE ca_address_sk = c_current_addr_sk and + c_preferred_cust_flag='Y' + group by ca_zip + having count(*) > 10)A1)A2) V1 + where ss_store_sk = s_store_sk + and ss_sold_date_sk = d_date_sk + and d_qoy = 2 and d_year = 1998 + and (substr(s_zip,1,2) = substr(V1.ca_zip,1,2)) + group by s_store_name + order by s_store_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query80.sql b/tools/tpcds-tools/queries/sf1000/query80.sql new file mode 100644 index 000000000000000..82f57c72361d7fb --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query80.sql @@ -0,0 +1,93 @@ +with ssr as + (select s_store_id as store_id, + sum(ss_ext_sales_price) as sales, + sum(coalesce(sr_return_amt, 0)) as returns, + sum(ss_net_profit - coalesce(sr_net_loss, 0)) as profit + from store_sales left outer join store_returns on + (ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number), + date_dim, + store, + item, + promotion + where ss_sold_date_sk = d_date_sk + and d_date between cast('2002-08-14' as date) + and (cast('2002-08-14' as date) + interval 30 day) + and ss_store_sk = s_store_sk + and ss_item_sk = i_item_sk + and i_current_price > 50 + and ss_promo_sk = p_promo_sk + and p_channel_tv = 'N' + group by s_store_id) + , + csr as + (select cp_catalog_page_id as catalog_page_id, + sum(cs_ext_sales_price) as sales, + sum(coalesce(cr_return_amount, 0)) as returns, + sum(cs_net_profit - coalesce(cr_net_loss, 0)) as profit + from catalog_sales left outer join catalog_returns on + (cs_item_sk = cr_item_sk and cs_order_number = cr_order_number), + date_dim, + catalog_page, + item, + promotion + where cs_sold_date_sk = d_date_sk + and d_date between cast('2002-08-14' as date) + and (cast('2002-08-14' as date) + interval 30 day) + and cs_catalog_page_sk = cp_catalog_page_sk + and cs_item_sk = i_item_sk + and i_current_price > 50 + and cs_promo_sk = p_promo_sk + and p_channel_tv = 'N' +group by cp_catalog_page_id) + , + wsr as + (select web_site_id, + sum(ws_ext_sales_price) as sales, + sum(coalesce(wr_return_amt, 0)) as returns, + sum(ws_net_profit - coalesce(wr_net_loss, 0)) as profit + from web_sales left outer join web_returns on + (ws_item_sk = wr_item_sk and ws_order_number = wr_order_number), + date_dim, + web_site, + item, + promotion + where ws_sold_date_sk = d_date_sk + and d_date between cast('2002-08-14' as date) + and (cast('2002-08-14' as date) + interval 30 day) + and ws_web_site_sk = web_site_sk + and ws_item_sk = i_item_sk + and i_current_price > 50 + and ws_promo_sk = p_promo_sk + and p_channel_tv = 'N' +group by web_site_id) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , concat('store', store_id) as id + , sales + , returns + , profit + from ssr + union all + select 'catalog channel' as channel + , concat('catalog_page', catalog_page_id) as id + , sales + , returns + , profit + from csr + union all + select 'web channel' as channel + , concat('web_site', web_site_id) as id + , sales + , returns + , profit + from wsr + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query81.sql b/tools/tpcds-tools/queries/sf1000/query81.sql new file mode 100644 index 000000000000000..127d49bdd3c90f9 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query81.sql @@ -0,0 +1,28 @@ +with customer_total_return as + (select cr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(cr_return_amt_inc_tax) as ctr_total_return + from catalog_returns + ,date_dim + ,customer_address + where cr_returned_date_sk = d_date_sk + and d_year =2001 + and cr_returning_addr_sk = ca_address_sk + group by cr_returning_customer_sk + ,ca_state ) + select c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'TN' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query82.sql b/tools/tpcds-tools/queries/sf1000/query82.sql new file mode 100644 index 000000000000000..c36ca8aecb84967 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query82.sql @@ -0,0 +1,14 @@ +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, store_sales + where i_current_price between 58 and 58+30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('2001-01-13' as date) and (cast('2001-01-13' as date) + interval 60 day) + and i_manufact_id in (259,559,580,485) + and inv_quantity_on_hand between 100 and 500 + and ss_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query83.sql b/tools/tpcds-tools/queries/sf1000/query83.sql new file mode 100644 index 000000000000000..321cdc470751842 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query83.sql @@ -0,0 +1,64 @@ +with sr_items as + (select i_item_id item_id, + sum(sr_return_quantity) sr_item_qty + from store_returns, + item, + date_dim + where sr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2001-07-13','2001-09-10','2001-11-16'))) + and sr_returned_date_sk = d_date_sk + group by i_item_id), + cr_items as + (select i_item_id item_id, + sum(cr_return_quantity) cr_item_qty + from catalog_returns, + item, + date_dim + where cr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2001-07-13','2001-09-10','2001-11-16'))) + and cr_returned_date_sk = d_date_sk + group by i_item_id), + wr_items as + (select i_item_id item_id, + sum(wr_return_quantity) wr_item_qty + from web_returns, + item, + date_dim + where wr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2001-07-13','2001-09-10','2001-11-16'))) + and wr_returned_date_sk = d_date_sk + group by i_item_id) + select sr_items.item_id + ,sr_item_qty + ,sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev + ,cr_item_qty + ,cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev + ,wr_item_qty + ,wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev + ,(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 average + from sr_items + ,cr_items + ,wr_items + where sr_items.item_id=cr_items.item_id + and sr_items.item_id=wr_items.item_id + order by sr_items.item_id + ,sr_item_qty + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query84.sql b/tools/tpcds-tools/queries/sf1000/query84.sql new file mode 100644 index 000000000000000..7d9fe1930a94541 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query84.sql @@ -0,0 +1,18 @@ +select c_customer_id as customer_id + , concat(concat(coalesce(c_last_name,''), ','), coalesce(c_first_name,'')) as customername + from customer + ,customer_address + ,customer_demographics + ,household_demographics + ,income_band + ,store_returns + where ca_city = 'Woodland' + and c_current_addr_sk = ca_address_sk + and ib_lower_bound >= 60306 + and ib_upper_bound <= 60306 + 50000 + and ib_income_band_sk = hd_income_band_sk + and cd_demo_sk = c_current_cdemo_sk + and hd_demo_sk = c_current_hdemo_sk + and sr_cdemo_sk = cd_demo_sk + order by c_customer_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query85.sql b/tools/tpcds-tools/queries/sf1000/query85.sql new file mode 100644 index 000000000000000..7ff852ac02cd5bc --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query85.sql @@ -0,0 +1,81 @@ +select substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) + from web_sales, web_returns, web_page, customer_demographics cd1, + customer_demographics cd2, customer_address, date_dim, reason + where ws_web_page_sk = wp_web_page_sk + and ws_item_sk = wr_item_sk + and ws_order_number = wr_order_number + and ws_sold_date_sk = d_date_sk and d_year = 1998 + and cd1.cd_demo_sk = wr_refunded_cdemo_sk + and cd2.cd_demo_sk = wr_returning_cdemo_sk + and ca_address_sk = wr_refunded_addr_sk + and r_reason_sk = wr_reason_sk + and + ( + ( + cd1.cd_marital_status = 'D' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Primary' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 100.00 and 150.00 + ) + or + ( + cd1.cd_marital_status = 'S' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'College' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 50.00 and 100.00 + ) + or + ( + cd1.cd_marital_status = 'U' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Advanced Degree' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ca_country = 'United States' + and + ca_state in ('NC', 'TX', 'IA') + and ws_net_profit between 100 and 200 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('WI', 'WV', 'GA') + and ws_net_profit between 150 and 300 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('OK', 'VA', 'KY') + and ws_net_profit between 50 and 250 + ) + ) +group by r_reason_desc +order by substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query86.sql b/tools/tpcds-tools/queries/sf1000/query86.sql new file mode 100644 index 000000000000000..173fff760f66dee --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query86.sql @@ -0,0 +1,23 @@ +select + sum(ws_net_paid) as total_sum + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ws_net_paid) desc) as rank_within_parent + from + web_sales + ,date_dim d1 + ,item + where + d1.d_month_seq between 1186 and 1186+11 + and d1.d_date_sk = ws_sold_date_sk + and i_item_sk = ws_item_sk + group by rollup(i_category,i_class) + order by + lochierarchy desc, + case when lochierarchy = 0 then i_category end, + rank_within_parent + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query87.sql b/tools/tpcds-tools/queries/sf1000/query87.sql new file mode 100644 index 000000000000000..8d27131f21f4952 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query87.sql @@ -0,0 +1,20 @@ +select count(*) +from ((select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1202 and 1202+11) + except + (select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1202 and 1202+11) + except + (select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1202 and 1202+11) +) cool_cust +; diff --git a/tools/tpcds-tools/queries/sf1000/query88.sql b/tools/tpcds-tools/queries/sf1000/query88.sql new file mode 100644 index 000000000000000..0d8eea8d116dcf8 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query88.sql @@ -0,0 +1,91 @@ +select * +from + (select count(*) h8_30_to_9 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 8 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s1, + (select count(*) h9_to_9_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s2, + (select count(*) h9_30_to_10 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s3, + (select count(*) h10_to_10_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s4, + (select count(*) h10_30_to_11 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s5, + (select count(*) h11_to_11_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s6, + (select count(*) h11_30_to_12 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s7, + (select count(*) h12_to_12_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 12 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = -1 and household_demographics.hd_vehicle_count<=-1+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s8 +; diff --git a/tools/tpcds-tools/queries/sf1000/query89.sql b/tools/tpcds-tools/queries/sf1000/query89.sql new file mode 100644 index 000000000000000..d88f0530687b1fb --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query89.sql @@ -0,0 +1,25 @@ +select * +from( +select i_category, i_class, i_brand, + s_store_name, s_company_name, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, s_store_name, s_company_name) + avg_monthly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + d_year in (2001) and + ((i_category in ('Books','Children','Electronics') and + i_class in ('history','school-uniforms','audio') + ) + or (i_category in ('Men','Sports','Shoes') and + i_class in ('pants','tennis','womens') + )) +group by i_category, i_class, i_brand, + s_store_name, s_company_name, d_moy) tmp1 +where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1 +order by sum_sales - avg_monthly_sales, s_store_name +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query9.sql b/tools/tpcds-tools/queries/sf1000/query9.sql new file mode 100644 index 000000000000000..df99d02dd2e1d2b --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query9.sql @@ -0,0 +1,48 @@ +select case when (select count(*) + from store_sales + where ss_quantity between 1 and 20) > 1071 + then (select avg(ss_ext_tax) + from store_sales + where ss_quantity between 1 and 20) + else (select avg(ss_net_paid_inc_tax) + from store_sales + where ss_quantity between 1 and 20) end bucket1 , + case when (select count(*) + from store_sales + where ss_quantity between 21 and 40) > 39161 + then (select avg(ss_ext_tax) + from store_sales + where ss_quantity between 21 and 40) + else (select avg(ss_net_paid_inc_tax) + from store_sales + where ss_quantity between 21 and 40) end bucket2, + case when (select count(*) + from store_sales + where ss_quantity between 41 and 60) > 29434 + then (select avg(ss_ext_tax) + from store_sales + where ss_quantity between 41 and 60) + else (select avg(ss_net_paid_inc_tax) + from store_sales + where ss_quantity between 41 and 60) end bucket3, + case when (select count(*) + from store_sales + where ss_quantity between 61 and 80) > 6568 + then (select avg(ss_ext_tax) + from store_sales + where ss_quantity between 61 and 80) + else (select avg(ss_net_paid_inc_tax) + from store_sales + where ss_quantity between 61 and 80) end bucket4, + case when (select count(*) + from store_sales + where ss_quantity between 81 and 100) > 21216 + then (select avg(ss_ext_tax) + from store_sales + where ss_quantity between 81 and 100) + else (select avg(ss_net_paid_inc_tax) + from store_sales + where ss_quantity between 81 and 100) end bucket5 +from reason +where r_reason_sk = 1 +; diff --git a/tools/tpcds-tools/queries/sf1000/query90.sql b/tools/tpcds-tools/queries/sf1000/query90.sql new file mode 100644 index 000000000000000..45dce0ab06a282a --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query90.sql @@ -0,0 +1,19 @@ +select cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio + from ( select count(*) amc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 12 and 12+1 + and household_demographics.hd_dep_count = 6 + and web_page.wp_char_count between 5000 and 5200) at, + ( select count(*) pmc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 14 and 14+1 + and household_demographics.hd_dep_count = 6 + and web_page.wp_char_count between 5000 and 5200) pt + order by am_pm_ratio + limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query91.sql b/tools/tpcds-tools/queries/sf1000/query91.sql new file mode 100644 index 000000000000000..b8354f62c69e346 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query91.sql @@ -0,0 +1,28 @@ +select + cc_call_center_id Call_Center, + cc_name Call_Center_Name, + cc_manager Manager, + sum(cr_net_loss) Returns_Loss +from + call_center, + catalog_returns, + date_dim, + customer, + customer_address, + customer_demographics, + household_demographics +where + cr_call_center_sk = cc_call_center_sk +and cr_returned_date_sk = d_date_sk +and cr_returning_customer_sk= c_customer_sk +and cd_demo_sk = c_current_cdemo_sk +and hd_demo_sk = c_current_hdemo_sk +and ca_address_sk = c_current_addr_sk +and d_year = 2000 +and d_moy = 12 +and ( (cd_marital_status = 'M' and cd_education_status = 'Unknown') + or(cd_marital_status = 'W' and cd_education_status = 'Advanced Degree')) +and hd_buy_potential like 'Unknown%' +and ca_gmt_offset = -7 +group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status +order by sum(cr_net_loss) desc; diff --git a/tools/tpcds-tools/queries/sf1000/query92.sql b/tools/tpcds-tools/queries/sf1000/query92.sql new file mode 100644 index 000000000000000..fc79c8bdeba02ef --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query92.sql @@ -0,0 +1,27 @@ +select + sum(ws_ext_discount_amt) as "Excess Discount Amount" +from + web_sales + ,item + ,date_dim +where +i_manufact_id = 714 +and i_item_sk = ws_item_sk +and d_date between '2000-02-01' and + (cast('2000-02-01' as date) + interval 90 day) +and d_date_sk = ws_sold_date_sk +and ws_ext_discount_amt + > ( + SELECT + 1.3 * avg(ws_ext_discount_amt) + FROM + web_sales + ,date_dim + WHERE + ws_item_sk = i_item_sk + and d_date between '2000-02-01' and + (cast('2000-02-01' as date) + interval 90 day) + and d_date_sk = ws_sold_date_sk + ) +order by sum(ws_ext_discount_amt) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query93.sql b/tools/tpcds-tools/queries/sf1000/query93.sql new file mode 100644 index 000000000000000..43cf18d6a16e87f --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query93.sql @@ -0,0 +1,15 @@ +select ss_customer_sk + ,sum(act_sales) sumsales + from (select ss_item_sk + ,ss_ticket_number + ,ss_customer_sk + ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price + else (ss_quantity*ss_sales_price) end act_sales + from store_sales left outer join store_returns on (sr_item_sk = ss_item_sk + and sr_ticket_number = ss_ticket_number) + ,reason + where sr_reason_sk = r_reason_sk + and r_reason_desc = 'reason 58') t + group by ss_customer_sk + order by sumsales, ss_customer_sk +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query94.sql b/tools/tpcds-tools/queries/sf1000/query94.sql new file mode 100644 index 000000000000000..5d37954d82a672d --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query94.sql @@ -0,0 +1,26 @@ +select + count(distinct ws_order_number) as "order count" + ,sum(ws_ext_ship_cost) as "total shipping cost" + ,sum(ws_net_profit) as "total net profit" +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '2002-5-01' and + (cast('2002-5-01' as date) + interval 60 day) +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'OK' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and exists (select * + from web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) +and not exists(select * + from web_returns wr1 + where ws1.ws_order_number = wr1.wr_order_number) +order by count(distinct ws_order_number) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query95.sql b/tools/tpcds-tools/queries/sf1000/query95.sql new file mode 100644 index 000000000000000..8927ea96b88b270 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query95.sql @@ -0,0 +1,29 @@ +with ws_wh as +(select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2 + from web_sales ws1,web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) + select + count(distinct ws_order_number) as "order count" + ,sum(ws_ext_ship_cost) as "total shipping cost" + ,sum(ws_net_profit) as "total net profit" +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '2001-4-01' and + (cast('2001-4-01' as date) + interval 60 day) +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'VA' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and ws1.ws_order_number in (select ws_order_number + from ws_wh) +and ws1.ws_order_number in (select wr_order_number + from web_returns,ws_wh + where wr_order_number = ws_wh.ws_order_number) +order by count(distinct ws_order_number) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query96.sql b/tools/tpcds-tools/queries/sf1000/query96.sql new file mode 100644 index 000000000000000..84e2eaad2a62355 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query96.sql @@ -0,0 +1,13 @@ +select count(*) +from store_sales + ,household_demographics + ,time_dim, store +where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 8 + and time_dim.t_minute >= 30 + and household_demographics.hd_dep_count = 0 + and store.s_store_name = 'ese' +order by count(*) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query97.sql b/tools/tpcds-tools/queries/sf1000/query97.sql new file mode 100644 index 000000000000000..6b63455bc596906 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query97.sql @@ -0,0 +1,22 @@ +with ssci as ( +select ss_customer_sk customer_sk + ,ss_item_sk item_sk +from store_sales,date_dim +where ss_sold_date_sk = d_date_sk + and d_month_seq between 1199 and 1199 + 11 and ss_sold_date_sk IS NOT NULL +group by ss_customer_sk + ,ss_item_sk), +csci as( + select cs_bill_customer_sk customer_sk + ,cs_item_sk item_sk +from catalog_sales,date_dim +where cs_sold_date_sk = d_date_sk + and d_month_seq between 1199 and 1199 + 11 and cs_sold_date_sk IS NOT NULL +group by cs_bill_customer_sk + ,cs_item_sk) + select sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only + ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only + ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog +from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk + and ssci.item_sk = csci.item_sk) +limit 100; diff --git a/tools/tpcds-tools/queries/sf1000/query98.sql b/tools/tpcds-tools/queries/sf1000/query98.sql new file mode 100644 index 000000000000000..0d16696158915f7 --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query98.sql @@ -0,0 +1,30 @@ +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ss_ext_sales_price) as itemrevenue + ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over + (partition by i_class) as revenueratio +from + store_sales + ,item + ,date_dim +where + ss_item_sk = i_item_sk + and i_category in ('Men', 'Sports', 'Jewelry') + and ss_sold_date_sk = d_date_sk + and d_date between cast('1999-02-05' as date) + and (cast('1999-02-05' as date) + interval 30 day) +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio; diff --git a/tools/tpcds-tools/queries/sf1000/query99.sql b/tools/tpcds-tools/queries/sf1000/query99.sql new file mode 100644 index 000000000000000..f144236d6b83dfc --- /dev/null +++ b/tools/tpcds-tools/queries/sf1000/query99.sql @@ -0,0 +1,32 @@ +select + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and + (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and + (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and + (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 120) then 1 else 0 end) as ">120 days" +from + catalog_sales + ,warehouse + ,ship_mode + ,call_center + ,date_dim +where + d_month_seq between 1194 and 1194 + 11 +and cs_ship_date_sk = d_date_sk +and cs_warehouse_sk = w_warehouse_sk +and cs_ship_mode_sk = sm_ship_mode_sk +and cs_call_center_sk = cc_call_center_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query1.sql b/tools/tpcds-tools/queries/sf10000/query1.sql new file mode 100644 index 000000000000000..3d525b001c42e4b --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query1.sql @@ -0,0 +1,22 @@ +with customer_total_return as +(select sr_customer_sk as ctr_customer_sk +,sr_store_sk as ctr_store_sk +,sum(SR_FEE) as ctr_total_return +from store_returns +,date_dim +where sr_returned_date_sk = d_date_sk +and d_year =2000 +group by sr_customer_sk +,sr_store_sk) + select c_customer_id +from customer_total_return ctr1 +,store +,customer +where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 +from customer_total_return ctr2 +where ctr1.ctr_store_sk = ctr2.ctr_store_sk) +and s_store_sk = ctr1.ctr_store_sk +and s_state = 'NM' +and ctr1.ctr_customer_sk = c_customer_sk +order by c_customer_id +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query10.sql b/tools/tpcds-tools/queries/sf10000/query10.sql new file mode 100644 index 000000000000000..28a37c87db7703c --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query10.sql @@ -0,0 +1,56 @@ +select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3, + cd_dep_count, + count(*) cnt4, + cd_dep_employed_count, + count(*) cnt5, + cd_dep_college_count, + count(*) cnt6 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_county in ('Fillmore County','McPherson County','Bonneville County','Boone County','Brown County') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2000 and + d_moy between 3 and 3+3) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2000 and + d_moy between 3 ANd 3+3) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2000 and + d_moy between 3 and 3+3)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query11.sql b/tools/tpcds-tools/queries/sf10000/query11.sql new file mode 100644 index 000000000000000..cc8da6f3a88ce11 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query11.sql @@ -0,0 +1,78 @@ +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(ss_ext_list_price-ss_ext_discount_amt) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(ws_ext_list_price-ws_ext_discount_amt) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + ) + select + t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_birth_country + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.dyear = 1999 + and t_s_secyear.dyear = 1999+1 + and t_w_firstyear.dyear = 1999 + and t_w_secyear.dyear = 1999+1 + and t_s_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else 0.0 end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else 0.0 end + order by t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_birth_country +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query12.sql b/tools/tpcds-tools/queries/sf10000/query12.sql new file mode 100644 index 000000000000000..e7812e4630d34ee --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query12.sql @@ -0,0 +1,31 @@ +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ws_ext_sales_price) as itemrevenue + ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over + (partition by i_class) as revenueratio +from + web_sales + ,item + ,date_dim +where + ws_item_sk = i_item_sk + and i_category in ('Electronics', 'Books', 'Women') + and ws_sold_date_sk = d_date_sk + and d_date between cast('1998-01-06' as date) + and (cast('1998-01-06' as date) + interval 30 day) +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query13.sql b/tools/tpcds-tools/queries/sf10000/query13.sql new file mode 100644 index 000000000000000..e45eb9183d66079 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query13.sql @@ -0,0 +1,49 @@ +select avg(ss_quantity) + ,avg(ss_ext_sales_price) + ,avg(ss_ext_wholesale_cost) + ,sum(ss_ext_wholesale_cost) + from store_sales + ,store + ,customer_demographics + ,household_demographics + ,customer_address + ,date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 2001 + and((ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'U' + and cd_education_status = 'Secondary' + and ss_sales_price between 100.00 and 150.00 + and hd_dep_count = 3 + )or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'W' + and cd_education_status = 'College' + and ss_sales_price between 50.00 and 100.00 + and hd_dep_count = 1 + ) or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'D' + and cd_education_status = 'Primary' + and ss_sales_price between 150.00 and 200.00 + and hd_dep_count = 1 + )) + and((ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('TX', 'OK', 'MI') + and ss_net_profit between 100 and 200 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('WA', 'NC', 'OH') + and ss_net_profit between 150 and 300 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('MT', 'FL', 'GA') + and ss_net_profit between 50 and 250 + )) +; diff --git a/tools/tpcds-tools/queries/sf10000/query14.sql b/tools/tpcds-tools/queries/sf10000/query14.sql new file mode 100644 index 000000000000000..4e6947c30ae1de0 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query14.sql @@ -0,0 +1,207 @@ +with cross_items as + (select i_item_sk ss_item_sk + from item, + (select iss.i_brand_id brand_id + ,iss.i_class_id class_id + ,iss.i_category_id category_id + from store_sales + ,item iss + ,date_dim d1 + where ss_item_sk = iss.i_item_sk + and ss_sold_date_sk = d1.d_date_sk + and d1.d_year between 2000 AND 2000 + 2 + intersect + select ics.i_brand_id + ,ics.i_class_id + ,ics.i_category_id + from catalog_sales + ,item ics + ,date_dim d2 + where cs_item_sk = ics.i_item_sk + and cs_sold_date_sk = d2.d_date_sk + and d2.d_year between 2000 AND 2000 + 2 + intersect + select iws.i_brand_id + ,iws.i_class_id + ,iws.i_category_id + from web_sales + ,item iws + ,date_dim d3 + where ws_item_sk = iws.i_item_sk + and ws_sold_date_sk = d3.d_date_sk + and d3.d_year between 2000 AND 2000 + 2) + t where i_brand_id = brand_id + and i_class_id = class_id + and i_category_id = category_id +), + avg_sales as + (select avg(quantity*list_price) average_sales + from (select ss_quantity quantity + ,ss_list_price list_price + from store_sales + ,date_dim + where ss_sold_date_sk = d_date_sk + and d_year between 2000 and 2000 + 2 + union all + select cs_quantity quantity + ,cs_list_price list_price + from catalog_sales + ,date_dim + where cs_sold_date_sk = d_date_sk + and d_year between 2000 and 2000 + 2 + union all + select ws_quantity quantity + ,ws_list_price list_price + from web_sales + ,date_dim + where ws_sold_date_sk = d_date_sk + and d_year between 2000 and 2000 + 2) x) + select channel, i_brand_id,i_class_id,i_category_id,sum(sales), sum(number_sales) + from( + select 'store' channel, i_brand_id,i_class_id + ,i_category_id,sum(ss_quantity*ss_list_price) sales + , count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2000+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales) + union all + select 'catalog' channel, i_brand_id,i_class_id,i_category_id, sum(cs_quantity*cs_list_price) sales, count(*) number_sales + from catalog_sales + ,item + ,date_dim + where cs_item_sk in (select ss_item_sk from cross_items) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2000+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(cs_quantity*cs_list_price) > (select average_sales from avg_sales) + union all + select 'web' channel, i_brand_id,i_class_id,i_category_id, sum(ws_quantity*ws_list_price) sales , count(*) number_sales + from web_sales + ,item + ,date_dim + where ws_item_sk in (select ss_item_sk from cross_items) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2000+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(ws_quantity*ws_list_price) > (select average_sales from avg_sales) + ) y + group by rollup (channel, i_brand_id,i_class_id,i_category_id) + order by channel,i_brand_id,i_class_id,i_category_id + limit 100; +with cross_items as + (select i_item_sk ss_item_sk + from item, + (select iss.i_brand_id brand_id + ,iss.i_class_id class_id + ,iss.i_category_id category_id + from store_sales + ,item iss + ,date_dim d1 + where ss_item_sk = iss.i_item_sk + and ss_sold_date_sk = d1.d_date_sk + and d1.d_year between 2000 AND 2000 + 2 + intersect + select ics.i_brand_id + ,ics.i_class_id + ,ics.i_category_id + from catalog_sales + ,item ics + ,date_dim d2 + where cs_item_sk = ics.i_item_sk + and cs_sold_date_sk = d2.d_date_sk + and d2.d_year between 2000 AND 2000 + 2 + intersect + select iws.i_brand_id + ,iws.i_class_id + ,iws.i_category_id + from web_sales + ,item iws + ,date_dim d3 + where ws_item_sk = iws.i_item_sk + and ws_sold_date_sk = d3.d_date_sk + and d3.d_year between 2000 AND 2000 + 2) x + where i_brand_id = brand_id + and i_class_id = class_id + and i_category_id = category_id +), + avg_sales as +(select avg(quantity*list_price) average_sales + from (select ss_quantity quantity + ,ss_list_price list_price + from store_sales + ,date_dim + where ss_sold_date_sk = d_date_sk + and d_year between 2000 and 2000 + 2 + union all + select cs_quantity quantity + ,cs_list_price list_price + from catalog_sales + ,date_dim + where cs_sold_date_sk = d_date_sk + and d_year between 2000 and 2000 + 2 + union all + select ws_quantity quantity + ,ws_list_price list_price + from web_sales + ,date_dim + where ws_sold_date_sk = d_date_sk + and d_year between 2000 and 2000 + 2) x) + select this_year.channel ty_channel + ,this_year.i_brand_id ty_brand + ,this_year.i_class_id ty_class + ,this_year.i_category_id ty_category + ,this_year.sales ty_sales + ,this_year.number_sales ty_number_sales + ,last_year.channel ly_channel + ,last_year.i_brand_id ly_brand + ,last_year.i_class_id ly_class + ,last_year.i_category_id ly_category + ,last_year.sales ly_sales + ,last_year.number_sales ly_number_sales + from + (select 'store' channel, i_brand_id,i_class_id,i_category_id + ,sum(ss_quantity*ss_list_price) sales, count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_week_seq = (select d_week_seq + from date_dim + where d_year = 2000 + 1 + and d_moy = 12 + and d_dom = 15) + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) this_year, + (select 'store' channel, i_brand_id,i_class_id + ,i_category_id, sum(ss_quantity*ss_list_price) sales, count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_week_seq = (select d_week_seq + from date_dim + where d_year = 2000 + and d_moy = 12 + and d_dom = 15) + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) last_year + where this_year.i_brand_id= last_year.i_brand_id + and this_year.i_class_id = last_year.i_class_id + and this_year.i_category_id = last_year.i_category_id + order by this_year.channel, this_year.i_brand_id, this_year.i_class_id, this_year.i_category_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query15.sql b/tools/tpcds-tools/queries/sf10000/query15.sql new file mode 100644 index 000000000000000..fa9a6e0dfdbd30b --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query15.sql @@ -0,0 +1,17 @@ +select ca_zip + ,sum(cs_sales_price) + from catalog_sales + ,customer + ,customer_address + ,date_dim + where cs_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', + '85392', '85460', '80348', '81792') + or ca_state in ('CA','WA','GA') + or cs_sales_price > 500) + and cs_sold_date_sk = d_date_sk + and d_qoy = 2 and d_year = 1998 + group by ca_zip + order by ca_zip + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query16.sql b/tools/tpcds-tools/queries/sf10000/query16.sql new file mode 100644 index 000000000000000..168756d19804a26 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query16.sql @@ -0,0 +1,28 @@ +select + count(distinct cs_order_number) as "order count" + ,sum(cs_ext_ship_cost) as "total shipping cost" + ,sum(cs_net_profit) as "total net profit" +from + catalog_sales cs1 + ,date_dim + ,customer_address + ,call_center +where + d_date between '1999-4-01' and + (cast('1999-4-01' as date) + interval 60 day) +and cs1.cs_ship_date_sk = d_date_sk +and cs1.cs_ship_addr_sk = ca_address_sk +and ca_state = 'IL' +and cs1.cs_call_center_sk = cc_call_center_sk +and cc_county in ('Richland County','Bronx County','Maverick County','Mesa County', + 'Raleigh County' +) +and exists (select * + from catalog_sales cs2 + where cs1.cs_order_number = cs2.cs_order_number + and cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) +and not exists(select * + from catalog_returns cr1 + where cs1.cs_order_number = cr1.cr_order_number) +order by count(distinct cs_order_number) +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query17.sql b/tools/tpcds-tools/queries/sf10000/query17.sql new file mode 100644 index 000000000000000..92a50896739c739 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query17.sql @@ -0,0 +1,42 @@ +select i_item_id + ,i_item_desc + ,s_state + ,count(ss_quantity) as store_sales_quantitycount + ,avg(ss_quantity) as store_sales_quantityave + ,stddev_samp(ss_quantity) as store_sales_quantitystdev + ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov + ,count(sr_return_quantity) as store_returns_quantitycount + ,avg(sr_return_quantity) as store_returns_quantityave + ,stddev_samp(sr_return_quantity) as store_returns_quantitystdev + ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov + ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave + ,stddev_samp(cs_quantity) as catalog_sales_quantitystdev + ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov + from store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where d1.d_quarter_name = '2000Q1' + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_quarter_name in ('2000Q1','2000Q2','2000Q3') + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_quarter_name in ('2000Q1','2000Q2','2000Q3') + group by i_item_id + ,i_item_desc + ,s_state + order by i_item_id + ,i_item_desc + ,s_state +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query18.sql b/tools/tpcds-tools/queries/sf10000/query18.sql new file mode 100644 index 000000000000000..705cb5dc34e3793 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query18.sql @@ -0,0 +1,31 @@ +select i_item_id, + ca_country, + ca_state, + ca_county, + avg( cast(cs_quantity as decimal(12,2))) agg1, + avg( cast(cs_list_price as decimal(12,2))) agg2, + avg( cast(cs_coupon_amt as decimal(12,2))) agg3, + avg( cast(cs_sales_price as decimal(12,2))) agg4, + avg( cast(cs_net_profit as decimal(12,2))) agg5, + avg( cast(c_birth_year as decimal(12,2))) agg6, + avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7 + from catalog_sales, customer_demographics cd1, + customer_demographics cd2, customer, customer_address, date_dim, item + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd1.cd_demo_sk and + cs_bill_customer_sk = c_customer_sk and + cd1.cd_gender = 'M' and + cd1.cd_education_status = 'Unknown' and + c_current_cdemo_sk = cd2.cd_demo_sk and + c_current_addr_sk = ca_address_sk and + c_birth_month in (5,1,4,7,8,9) and + d_year = 2002 and + ca_state in ('AR','TX','NC' + ,'GA','MS','WV','AL') + group by rollup (i_item_id, ca_country, ca_state, ca_county) + order by ca_country, + ca_state, + ca_county, + i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query19.sql b/tools/tpcds-tools/queries/sf10000/query19.sql new file mode 100644 index 000000000000000..545ae56059c78e2 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query19.sql @@ -0,0 +1,22 @@ +select i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item,customer,customer_address,store + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=16 + and d_moy=12 + and d_year=1998 + and ss_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and substr(ca_zip,1,5) <> substr(s_zip,1,5) + and ss_store_sk = s_store_sk + group by i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact + order by ext_price desc + ,i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact +limit 100 ; diff --git a/tools/tpcds-tools/queries/sf10000/query2.sql b/tools/tpcds-tools/queries/sf10000/query2.sql new file mode 100644 index 000000000000000..5dc50b1156c022e --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query2.sql @@ -0,0 +1,57 @@ +with wscs as + (select sold_date_sk + ,sales_price + from (select ws_sold_date_sk sold_date_sk + ,ws_ext_sales_price sales_price + from web_sales + union all + select cs_sold_date_sk sold_date_sk + ,cs_ext_sales_price sales_price + from catalog_sales) t), + wswscs as + (select d_week_seq, + sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales + from wscs + ,date_dim + where d_date_sk = sold_date_sk + group by d_week_seq) + select d_week_seq1 + ,round(sun_sales1/sun_sales2,2) + ,round(mon_sales1/mon_sales2,2) + ,round(tue_sales1/tue_sales2,2) + ,round(wed_sales1/wed_sales2,2) + ,round(thu_sales1/thu_sales2,2) + ,round(fri_sales1/fri_sales2,2) + ,round(sat_sales1/sat_sales2,2) + from + (select wswscs.d_week_seq d_week_seq1 + ,sun_sales sun_sales1 + ,mon_sales mon_sales1 + ,tue_sales tue_sales1 + ,wed_sales wed_sales1 + ,thu_sales thu_sales1 + ,fri_sales fri_sales1 + ,sat_sales sat_sales1 + from wswscs,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 1998) y, + (select wswscs.d_week_seq d_week_seq2 + ,sun_sales sun_sales2 + ,mon_sales mon_sales2 + ,tue_sales tue_sales2 + ,wed_sales wed_sales2 + ,thu_sales thu_sales2 + ,fri_sales fri_sales2 + ,sat_sales sat_sales2 + from wswscs + ,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 1998+1) z + where d_week_seq1=d_week_seq2-53 + order by d_week_seq1; diff --git a/tools/tpcds-tools/queries/sf10000/query20.sql b/tools/tpcds-tools/queries/sf10000/query20.sql new file mode 100644 index 000000000000000..0584c8492cdb11f --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query20.sql @@ -0,0 +1,27 @@ +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(cs_ext_sales_price) as itemrevenue + ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over + (partition by i_class) as revenueratio + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and i_category in ('Shoes', 'Electronics', 'Children') + and cs_sold_date_sk = d_date_sk + and d_date between cast('2001-03-14' as date) + and (cast('2001-03-14' as date) + interval 30 day) + group by i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + order by i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query21.sql b/tools/tpcds-tools/queries/sf10000/query21.sql new file mode 100644 index 000000000000000..cdf32b08d584c5f --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query21.sql @@ -0,0 +1,27 @@ +select * + from(select w_warehouse_name + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('1999-03-20' as date)) + then inv_quantity_on_hand + else 0 end) as inv_before + ,sum(case when (cast(d_date as date) >= cast ('1999-03-20' as date)) + then inv_quantity_on_hand + else 0 end) as inv_after + from inventory + ,warehouse + ,item + ,date_dim + where i_current_price between 0.99 and 1.49 + and i_item_sk = inv_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_date between (cast ('1999-03-20' as date) - interval 30 day) + and (cast ('1999-03-20' as date) + interval 30 day) + group by w_warehouse_name, i_item_id) x + where (case when inv_before > 0 + then inv_after / inv_before + else null + end) between 2.0/3.0 and 3.0/2.0 + order by w_warehouse_name + ,i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query22.sql b/tools/tpcds-tools/queries/sf10000/query22.sql new file mode 100644 index 000000000000000..9732e81a6399a04 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query22.sql @@ -0,0 +1,17 @@ +select i_product_name + ,i_brand + ,i_class + ,i_category + ,avg(inv_quantity_on_hand) qoh + from inventory + ,date_dim + ,item + where inv_date_sk=d_date_sk + and inv_item_sk=i_item_sk + and d_month_seq between 1186 and 1186 + 11 + group by rollup(i_product_name + ,i_brand + ,i_class + ,i_category) +order by qoh, i_product_name, i_brand, i_class, i_category +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query23.sql b/tools/tpcds-tools/queries/sf10000/query23.sql new file mode 100644 index 000000000000000..ff73709f4ebc95a --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query23.sql @@ -0,0 +1,104 @@ +with frequent_ss_items as + (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt + from store_sales + ,date_dim + ,item + where ss_sold_date_sk = d_date_sk + and ss_item_sk = i_item_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by substr(i_item_desc,1,30),i_item_sk,d_date + having count(*) >4), + max_store_sales as + (select max(csales) tpcds_cmax + from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales + from store_sales + ,date_dim + ,customer + where ss_customer_sk = c_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by c_customer_sk) t), + best_ss_customer as + (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales + from store_sales + ,customer + where ss_customer_sk = c_customer_sk + group by c_customer_sk + having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select + * +from + max_store_sales)) + select sum(sales) + from (select cs_quantity*cs_list_price sales + from catalog_sales + ,date_dim + where d_year = 2000 + and d_moy = 3 + and cs_sold_date_sk = d_date_sk + and cs_item_sk in (select item_sk from frequent_ss_items) + and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) + union all + select ws_quantity*ws_list_price sales + from web_sales + ,date_dim + where d_year = 2000 + and d_moy = 3 + and ws_sold_date_sk = d_date_sk + and ws_item_sk in (select item_sk from frequent_ss_items) + and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)) t2 + limit 100; +with frequent_ss_items as + (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt + from store_sales + ,date_dim + ,item + where ss_sold_date_sk = d_date_sk + and ss_item_sk = i_item_sk + and d_year in (2000,2000 + 1,2000 + 2,2000 + 3) + group by substr(i_item_desc,1,30),i_item_sk,d_date + having count(*) >4), + max_store_sales as + (select max(csales) tpcds_cmax + from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales + from store_sales + ,customer + ,date_dim + where ss_customer_sk = c_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by c_customer_sk) t), + best_ss_customer as + (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales + from store_sales + ,customer + where ss_customer_sk = c_customer_sk + group by c_customer_sk + having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select + * + from max_store_sales)) + select c_last_name,c_first_name,sales + from (select c_last_name,c_first_name,sum(cs_quantity*cs_list_price) sales + from catalog_sales + ,customer + ,date_dim + where d_year = 2000 + and d_moy = 3 + and cs_sold_date_sk = d_date_sk + and cs_item_sk in (select item_sk from frequent_ss_items) + and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) + and cs_bill_customer_sk = c_customer_sk + group by c_last_name,c_first_name + union all + select c_last_name,c_first_name,sum(ws_quantity*ws_list_price) sales + from web_sales + ,customer + ,date_dim + where d_year = 2000 + and d_moy = 3 + and ws_sold_date_sk = d_date_sk + and ws_item_sk in (select item_sk from frequent_ss_items) + and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer) + and ws_bill_customer_sk = c_customer_sk + group by c_last_name,c_first_name) t2 + order by c_last_name,c_first_name,sales + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query24.sql b/tools/tpcds-tools/queries/sf10000/query24.sql new file mode 100644 index 000000000000000..98102afa7170e8e --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query24.sql @@ -0,0 +1,104 @@ +with ssales as +(select c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size + ,sum(ss_sales_price) netpaid +from store_sales + ,store_returns + ,store + ,item + ,customer + ,customer_address +where ss_ticket_number = sr_ticket_number + and ss_item_sk = sr_item_sk + and ss_customer_sk = c_customer_sk + and ss_item_sk = i_item_sk + and ss_store_sk = s_store_sk + and c_current_addr_sk = ca_address_sk + and c_birth_country <> upper(ca_country) + and s_zip = ca_zip +and s_market_id=10 +group by c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size) +select c_last_name + ,c_first_name + ,s_store_name + ,sum(netpaid) paid +from ssales +where i_color = 'snow' +group by c_last_name + ,c_first_name + ,s_store_name +having sum(netpaid) > (select 0.05*avg(netpaid) + from ssales) +order by c_last_name + ,c_first_name + ,s_store_name +; +with ssales as +(select c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size + ,sum(ss_sales_price) netpaid +from store_sales + ,store_returns + ,store + ,item + ,customer + ,customer_address +where ss_ticket_number = sr_ticket_number + and ss_item_sk = sr_item_sk + and ss_customer_sk = c_customer_sk + and ss_item_sk = i_item_sk + and ss_store_sk = s_store_sk + and c_current_addr_sk = ca_address_sk + and c_birth_country <> upper(ca_country) + and s_zip = ca_zip + and s_market_id = 10 +group by c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size) +select c_last_name + ,c_first_name + ,s_store_name + ,sum(netpaid) paid +from ssales +where i_color = 'chiffon' +group by c_last_name + ,c_first_name + ,s_store_name +having sum(netpaid) > (select 0.05*avg(netpaid) + from ssales) +order by c_last_name + ,c_first_name + ,s_store_name +; diff --git a/tools/tpcds-tools/queries/sf10000/query25.sql b/tools/tpcds-tools/queries/sf10000/query25.sql new file mode 100644 index 000000000000000..45071c93b7ad05f --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query25.sql @@ -0,0 +1,45 @@ +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,sum(ss_net_profit) as store_sales_profit + ,sum(sr_net_loss) as store_returns_loss + ,sum(cs_net_profit) as catalog_sales_profit + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 2000 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 10 + and d2.d_year = 2000 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_moy between 4 and 10 + and d3.d_year = 2000 + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query26.sql b/tools/tpcds-tools/queries/sf10000/query26.sql new file mode 100644 index 000000000000000..b28f1b37a191016 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query26.sql @@ -0,0 +1,18 @@ +select i_item_id, + avg(cs_quantity) agg1, + avg(cs_list_price) agg2, + avg(cs_coupon_amt) agg3, + avg(cs_sales_price) agg4 + from catalog_sales, customer_demographics, date_dim, item, promotion + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd_demo_sk and + cs_promo_sk = p_promo_sk and + cd_gender = 'F' and + cd_marital_status = 'S' and + cd_education_status = 'College' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 1998 + group by i_item_id + order by i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query27.sql b/tools/tpcds-tools/queries/sf10000/query27.sql new file mode 100644 index 000000000000000..4615e63de098d00 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query27.sql @@ -0,0 +1,20 @@ +select i_item_id, + s_state, grouping(s_state) g_state, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, store, item + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_store_sk = s_store_sk and + ss_cdemo_sk = cd_demo_sk and + cd_gender = 'F' and + cd_marital_status = 'U' and + cd_education_status = '2 yr Degree' and + d_year = 2000 and + s_state in ('AL','IN', 'SC', 'NY', 'OH', 'FL') + group by rollup (i_item_id, s_state) + order by i_item_id + ,s_state + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query28.sql b/tools/tpcds-tools/queries/sf10000/query28.sql new file mode 100644 index 000000000000000..f85c3f3f7bc559c --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query28.sql @@ -0,0 +1,50 @@ +select * +from (select avg(ss_list_price) B1_LP + ,count(ss_list_price) B1_CNT + ,count(distinct ss_list_price) B1_CNTD + from store_sales + where ss_quantity between 0 and 5 + and (ss_list_price between 73 and 73+10 + or ss_coupon_amt between 7826 and 7826+1000 + or ss_wholesale_cost between 70 and 70+20)) B1, + (select avg(ss_list_price) B2_LP + ,count(ss_list_price) B2_CNT + ,count(distinct ss_list_price) B2_CNTD + from store_sales + where ss_quantity between 6 and 10 + and (ss_list_price between 152 and 152+10 + or ss_coupon_amt between 2196 and 2196+1000 + or ss_wholesale_cost between 56 and 56+20)) B2, + (select avg(ss_list_price) B3_LP + ,count(ss_list_price) B3_CNT + ,count(distinct ss_list_price) B3_CNTD + from store_sales + where ss_quantity between 11 and 15 + and (ss_list_price between 53 and 53+10 + or ss_coupon_amt between 3430 and 3430+1000 + or ss_wholesale_cost between 13 and 13+20)) B3, + (select avg(ss_list_price) B4_LP + ,count(ss_list_price) B4_CNT + ,count(distinct ss_list_price) B4_CNTD + from store_sales + where ss_quantity between 16 and 20 + and (ss_list_price between 182 and 182+10 + or ss_coupon_amt between 3262 and 3262+1000 + or ss_wholesale_cost between 20 and 20+20)) B4, + (select avg(ss_list_price) B5_LP + ,count(ss_list_price) B5_CNT + ,count(distinct ss_list_price) B5_CNTD + from store_sales + where ss_quantity between 21 and 25 + and (ss_list_price between 85 and 85+10 + or ss_coupon_amt between 3310 and 3310+1000 + or ss_wholesale_cost between 37 and 37+20)) B5, + (select avg(ss_list_price) B6_LP + ,count(ss_list_price) B6_CNT + ,count(distinct ss_list_price) B6_CNTD + from store_sales + where ss_quantity between 26 and 30 + and (ss_list_price between 180 and 180+10 + or ss_coupon_amt between 12592 and 12592+1000 + or ss_wholesale_cost between 22 and 22+20)) B6 +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query29.sql b/tools/tpcds-tools/queries/sf10000/query29.sql new file mode 100644 index 000000000000000..660431e1f8ce608 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query29.sql @@ -0,0 +1,44 @@ +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,stddev_samp(ss_quantity) as store_sales_quantity + ,stddev_samp(sr_return_quantity) as store_returns_quantity + ,stddev_samp(cs_quantity) as catalog_sales_quantity + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 1998 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 4 + 3 + and d2.d_year = 1998 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_year in (1998,1998+1,1998+2) + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query3.sql b/tools/tpcds-tools/queries/sf10000/query3.sql new file mode 100644 index 000000000000000..6f368b3a2b52800 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query3.sql @@ -0,0 +1,18 @@ +select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_sales_price) sum_agg + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 816 + and dt.d_moy=11 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,sum_agg desc + ,brand_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query30.sql b/tools/tpcds-tools/queries/sf10000/query30.sql new file mode 100644 index 000000000000000..db07e4af929b22a --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query30.sql @@ -0,0 +1,28 @@ +with customer_total_return as + (select wr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(wr_return_amt) as ctr_total_return + from web_returns + ,date_dim + ,customer_address + where wr_returned_date_sk = d_date_sk + and d_year =2000 + and wr_returning_addr_sk = ca_address_sk + group by wr_returning_customer_sk + ,ca_state) + select c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date_sk,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'GA' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date_sk,ctr_total_return +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query31.sql b/tools/tpcds-tools/queries/sf10000/query31.sql new file mode 100644 index 000000000000000..15457ad1f4cae05 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query31.sql @@ -0,0 +1,49 @@ +with ss as + (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales + from store_sales,date_dim,customer_address + where ss_sold_date_sk = d_date_sk + and ss_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year), + ws as + (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales + from web_sales,date_dim,customer_address + where ws_sold_date_sk = d_date_sk + and ws_bill_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year) + select + ss1.ca_county + ,ss1.d_year + ,ws2.web_sales/ws1.web_sales web_q1_q2_increase + ,ss2.store_sales/ss1.store_sales store_q1_q2_increase + ,ws3.web_sales/ws2.web_sales web_q2_q3_increase + ,ss3.store_sales/ss2.store_sales store_q2_q3_increase + from + ss ss1 + ,ss ss2 + ,ss ss3 + ,ws ws1 + ,ws ws2 + ,ws ws3 + where + ss1.d_qoy = 1 + and ss1.d_year = 1999 + and ss1.ca_county = ss2.ca_county + and ss2.d_qoy = 2 + and ss2.d_year = 1999 + and ss2.ca_county = ss3.ca_county + and ss3.d_qoy = 3 + and ss3.d_year = 1999 + and ss1.ca_county = ws1.ca_county + and ws1.d_qoy = 1 + and ws1.d_year = 1999 + and ws1.ca_county = ws2.ca_county + and ws2.d_qoy = 2 + and ws2.d_year = 1999 + and ws1.ca_county = ws3.ca_county + and ws3.d_qoy = 3 + and ws3.d_year =1999 + and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end + > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end + and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end + > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end + order by ss1.d_year; diff --git a/tools/tpcds-tools/queries/sf10000/query32.sql b/tools/tpcds-tools/queries/sf10000/query32.sql new file mode 100644 index 000000000000000..9605ddee8e5f02d --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query32.sql @@ -0,0 +1,25 @@ +select sum(cs_ext_discount_amt) as "excess discount amount" +from + catalog_sales + ,item + ,date_dim +where +i_manufact_id = 66 +and i_item_sk = cs_item_sk +and d_date between '2002-03-29' and + (cast('2002-03-29' as date) + interval 90 day) +and d_date_sk = cs_sold_date_sk +and cs_ext_discount_amt + > ( + select + 1.3 * avg(cs_ext_discount_amt) + from + catalog_sales + ,date_dim + where + cs_item_sk = i_item_sk + and d_date between '2002-03-29' and + (cast('2002-03-29' as date) + interval 90 day) + and d_date_sk = cs_sold_date_sk + ) +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query33.sql b/tools/tpcds-tools/queries/sf10000/query33.sql new file mode 100644 index 000000000000000..039fab4cb45b671 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query33.sql @@ -0,0 +1,72 @@ +with ss as ( + select + i_manufact_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Home')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 5 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_manufact_id), + cs as ( + select + i_manufact_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Home')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 5 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_manufact_id), + ws as ( + select + i_manufact_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Home')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 5 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_manufact_id) + select i_manufact_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_manufact_id + order by total_sales +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query34.sql b/tools/tpcds-tools/queries/sf10000/query34.sql new file mode 100644 index 000000000000000..25bb51aefe1befd --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query34.sql @@ -0,0 +1,28 @@ +select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28) + and (household_demographics.hd_buy_potential = '>10000' or + household_demographics.hd_buy_potential = 'Unknown') + and household_demographics.hd_vehicle_count > 0 + and (case when household_demographics.hd_vehicle_count > 0 + then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count + else null + end) > 1.2 + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_county in ('Salem County','Terrell County','Arthur County','Oglethorpe County', + 'Lunenburg County','Perry County','Halifax County','Sumner County') + group by ss_ticket_number,ss_customer_sk) dn,customer + where ss_customer_sk = c_customer_sk + and cnt between 15 and 20 + order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc, ss_ticket_number; diff --git a/tools/tpcds-tools/queries/sf10000/query35.sql b/tools/tpcds-tools/queries/sf10000/query35.sql new file mode 100644 index 000000000000000..02ecebe57fad539 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query35.sql @@ -0,0 +1,55 @@ +select + ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + count(*) cnt1, + avg(cd_dep_count), + min(cd_dep_count), + stddev_samp(cd_dep_count), + cd_dep_employed_count, + count(*) cnt2, + avg(cd_dep_employed_count), + min(cd_dep_employed_count), + stddev_samp(cd_dep_employed_count), + cd_dep_college_count, + count(*) cnt3, + avg(cd_dep_college_count), + min(cd_dep_college_count), + stddev_samp(cd_dep_college_count) + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2001 and + d_qoy < 4) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2001 and + d_qoy < 4) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2001 and + d_qoy < 4)) + group by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query36.sql b/tools/tpcds-tools/queries/sf10000/query36.sql new file mode 100644 index 000000000000000..e187c0b52e1518c --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query36.sql @@ -0,0 +1,27 @@ +select + sum(ss_net_profit)/sum(ss_ext_sales_price) as gross_margin + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,item + ,store + where + d1.d_year = 1999 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and s_state in ('IN','AL','MI','MN', + 'TN','LA','FL','NM') + group by rollup(i_category,i_class) + order by + lochierarchy desc + ,case when lochierarchy = 0 then i_category end + ,rank_within_parent + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query37.sql b/tools/tpcds-tools/queries/sf10000/query37.sql new file mode 100644 index 000000000000000..bca739219452ecb --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query37.sql @@ -0,0 +1,14 @@ +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, catalog_sales + where i_current_price between 39 and 39 + 30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('2001-01-16' as date) and (cast('2001-01-16' as date) + interval 60 day) + and i_manufact_id in (765,886,889,728) + and inv_quantity_on_hand between 100 and 500 + and cs_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query38.sql b/tools/tpcds-tools/queries/sf10000/query38.sql new file mode 100644 index 000000000000000..03c72adccfd946b --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query38.sql @@ -0,0 +1,20 @@ +select count(*) from ( + select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1186 and 1186 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1186 and 1186 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1186 and 1186 + 11 +) hot_cust +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query39.sql b/tools/tpcds-tools/queries/sf10000/query39.sql new file mode 100644 index 000000000000000..47faacbdd55457b --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query39.sql @@ -0,0 +1,51 @@ +with inv as +(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stdev,mean, case mean when 0 then null else stdev/mean end cov + from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean + from inventory + ,item + ,warehouse + ,date_dim + where inv_item_sk = i_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_year =2000 + group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo + where case mean when 0 then 0 else stdev/mean end > 1) +select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov + ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov +from inv inv1,inv inv2 +where inv1.i_item_sk = inv2.i_item_sk + and inv1.w_warehouse_sk = inv2.w_warehouse_sk + and inv1.d_moy=2 + and inv2.d_moy=2+1 +order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov + ,inv2.d_moy,inv2.mean, inv2.cov +; +with inv as +(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stdev,mean, case mean when 0 then null else stdev/mean end cov + from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean + from inventory + ,item + ,warehouse + ,date_dim + where inv_item_sk = i_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_year =2000 + group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo + where case mean when 0 then 0 else stdev/mean end > 1) +select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov + ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov +from inv inv1,inv inv2 +where inv1.i_item_sk = inv2.i_item_sk + and inv1.w_warehouse_sk = inv2.w_warehouse_sk + and inv1.d_moy=2 + and inv2.d_moy=2+1 + and inv1.cov > 1.5 +order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov + ,inv2.d_moy,inv2.mean, inv2.cov +; diff --git a/tools/tpcds-tools/queries/sf10000/query4.sql b/tools/tpcds-tools/queries/sf10000/query4.sql new file mode 100644 index 000000000000000..99d6806f73d77a8 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query4.sql @@ -0,0 +1,113 @@ +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(((ss_ext_list_price-ss_ext_wholesale_cost-ss_ext_discount_amt)+ss_ext_sales_price)/2) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum((((cs_ext_list_price-cs_ext_wholesale_cost-cs_ext_discount_amt)+cs_ext_sales_price)/2) ) year_total + ,'c' sale_type + from customer + ,catalog_sales + ,date_dim + where c_customer_sk = cs_bill_customer_sk + and cs_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year +union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum((((ws_ext_list_price-ws_ext_wholesale_cost-ws_ext_discount_amt)+ws_ext_sales_price)/2) ) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + ) + select + t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_birth_country + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_c_firstyear + ,year_total t_c_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_c_secyear.customer_id + and t_s_firstyear.customer_id = t_c_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_c_firstyear.sale_type = 'c' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_c_secyear.sale_type = 'c' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.dyear = 1999 + and t_s_secyear.dyear = 1999+1 + and t_c_firstyear.dyear = 1999 + and t_c_secyear.dyear = 1999+1 + and t_w_firstyear.dyear = 1999 + and t_w_secyear.dyear = 1999+1 + and t_s_firstyear.year_total > 0 + and t_c_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end + and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end + > case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end + order by t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_birth_country +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query40.sql b/tools/tpcds-tools/queries/sf10000/query40.sql new file mode 100644 index 000000000000000..5054c79e820dbbf --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query40.sql @@ -0,0 +1,25 @@ +select + w_state + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('2000-03-18' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_before + ,sum(case when (cast(d_date as date) >= cast ('2000-03-18' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_after + from + catalog_sales left outer join catalog_returns on + (cs_order_number = cr_order_number + and cs_item_sk = cr_item_sk) + ,warehouse + ,item + ,date_dim + where + i_current_price between 0.99 and 1.49 + and i_item_sk = cs_item_sk + and cs_warehouse_sk = w_warehouse_sk + and cs_sold_date_sk = d_date_sk + and d_date between (cast ('2000-03-18' as date) - interval 30 day) + and (cast ('2000-03-18' as date) + interval 30 day) + group by + w_state,i_item_id + order by w_state,i_item_id +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query41.sql b/tools/tpcds-tools/queries/sf10000/query41.sql new file mode 100644 index 000000000000000..8088b80e5237160 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query41.sql @@ -0,0 +1,49 @@ +select distinct(i_product_name) + from item i1 + where i_manufact_id between 970 and 970+40 + and (select count(*) as item_cnt + from item + where (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'frosted' or i_color = 'rose') and + (i_units = 'Lb' or i_units = 'Gross') and + (i_size = 'medium' or i_size = 'large') + ) or + (i_category = 'Women' and + (i_color = 'chocolate' or i_color = 'black') and + (i_units = 'Box' or i_units = 'Dram') and + (i_size = 'economy' or i_size = 'petite') + ) or + (i_category = 'Men' and + (i_color = 'slate' or i_color = 'magenta') and + (i_units = 'Carton' or i_units = 'Bundle') and + (i_size = 'N/A' or i_size = 'small') + ) or + (i_category = 'Men' and + (i_color = 'cornflower' or i_color = 'firebrick') and + (i_units = 'Pound' or i_units = 'Oz') and + (i_size = 'medium' or i_size = 'large') + ))) or + (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'almond' or i_color = 'steel') and + (i_units = 'Tsp' or i_units = 'Case') and + (i_size = 'medium' or i_size = 'large') + ) or + (i_category = 'Women' and + (i_color = 'purple' or i_color = 'aquamarine') and + (i_units = 'Bunch' or i_units = 'Gram') and + (i_size = 'economy' or i_size = 'petite') + ) or + (i_category = 'Men' and + (i_color = 'lavender' or i_color = 'papaya') and + (i_units = 'Pallet' or i_units = 'Cup') and + (i_size = 'N/A' or i_size = 'small') + ) or + (i_category = 'Men' and + (i_color = 'maroon' or i_color = 'cyan') and + (i_units = 'Each' or i_units = 'N/A') and + (i_size = 'medium' or i_size = 'large') + )))) > 0 + order by i_product_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query42.sql b/tools/tpcds-tools/queries/sf10000/query42.sql new file mode 100644 index 000000000000000..be2813175154ebb --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query42.sql @@ -0,0 +1,19 @@ +select dt.d_year + ,item.i_category_id + ,item.i_category + ,sum(ss_ext_sales_price) + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=12 + and dt.d_year=1998 + group by dt.d_year + ,item.i_category_id + ,item.i_category + order by sum(ss_ext_sales_price) desc,dt.d_year + ,item.i_category_id + ,item.i_category +limit 100 ; diff --git a/tools/tpcds-tools/queries/sf10000/query43.sql b/tools/tpcds-tools/queries/sf10000/query43.sql new file mode 100644 index 000000000000000..e9359a13649ff55 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query43.sql @@ -0,0 +1,16 @@ +select s_store_name, s_store_id, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from date_dim, store_sales, store + where d_date_sk = ss_sold_date_sk and + s_store_sk = ss_store_sk and + s_gmt_offset = -6 and + d_year = 2001 + group by s_store_name, s_store_id + order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query44.sql b/tools/tpcds-tools/queries/sf10000/query44.sql new file mode 100644 index 000000000000000..ad1e178cbd38b4d --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query44.sql @@ -0,0 +1,32 @@ +select asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing +from(select * + from (select item_sk,rank() over (order by rank_col asc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 366 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 366 + and ss_cdemo_sk is null + group by ss_store_sk))V1)V11 + where rnk < 11) asceding, + (select * + from (select item_sk,rank() over (order by rank_col desc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 366 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 366 + and ss_cdemo_sk is null + group by ss_store_sk))V2)V21 + where rnk < 11) descending, +item i1, +item i2 +where asceding.rnk = descending.rnk + and i1.i_item_sk=asceding.item_sk + and i2.i_item_sk=descending.item_sk +order by asceding.rnk +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query45.sql b/tools/tpcds-tools/queries/sf10000/query45.sql new file mode 100644 index 000000000000000..ba3e6131abef52c --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query45.sql @@ -0,0 +1,17 @@ +select ca_zip, ca_county, sum(ws_sales_price) + from web_sales, customer, customer_address, date_dim, item + where ws_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ws_item_sk = i_item_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792') + or + i_item_id in (select i_item_id + from item + where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) + ) + ) + and ws_sold_date_sk = d_date_sk + and d_qoy = 1 and d_year = 1998 + group by ca_zip, ca_county + order by ca_zip, ca_county + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query46.sql b/tools/tpcds-tools/queries/sf10000/query46.sql new file mode 100644 index 000000000000000..dfaf1212927b36f --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query46.sql @@ -0,0 +1,32 @@ +select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and (household_demographics.hd_dep_count = 0 or + household_demographics.hd_vehicle_count= 1) + and date_dim.d_dow in (6,0) + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_city in ('Five Forks','Oakland','Fairview','Winchester','Farmington') + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query47.sql b/tools/tpcds-tools/queries/sf10000/query47.sql new file mode 100644 index 000000000000000..4784c25e7233c67 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query47.sql @@ -0,0 +1,48 @@ +with v1 as( + select i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, + s_store_name, s_company_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + s_store_name, s_company_name + order by d_year, d_moy) rn + from item, store_sales, date_dim, store + where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + ( + d_year = 1999 or + ( d_year = 1999-1 and d_moy =12) or + ( d_year = 1999+1 and d_moy =1) + ) + group by i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy), + v2 as( + select v1.s_store_name + ,v1.d_year, v1.d_moy + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1.s_store_name = v1_lag.s_store_name and + v1.s_store_name = v1_lead.s_store_name and + v1.s_company_name = v1_lag.s_company_name and + v1.s_company_name = v1_lead.s_company_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 1999 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, sum_sales + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query48.sql b/tools/tpcds-tools/queries/sf10000/query48.sql new file mode 100644 index 000000000000000..841863f1db6ffe2 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query48.sql @@ -0,0 +1,64 @@ +select sum (ss_quantity) + from store_sales, store, customer_demographics, customer_address, date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 1998 + and + ( + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'M' + and + cd_education_status = 'Unknown' + and + ss_sales_price between 100.00 and 150.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'W' + and + cd_education_status = 'College' + and + ss_sales_price between 50.00 and 100.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'D' + and + cd_education_status = 'Primary' + and + ss_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('MI', 'GA', 'NH') + and ss_net_profit between 0 and 2000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('TX', 'KY', 'SD') + and ss_net_profit between 150 and 3000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('NY', 'OH', 'FL') + and ss_net_profit between 50 and 25000 + ) + ) +; diff --git a/tools/tpcds-tools/queries/sf10000/query49.sql b/tools/tpcds-tools/queries/sf10000/query49.sql new file mode 100644 index 000000000000000..6c45212cf32943d --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query49.sql @@ -0,0 +1,126 @@ +select channel, item, return_ratio, return_rank, currency_rank from + (select + 'web' as channel + ,web.item + ,web.return_ratio + ,web.return_rank + ,web.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select ws.ws_item_sk as item + ,(cast(sum(coalesce(wr.wr_return_quantity,0)) as decimal(15,4))/ + cast(sum(coalesce(ws.ws_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(wr.wr_return_amt,0)) as decimal(15,4))/ + cast(sum(coalesce(ws.ws_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + web_sales ws left outer join web_returns wr + on (ws.ws_order_number = wr.wr_order_number and + ws.ws_item_sk = wr.wr_item_sk) + ,date_dim + where + wr.wr_return_amt > 10000 + and ws.ws_net_profit > 1 + and ws.ws_net_paid > 0 + and ws.ws_quantity > 0 + and ws_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 12 + group by ws.ws_item_sk + ) in_web + ) web + where + ( + web.return_rank <= 10 + or + web.currency_rank <= 10 + ) + union + select + 'catalog' as channel + ,catalog.item + ,catalog.return_ratio + ,catalog.return_rank + ,catalog.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select + cs.cs_item_sk as item + ,(cast(sum(coalesce(cr.cr_return_quantity,0)) as decimal(15,4))/ + cast(sum(coalesce(cs.cs_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(cr.cr_return_amount,0)) as decimal(15,4))/ + cast(sum(coalesce(cs.cs_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + catalog_sales cs left outer join catalog_returns cr + on (cs.cs_order_number = cr.cr_order_number and + cs.cs_item_sk = cr.cr_item_sk) + ,date_dim + where + cr.cr_return_amount > 10000 + and cs.cs_net_profit > 1 + and cs.cs_net_paid > 0 + and cs.cs_quantity > 0 + and cs_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 12 + group by cs.cs_item_sk + ) in_cat + ) catalog + where + ( + catalog.return_rank <= 10 + or + catalog.currency_rank <=10 + ) + union + select + 'store' as channel + ,store.item + ,store.return_ratio + ,store.return_rank + ,store.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select sts.ss_item_sk as item + ,(cast(sum(coalesce(sr.sr_return_quantity,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(sr.sr_return_amt,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + store_sales sts left outer join store_returns sr + on (sts.ss_ticket_number = sr.sr_ticket_number and sts.ss_item_sk = sr.sr_item_sk) + ,date_dim + where + sr.sr_return_amt > 10000 + and sts.ss_net_profit > 1 + and sts.ss_net_paid > 0 + and sts.ss_quantity > 0 + and ss_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 12 + group by sts.ss_item_sk + ) in_store + ) store + where ( + store.return_rank <= 10 + or + store.currency_rank <= 10 + ) + ) + t order by 1,4,5,2 + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query5.sql b/tools/tpcds-tools/queries/sf10000/query5.sql new file mode 100644 index 000000000000000..96c97b7c8a1636a --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query5.sql @@ -0,0 +1,125 @@ +with ssr as + (select s_store_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( select ss_store_sk as store_sk, + ss_sold_date_sk as date_sk, + ss_ext_sales_price as sales_price, + ss_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from store_sales + union all + select sr_store_sk as store_sk, + sr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + sr_return_amt as return_amt, + sr_net_loss as net_loss + from store_returns + ) salesreturns, + date_dim, + store + where date_sk = d_date_sk + and d_date between cast('2000-08-19' as date) + and (cast('2000-08-19' as date) + interval 14 day) + and store_sk = s_store_sk + group by s_store_id) + , + csr as + (select cp_catalog_page_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( select cs_catalog_page_sk as page_sk, + cs_sold_date_sk as date_sk, + cs_ext_sales_price as sales_price, + cs_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from catalog_sales + union all + select cr_catalog_page_sk as page_sk, + cr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + cr_return_amount as return_amt, + cr_net_loss as net_loss + from catalog_returns + ) salesreturns, + date_dim, + catalog_page + where date_sk = d_date_sk + and d_date between cast('2000-08-19' as date) + and (cast('2000-08-19' as date) + interval 14 day) + and page_sk = cp_catalog_page_sk + group by cp_catalog_page_id) + , + wsr as + (select web_site_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( select ws_web_site_sk as wsr_web_site_sk, + ws_sold_date_sk as date_sk, + ws_ext_sales_price as sales_price, + ws_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from web_sales + union all + select ws_web_site_sk as wsr_web_site_sk, + wr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + wr_return_amt as return_amt, + wr_net_loss as net_loss + from web_returns left outer join web_sales on + ( wr_item_sk = ws_item_sk + and wr_order_number = ws_order_number) + ) salesreturns, + date_dim, + web_site + where date_sk = d_date_sk + and d_date between cast('2000-08-19' as date) + and (cast('2000-08-19' as date) + interval 14 day) + and wsr_web_site_sk = web_site_sk + group by web_site_id) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , concat('store', s_store_id) id + , sales + , returns + , (profit - profit_loss) as profit + from ssr + union all + select 'catalog channel' as channel + , concat('catalog_page', cp_catalog_page_id) id + , sales + , returns + , (profit - profit_loss) as profit + from csr + union all + select 'web channel' as channel + , concat('web_site', web_site_id) id + , sales + , returns + , (profit - profit_loss) as profit + from wsr + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query50.sql b/tools/tpcds-tools/queries/sf10000/query50.sql new file mode 100644 index 000000000000000..0b0d70d726a6a25 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query50.sql @@ -0,0 +1,56 @@ +select + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 30) and + (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 60) and + (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 90) and + (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 120) then 1 else 0 end) as ">120 days" +from + store_sales + ,store_returns + ,store + ,date_dim d1 + ,date_dim d2 +where + d2.d_year = 1998 +and d2.d_moy = 9 +and ss_ticket_number = sr_ticket_number +and ss_item_sk = sr_item_sk +and ss_sold_date_sk = d1.d_date_sk +and sr_returned_date_sk = d2.d_date_sk +and ss_customer_sk = sr_customer_sk +and ss_store_sk = s_store_sk +group by + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip +order by s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query51.sql b/tools/tpcds-tools/queries/sf10000/query51.sql new file mode 100644 index 000000000000000..c83e2325c060be0 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query51.sql @@ -0,0 +1,42 @@ +WITH web_v1 as ( +select + ws_item_sk item_sk, d_date, + sum(sum(ws_sales_price)) + over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from web_sales + ,date_dim +where ws_sold_date_sk=d_date_sk + and d_month_seq between 1214 and 1214+11 + and ws_item_sk is not NULL +group by ws_item_sk, d_date), +store_v1 as ( +select + ss_item_sk item_sk, d_date, + sum(sum(ss_sales_price)) + over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from store_sales + ,date_dim +where ss_sold_date_sk=d_date_sk + and d_month_seq between 1214 and 1214+11 + and ss_item_sk is not NULL +group by ss_item_sk, d_date) + select * +from (select item_sk + ,d_date + ,web_sales + ,store_sales + ,max(web_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) web_cumulative + ,max(store_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) store_cumulative + from (select case when web.item_sk is not null then web.item_sk else store.item_sk end item_sk + ,case when web.d_date is not null then web.d_date else store.d_date end d_date + ,web.cume_sales web_sales + ,store.cume_sales store_sales + from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk + and web.d_date = store.d_date) + )x )y +where web_cumulative > store_cumulative +order by item_sk + ,d_date +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query52.sql b/tools/tpcds-tools/queries/sf10000/query52.sql new file mode 100644 index 000000000000000..2fce4c3fdf1c278 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query52.sql @@ -0,0 +1,19 @@ +select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_ext_sales_price) ext_price + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=12 + and dt.d_year=2000 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,ext_price desc + ,brand_id +limit 100 ; diff --git a/tools/tpcds-tools/queries/sf10000/query53.sql b/tools/tpcds-tools/queries/sf10000/query53.sql new file mode 100644 index 000000000000000..dcbc31a03661082 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query53.sql @@ -0,0 +1,25 @@ +select * from +(select i_manufact_id, +sum(ss_sales_price) sum_sales, +avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and +ss_sold_date_sk = d_date_sk and +ss_store_sk = s_store_sk and +d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11) and +((i_category in ('Books','Children','Electronics') and +i_class in ('personal','portable','reference','self-help') and +i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7', + 'exportiunivamalg #9','scholaramalgamalg #9')) +or(i_category in ('Women','Music','Men') and +i_class in ('accessories','classical','fragrances','pants') and +i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1', + 'importoamalg #1'))) +group by i_manufact_id, d_qoy ) tmp1 +where case when avg_quarterly_sales > 0 + then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales + else null end > 0.1 +order by avg_quarterly_sales, + sum_sales, + i_manufact_id +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query54.sql b/tools/tpcds-tools/queries/sf10000/query54.sql new file mode 100644 index 000000000000000..d68d9f67549ab05 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query54.sql @@ -0,0 +1,53 @@ +with my_customers as ( + select distinct c_customer_sk + , c_current_addr_sk + from + ( select cs_sold_date_sk sold_date_sk, + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + from catalog_sales + union all + select ws_sold_date_sk sold_date_sk, + ws_bill_customer_sk customer_sk, + ws_item_sk item_sk + from web_sales + ) cs_or_ws_sales, + item, + date_dim, + customer + where sold_date_sk = d_date_sk + and item_sk = i_item_sk + and i_category = 'Books' + and i_class = 'business' + and c_customer_sk = cs_or_ws_sales.customer_sk + and d_moy = 2 + and d_year = 2000 + ) + , my_revenue as ( + select c_customer_sk, + sum(ss_ext_sales_price) as revenue + from my_customers, + store_sales, + customer_address, + store, + date_dim + where c_current_addr_sk = ca_address_sk + and ca_county = s_county + and ca_state = s_state + and ss_sold_date_sk = d_date_sk + and c_customer_sk = ss_customer_sk + and d_month_seq between (select distinct d_month_seq+1 + from date_dim where d_year = 2000 and d_moy = 2) + and (select distinct d_month_seq+3 + from date_dim where d_year = 2000 and d_moy = 2) + group by c_customer_sk + ) + , segments as + (select cast((revenue/50) as int) as segment + from my_revenue + ) + select segment, count(*) as num_customers, segment*50 as segment_base + from segments + group by segment + order by segment, num_customers + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query55.sql b/tools/tpcds-tools/queries/sf10000/query55.sql new file mode 100644 index 000000000000000..86b815c48efeb89 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query55.sql @@ -0,0 +1,11 @@ +select i_brand_id brand_id, i_brand brand, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=13 + and d_moy=11 + and d_year=1999 + group by i_brand, i_brand_id + order by ext_price desc, i_brand_id +limit 100 ; diff --git a/tools/tpcds-tools/queries/sf10000/query56.sql b/tools/tpcds-tools/queries/sf10000/query56.sql new file mode 100644 index 000000000000000..0c3ce2733190bb0 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query56.sql @@ -0,0 +1,66 @@ +with ss as ( + select i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where i_item_id in (select + i_item_id +from item +where i_color in ('chiffon','smoke','lace')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 5 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + cs as ( + select i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('chiffon','smoke','lace')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 5 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + ws as ( + select i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('chiffon','smoke','lace')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 5 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id) + select i_item_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by total_sales, + i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query57.sql b/tools/tpcds-tools/queries/sf10000/query57.sql new file mode 100644 index 000000000000000..0d8f7528fa934b4 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query57.sql @@ -0,0 +1,45 @@ +with v1 as( + select i_category, i_brand, + cc_name, + d_year, d_moy, + sum(cs_sales_price) sum_sales, + avg(sum(cs_sales_price)) over + (partition by i_category, i_brand, + cc_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + cc_name + order by d_year, d_moy) rn + from item, catalog_sales, date_dim, call_center + where cs_item_sk = i_item_sk and + cs_sold_date_sk = d_date_sk and + cc_call_center_sk= cs_call_center_sk and + ( + d_year = 1999 or + ( d_year = 1999-1 and d_moy =12) or + ( d_year = 1999+1 and d_moy =1) + ) + group by i_category, i_brand, + cc_name , d_year, d_moy), + v2 as( + select v1.i_category, v1.i_brand + ,v1.d_year, v1.d_moy + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1. cc_name = v1_lag. cc_name and + v1. cc_name = v1_lead. cc_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 1999 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, avg_monthly_sales + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query58.sql b/tools/tpcds-tools/queries/sf10000/query58.sql new file mode 100644 index 000000000000000..693719c04e81c98 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query58.sql @@ -0,0 +1,62 @@ +with ss_items as + (select i_item_id item_id + ,sum(ss_ext_sales_price) ss_item_rev + from store_sales + ,item + ,date_dim + where ss_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '1998-02-21')) + and ss_sold_date_sk = d_date_sk + group by i_item_id), + cs_items as + (select i_item_id item_id + ,sum(cs_ext_sales_price) cs_item_rev + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '1998-02-21')) + and cs_sold_date_sk = d_date_sk + group by i_item_id), + ws_items as + (select i_item_id item_id + ,sum(ws_ext_sales_price) ws_item_rev + from web_sales + ,item + ,date_dim + where ws_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq =(select d_week_seq + from date_dim + where d_date = '1998-02-21')) + and ws_sold_date_sk = d_date_sk + group by i_item_id) + select ss_items.item_id + ,ss_item_rev + ,ss_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ss_dev + ,cs_item_rev + ,cs_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 cs_dev + ,ws_item_rev + ,ws_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ws_dev + ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average + from ss_items,cs_items,ws_items + where ss_items.item_id=cs_items.item_id + and ss_items.item_id=ws_items.item_id + and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + order by item_id + ,ss_item_rev + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query59.sql b/tools/tpcds-tools/queries/sf10000/query59.sql new file mode 100644 index 000000000000000..05d2165771143a8 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query59.sql @@ -0,0 +1,41 @@ +with wss as + (select d_week_seq, + ss_store_sk, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + group by d_week_seq,ss_store_sk + ) + select s_store_name1,s_store_id1,d_week_seq1 + ,sun_sales1/sun_sales2,mon_sales1/mon_sales2 + ,tue_sales1/tue_sales2,wed_sales1/wed_sales2,thu_sales1/thu_sales2 + ,fri_sales1/fri_sales2,sat_sales1/sat_sales2 + from + (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1 + ,s_store_id s_store_id1,sun_sales sun_sales1 + ,mon_sales mon_sales1,tue_sales tue_sales1 + ,wed_sales wed_sales1,thu_sales thu_sales1 + ,fri_sales fri_sales1,sat_sales sat_sales1 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + ss_store_sk = s_store_sk and + d_month_seq between 1205 and 1205 + 11) y, + (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2 + ,s_store_id s_store_id2,sun_sales sun_sales2 + ,mon_sales mon_sales2,tue_sales tue_sales2 + ,wed_sales wed_sales2,thu_sales thu_sales2 + ,fri_sales fri_sales2,sat_sales sat_sales2 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + ss_store_sk = s_store_sk and + d_month_seq between 1205+ 12 and 1205 + 23) x + where s_store_id1=s_store_id2 + and d_week_seq1=d_week_seq2-52 + order by s_store_name1,s_store_id1,d_week_seq1 +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query6.sql b/tools/tpcds-tools/queries/sf10000/query6.sql new file mode 100644 index 000000000000000..00decd2cdc48a55 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query6.sql @@ -0,0 +1,23 @@ +select a.ca_state state, count(*) cnt + from customer_address a + ,customer c + ,store_sales s + ,date_dim d + ,item i + where a.ca_address_sk = c.c_current_addr_sk + and c.c_customer_sk = s.ss_customer_sk + and s.ss_sold_date_sk = d.d_date_sk + and s.ss_item_sk = i.i_item_sk + and d.d_month_seq = + (select distinct (d_month_seq) + from date_dim + where d_year = 2002 + and d_moy = 3 ) + and i.i_current_price > 1.2 * + (select avg(j.i_current_price) + from item j + where j.i_category = i.i_category) + group by a.ca_state + having count(*) >= 10 + order by cnt, a.ca_state + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query60.sql b/tools/tpcds-tools/queries/sf10000/query60.sql new file mode 100644 index 000000000000000..666a809d715a028 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query60.sql @@ -0,0 +1,75 @@ +with ss as ( + select + i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Children')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 10 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_item_id), + cs as ( + select + i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Children')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 10 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_item_id), + ws as ( + select + i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Children')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 10 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_item_id) + select + i_item_id +,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by i_item_id + ,total_sales + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query61.sql b/tools/tpcds-tools/queries/sf10000/query61.sql new file mode 100644 index 000000000000000..2c818159df1cea7 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query61.sql @@ -0,0 +1,41 @@ +select promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100 +from + (select sum(ss_ext_sales_price) promotions + from store_sales + ,store + ,promotion + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_promo_sk = p_promo_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -6 + and i_category = 'Sports' + and (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y') + and s_gmt_offset = -6 + and d_year = 2001 + and d_moy = 12) promotional_sales, + (select sum(ss_ext_sales_price) total + from store_sales + ,store + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -6 + and i_category = 'Sports' + and s_gmt_offset = -6 + and d_year = 2001 + and d_moy = 12) all_sales +order by promotions, total +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query62.sql b/tools/tpcds-tools/queries/sf10000/query62.sql new file mode 100644 index 000000000000000..2d097a9089cb17e --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query62.sql @@ -0,0 +1,32 @@ +select + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 30) and + (ws_ship_date_sk - ws_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 60) and + (ws_ship_date_sk - ws_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 90) and + (ws_ship_date_sk - ws_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 120) then 1 else 0 end) as ">120 days" +from + web_sales + ,warehouse + ,ship_mode + ,web_site + ,date_dim +where + d_month_seq between 1215 and 1215 + 11 +and ws_ship_date_sk = d_date_sk +and ws_warehouse_sk = w_warehouse_sk +and ws_ship_mode_sk = sm_ship_mode_sk +and ws_web_site_sk = web_site_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,web_name +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query63.sql b/tools/tpcds-tools/queries/sf10000/query63.sql new file mode 100644 index 000000000000000..795fb596c8c33d6 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query63.sql @@ -0,0 +1,26 @@ +select * +from (select i_manager_id + ,sum(ss_sales_price) sum_sales + ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales + from item + ,store_sales + ,date_dim + ,store + where ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and d_month_seq in (1211,1211+1,1211+2,1211+3,1211+4,1211+5,1211+6,1211+7,1211+8,1211+9,1211+10,1211+11) + and (( i_category in ('Books','Children','Electronics') + and i_class in ('personal','portable','reference','self-help') + and i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7', + 'exportiunivamalg #9','scholaramalgamalg #9')) + or( i_category in ('Women','Music','Men') + and i_class in ('accessories','classical','fragrances','pants') + and i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1', + 'importoamalg #1'))) +group by i_manager_id, d_moy) tmp1 +where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 +order by i_manager_id + ,avg_monthly_sales + ,sum_sales +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query64.sql b/tools/tpcds-tools/queries/sf10000/query64.sql new file mode 100644 index 000000000000000..13be8a4687a9014 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query64.sql @@ -0,0 +1,118 @@ +with cs_ui as + (select cs_item_sk + ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund + from catalog_sales + ,catalog_returns + where cs_item_sk = cr_item_sk + and cs_order_number = cr_order_number + group by cs_item_sk + having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)), +cross_sales as + (select i_product_name product_name + ,i_item_sk item_sk + ,s_store_name store_name + ,s_zip store_zip + ,ad1.ca_street_number b_street_number + ,ad1.ca_street_name b_street_name + ,ad1.ca_city b_city + ,ad1.ca_zip b_zip + ,ad2.ca_street_number c_street_number + ,ad2.ca_street_name c_street_name + ,ad2.ca_city c_city + ,ad2.ca_zip c_zip + ,d1.d_year as syear + ,d2.d_year as fsyear + ,d3.d_year s2year + ,count(*) cnt + ,sum(ss_wholesale_cost) s1 + ,sum(ss_list_price) s2 + ,sum(ss_coupon_amt) s3 + FROM store_sales + ,store_returns + ,cs_ui + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,customer + ,customer_demographics cd1 + ,customer_demographics cd2 + ,promotion + ,household_demographics hd1 + ,household_demographics hd2 + ,customer_address ad1 + ,customer_address ad2 + ,income_band ib1 + ,income_band ib2 + ,item + WHERE ss_store_sk = s_store_sk AND + ss_sold_date_sk = d1.d_date_sk AND + ss_customer_sk = c_customer_sk AND + ss_cdemo_sk= cd1.cd_demo_sk AND + ss_hdemo_sk = hd1.hd_demo_sk AND + ss_addr_sk = ad1.ca_address_sk and + ss_item_sk = i_item_sk and + ss_item_sk = sr_item_sk and + ss_ticket_number = sr_ticket_number and + ss_item_sk = cs_ui.cs_item_sk and + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_hdemo_sk = hd2.hd_demo_sk AND + c_current_addr_sk = ad2.ca_address_sk and + c_first_sales_date_sk = d2.d_date_sk and + c_first_shipto_date_sk = d3.d_date_sk and + ss_promo_sk = p_promo_sk and + hd1.hd_income_band_sk = ib1.ib_income_band_sk and + hd2.hd_income_band_sk = ib2.ib_income_band_sk and + cd1.cd_marital_status <> cd2.cd_marital_status and + i_color in ('azure','gainsboro','misty','blush','hot','lemon') and + i_current_price between 80 and 80 + 10 and + i_current_price between 80 + 1 and 80 + 15 +group by i_product_name + ,i_item_sk + ,s_store_name + ,s_zip + ,ad1.ca_street_number + ,ad1.ca_street_name + ,ad1.ca_city + ,ad1.ca_zip + ,ad2.ca_street_number + ,ad2.ca_street_name + ,ad2.ca_city + ,ad2.ca_zip + ,d1.d_year + ,d2.d_year + ,d3.d_year +) +select cs1.product_name + ,cs1.store_name + ,cs1.store_zip + ,cs1.b_street_number + ,cs1.b_street_name + ,cs1.b_city + ,cs1.b_zip + ,cs1.c_street_number + ,cs1.c_street_name + ,cs1.c_city + ,cs1.c_zip + ,cs1.syear + ,cs1.cnt + ,cs1.s1 as s11 + ,cs1.s2 as s21 + ,cs1.s3 as s31 + ,cs2.s1 as s12 + ,cs2.s2 as s22 + ,cs2.s3 as s32 + ,cs2.syear + ,cs2.cnt +from cross_sales cs1,cross_sales cs2 +where cs1.item_sk=cs2.item_sk and + cs1.syear = 1999 and + cs2.syear = 1999 + 1 and + cs2.cnt <= cs1.cnt and + cs1.store_name = cs2.store_name and + cs1.store_zip = cs2.store_zip +order by cs1.product_name + ,cs1.store_name + ,cs2.cnt + ,cs1.s1 + ,cs2.s1; diff --git a/tools/tpcds-tools/queries/sf10000/query65.sql b/tools/tpcds-tools/queries/sf10000/query65.sql new file mode 100644 index 000000000000000..132241a0b38567a --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query65.sql @@ -0,0 +1,26 @@ +select + s_store_name, + i_item_desc, + sc.revenue, + i_current_price, + i_wholesale_cost, + i_brand + from store, item, + (select ss_store_sk, avg(revenue) as ave + from + (select ss_store_sk, ss_item_sk, + sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1186 and 1186+11 + group by ss_store_sk, ss_item_sk) sa + group by ss_store_sk) sb, + (select ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1186 and 1186+11 + group by ss_store_sk, ss_item_sk) sc + where sb.ss_store_sk = sc.ss_store_sk and + sc.revenue <= 0.1 * sb.ave and + s_store_sk = sc.ss_store_sk and + i_item_sk = sc.ss_item_sk + order by s_store_name, i_item_desc +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query66.sql b/tools/tpcds-tools/queries/sf10000/query66.sql new file mode 100644 index 000000000000000..554185b6bbe5f80 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query66.sql @@ -0,0 +1,217 @@ +select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,ship_carriers + ,year + ,sum(jan_sales) as jan_sales + ,sum(feb_sales) as feb_sales + ,sum(mar_sales) as mar_sales + ,sum(apr_sales) as apr_sales + ,sum(may_sales) as may_sales + ,sum(jun_sales) as jun_sales + ,sum(jul_sales) as jul_sales + ,sum(aug_sales) as aug_sales + ,sum(sep_sales) as sep_sales + ,sum(oct_sales) as oct_sales + ,sum(nov_sales) as nov_sales + ,sum(dec_sales) as dec_sales + ,sum(jan_sales/w_warehouse_sq_ft) as jan_sales_per_sq_foot + ,sum(feb_sales/w_warehouse_sq_ft) as feb_sales_per_sq_foot + ,sum(mar_sales/w_warehouse_sq_ft) as mar_sales_per_sq_foot + ,sum(apr_sales/w_warehouse_sq_ft) as apr_sales_per_sq_foot + ,sum(may_sales/w_warehouse_sq_ft) as may_sales_per_sq_foot + ,sum(jun_sales/w_warehouse_sq_ft) as jun_sales_per_sq_foot + ,sum(jul_sales/w_warehouse_sq_ft) as jul_sales_per_sq_foot + ,sum(aug_sales/w_warehouse_sq_ft) as aug_sales_per_sq_foot + ,sum(sep_sales/w_warehouse_sq_ft) as sep_sales_per_sq_foot + ,sum(oct_sales/w_warehouse_sq_ft) as oct_sales_per_sq_foot + ,sum(nov_sales/w_warehouse_sq_ft) as nov_sales_per_sq_foot + ,sum(dec_sales/w_warehouse_sq_ft) as dec_sales_per_sq_foot + ,sum(jan_net) as jan_net + ,sum(feb_net) as feb_net + ,sum(mar_net) as mar_net + ,sum(apr_net) as apr_net + ,sum(may_net) as may_net + ,sum(jun_net) as jun_net + ,sum(jul_net) as jul_net + ,sum(aug_net) as aug_net + ,sum(sep_net) as sep_net + ,sum(oct_net) as oct_net + ,sum(nov_net) as nov_net + ,sum(dec_net) as dec_net + from ( + select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,concat(concat('MSC ', ','), ' GERMA') as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then ws_sales_price* ws_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then ws_sales_price* ws_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then ws_sales_price* ws_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then ws_sales_price* ws_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then ws_sales_price* ws_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then ws_sales_price* ws_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then ws_sales_price* ws_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then ws_sales_price* ws_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then ws_sales_price* ws_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then ws_sales_price* ws_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then ws_sales_price* ws_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then ws_sales_price* ws_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then ws_net_paid_inc_ship_tax * ws_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then ws_net_paid_inc_ship_tax * ws_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then ws_net_paid_inc_ship_tax * ws_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then ws_net_paid_inc_ship_tax * ws_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then ws_net_paid_inc_ship_tax * ws_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then ws_net_paid_inc_ship_tax * ws_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then ws_net_paid_inc_ship_tax * ws_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then ws_net_paid_inc_ship_tax * ws_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then ws_net_paid_inc_ship_tax * ws_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then ws_net_paid_inc_ship_tax * ws_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then ws_net_paid_inc_ship_tax * ws_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then ws_net_paid_inc_ship_tax * ws_quantity else 0 end) as dec_net + from + web_sales + ,warehouse + ,date_dim + ,time_dim + ,ship_mode + where + ws_warehouse_sk = w_warehouse_sk + and ws_sold_date_sk = d_date_sk + and ws_sold_time_sk = t_time_sk + and ws_ship_mode_sk = sm_ship_mode_sk + and d_year = 2001 + and t_time between 9453 and 9453+28800 + and sm_carrier in ('MSC','GERMA') + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,d_year + union all + select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,concat(concat('MSC ', ','), ' GERMA') as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then cs_ext_list_price* cs_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then cs_ext_list_price* cs_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then cs_ext_list_price* cs_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then cs_ext_list_price* cs_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then cs_ext_list_price* cs_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then cs_ext_list_price* cs_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then cs_ext_list_price* cs_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then cs_ext_list_price* cs_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then cs_ext_list_price* cs_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then cs_ext_list_price* cs_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then cs_ext_list_price* cs_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then cs_ext_list_price* cs_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then cs_net_paid_inc_ship * cs_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then cs_net_paid_inc_ship * cs_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then cs_net_paid_inc_ship * cs_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then cs_net_paid_inc_ship * cs_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then cs_net_paid_inc_ship * cs_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then cs_net_paid_inc_ship * cs_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then cs_net_paid_inc_ship * cs_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then cs_net_paid_inc_ship * cs_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then cs_net_paid_inc_ship * cs_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then cs_net_paid_inc_ship * cs_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then cs_net_paid_inc_ship * cs_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then cs_net_paid_inc_ship * cs_quantity else 0 end) as dec_net + from + catalog_sales + ,warehouse + ,date_dim + ,time_dim + ,ship_mode + where + cs_warehouse_sk = w_warehouse_sk + and cs_sold_date_sk = d_date_sk + and cs_sold_time_sk = t_time_sk + and cs_ship_mode_sk = sm_ship_mode_sk + and d_year = 2001 + and t_time between 9453 AND 9453+28800 + and sm_carrier in ('MSC','GERMA') + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,d_year + ) x + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,ship_carriers + ,year + order by w_warehouse_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query67.sql b/tools/tpcds-tools/queries/sf10000/query67.sql new file mode 100644 index 000000000000000..8954025ce6b2090 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query67.sql @@ -0,0 +1,41 @@ +select * +from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rank() over (partition by i_category order by sumsales desc) rk + from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sum(coalesce(ss_sales_price*ss_quantity,0)) sumsales + from store_sales + ,date_dim + ,store + ,item + where ss_sold_date_sk=d_date_sk + and ss_item_sk=i_item_sk + and ss_store_sk = s_store_sk + and d_month_seq between 1185 and 1185+11 + group by rollup(i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy,s_store_id))dw1) dw2 +where rk <= 100 +order by i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rk +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query68.sql b/tools/tpcds-tools/queries/sf10000/query68.sql new file mode 100644 index 000000000000000..1fc63a89a54cfcf --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query68.sql @@ -0,0 +1,39 @@ +select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,extended_price + ,extended_tax + ,list_price + from (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_ext_sales_price) extended_price + ,sum(ss_ext_list_price) list_price + ,sum(ss_ext_tax) extended_tax + from store_sales + ,date_dim + ,store + ,household_demographics + ,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_dep_count = 4 or + household_demographics.hd_vehicle_count= 0) + and date_dim.d_year in (1999,1999+1,1999+2) + and store.s_city in ('Pleasant Hill','Bethel') + group by ss_ticket_number + ,ss_customer_sk + ,ss_addr_sk,ca_city) dn + ,customer + ,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,ss_ticket_number + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query69.sql b/tools/tpcds-tools/queries/sf10000/query69.sql new file mode 100644 index 000000000000000..e5ef37541d1c2dc --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query69.sql @@ -0,0 +1,44 @@ +select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_state in ('MO','MN','AZ') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2003 and + d_moy between 2 and 2+2) and + (not exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2003 and + d_moy between 2 and 2+2) and + not exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2003 and + d_moy between 2 and 2+2)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query7.sql b/tools/tpcds-tools/queries/sf10000/query7.sql new file mode 100644 index 000000000000000..c4b33b79d18b8a8 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query7.sql @@ -0,0 +1,18 @@ +select i_item_id, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, item, promotion + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_cdemo_sk = cd_demo_sk and + ss_promo_sk = p_promo_sk and + cd_gender = 'F' and + cd_marital_status = 'W' and + cd_education_status = 'College' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 2001 + group by i_item_id + order by i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query70.sql b/tools/tpcds-tools/queries/sf10000/query70.sql new file mode 100644 index 000000000000000..1dfd094f4f26a04 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query70.sql @@ -0,0 +1,35 @@ +select + sum(ss_net_profit) as total_sum + ,s_state + ,s_county + ,grouping(s_state)+grouping(s_county) as lochierarchy + ,rank() over ( + partition by grouping(s_state)+grouping(s_county), + case when grouping(s_county) = 0 then s_state end + order by sum(ss_net_profit) desc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,store + where + d1.d_month_seq between 1218 and 1218+11 + and d1.d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + and s_state in + ( select s_state + from (select s_state as s_state, + rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking + from store_sales, store, date_dim + where d_month_seq between 1218 and 1218+11 + and d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + group by s_state + ) tmp1 + where ranking <= 5 + ) + group by rollup(s_state,s_county) + order by + lochierarchy desc + ,case when lochierarchy = 0 then s_state end + ,rank_within_parent + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query71.sql b/tools/tpcds-tools/queries/sf10000/query71.sql new file mode 100644 index 000000000000000..040ba7278e98f64 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query71.sql @@ -0,0 +1,37 @@ +select i_brand_id brand_id, i_brand brand,t_hour,t_minute, + sum(ext_price) ext_price + from item, (select ws_ext_sales_price as ext_price, + ws_sold_date_sk as sold_date_sk, + ws_item_sk as sold_item_sk, + ws_sold_time_sk as time_sk + from web_sales,date_dim + where d_date_sk = ws_sold_date_sk + and d_moy=12 + and d_year=2000 + union all + select cs_ext_sales_price as ext_price, + cs_sold_date_sk as sold_date_sk, + cs_item_sk as sold_item_sk, + cs_sold_time_sk as time_sk + from catalog_sales,date_dim + where d_date_sk = cs_sold_date_sk + and d_moy=12 + and d_year=2000 + union all + select ss_ext_sales_price as ext_price, + ss_sold_date_sk as sold_date_sk, + ss_item_sk as sold_item_sk, + ss_sold_time_sk as time_sk + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + and d_moy=12 + and d_year=2000 + ) tmp,time_dim + where + sold_item_sk = i_item_sk + and i_manager_id=1 + and time_sk = t_time_sk + and (t_meal_time = 'breakfast' or t_meal_time = 'dinner') + group by i_brand, i_brand_id,t_hour,t_minute + order by ext_price desc, i_brand_id + ; diff --git a/tools/tpcds-tools/queries/sf10000/query72.sql b/tools/tpcds-tools/queries/sf10000/query72.sql new file mode 100644 index 000000000000000..61d0ca7110e0cbe --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query72.sql @@ -0,0 +1,26 @@ +select i_item_desc + ,w_warehouse_name + ,d1.d_week_seq + ,sum(case when p_promo_sk is null then 1 else 0 end) no_promo + ,sum(case when p_promo_sk is not null then 1 else 0 end) promo + ,count(*) total_cnt +from catalog_sales +join inventory on (cs_item_sk = inv_item_sk) +join warehouse on (w_warehouse_sk=inv_warehouse_sk) +join item on (i_item_sk = cs_item_sk) +join customer_demographics on (cs_bill_cdemo_sk = cd_demo_sk) +join household_demographics on (cs_bill_hdemo_sk = hd_demo_sk) +join date_dim d1 on (cs_sold_date_sk = d1.d_date_sk) +join date_dim d2 on (inv_date_sk = d2.d_date_sk) +join date_dim d3 on (cs_ship_date_sk = d3.d_date_sk) +left outer join promotion on (cs_promo_sk=p_promo_sk) +left outer join catalog_returns on (cr_item_sk = cs_item_sk and cr_order_number = cs_order_number) +where d1.d_week_seq = d2.d_week_seq + and inv_quantity_on_hand < cs_quantity + and d3.d_date > d1.d_date + 5 + and hd_buy_potential = '1001-5000' + and d1.d_year = 2000 + and cd_marital_status = 'D' +group by i_item_desc,w_warehouse_name,d1.d_week_seq +order by total_cnt desc, i_item_desc, w_warehouse_name, d_week_seq +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query73.sql b/tools/tpcds-tools/queries/sf10000/query73.sql new file mode 100644 index 000000000000000..aebfd074dd6056e --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query73.sql @@ -0,0 +1,25 @@ +select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_buy_potential = '>10000' or + household_demographics.hd_buy_potential = '5001-10000') + and household_demographics.hd_vehicle_count > 0 + and case when household_demographics.hd_vehicle_count > 0 then + household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1 + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_county in ('Lea County','Furnas County','Pennington County','Bronx County') + group by ss_ticket_number,ss_customer_sk) dj,customer + where ss_customer_sk = c_customer_sk + and cnt between 1 and 5 + order by cnt desc, c_last_name asc; diff --git a/tools/tpcds-tools/queries/sf10000/query74.sql b/tools/tpcds-tools/queries/sf10000/query74.sql new file mode 100644 index 000000000000000..ce9dc4b627eee67 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query74.sql @@ -0,0 +1,58 @@ +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,sum(ss_net_paid) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (1998,1998+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,sum(ws_net_paid) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + and d_year in (1998,1998+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + ) + select + t_s_secyear.customer_id, t_s_secyear.customer_first_name, t_s_secyear.customer_last_name + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.year = 1998 + and t_s_secyear.year = 1998+1 + and t_w_firstyear.year = 1998 + and t_w_secyear.year = 1998+1 + and t_s_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end + order by 3,1,2 +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query75.sql b/tools/tpcds-tools/queries/sf10000/query75.sql new file mode 100644 index 000000000000000..89f55e20b2f7537 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query75.sql @@ -0,0 +1,67 @@ +WITH all_sales AS ( + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,SUM(sales_cnt) AS sales_cnt + ,SUM(sales_amt) AS sales_amt + FROM (SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt + ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt + FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk + JOIN date_dim ON d_date_sk=cs_sold_date_sk + LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number + AND cs_item_sk=cr_item_sk) + WHERE i_category='Sports' + UNION + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt + ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt + FROM store_sales JOIN item ON i_item_sk=ss_item_sk + JOIN date_dim ON d_date_sk=ss_sold_date_sk + LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number + AND ss_item_sk=sr_item_sk) + WHERE i_category='Sports' + UNION + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt + ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt + FROM web_sales JOIN item ON i_item_sk=ws_item_sk + JOIN date_dim ON d_date_sk=ws_sold_date_sk + LEFT JOIN web_returns ON (ws_order_number=wr_order_number + AND ws_item_sk=wr_item_sk) + WHERE i_category='Sports') sales_detail + GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) + SELECT prev_yr.d_year AS prev_year + ,curr_yr.d_year AS year + ,curr_yr.i_brand_id + ,curr_yr.i_class_id + ,curr_yr.i_category_id + ,curr_yr.i_manufact_id + ,prev_yr.sales_cnt AS prev_yr_cnt + ,curr_yr.sales_cnt AS curr_yr_cnt + ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff + ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff + FROM all_sales curr_yr, all_sales prev_yr + WHERE curr_yr.i_brand_id=prev_yr.i_brand_id + AND curr_yr.i_class_id=prev_yr.i_class_id + AND curr_yr.i_category_id=prev_yr.i_category_id + AND curr_yr.i_manufact_id=prev_yr.i_manufact_id + AND curr_yr.d_year=2001 + AND prev_yr.d_year=2001-1 + AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9 + ORDER BY sales_cnt_diff,sales_amt_diff + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query76.sql b/tools/tpcds-tools/queries/sf10000/query76.sql new file mode 100644 index 000000000000000..bc1a292eac01344 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query76.sql @@ -0,0 +1,21 @@ +select channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM ( + SELECT 'store' as channel, 'ss_customer_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price + FROM store_sales, item, date_dim + WHERE ss_customer_sk IS NULL + AND ss_sold_date_sk=d_date_sk + AND ss_item_sk=i_item_sk + UNION ALL + SELECT 'web' as channel, 'ws_ship_addr_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price + FROM web_sales, item, date_dim + WHERE ws_ship_addr_sk IS NULL + AND ws_sold_date_sk=d_date_sk + AND ws_item_sk=i_item_sk + UNION ALL + SELECT 'catalog' as channel, 'cs_ship_mode_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price + FROM catalog_sales, item, date_dim + WHERE cs_ship_mode_sk IS NULL + AND cs_sold_date_sk=d_date_sk + AND cs_item_sk=i_item_sk) foo +GROUP BY channel, col_name, d_year, d_qoy, i_category +ORDER BY channel, col_name, d_year, d_qoy, i_category +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query77.sql b/tools/tpcds-tools/queries/sf10000/query77.sql new file mode 100644 index 000000000000000..e61f06a1e22e040 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query77.sql @@ -0,0 +1,105 @@ +with ss as + (select s_store_sk, + sum(ss_ext_sales_price) as sales, + sum(ss_net_profit) as profit + from store_sales, + date_dim, + store + where ss_sold_date_sk = d_date_sk + and d_date between cast('2000-08-16' as date) + and (cast('2000-08-16' as date) + interval 30 day) + and ss_store_sk = s_store_sk + group by s_store_sk) + , + sr as + (select s_store_sk, + sum(sr_return_amt) as returns, + sum(sr_net_loss) as profit_loss + from store_returns, + date_dim, + store + where sr_returned_date_sk = d_date_sk + and d_date between cast('2000-08-16' as date) + and (cast('2000-08-16' as date) + interval 30 day) + and sr_store_sk = s_store_sk + group by s_store_sk), + cs as + (select cs_call_center_sk, + sum(cs_ext_sales_price) as sales, + sum(cs_net_profit) as profit + from catalog_sales, + date_dim + where cs_sold_date_sk = d_date_sk + and d_date between cast('2000-08-16' as date) + and (cast('2000-08-16' as date) + interval 30 day) + group by cs_call_center_sk + ), + cr as + (select cr_call_center_sk, + sum(cr_return_amount) as returns, + sum(cr_net_loss) as profit_loss + from catalog_returns, + date_dim + where cr_returned_date_sk = d_date_sk + and d_date between cast('2000-08-16' as date) + and (cast('2000-08-16' as date) + interval 30 day) + group by cr_call_center_sk + ), + ws as + ( select wp_web_page_sk, + sum(ws_ext_sales_price) as sales, + sum(ws_net_profit) as profit + from web_sales, + date_dim, + web_page + where ws_sold_date_sk = d_date_sk + and d_date between cast('2000-08-16' as date) + and (cast('2000-08-16' as date) + interval 30 day) + and ws_web_page_sk = wp_web_page_sk + group by wp_web_page_sk), + wr as + (select wp_web_page_sk, + sum(wr_return_amt) as returns, + sum(wr_net_loss) as profit_loss + from web_returns, + date_dim, + web_page + where wr_returned_date_sk = d_date_sk + and d_date between cast('2000-08-16' as date) + and (cast('2000-08-16' as date) + interval 30 day) + and wr_web_page_sk = wp_web_page_sk + group by wp_web_page_sk) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , ss.s_store_sk as id + , sales + , coalesce(returns, 0) as returns + , (profit - coalesce(profit_loss,0)) as profit + from ss left join sr + on ss.s_store_sk = sr.s_store_sk + union all + select 'catalog channel' as channel + , cs_call_center_sk as id + , sales + , returns + , (profit - profit_loss) as profit + from cs + , cr + union all + select 'web channel' as channel + , ws.wp_web_page_sk as id + , sales + , coalesce(returns, 0) returns + , (profit - coalesce(profit_loss,0)) as profit + from ws left join wr + on ws.wp_web_page_sk = wr.wp_web_page_sk + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query78.sql b/tools/tpcds-tools/queries/sf10000/query78.sql new file mode 100644 index 000000000000000..1c34f237f09f885 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query78.sql @@ -0,0 +1,55 @@ +with ws as + (select d_year AS ws_sold_year, ws_item_sk, + ws_bill_customer_sk ws_customer_sk, + sum(ws_quantity) ws_qty, + sum(ws_wholesale_cost) ws_wc, + sum(ws_sales_price) ws_sp + from web_sales + left join web_returns on wr_order_number=ws_order_number and ws_item_sk=wr_item_sk + join date_dim on ws_sold_date_sk = d_date_sk + where wr_order_number is null + group by d_year, ws_item_sk, ws_bill_customer_sk + ), +cs as + (select d_year AS cs_sold_year, cs_item_sk, + cs_bill_customer_sk cs_customer_sk, + sum(cs_quantity) cs_qty, + sum(cs_wholesale_cost) cs_wc, + sum(cs_sales_price) cs_sp + from catalog_sales + left join catalog_returns on cr_order_number=cs_order_number and cs_item_sk=cr_item_sk + join date_dim on cs_sold_date_sk = d_date_sk + where cr_order_number is null + group by d_year, cs_item_sk, cs_bill_customer_sk + ), +ss as + (select d_year AS ss_sold_year, ss_item_sk, + ss_customer_sk, + sum(ss_quantity) ss_qty, + sum(ss_wholesale_cost) ss_wc, + sum(ss_sales_price) ss_sp + from store_sales + left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk + join date_dim on ss_sold_date_sk = d_date_sk + where sr_ticket_number is null + group by d_year, ss_item_sk, ss_customer_sk + ) + select +ss_customer_sk, +round(ss_qty/(coalesce(ws_qty,0)+coalesce(cs_qty,0)),2) ratio, +ss_qty store_qty, ss_wc store_wholesale_cost, ss_sp store_sales_price, +coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty, +coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost, +coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price +from ss +left join ws on (ws_sold_year=ss_sold_year and ws_item_sk=ss_item_sk and ws_customer_sk=ss_customer_sk) +left join cs on (cs_sold_year=ss_sold_year and cs_item_sk=ss_item_sk and cs_customer_sk=ss_customer_sk) +where (coalesce(ws_qty,0)>0 or coalesce(cs_qty, 0)>0) and ss_sold_year=2001 +order by + ss_customer_sk, + ss_qty desc, ss_wc desc, ss_sp desc, + other_chan_qty, + other_chan_wholesale_cost, + other_chan_sales_price, + ratio +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query79.sql b/tools/tpcds-tools/queries/sf10000/query79.sql new file mode 100644 index 000000000000000..444460723981425 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query79.sql @@ -0,0 +1,20 @@ +select + c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,store.s_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (household_demographics.hd_dep_count = 0 or household_demographics.hd_vehicle_count > 3) + and date_dim.d_dow = 1 + and date_dim.d_year in (1998,1998+1,1998+2) + and store.s_number_employees between 200 and 295 + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer + where ss_customer_sk = c_customer_sk + order by c_last_name,c_first_name,substr(s_city,1,30), profit +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query8.sql b/tools/tpcds-tools/queries/sf10000/query8.sql new file mode 100644 index 000000000000000..8909d5ba1d8b17b --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query8.sql @@ -0,0 +1,105 @@ +select s_store_name + ,sum(ss_net_profit) + from store_sales + ,date_dim + ,store, + (select ca_zip + from ( + SELECT substr(ca_zip,1,5) ca_zip + FROM customer_address + WHERE substr(ca_zip,1,5) IN ( + '47602','16704','35863','28577','83910','36201', + '58412','48162','28055','41419','80332', + '38607','77817','24891','16226','18410', + '21231','59345','13918','51089','20317', + '17167','54585','67881','78366','47770', + '18360','51717','73108','14440','21800', + '89338','45859','65501','34948','25973', + '73219','25333','17291','10374','18829', + '60736','82620','41351','52094','19326', + '25214','54207','40936','21814','79077', + '25178','75742','77454','30621','89193', + '27369','41232','48567','83041','71948', + '37119','68341','14073','16891','62878', + '49130','19833','24286','27700','40979', + '50412','81504','94835','84844','71954', + '39503','57649','18434','24987','12350', + '86379','27413','44529','98569','16515', + '27287','24255','21094','16005','56436', + '91110','68293','56455','54558','10298', + '83647','32754','27052','51766','19444', + '13869','45645','94791','57631','20712', + '37788','41807','46507','21727','71836', + '81070','50632','88086','63991','20244', + '31655','51782','29818','63792','68605', + '94898','36430','57025','20601','82080', + '33869','22728','35834','29086','92645', + '98584','98072','11652','78093','57553', + '43830','71144','53565','18700','90209', + '71256','38353','54364','28571','96560', + '57839','56355','50679','45266','84680', + '34306','34972','48530','30106','15371', + '92380','84247','92292','68852','13338', + '34594','82602','70073','98069','85066', + '47289','11686','98862','26217','47529', + '63294','51793','35926','24227','14196', + '24594','32489','99060','49472','43432', + '49211','14312','88137','47369','56877', + '20534','81755','15794','12318','21060', + '73134','41255','63073','81003','73873', + '66057','51184','51195','45676','92696', + '70450','90669','98338','25264','38919', + '59226','58581','60298','17895','19489', + '52301','80846','95464','68770','51634', + '19988','18367','18421','11618','67975', + '25494','41352','95430','15734','62585', + '97173','33773','10425','75675','53535', + '17879','41967','12197','67998','79658', + '59130','72592','14851','43933','68101', + '50636','25717','71286','24660','58058', + '72991','95042','15543','33122','69280', + '11912','59386','27642','65177','17672', + '33467','64592','36335','54010','18767', + '63193','42361','49254','33113','33159', + '36479','59080','11855','81963','31016', + '49140','29392','41836','32958','53163', + '13844','73146','23952','65148','93498', + '14530','46131','58454','13376','13378', + '83986','12320','17193','59852','46081', + '98533','52389','13086','68843','31013', + '13261','60560','13443','45533','83583', + '11489','58218','19753','22911','25115', + '86709','27156','32669','13123','51933', + '39214','41331','66943','14155','69998', + '49101','70070','35076','14242','73021', + '59494','15782','29752','37914','74686', + '83086','34473','15751','81084','49230', + '91894','60624','17819','28810','63180', + '56224','39459','55233','75752','43639', + '55349','86057','62361','50788','31830', + '58062','18218','85761','60083','45484', + '21204','90229','70041','41162','35390', + '16364','39500','68908','26689','52868', + '81335','40146','11340','61527','61794', + '71997','30415','59004','29450','58117', + '69952','33562','83833','27385','61860', + '96435','48333','23065','32961','84919', + '61997','99132','22815','56600','68730', + '48017','95694','32919','88217','27116', + '28239','58032','18884','16791','21343', + '97462','18569','75660','15475') + intersect + select ca_zip + from (SELECT substr(ca_zip,1,5) ca_zip,count(*) cnt + FROM customer_address, customer + WHERE ca_address_sk = c_current_addr_sk and + c_preferred_cust_flag='Y' + group by ca_zip + having count(*) > 10)A1)A2) V1 + where ss_store_sk = s_store_sk + and ss_sold_date_sk = d_date_sk + and d_qoy = 2 and d_year = 1998 + and (substr(s_zip,1,2) = substr(V1.ca_zip,1,2)) + group by s_store_name + order by s_store_name + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query80.sql b/tools/tpcds-tools/queries/sf10000/query80.sql new file mode 100644 index 000000000000000..12dcd06c46c1cbc --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query80.sql @@ -0,0 +1,93 @@ +with ssr as + (select s_store_id as store_id, + sum(ss_ext_sales_price) as sales, + sum(coalesce(sr_return_amt, 0)) as returns, + sum(ss_net_profit - coalesce(sr_net_loss, 0)) as profit + from store_sales left outer join store_returns on + (ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number), + date_dim, + store, + item, + promotion + where ss_sold_date_sk = d_date_sk + and d_date between cast('2002-08-06' as date) + and (cast('2002-08-06' as date) + interval 30 day) + and ss_store_sk = s_store_sk + and ss_item_sk = i_item_sk + and i_current_price > 50 + and ss_promo_sk = p_promo_sk + and p_channel_tv = 'N' + group by s_store_id) + , + csr as + (select cp_catalog_page_id as catalog_page_id, + sum(cs_ext_sales_price) as sales, + sum(coalesce(cr_return_amount, 0)) as returns, + sum(cs_net_profit - coalesce(cr_net_loss, 0)) as profit + from catalog_sales left outer join catalog_returns on + (cs_item_sk = cr_item_sk and cs_order_number = cr_order_number), + date_dim, + catalog_page, + item, + promotion + where cs_sold_date_sk = d_date_sk + and d_date between cast('2002-08-06' as date) + and (cast('2002-08-06' as date) + interval 30 day) + and cs_catalog_page_sk = cp_catalog_page_sk + and cs_item_sk = i_item_sk + and i_current_price > 50 + and cs_promo_sk = p_promo_sk + and p_channel_tv = 'N' +group by cp_catalog_page_id) + , + wsr as + (select web_site_id, + sum(ws_ext_sales_price) as sales, + sum(coalesce(wr_return_amt, 0)) as returns, + sum(ws_net_profit - coalesce(wr_net_loss, 0)) as profit + from web_sales left outer join web_returns on + (ws_item_sk = wr_item_sk and ws_order_number = wr_order_number), + date_dim, + web_site, + item, + promotion + where ws_sold_date_sk = d_date_sk + and d_date between cast('2002-08-06' as date) + and (cast('2002-08-06' as date) + interval 30 day) + and ws_web_site_sk = web_site_sk + and ws_item_sk = i_item_sk + and i_current_price > 50 + and ws_promo_sk = p_promo_sk + and p_channel_tv = 'N' +group by web_site_id) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , concat('store', store_id) as id + , sales + , returns + , profit + from ssr + union all + select 'catalog channel' as channel + , concat('catalog_page', catalog_page_id) as id + , sales + , returns + , profit + from csr + union all + select 'web channel' as channel + , concat('web_site', web_site_id) as id + , sales + , returns + , profit + from wsr + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query81.sql b/tools/tpcds-tools/queries/sf10000/query81.sql new file mode 100644 index 000000000000000..7fbc09767ddb127 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query81.sql @@ -0,0 +1,28 @@ +with customer_total_return as + (select cr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(cr_return_amt_inc_tax) as ctr_total_return + from catalog_returns + ,date_dim + ,customer_address + where cr_returned_date_sk = d_date_sk + and d_year =1998 + and cr_returning_addr_sk = ca_address_sk + group by cr_returning_customer_sk + ,ca_state ) + select c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'TX' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query82.sql b/tools/tpcds-tools/queries/sf10000/query82.sql new file mode 100644 index 000000000000000..2ac1a5299a55dce --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query82.sql @@ -0,0 +1,14 @@ +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, store_sales + where i_current_price between 49 and 49+30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('2001-01-28' as date) and (cast('2001-01-28' as date) + interval 60 day) + and i_manufact_id in (80,675,292,17) + and inv_quantity_on_hand between 100 and 500 + and ss_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query83.sql b/tools/tpcds-tools/queries/sf10000/query83.sql new file mode 100644 index 000000000000000..81e76c390630bc0 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query83.sql @@ -0,0 +1,64 @@ +with sr_items as + (select i_item_id item_id, + sum(sr_return_quantity) sr_item_qty + from store_returns, + item, + date_dim + where sr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2000-06-17','2000-08-22','2000-11-17'))) + and sr_returned_date_sk = d_date_sk + group by i_item_id), + cr_items as + (select i_item_id item_id, + sum(cr_return_quantity) cr_item_qty + from catalog_returns, + item, + date_dim + where cr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2000-06-17','2000-08-22','2000-11-17'))) + and cr_returned_date_sk = d_date_sk + group by i_item_id), + wr_items as + (select i_item_id item_id, + sum(wr_return_quantity) wr_item_qty + from web_returns, + item, + date_dim + where wr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2000-06-17','2000-08-22','2000-11-17'))) + and wr_returned_date_sk = d_date_sk + group by i_item_id) + select sr_items.item_id + ,sr_item_qty + ,sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev + ,cr_item_qty + ,cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev + ,wr_item_qty + ,wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev + ,(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 average + from sr_items + ,cr_items + ,wr_items + where sr_items.item_id=cr_items.item_id + and sr_items.item_id=wr_items.item_id + order by sr_items.item_id + ,sr_item_qty + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query84.sql b/tools/tpcds-tools/queries/sf10000/query84.sql new file mode 100644 index 000000000000000..ba0cd108aa39858 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query84.sql @@ -0,0 +1,18 @@ +select c_customer_id as customer_id + , concat(concat(coalesce(c_last_name,''), ','), coalesce(c_first_name,'')) as customername + from customer + ,customer_address + ,customer_demographics + ,household_demographics + ,income_band + ,store_returns + where ca_city = 'Hopewell' + and c_current_addr_sk = ca_address_sk + and ib_lower_bound >= 37855 + and ib_upper_bound <= 37855 + 50000 + and ib_income_band_sk = hd_income_band_sk + and cd_demo_sk = c_current_cdemo_sk + and hd_demo_sk = c_current_hdemo_sk + and sr_cdemo_sk = cd_demo_sk + order by c_customer_id + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query85.sql b/tools/tpcds-tools/queries/sf10000/query85.sql new file mode 100644 index 000000000000000..64ce0b40dc41e6d --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query85.sql @@ -0,0 +1,81 @@ +select substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) + from web_sales, web_returns, web_page, customer_demographics cd1, + customer_demographics cd2, customer_address, date_dim, reason + where ws_web_page_sk = wp_web_page_sk + and ws_item_sk = wr_item_sk + and ws_order_number = wr_order_number + and ws_sold_date_sk = d_date_sk and d_year = 2001 + and cd1.cd_demo_sk = wr_refunded_cdemo_sk + and cd2.cd_demo_sk = wr_returning_cdemo_sk + and ca_address_sk = wr_refunded_addr_sk + and r_reason_sk = wr_reason_sk + and + ( + ( + cd1.cd_marital_status = 'M' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = '4 yr Degree' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 100.00 and 150.00 + ) + or + ( + cd1.cd_marital_status = 'S' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'College' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 50.00 and 100.00 + ) + or + ( + cd1.cd_marital_status = 'D' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Secondary' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ca_country = 'United States' + and + ca_state in ('TX', 'VA', 'CA') + and ws_net_profit between 100 and 200 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('AR', 'NE', 'MO') + and ws_net_profit between 150 and 300 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('IA', 'MS', 'WA') + and ws_net_profit between 50 and 250 + ) + ) +group by r_reason_desc +order by substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query86.sql b/tools/tpcds-tools/queries/sf10000/query86.sql new file mode 100644 index 000000000000000..5334ea766ada13e --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query86.sql @@ -0,0 +1,23 @@ +select + sum(ws_net_paid) as total_sum + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ws_net_paid) desc) as rank_within_parent + from + web_sales + ,date_dim d1 + ,item + where + d1.d_month_seq between 1215 and 1215+11 + and d1.d_date_sk = ws_sold_date_sk + and i_item_sk = ws_item_sk + group by rollup(i_category,i_class) + order by + lochierarchy desc, + case when lochierarchy = 0 then i_category end, + rank_within_parent + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query87.sql b/tools/tpcds-tools/queries/sf10000/query87.sql new file mode 100644 index 000000000000000..7481b1dff171080 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query87.sql @@ -0,0 +1,20 @@ +select count(*) +from ((select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1221 and 1221+11) + except + (select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1221 and 1221+11) + except + (select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1221 and 1221+11) +) cool_cust +; diff --git a/tools/tpcds-tools/queries/sf10000/query88.sql b/tools/tpcds-tools/queries/sf10000/query88.sql new file mode 100644 index 000000000000000..422652669c1fdaf --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query88.sql @@ -0,0 +1,91 @@ +select * +from + (select count(*) h8_30_to_9 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 8 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s1, + (select count(*) h9_to_9_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s2, + (select count(*) h9_30_to_10 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s3, + (select count(*) h10_to_10_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s4, + (select count(*) h10_30_to_11 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s5, + (select count(*) h11_to_11_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s6, + (select count(*) h11_30_to_12 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s7, + (select count(*) h12_to_12_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 12 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2) or + (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) + and store.s_store_name = 'ese') s8 +; diff --git a/tools/tpcds-tools/queries/sf10000/query89.sql b/tools/tpcds-tools/queries/sf10000/query89.sql new file mode 100644 index 000000000000000..175af823b3ad8d5 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query89.sql @@ -0,0 +1,25 @@ +select * +from( +select i_category, i_class, i_brand, + s_store_name, s_company_name, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, s_store_name, s_company_name) + avg_monthly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + d_year in (2000) and + ((i_category in ('Home','Music','Books') and + i_class in ('glassware','classical','fiction') + ) + or (i_category in ('Jewelry','Sports','Women') and + i_class in ('semi-precious','baseball','dresses') + )) +group by i_category, i_class, i_brand, + s_store_name, s_company_name, d_moy) tmp1 +where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1 +order by sum_sales - avg_monthly_sales, s_store_name +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query9.sql b/tools/tpcds-tools/queries/sf10000/query9.sql new file mode 100644 index 000000000000000..d0b83d9c21b720a --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query9.sql @@ -0,0 +1,48 @@ +select case when (select count(*) + from store_sales + where ss_quantity between 1 and 20) > 98972190 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 1 and 20) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 1 and 20) end bucket1 , + case when (select count(*) + from store_sales + where ss_quantity between 21 and 40) > 160856845 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 21 and 40) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 21 and 40) end bucket2, + case when (select count(*) + from store_sales + where ss_quantity between 41 and 60) > 12733327 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 41 and 60) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 41 and 60) end bucket3, + case when (select count(*) + from store_sales + where ss_quantity between 61 and 80) > 96251173 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 61 and 80) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 61 and 80) end bucket4, + case when (select count(*) + from store_sales + where ss_quantity between 81 and 100) > 80049606 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 81 and 100) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 81 and 100) end bucket5 +from reason +where r_reason_sk = 1 +; diff --git a/tools/tpcds-tools/queries/sf10000/query90.sql b/tools/tpcds-tools/queries/sf10000/query90.sql new file mode 100644 index 000000000000000..08118f64955cd0b --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query90.sql @@ -0,0 +1,19 @@ +select cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio + from ( select count(*) amc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 9 and 9+1 + and household_demographics.hd_dep_count = 3 + and web_page.wp_char_count between 5000 and 5200) at, + ( select count(*) pmc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 16 and 16+1 + and household_demographics.hd_dep_count = 3 + and web_page.wp_char_count between 5000 and 5200) pt + order by am_pm_ratio + limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query91.sql b/tools/tpcds-tools/queries/sf10000/query91.sql new file mode 100644 index 000000000000000..b8354f62c69e346 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query91.sql @@ -0,0 +1,28 @@ +select + cc_call_center_id Call_Center, + cc_name Call_Center_Name, + cc_manager Manager, + sum(cr_net_loss) Returns_Loss +from + call_center, + catalog_returns, + date_dim, + customer, + customer_address, + customer_demographics, + household_demographics +where + cr_call_center_sk = cc_call_center_sk +and cr_returned_date_sk = d_date_sk +and cr_returning_customer_sk= c_customer_sk +and cd_demo_sk = c_current_cdemo_sk +and hd_demo_sk = c_current_hdemo_sk +and ca_address_sk = c_current_addr_sk +and d_year = 2000 +and d_moy = 12 +and ( (cd_marital_status = 'M' and cd_education_status = 'Unknown') + or(cd_marital_status = 'W' and cd_education_status = 'Advanced Degree')) +and hd_buy_potential like 'Unknown%' +and ca_gmt_offset = -7 +group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status +order by sum(cr_net_loss) desc; diff --git a/tools/tpcds-tools/queries/sf10000/query92.sql b/tools/tpcds-tools/queries/sf10000/query92.sql new file mode 100644 index 000000000000000..a45c64ec2f08532 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query92.sql @@ -0,0 +1,27 @@ +select + sum(ws_ext_discount_amt) as "Excess Discount Amount" +from + web_sales + ,item + ,date_dim +where +i_manufact_id = 356 +and i_item_sk = ws_item_sk +and d_date between '2001-03-12' and + (cast('2001-03-12' as date) + interval 90 day) +and d_date_sk = ws_sold_date_sk +and ws_ext_discount_amt + > ( + SELECT + 1.3 * avg(ws_ext_discount_amt) + FROM + web_sales + ,date_dim + WHERE + ws_item_sk = i_item_sk + and d_date between '2001-03-12' and + (cast('2001-03-12' as date) + interval 90 day) + and d_date_sk = ws_sold_date_sk + ) +order by sum(ws_ext_discount_amt) +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query93.sql b/tools/tpcds-tools/queries/sf10000/query93.sql new file mode 100644 index 000000000000000..8351581a919c53e --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query93.sql @@ -0,0 +1,15 @@ +select ss_customer_sk + ,sum(act_sales) sumsales + from (select ss_item_sk + ,ss_ticket_number + ,ss_customer_sk + ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price + else (ss_quantity*ss_sales_price) end act_sales + from store_sales left outer join store_returns on (sr_item_sk = ss_item_sk + and sr_ticket_number = ss_ticket_number) + ,reason + where sr_reason_sk = r_reason_sk + and r_reason_desc = 'reason 66') t + group by ss_customer_sk + order by sumsales, ss_customer_sk +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query94.sql b/tools/tpcds-tools/queries/sf10000/query94.sql new file mode 100644 index 000000000000000..877a8b055c5593f --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query94.sql @@ -0,0 +1,26 @@ +select + count(distinct ws_order_number) as "order count" + ,sum(ws_ext_ship_cost) as "total shipping cost" + ,sum(ws_net_profit) as "total net profit" +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '1999-4-01' and + (cast('1999-4-01' as date) + interval 60 day) +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'NE' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and exists (select * + from web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) +and not exists(select * + from web_returns wr1 + where ws1.ws_order_number = wr1.wr_order_number) +order by count(distinct ws_order_number) +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query95.sql b/tools/tpcds-tools/queries/sf10000/query95.sql new file mode 100644 index 000000000000000..62a7d1f405367c3 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query95.sql @@ -0,0 +1,29 @@ +with ws_wh as +(select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2 + from web_sales ws1,web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) + select + count(distinct ws_order_number) as "order count" + ,sum(ws_ext_ship_cost) as "total shipping cost" + ,sum(ws_net_profit) as "total net profit" +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '2002-4-01' and + (cast('2002-4-01' as date) + interval 60 day) +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'AL' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and ws1.ws_order_number in (select ws_order_number + from ws_wh) +and ws1.ws_order_number in (select wr_order_number + from web_returns,ws_wh + where wr_order_number = ws_wh.ws_order_number) +order by count(distinct ws_order_number) +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query96.sql b/tools/tpcds-tools/queries/sf10000/query96.sql new file mode 100644 index 000000000000000..eab40ea5500c9b6 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query96.sql @@ -0,0 +1,13 @@ +select count(*) +from store_sales + ,household_demographics + ,time_dim, store +where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 16 + and time_dim.t_minute >= 30 + and household_demographics.hd_dep_count = 6 + and store.s_store_name = 'ese' +order by count(*) +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query97.sql b/tools/tpcds-tools/queries/sf10000/query97.sql new file mode 100644 index 000000000000000..5ab79672fdf15d6 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query97.sql @@ -0,0 +1,22 @@ +with ssci as ( +select ss_customer_sk customer_sk + ,ss_item_sk item_sk +from store_sales,date_dim +where ss_sold_date_sk = d_date_sk + and d_month_seq between 1190 and 1190 + 11 +group by ss_customer_sk + ,ss_item_sk), +csci as( + select cs_bill_customer_sk customer_sk + ,cs_item_sk item_sk +from catalog_sales,date_dim +where cs_sold_date_sk = d_date_sk + and d_month_seq between 1190 and 1190 + 11 +group by cs_bill_customer_sk + ,cs_item_sk) + select sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only + ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only + ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog +from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk + and ssci.item_sk = csci.item_sk) +limit 100; diff --git a/tools/tpcds-tools/queries/sf10000/query98.sql b/tools/tpcds-tools/queries/sf10000/query98.sql new file mode 100644 index 000000000000000..0003497e22aa65e --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query98.sql @@ -0,0 +1,30 @@ +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ss_ext_sales_price) as itemrevenue + ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over + (partition by i_class) as revenueratio +from + store_sales + ,item + ,date_dim +where + ss_item_sk = i_item_sk + and i_category in ('Home', 'Sports', 'Men') + and ss_sold_date_sk = d_date_sk + and d_date between cast('2002-01-05' as date) + and (cast('2002-01-05' as date) + interval 30 day) +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio; diff --git a/tools/tpcds-tools/queries/sf10000/query99.sql b/tools/tpcds-tools/queries/sf10000/query99.sql new file mode 100644 index 000000000000000..5faeed321c6a2d2 --- /dev/null +++ b/tools/tpcds-tools/queries/sf10000/query99.sql @@ -0,0 +1,32 @@ +select + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and + (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and + (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and + (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 120) then 1 else 0 end) as ">120 days" +from + catalog_sales + ,warehouse + ,ship_mode + ,call_center + ,date_dim +where + d_month_seq between 1178 and 1178 + 11 +and cs_ship_date_sk = d_date_sk +and cs_warehouse_sk = w_warehouse_sk +and cs_ship_mode_sk = sm_ship_mode_sk +and cs_call_center_sk = cc_call_center_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name +limit 100;