Skip to content

Commit

Permalink
1. hide password and other sensitive information in log and audit log
Browse files Browse the repository at this point in the history
2. add 2 new proc '/current_queries' and '/current_backend_instances' to monitor the current running queries.
3. add a manual compaction api on Backend to trigger cumulative or base compaction manually.
4. add Frontend config 'max_bytes_per_broker_scanner' to limit to bytes per one broker scanner. This is to limit the memory cost of a single broker load job
5. add Frontend config 'max_unfinished_load_job' to limit load job number: if number of running load jobs exceed the limit, no more load job is allowed to be submmitted.
6. a log of bug fixed
  • Loading branch information
lide committed Sep 19, 2018
1 parent 050cbf2 commit bea10e4
Show file tree
Hide file tree
Showing 101 changed files with 2,479 additions and 404 deletions.
10 changes: 8 additions & 2 deletions be/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -303,7 +303,7 @@ SET(CXX_FLAGS_TSAN "${CXX_GCC_FLAGS} -O1 -ggdb3 -fsanitize=thread -DTHREAD_SANIT
# Set compile flags based on the build type.
if ("${CMAKE_BUILD_TYPE}" STREQUAL "DEBUG")
SET(CMAKE_CXX_FLAGS ${CXX_FLAGS_DEBUG})
elseif ("${CMAKE_BUILD_TYPE}" STREQUAL "RELEASE")
elseif ("${CMAKE_BUILD_TYPE}" STREQUAL "RELEASE" OR "${CMAKE_BUILD_TYPE}" STREQUAL "BCC")
SET(CMAKE_CXX_FLAGS ${CXX_FLAGS_RELEASE})
elseif ("${CMAKE_BUILD_TYPE}" STREQUAL "ASAN")
SET(CMAKE_CXX_FLAGS "${CXX_FLAGS_ASAN}")
Expand Down Expand Up @@ -485,8 +485,14 @@ set(PALO_LINK_LIBS ${PALO_LINK_LIBS}
-static-libgcc
)

if ("${CMAKE_BUILD_TYPE}" STREQUAL "BCC")
set(PALO_LINK_LIBS ${PALO_LINK_LIBS}
-Wl,--dynamic-linker=/lib64/ld-linux-x86-64.so.2
)
endif()

# Add sanitize static link flags or tcmalloc
if ("${CMAKE_BUILD_TYPE}" STREQUAL "DEBUG" OR "${CMAKE_BUILD_TYPE}" STREQUAL "RELEASE")
if ("${CMAKE_BUILD_TYPE}" STREQUAL "DEBUG" OR "${CMAKE_BUILD_TYPE}" STREQUAL "RELEASE" OR "${CMAKE_BUILD_TYPE}" STREQUAL "BCC")
set(PALO_LINK_LIBS ${PALO_LINK_LIBS} tcmalloc)
elseif ("${CMAKE_BUILD_TYPE}" STREQUAL "ASAN")
set(PALO_LINK_LIBS ${PALO_LINK_LIBS} -static-libasan)
Expand Down
1 change: 1 addition & 0 deletions be/src/exec/cross_join_node.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@ void CrossJoinNode::init_get_next(TupleRow* first_left_row) {
Status CrossJoinNode::get_next(RuntimeState* state, RowBatch* output_batch, bool* eos) {
// RETURN_IF_ERROR(exec_debug_action(TExecNodePhase::GETNEXT, state));
RETURN_IF_CANCELLED(state);
*eos = false;
// TOOD(zhaochun)
// RETURN_IF_ERROR(state->check_query_state());
SCOPED_TIMER(_runtime_profile->total_time_counter());
Expand Down
5 changes: 5 additions & 0 deletions be/src/exec/exchange_node.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -116,8 +116,10 @@ Status ExchangeNode::fill_input_row_batch(RuntimeState* state) {
DCHECK(!_is_merging);
Status ret_status;
{
state->set_query_state_for_wait();
// SCOPED_TIMER(state->total_network_receive_timer());
ret_status = _stream_recvr->get_batch(&_input_batch);
state->set_query_state_for_running();
}
VLOG_FILE << "exch: has batch=" << (_input_batch == NULL ? "false" : "true")
<< " #rows=" << (_input_batch != NULL ? _input_batch->num_rows() : 0)
Expand Down Expand Up @@ -211,7 +213,10 @@ Status ExchangeNode::get_next_merging(RuntimeState* state, RowBatch* output_batc
RETURN_IF_CANCELLED(state);
// RETURN_IF_ERROR(QueryMaintenance(state));
RETURN_IF_ERROR(state->check_query_state());

state->set_query_state_for_wait();
RETURN_IF_ERROR(_stream_recvr->get_next(output_batch, eos));
state->set_query_state_for_running();

while ((_num_rows_skipped < _offset)) {
_num_rows_skipped += output_batch->num_rows();
Expand Down
1 change: 1 addition & 0 deletions be/src/exec/hash_join_node.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ HashJoinNode::HashJoinNode(
ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) :
ExecNode(pool, tnode, descs),
_join_op(tnode.hash_join_node.join_op),
_probe_eos(false),
_codegen_process_build_batch_fn(NULL),
_process_build_batch_fn(NULL),
_process_probe_batch_fn(NULL),
Expand Down
3 changes: 2 additions & 1 deletion be/src/exec/olap_scan_node.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -238,8 +238,8 @@ Status OlapScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eo
// wait for batch from queue
RowBatch* materialized_batch = NULL;
{
state->set_query_state_for_wait();
boost::unique_lock<boost::mutex> l(_row_batches_lock);

while (_materialized_row_batches.empty() && !_transfer_done) {
if (state->is_cancelled()) {
_transfer_done = true;
Expand All @@ -253,6 +253,7 @@ Status OlapScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eo
DCHECK(materialized_batch != NULL);
_materialized_row_batches.pop_front();
}
state->set_query_state_for_running();
}

// return batch
Expand Down
1 change: 0 additions & 1 deletion be/src/exprs/aggregate_functions.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1050,7 +1050,6 @@ int64_t AggregateFunctions::hll_algorithm(const palo_udf::StringVal& src) {

harmonic_mean = 1.0f / harmonic_mean;
double estimate = alpha * num_streams * num_streams * harmonic_mean;
double tmp = 0.f;
// according to HerperLogLog current correction, if E is cardinal
// E =< num_streams * 2.5 , LC has higher accuracy.
// num_streams * 2.5 < E , HerperLogLog has higher accuracy.
Expand Down
1 change: 1 addition & 0 deletions be/src/http/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ add_library(Webserver STATIC
monitor_action.cpp
default_path_handlers.cpp
ev_http_server.cpp
action/compaction_action.cpp
action/mini_load.cpp
action/health_action.cpp
action/checksum_action.cpp
Expand Down
98 changes: 98 additions & 0 deletions be/src/http/action/compaction_action.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved

// Licensed 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 "http/action/compaction_action.h"

#include <string>
#include <sstream>

#include <boost/algorithm/string.hpp>
#include "boost/lexical_cast.hpp"

#include "agent/cgroups_mgr.h"
#include "http/http_channel.h"
#include "http/http_headers.h"
#include "http/http_request.h"
#include "http/http_response.h"
#include "http/http_status.h"
#include "olap/base_compaction.h"
#include "olap/cumulative_compaction.h"
#include "olap/olap_engine.h"

namespace palo {

//example:
// http://host:port/api/compaction?tablet_id=10001&schema_hash=10001&compaction_type=base_compaction
// http://host:port/api/compaction?tablet_id=10001&schema_hash=10001&compaction_type=cumulative_compaction
const std::string TABLET_ID = "tablet_id";
const std::string SCHEMA_HASH = "schema_hash";
const std::string COMPACTION_TYPE = "compaction_type";

void CompactionAction::handle(HttpRequest *req) {
LOG(INFO) << "accept one request " << req->debug_string();

// add tid to cgroup in order to limit read bandwidth
CgroupsMgr::apply_system_cgroup();
// Get tablet id
const std::string tablet_id_str = req->param(TABLET_ID);
if (tablet_id_str.empty()) {
std::string error_msg = std::string(
"parameter " + TABLET_ID + " not specified in url.");

HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, error_msg);
return;
}

// Get schema hash
const std::string schema_hash_str = req->param(SCHEMA_HASH);
if (schema_hash_str.empty()) {
std::string error_msg = std::string(
"parameter " + SCHEMA_HASH + " not specified in url.");
HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, error_msg);
return;
}

// valid str format
int64_t tablet_id;
int32_t schema_hash;
try {
tablet_id = boost::lexical_cast<int64_t>(tablet_id_str);
schema_hash = boost::lexical_cast<int64_t>(schema_hash_str);
} catch (boost::bad_lexical_cast& e) {
std::string error_msg = std::string("param format is invalid: ") + std::string(e.what());
HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, error_msg);
return;
}

// Get compaction type
const std::string& compaction_type = req->param(COMPACTION_TYPE);
if (!boost::iequals(compaction_type, "base_compaction")
&& !boost::iequals(compaction_type, "cumulative_compaction")) {
std::string error_msg = std::string(
"parameter " + COMPACTION_TYPE + " not specified in url.");
HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, error_msg);
return;
}

TableInfo tablet_info(tablet_id, schema_hash);
if (boost::iequals(compaction_type, "base_compaction")) {
OLAPEngine::get_instance()->add_tablet_to_base_compaction_queue(tablet_info);
} else {
OLAPEngine::get_instance()->add_tablet_to_cumulative_compaction_queue(tablet_info);
}
HttpChannel::send_reply(req, "succeed add compaction to queue");
}

} // end namespace palo
32 changes: 32 additions & 0 deletions be/src/http/action/compaction_action.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved

// Licensed 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.

#ifndef BDG_PALO_BE_SRC_HTTP_COMPACTION_ACTION_H
#define BDG_PALO_BE_SRC_HTTP_COMPACTION_ACTION_H

#include <boost/scoped_ptr.hpp>

#include "http/http_handler.h"

namespace palo {

class CompactionAction : public HttpHandler {
public:
void handle(HttpRequest *req) override;
}; // end class ChecksumAction

} // end namespace palo

#endif // BDG_PALO_BE_SRC_HTTP_COMPACTION_ACTION_H
2 changes: 2 additions & 0 deletions be/src/olap/base_compaction.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -186,6 +186,8 @@ OLAPStatus BaseCompaction::run() {
_table->set_base_compaction_status(BASE_COMPACTION_WAITING, -1);
_release_base_compaction_lock();

LOG(INFO) << "succeed to do base compaction. table=" << _table->full_name() << ", "
<< "base_version=" << _new_base_version.first << "-" << _new_base_version.second;
return OLAP_SUCCESS;
}

Expand Down
91 changes: 65 additions & 26 deletions be/src/olap/olap_engine.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -891,37 +891,45 @@ OLAPStatus OLAPEngine::report_all_tablets_info(
return OLAP_SUCCESS;
}

bool OLAPEngine::_can_do_compaction(SmartOLAPTable table) {
// 如果table正在做schema change,则通过选路判断数据是否转换完成
// 如果选路成功,则转换完成,可以进行BE
// 如果选路失败,则转换未完成,不能进行BE
table->obtain_header_rdlock();
const FileVersionMessage* latest_version = table->latest_version();
if (latest_version == NULL) {
table->release_header_lock();
return false;
}

if (table->is_schema_changing()) {
Version test_version = Version(0, latest_version->end_version());
vector<Version> path_versions;
if (OLAP_SUCCESS != table->select_versions_to_span(test_version, &path_versions)) {
table->release_header_lock();
return false;
}
}
table->release_header_lock();

return true;
}

void OLAPEngine::start_clean_fd_cache() {
OLAP_LOG_TRACE("start clean file descritpor cache");
_file_descriptor_lru_cache->prune();
OLAP_LOG_TRACE("end clean file descritpor cache");
}

void OLAPEngine::start_base_compaction(string* last_base_compaction_fs, TTabletId* last_base_compaction_tablet_id) {

{
std::lock_guard<std::mutex> l(_base_compaction_queue_lock);
if (!_base_compaction_tablet_queue.empty()) {
TableInfo& tablet_info = _base_compaction_tablet_queue.front();
_tablet_map_lock.rdlock();
SmartOLAPTable table = OLAPEngine::get_instance()->get_table(
tablet_info.tablet_id, tablet_info.schema_hash);
_tablet_map_lock.unlock();
_base_compaction_tablet_queue.pop();
if (table == nullptr) {
return;
}
if (!table->is_loaded() || !table->can_do_compaction()) {
return;
}
BaseCompaction base_compaction;
OLAPStatus res = base_compaction.init(table, true);
if (res != OLAP_SUCCESS) {
LOG(WARNING) << "failed to init base compaction. table=" << table->full_name();
return;
}

res = base_compaction.run();
if (res != OLAP_SUCCESS) {
LOG(WARNING) << "failed to do base compaction. table=" << table->full_name();
return;
}
return;
}
}

uint64_t base_compaction_start_hour = config::base_compaction_start_hour;
uint64_t base_compaction_end_hour = config::base_compaction_end_hour;
time_t current_time = time(NULL);
Expand Down Expand Up @@ -974,7 +982,7 @@ void OLAPEngine::start_base_compaction(string* last_base_compaction_fs, TTabletI
}

// 跳过正在做schema change的tablet
if (!_can_do_compaction(j)) {
if (!j->can_do_compaction()) {
OLAP_LOG_DEBUG("skip tablet, it is schema changing. [tablet=%s]",
j->full_name().c_str());
continue;
Expand Down Expand Up @@ -1060,6 +1068,37 @@ void OLAPEngine::_select_candidate() {
}

void OLAPEngine::start_cumulative_priority() {
{
std::lock_guard<std::mutex> l(_cumulative_compaction_queue_lock);
if (!_cumulative_compaction_tablet_queue.empty()) {
TableInfo& tablet_info = _cumulative_compaction_tablet_queue.front();
_tablet_map_lock.rdlock();
SmartOLAPTable table = OLAPEngine::get_instance()->get_table(
tablet_info.tablet_id, tablet_info.schema_hash);
_tablet_map_lock.unlock();
_cumulative_compaction_tablet_queue.pop();
if (table == nullptr) {
return;
}
if (!table->is_loaded() || !table->can_do_compaction()) {
return;
}
CumulativeCompaction cumulative_compaction;
OLAPStatus res = cumulative_compaction.init(table);
if (res != OLAP_SUCCESS) {
LOG(WARNING) << "failed to init cumulative compaction. table=" << table->full_name();
return;
}

res = cumulative_compaction.run();
if (res != OLAP_SUCCESS) {
LOG(WARNING) << "failed to do cumulative compaction. table=" << table->full_name();
return;
}
return;
}
}

_tablet_map_lock.rdlock();
_fs_task_mutex.lock();

Expand Down Expand Up @@ -1102,7 +1141,7 @@ void OLAPEngine::start_cumulative_priority() {
}

for (SmartOLAPTable j : i->second.table_arr) {
if (!_can_do_compaction(j)) {
if (!j->can_do_compaction()) {
OLAP_LOG_DEBUG("skip tablet, it is schema changing. [tablet=%s]",
j->full_name().c_str());
continue;
Expand Down
Loading

0 comments on commit bea10e4

Please sign in to comment.