Skip to content

Commit 7a27d40

Browse files
keith-turnerddanielrctubbsii
authored
Offers new ways of computing bulk load plans (apache#4898)
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>
1 parent 51e152a commit 7a27d40

File tree

12 files changed

+792
-27
lines changed

12 files changed

+792
-27
lines changed

core/src/main/java/org/apache/accumulo/core/Constants.java

+1
Original file line numberDiff line numberDiff line change
@@ -104,6 +104,7 @@ public class Constants {
104104
public static final String BULK_PREFIX = "b-";
105105
public static final String BULK_RENAME_FILE = "renames.json";
106106
public static final String BULK_LOAD_MAPPING = "loadmap.json";
107+
public static final String BULK_WORKING_PREFIX = "accumulo-bulk-";
107108

108109
public static final String CLONE_PREFIX = "c-";
109110
public static final byte[] CLONE_PREFIX_BYTES = CLONE_PREFIX.getBytes(UTF_8);
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,131 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* https://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
package org.apache.accumulo.core.client.rfile;
20+
21+
import java.util.HashSet;
22+
import java.util.Set;
23+
24+
import org.apache.accumulo.core.data.Key;
25+
import org.apache.accumulo.core.data.LoadPlan;
26+
import org.apache.accumulo.core.data.TableId;
27+
import org.apache.accumulo.core.dataImpl.KeyExtent;
28+
import org.apache.hadoop.io.Text;
29+
30+
import com.google.common.base.Preconditions;
31+
32+
class LoadPlanCollector {
33+
34+
private final LoadPlan.SplitResolver splitResolver;
35+
private boolean finished = false;
36+
private Text lgFirstRow;
37+
private Text lgLastRow;
38+
private Text firstRow;
39+
private Text lastRow;
40+
private Set<KeyExtent> overlappingExtents;
41+
private KeyExtent currentExtent;
42+
private long appended = 0;
43+
44+
LoadPlanCollector(LoadPlan.SplitResolver splitResolver) {
45+
this.splitResolver = splitResolver;
46+
this.overlappingExtents = new HashSet<>();
47+
}
48+
49+
LoadPlanCollector() {
50+
splitResolver = null;
51+
this.overlappingExtents = null;
52+
53+
}
54+
55+
private void appendNoSplits(Key key) {
56+
if (lgFirstRow == null) {
57+
lgFirstRow = key.getRow();
58+
lgLastRow = lgFirstRow;
59+
} else {
60+
var row = key.getRow();
61+
lgLastRow = row;
62+
}
63+
}
64+
65+
private static final TableId FAKE_ID = TableId.of("123");
66+
67+
private void appendSplits(Key key) {
68+
var row = key.getRow();
69+
if (currentExtent == null || !currentExtent.contains(row)) {
70+
var tableSplits = splitResolver.apply(row);
71+
var extent = new KeyExtent(FAKE_ID, tableSplits.getEndRow(), tableSplits.getPrevRow());
72+
Preconditions.checkState(extent.contains(row), "%s does not contain %s", tableSplits, row);
73+
if (currentExtent != null) {
74+
overlappingExtents.add(currentExtent);
75+
}
76+
currentExtent = extent;
77+
}
78+
}
79+
80+
public void append(Key key) {
81+
if (splitResolver == null) {
82+
appendNoSplits(key);
83+
} else {
84+
appendSplits(key);
85+
}
86+
appended++;
87+
}
88+
89+
public void startLocalityGroup() {
90+
if (lgFirstRow != null) {
91+
if (firstRow == null) {
92+
firstRow = lgFirstRow;
93+
lastRow = lgLastRow;
94+
} else {
95+
// take the minimum
96+
firstRow = firstRow.compareTo(lgFirstRow) < 0 ? firstRow : lgFirstRow;
97+
// take the maximum
98+
lastRow = lastRow.compareTo(lgLastRow) > 0 ? lastRow : lgLastRow;
99+
}
100+
lgFirstRow = null;
101+
lgLastRow = null;
102+
}
103+
}
104+
105+
public LoadPlan getLoadPlan(String filename) {
106+
Preconditions.checkState(finished, "Attempted to get load plan before closing");
107+
108+
if (appended == 0) {
109+
return LoadPlan.builder().build();
110+
}
111+
112+
if (splitResolver == null) {
113+
return LoadPlan.builder().loadFileTo(filename, LoadPlan.RangeType.FILE, firstRow, lastRow)
114+
.build();
115+
} else {
116+
var builder = LoadPlan.builder();
117+
overlappingExtents.add(currentExtent);
118+
for (var extent : overlappingExtents) {
119+
builder.loadFileTo(filename, LoadPlan.RangeType.TABLE, extent.prevEndRow(),
120+
extent.endRow());
121+
}
122+
return builder.build();
123+
}
124+
}
125+
126+
public void close() {
127+
finished = true;
128+
// compute the overall min and max rows
129+
startLocalityGroup();
130+
}
131+
}

core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java

+10
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@
3434
import org.apache.accumulo.core.client.summary.Summary.FileStatistics;
3535
import org.apache.accumulo.core.conf.Property;
3636
import org.apache.accumulo.core.data.Key;
37+
import org.apache.accumulo.core.data.LoadPlan;
3738
import org.apache.accumulo.core.data.Range;
3839
import org.apache.accumulo.core.security.Authorizations;
3940
import org.apache.hadoop.fs.FileSystem;
@@ -428,6 +429,15 @@ default WriterOptions withSummarizers(SummarizerConfiguration... summarizerConf)
428429
*/
429430
WriterOptions withVisibilityCacheSize(int maxSize);
430431

432+
/**
433+
* @param splitResolver builds a {@link LoadPlan} using table split points provided by the given
434+
* splitResolver.
435+
* @return this
436+
* @see RFileWriter#getLoadPlan(String)
437+
* @since 2.1.4
438+
*/
439+
WriterOptions withSplitResolver(LoadPlan.SplitResolver splitResolver);
440+
431441
/**
432442
* @return a new RfileWriter created with the options previously specified.
433443
*/

core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriter.java

+34-2
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
import org.apache.accumulo.core.data.ArrayByteSequence;
3030
import org.apache.accumulo.core.data.ByteSequence;
3131
import org.apache.accumulo.core.data.Key;
32+
import org.apache.accumulo.core.data.LoadPlan;
3233
import org.apache.accumulo.core.data.Value;
3334
import org.apache.accumulo.core.file.FileSKVWriter;
3435
import org.apache.accumulo.core.security.ColumnVisibility;
@@ -92,12 +93,15 @@ public class RFileWriter implements AutoCloseable {
9293

9394
private final FileSKVWriter writer;
9495
private final LRUMap<ByteSequence,Boolean> validVisibilities;
96+
97+
private final LoadPlanCollector loadPlanCollector;
9598
private boolean startedLG;
9699
private boolean startedDefaultLG;
97100

98-
RFileWriter(FileSKVWriter fileSKVWriter, int visCacheSize) {
101+
RFileWriter(FileSKVWriter fileSKVWriter, int visCacheSize, LoadPlanCollector loadPlanCollector) {
99102
this.writer = fileSKVWriter;
100103
this.validVisibilities = new LRUMap<>(visCacheSize);
104+
this.loadPlanCollector = loadPlanCollector;
101105
}
102106

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

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

176181
public void startDefaultLocalityGroup() throws IOException {
177182
Preconditions.checkState(!startedDefaultLG);
183+
loadPlanCollector.startLocalityGroup();
178184
writer.startDefaultLocalityGroup();
179185
startedDefaultLG = true;
180186
startedLG = true;
@@ -204,6 +210,7 @@ public void append(Key key, Value val) throws IOException {
204210
validVisibilities.put(new ArrayByteSequence(Arrays.copyOf(cv, cv.length)), Boolean.TRUE);
205211
}
206212
writer.append(key, val);
213+
loadPlanCollector.append(key);
207214
}
208215

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

250257
@Override
251258
public void close() throws IOException {
252-
writer.close();
259+
try {
260+
writer.close();
261+
} finally {
262+
loadPlanCollector.close();
263+
}
264+
}
265+
266+
/**
267+
* If no split resolver was provided when the RFileWriter was built then this method will return a
268+
* simple load plan of type {@link org.apache.accumulo.core.data.LoadPlan.RangeType#FILE} using
269+
* the first and last row seen. If a splitResolver was provided then this will return a load plan
270+
* of type {@link org.apache.accumulo.core.data.LoadPlan.RangeType#TABLE} that has the split
271+
* ranges the rows written overlapped.
272+
*
273+
* @param filename This file name will be used in the load plan and it should match the name that
274+
* will be used when bulk importing this file. Only a filename is needed, not a full path.
275+
* @return load plan computed from the keys written to the rfile.
276+
* @see org.apache.accumulo.core.client.rfile.RFile.WriterOptions#withSplitResolver(LoadPlan.SplitResolver)
277+
* @since 2.1.4
278+
* @throws IllegalStateException is attempting to get load plan before calling {@link #close()}
279+
* @throws IllegalArgumentException is a full path is passed instead of a filename
280+
*/
281+
public LoadPlan getLoadPlan(String filename) {
282+
Preconditions.checkArgument(!filename.contains("/"),
283+
"Unexpected path %s seen instead of file name", filename);
284+
return loadPlanCollector.getLoadPlan(filename);
253285
}
254286
}

core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java

+24-11
Original file line numberDiff line numberDiff line change
@@ -19,14 +19,14 @@
1919
package org.apache.accumulo.core.client.rfile;
2020

2121
import static com.google.common.base.Preconditions.checkArgument;
22+
import static java.util.Objects.requireNonNull;
2223

2324
import java.io.IOException;
2425
import java.io.OutputStream;
2526
import java.util.Collections;
2627
import java.util.HashMap;
2728
import java.util.Map;
2829
import java.util.Map.Entry;
29-
import java.util.Objects;
3030
import java.util.stream.Stream;
3131

3232
import org.apache.accumulo.core.client.rfile.RFile.WriterFSOptions;
@@ -37,6 +37,7 @@
3737
import org.apache.accumulo.core.conf.ConfigurationCopy;
3838
import org.apache.accumulo.core.conf.DefaultConfiguration;
3939
import org.apache.accumulo.core.crypto.CryptoFactoryLoader;
40+
import org.apache.accumulo.core.data.LoadPlan;
4041
import org.apache.accumulo.core.file.FileOperations;
4142
import org.apache.accumulo.core.metadata.ValidationUtil;
4243
import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
@@ -72,6 +73,7 @@ OutputStream getOutputStream() {
7273
private int visCacheSize = 1000;
7374
private Map<String,String> samplerProps = Collections.emptyMap();
7475
private Map<String,String> summarizerProps = Collections.emptyMap();
76+
private LoadPlan.SplitResolver splitResolver;
7577

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

8284
@Override
8385
public WriterOptions withSampler(SamplerConfiguration samplerConf) {
84-
Objects.requireNonNull(samplerConf);
86+
requireNonNull(samplerConf);
8587
Map<String,String> tmp = new SamplerConfigurationImpl(samplerConf).toTablePropertiesMap();
8688
checkDisjoint(tableConfig, tmp, "sampler");
8789
this.samplerProps = tmp;
@@ -106,6 +108,9 @@ public RFileWriter build() throws IOException {
106108
CryptoService cs =
107109
CryptoFactoryLoader.getServiceForClient(CryptoEnvironment.Scope.TABLE, tableConfig);
108110

111+
var loadPlanCollector =
112+
splitResolver == null ? new LoadPlanCollector() : new LoadPlanCollector(splitResolver);
113+
109114
if (out.getOutputStream() != null) {
110115
FSDataOutputStream fsdo;
111116
if (out.getOutputStream() instanceof FSDataOutputStream) {
@@ -116,17 +121,19 @@ public RFileWriter build() throws IOException {
116121
return new RFileWriter(
117122
fileops.newWriterBuilder().forOutputStream(".rf", fsdo, out.getConf(), cs)
118123
.withTableConfiguration(acuconf).withStartDisabled().build(),
119-
visCacheSize);
124+
visCacheSize, loadPlanCollector);
120125
} else {
121-
return new RFileWriter(fileops.newWriterBuilder()
122-
.forFile(out.path.toString(), out.getFileSystem(out.path), out.getConf(), cs)
123-
.withTableConfiguration(acuconf).withStartDisabled().build(), visCacheSize);
126+
return new RFileWriter(
127+
fileops.newWriterBuilder()
128+
.forFile(out.path.toString(), out.getFileSystem(out.path), out.getConf(), cs)
129+
.withTableConfiguration(acuconf).withStartDisabled().build(),
130+
visCacheSize, loadPlanCollector);
124131
}
125132
}
126133

127134
@Override
128135
public WriterOptions withFileSystem(FileSystem fs) {
129-
Objects.requireNonNull(fs);
136+
requireNonNull(fs);
130137
out.fs = fs;
131138
return this;
132139
}
@@ -140,14 +147,14 @@ public WriterFSOptions to(String filename) {
140147

141148
@Override
142149
public WriterOptions to(OutputStream out) {
143-
Objects.requireNonNull(out);
150+
requireNonNull(out);
144151
this.out = new OutputArgs(out);
145152
return this;
146153
}
147154

148155
@Override
149156
public WriterOptions withTableProperties(Iterable<Entry<String,String>> tableConfig) {
150-
Objects.requireNonNull(tableConfig);
157+
requireNonNull(tableConfig);
151158
HashMap<String,String> cfg = new HashMap<>();
152159
for (Entry<String,String> entry : tableConfig) {
153160
cfg.put(entry.getKey(), entry.getValue());
@@ -161,7 +168,7 @@ public WriterOptions withTableProperties(Iterable<Entry<String,String>> tableCon
161168

162169
@Override
163170
public WriterOptions withTableProperties(Map<String,String> tableConfig) {
164-
Objects.requireNonNull(tableConfig);
171+
requireNonNull(tableConfig);
165172
return withTableProperties(tableConfig.entrySet());
166173
}
167174

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

182+
@Override
183+
public WriterOptions withSplitResolver(LoadPlan.SplitResolver splitResolver) {
184+
this.splitResolver = requireNonNull(splitResolver);
185+
return this;
186+
}
187+
175188
@Override
176189
public WriterOptions withSummarizers(SummarizerConfiguration... summarizerConf) {
177-
Objects.requireNonNull(summarizerConf);
190+
requireNonNull(summarizerConf);
178191
Map<String,String> tmp = SummarizerConfiguration.toTableProperties(summarizerConf);
179192
checkDisjoint(tableConfig, tmp, "summarizer");
180193
this.summarizerProps = tmp;

0 commit comments

Comments
 (0)