Skip to content

Commit

Permalink
Fix C++ CVEs
Browse files Browse the repository at this point in the history
Use of Uninitialized Variables
  • Loading branch information
jkhaliqi committed Jan 8, 2025
1 parent 1615e84 commit bdab7c5
Show file tree
Hide file tree
Showing 6 changed files with 44 additions and 29 deletions.
5 changes: 4 additions & 1 deletion cpp/velox/compute/WholeStageResultIterator.cc
Original file line number Diff line number Diff line change
Expand Up @@ -192,10 +192,13 @@ std::shared_ptr<velox::core::QueryCtx> WholeStageResultIterator::createNewVeloxQ

std::shared_ptr<ColumnarBatch> WholeStageResultIterator::next() {
tryAddSplitsToTask();
if (task_ == nullptr) {
return nullptr;
}
if (task_->isFinished()) {
return nullptr;
}
velox::RowVectorPtr vector;
velox::RowVectorPtr vector = nullptr;
while (true) {
auto future = velox::ContinueFuture::makeEmpty();
auto out = task_->next(&future);
Expand Down
36 changes: 24 additions & 12 deletions cpp/velox/memory/VeloxMemoryManager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator {
uint64_t shrinkCapacity(uint64_t targetBytes, bool allowSpill, bool allowAbort) override {
velox::memory::ScopedMemoryArbitrationContext ctx{};
facebook::velox::exec::MemoryReclaimer::Stats status;
velox::memory::MemoryPool* pool;
velox::memory::MemoryPool* pool = nullptr;
{
std::unique_lock guard{mutex_};
VELOX_CHECK_EQ(candidates_.size(), 1, "ListenableArbitrator should only be used within a single root pool");
Expand Down Expand Up @@ -178,7 +178,7 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator {
return freeBytes;
}

gluten::AllocationListener* listener_;
gluten::AllocationListener* listener_ = nullptr;
const uint64_t memoryPoolInitialCapacity_; // FIXME: Unused.
const uint64_t memoryPoolTransferCapacity_;
const uint64_t memoryReclaimMaxWaitMs_;
Expand Down Expand Up @@ -215,7 +215,11 @@ class ArbitratorFactoryRegister {
};

VeloxMemoryManager::VeloxMemoryManager(const std::string& kind, std::unique_ptr<AllocationListener> listener)
: MemoryManager(kind), listener_(std::move(listener)) {
: MemoryManager(kind) {
if (listener == nullptr) {
throw gluten::GlutenException("VeloxMemoryManager failed");
}
listener_(std::move(listener))
auto reservationBlockSize = VeloxBackend::get()->getBackendConf()->get<uint64_t>(
kMemoryReservationBlockSize, kMemoryReservationBlockSizeDefault);
auto memInitCapacity =
Expand Down Expand Up @@ -253,7 +257,10 @@ VeloxMemoryManager::VeloxMemoryManager(const std::string& kind, std::unique_ptr<

namespace {
MemoryUsageStats collectVeloxMemoryUsageStats(const velox::memory::MemoryPool* pool) {
MemoryUsageStats stats;
if (pool == nullptr) {
throw gluten::GlutenException("MemoryPool is nullptr");
}
MemoryUsageStats stats{};
stats.set_current(pool->usedBytes());
stats.set_peak(pool->peakBytes());
// walk down root and all children
Expand All @@ -265,20 +272,25 @@ MemoryUsageStats collectVeloxMemoryUsageStats(const velox::memory::MemoryPool* p
}

MemoryUsageStats collectGlutenAllocatorMemoryUsageStats(const MemoryAllocator* allocator) {
MemoryUsageStats stats;
MemoryUsageStats stats{};
stats.set_current(allocator->getBytes());
stats.set_peak(allocator->peakBytes());
return stats;
}

int64_t shrinkVeloxMemoryPool(velox::memory::MemoryManager* mm, velox::memory::MemoryPool* pool, int64_t size) {
std::string poolName{pool->root()->name() + "/" + pool->name()};
std::string logPrefix{"Shrink[" + poolName + "]: "};
VLOG(2) << logPrefix << "Trying to shrink " << size << " bytes of data...";
VLOG(2) << logPrefix << "Pool has reserved " << pool->usedBytes() << "/" << pool->root()->reservedBytes() << "/"
<< pool->root()->capacity() << "/" << pool->root()->maxCapacity() << " bytes.";
VLOG(2) << logPrefix << "Shrinking...";
auto shrunken = mm->arbitrator()->shrinkCapacity(pool, 0);
if (pool != nullptr) {
std::string poolName{pool->root()->name() + "/" + pool->name()};
std::string logPrefix{"Shrink[" + poolName + "]: "};
VLOG(2) << logPrefix << "Trying to shrink " << size << " bytes of data...";
VLOG(2) << logPrefix << "Pool has reserved " << pool->usedBytes() << "/" << pool->root()->reservedBytes() << "/"
<< pool->root()->capacity() << "/" << pool->root()->maxCapacity() << " bytes.";
VLOG(2) << logPrefix << "Shrinking...";
}
auto shrunken = 0;
if (mm != nullptr) {
shrunken = mm->arbitrator()->shrinkCapacity(pool, 0);
}
VLOG(2) << logPrefix << shrunken << " bytes released from shrinking.";
return shrunken;
}
Expand Down
2 changes: 1 addition & 1 deletion cpp/velox/shuffle/VeloxShuffleReader.cc
Original file line number Diff line number Diff line change
Expand Up @@ -182,7 +182,7 @@ std::unique_ptr<ByteInputStream> toByteStream(uint8_t* data, int32_t size) {
}

RowVectorPtr readComplexType(BufferPtr buffer, RowTypePtr& rowType, memory::MemoryPool* pool) {
RowVectorPtr result;
RowVectorPtr result = nullptr;
auto byteStream = toByteStream(const_cast<uint8_t*>(buffer->as<uint8_t>()), buffer->size());
auto serde = std::make_unique<serializer::presto::PrestoVectorSerde>();
serializer::presto::PrestoVectorSerde::PrestoOptions options;
Expand Down
8 changes: 4 additions & 4 deletions cpp/velox/substrait/SubstraitToVeloxExpr.cc
Original file line number Diff line number Diff line change
Expand Up @@ -312,7 +312,7 @@ std::shared_ptr<const core::ConstantTypedExpr> SubstraitVeloxExprConverter::lite
std::vector<variant> variants;
variants.reserve(literals.size());
VELOX_CHECK_GE(literals.size(), 0, "List should have at least one item.");
std::optional<TypePtr> literalType;
std::optional<TypePtr> literalType = std::nullopt;
for (const auto& literal : literals) {
auto veloxVariant = toVeloxExpr(literal);
if (!literalType.has_value()) {
Expand Down Expand Up @@ -433,7 +433,7 @@ VectorPtr SubstraitVeloxExprConverter::literalsToVector(
// Handle EmptyList and List together since the children could be either case.
case ::substrait::Expression_Literal::LiteralTypeCase::kEmptyList:
case ::substrait::Expression_Literal::LiteralTypeCase::kList: {
ArrayVectorPtr elements;
ArrayVectorPtr elements = nullptr;
for (int i = 0; i < childSize; i++) {
auto child = elementAtFunc(i);
auto childType = child.literal_type_case();
Expand All @@ -456,7 +456,7 @@ VectorPtr SubstraitVeloxExprConverter::literalsToVector(
// Handle EmptyMap and Map together since the children could be either case.
case ::substrait::Expression_Literal::LiteralTypeCase::kEmptyMap:
case ::substrait::Expression_Literal::LiteralTypeCase::kMap: {
MapVectorPtr mapVector;
MapVectorPtr mapVector = nullptr;
for (int i = 0; i < childSize; i++) {
auto child = elementAtFunc(i);
auto childType = child.literal_type_case();
Expand All @@ -478,7 +478,7 @@ VectorPtr SubstraitVeloxExprConverter::literalsToVector(
return mapVector;
}
case ::substrait::Expression_Literal::LiteralTypeCase::kStruct: {
RowVectorPtr rowVector;
RowVectorPtr rowVector = nullptr;
for (int i = 0; i < childSize; i++) {
auto element = elementAtFunc(i);
RowVectorPtr grandVector = literalsToRowVector(element);
Expand Down
18 changes: 9 additions & 9 deletions cpp/velox/substrait/SubstraitToVeloxPlan.cc
Original file line number Diff line number Diff line change
Expand Up @@ -286,7 +286,7 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::
rightKeys.emplace_back(exprConverter_->toVeloxExpr(*rightExprs[i], inputRowType));
}

core::TypedExprPtr filter;
core::TypedExprPtr filter = nullptr;
if (sJoin.has_post_join_filter()) {
filter = exprConverter_->toVeloxExpr(sJoin.post_join_filter(), inputRowType);
}
Expand Down Expand Up @@ -345,7 +345,7 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::
}

auto inputRowType = getJoinInputType(leftNode, rightNode);
core::TypedExprPtr joinConditions;
core::TypedExprPtr joinConditions = nullptr;
if (crossRel.has_expression()) {
joinConditions = exprConverter_->toVeloxExpr(crossRel.expression(), inputRowType);
}
Expand Down Expand Up @@ -581,7 +581,7 @@ std::shared_ptr<connector::hive::HiveInsertTableHandle> makeHiveInsertTableHandl
}

core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::WriteRel& writeRel) {
core::PlanNodePtr childNode;
core::PlanNodePtr childNode = nullptr;
if (writeRel.has_input()) {
childNode = toVeloxPlan(writeRel.input());
} else {
Expand Down Expand Up @@ -693,7 +693,7 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::
}

core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::ExpandRel& expandRel) {
core::PlanNodePtr childNode;
core::PlanNodePtr childNode = nullptr;
if (expandRel.has_input()) {
childNode = toVeloxPlan(expandRel.input());
} else {
Expand Down Expand Up @@ -758,7 +758,7 @@ void extractUnnestFieldExpr(
} // namespace

core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::GenerateRel& generateRel) {
core::PlanNodePtr childNode;
core::PlanNodePtr childNode = nullptr;
if (generateRel.has_input()) {
childNode = toVeloxPlan(generateRel.input());
} else {
Expand Down Expand Up @@ -920,7 +920,7 @@ const core::WindowNode::Frame SubstraitToVeloxPlanConverter::createWindowFrame(
}

core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::WindowRel& windowRel) {
core::PlanNodePtr childNode;
core::PlanNodePtr childNode = nullptr;
if (windowRel.has_input()) {
childNode = toVeloxPlan(windowRel.input());
} else {
Expand Down Expand Up @@ -1015,7 +1015,7 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::

core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(
const ::substrait::WindowGroupLimitRel& windowGroupLimitRel) {
core::PlanNodePtr childNode;
core::PlanNodePtr childNode = nullptr;
if (windowGroupLimitRel.has_input()) {
childNode = toVeloxPlan(windowGroupLimitRel.input());
} else {
Expand Down Expand Up @@ -1193,7 +1193,7 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::constructValueStreamNode(
}

auto outputType = ROW(std::move(outNames), std::move(veloxTypeList));
std::shared_ptr<ResultIterator> iterator;
std::shared_ptr<ResultIterator> iterator = nullptr;
if (!validationMode_) {
VELOX_CHECK_LT(streamIdx, inputIters_.size(), "Could not find stream index {} in input iterator list.", streamIdx);
iterator = inputIters_[streamIdx];
Expand Down Expand Up @@ -1254,7 +1254,7 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::

// Velox requires Filter Pushdown must being enabled.
bool filterPushdownEnabled = true;
std::shared_ptr<connector::hive::HiveTableHandle> tableHandle;
std::shared_ptr<connector::hive::HiveTableHandle> tableHandle = nullptr;
if (!readRel.has_filter()) {
tableHandle = std::make_shared<connector::hive::HiveTableHandle>(
kHiveConnectorId, "hive_table", filterPushdownEnabled, connector::hive::SubfieldFilters{}, nullptr);
Expand Down
4 changes: 2 additions & 2 deletions cpp/velox/tests/Substrait2VeloxPlanValidatorTest.cc
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ class Substrait2VeloxPlanValidatorTest : public exec::test::HiveConnectorTestBas
bool validatePlan(std::string file) {
std::string subPlanPath = FilePathGenerator::getDataFilePath(file);

::substrait::Plan substraitPlan;
::substrait::Plan substraitPlan{};
JsonToProtoConverter::readFromFile(subPlanPath, substraitPlan);
return validatePlan(substraitPlan);
}
Expand All @@ -60,7 +60,7 @@ class Substrait2VeloxPlanValidatorTest : public exec::test::HiveConnectorTestBas
TEST_F(Substrait2VeloxPlanValidatorTest, group) {
std::string subPlanPath = FilePathGenerator::getDataFilePath("group.json");

::substrait::Plan substraitPlan;
::substrait::Plan substraitPlan{};
JsonToProtoConverter::readFromFile(subPlanPath, substraitPlan);

ASSERT_FALSE(validatePlan(substraitPlan));
Expand Down

0 comments on commit bdab7c5

Please sign in to comment.