Skip to content

Commit

Permalink
Offers new ways of computing bulk load plans (#4898)
Browse files Browse the repository at this point in the history
Two new ways of computing bulk import load plans are offered in these change.  First the RFile API was modified to support computing a LoadPlan as the RFile is written.  Second a new LoadPlan.compute() method was added that creates a LoadPlan from an existing RFile. In addition to these changes methods were added to LoadPlan that support serializing and deserializing load plans to/from json.

All of these changes together support the use case of computing load plans in a distributed manner. For example, with a bulk import directory with N files the following use case is now supported.

 1. For eack file a task is spun up on a remote server that calls the new LoadPlan.compute() API to determine what tablets the file overlaps. Then the new LoadPlan.toJson() method is called to serialize the load plan and send it to a central place.
 2. All the load plans from the remote servers are deserialized calling the new LoadPlan.fromJson() method and merged into a single load plan that is used to do the bulk import.

Another use case these new APIs could support is running this new code in the map reduce job that generates bulk import data.

  1. In each reducer as it writes to an rfile it could also be building a LoadPlan.  A load plan can be obtained from the Rfile after closing it and serialized using LoadPlan.toJson() and the result saved to a file.  So after the map reduce job completes each rfile would have  corresponding file with a load plan for that file.
  2. Another process that runs after the map reduce job can load all the load plans from files and merge them using the new LoadPlan.fromJson() method.  Then the merged LoadPlan can be used to do the bulk import.


Both of these use cases avoid doing the analysis of files on a single machine doing the bulk import.  Bulk import V1 had this functionality and would ask random tservers to do the file analysis.  This could cause unexpected load on those tservers.  Bulk V1 would interleave analyzing files and adding them to tablets.  This could lead to odd situations where files are partially imported to some tablets and analysis fails, leaving the file partially imported.  Bulk v2 does all analysis before any files are added to tablets, however it lacks this distributed analysis capability. These changes provide the building blocks to do the distributed analysis that bulk v1 did for bulk v2.


Co-authored-by: Daniel Roberts <ddanielr@gmail.com>
Co-authored-by: Christopher Tubbs <ctubbsii@apache.org>
  • Loading branch information
3 people authored Feb 1, 2025
1 parent 51e152a commit 7a27d40
Show file tree
Hide file tree
Showing 12 changed files with 792 additions and 27 deletions.
1 change: 1 addition & 0 deletions core/src/main/java/org/apache/accumulo/core/Constants.java
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,7 @@ public class Constants {
public static final String BULK_PREFIX = "b-";
public static final String BULK_RENAME_FILE = "renames.json";
public static final String BULK_LOAD_MAPPING = "loadmap.json";
public static final String BULK_WORKING_PREFIX = "accumulo-bulk-";

public static final String CLONE_PREFIX = "c-";
public static final byte[] CLONE_PREFIX_BYTES = CLONE_PREFIX.getBytes(UTF_8);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,131 @@
/*
* 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.core.client.rfile;

import java.util.HashSet;
import java.util.Set;

import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.LoadPlan;
import org.apache.accumulo.core.data.TableId;
import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.hadoop.io.Text;

import com.google.common.base.Preconditions;

class LoadPlanCollector {

private final LoadPlan.SplitResolver splitResolver;
private boolean finished = false;
private Text lgFirstRow;
private Text lgLastRow;
private Text firstRow;
private Text lastRow;
private Set<KeyExtent> overlappingExtents;
private KeyExtent currentExtent;
private long appended = 0;

LoadPlanCollector(LoadPlan.SplitResolver splitResolver) {
this.splitResolver = splitResolver;
this.overlappingExtents = new HashSet<>();
}

LoadPlanCollector() {
splitResolver = null;
this.overlappingExtents = null;

}

private void appendNoSplits(Key key) {
if (lgFirstRow == null) {
lgFirstRow = key.getRow();
lgLastRow = lgFirstRow;
} else {
var row = key.getRow();
lgLastRow = row;
}
}

private static final TableId FAKE_ID = TableId.of("123");

private void appendSplits(Key key) {
var row = key.getRow();
if (currentExtent == null || !currentExtent.contains(row)) {
var tableSplits = splitResolver.apply(row);
var extent = new KeyExtent(FAKE_ID, tableSplits.getEndRow(), tableSplits.getPrevRow());
Preconditions.checkState(extent.contains(row), "%s does not contain %s", tableSplits, row);
if (currentExtent != null) {
overlappingExtents.add(currentExtent);
}
currentExtent = extent;
}
}

public void append(Key key) {
if (splitResolver == null) {
appendNoSplits(key);
} else {
appendSplits(key);
}
appended++;
}

public void startLocalityGroup() {
if (lgFirstRow != null) {
if (firstRow == null) {
firstRow = lgFirstRow;
lastRow = lgLastRow;
} else {
// take the minimum
firstRow = firstRow.compareTo(lgFirstRow) < 0 ? firstRow : lgFirstRow;
// take the maximum
lastRow = lastRow.compareTo(lgLastRow) > 0 ? lastRow : lgLastRow;
}
lgFirstRow = null;
lgLastRow = null;
}
}

public LoadPlan getLoadPlan(String filename) {
Preconditions.checkState(finished, "Attempted to get load plan before closing");

if (appended == 0) {
return LoadPlan.builder().build();
}

if (splitResolver == null) {
return LoadPlan.builder().loadFileTo(filename, LoadPlan.RangeType.FILE, firstRow, lastRow)
.build();
} else {
var builder = LoadPlan.builder();
overlappingExtents.add(currentExtent);
for (var extent : overlappingExtents) {
builder.loadFileTo(filename, LoadPlan.RangeType.TABLE, extent.prevEndRow(),
extent.endRow());
}
return builder.build();
}
}

public void close() {
finished = true;
// compute the overall min and max rows
startLocalityGroup();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.accumulo.core.client.summary.Summary.FileStatistics;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.LoadPlan;
import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.security.Authorizations;
import org.apache.hadoop.fs.FileSystem;
Expand Down Expand Up @@ -428,6 +429,15 @@ default WriterOptions withSummarizers(SummarizerConfiguration... summarizerConf)
*/
WriterOptions withVisibilityCacheSize(int maxSize);

/**
* @param splitResolver builds a {@link LoadPlan} using table split points provided by the given
* splitResolver.
* @return this
* @see RFileWriter#getLoadPlan(String)
* @since 2.1.4
*/
WriterOptions withSplitResolver(LoadPlan.SplitResolver splitResolver);

/**
* @return a new RfileWriter created with the options previously specified.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.accumulo.core.data.ArrayByteSequence;
import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.LoadPlan;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.file.FileSKVWriter;
import org.apache.accumulo.core.security.ColumnVisibility;
Expand Down Expand Up @@ -92,12 +93,15 @@ public class RFileWriter implements AutoCloseable {

private final FileSKVWriter writer;
private final LRUMap<ByteSequence,Boolean> validVisibilities;

private final LoadPlanCollector loadPlanCollector;
private boolean startedLG;
private boolean startedDefaultLG;

RFileWriter(FileSKVWriter fileSKVWriter, int visCacheSize) {
RFileWriter(FileSKVWriter fileSKVWriter, int visCacheSize, LoadPlanCollector loadPlanCollector) {
this.writer = fileSKVWriter;
this.validVisibilities = new LRUMap<>(visCacheSize);
this.loadPlanCollector = loadPlanCollector;
}

private void _startNewLocalityGroup(String name, Set<ByteSequence> columnFamilies)
Expand All @@ -106,6 +110,7 @@ private void _startNewLocalityGroup(String name, Set<ByteSequence> columnFamilie
"Cannot start a locality group after starting the default locality group");
writer.startNewLocalityGroup(name, columnFamilies);
startedLG = true;
loadPlanCollector.startLocalityGroup();
}

/**
Expand Down Expand Up @@ -175,6 +180,7 @@ public void startNewLocalityGroup(String name, String... families) throws IOExce

public void startDefaultLocalityGroup() throws IOException {
Preconditions.checkState(!startedDefaultLG);
loadPlanCollector.startLocalityGroup();
writer.startDefaultLocalityGroup();
startedDefaultLG = true;
startedLG = true;
Expand Down Expand Up @@ -204,6 +210,7 @@ public void append(Key key, Value val) throws IOException {
validVisibilities.put(new ArrayByteSequence(Arrays.copyOf(cv, cv.length)), Boolean.TRUE);
}
writer.append(key, val);
loadPlanCollector.append(key);
}

/**
Expand Down Expand Up @@ -249,6 +256,31 @@ public void append(Iterable<Entry<Key,Value>> keyValues) throws IOException {

@Override
public void close() throws IOException {
writer.close();
try {
writer.close();
} finally {
loadPlanCollector.close();
}
}

/**
* If no split resolver was provided when the RFileWriter was built then this method will return a
* simple load plan of type {@link org.apache.accumulo.core.data.LoadPlan.RangeType#FILE} using
* the first and last row seen. If a splitResolver was provided then this will return a load plan
* of type {@link org.apache.accumulo.core.data.LoadPlan.RangeType#TABLE} that has the split
* ranges the rows written overlapped.
*
* @param filename This file name will be used in the load plan and it should match the name that
* will be used when bulk importing this file. Only a filename is needed, not a full path.
* @return load plan computed from the keys written to the rfile.
* @see org.apache.accumulo.core.client.rfile.RFile.WriterOptions#withSplitResolver(LoadPlan.SplitResolver)
* @since 2.1.4
* @throws IllegalStateException is attempting to get load plan before calling {@link #close()}
* @throws IllegalArgumentException is a full path is passed instead of a filename
*/
public LoadPlan getLoadPlan(String filename) {
Preconditions.checkArgument(!filename.contains("/"),
"Unexpected path %s seen instead of file name", filename);
return loadPlanCollector.getLoadPlan(filename);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,14 @@
package org.apache.accumulo.core.client.rfile;

import static com.google.common.base.Preconditions.checkArgument;
import static java.util.Objects.requireNonNull;

import java.io.IOException;
import java.io.OutputStream;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Objects;
import java.util.stream.Stream;

import org.apache.accumulo.core.client.rfile.RFile.WriterFSOptions;
Expand All @@ -37,6 +37,7 @@
import org.apache.accumulo.core.conf.ConfigurationCopy;
import org.apache.accumulo.core.conf.DefaultConfiguration;
import org.apache.accumulo.core.crypto.CryptoFactoryLoader;
import org.apache.accumulo.core.data.LoadPlan;
import org.apache.accumulo.core.file.FileOperations;
import org.apache.accumulo.core.metadata.ValidationUtil;
import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
Expand Down Expand Up @@ -72,6 +73,7 @@ OutputStream getOutputStream() {
private int visCacheSize = 1000;
private Map<String,String> samplerProps = Collections.emptyMap();
private Map<String,String> summarizerProps = Collections.emptyMap();
private LoadPlan.SplitResolver splitResolver;

private void checkDisjoint(Map<String,String> props, Map<String,String> derivedProps,
String kind) {
Expand All @@ -81,7 +83,7 @@ private void checkDisjoint(Map<String,String> props, Map<String,String> derivedP

@Override
public WriterOptions withSampler(SamplerConfiguration samplerConf) {
Objects.requireNonNull(samplerConf);
requireNonNull(samplerConf);
Map<String,String> tmp = new SamplerConfigurationImpl(samplerConf).toTablePropertiesMap();
checkDisjoint(tableConfig, tmp, "sampler");
this.samplerProps = tmp;
Expand All @@ -106,6 +108,9 @@ public RFileWriter build() throws IOException {
CryptoService cs =
CryptoFactoryLoader.getServiceForClient(CryptoEnvironment.Scope.TABLE, tableConfig);

var loadPlanCollector =
splitResolver == null ? new LoadPlanCollector() : new LoadPlanCollector(splitResolver);

if (out.getOutputStream() != null) {
FSDataOutputStream fsdo;
if (out.getOutputStream() instanceof FSDataOutputStream) {
Expand All @@ -116,17 +121,19 @@ public RFileWriter build() throws IOException {
return new RFileWriter(
fileops.newWriterBuilder().forOutputStream(".rf", fsdo, out.getConf(), cs)
.withTableConfiguration(acuconf).withStartDisabled().build(),
visCacheSize);
visCacheSize, loadPlanCollector);
} else {
return new RFileWriter(fileops.newWriterBuilder()
.forFile(out.path.toString(), out.getFileSystem(out.path), out.getConf(), cs)
.withTableConfiguration(acuconf).withStartDisabled().build(), visCacheSize);
return new RFileWriter(
fileops.newWriterBuilder()
.forFile(out.path.toString(), out.getFileSystem(out.path), out.getConf(), cs)
.withTableConfiguration(acuconf).withStartDisabled().build(),
visCacheSize, loadPlanCollector);
}
}

@Override
public WriterOptions withFileSystem(FileSystem fs) {
Objects.requireNonNull(fs);
requireNonNull(fs);
out.fs = fs;
return this;
}
Expand All @@ -140,14 +147,14 @@ public WriterFSOptions to(String filename) {

@Override
public WriterOptions to(OutputStream out) {
Objects.requireNonNull(out);
requireNonNull(out);
this.out = new OutputArgs(out);
return this;
}

@Override
public WriterOptions withTableProperties(Iterable<Entry<String,String>> tableConfig) {
Objects.requireNonNull(tableConfig);
requireNonNull(tableConfig);
HashMap<String,String> cfg = new HashMap<>();
for (Entry<String,String> entry : tableConfig) {
cfg.put(entry.getKey(), entry.getValue());
Expand All @@ -161,7 +168,7 @@ public WriterOptions withTableProperties(Iterable<Entry<String,String>> tableCon

@Override
public WriterOptions withTableProperties(Map<String,String> tableConfig) {
Objects.requireNonNull(tableConfig);
requireNonNull(tableConfig);
return withTableProperties(tableConfig.entrySet());
}

Expand All @@ -172,9 +179,15 @@ public WriterOptions withVisibilityCacheSize(int maxSize) {
return this;
}

@Override
public WriterOptions withSplitResolver(LoadPlan.SplitResolver splitResolver) {
this.splitResolver = requireNonNull(splitResolver);
return this;
}

@Override
public WriterOptions withSummarizers(SummarizerConfiguration... summarizerConf) {
Objects.requireNonNull(summarizerConf);
requireNonNull(summarizerConf);
Map<String,String> tmp = SummarizerConfiguration.toTableProperties(summarizerConf);
checkDisjoint(tableConfig, tmp, "summarizer");
this.summarizerProps = tmp;
Expand Down
Loading

0 comments on commit 7a27d40

Please sign in to comment.