Skip to content

Commit

Permalink
Make key tuple
Browse files Browse the repository at this point in the history
  • Loading branch information
acvictor committed Mar 30, 2024
1 parent 4ac49e1 commit b3e697a
Show file tree
Hide file tree
Showing 6 changed files with 21 additions and 22 deletions.
16 changes: 7 additions & 9 deletions velox/connectors/hive/FileHandle.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -34,9 +34,7 @@ std::string groupName(const std::string& filename) {
} // namespace

std::shared_ptr<FileHandle> FileHandleGenerator::operator()(
const std::string& filename,
const std::string& fileSize = "",
const std::string& modificationTime = "") {
std::tuple<const std::string, const std::string, const std::string> params) {
// We have seen cases where drivers are stuck when creating file handles.
// Adding a trace here to spot this more easily in future.
process::TraceContext trace("FileHandleGenerator::operator()");
Expand All @@ -46,13 +44,13 @@ std::shared_ptr<FileHandle> FileHandleGenerator::operator()(
MicrosecondTimer timer(&elapsedTimeUs);
fileHandle = std::make_shared<FileHandle>();
filesystems::FileOptions options;
options.values["fileSize"] = fileSize;
options.values["fileSize"] = std::get<1>(params);
// Add length and modification time here to create a unique handle?
fileHandle->file = filesystems::getFileSystem(filename, properties_)
->openFileForRead(filename, options);
fileHandle->uuid = StringIdLease(fileIds(), filename);
fileHandle->groupId = StringIdLease(fileIds(), groupName(filename));
VLOG(1) << "Generating file handle for: " << filename
fileHandle->file = filesystems::getFileSystem(std::get<0>(params), properties_)
->openFileForRead(std::get<0>(params), options);
fileHandle->uuid = StringIdLease(fileIds(), std::get<0>(params));
fileHandle->groupId = StringIdLease(fileIds(), groupName(std::get<0>(params)));
VLOG(1) << "Generating file handle for: " << std::get<0>(params)
<< " uuid: " << fileHandle->uuid.id();
}
RECORD_HISTOGRAM_METRIC_VALUE(
Expand Down
5 changes: 3 additions & 2 deletions velox/connectors/hive/FileHandle.h
Original file line number Diff line number Diff line change
Expand Up @@ -62,14 +62,15 @@ class FileHandleGenerator {
FileHandleGenerator() {}
FileHandleGenerator(std::shared_ptr<const Config> properties)
: properties_(std::move(properties)) {}
std::shared_ptr<FileHandle> operator()(const std::string& filename);
std::shared_ptr<FileHandle> operator()(
std::tuple<const std::string, const std::string, const std::string> params);

private:
const std::shared_ptr<const Config> properties_;
};

using FileHandleFactory = CachedFactory<
std::string,
std::tuple<const std::string, const std::string, const std::string>,
std::shared_ptr<FileHandle>,
FileHandleGenerator>;

Expand Down
2 changes: 1 addition & 1 deletion velox/connectors/hive/HiveConnector.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ HiveConnector::HiveConnector(
fileHandleFactory_(
hiveConfig_->isFileHandleCacheEnabled()
? std::make_unique<
SimpleLRUCache<std::string, std::shared_ptr<FileHandle>>>(
SimpleLRUCache<std::tuple<const std::string, const std::string, const std::string>, std::shared_ptr<FileHandle>>>(
hiveConfig_->numCacheFileHandles())
: nullptr,
std::make_unique<FileHandleGenerator>(config)),
Expand Down
14 changes: 7 additions & 7 deletions velox/connectors/hive/SplitReader.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -146,18 +146,18 @@ void SplitReader::prepareSplit(

std::shared_ptr<FileHandle> fileHandle;
try {
auto fileSizeIter = infoColumns_.find(kFileSize);
auto fileModificationTimeIter = infoColumns_.find(kModificationTime);
if (fileSizeIter != infoColumns_.end() &&
fileModifiicationTimeIter != infoColumns_.end()) {
auto fileSizeIter = hiveSplit_->infoColumns.find(kFileSize);
auto fileModificationTimeIter = hiveSplit_->infoColumns.find(kModificationTime);
if (fileSizeIter != hiveSplit_->infoColumns.end() &&
fileModificationTimeIter != hiveSplit_->infoColumns.end()) {
fileHandle = fileHandleFactory_
->generate(
hiveSplit_->filePath,
std::make_tuple(hiveSplit_->filePath,
fileSizeIter->second,
fileModificationTimeIter->second)
fileModificationTimeIter->second))
.second;
} else {
fileHandle = fileHandleFactory_->generate(hiveSplit_->filePath).second;
fileHandle = fileHandleFactory_->generate(std::make_tuple(hiveSplit_->filePath, "", "")).second;
}
} catch (const VeloxRuntimeError& e) {
if (e.errorCode() == error_code::kFileNotFound &&
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ PositionalDeleteFileReader::PositionalDeleteFileReader(
deleteSplit_);

auto deleteFileHandle =
fileHandleFactory_->generate(deleteFile_.filePath).second;
fileHandleFactory_->generate(make_tuple(deleteFile_.filePath, "", "")).second;
auto deleteFileInput = createBufferedInput(
*deleteFileHandle,
deleteReaderOpts,
Expand Down
4 changes: 2 additions & 2 deletions velox/connectors/hive/tests/FileHandleTest.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -38,9 +38,9 @@ TEST(FileHandleTest, localFile) {

FileHandleFactory factory(
std::make_unique<
SimpleLRUCache<std::string, std::shared_ptr<FileHandle>>>(1000),
SimpleLRUCache<std::tuple<const std::string, const std::string, const std::string>, std::shared_ptr<FileHandle>>>(1000),
std::make_unique<FileHandleGenerator>());
auto fileHandle = factory.generate(filename).second;
auto fileHandle = factory.generate(std::make_tuple(filename, "", "")).second;
ASSERT_EQ(fileHandle->file->size(), 3);
char buffer[3];
ASSERT_EQ(fileHandle->file->pread(0, 3, &buffer), "foo");
Expand Down

0 comments on commit b3e697a

Please sign in to comment.