Skip to content

Commit

Permalink
remove excessive logging, remove json from buffer manager
Browse files Browse the repository at this point in the history
  • Loading branch information
andyfengHKU authored and ray6080 committed Nov 27, 2022
1 parent c6d1dc0 commit cb099e8
Show file tree
Hide file tree
Showing 38 changed files with 218 additions and 281 deletions.
2 changes: 1 addition & 1 deletion README.md
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ Please refer to [our website](https://kuzudb.com/) for installation and usage in
## Development
For development, Kùzu requires Cmake(>=3.11), Python 3, and a compiler that supports `C++20`.

### Bazel build
### Build

- To do a full clean build
- `make clean`
Expand Down
6 changes: 3 additions & 3 deletions src/binder/bind/bind_reading_clause.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -9,12 +9,12 @@ unique_ptr<BoundReadingClause> Binder::bindReadingClause(const ReadingClause& re
switch (readingClause.getClauseType()) {
case ClauseType::MATCH: {
return bindMatchClause((MatchClause&)readingClause);
} break;
}
case ClauseType::UNWIND: {
return bindUnwindClause((UnwindClause&)readingClause);
} break;
}
default:
assert(false);
throw NotImplementedException("bindReadingClause().");
}
}

Expand Down
2 changes: 1 addition & 1 deletion src/binder/bind/bind_updating_clause.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ unique_ptr<BoundUpdatingClause> Binder::bindUpdatingClause(const UpdatingClause&
return bindDeleteClause(updatingClause);
}
default:
assert(false);
throw NotImplementedException("bindUpdatingClause().");
}
}

Expand Down
17 changes: 9 additions & 8 deletions src/binder/expression_binder.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -42,8 +42,9 @@ shared_ptr<Expression> ExpressionBinder::bindExpression(const ParsedExpression&
expression = bindVariableExpression(parsedExpression);
} else if (EXISTENTIAL_SUBQUERY == expressionType) {
expression = bindExistentialSubqueryExpression(parsedExpression);
} else if (!expression) {
assert(false);
} else {
throw NotImplementedException(
"bindExpression(" + expressionTypeToString(expressionType) + ").");
}
if (parsedExpression.hasAlias()) {
expression->setAlias(parsedExpression.getAlias());
Expand Down Expand Up @@ -133,10 +134,10 @@ shared_ptr<Expression> ExpressionBinder::bindPropertyExpression(
validateExpectedDataType(*child, unordered_set<DataTypeID>{NODE, REL});
if (NODE == child->dataType.typeID) {
return bindNodePropertyExpression(child, propertyName);
} else if (REL == child->dataType.typeID) {
} else {
assert(REL == child->dataType.typeID);
return bindRelPropertyExpression(child, propertyName);
}
assert(false);
}

shared_ptr<Expression> ExpressionBinder::bindNodePropertyExpression(
Expand Down Expand Up @@ -290,10 +291,10 @@ shared_ptr<Expression> ExpressionBinder::staticEvaluate(const string& functionNa
auto strVal = ((LiteralExpression*)children[0].get())->literal->strVal;
return make_shared<LiteralExpression>(DataType(INTERVAL),
make_unique<Literal>(Interval::FromCString(strVal.c_str(), strVal.length())));
} else if (functionName == ID_FUNC_NAME) {
} else {
assert(functionName == ID_FUNC_NAME);
return bindInternalIDExpression(parsedExpression);
}
assert(false);
}

shared_ptr<Expression> ExpressionBinder::bindInternalIDExpression(
Expand All @@ -307,13 +308,13 @@ shared_ptr<Expression> ExpressionBinder::bindInternalIDExpression(
shared_ptr<Expression> nodeOrRel) {
if (nodeOrRel->dataType.typeID == NODE) {
return ((NodeExpression*)nodeOrRel.get())->getNodeIDPropertyExpression();
} else if (nodeOrRel->dataType.typeID == REL) {
} else {
assert(nodeOrRel->dataType.typeID == REL);
auto rel = (RelExpression*)nodeOrRel.get();
auto relTableSchema =
binder->catalog.getReadOnlyVersion()->getRelTableSchema(rel->getTableID());
return bindRelPropertyExpression(nodeOrRel, relTableSchema->getRelIDDefinition());
}
assert(false);
}

shared_ptr<Expression> ExpressionBinder::bindParameterExpression(
Expand Down
12 changes: 2 additions & 10 deletions src/catalog/catalog.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -264,7 +264,7 @@ const Property& CatalogContent::getNodeProperty(
return property;
}
}
assert(false);
throw CatalogException("Cannot find node property " + propertyName + ".");
}

const Property& CatalogContent::getRelProperty(
Expand All @@ -274,21 +274,13 @@ const Property& CatalogContent::getRelProperty(
return property;
}
}
assert(false);
throw CatalogException("Cannot find rel property " + propertyName + ".");
}

vector<Property> CatalogContent::getAllNodeProperties(table_id_t tableID) const {
return nodeTableSchemas.at(tableID)->getAllNodeProperties();
}

const unordered_set<table_id_t>& CatalogContent::getRelTableIDsForNodeTableDirection(
table_id_t tableID, RelDirection direction) const {
if (FWD == direction) {
return nodeTableSchemas.at(tableID)->fwdRelTableIDSet;
}
return nodeTableSchemas.at(tableID)->bwdRelTableIDSet;
}

void CatalogContent::saveToFile(const string& directory, DBFileType dbFileType) {
auto catalogPath = StorageUtils::getCatalogFilePath(directory, dbFileType);
auto fileInfo = FileUtils::openFile(catalogPath, O_WRONLY | O_CREAT);
Expand Down
4 changes: 2 additions & 2 deletions src/catalog/catalog_structs.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ RelMultiplicity getRelMultiplicityFromString(const string& relMultiplicityString
} else if ("MANY_MANY" == relMultiplicityString) {
return MANY_MANY;
}
throw CatalogException("Invalid relMultiplicity string \"" + relMultiplicityString + "\"");
throw CatalogException("Invalid relMultiplicity string '" + relMultiplicityString + "'.");
}

string getRelMultiplicityAsString(RelMultiplicity relMultiplicity) {
Expand All @@ -33,7 +33,7 @@ string getRelMultiplicityAsString(RelMultiplicity relMultiplicity) {
return "ONE_MANY";
}
default:
assert(false);
throw CatalogException("Cannot convert rel multiplicity to string.");
}
}

Expand Down
2 changes: 1 addition & 1 deletion src/common/logging_level_utils.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ string LoggingLevelUtils::convertLevelEnumToStr(spdlog::level::level_enum levelE
return "off";
}
default:
assert(false);
throw ConversionException("Unsupported logging level conversion to string.");
}
}

Expand Down
26 changes: 0 additions & 26 deletions src/common/task_system/task_scheduler.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -30,14 +30,9 @@ shared_ptr<ScheduledTask> TaskScheduler::scheduleTask(const shared_ptr<Task>& ta
}

void TaskScheduler::waitAllTasksToCompleteOrError() {
logger->debug("Thread {} called waitAllTasksToCompleteOrError. Beginning to wait.",
ThreadUtils::getThreadIDString());
while (true) {
lock_t lck{mtx};
if (taskQueue.empty()) {
logger->debug("Thread {} successfully waited all tasks to be complete. Returning from "
"waitAllTasksToCompleteOrError.",
ThreadUtils::getThreadIDString());
return;
}
for (auto it = taskQueue.begin(); it != taskQueue.end(); ++it) {
Expand All @@ -56,8 +51,6 @@ void TaskScheduler::waitAllTasksToCompleteOrError() {
}

void TaskScheduler::scheduleTaskAndWaitOrError(const shared_ptr<Task>& task) {
logger->debug("Thread {} called scheduleTaskAndWaitOrError. Scheduling task.",
ThreadUtils::getThreadIDString());
for (auto& dependency : task->children) {
scheduleTaskAndWaitOrError(dependency);
}
Expand All @@ -66,13 +59,9 @@ void TaskScheduler::scheduleTaskAndWaitOrError(const shared_ptr<Task>& task) {
this_thread::sleep_for(chrono::microseconds(THREAD_SLEEP_TIME_WHEN_WAITING_IN_MICROS));
}
if (task->hasException()) {
logger->debug("Thread {} found a task with exception. Will call removeErroringTask.",
ThreadUtils::getThreadIDString());
removeErroringTask(scheduledTask->ID);
std::rethrow_exception(task->getExceptionPtr());
}
logger->debug("Thread {} exiting scheduleTaskAndWaitOrError (task was successfully complete)",
ThreadUtils::getThreadIDString());
}

shared_ptr<ScheduledTask> TaskScheduler::getTaskAndRegister() {
Expand All @@ -91,15 +80,11 @@ shared_ptr<ScheduledTask> TaskScheduler::getTaskAndRegister() {
// queue. For (ii) and (iii) we keep the task in queue. Recall erroring tasks need to be
// manually removed.
if (task->isCompletedSuccessfully()) { // option (i)
logger->debug("Thread {} is removing completed schedule task {} from queue.",
ThreadUtils::getThreadIDString(), (*it)->ID);
it = taskQueue.erase(it);
} else { // option (ii) or (iii): keep the task in the queue.
++it;
}
} else {
logger->debug("Registered thread {} to schedule task {}.",
ThreadUtils::getThreadIDString(), (*it)->ID);
return *it;
}
}
Expand All @@ -108,19 +93,12 @@ shared_ptr<ScheduledTask> TaskScheduler::getTaskAndRegister() {

void TaskScheduler::removeErroringTask(uint64_t scheduledTaskID) {
lock_t lck{mtx};
logger->debug("RemovErroringTask is called.Thread {}", ThreadUtils::getThreadIDString());
for (auto it = taskQueue.begin(); it != taskQueue.end(); ++it) {
if (scheduledTaskID == (*it)->ID) {
logger->debug(
"Inside removeErroringTask.Thread {} is removing an erroring task {} from queue.",
ThreadUtils::getThreadIDString(), (*it)->ID);
taskQueue.erase(it);
return;
}
}
logger->debug(
"Inside removeErroringTask. Thread {} could not find the task to remove from queue.",
ThreadUtils::getThreadIDString());
}

void TaskScheduler::runWorkerThread() {
Expand All @@ -136,11 +114,7 @@ void TaskScheduler::runWorkerThread() {
try {
scheduledTask->task->run();
scheduledTask->task->deRegisterThreadAndFinalizeTaskIfNecessary();
logger->debug(
"Thread {} completed task successfully.", ThreadUtils::getThreadIDString());
} catch (exception& e) {
logger->info("Thread {} caught an exception {}. Setting the exception of the task.",
e.what(), ThreadUtils::getThreadIDString());
scheduledTask->task->setException(current_exception());
scheduledTask->task->deRegisterThreadAndFinalizeTaskIfNecessary();
continue;
Expand Down
6 changes: 4 additions & 2 deletions src/common/type_utils.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,8 @@ string TypeUtils::elementToString(const DataType& dataType, uint8_t* overflowPtr
case LIST:
return TypeUtils::toString(((ku_list_t*)overflowPtr)[pos], dataType);
default:
assert(false);
throw RuntimeException("Invalid data type " + Types::dataTypeToString(dataType) +
" for TypeUtils::elementToString.");
}
}

Expand Down Expand Up @@ -127,7 +128,8 @@ string TypeUtils::toString(const Literal& literal) {
return result;
}
default:
assert(false);
throw RuntimeException("Invalid data type " + Types::dataTypeToString(literal.dataType) +
" for TypeUtils::toString.");
}
}

Expand Down
9 changes: 6 additions & 3 deletions src/function/aggregate_function.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,8 @@ unique_ptr<AggregateFunction> AggregateFunctionUtil::getAvgFunction(
AvgFunction<double_t>::updateAll, AvgFunction<double_t>::updatePos,
AvgFunction<double_t>::combine, AvgFunction<double_t>::finalize, inputType, isDistinct);
default:
assert(false);
throw RuntimeException("Unsupported input data type " + Types::dataTypeToString(inputType) +
" for AggregateFunctionUtil::getAvgFunction.");
}
}

Expand All @@ -50,7 +51,8 @@ unique_ptr<AggregateFunction> AggregateFunctionUtil::getSumFunction(
SumFunction<double_t>::updateAll, SumFunction<double_t>::updatePos,
SumFunction<double_t>::combine, SumFunction<double_t>::finalize, inputType, isDistinct);
default:
assert(false);
throw RuntimeException("Unsupported input data type " + Types::dataTypeToString(inputType) +
" for AggregateFunctionUtil::getSumFunction.");
}
}

Expand Down Expand Up @@ -100,7 +102,8 @@ unique_ptr<AggregateFunction> AggregateFunctionUtil::getMinMaxFunction(
MinMaxFunction<nodeID_t>::combine<FUNC>, MinMaxFunction<nodeID_t>::finalize, inputType,
isDistinct);
default:
assert(false);
throw RuntimeException("Unsupported input data type " + Types::dataTypeToString(inputType) +
" for AggregateFunctionUtil::getMinMaxFunction.");
}
}

Expand Down
20 changes: 12 additions & 8 deletions src/function/vector_boolean_operations.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -9,20 +9,20 @@ scalar_exec_func VectorBooleanOperations::bindExecFunction(
ExpressionType expressionType, const expression_vector& children) {
if (isExpressionBinary(expressionType)) {
return bindBinaryExecFunction(expressionType, children);
} else if (isExpressionUnary(expressionType)) {
} else {
assert(isExpressionUnary(expressionType));
return bindUnaryExecFunction(expressionType, children);
}
assert(false);
}

scalar_select_func VectorBooleanOperations::bindSelectFunction(
ExpressionType expressionType, const expression_vector& children) {
if (isExpressionBinary(expressionType)) {
return bindBinarySelectFunction(expressionType, children);
} else if (isExpressionUnary(expressionType)) {
} else {
assert(isExpressionUnary(expressionType));
return bindUnarySelectFunction(expressionType, children);
}
assert(false);
}

scalar_exec_func VectorBooleanOperations::bindBinaryExecFunction(
Expand All @@ -42,7 +42,8 @@ scalar_exec_func VectorBooleanOperations::bindBinaryExecFunction(
return BinaryBooleanExecFunction<operation::Xor>;
}
default:
assert(false);
throw RuntimeException("Invalid expression type " + expressionTypeToString(expressionType) +
" for VectorBooleanOperations::bindBinaryExecFunction.");
}
}

Expand All @@ -63,7 +64,8 @@ scalar_select_func VectorBooleanOperations::bindBinarySelectFunction(
return BinaryBooleanSelectFunction<operation::Xor>;
}
default:
assert(false);
throw RuntimeException("Invalid expression type " + expressionTypeToString(expressionType) +
" for VectorBooleanOperations::bindBinarySelectFunction.");
}
}

Expand All @@ -75,7 +77,8 @@ scalar_exec_func VectorBooleanOperations::bindUnaryExecFunction(
return UnaryBooleanExecFunction<operation::Not>;
}
default:
assert(false);
throw RuntimeException("Invalid expression type " + expressionTypeToString(expressionType) +
" for VectorBooleanOperations::bindUnaryExecFunction.");
}
}

Expand All @@ -87,7 +90,8 @@ scalar_select_func VectorBooleanOperations::bindUnarySelectFunction(
return UnaryBooleanSelectFunction<operation::Not>;
}
default:
assert(false);
throw RuntimeException("Invalid expression type " + expressionTypeToString(expressionType) +
" for VectorBooleanOperations::bindUnaryExecFunction.");
}
}

Expand Down
6 changes: 4 additions & 2 deletions src/function/vector_null_operations.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,8 @@ scalar_exec_func VectorNullOperations::bindUnaryExecFunction(
return UnaryNullExecFunction<operation::IsNotNull>;
}
default:
assert(false);
throw RuntimeException("Invalid expression type " + expressionTypeToString(expressionType) +
"for VectorNullOperations::bindUnaryExecFunction.");
}
}

Expand All @@ -43,7 +44,8 @@ scalar_select_func VectorNullOperations::bindUnarySelectFunction(
return UnaryNullSelectFunction<operation::IsNotNull>;
}
default:
assert(false);
throw RuntimeException("Invalid expression type " + expressionTypeToString(expressionType) +
"for VectorNullOperations::bindUnarySelectFunction.");
}
}

Expand Down
8 changes: 0 additions & 8 deletions src/include/catalog/catalog.h
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@
#include "common/utils.h"
#include "function/aggregate/built_in_aggregate_functions.h"
#include "function/built_in_vector_operations.h"
#include "json.hpp"
#include "storage/wal/wal.h"

using namespace kuzu::common;
Expand Down Expand Up @@ -111,13 +110,6 @@ class CatalogContent {

void removeTableSchema(TableSchema* tableSchema);

/**
* Graph topology functions.
*/
// TODO(Xiyang): remove
virtual const unordered_set<table_id_t>& getRelTableIDsForNodeTableDirection(
table_id_t tableID, RelDirection direction) const;

void saveToFile(const string& directory, DBFileType dbFileType);
void readFromFile(const string& directory, DBFileType dbFileType);

Expand Down
Loading

0 comments on commit cb099e8

Please sign in to comment.