forked from apache/accumulo
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Add initial TabletMergeability support to client api
This is a follow on PR to apache#5187 for Issue apache#5014 This commit adds the first part of the needed client API changes to support setting TabletMergeability on splits. Two inital updates have been made to support table creation and adding splits later. 1) A new withSplits() method has been added to NewTableConfiguration that takes a map of splits and associated TabletMergeability. The exiting withSplits() method that takes a set just delegates to the new method and sets a default for TabletMergeability to never because by default user created tables should never be automatically mergeable. 2) A new putSplits() method has been added that also takes a map and will create new splits with an associated TabletMergeability. The existing addSplits() method delgates to this as well. Note that this method is only partially done. The intent is to support updating existing splits as well but this will be done in another PR. The support the changes, where necessary the splits are now encoded with the TabletMergeability value as json instead of just passing the split bytes.
- Loading branch information
Showing
24 changed files
with
504 additions
and
110 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
104 changes: 104 additions & 0 deletions
104
core/src/main/java/org/apache/accumulo/core/clientImpl/TabletMergeabilityUtil.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,104 @@ | ||
/* | ||
* 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.clientImpl; | ||
|
||
import static java.nio.charset.StandardCharsets.UTF_8; | ||
|
||
import java.nio.ByteBuffer; | ||
import java.time.Duration; | ||
import java.util.Comparator; | ||
import java.util.Objects; | ||
import java.util.Optional; | ||
import java.util.SortedMap; | ||
import java.util.SortedSet; | ||
import java.util.function.Function; | ||
|
||
import org.apache.accumulo.core.client.admin.TabletMergeability; | ||
import org.apache.accumulo.core.util.ByteBufferUtil; | ||
import org.apache.accumulo.core.util.Pair; | ||
import org.apache.accumulo.core.util.TextUtil; | ||
import org.apache.accumulo.core.util.json.ByteArrayToBase64TypeAdapter; | ||
import org.apache.hadoop.io.Text; | ||
|
||
import com.google.common.base.Preconditions; | ||
import com.google.common.collect.ImmutableSortedMap; | ||
import com.google.gson.Gson; | ||
|
||
public class TabletMergeabilityUtil { | ||
|
||
private static final Gson gson = ByteArrayToBase64TypeAdapter.createBase64Gson(); | ||
|
||
public static SortedMap<Text,TabletMergeability> userDefaultSplits(SortedSet<Text> splits) { | ||
return splitsWithDefault(splits, TabletMergeability.never()); | ||
} | ||
|
||
public static SortedMap<Text,TabletMergeability> systemDefaultSplits(SortedSet<Text> splits) { | ||
return splitsWithDefault(splits, TabletMergeability.always()); | ||
} | ||
|
||
@SuppressWarnings("unchecked") | ||
public static SortedMap<Text,TabletMergeability> splitsWithDefault(SortedSet<Text> splits, | ||
TabletMergeability tabletMergeability) { | ||
Objects.requireNonNull(tabletMergeability); | ||
return splits.stream() | ||
.collect(ImmutableSortedMap.toImmutableSortedMap(Optional | ||
.ofNullable((Comparator<Text>) splits.comparator()).orElse(Comparator.naturalOrder()), | ||
Function.identity(), t -> tabletMergeability)); | ||
} | ||
|
||
public static ByteBuffer encodeAsBuffer(Text split, TabletMergeability tabletMergeability) { | ||
return ByteBuffer.wrap(encode(split, tabletMergeability).getBytes(UTF_8)); | ||
} | ||
|
||
// Encode A split and TabletMergeability as json. The split will be Base64 encoded | ||
public static String encode(Text split, TabletMergeability tabletMergeability) { | ||
GSonData jData = new GSonData(); | ||
jData.split = TextUtil.getBytes(split); | ||
jData.never = tabletMergeability.isNever(); | ||
jData.delay = tabletMergeability.getDelay().map(Duration::toNanos).orElse(null); | ||
return gson.toJson(jData); | ||
} | ||
|
||
public static ByteBuffer encodeAsBuffer(Pair<Text,TabletMergeability> split) { | ||
return ByteBuffer.wrap(encode(split).getBytes(UTF_8)); | ||
} | ||
|
||
public static String encode(Pair<Text,TabletMergeability> split) { | ||
return encode(split.getFirst(), split.getSecond()); | ||
} | ||
|
||
public static Pair<Text,TabletMergeability> decode(ByteBuffer data) { | ||
return decode(ByteBufferUtil.toString(data)); | ||
} | ||
|
||
public static Pair<Text,TabletMergeability> decode(String data) { | ||
GSonData jData = gson.fromJson(data, GSonData.class); | ||
Preconditions.checkArgument(jData.never == (jData.delay == null), | ||
"delay should both be set if and only if mergeability 'never' is false"); | ||
return new Pair<>(new Text(jData.split), jData.never ? TabletMergeability.never() | ||
: TabletMergeability.after(Duration.ofNanos(jData.delay))); | ||
} | ||
|
||
private static class GSonData { | ||
byte[] split; | ||
boolean never; | ||
Long delay; | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.