Skip to content
Open
Show file tree
Hide file tree
Changes from 1 commit
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
@@ -0,0 +1,134 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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
*
* https://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 org.apache.accumulo.tserver.tablet;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.UUID;

import org.apache.accumulo.core.client.rfile.RFile;
import org.apache.accumulo.core.client.rfile.RFileWriter;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.file.FileOperations;
import org.apache.accumulo.core.file.FileSKVIterator;
import org.apache.accumulo.core.metadata.StoredTabletFile;
import org.apache.accumulo.server.conf.TableConfiguration;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class SplitPointUtil {
Comment thread
ArbaazKhan1 marked this conversation as resolved.
Outdated
private static final Logger log = LoggerFactory.getLogger(SplitPointUtil.class);
private static final int DEFAULT_BATCH_SIZE = 200;

public static Optional<Key> findSplitPoint(List<StoredTabletFile> files, int maxOpen,
FileSystem fs, Configuration hadoopConf, TableConfiguration tableConf) throws IOException {

if (files.size() <= maxOpen) {
return computeMidpoint(files, fs, hadoopConf, tableConf);
}

log.info("Tablet has {} files, reducing using batch size {}", files.size(), DEFAULT_BATCH_SIZE);
List<StoredTabletFile> reduced =
reduceFiles(files, DEFAULT_BATCH_SIZE, fs, hadoopConf, tableConf);

return computeMidpoint(reduced, fs, hadoopConf, tableConf);
}

private static Optional<Key> computeMidpoint(List<StoredTabletFile> files, FileSystem fs,
Configuration conf, TableConfiguration tableConf) throws IOException {
List<Key> allKeys = new ArrayList<>();

for (StoredTabletFile file : files) {
try (FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
.forFile(file, fs, conf, tableConf.getCryptoService()).withTableConfiguration(tableConf)
.build()) {

reader.seek(new Range(), Collections.emptyList(), false);

while (reader.hasTop()) {
allKeys.add(new Key(reader.getTopKey()));
reader.next();
}
}
}

if (allKeys.isEmpty()) {
return Optional.empty();
}

Collections.sort(allKeys);
return Optional.of(allKeys.get(allKeys.size() / 2));
}

private static List<StoredTabletFile> reduceFiles(List<StoredTabletFile> files, int batchSize,
FileSystem fs, Configuration conf, TableConfiguration tableConf) throws IOException {
List<StoredTabletFile> reduced = new ArrayList<>();

for (int i = 0; i < files.size(); i += batchSize) {
List<StoredTabletFile> batch = files.subList(i, Math.min(i + batchSize, files.size()));
StoredTabletFile merged = mergeBatch(batch, fs, conf, tableConf);
reduced.add(merged);
}

if (reduced.size() > batchSize) {
return reduceFiles(reduced, batchSize, fs, conf, tableConf);
}

return reduced;
}

private static StoredTabletFile mergeBatch(List<StoredTabletFile> batch, FileSystem fs,
Configuration conf, TableConfiguration tableConf) throws IOException {
Path tmpDir = new Path("/tmp/idxReduce_" + System.nanoTime());
fs.mkdirs(tmpDir);
Path mergedPath = new Path(tmpDir, "merged_" + UUID.randomUUID() + ".rf");

Map<String,String> props = new HashMap<>();
tableConf.iterator().forEachRemaining(e -> props.put(e.getKey(), e.getValue()));

try (RFileWriter writer = RFile.newWriter().to(mergedPath.toString()).withFileSystem(fs)
.withTableProperties(props).build()) {

for (StoredTabletFile file : batch) {
try (FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
.forFile(file, fs, conf, tableConf.getCryptoService()).withTableConfiguration(tableConf)
.build()) {

reader.seek(new Range(), Collections.emptyList(), false);
while (reader.hasTop()) {
writer.append(reader.getTopKey(), reader.getTopValue());
reader.next();
}
}
}

}

return new StoredTabletFile(mergedPath.toString());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.stream.Collectors.toList;
import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
import static org.apache.accumulo.server.tablets.TabletNameGenerator.createTabletDirectoryName;

import java.io.FileNotFoundException;
import java.io.IOException;
Expand Down Expand Up @@ -75,6 +76,7 @@
import org.apache.accumulo.core.metadata.schema.Ample.ConditionalTabletMutator;
import org.apache.accumulo.core.metadata.schema.Ample.ConditionalTabletsMutator;
import org.apache.accumulo.core.metadata.schema.DataFileValue;
import org.apache.accumulo.core.metadata.schema.MetadataTime;
import org.apache.accumulo.core.metadata.schema.TabletMetadata;
import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
Expand All @@ -87,6 +89,7 @@
import org.apache.accumulo.core.util.Halt;
import org.apache.accumulo.core.util.Pair;
import org.apache.accumulo.core.util.UtilWaitThread;
import org.apache.accumulo.server.ServerContext;
import org.apache.accumulo.server.compaction.CompactionStats;
import org.apache.accumulo.server.fs.VolumeManager;
import org.apache.accumulo.server.tablets.ConditionCheckerContext.ConditionChecker;
Expand All @@ -106,6 +109,7 @@
import org.apache.accumulo.tserver.scan.ScanParameters;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.slf4j.Logger;
Expand Down Expand Up @@ -409,6 +413,11 @@ DataFileValue minorCompact(InMemoryMap memTable, ReferencedTabletFile tmpDatafil
bringMinorCompactionOnline(tmpDatafile, newDatafile,
new DataFileValue(stats.getFileSize(), stats.getEntriesWritten()), commitSession,
flushId, mincReason);
try {
maybeSplitTablet();
} catch (IOException e) {
log.error("Error while attempting to split tablet after minor compaction", e);
}
} catch (Exception e) {
final ServiceLock tserverLock = tabletServer.getLock();
if (tserverLock == null || !tserverLock.verifyLockAtSource()) {
Expand Down Expand Up @@ -1125,6 +1134,102 @@ public Map<StoredTabletFile,DataFileValue> getDatafiles() {
return getMetadata().getFilesMap();
}

public long estimateTabletSize() {
long size = 0L;

for (DataFileValue sz : getDatafiles().values()) {
size += sz.getSize();
}

return size;
}

private boolean isSplitPossible() {

long splitThreshold = tableConfiguration.getAsBytes(Property.TABLE_SPLIT_THRESHOLD);

return !extent.isRootTablet() && !isFindSplitsSuppressed()
&& estimateTabletSize() > splitThreshold;
}

public boolean needsSplit() {
var files = getDatafiles().keySet();
return files.size() > 1 && !supressFindSplits;
}

private boolean supressFindSplits = false;
private long timeOfLastMinCWhenFindSplitsWasSupressed = 0;

private boolean isFindSplitsSuppressed() {
if (supressFindSplits) {
if (timeOfLastMinCWhenFindSplitsWasSupressed != lastMinorCompactionFinishTime) {
supressFindSplits = false;
} else {
// nothing changed, do not split
return true;
}
}

return false;
}

private void suppressFindSplits() {
supressFindSplits = true;
timeOfLastMinCWhenFindSplitsWasSupressed = lastMinorCompactionFinishTime;
}

public void maybeSplitTablet() throws IOException {
if (!isSplitPossible() || !needsSplit()) {
return;
}

Optional<Key> optSplit = computeSplitPoint();
if (optSplit.isEmpty()) {
suppressFindSplits();
return;
}

Text midRow = optSplit
.orElseThrow(() -> new IllegalStateException("Split point should be present")).getRow();
KeyExtent low = new KeyExtent(extent.tableId(), midRow, extent.prevEndRow());
KeyExtent high = new KeyExtent(extent.tableId(), extent.endRow(), midRow);

ServerContext context = tabletServer.getContext();
Ample.TabletsMutator mutator = context.getAmple().mutateTablets();

String lowDir = createTabletDirectoryName(context, midRow);
String highDir = getMetadata().getDirName();
MetadataTime time = tabletTime.getMetadataTime();

mutator.mutateTablet(low).putDirName(lowDir).putTime(time);

mutator.mutateTablet(high).putDirName(highDir).putTime(time);

mutator.close();

log.info("Split tablet {} at {} into {} and {}", extent, midRow, low, high);
}

public Optional<Key> computeSplitPoint() {
try {
List<StoredTabletFile> files = new ArrayList<>(getDatafiles().keySet());
if (files.isEmpty()) {
return Optional.empty();
}

var context = tabletServer.getContext();
var fs = context.getVolumeManager().getFileSystemByPath(files.get(0).getPath());
var conf = context.getHadoopConf();
var maxOpen = tableConfiguration.getCount(Property.SPLIT_MAXOPEN);

return SplitPointUtil.findSplitPoint(files, maxOpen, fs, conf, tableConfiguration);

} catch (IOException e) {
log.warn("Error computing split point for tablet {}", extent, e);
return Optional.empty();
}
}

@Override
public void addToYieldMetric(int i) {
getTabletServer().getScanMetrics().addYield(i);
Expand Down