diff --git a/velox/common/file/tests/CMakeLists.txt b/velox/common/file/tests/CMakeLists.txt index 446ef6a859e1..37cfa9c6ceb0 100644 --- a/velox/common/file/tests/CMakeLists.txt +++ b/velox/common/file/tests/CMakeLists.txt @@ -11,9 +11,10 @@ # 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. +add_subdirectory(utils) add_library(velox_file_test_utils TestUtils.cpp) -target_link_libraries(velox_file_test_utils PUBLIC velox_file) +target_link_libraries(velox_file_test_utils PUBLIC velox_file velox_file_test_lib) add_executable(velox_file_test FileTest.cpp UtilsTest.cpp) add_test(velox_file_test velox_file_test) diff --git a/velox/common/file/tests/utils/CMakeLists.txt b/velox/common/file/tests/utils/CMakeLists.txt new file mode 100644 index 000000000000..56bdede2f417 --- /dev/null +++ b/velox/common/file/tests/utils/CMakeLists.txt @@ -0,0 +1,22 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# 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. + +add_library( + velox_file_test_lib + FaultyFile.cpp + FaultyFileSystem.cpp) + +target_link_libraries( + velox_file_test_lib + velox_file) diff --git a/velox/common/file/tests/utils/FaultyFile.cpp b/velox/common/file/tests/utils/FaultyFile.cpp new file mode 100644 index 000000000000..11592ee6e571 --- /dev/null +++ b/velox/common/file/tests/utils/FaultyFile.cpp @@ -0,0 +1,76 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * 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 "velox/common/file/tests/utils/FaultyFile.h" + +namespace facebook::velox::tests::utils { + +FaultyReadFile::FaultyReadFile( + std::shared_ptr delegatedFile, + FileFaultInjectionHook injectionHook) + : delegatedFile_(std::move(delegatedFile)), + injectionHook_(std::move(injectionHook)) { + VELOX_CHECK_NOT_NULL(delegatedFile_); +} + +std::string_view +FaultyReadFile::pread(uint64_t offset, uint64_t length, void* buf) const { + if (injectionHook_ != nullptr) { + FileReadFaultInjection injection(offset, length, buf); + injectionHook_(&injection); + if (!injection.delegate) { + return injection.injectedReadBuf; + } + } + return delegatedFile_->pread(offset, length, buf); +} + +uint64_t FaultyReadFile::preadv( + uint64_t offset, + const std::vector>& buffers) const { + if (injectionHook_ != nullptr) { + FileReadvFaultInjection injection(offset, buffers); + injectionHook_(&injection); + if (!injection.delegate) { + return injection.readBytes; + } + } + return delegatedFile_->preadv(offset, buffers); +} + +FaultyWriteFile::FaultyWriteFile( + std::shared_ptr delegatedFile, + FileFaultInjectionHook injectionHook) + : delegatedFile_(std::move(delegatedFile)), + injectionHook_(std::move(injectionHook)) { + VELOX_CHECK_NOT_NULL(delegatedFile_); +} +void FaultyWriteFile::append(std::string_view data) { + delegatedFile_->append(data); +} + +void FaultyWriteFile::append(std::unique_ptr data) { + delegatedFile_->append(std::move(data)); +} + +void FaultyWriteFile::flush() { + delegatedFile_->flush(); +} + +void FaultyWriteFile::close() { + delegatedFile_->close(); +} +} // namespace facebook::velox::exec::test diff --git a/velox/common/file/tests/utils/FaultyFile.h b/velox/common/file/tests/utils/FaultyFile.h new file mode 100644 index 000000000000..d78617574a87 --- /dev/null +++ b/velox/common/file/tests/utils/FaultyFile.h @@ -0,0 +1,138 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * 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. + */ + +#pragma once + +#include "velox/common/file/File.h" + +namespace facebook::velox::tests::utils { + + /// Defines the per-file operation fault injection. +struct FileFaultInjectionBase { + enum class Type { + /// Injects fault for file read operations. + kRead, + kReadv, + /// TODO: add to support fault injections for the other operator types. + }; + + const Type type; + + /// Indicates to forward this operation to the delegate file or not. If not, + /// then the file fault injection hook must have processed the request. For + /// instance, if this is a file read injection, then the hook must have filled + /// the fake data for read. + bool delegate{true}; + + explicit FileFaultInjectionBase(Type _type) : type(_type) {} +}; + +/// Defines fault injection parameters for file read. +struct FileReadFaultInjection : FileFaultInjectionBase { + const uint64_t offset; + const uint64_t length; + void* const buf; + std::string_view injectedReadBuf; + + FileReadFaultInjection(uint64_t _offset, uint64_t _length, void* _buf) + : FileFaultInjectionBase(FileFaultInjectionBase::Type::kRead), + offset(_offset), + length(_length), + buf(_buf) {} +}; + +/// Defines fault injection parameters for file readv. +struct FileReadvFaultInjection : FileFaultInjectionBase { + const uint64_t offset; + const std::vector>& buffers; + uint64_t readBytes{0}; + + FileReadvFaultInjection( + uint64_t _offset, + const std::vector>& _buffers) + : FileFaultInjectionBase(FileFaultInjectionBase::Type::kReadv), + offset(_offset), + buffers(_buffers) {} +}; + +using FileFaultInjectionHook = std::function; + +class FaultyReadFile : public ReadFile { + public: + FaultyReadFile( + std::shared_ptr delegatedFile, + FileFaultInjectionHook injectionHook); + + ~FaultyReadFile() override{}; + + uint64_t size() const override { + return delegatedFile_->size(); + } + + std::string_view pread(uint64_t offset, uint64_t length, void* buf) + const override; + + uint64_t preadv( + uint64_t offset, + const std::vector>& buffers) const override; + + uint64_t memoryUsage() const override { + return delegatedFile_->memoryUsage(); + } + + bool shouldCoalesce() const override { + return delegatedFile_->shouldCoalesce(); + } + + std::string getName() const override { + return delegatedFile_->getName(); + } + + uint64_t getNaturalReadSize() const override { + return delegatedFile_->getNaturalReadSize(); + } + + private: + const std::shared_ptr delegatedFile_; + const FileFaultInjectionHook injectionHook_; +}; + +class FaultyWriteFile : public WriteFile { + public: + FaultyWriteFile( + std::shared_ptr delegatedFile, + FileFaultInjectionHook injectionHook); + + ~FaultyWriteFile() override{}; + + void append(std::string_view data) override; + + void append(std::unique_ptr data) override; + + void flush() override; + + void close() override; + + uint64_t size() const final { + return delegatedFile_->size(); + } + + private: + const std::shared_ptr delegatedFile_; + const FileFaultInjectionHook injectionHook_; +}; + +} // namespace facebook::velox::exec::test diff --git a/velox/common/file/tests/utils/FaultyFileSystem.cpp b/velox/common/file/tests/utils/FaultyFileSystem.cpp new file mode 100644 index 000000000000..81f883a788d3 --- /dev/null +++ b/velox/common/file/tests/utils/FaultyFileSystem.cpp @@ -0,0 +1,129 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * 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 "velox/common/file/tests/utils/FaultyFileSystem.h" +#include + +#include + +namespace facebook::velox::tests::utils { +namespace { +inline std::string extractPath(std::string_view path) { + VELOX_CHECK_EQ(path.find(FaultyFileSystem::scheme()), 0); + return std::string(path.substr(FaultyFileSystem::scheme().length())); +} + +std::function schemeMatcher() { + // Note: presto behavior is to prefix local paths with 'file:'. + // Check for that prefix and prune to absolute regular paths as needed. + return [](std::string_view filePath) { + return filePath.find(FaultyFileSystem::scheme()) == 0; + }; +} + +folly::once_flag faultFilesystemInitOnceFlag; + +std::function(std::shared_ptr, std::string_view)> +fileSystemGenerator() { + return + [](std::shared_ptr properties, std::string_view filePath) { + // One instance of faulty FileSystem is sufficient. Initializes on first + // access and reuse after that. + static std::shared_ptr lfs; + folly::call_once(faultFilesystemInitOnceFlag, [&properties]() { + lfs = std::make_shared(properties); + }); + return lfs; + }; +} +} // namespace + +std::unique_ptr FaultyFileSystem::openFileForRead( + std::string_view path, + const FileOptions& options) { + const std::string delegatedPath = extractPath(path); + auto delegatedFile = getFileSystem(delegatedPath, config_) + ->openFileForRead(delegatedPath, options); + return std::make_unique( + std::move(delegatedFile), fileInjectionHook_); +} + +std::unique_ptr FaultyFileSystem::openFileForWrite( + std::string_view path, + const FileOptions& options) { + const std::string delegatedPath = extractPath(path); + auto delegatedFile = getFileSystem(delegatedPath, config_) + ->openFileForWrite(delegatedPath, options); + return std::make_unique( + std::move(delegatedFile), fileInjectionHook_); +} + +void FaultyFileSystem::remove(std::string_view path) { + const std::string delegatedPath = extractPath(path); + getFileSystem(delegatedPath, config_)->remove(path); +} + +void FaultyFileSystem::rename( + std::string_view oldPath, + std::string_view newPath, + bool overwrite) { + const auto delegatedOldPath = extractPath(oldPath); + const auto delegatedNewPath = extractPath(newPath); + getFileSystem(delegatedOldPath, config_) + ->rename(delegatedOldPath, delegatedNewPath); +} + +bool FaultyFileSystem::exists(std::string_view path) { + const auto delegatedPath = extractPath(path); + return getFileSystem(delegatedPath, config_)->exists(delegatedPath); +} + +std::vector FaultyFileSystem::list(std::string_view path) { + const auto delegatedDirPath = extractPath(path); + return getFileSystem(delegatedDirPath, config_)->list(delegatedDirPath); +} + +void FaultyFileSystem::mkdir(std::string_view path) { + const auto delegatedDirPath = extractPath(path); + getFileSystem(delegatedDirPath, config_)->mkdir(delegatedDirPath); +} + +void FaultyFileSystem::rmdir(std::string_view path) { + const auto delegatedDirPath = extractPath(path); + getFileSystem(delegatedDirPath, config_)->rmdir(delegatedDirPath); +} + +void FaultyFileSystem::setFileFaultInjection( + FileFaultInjectionHook injectionHook) { + std::lock_guard l(mu_); + fileInjectionHook_ = std::move(injectionHook); +} + +void FaultyFileSystem::clearFileFaultInjection() { + std::lock_guard l(mu_); + fileInjectionHook_ = nullptr; +} + +void registerFaultyFileSystem() { + registerFileSystem(schemeMatcher(), fileSystemGenerator()); +} + +std::shared_ptr faultyFileSystem() { + return std::dynamic_pointer_cast( + getFileSystem(FaultyFileSystem::scheme(), {})); +} +} // namespace facebook::velox::exec::test diff --git a/velox/common/file/tests/utils/FaultyFileSystem.h b/velox/common/file/tests/utils/FaultyFileSystem.h new file mode 100644 index 000000000000..3d4ccc7074b8 --- /dev/null +++ b/velox/common/file/tests/utils/FaultyFileSystem.h @@ -0,0 +1,86 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * 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. + */ +#pragma once + +#include "velox/common/file/FileSystems.h" + +#include +#include +#include +#include "velox/common/file/tests/utils/FaultyFile.h" + +namespace facebook::velox::tests::utils { + +using namespace filesystems; + +/// Implements fault filesystem which is a wrapper on top of a real file system. +/// It is used by unit test for io fault injection. By default, it delegates the +/// the file operation to the file system underneath. +class FaultyFileSystem : public FileSystem { + public: + explicit FaultyFileSystem(std::shared_ptr config) + : FileSystem(std::move(config)) {} + + ~FaultyFileSystem() override {} + + static inline std::string scheme() { + return "faulty:"; + } + + std::string name() const override { + return "Faulty FS"; + } + + std::unique_ptr openFileForRead( + std::string_view path, + const FileOptions& options) override; + + std::unique_ptr openFileForWrite( + std::string_view path, + const FileOptions& options) override; + + void remove(std::string_view path) override; + + void rename( + std::string_view oldPath, + std::string_view newPath, + bool overwrite) override; + + bool exists(std::string_view path) override; + + std::vector list(std::string_view path) override; + + void mkdir(std::string_view path) override; + + void rmdir(std::string_view path) override; + + /// Setups hook for file fault injection. + void setFileFaultInjection(FileFaultInjectionHook injectionHook); + + /// Clear hooks for file fault injections. + void clearFileFaultInjection(); + + private: + mutable std::mutex mu_; + FileFaultInjectionHook fileInjectionHook_; +}; + +/// Registers the faulty filesystem. +void registerFaultyFileSystem(); + +/// Gets the fault filesystem instance. +std::shared_ptr faultyFileSystem(); +} // namespace facebook::velox::tests::utils diff --git a/velox/exec/tests/utils/ArbitratorTestUtil.cpp b/velox/exec/tests/utils/ArbitratorTestUtil.cpp index 58c5d0af4b25..d5311e7dc66e 100644 --- a/velox/exec/tests/utils/ArbitratorTestUtil.cpp +++ b/velox/exec/tests/utils/ArbitratorTestUtil.cpp @@ -94,7 +94,7 @@ QueryTestResult runHashJoinTask( if (enableSpilling) { const auto spillDirectory = exec::test::TempDirectoryPath::create(); result.data = AssertQueryBuilder(plan) - .spillDirectory(spillDirectory->path) + .spillDirectory(spillDirectory->path()) .config(core::QueryConfig::kSpillEnabled, true) .config(core::QueryConfig::kJoinSpillEnabled, true) .config(core::QueryConfig::kSpillStartPartitionBit, "29") @@ -137,7 +137,7 @@ QueryTestResult runAggregateTask( const auto spillDirectory = exec::test::TempDirectoryPath::create(); result.data = AssertQueryBuilder(plan) - .spillDirectory(spillDirectory->path) + .spillDirectory(spillDirectory->path()) .config(core::QueryConfig::kSpillEnabled, "true") .config(core::QueryConfig::kAggregationSpillEnabled, "true") .queryCtx(queryCtx) @@ -179,7 +179,7 @@ QueryTestResult runOrderByTask( if (enableSpilling) { const auto spillDirectory = exec::test::TempDirectoryPath::create(); result.data = AssertQueryBuilder(plan) - .spillDirectory(spillDirectory->path) + .spillDirectory(spillDirectory->path()) .config(core::QueryConfig::kSpillEnabled, "true") .config(core::QueryConfig::kOrderBySpillEnabled, "true") .queryCtx(queryCtx) @@ -221,7 +221,7 @@ QueryTestResult runRowNumberTask( if (enableSpilling) { const auto spillDirectory = exec::test::TempDirectoryPath::create(); result.data = AssertQueryBuilder(plan) - .spillDirectory(spillDirectory->path) + .spillDirectory(spillDirectory->path()) .config(core::QueryConfig::kSpillEnabled, "true") .config(core::QueryConfig::kRowNumberSpillEnabled, "true") .queryCtx(queryCtx) @@ -264,7 +264,7 @@ QueryTestResult runTopNTask( const auto spillDirectory = exec::test::TempDirectoryPath::create(); result.data = AssertQueryBuilder(plan) - .spillDirectory(spillDirectory->path) + .spillDirectory(spillDirectory->path()) .config(core::QueryConfig::kSpillEnabled, "true") .config(core::QueryConfig::kTopNRowNumberSpillEnabled, "true") .queryCtx(queryCtx) @@ -306,12 +306,12 @@ QueryTestResult runWriteTask( const RowVectorPtr& expectedResult) { QueryTestResult result; const auto outputDirectory = TempDirectoryPath::create(); - auto plan = writePlan(vectors, outputDirectory->path, result.planNodeId); + auto plan = writePlan(vectors, outputDirectory->path(), result.planNodeId); if (enableSpilling) { const auto spillDirectory = exec::test::TempDirectoryPath::create(); result.data = AssertQueryBuilder(plan) - .spillDirectory(spillDirectory->path) + .spillDirectory(spillDirectory->path()) .config(core::QueryConfig::kSpillEnabled, "true") .config(core::QueryConfig::kAggregationSpillEnabled, "false") .config(core::QueryConfig::kWriterSpillEnabled, "true") diff --git a/velox/exec/tests/utils/TempDirectoryPath.cpp b/velox/exec/tests/utils/TempDirectoryPath.cpp index b34815a0cd5a..662c18aa5fe5 100644 --- a/velox/exec/tests/utils/TempDirectoryPath.cpp +++ b/velox/exec/tests/utils/TempDirectoryPath.cpp @@ -20,21 +20,28 @@ namespace facebook::velox::exec::test { -std::shared_ptr TempDirectoryPath::create() { - struct SharedTempDirectoryPath : public TempDirectoryPath { - SharedTempDirectoryPath() : TempDirectoryPath() {} - }; - return std::make_shared(); +std::shared_ptr TempDirectoryPath::create(bool injectFault) { + auto* tempDirPathPtr = new TempDirectoryPath(injectFault); + return std::shared_ptr(tempDirPathPtr); } TempDirectoryPath::~TempDirectoryPath() { - LOG(INFO) << "TempDirectoryPath:: removing all files from " << path; + LOG(INFO) << "TempDirectoryPath:: removing all files from " << path_; try { - boost::filesystem::remove_all(path.c_str()); + boost::filesystem::remove_all(path_.c_str()); } catch (...) { LOG(WARNING) << "TempDirectoryPath:: destructor failed while calling boost::filesystem::remove_all"; } } +std::string TempDirectoryPath::createTempDirectory() { + char path[] = "/tmp/velox_test_XXXXXX"; + const char* tempDirectoryPath = ::mkdtemp(path); + if (tempDirectoryPath == nullptr) { + VELOX_FAIL("Cannot open temp directory"); + } + return tempDirectoryPath; +} + } // namespace facebook::velox::exec::test diff --git a/velox/exec/tests/utils/TempDirectoryPath.h b/velox/exec/tests/utils/TempDirectoryPath.h index 31d8d5dbea16..4d1508b0bb06 100644 --- a/velox/exec/tests/utils/TempDirectoryPath.h +++ b/velox/exec/tests/utils/TempDirectoryPath.h @@ -24,27 +24,30 @@ namespace facebook::velox::exec::test { -// It manages the lifetime of a temporary directory. +/// Manages the lifetime of a temporary directory. class TempDirectoryPath { public: - static std::shared_ptr create(); + static std::shared_ptr create( + bool enableFaultInjection = false); virtual ~TempDirectoryPath(); - const std::string path; - TempDirectoryPath(const TempDirectoryPath&) = delete; TempDirectoryPath& operator=(const TempDirectoryPath&) = delete; - TempDirectoryPath() : path(createTempDirectory()) {} - - static std::string createTempDirectory() { - char path[] = "/tmp/velox_test_XXXXXX"; - const char* tempDirectoryPath = mkdtemp(path); - if (tempDirectoryPath == nullptr) { - throw std::logic_error("Cannot open temp directory"); - } - return tempDirectoryPath; + std::string path() const { + return enableFaultInjection_ ? fmt::format("faulty:{}", path_) : path_; } + + private: + static std::string createTempDirectory(); + + explicit TempDirectoryPath(bool enableFaultInjection) + : path_(createTempDirectory()), + enableFaultInjection_(enableFaultInjection) {} + + const std::string path_; + + bool enableFaultInjection_{false}; }; } // namespace facebook::velox::exec::test diff --git a/velox/exec/tests/utils/TempFilePath.cpp b/velox/exec/tests/utils/TempFilePath.cpp index 7c5cbe7370c1..b7e444ea2eb3 100644 --- a/velox/exec/tests/utils/TempFilePath.cpp +++ b/velox/exec/tests/utils/TempFilePath.cpp @@ -18,11 +18,17 @@ namespace facebook::velox::exec::test { -std::shared_ptr TempFilePath::create() { - struct SharedTempFilePath : public TempFilePath { - SharedTempFilePath() : TempFilePath() {} - }; - return std::make_shared(); +std::shared_ptr TempFilePath::create(bool faultInjectionEnable) { + auto* tempFilePathPtr = new TempFilePath(); + return std::shared_ptr(tempFilePathPtr); } +std::string TempFilePath::createTempFile(TempFilePath* tempFilePath) { + char path[] = "/tmp/velox_test_XXXXXX"; + tempFilePath->fd_ = mkstemp(path); + if (tempFilePath->fd_ == -1) { + VELOX_FAIL("Cannot open temp file"); + } + return path; +} } // namespace facebook::velox::exec::test diff --git a/velox/exec/tests/utils/TempFilePath.h b/velox/exec/tests/utils/TempFilePath.h index d993795f1e3a..7995926fec7b 100644 --- a/velox/exec/tests/utils/TempFilePath.h +++ b/velox/exec/tests/utils/TempFilePath.h @@ -26,23 +26,22 @@ namespace facebook::velox::exec::test { -// It manages the lifetime of a temporary file. +/// Manages the lifetime of a temporary file. class TempFilePath { public: - static std::shared_ptr create(); + static std::shared_ptr create( + bool faultInjectionEnable = false); virtual ~TempFilePath() { - unlink(path.c_str()); - close(fd); + ::unlink(path_.c_str()); + ::close(fd_); } - const std::string path; - TempFilePath(const TempFilePath&) = delete; TempFilePath& operator=(const TempFilePath&) = delete; void append(std::string data) { - std::ofstream file(path, std::ios_base::app); + std::ofstream file(path_, std::ios_base::app); file << data; file.flush(); file.close(); @@ -50,31 +49,25 @@ class TempFilePath { const int64_t fileSize() { struct stat st; - stat(path.data(), &st); + stat(path_.data(), &st); return st.st_size; } const int64_t fileModifiedTime() { struct stat st; - stat(path.data(), &st); + ::stat(path_.data(), &st); return st.st_mtime; } private: - int fd; + static std::string createTempFile(TempFilePath* tempFilePath); - TempFilePath() : path(createTempFile(this)) { - VELOX_CHECK_NE(fd, -1); + TempFilePath() : path_(createTempFile(this)) { + VELOX_CHECK_NE(fd_, -1); } - static std::string createTempFile(TempFilePath* tempFilePath) { - char path[] = "/tmp/velox_test_XXXXXX"; - tempFilePath->fd = mkstemp(path); - if (tempFilePath->fd == -1) { - throw std::logic_error("Cannot open temp file"); - } - return path; - } + const std::string path_; + int fd_; }; } // namespace facebook::velox::exec::test