Skip to content

Commit

Permalink
Merge branch 'master' into nereids-sql-cache
Browse files Browse the repository at this point in the history
  • Loading branch information
htyoung authored Nov 29, 2024
2 parents 81b2f89 + 851d780 commit ed7b477
Show file tree
Hide file tree
Showing 14 changed files with 29 additions and 30 deletions.
2 changes: 2 additions & 0 deletions be/src/exprs/bloom_filter_func.h
Original file line number Diff line number Diff line change
Expand Up @@ -233,6 +233,8 @@ class BloomFilterFuncBase : public RuntimeFilterFuncBase {
uint16_t* offsets, int number,
bool is_parse_column) = 0;

bool inited() const { return _inited; }

private:
void _limit_length() {
if (_runtime_bloom_filter_min_size > 0) {
Expand Down
7 changes: 7 additions & 0 deletions be/src/exprs/runtime_filter.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1283,6 +1283,13 @@ PrimitiveType IRuntimeFilter::column_type() const {

void IRuntimeFilter::signal() {
DCHECK(is_consumer());

if (!_wrapper->is_ignored() && _wrapper->is_bloomfilter() &&
!_wrapper->get_bloomfilter()->inited()) {
throw Exception(ErrorCode::INTERNAL_ERROR, "bf not inited and not ignored, rf: {}",
debug_string());
}

COUNTER_SET(_wait_timer, int64_t((MonotonicMillis() - registration_time_) * NANOS_PER_MILLIS));
_rf_state_atomic.store(RuntimeFilterState::READY);
if (!_filter_timer.empty()) {
Expand Down
12 changes: 1 addition & 11 deletions be/src/vec/columns/column.h
Original file line number Diff line number Diff line change
Expand Up @@ -565,10 +565,6 @@ class IColumn : public COW<IColumn> {
/// It's true for ColumnNullable, can be true or false for ColumnConst, etc.
virtual bool is_concrete_nullable() const { return false; }

virtual bool is_bitmap() const { return false; }

virtual bool is_hll() const { return false; }

// true if column has null element
virtual bool has_null() const { return false; }

Expand Down Expand Up @@ -607,19 +603,13 @@ class IColumn : public COW<IColumn> {
/// Checks only @sample_ratio ratio of rows.
virtual double get_ratio_of_default_rows(double sample_ratio = 1.0) const { return 0.0; }

/// Column is ColumnVector of numbers or ColumnConst of it. Note that Nullable columns are not numeric.
/// Implies is_fixed_and_contiguous.
virtual bool is_numeric() const { return false; }

// Column is ColumnString/ColumnArray/ColumnMap or other variable length column at every row
virtual bool is_variable_length() const { return false; }

virtual bool is_column_string() const { return false; }

virtual bool is_column_string64() const { return false; }

virtual bool is_column_decimal() const { return false; }

virtual bool is_column_dictionary() const { return false; }

virtual bool is_column_array() const { return false; }
Expand Down Expand Up @@ -752,4 +742,4 @@ struct ColumnPtrWrapper {

ColumnPtrWrapper(vectorized::ColumnPtr col) : column_ptr(std::move(col)) {}
};
} // namespace doris
} // namespace doris
5 changes: 0 additions & 5 deletions be/src/vec/columns/column_complex.h
Original file line number Diff line number Diff line change
Expand Up @@ -48,11 +48,6 @@ class ColumnComplexType final : public COWHelper<IColumn, ColumnComplexType<T>>
using value_type = T;
using Container = std::vector<value_type>;

bool is_numeric() const override { return false; }

bool is_bitmap() const override { return std::is_same_v<T, BitmapValue>; }
bool is_hll() const override { return std::is_same_v<T, HyperLogLog>; }

size_t size() const override { return data.size(); }

StringRef get_data_at(size_t n) const override {
Expand Down
1 change: 0 additions & 1 deletion be/src/vec/columns/column_const.h
Original file line number Diff line number Diff line change
Expand Up @@ -253,7 +253,6 @@ class ColumnConst final : public COWHelper<IColumn, ColumnConst> {
// ColumnConst is not nullable, but may be concrete nullable.
bool is_concrete_nullable() const override { return is_column_nullable(*data); }
bool only_null() const override { return data->is_null_at(0); }
bool is_numeric() const override { return data->is_numeric(); }
StringRef get_raw_data() const override { return data->get_raw_data(); }

/// Not part of the common interface.
Expand Down
3 changes: 0 additions & 3 deletions be/src/vec/columns/column_decimal.h
Original file line number Diff line number Diff line change
Expand Up @@ -105,9 +105,6 @@ class ColumnDecimal final : public COWHelper<IColumn, ColumnDecimal<T>> {
public:
std::string get_name() const override { return TypeName<T>::get(); }

bool is_numeric() const override { return false; }
bool is_column_decimal() const override { return true; }

size_t size() const override { return data.size(); }
size_t byte_size() const override { return data.size() * sizeof(data[0]); }
size_t allocated_bytes() const override { return data.allocated_bytes(); }
Expand Down
3 changes: 0 additions & 3 deletions be/src/vec/columns/column_nullable.h
Original file line number Diff line number Diff line change
Expand Up @@ -328,9 +328,6 @@ class ColumnNullable final : public COWHelper<IColumn, ColumnNullable>, public N

bool is_nullable() const override { return true; }
bool is_concrete_nullable() const override { return true; }
bool is_bitmap() const override { return get_nested_column().is_bitmap(); }
bool is_hll() const override { return get_nested_column().is_hll(); }
bool is_column_decimal() const override { return get_nested_column().is_column_decimal(); }
bool is_column_string() const override { return get_nested_column().is_column_string(); }
bool is_column_array() const override { return get_nested_column().is_column_array(); }
bool is_column_map() const override { return get_nested_column().is_column_map(); }
Expand Down
2 changes: 0 additions & 2 deletions be/src/vec/columns/column_vector.h
Original file line number Diff line number Diff line change
Expand Up @@ -152,8 +152,6 @@ class ColumnVector final : public COWHelper<IColumn, ColumnVector<T>> {
ColumnVector(std::initializer_list<T> il) : data {il} {}

public:
bool is_numeric() const override { return IsNumber<T>; }

size_t size() const override { return data.size(); }

StringRef get_data_at(size_t n) const override {
Expand Down
2 changes: 0 additions & 2 deletions be/src/vec/columns/predicate_column.h
Original file line number Diff line number Diff line change
Expand Up @@ -102,8 +102,6 @@ class PredicateColumnType final : public COWHelper<IColumn, PredicateColumnType<
using value_type = T;
using Container = PaddedPODArray<value_type>;

bool is_numeric() const override { return false; }

size_t size() const override { return data.size(); }

StringRef get_data_at(size_t n) const override {
Expand Down
1 change: 1 addition & 0 deletions be/src/vec/data_types/data_type.h
Original file line number Diff line number Diff line change
Expand Up @@ -289,6 +289,7 @@ struct WhichDataType {
bool is_aggregate_function() const { return idx == TypeIndex::AggregateFunction; }
bool is_variant_type() const { return idx == TypeIndex::VARIANT; }
bool is_simple() const { return is_int() || is_uint() || is_float() || is_string(); }
bool is_num_can_compare() const { return is_int_or_uint() || is_float() || is_ip(); }
};

/// IDataType helpers (alternative for IDataType virtual methods with single point of truth)
Expand Down
6 changes: 3 additions & 3 deletions be/src/vec/functions/functions_comparison.h
Original file line number Diff line number Diff line change
Expand Up @@ -637,8 +637,8 @@ class FunctionComparison : public IFunction {
WhichDataType which_left {left_type};
WhichDataType which_right {right_type};

const bool left_is_num = col_left_untyped->is_numeric();
const bool right_is_num = col_right_untyped->is_numeric();
const bool left_is_num_can_compare = which_left.is_num_can_compare();
const bool right_is_num_can_compare = which_right.is_num_can_compare();

const bool left_is_string = which_left.is_string_or_fixed_string();
const bool right_is_string = which_right.is_string_or_fixed_string();
Expand All @@ -648,7 +648,7 @@ class FunctionComparison : public IFunction {
// bool date_and_datetime = (left_type != right_type) && which_left.is_date_or_datetime() &&
// which_right.is_date_or_datetime();

if (left_is_num && right_is_num) {
if (left_is_num_can_compare && right_is_num_can_compare) {
if (!(execute_num_left_type<UInt8>(block, result, col_left_untyped,
col_right_untyped) ||
execute_num_left_type<UInt16>(block, result, col_left_untyped,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -342,6 +342,11 @@ public String removeLastDBOfCatalog(String catalog) {
return lastDBOfCatalog.get(catalog);
}

// Used by COM_RESET_CONNECTION
public void clearLastDBOfCatalog() {
lastDBOfCatalog.clear();
}

public void setNotEvalNondeterministicFunction(boolean notEvalNondeterministicFunction) {
this.notEvalNondeterministicFunction = notEvalNondeterministicFunction;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import org.apache.doris.common.util.SqlUtils;
import org.apache.doris.common.util.Util;
import org.apache.doris.datasource.CatalogIf;
import org.apache.doris.datasource.InternalCatalog;
import org.apache.doris.metric.MetricRepo;
import org.apache.doris.mysql.MysqlChannel;
import org.apache.doris.mysql.MysqlPacket;
Expand Down Expand Up @@ -198,6 +199,12 @@ protected void handleDebug() {
ctx.getState().setOk();
}

protected void handleResetConnection() {
ctx.changeDefaultCatalog(InternalCatalog.INTERNAL_CATALOG_NAME);
ctx.clearLastDBOfCatalog();
ctx.getState().setOk();
}

protected void handleStmtReset() {
ctx.getState().setOk();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -264,6 +264,9 @@ private void dispatch() throws IOException {
case COM_SET_OPTION:
handleSetOption();
break;
case COM_RESET_CONNECTION:
handleResetConnection();
break;
default:
ctx.getState().setError(ErrorCode.ERR_UNKNOWN_COM_ERROR, "Unsupported command(" + command + ")");
LOG.warn("Unsupported command(" + command + ")");
Expand Down

0 comments on commit ed7b477

Please sign in to comment.