Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Rename files written by PageSink #14974

Merged
merged 6 commits into from
Oct 1, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
import com.facebook.presto.hive.HiveFileContext;
import com.facebook.presto.hive.HiveFileInfo;
import com.facebook.presto.hive.filesystem.ExtendedFileSystem;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.ContentSummary;
Expand Down Expand Up @@ -498,4 +499,11 @@ public RemoteIterator<HiveFileInfo> listFiles(Path path)
{
return dataTier.listFiles(path);
}

@Override
public ListenableFuture<Void> renameFileAsync(Path source, Path destination)
throws IOException
{
return dataTier.renameFileAsync(source, destination);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

import com.facebook.presto.hive.HiveFileContext;
import com.facebook.presto.hive.HiveFileInfo;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
Expand Down Expand Up @@ -43,4 +44,10 @@ public RemoteIterator<LocatedFileStatus> listDirectory(Path path)
{
throw new UnsupportedOperationException();
}

public ListenableFuture<Void> renameFileAsync(Path source, Path destination)
throws IOException
{
throw new UnsupportedOperationException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

import com.facebook.presto.hive.HiveFileContext;
import com.facebook.presto.hive.filesystem.ExtendedFileSystem;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus;
Expand All @@ -29,6 +30,8 @@
import java.util.List;
import java.util.Map;

import static com.google.common.util.concurrent.Futures.immediateFuture;

public class HadoopExtendedFileSystem
extends ExtendedFileSystem
{
Expand Down Expand Up @@ -189,6 +192,14 @@ public boolean rename(Path src, Path dst)
return fs.rename(src, dst);
}

@Override
public ListenableFuture<Void> renameFileAsync(Path src, Path dst)
throws IOException
{
fs.rename(src, dst);
return immediateFuture(null);
}

@Override
protected void rename(Path src, Path dst, Options.Rename... options)
throws IOException
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,8 @@ public class HiveClientConfig
private boolean isPartialAggregationPushdownEnabled;
private boolean isPartialAggregationPushdownForVariableLengthDatatypesEnabled;

private boolean fileRenamingEnabled;

public int getMaxInitialSplits()
{
return maxInitialSplits;
Expand Down Expand Up @@ -1509,4 +1511,17 @@ public boolean isPartialAggregationPushdownForVariableLengthDatatypesEnabled()
{
return this.isPartialAggregationPushdownForVariableLengthDatatypesEnabled;
}

@Config("hive.file_renaming_enabled")
@ConfigDescription("enable file renaming")
public HiveClientConfig setFileRenamingEnabled(boolean fileRenamingEnabled)
{
this.fileRenamingEnabled = fileRenamingEnabled;
return this;
}

public boolean isFileRenamingEnabled()
{
return this.fileRenamingEnabled;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,9 @@ public void configure(Binder binder)
binder.bind(HiveWriterStats.class).in(Scopes.SINGLETON);
newExporter(binder).export(HiveWriterStats.class).as(generatedNameOf(HiveWriterStats.class, connectorId));

binder.bind(HiveFileRenamer.class).in(Scopes.SINGLETON);
newExporter(binder).export(HiveFileRenamer.class).as(generatedNameOf(HiveFileRenamer.class, connectorId));

newSetBinder(binder, EventClient.class).addBinding().to(HiveEventClient.class).in(Scopes.SINGLETON);
binder.bind(HivePartitionManager.class).in(Scopes.SINGLETON);
binder.bind(LocationService.class).to(HiveLocationService.class).in(Scopes.SINGLETON);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,140 @@
/*
* 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.
*/
package com.facebook.presto.hive;

import com.facebook.presto.spi.ConnectorMetadataUpdateHandle;
import com.facebook.presto.spi.QueryId;
import com.facebook.presto.spi.SchemaTableName;
import com.google.common.collect.ImmutableList;
import org.weakref.jmx.Managed;

import javax.annotation.PreDestroy;

import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;

import static com.google.common.base.Verify.verify;

public class HiveFileRenamer
NikhilCollooru marked this conversation as resolved.
Show resolved Hide resolved
{
private final Map<QueryId, Map<HiveMetadataUpdateKey, AtomicLong>> queryPartitionFileCounterMap = new ConcurrentHashMap<>();
private final Map<QueryId, Map<HiveMetadataUpdateHandle, String>> queryHiveMetadataResultMap = new ConcurrentHashMap<>();

public List<ConnectorMetadataUpdateHandle> getMetadataUpdateResults(List<ConnectorMetadataUpdateHandle> metadataUpdateRequests, QueryId queryId)
{
ImmutableList.Builder<ConnectorMetadataUpdateHandle> metadataUpdateResults = ImmutableList.builder();

for (ConnectorMetadataUpdateHandle connectorMetadataUpdateHandle : metadataUpdateRequests) {
HiveMetadataUpdateHandle request = (HiveMetadataUpdateHandle) connectorMetadataUpdateHandle;
String fileName = getFileName(request, queryId);
metadataUpdateResults.add(new HiveMetadataUpdateHandle(request.getRequestId(), request.getSchemaTableName(), request.getPartitionName(), Optional.of(fileName)));
}
return metadataUpdateResults.build();
}

public void cleanup(QueryId queryId)
{
queryPartitionFileCounterMap.remove(queryId);
queryHiveMetadataResultMap.remove(queryId);
}

private String getFileName(HiveMetadataUpdateHandle request, QueryId queryId)
{
if (!queryPartitionFileCounterMap.containsKey(queryId) || !queryHiveMetadataResultMap.containsKey(queryId)) {
queryPartitionFileCounterMap.putIfAbsent(queryId, new ConcurrentHashMap<>());
queryHiveMetadataResultMap.putIfAbsent(queryId, new ConcurrentHashMap<>());
}

// To keep track of the file counter per query per partition
Map<HiveMetadataUpdateKey, AtomicLong> partitionFileCounterMap = queryPartitionFileCounterMap.get(queryId);

// To keep track of the file name result per query per request
// This is to make sure that request - fileName mapping is 1:1
Map<HiveMetadataUpdateHandle, String> hiveMetadataResultMap = queryHiveMetadataResultMap.get(queryId);

// If we have seen this request before then directly return the result.
if (hiveMetadataResultMap.containsKey(request)) {
// We come here if for some reason the worker did not receive the fileName and it retried the request.
return hiveMetadataResultMap.get(request);
}

HiveMetadataUpdateKey key = new HiveMetadataUpdateKey(request);
// File names start from 0
partitionFileCounterMap.putIfAbsent(key, new AtomicLong(0));

AtomicLong fileCount = partitionFileCounterMap.get(key);
String fileName = Long.valueOf(fileCount.getAndIncrement()).toString();

// Store the request - fileName mapping
hiveMetadataResultMap.put(request, fileName);

return fileName;
}

@PreDestroy
public void stop()
{
// Mappings should be deleted when query finishes. So verify that map is empty before its closed.
verify(queryPartitionFileCounterMap.isEmpty(), "Query partition file counter map has %s entries left behind", queryPartitionFileCounterMap.size());
verify(queryHiveMetadataResultMap.isEmpty(), "Query hive metadata result map has %s entries left behind", queryHiveMetadataResultMap.size());
}

@Managed
public int getQueryPartitionFileCounterMapSize()
{
return queryPartitionFileCounterMap.size();
}

@Managed
public int getHiveMetadataUpdateResultMapSize()
{
return queryHiveMetadataResultMap.size();
}

private static class HiveMetadataUpdateKey
{
private final SchemaTableName schemaTableName;
private final Optional<String> partitionName;

private HiveMetadataUpdateKey(HiveMetadataUpdateHandle hiveMetadataUpdateHandle)
{
this.schemaTableName = hiveMetadataUpdateHandle.getSchemaTableName();
this.partitionName = hiveMetadataUpdateHandle.getPartitionName();
}

@Override
public boolean equals(Object obj)
{
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
HiveMetadataUpdateKey o = (HiveMetadataUpdateKey) obj;
return schemaTableName.equals(o.schemaTableName) &&
partitionName.equals(o.partitionName);
}

@Override
public int hashCode()
{
return Objects.hash(schemaTableName, partitionName);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ColumnMetadata;
import com.facebook.presto.spi.ConnectorInsertTableHandle;
import com.facebook.presto.spi.ConnectorMetadataUpdateHandle;
import com.facebook.presto.spi.ConnectorNewTableLayout;
import com.facebook.presto.spi.ConnectorOutputTableHandle;
import com.facebook.presto.spi.ConnectorSession;
Expand All @@ -61,6 +62,7 @@
import com.facebook.presto.spi.DiscretePredicates;
import com.facebook.presto.spi.InMemoryRecordSet;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.QueryId;
import com.facebook.presto.spi.RecordCursor;
import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.spi.SchemaTablePrefix;
Expand Down Expand Up @@ -356,6 +358,7 @@ public class HiveMetadata
private final PartitionObjectBuilder partitionObjectBuilder;
private final HiveEncryptionInformationProvider encryptionInformationProvider;
private final HivePartitionStats hivePartitionStats;
private final HiveFileRenamer hiveFileRenamer;

public HiveMetadata(
SemiTransactionalHiveMetastore metastore,
Expand All @@ -380,7 +383,8 @@ public HiveMetadata(
ZeroRowFileCreator zeroRowFileCreator,
PartitionObjectBuilder partitionObjectBuilder,
HiveEncryptionInformationProvider encryptionInformationProvider,
HivePartitionStats hivePartitionStats)
HivePartitionStats hivePartitionStats,
HiveFileRenamer hiveFileRenamer)
{
this.allowCorruptWritesForTesting = allowCorruptWritesForTesting;

Expand All @@ -406,6 +410,7 @@ public HiveMetadata(
this.partitionObjectBuilder = requireNonNull(partitionObjectBuilder, "partitionObjectBuilder is null");
this.encryptionInformationProvider = requireNonNull(encryptionInformationProvider, "encryptionInformationProvider is null");
this.hivePartitionStats = requireNonNull(hivePartitionStats, "hivePartitionStats is null");
this.hiveFileRenamer = requireNonNull(hiveFileRenamer, "hiveFileRenamer is null");
}

public SemiTransactionalHiveMetastore getMetastore()
Expand Down Expand Up @@ -2879,6 +2884,18 @@ public CompletableFuture<Void> commitPageSinkAsync(ConnectorSession session, Con
return toCompletableFuture(stagingFileCommitter.commitFiles(session, handle.getSchemaName(), handle.getTableName(), getPartitionUpdates(fragments)));
}

@Override
public List<ConnectorMetadataUpdateHandle> getMetadataUpdateResults(List<ConnectorMetadataUpdateHandle> metadataUpdateRequests, QueryId queryId)
{
return hiveFileRenamer.getMetadataUpdateResults(metadataUpdateRequests, queryId);
}

@Override
public void doMetadataUpdateCleanup(QueryId queryId)
{
hiveFileRenamer.cleanup(queryId);
}

private List<GrantInfo> buildGrants(SchemaTableName tableName, PrestoPrincipal principal)
{
ImmutableList.Builder<GrantInfo> result = ImmutableList.builder();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@ public class HiveMetadataFactory
private final PartitionObjectBuilder partitionObjectBuilder;
private final HiveEncryptionInformationProvider encryptionInformationProvider;
private final HivePartitionStats hivePartitionStats;
private final HiveFileRenamer hiveFileRenamer;

@Inject
@SuppressWarnings("deprecation")
Expand All @@ -86,7 +87,8 @@ public HiveMetadataFactory(
NodeVersion nodeVersion,
PartitionObjectBuilder partitionObjectBuilder,
HiveEncryptionInformationProvider encryptionInformationProvider,
HivePartitionStats hivePartitionStats)
HivePartitionStats hivePartitionStats,
HiveFileRenamer hiveFileRenamer)
{
this(
metastore,
Expand Down Expand Up @@ -114,7 +116,8 @@ public HiveMetadataFactory(
nodeVersion.toString(),
partitionObjectBuilder,
encryptionInformationProvider,
hivePartitionStats);
hivePartitionStats,
hiveFileRenamer);
}

public HiveMetadataFactory(
Expand Down Expand Up @@ -143,7 +146,8 @@ public HiveMetadataFactory(
String prestoVersion,
PartitionObjectBuilder partitionObjectBuilder,
HiveEncryptionInformationProvider encryptionInformationProvider,
HivePartitionStats hivePartitionStats)
HivePartitionStats hivePartitionStats,
HiveFileRenamer hiveFileRenamer)
{
this.allowCorruptWritesForTesting = allowCorruptWritesForTesting;
this.skipDeletionForAlter = skipDeletionForAlter;
Expand Down Expand Up @@ -172,6 +176,7 @@ public HiveMetadataFactory(
this.partitionObjectBuilder = requireNonNull(partitionObjectBuilder, "partitionObjectBuilder is null");
this.encryptionInformationProvider = requireNonNull(encryptionInformationProvider, "encryptionInformationProvider is null");
this.hivePartitionStats = requireNonNull(hivePartitionStats, "hivePartitionStats is null");
this.hiveFileRenamer = requireNonNull(hiveFileRenamer, "hiveFileRenamer is null");

if (!allowCorruptWritesForTesting && !timeZone.equals(DateTimeZone.getDefault())) {
log.warn("Hive writes are disabled. " +
Expand Down Expand Up @@ -214,6 +219,7 @@ public HiveMetadata get()
zeroRowFileCreator,
partitionObjectBuilder,
encryptionInformationProvider,
hivePartitionStats);
hivePartitionStats,
hiveFileRenamer);
}
}
Loading