[10/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java index 95945c6..e1f31bb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java @@ -19,13 +19,12 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; -import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -33,12 +32,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.KeyValue; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection; import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables; /** * Default implementation of StoreFileManager. Not thread-safe. @@ -47,27 +46,27 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; class DefaultStoreFileManager implements StoreFileManager { private static final Log LOG = LogFactory.getLog(DefaultStoreFileManager.class); - private final CellComparator kvComparator; + private final CellComparator cellComparator; private final CompactionConfiguration comConf; private final int blockingFileCount; - private final Comparator storeFileComparator; + private final Comparator storeFileComparator; /** * List of store files inside this store. This is an immutable list that * is atomically replaced when its contents change. */ - private volatile ImmutableList storefiles = null; + private volatile ImmutableList storefiles = ImmutableList.of(); /** * List of compacted files inside this store that needs to be excluded in reads * because further new reads will be using only the newly created files out of compaction. * These compacted files will be deleted/cleared once all the existing readers on these * compacted files are done. */ - private volatile List compactedfiles = null; + private volatile ImmutableList compactedfiles = ImmutableList.of(); - public DefaultStoreFileManager(CellComparator kvComparator, - Comparator storeFileComparator, Configuration conf, + public DefaultStoreFileManager(CellComparator cellComparator, + Comparator storeFileComparator, Configuration conf, CompactionConfiguration comConf) { -this.kvComparator = kvComparator; +this.cellComparator = cellComparator; this.storeFileComparator = storeFileComparator; this.comConf = comConf; this.blockingFileCount = @@ -75,39 +74,37 @@ class DefaultStoreFileManager implements StoreFileManager { } @Override - public void loadFiles(List storeFiles) { -sortAndSetStoreFiles(storeFiles); + public void loadFiles(List storeFiles) { +this.storefiles = ImmutableList.sortedCopyOf(storeFileComparator, storeFiles); } @Override - public final Collection getStorefiles() { -// TODO: I can return a null list of StoreFiles? That'll mess up clients. St.Ack 2015 + public final Collection getStorefiles() { return storefiles; } @Override - public Collection getCompactedfiles() { + public Collection getCompactedfiles() { return compactedfiles; } @Override - public void insertNewFiles(Collection sfs) throws IOException { -ArrayList newFiles = new ArrayList<>(storefiles); -newFiles.addAll(sfs); -sortAndSetStoreFiles(newFiles); + public void insertNewFiles(Collection sfs) throws IOException { +this.storefiles = +ImmutableList.sortedCopyOf(storeFileComparator, Iterables.concat(this.storefiles, sfs)); } @Override - public ImmutableCollection clearFiles() { -ImmutableList result = storefiles; + public ImmutableCollection clearFiles() { +ImmutableList result = storefiles; storefiles = ImmutableList.of(); return result; } @Override - public Collection clearCompactedFiles() { -List result = compactedfiles; -compactedfiles
[04/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java new file mode 100644 index 000..b685115 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java @@ -0,0 +1,1743 @@ +/* + * + * 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 + * + * 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 org.apache.hadoop.hbase.regionserver; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.io.IOException; +import java.lang.ref.SoftReference; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; +import java.util.NavigableSet; +import java.util.TreeSet; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FilterFileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MemoryCompactionPolicy; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.FilterBase; +import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFileContext; +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hadoop.hbase.monitoring.MonitoredTask; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration; +import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor; +import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher; +import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge; +import
[11/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface
HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a5f84430 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a5f84430 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a5f84430 Branch: refs/heads/master Commit: a5f84430a305db6a1eddd45f82d19babf43a8d01 Parents: 1540483 Author: zhangduo <zhang...@apache.org> Authored: Sun Sep 24 19:22:16 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Mon Sep 25 09:35:39 2017 +0800 -- .../hadoop/hbase/backup/util/RestoreTool.java |4 +- .../example/ZooKeeperScanPolicyObserver.java| 68 +- .../hbase/mapreduce/HFileOutputFormat2.java | 30 +- .../hbase/regionserver/CompactionTool.java |6 +- .../hbase/mapreduce/TestHFileOutputFormat2.java |4 +- .../hadoop/hbase/backup/HFileArchiver.java | 30 +- .../hbase/coprocessor/RegionObserver.java |4 +- .../hadoop/hbase/io/HalfStoreFileReader.java| 23 +- .../org/apache/hadoop/hbase/io/hfile/HFile.java | 11 +- .../hbase/io/hfile/HFilePrettyPrinter.java |2 +- .../hadoop/hbase/io/hfile/HFileReaderImpl.java | 36 +- .../assignment/SplitTableRegionProcedure.java | 31 +- .../master/balancer/StochasticLoadBalancer.java |5 +- .../apache/hadoop/hbase/mob/CachedMobFile.java |7 +- .../hbase/mob/DefaultMobStoreCompactor.java |2 +- .../hbase/mob/DefaultMobStoreFlusher.java |9 +- .../org/apache/hadoop/hbase/mob/MobFile.java| 15 +- .../apache/hadoop/hbase/mob/MobStoreEngine.java |8 +- .../org/apache/hadoop/hbase/mob/MobUtils.java | 11 +- .../PartitionedMobCompactionRequest.java| 13 +- .../compactions/PartitionedMobCompactor.java| 58 +- .../regionserver/ChangedReadersObserver.java|4 +- .../hbase/regionserver/CompactingMemStore.java |4 +- .../regionserver/DateTieredStoreEngine.java |8 +- .../hbase/regionserver/DefaultStoreEngine.java | 20 +- .../regionserver/DefaultStoreFileManager.java | 155 +- .../hbase/regionserver/DefaultStoreFlusher.java |2 +- .../hadoop/hbase/regionserver/HMobStore.java|8 +- .../hadoop/hbase/regionserver/HRegion.java | 90 +- .../hbase/regionserver/HRegionFileSystem.java | 44 +- .../hbase/regionserver/HRegionServer.java | 78 +- .../hadoop/hbase/regionserver/HStore.java | 524 +++--- .../hadoop/hbase/regionserver/HStoreFile.java | 170 +- .../hbase/regionserver/MemStoreCompactor.java | 31 +- .../MemStoreCompactorSegmentsIterator.java |8 +- .../hbase/regionserver/MobStoreScanner.java |4 +- .../regionserver/RegionCoprocessorHost.java | 54 +- .../hbase/regionserver/RegionSplitPolicy.java | 15 +- .../regionserver/ReversedMobStoreScanner.java |4 +- .../regionserver/ReversedStoreScanner.java |4 +- .../apache/hadoop/hbase/regionserver/Store.java | 121 +- .../hadoop/hbase/regionserver/StoreEngine.java | 16 +- .../hadoop/hbase/regionserver/StoreFile.java| 126 +- .../regionserver/StoreFileComparators.java | 28 +- .../hbase/regionserver/StoreFileManager.java| 42 +- .../hbase/regionserver/StoreFileReader.java | 46 +- .../hbase/regionserver/StoreFileScanner.java| 24 +- .../hbase/regionserver/StoreFileWriter.java | 42 +- .../hadoop/hbase/regionserver/StoreFlusher.java |6 +- .../hadoop/hbase/regionserver/StoreScanner.java | 29 +- .../hadoop/hbase/regionserver/StoreUtils.java | 86 +- .../hbase/regionserver/StripeStoreEngine.java |8 +- .../regionserver/StripeStoreFileManager.java| 225 +-- .../hbase/regionserver/StripeStoreFlusher.java |6 +- .../AbstractMultiOutputCompactor.java |6 +- .../compactions/CompactionContext.java | 11 +- .../compactions/CompactionPolicy.java |8 +- .../compactions/CompactionRequest.java | 12 +- .../regionserver/compactions/Compactor.java | 26 +- .../compactions/DateTieredCompactionPolicy.java | 38 +- .../DateTieredCompactionRequest.java|4 +- .../compactions/DateTieredCompactor.java|6 +- .../compactions/DefaultCompactor.java | 10 +- .../compactions/ExploringCompactionPolicy.java | 40 +- .../compactions/FIFOCompactionPolicy.java | 29 +- .../compactions/RatioBasedCompactionPolicy.java | 18 +- .../compactions/SortedCompactionPolicy.java | 60 +- .../compactions/StripeCompactionPolicy.java | 92 +- .../compactions/StripeCompactor.java|6 +- .../hadoop/hbase/snapshot/SnapshotManifest.java | 21 +- .../hbase/tool/LoadIncrementalHFiles.java | 25 +- .../hadoop/hbase/util/BloomFilterFactory.java | 14 +- .../org/apache/
[02/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java index 76959c6..6e5aeed 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java @@ -22,7 +22,6 @@ import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_K import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -87,10 +86,10 @@ public class TestStripeStoreFileManager { @Test public void testInsertFilesIntoL0() throws Exception { StripeStoreFileManager manager = createManager(); -MockStoreFile sf = createFile(); +MockHStoreFile sf = createFile(); manager.insertNewFiles(al(sf)); assertEquals(1, manager.getStorefileCount()); -Collection filesForGet = manager.getFilesForScan(KEY_A, true, KEY_A, true); +Collection filesForGet = manager.getFilesForScan(KEY_A, true, KEY_A, true); assertEquals(1, filesForGet.size()); assertTrue(filesForGet.contains(sf)); @@ -109,14 +108,14 @@ public class TestStripeStoreFileManager { manager.addCompactionResults(al(), al(createFile(OPEN_KEY, KEY_B), createFile(KEY_B, OPEN_KEY))); assertEquals(4, manager.getStorefileCount()); -Collection allFiles = manager.clearFiles(); +Collection allFiles = manager.clearFiles(); assertEquals(4, allFiles.size()); assertEquals(0, manager.getStorefileCount()); assertEquals(0, manager.getStorefiles().size()); } - private static ArrayList dumpIterator(Iterator iter) { -ArrayList result = new ArrayList<>(); + private static ArrayList dumpIterator(Iterator iter) { +ArrayList result = new ArrayList<>(); for (; iter.hasNext(); result.add(iter.next())); return result; } @@ -124,23 +123,23 @@ public class TestStripeStoreFileManager { @Test public void testRowKeyBefore() throws Exception { StripeStoreFileManager manager = createManager(); -StoreFile l0File = createFile(), l0File2 = createFile(); +HStoreFile l0File = createFile(), l0File2 = createFile(); manager.insertNewFiles(al(l0File)); manager.insertNewFiles(al(l0File2)); // Get candidate files. -Iterator sfs = manager.getCandidateFilesForRowKeyBefore(KV_B); +Iterator sfs = manager.getCandidateFilesForRowKeyBefore(KV_B); sfs.next(); sfs.remove(); // Suppose we found a candidate in this file... make sure L0 file remaining is not removed. sfs = manager.updateCandidateFilesForRowKeyBefore(sfs, KV_B, KV_A); assertTrue(sfs.hasNext()); // Now add some stripes (remove L0 file too) -MockStoreFile stripe0a = createFile(0, 100, OPEN_KEY, KEY_B), +MockHStoreFile stripe0a = createFile(0, 100, OPEN_KEY, KEY_B), stripe1 = createFile(KEY_B, OPEN_KEY); manager.addCompactionResults(al(l0File), al(stripe0a, stripe1)); manager.removeCompactedFiles(al(l0File)); // If we want a key <= KEY_A, we should get everything except stripe1. -ArrayList sfsDump = dumpIterator(manager.getCandidateFilesForRowKeyBefore(KV_A)); +ArrayList sfsDump = dumpIterator(manager.getCandidateFilesForRowKeyBefore(KV_A)); assertEquals(2, sfsDump.size()); assertTrue(sfsDump.contains(stripe0a)); assertFalse(sfsDump.contains(stripe1)); @@ -162,7 +161,7 @@ public class TestStripeStoreFileManager { // Add one more, later, file to stripe0, remove the last annoying L0 file. // This file should be returned in preference to older L0 file; also, after we get // a candidate from the first file, the old one should not be removed. -StoreFile stripe0b = createFile(0, 101, OPEN_KEY, KEY_B); +HStoreFile stripe0b = createFile(0, 101, OPEN_KEY, KEY_B); manager.addCompactionResults(al(l0File2), al(stripe0b)); manager.removeCompactedFiles(al(l0File2)); sfs = manager.getCandidateFilesForRowKeyBefore(KV_A); @@ -176,24 +175,24 @@ public class TestStripeStoreFileManager { public void testGetSplitPointEdgeCases() throws Exception { StripeStoreFileManager manager = createManager(); // No files => no split. -assertNull(manager.getSplitPoint()); +assertFalse(manager.getSplitPoint().isPresent()); // If there are no stripes, should pick midpoint from the biggest file in L0. -MockStoreFile sf5 = createFile(5, 0); +MockHStoreFile sf5 =
[08/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java index b7e83bf..6cece0f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -28,8 +28,6 @@ import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.yetus.audience.InterfaceStability; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver; @@ -40,10 +38,10 @@ import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; -import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.security.User; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; /** * Interface for objects that hold a column family in a Region. Its a memstore and a set of zero or @@ -63,9 +61,9 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf // General Accessors CellComparator getComparator(); - Collection getStorefiles(); + Collection getStorefiles(); - Collection getCompactedFiles(); + Collection getCompactedFiles(); /** * Close all the readers We don't need to worry about subsequent requests because the Region @@ -73,7 +71,7 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf * @return the {@link StoreFile StoreFiles} that were previously being used. * @throws IOException on failure */ - Collection close() throws IOException; + Collection close() throws IOException; /** * Return a scanner for both the memstore and the HStore files. Assumes we are not in a @@ -86,105 +84,6 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf KeyValueScanner getScanner(Scan scan, final NavigableSettargetCols, long readPt) throws IOException; - /** - * Get all scanners with no filtering based on TTL (that happens further down the line). - * @param cacheBlocks cache the blocks or not - * @param usePread true to use pread, false if not - * @param isCompaction true if the scanner is created for compaction - * @param matcher the scan query matcher - * @param startRow the start row - * @param stopRow the stop row - * @param readPt the read point of the current scan - * @return all scanners for this store - */ - default List getScanners(boolean cacheBlocks, boolean isGet, boolean usePread, - boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt) - throws IOException { -return getScanners(cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow, false, - readPt); - } - - /** - * Get all scanners with no filtering based on TTL (that happens further down the line). - * @param cacheBlocks cache the blocks or not - * @param usePread true to use pread, false if not - * @param isCompaction true if the scanner is created for compaction - * @param matcher the scan query matcher - * @param startRow the start row - * @param includeStartRow true to include start row, false if not - * @param stopRow the stop row - * @param includeStopRow true to include stop row, false if not - * @param readPt the read point of the current scan - * @return all scanners for this store - */ - List getScanners(boolean cacheBlocks, boolean usePread, boolean isCompaction, - ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow, - boolean includeStopRow, long readPt) throws IOException; - - /** - * Recreates the scanners on the current list of active store file scanners - * @param currentFileScanners the current set of active store file scanners - * @param cacheBlocks cache the blocks or not - * @param usePread use pread or not - * @param isCompaction is the scanner for compaction - * @param matcher the scan query matcher - * @param startRow the scan's
[03/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java index 3ba2299..c2c3171 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java @@ -82,12 +82,6 @@ import org.apache.hadoop.hbase.master.assignment.RegionStates; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; -import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController; -import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -109,6 +103,13 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.junit.rules.TestRule; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse; + /** * The below tests are testing split region against a running cluster */ @@ -750,8 +751,8 @@ public class TestSplitTransactionOnCluster { region.put(p); } region.flush(true); - Store store = region.getStore(Bytes.toBytes("f")); - Collection storefiles = store.getStorefiles(); + HStore store = region.getStore(Bytes.toBytes("f")); + Collection storefiles = store.getStorefiles(); assertEquals(storefiles.size(), 1); assertFalse(region.hasReferences()); Path referencePath = http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java deleted file mode 100644 index 2095dcd..000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java +++ /dev/null @@ -1,1740 +0,0 @@ -/* - * - * 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 - * - * 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 org.apache.hadoop.hbase.regionserver; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - -import java.io.IOException; -import java.lang.ref.SoftReference; -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import
[09/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index daad241..de41087 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -31,6 +31,7 @@ import java.util.List; import java.util.Map; import java.util.NavigableSet; import java.util.Optional; +import java.util.OptionalDouble; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.CompletionService; @@ -43,6 +44,9 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Predicate; +import java.util.function.ToLongFunction; +import java.util.stream.Collectors; +import java.util.stream.LongStream; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -171,7 +175,7 @@ public class HStore implements Store { private ScanInfo scanInfo; // TODO: ideally, this should be part of storeFileManager, as we keep passing this to it. - final List filesCompacting = Lists.newArrayList(); + final List filesCompacting = Lists.newArrayList(); // All access must be synchronized. private final Set changedReaderObservers = @@ -335,7 +339,7 @@ public class HStore implements Store { * @param kvComparator KVComparator for storeFileManager. * @return StoreEngine to use. */ - protected StoreEngine createStoreEngine(Store store, Configuration conf, + protected StoreEngine createStoreEngine(HStore store, Configuration conf, CellComparator kvComparator) throws IOException { return StoreEngine.create(store, conf, comparator); } @@ -517,12 +521,12 @@ public class HStore implements Store { * from the given directory. * @throws IOException */ - private List loadStoreFiles() throws IOException { + private List loadStoreFiles() throws IOException { Collection files = fs.getStoreFiles(getColumnFamilyName()); return openStoreFiles(files); } - private List openStoreFiles(Collection files) throws IOException { + private List openStoreFiles(Collection files) throws IOException { if (files == null || files.isEmpty()) { return new ArrayList<>(); } @@ -530,28 +534,21 @@ public class HStore implements Store { ThreadPoolExecutor storeFileOpenerThreadPool = this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpenerThread-" + this.getColumnFamilyName()); -CompletionService completionService = new ExecutorCompletionService<>(storeFileOpenerThreadPool); +CompletionService completionService = new ExecutorCompletionService<>(storeFileOpenerThreadPool); int totalValidStoreFile = 0; -for (final StoreFileInfo storeFileInfo: files) { +for (StoreFileInfo storeFileInfo : files) { // open each store file in parallel - completionService.submit(new Callable() { -@Override -public StoreFile call() throws IOException { - StoreFile storeFile = createStoreFileAndReader(storeFileInfo); - return storeFile; -} - }); + completionService.submit(() -> this.createStoreFileAndReader(storeFileInfo)); totalValidStoreFile++; } -ArrayList results = new ArrayList<>(files.size()); +ArrayList results = new ArrayList<>(files.size()); IOException ioe = null; try { for (int i = 0; i < totalValidStoreFile; i++) { try { - Future future = completionService.take(); - StoreFile storeFile = future.get(); + HStoreFile storeFile = completionService.take().get(); if (storeFile != null) { long length = storeFile.getReader().length(); this.storeSize += length; @@ -574,9 +571,9 @@ public class HStore implements Store { // close StoreFile readers boolean evictOnClose = cacheConf != null? cacheConf.shouldEvictOnClose(): true; - for (StoreFile file : results) { + for (HStoreFile file : results) { try { - if (file != null) file.closeReader(evictOnClose); + if (file != null) file.closeStoreFile(evictOnClose); } catch (IOException e) { LOG.warn(e.getMessage()); } @@ -618,19 +615,18 @@ public class HStore implements Store { */ private void refreshStoreFilesInternal(Collection newFiles) throws IOException { StoreFileManager sfm = storeEngine.getStoreFileManager(); -Collection currentFiles = sfm.getStorefiles(); -Collection compactedFiles = sfm.getCompactedfiles(); +Collection
hbase git commit: HBASE-18160 Fix incorrect logic in FilterList.filterKeyValue
Repository: hbase Updated Branches: refs/heads/branch-2 3e1ceadc8 -> 7c2622baf HBASE-18160 Fix incorrect logic in FilterList.filterKeyValue Signed-off-by: zhangduo <zhang...@apache.org> Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/7c2622ba Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/7c2622ba Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/7c2622ba Branch: refs/heads/branch-2 Commit: 7c2622baf75ac414547488799216cdf2b37be7bd Parents: 3e1cead Author: huzheng <open...@gmail.com> Authored: Thu Jun 8 15:58:42 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Sep 26 11:23:38 2017 +0800 -- .../apache/hadoop/hbase/filter/FilterList.java | 542 --- .../hadoop/hbase/filter/TestFilterList.java | 146 +++-- 2 files changed, 469 insertions(+), 219 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/7c2622ba/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index 1e80a7e..7f2405d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -90,62 +90,53 @@ final public class FilterList extends FilterBase { private Cell transformedCell = null; /** - * Constructor that takes a set of {@link Filter}s. The default operator - * MUST_PASS_ALL is assumed. + * Constructor that takes a set of {@link Filter}s and an operator. + * @param operator Operator to process filter set with. + * @param rowFilters Set of row filters. + */ + public FilterList(final Operator operator, final List rowFilters) { +reversed = checkAndGetReversed(rowFilters, reversed); +this.filters = new ArrayList<>(rowFilters); +this.operator = operator; +initPrevListForMustPassOne(rowFilters.size()); + } + + /** + * Constructor that takes a set of {@link Filter}s. The default operator MUST_PASS_ALL is assumed. * All filters are cloned to internal list. * @param rowFilters list of filters */ public FilterList(final List rowFilters) { -reversed = getReversed(rowFilters, reversed); -this.filters = new ArrayList<>(rowFilters); -initPrevListForMustPassOne(rowFilters.size()); +this(Operator.MUST_PASS_ALL, rowFilters); } /** - * Constructor that takes a var arg number of {@link Filter}s. The fefault operator - * MUST_PASS_ALL is assumed. + * Constructor that takes a var arg number of {@link Filter}s. The default operator MUST_PASS_ALL + * is assumed. * @param rowFilters */ public FilterList(final Filter... rowFilters) { -this(Arrays.asList(rowFilters)); +this(Operator.MUST_PASS_ALL, Arrays.asList(rowFilters)); } /** * Constructor that takes an operator. - * * @param operator Operator to process filter set with. */ public FilterList(final Operator operator) { -this.operator = operator; -this.filters = new ArrayList<>(); -initPrevListForMustPassOne(filters.size()); - } - - /** - * Constructor that takes a set of {@link Filter}s and an operator. - * - * @param operator Operator to process filter set with. - * @param rowFilters Set of row filters. - */ - public FilterList(final Operator operator, final List rowFilters) { -this(rowFilters); -this.operator = operator; -initPrevListForMustPassOne(rowFilters.size()); +this(operator, new ArrayList<>()); } /** * Constructor that takes a var arg number of {@link Filter}s and an operator. - * * @param operator Operator to process filter set with. * @param rowFilters Filters to use */ public FilterList(final Operator operator, final Filter... rowFilters) { -this(rowFilters); -this.operator = operator; -initPrevListForMustPassOne(rowFilters.length); +this(operator, Arrays.asList(rowFilters)); } - public void initPrevListForMustPassOne(int size) { + private void initPrevListForMustPassOne(int size) { if (operator == Operator.MUST_PASS_ONE) { if (this.prevFilterRCList == null) { prevFilterRCList = new ArrayList<>(Collections.nCopies(size, null)); @@ -156,10 +147,8 @@ final public class FilterList extends FilterBase { } } - /** * Get the operator. - * * @return operator */ public Operator getOperator() { @@ -168,7 +157,6 @@ final public class FilterList extends FilterBase { /** * Get the filters. - * * @return filters */ public List getF
hbase git commit: HBASE-18845 TestReplicationSmallTests fails after HBASE-14004
Repository: hbase Updated Branches: refs/heads/branch-2 0658252ed -> 2e4c1b628 HBASE-18845 TestReplicationSmallTests fails after HBASE-14004 Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/2e4c1b62 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/2e4c1b62 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/2e4c1b62 Branch: refs/heads/branch-2 Commit: 2e4c1b62884026ba8fc2d743d33a7f9d9125393e Parents: 0658252 Author: zhangduo <zhang...@apache.org> Authored: Mon Sep 25 12:07:19 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Fri Sep 29 14:32:33 2017 +0800 -- .../replication/TestReplicationSmallTests.java | 115 +-- .../hbase/replication/TestReplicationBase.java | 2 +- 2 files changed, 56 insertions(+), 61 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/2e4c1b62/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java -- diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java index 6105a0d..28bf249 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java @@ -26,7 +26,6 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.NavigableMap; import java.util.TreeMap; @@ -39,13 +38,13 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; @@ -57,10 +56,14 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.client.replication.TableCFs; import org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; +import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL; import org.apache.hadoop.hbase.replication.regionserver.Replication; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSource; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface; @@ -73,8 +76,8 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.WAL; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.mapreduce.Job; import org.junit.Before; import org.junit.Rule; @@ -162,7 +165,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { htable1.put(put); Get get = new Get(row); -get.setMaxVersions(); +get.readAllVersions(); for (int i = 0; i < NB_RETRIES; i++) { if (i==NB_RETRIES-1) { fail("Waited too much time for put replication"); @@ -184,7 +187,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { htable1.delete(d); get = new Get(row); -get.setMaxVersions(); +get.readAllVersions(); for (int i = 0; i < NB_RETRIES; i++) { if (i==NB_RETRIES-1) { fail("Waited too much time for put replication"); @@ -327,7 +330,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { public void testDisableEnable() throws Excepti
hbase git commit: HBASE-18845 TestReplicationSmallTests fails after HBASE-14004
Repository: hbase Updated Branches: refs/heads/master afce850cf -> 239e68726 HBASE-18845 TestReplicationSmallTests fails after HBASE-14004 Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/239e6872 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/239e6872 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/239e6872 Branch: refs/heads/master Commit: 239e6872674ff122ecec2d8d6a557b269e6ae54b Parents: afce850 Author: zhangduo <zhang...@apache.org> Authored: Mon Sep 25 12:07:19 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Fri Sep 29 14:32:26 2017 +0800 -- .../replication/TestReplicationSmallTests.java | 115 +-- .../hbase/replication/TestReplicationBase.java | 2 +- 2 files changed, 56 insertions(+), 61 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/239e6872/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java -- diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java index 6105a0d..28bf249 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java @@ -26,7 +26,6 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.NavigableMap; import java.util.TreeMap; @@ -39,13 +38,13 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; @@ -57,10 +56,14 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.client.replication.TableCFs; import org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; +import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL; import org.apache.hadoop.hbase.replication.regionserver.Replication; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSource; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface; @@ -73,8 +76,8 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.WAL; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.mapreduce.Job; import org.junit.Before; import org.junit.Rule; @@ -162,7 +165,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { htable1.put(put); Get get = new Get(row); -get.setMaxVersions(); +get.readAllVersions(); for (int i = 0; i < NB_RETRIES; i++) { if (i==NB_RETRIES-1) { fail("Waited too much time for put replication"); @@ -184,7 +187,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { htable1.delete(d); get = new Get(row); -get.setMaxVersions(); +get.readAllVersions(); for (int i = 0; i < NB_RETRIES; i++) { if (i==NB_RETRIES-1) { fail("Waited too much time for put replication"); @@ -327,7 +330,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { public void testDisableEnable() throws Excepti
hbase git commit: HBASE-18160 Fix incorrect logic in FilterList.filterKeyValue
Repository: hbase Updated Branches: refs/heads/master 3c7ab8107 -> f54cc1ca5 HBASE-18160 Fix incorrect logic in FilterList.filterKeyValue Signed-off-by: zhangduo <zhang...@apache.org> Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/f54cc1ca Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f54cc1ca Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f54cc1ca Branch: refs/heads/master Commit: f54cc1ca51440ee0f445a008f2f31697730fd7fe Parents: 3c7ab81 Author: huzheng <open...@gmail.com> Authored: Thu Jun 8 15:58:42 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Sep 26 11:46:01 2017 +0800 -- .../apache/hadoop/hbase/filter/FilterList.java | 542 --- .../hadoop/hbase/filter/TestFilterList.java | 146 +++-- 2 files changed, 469 insertions(+), 219 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/f54cc1ca/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index 1e80a7e..7f2405d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -90,62 +90,53 @@ final public class FilterList extends FilterBase { private Cell transformedCell = null; /** - * Constructor that takes a set of {@link Filter}s. The default operator - * MUST_PASS_ALL is assumed. + * Constructor that takes a set of {@link Filter}s and an operator. + * @param operator Operator to process filter set with. + * @param rowFilters Set of row filters. + */ + public FilterList(final Operator operator, final List rowFilters) { +reversed = checkAndGetReversed(rowFilters, reversed); +this.filters = new ArrayList<>(rowFilters); +this.operator = operator; +initPrevListForMustPassOne(rowFilters.size()); + } + + /** + * Constructor that takes a set of {@link Filter}s. The default operator MUST_PASS_ALL is assumed. * All filters are cloned to internal list. * @param rowFilters list of filters */ public FilterList(final List rowFilters) { -reversed = getReversed(rowFilters, reversed); -this.filters = new ArrayList<>(rowFilters); -initPrevListForMustPassOne(rowFilters.size()); +this(Operator.MUST_PASS_ALL, rowFilters); } /** - * Constructor that takes a var arg number of {@link Filter}s. The fefault operator - * MUST_PASS_ALL is assumed. + * Constructor that takes a var arg number of {@link Filter}s. The default operator MUST_PASS_ALL + * is assumed. * @param rowFilters */ public FilterList(final Filter... rowFilters) { -this(Arrays.asList(rowFilters)); +this(Operator.MUST_PASS_ALL, Arrays.asList(rowFilters)); } /** * Constructor that takes an operator. - * * @param operator Operator to process filter set with. */ public FilterList(final Operator operator) { -this.operator = operator; -this.filters = new ArrayList<>(); -initPrevListForMustPassOne(filters.size()); - } - - /** - * Constructor that takes a set of {@link Filter}s and an operator. - * - * @param operator Operator to process filter set with. - * @param rowFilters Set of row filters. - */ - public FilterList(final Operator operator, final List rowFilters) { -this(rowFilters); -this.operator = operator; -initPrevListForMustPassOne(rowFilters.size()); +this(operator, new ArrayList<>()); } /** * Constructor that takes a var arg number of {@link Filter}s and an operator. - * * @param operator Operator to process filter set with. * @param rowFilters Filters to use */ public FilterList(final Operator operator, final Filter... rowFilters) { -this(rowFilters); -this.operator = operator; -initPrevListForMustPassOne(rowFilters.length); +this(operator, Arrays.asList(rowFilters)); } - public void initPrevListForMustPassOne(int size) { + private void initPrevListForMustPassOne(int size) { if (operator == Operator.MUST_PASS_ONE) { if (this.prevFilterRCList == null) { prevFilterRCList = new ArrayList<>(Collections.nCopies(size, null)); @@ -156,10 +147,8 @@ final public class FilterList extends FilterBase { } } - /** * Get the operator. - * * @return operator */ public Operator getOperator() { @@ -168,7 +157,6 @@ final public class FilterList extends FilterBase { /** * Get the filters. - * * @return filters */ public List getF
[1/3] hbase git commit: HBASE-18826 Use HStore instead of Store in our own code base and remove unnecessary methods in Store interface
Repository: hbase Updated Branches: refs/heads/branch-2 f73a3a6fb -> d26b8f8dd http://git-wip-us.apache.org/repos/asf/hbase/blob/d26b8f8d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java index d93152a..7edcf54 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java @@ -99,7 +99,7 @@ public class TestKeepDeletes { // keep 3 versions, rows do not expire HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HConstants.FOREVER, KeepDeletedCells.TRUE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -241,7 +241,7 @@ public class TestKeepDeletes { // KEEP_DELETED_CELLS is NOT enabled HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HConstants.FOREVER, KeepDeletedCells.FALSE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -408,7 +408,7 @@ public class TestKeepDeletes { public void testDeleteMarkerExpirationEmptyStore() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -451,7 +451,7 @@ public class TestKeepDeletes { public void testDeleteMarkerExpiration() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -514,7 +514,7 @@ public class TestKeepDeletes { public void testWithOldRow() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -674,7 +674,7 @@ public class TestKeepDeletes { public void testDeleteMarkerVersioning() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -818,7 +818,7 @@ public class TestKeepDeletes { public void testWithMinVersions() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 3, 1000, 1, KeepDeletedCells.TRUE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime() - 2000; // 2s in the past @@ -897,7 +897,7 @@ public class TestKeepDeletes { public void testWithTTL() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 1, 1000, 1, KeepDeletedCells.TTL); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime() - 2000; // 2s in the past @@ -945,7 +945,7 @@ public class TestKeepDeletes { } - private int countDeleteMarkers(Region region) throws IOException { + private int countDeleteMarkers(HRegion region) throws IOException { Scan s = new Scan(); s.setRaw(true); // use max versions from the store(s) http://git-wip-us.apache.org/repos/asf/hbase/blob/d26b8f8d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java index 0c33bdb..71f18c0 100644 ---
[2/3] hbase git commit: HBASE-18826 Use HStore instead of Store in our own code base and remove unnecessary methods in Store interface
http://git-wip-us.apache.org/repos/asf/hbase/blob/7f4c3b35/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java index d558307..dec28f3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java @@ -53,11 +53,11 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CachedBlock; import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScannerContext; -import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -65,9 +65,9 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; -import org.junit.Ignore; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; @@ -184,8 +184,9 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); - Store store = region.getStores().iterator().next(); + HRegion region = (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName) + .getRegion(regionName); + HStore store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); cacheConf.setCacheDataOnWrite(true); cacheConf.setEvictOnClose(true); @@ -274,8 +275,9 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); - Store store = region.getStores().iterator().next(); + HRegion region = + (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); + HStore store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); cacheConf.setCacheDataOnWrite(true); cacheConf.setEvictOnClose(true); @@ -332,8 +334,9 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); - Store store = region.getStores().iterator().next(); + HRegion region = + (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); + HStore store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); cacheConf.setCacheDataOnWrite(true); cacheConf.setEvictOnClose(true); @@ -393,7 +396,8 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); + HRegion region = + (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); BlockCache cache = setCacheProperties(region); Put put = new Put(ROW); put.addColumn(FAMILY, QUALIFIER, data); @@ -485,7 +489,8 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); -
[3/3] hbase git commit: HBASE-18826 Use HStore instead of Store in our own code base and remove unnecessary methods in Store interface
HBASE-18826 Use HStore instead of Store in our own code base and remove unnecessary methods in Store interface Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/d26b8f8d Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d26b8f8d Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d26b8f8d Branch: refs/heads/branch-2 Commit: d26b8f8dddb19e9d888961a00fa597d7efb9fbd4 Parents: f73a3a6 Author: zhangduo <zhang...@apache.org> Authored: Thu Sep 28 15:24:41 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Thu Sep 28 15:41:56 2017 +0800 -- .../client/ColumnFamilyDescriptorBuilder.java | 5 + .../example/ZooKeeperScanPolicyObserver.java| 2 +- .../hbase/regionserver/CompactionTool.java | 7 +- .../hbase/mapreduce/TestHFileOutputFormat2.java | 6 +- .../regionserver/BusyRegionSplitPolicy.java | 2 +- .../hadoop/hbase/regionserver/CompactSplit.java | 16 +- .../regionserver/CompactedHFilesDischarger.java | 10 +- .../ConstantSizeRegionSplitPolicy.java | 2 +- .../regionserver/FlushLargeStoresPolicy.java| 4 +- .../hadoop/hbase/regionserver/HRegion.java | 34 +-- .../hbase/regionserver/HRegionServer.java | 2 +- .../hadoop/hbase/regionserver/HStore.java | 147 +--- ...IncreasingToUpperBoundRegionSplitPolicy.java | 2 +- .../hbase/regionserver/KeyValueScanner.java | 2 +- .../MetricsRegionServerWrapperImpl.java | 28 ++- .../regionserver/MetricsRegionWrapperImpl.java | 31 ++- .../regionserver/NonLazyKeyValueScanner.java| 2 +- .../hadoop/hbase/regionserver/Region.java | 2 +- .../hbase/regionserver/SegmentScanner.java | 2 +- .../apache/hadoop/hbase/regionserver/Store.java | 239 +++ .../hbase/regionserver/StoreFileManager.java| 2 +- .../hbase/regionserver/StoreFileScanner.java| 2 +- .../hadoop/hbase/regionserver/StoreScanner.java | 2 +- .../hadoop/hbase/regionserver/StoreUtils.java | 14 +- .../compactions/CompactionRequest.java | 5 +- .../compactions/DateTieredCompactor.java| 6 +- ...sureAwareCompactionThroughputController.java | 2 +- .../throttle/ThroughputControlUtil.java | 10 +- .../hbase-webapps/regionserver/region.jsp | 2 +- .../org/apache/hadoop/hbase/TestIOFencing.java | 2 +- .../TestZooKeeperTableArchiveClient.java| 15 +- ...estAvoidCellReferencesIntoShippedBlocks.java | 14 +- .../client/TestBlockEvictionFromClient.java | 72 +++--- .../hadoop/hbase/client/TestFromClientSide.java | 34 +-- .../TestRegionObserverScannerOpenHook.java | 3 +- .../io/hfile/TestForceCacheImportantBlocks.java | 7 +- .../io/hfile/TestScannerFromBucketCache.java| 6 +- .../regionserver/DelegatingKeyValueScanner.java | 2 +- .../regionserver/NoOpScanPolicyObserver.java| 15 +- .../hbase/regionserver/TestAtomicOperation.java | 32 +-- .../regionserver/TestCacheOnWriteInSchema.java | 2 +- .../hbase/regionserver/TestCompaction.java | 12 +- .../TestCompactionFileNotFound.java | 2 +- .../regionserver/TestCompoundBloomFilter.java | 9 +- .../hbase/regionserver/TestHMobStore.java | 4 +- .../hadoop/hbase/regionserver/TestHRegion.java | 32 +-- .../regionserver/TestHRegionReplayEvents.java | 109 - .../hadoop/hbase/regionserver/TestHStore.java | 238 +++--- .../hbase/regionserver/TestHStoreFile.java | 27 +-- .../hbase/regionserver/TestKeepDeletes.java | 18 +- .../hbase/regionserver/TestMajorCompaction.java | 33 ++- .../hbase/regionserver/TestMinorCompaction.java | 11 +- .../regionserver/TestMobStoreCompaction.java| 2 +- .../regionserver/TestPerColumnFamilyFlush.java | 50 ++-- .../regionserver/TestRegionSplitPolicy.java | 2 +- .../TestWalAndCompactingMemStoreFlush.java | 90 +++ .../compactions/TestDateTieredCompactor.java| 3 +- .../compactions/TestFIFOCompactionPolicy.java | 108 - .../TestCompactionWithThroughputController.java | 42 ++-- .../TestFlushWithThroughputController.java | 22 +- .../regionserver/wal/AbstractTestWALReplay.java | 5 +- .../hbase/util/TestCoprocessorScanPolicy.java | 15 +- 62 files changed, 759 insertions(+), 869 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/d26b8f8d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java index 5f4d256..14f7381 100644 --- a/hbase-client/src/m
[3/3] hbase git commit: HBASE-18826 Use HStore instead of Store in our own code base and remove unnecessary methods in Store interface
HBASE-18826 Use HStore instead of Store in our own code base and remove unnecessary methods in Store interface Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/7f4c3b35 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/7f4c3b35 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/7f4c3b35 Branch: refs/heads/master Commit: 7f4c3b3564dfef595780e8b98da5d2dd0b9c5056 Parents: 0cf15fa Author: zhangduo <zhang...@apache.org> Authored: Thu Sep 28 15:24:41 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Thu Sep 28 15:26:52 2017 +0800 -- .../client/ColumnFamilyDescriptorBuilder.java | 5 + .../example/ZooKeeperScanPolicyObserver.java| 2 +- .../hbase/regionserver/CompactionTool.java | 7 +- .../hbase/mapreduce/TestHFileOutputFormat2.java | 6 +- .../regionserver/BusyRegionSplitPolicy.java | 2 +- .../hadoop/hbase/regionserver/CompactSplit.java | 16 +- .../regionserver/CompactedHFilesDischarger.java | 10 +- .../ConstantSizeRegionSplitPolicy.java | 2 +- .../regionserver/FlushLargeStoresPolicy.java| 4 +- .../hadoop/hbase/regionserver/HRegion.java | 34 +-- .../hbase/regionserver/HRegionServer.java | 2 +- .../hadoop/hbase/regionserver/HStore.java | 147 +--- ...IncreasingToUpperBoundRegionSplitPolicy.java | 2 +- .../hbase/regionserver/KeyValueScanner.java | 2 +- .../MetricsRegionServerWrapperImpl.java | 28 ++- .../regionserver/MetricsRegionWrapperImpl.java | 31 ++- .../regionserver/NonLazyKeyValueScanner.java| 2 +- .../hadoop/hbase/regionserver/Region.java | 2 +- .../hbase/regionserver/SegmentScanner.java | 2 +- .../apache/hadoop/hbase/regionserver/Store.java | 239 +++ .../hbase/regionserver/StoreFileManager.java| 2 +- .../hbase/regionserver/StoreFileScanner.java| 2 +- .../hadoop/hbase/regionserver/StoreScanner.java | 2 +- .../hadoop/hbase/regionserver/StoreUtils.java | 14 +- .../compactions/CompactionRequest.java | 5 +- .../compactions/DateTieredCompactor.java| 6 +- ...sureAwareCompactionThroughputController.java | 2 +- .../throttle/ThroughputControlUtil.java | 10 +- .../hbase-webapps/regionserver/region.jsp | 2 +- .../org/apache/hadoop/hbase/TestIOFencing.java | 2 +- .../TestZooKeeperTableArchiveClient.java| 15 +- ...estAvoidCellReferencesIntoShippedBlocks.java | 14 +- .../client/TestBlockEvictionFromClient.java | 72 +++--- .../hadoop/hbase/client/TestFromClientSide.java | 34 +-- .../TestRegionObserverScannerOpenHook.java | 3 +- .../io/hfile/TestForceCacheImportantBlocks.java | 7 +- .../io/hfile/TestScannerFromBucketCache.java| 6 +- .../regionserver/DelegatingKeyValueScanner.java | 2 +- .../regionserver/NoOpScanPolicyObserver.java| 15 +- .../hbase/regionserver/TestAtomicOperation.java | 32 +-- .../regionserver/TestCacheOnWriteInSchema.java | 2 +- .../hbase/regionserver/TestCompaction.java | 12 +- .../TestCompactionFileNotFound.java | 2 +- .../regionserver/TestCompoundBloomFilter.java | 9 +- .../hbase/regionserver/TestHMobStore.java | 4 +- .../hadoop/hbase/regionserver/TestHRegion.java | 32 +-- .../regionserver/TestHRegionReplayEvents.java | 109 - .../hadoop/hbase/regionserver/TestHStore.java | 238 +++--- .../hbase/regionserver/TestHStoreFile.java | 27 +-- .../hbase/regionserver/TestKeepDeletes.java | 18 +- .../hbase/regionserver/TestMajorCompaction.java | 33 ++- .../hbase/regionserver/TestMinorCompaction.java | 11 +- .../regionserver/TestMobStoreCompaction.java| 2 +- .../regionserver/TestPerColumnFamilyFlush.java | 53 ++-- .../regionserver/TestRegionSplitPolicy.java | 2 +- .../TestWalAndCompactingMemStoreFlush.java | 90 +++ .../compactions/TestDateTieredCompactor.java| 3 +- .../compactions/TestFIFOCompactionPolicy.java | 108 - .../TestCompactionWithThroughputController.java | 42 ++-- .../TestFlushWithThroughputController.java | 22 +- .../regionserver/wal/AbstractTestWALReplay.java | 5 +- .../hbase/util/TestCoprocessorScanPolicy.java | 15 +- 62 files changed, 762 insertions(+), 869 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/7f4c3b35/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java index 5f4d256..14f7381 100644 --- a/hbase-client/src/m
[2/3] hbase git commit: HBASE-18826 Use HStore instead of Store in our own code base and remove unnecessary methods in Store interface
http://git-wip-us.apache.org/repos/asf/hbase/blob/d26b8f8d/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java index d558307..dec28f3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java @@ -53,11 +53,11 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CachedBlock; import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScannerContext; -import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -65,9 +65,9 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; -import org.junit.Ignore; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; @@ -184,8 +184,9 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); - Store store = region.getStores().iterator().next(); + HRegion region = (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName) + .getRegion(regionName); + HStore store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); cacheConf.setCacheDataOnWrite(true); cacheConf.setEvictOnClose(true); @@ -274,8 +275,9 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); - Store store = region.getStores().iterator().next(); + HRegion region = + (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); + HStore store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); cacheConf.setCacheDataOnWrite(true); cacheConf.setEvictOnClose(true); @@ -332,8 +334,9 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); - Store store = region.getStores().iterator().next(); + HRegion region = + (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); + HStore store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); cacheConf.setCacheDataOnWrite(true); cacheConf.setEvictOnClose(true); @@ -393,7 +396,8 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); + HRegion region = + (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); BlockCache cache = setCacheProperties(region); Put put = new Put(ROW); put.addColumn(FAMILY, QUALIFIER, data); @@ -485,7 +489,8 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); -
[1/3] hbase git commit: HBASE-18826 Use HStore instead of Store in our own code base and remove unnecessary methods in Store interface
Repository: hbase Updated Branches: refs/heads/master 0cf15fadd -> 7f4c3b356 http://git-wip-us.apache.org/repos/asf/hbase/blob/7f4c3b35/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java index d93152a..7edcf54 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java @@ -99,7 +99,7 @@ public class TestKeepDeletes { // keep 3 versions, rows do not expire HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HConstants.FOREVER, KeepDeletedCells.TRUE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -241,7 +241,7 @@ public class TestKeepDeletes { // KEEP_DELETED_CELLS is NOT enabled HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HConstants.FOREVER, KeepDeletedCells.FALSE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -408,7 +408,7 @@ public class TestKeepDeletes { public void testDeleteMarkerExpirationEmptyStore() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -451,7 +451,7 @@ public class TestKeepDeletes { public void testDeleteMarkerExpiration() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -514,7 +514,7 @@ public class TestKeepDeletes { public void testWithOldRow() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -674,7 +674,7 @@ public class TestKeepDeletes { public void testDeleteMarkerVersioning() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -818,7 +818,7 @@ public class TestKeepDeletes { public void testWithMinVersions() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 3, 1000, 1, KeepDeletedCells.TRUE); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime() - 2000; // 2s in the past @@ -897,7 +897,7 @@ public class TestKeepDeletes { public void testWithTTL() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 1, 1000, 1, KeepDeletedCells.TTL); -Region region = hbu.createLocalHRegion(htd, null, null); +HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime() - 2000; // 2s in the past @@ -945,7 +945,7 @@ public class TestKeepDeletes { } - private int countDeleteMarkers(Region region) throws IOException { + private int countDeleteMarkers(HRegion region) throws IOException { Scan s = new Scan(); s.setRaw(true); // use max versions from the store(s) http://git-wip-us.apache.org/repos/asf/hbase/blob/7f4c3b35/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java index 0c33bdb..71f18c0 100644 ---
[2/2] hbase git commit: HBASE-16324 Remove LegacyScanQueryMatcher
HBASE-16324 Remove LegacyScanQueryMatcher Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/95bc4645 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/95bc4645 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/95bc4645 Branch: refs/heads/branch-2 Commit: 95bc4645271a5fe0c88b4bcce2561da189882034 Parents: b55b952 Author: zhangduo <zhang...@apache.org> Authored: Fri Aug 25 17:02:03 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Sat Aug 26 08:00:20 2017 +0800 -- .../example/ZooKeeperScanPolicyObserver.java| 10 +- .../hbase/mob/DefaultMobStoreCompactor.java | 6 +- .../compactions/PartitionedMobCompactor.java| 8 +- .../hadoop/hbase/regionserver/HMobStore.java| 1 - .../MemStoreCompactorSegmentsIterator.java | 26 +- .../regionserver/ReversedStoreScanner.java | 8 +- .../hadoop/hbase/regionserver/StoreFlusher.java | 10 +- .../hadoop/hbase/regionserver/StoreScanner.java | 217 +++--- .../regionserver/compactions/Compactor.java | 18 +- .../querymatcher/LegacyScanQueryMatcher.java| 384 --- ...estAvoidCellReferencesIntoShippedBlocks.java | 11 +- .../hadoop/hbase/client/TestFromClientSide.java | 4 +- .../TestRegionObserverScannerOpenHook.java | 31 +- .../TestPartitionedMobCompactor.java| 6 +- .../regionserver/NoOpScanPolicyObserver.java| 24 +- .../regionserver/TestCompactingMemStore.java| 34 +- .../hbase/regionserver/TestDefaultMemStore.java | 66 +- .../regionserver/TestMobStoreCompaction.java| 5 +- .../regionserver/TestReversibleScanners.java| 22 +- .../hbase/regionserver/TestStoreScanner.java| 682 +-- .../hbase/util/TestCoprocessorScanPolicy.java | 24 +- 21 files changed, 552 insertions(+), 1045 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/95bc4645/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java -- diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java index 35f85f7..b489fe4 100644 --- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java +++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java @@ -19,9 +19,9 @@ package org.apache.hadoop.hbase.coprocessor.example; import java.io.IOException; -import java.util.Collections; import java.util.List; import java.util.NavigableSet; +import java.util.OptionalInt; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -194,9 +194,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver { // take default action return null; } -Scan scan = new Scan(); -scan.setMaxVersions(scanInfo.getMaxVersions()); -return new StoreScanner(store, scanInfo, scan, scanners, +return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP); } @@ -210,9 +208,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver { // take default action return null; } -Scan scan = new Scan(); -scan.setMaxVersions(scanInfo.getMaxVersions()); -return new StoreScanner(store, scanInfo, scan, scanners, scanType, +return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, scanType, store.getSmallestReadPoint(), earliestPutTs); } http://git-wip-us.apache.org/repos/asf/hbase/blob/95bc4645/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index c475b17..89d2958 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -22,6 +22,7 @@ import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Date; import java.util.List; +import java.util.OptionalInt; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -32,7 +33,6 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueU
[1/2] hbase git commit: HBASE-16324 Remove LegacyScanQueryMatcher
Repository: hbase Updated Branches: refs/heads/branch-2 b55b952d5 -> 95bc46452 http://git-wip-us.apache.org/repos/asf/hbase/blob/95bc4645/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java index 1653728..4082818 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java @@ -19,8 +19,13 @@ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.CellUtil.createCell; +import static org.apache.hadoop.hbase.KeyValueTestUtil.create; import static org.apache.hadoop.hbase.regionserver.KeyValueScanFixture.scanFixture; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; @@ -28,6 +33,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.NavigableSet; +import java.util.OptionalInt; import java.util.TreeSet; import java.util.concurrent.atomic.AtomicInteger; @@ -42,7 +48,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueTestUtil; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.ColumnCountGetFilter; @@ -51,7 +56,6 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdge; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; -import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -66,11 +70,10 @@ public class TestStoreScanner { @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). withLookingForStuckThread(true).build(); private static final String CF_STR = "cf"; - private static final byte [] CF = Bytes.toBytes(CF_STR); + private static final byte[] CF = Bytes.toBytes(CF_STR); static Configuration CONF = HBaseConfiguration.create(); private ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, Integer.MAX_VALUE, Long.MAX_VALUE, KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, CellComparator.COMPARATOR, false); - private ScanType scanType = ScanType.USER_SCAN; /** * From here on down, we have a bunch of defines and specific CELL_GRID of Cells. The @@ -79,15 +82,15 @@ public class TestStoreScanner { * {@link StoreScanner#optimize(org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode, * Cell)} is not overly enthusiastic. */ - private static final byte [] ZERO = new byte [] {'0'}; - private static final byte [] ZERO_POINT_ZERO = new byte [] {'0', '.', '0'}; - private static final byte [] ONE = new byte [] {'1'}; - private static final byte [] TWO = new byte [] {'2'}; - private static final byte [] TWO_POINT_TWO = new byte [] {'2', '.', '2'}; - private static final byte [] THREE = new byte [] {'3'}; - private static final byte [] FOUR = new byte [] {'4'}; - private static final byte [] FIVE = new byte [] {'5'}; - private static final byte [] VALUE = new byte [] {'v'}; + private static final byte[] ZERO = new byte[] {'0'}; + private static final byte[] ZERO_POINT_ZERO = new byte[] {'0', '.', '0'}; + private static final byte[] ONE = new byte[] {'1'}; + private static final byte[] TWO = new byte[] {'2'}; + private static final byte[] TWO_POINT_TWO = new byte[] {'2', '.', '2'}; + private static final byte[] THREE = new byte[] {'3'}; + private static final byte[] FOUR = new byte[] {'4'}; + private static final byte[] FIVE = new byte[] {'5'}; + private static final byte[] VALUE = new byte[] {'v'}; private static final int CELL_GRID_BLOCK2_BOUNDARY = 4; private static final int CELL_GRID_BLOCK3_BOUNDARY = 11; private static final int CELL_GRID_BLOCK4_BOUNDARY = 15; @@ -100,32 +103,32 @@ public class TestStoreScanner { * We will use this to test scan does the right thing as it * we do Gets, StoreScanner#optimize, and what we do on (faked) block boundaries. */ - private static final Cell [] CELL_GRID = new Cell [] { -CellUtil.createCell(ONE, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE), -CellUtil.createCell(ONE, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE), -
[2/2] hbase git commit: HBASE-16324 Remove LegacyScanQueryMatcher
HBASE-16324 Remove LegacyScanQueryMatcher Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/8d33949b Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/8d33949b Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/8d33949b Branch: refs/heads/master Commit: 8d33949b8db072902783f63cd9aaa68cbd6b905f Parents: 2773510 Author: zhangduo <zhang...@apache.org> Authored: Fri Aug 25 17:02:03 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Sat Aug 26 08:04:43 2017 +0800 -- .../example/ZooKeeperScanPolicyObserver.java| 10 +- .../hbase/mob/DefaultMobStoreCompactor.java | 6 +- .../compactions/PartitionedMobCompactor.java| 8 +- .../hadoop/hbase/regionserver/HMobStore.java| 1 - .../MemStoreCompactorSegmentsIterator.java | 26 +- .../regionserver/ReversedStoreScanner.java | 8 +- .../hadoop/hbase/regionserver/StoreFlusher.java | 10 +- .../hadoop/hbase/regionserver/StoreScanner.java | 217 +++--- .../regionserver/compactions/Compactor.java | 18 +- .../querymatcher/LegacyScanQueryMatcher.java| 384 --- ...estAvoidCellReferencesIntoShippedBlocks.java | 11 +- .../hadoop/hbase/client/TestFromClientSide.java | 4 +- .../TestRegionObserverScannerOpenHook.java | 31 +- .../TestPartitionedMobCompactor.java| 6 +- .../regionserver/NoOpScanPolicyObserver.java| 24 +- .../regionserver/TestCompactingMemStore.java| 34 +- .../hbase/regionserver/TestDefaultMemStore.java | 66 +- .../regionserver/TestMobStoreCompaction.java| 5 +- .../regionserver/TestReversibleScanners.java| 22 +- .../hbase/regionserver/TestStoreScanner.java| 682 +-- .../hbase/util/TestCoprocessorScanPolicy.java | 24 +- 21 files changed, 552 insertions(+), 1045 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/8d33949b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java -- diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java index 35f85f7..b489fe4 100644 --- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java +++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java @@ -19,9 +19,9 @@ package org.apache.hadoop.hbase.coprocessor.example; import java.io.IOException; -import java.util.Collections; import java.util.List; import java.util.NavigableSet; +import java.util.OptionalInt; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -194,9 +194,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver { // take default action return null; } -Scan scan = new Scan(); -scan.setMaxVersions(scanInfo.getMaxVersions()); -return new StoreScanner(store, scanInfo, scan, scanners, +return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP); } @@ -210,9 +208,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver { // take default action return null; } -Scan scan = new Scan(); -scan.setMaxVersions(scanInfo.getMaxVersions()); -return new StoreScanner(store, scanInfo, scan, scanners, scanType, +return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, scanType, store.getSmallestReadPoint(), earliestPutTs); } http://git-wip-us.apache.org/repos/asf/hbase/blob/8d33949b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index c475b17..89d2958 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -22,6 +22,7 @@ import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Date; import java.util.List; +import java.util.OptionalInt; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -32,7 +33,6 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueU
[1/2] hbase git commit: HBASE-16324 Remove LegacyScanQueryMatcher
Repository: hbase Updated Branches: refs/heads/master 2773510f1 -> 8d33949b8 http://git-wip-us.apache.org/repos/asf/hbase/blob/8d33949b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java index 1653728..4082818 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java @@ -19,8 +19,13 @@ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.CellUtil.createCell; +import static org.apache.hadoop.hbase.KeyValueTestUtil.create; import static org.apache.hadoop.hbase.regionserver.KeyValueScanFixture.scanFixture; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; @@ -28,6 +33,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.NavigableSet; +import java.util.OptionalInt; import java.util.TreeSet; import java.util.concurrent.atomic.AtomicInteger; @@ -42,7 +48,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueTestUtil; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.ColumnCountGetFilter; @@ -51,7 +56,6 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdge; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; -import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -66,11 +70,10 @@ public class TestStoreScanner { @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). withLookingForStuckThread(true).build(); private static final String CF_STR = "cf"; - private static final byte [] CF = Bytes.toBytes(CF_STR); + private static final byte[] CF = Bytes.toBytes(CF_STR); static Configuration CONF = HBaseConfiguration.create(); private ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, Integer.MAX_VALUE, Long.MAX_VALUE, KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, CellComparator.COMPARATOR, false); - private ScanType scanType = ScanType.USER_SCAN; /** * From here on down, we have a bunch of defines and specific CELL_GRID of Cells. The @@ -79,15 +82,15 @@ public class TestStoreScanner { * {@link StoreScanner#optimize(org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode, * Cell)} is not overly enthusiastic. */ - private static final byte [] ZERO = new byte [] {'0'}; - private static final byte [] ZERO_POINT_ZERO = new byte [] {'0', '.', '0'}; - private static final byte [] ONE = new byte [] {'1'}; - private static final byte [] TWO = new byte [] {'2'}; - private static final byte [] TWO_POINT_TWO = new byte [] {'2', '.', '2'}; - private static final byte [] THREE = new byte [] {'3'}; - private static final byte [] FOUR = new byte [] {'4'}; - private static final byte [] FIVE = new byte [] {'5'}; - private static final byte [] VALUE = new byte [] {'v'}; + private static final byte[] ZERO = new byte[] {'0'}; + private static final byte[] ZERO_POINT_ZERO = new byte[] {'0', '.', '0'}; + private static final byte[] ONE = new byte[] {'1'}; + private static final byte[] TWO = new byte[] {'2'}; + private static final byte[] TWO_POINT_TWO = new byte[] {'2', '.', '2'}; + private static final byte[] THREE = new byte[] {'3'}; + private static final byte[] FOUR = new byte[] {'4'}; + private static final byte[] FIVE = new byte[] {'5'}; + private static final byte[] VALUE = new byte[] {'v'}; private static final int CELL_GRID_BLOCK2_BOUNDARY = 4; private static final int CELL_GRID_BLOCK3_BOUNDARY = 11; private static final int CELL_GRID_BLOCK4_BOUNDARY = 15; @@ -100,32 +103,32 @@ public class TestStoreScanner { * We will use this to test scan does the right thing as it * we do Gets, StoreScanner#optimize, and what we do on (faked) block boundaries. */ - private static final Cell [] CELL_GRID = new Cell [] { -CellUtil.createCell(ONE, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE), -CellUtil.createCell(ONE, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE), -
hbase git commit: HBASE-18347 Implement a BufferedMutator for async client
Repository: hbase Updated Branches: refs/heads/master 25ee5f7f8 -> d12eb7a4a HBASE-18347 Implement a BufferedMutator for async client Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/d12eb7a4 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d12eb7a4 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d12eb7a4 Branch: refs/heads/master Commit: d12eb7a4aae5c2dc7b230bf2a12d2313b93b8ba9 Parents: 25ee5f7 Author: zhangduo <zhang...@apache.org> Authored: Mon Aug 21 18:37:26 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Thu Aug 24 15:55:43 2017 +0800 -- .../hbase/client/AsyncBufferedMutator.java | 84 +++ .../client/AsyncBufferedMutatorBuilder.java | 85 +++ .../client/AsyncBufferedMutatorBuilderImpl.java | 85 +++ .../hbase/client/AsyncBufferedMutatorImpl.java | 144 +++ .../hadoop/hbase/client/AsyncConnection.java| 39 + .../client/AsyncConnectionConfiguration.java| 9 ++ .../hbase/client/AsyncConnectionImpl.java | 11 ++ .../hbase/client/TestAsyncBufferMutator.java| 128 + 8 files changed, 585 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/d12eb7a4/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java new file mode 100644 index 000..ad9279b --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java @@ -0,0 +1,84 @@ +/** + * 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 + * + * 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 org.apache.hadoop.hbase.client; + +import java.io.Closeable; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Used to communicate with a single HBase table in batches. Obtain an instance from a + * {@link AsyncConnection} and call {@link #close()} afterwards. + * + * The implementation is required to be thread safe. + */ +@InterfaceAudience.Public +public interface AsyncBufferedMutator extends Closeable { + + /** + * Gets the fully qualified table name instance of the table that this + * {@code AsyncBufferedMutator} writes to. + */ + TableName getName(); + + /** + * Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance. + * + * The reference returned is not a copy, so any change made to it will affect this instance. + */ + Configuration getConfiguration(); + + /** + * Sends a {@link Mutation} to the table. The mutations will be buffered and sent over the wire as + * part of a batch. Currently only supports {@link Put} and {@link Delete} mutations. + * @param mutation The data to send. + */ + CompletableFuture mutate(Mutation mutation); + + /** + * Send some {@link Mutation}s to the table. The mutations will be buffered and sent over the wire + * as part of a batch. There is no guarantee of sending entire content of {@code mutations} in a + * single batch, the implementations are free to break it up according to the write buffer + * capacity. + * @param mutations The data to send. + */ + List<CompletableFuture> mutate(List mutations); + + /** + * Executes all the buffered, asynchronous operations. + */ + void flush(); + + /** + * Performs a {@link #flush()} and releases any resources held. + */ + @Override + void close(); + + /** + * Returns the maximum size in bytes of the write buffer. + * + * The default value comes from the configuration parameter {@code hbase.client.write.buffer}. + * @return The size of the write buffer in bytes. + */ + long getWriteBuffe
hbase git commit: HBASE-18347 Implement a BufferedMutator for async client
Repository: hbase Updated Branches: refs/heads/branch-2 45b20da23 -> 1ae9a3901 HBASE-18347 Implement a BufferedMutator for async client Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/1ae9a390 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/1ae9a390 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/1ae9a390 Branch: refs/heads/branch-2 Commit: 1ae9a39011f2ee2dfe013a32a990cff34f6ea428 Parents: 45b20da Author: zhangduo <zhang...@apache.org> Authored: Mon Aug 21 18:37:26 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Thu Aug 24 16:01:30 2017 +0800 -- .../hbase/client/AsyncBufferedMutator.java | 84 +++ .../client/AsyncBufferedMutatorBuilder.java | 85 +++ .../client/AsyncBufferedMutatorBuilderImpl.java | 85 +++ .../hbase/client/AsyncBufferedMutatorImpl.java | 144 +++ .../hadoop/hbase/client/AsyncConnection.java| 39 + .../client/AsyncConnectionConfiguration.java| 9 ++ .../hbase/client/AsyncConnectionImpl.java | 11 ++ .../hbase/client/TestAsyncBufferMutator.java| 128 + 8 files changed, 585 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/1ae9a390/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java new file mode 100644 index 000..ad9279b --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java @@ -0,0 +1,84 @@ +/** + * 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 + * + * 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 org.apache.hadoop.hbase.client; + +import java.io.Closeable; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Used to communicate with a single HBase table in batches. Obtain an instance from a + * {@link AsyncConnection} and call {@link #close()} afterwards. + * + * The implementation is required to be thread safe. + */ +@InterfaceAudience.Public +public interface AsyncBufferedMutator extends Closeable { + + /** + * Gets the fully qualified table name instance of the table that this + * {@code AsyncBufferedMutator} writes to. + */ + TableName getName(); + + /** + * Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance. + * + * The reference returned is not a copy, so any change made to it will affect this instance. + */ + Configuration getConfiguration(); + + /** + * Sends a {@link Mutation} to the table. The mutations will be buffered and sent over the wire as + * part of a batch. Currently only supports {@link Put} and {@link Delete} mutations. + * @param mutation The data to send. + */ + CompletableFuture mutate(Mutation mutation); + + /** + * Send some {@link Mutation}s to the table. The mutations will be buffered and sent over the wire + * as part of a batch. There is no guarantee of sending entire content of {@code mutations} in a + * single batch, the implementations are free to break it up according to the write buffer + * capacity. + * @param mutations The data to send. + */ + List<CompletableFuture> mutate(List mutations); + + /** + * Executes all the buffered, asynchronous operations. + */ + void flush(); + + /** + * Performs a {@link #flush()} and releases any resources held. + */ + @Override + void close(); + + /** + * Returns the maximum size in bytes of the write buffer. + * + * The default value comes from the configuration parameter {@code hbase.client.write.buffer}. + * @return The size of the write buffer in bytes. + */ + long
[4/4] hbase git commit: HBASE-18699 Copy LoadIncrementalHFiles to another package and mark the old one as deprecated
HBASE-18699 Copy LoadIncrementalHFiles to another package and mark the old one as deprecated Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/9e53f292 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/9e53f292 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/9e53f292 Branch: refs/heads/master Commit: 9e53f2927b3154eb703560933ddad489c2e232b5 Parents: 7c51d3f Author: zhangduo <zhang...@apache.org> Authored: Fri Sep 1 20:27:16 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Sun Sep 3 19:49:42 2017 +0800 -- .../hbase/backup/impl/RestoreTablesClient.java |6 +- .../backup/mapreduce/MapReduceRestoreJob.java |2 +- .../hadoop/hbase/backup/util/BackupUtils.java |2 +- .../hadoop/hbase/backup/util/RestoreTool.java |2 +- .../hadoop/hbase/backup/TestBackupBase.java |2 +- .../TestIncrementalBackupWithBulkLoad.java |5 +- .../client/ColumnFamilyDescriptorBuilder.java | 10 +- .../hbase/coprocessor/TestSecureExport.java |2 +- ...ReplicationSyncUpToolWithBulkLoadedData.java |2 +- .../mapreduce/IntegrationTestBulkLoad.java | 23 +- .../mapreduce/IntegrationTestImportTsv.java |5 +- .../hadoop/hbase/mapreduce/CopyTable.java |1 + .../apache/hadoop/hbase/mapreduce/Driver.java |1 + .../hbase/mapreduce/HRegionPartitioner.java |2 +- ...opSecurityEnabledUserProviderForTesting.java | 41 - .../hbase/mapreduce/TestHFileOutputFormat2.java |6 +- .../TestLoadIncrementalHFilesSplitRecovery.java | 669 - .../TestSecureLoadIncrementalHFiles.java| 70 - ...ecureLoadIncrementalHFilesSplitRecovery.java | 69 - .../snapshot/TestMobSecureExportSnapshot.java |2 +- .../snapshot/TestSecureExportSnapshot.java |2 +- .../hbase/mapreduce/LoadIncrementalHFiles.java | 1284 +- .../compactions/PartitionedMobCompactor.java|2 +- .../regionserver/HFileReplicator.java |4 +- .../hbase/tool/LoadIncrementalHFiles.java | 1251 + .../org/apache/hadoop/hbase/util/HBaseFsck.java |2 +- .../TestRegionObserverInterface.java|5 +- .../mapreduce/TestLoadIncrementalHFiles.java| 763 --- .../regionserver/TestScannerWithBulkload.java |2 +- .../replication/TestMasterReplication.java |2 +- ...opSecurityEnabledUserProviderForTesting.java | 41 + .../security/access/TestAccessController.java | 19 +- .../hadoop/hbase/tool/MapreduceTestingShim.java | 171 +++ .../hbase/tool/TestLoadIncrementalHFiles.java | 723 ++ .../TestLoadIncrementalHFilesSplitRecovery.java | 628 + .../tool/TestSecureLoadIncrementalHFiles.java | 66 + ...ecureLoadIncrementalHFilesSplitRecovery.java | 66 + .../spark/IntegrationTestSparkBulkLoad.java |2 +- .../hbasecontext/JavaHBaseBulkLoadExample.java |2 +- .../hbase/spark/TestJavaHBaseContext.java |2 +- .../hadoop/hbase/spark/BulkLoadSuite.scala |2 +- src/main/asciidoc/_chapters/ops_mgt.adoc|2 +- 42 files changed, 3041 insertions(+), 2922 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/9e53f292/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java -- diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java index ea7a7b8..ff79533 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java @@ -44,8 +44,8 @@ import org.apache.hadoop.hbase.backup.util.RestoreTool; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles; -import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles.LoadQueueItem; +import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; +import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem; /** * Restore table implementation @@ -231,7 +231,7 @@ public class RestoreTablesClient { LoadIncrementalHFiles loader = BackupUtils.createLoader(conf); for (int i = 0; i < sTableList.size(); i++) { if (mapForSrc[i] != null && !mapForSrc[i].isEmpty()) { -loaderResult = loader.run(null, mapForSrc[i], tTableArray[i]); +loaderResult = loader.run(mapForSrc[i], tTableArray[i]); LOG.debug("bulk
[2/4] hbase git commit: HBASE-18699 Copy LoadIncrementalHFiles to another package and mark the old one as deprecated
http://git-wip-us.apache.org/repos/asf/hbase/blob/9e53f292/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java new file mode 100644 index 000..1f27d04 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java @@ -0,0 +1,1251 @@ +/** + * 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 + * + * 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 org.apache.hadoop.hbase.tool; + +import static java.lang.String.format; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Deque; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import org.apache.commons.lang.mutable.MutableInt; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ClientServiceCallable; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; +import org.apache.hadoop.hbase.client.SecureBulkLoadClient; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.io.HFileLink; +import org.apache.hadoop.hbase.io.HalfStoreFileReader; +import org.apache.hadoop.hbase.io.Reference; +import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFileContext; +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; +import org.apache.hadoop.hbase.io.hfile.HFileScanner; +import org.apache.hadoop.hbase.ipc.RpcControllerFactory; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; +import org.apache.hadoop.hbase.security.UserProvider; +import org.apache.hadoop.hbase.security.token.FsDelegationToken; +import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.HashMultimap; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap; +import
[3/4] hbase git commit: HBASE-18699 Copy LoadIncrementalHFiles to another package and mark the old one as deprecated
http://git-wip-us.apache.org/repos/asf/hbase/blob/9e53f292/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java index 7b4a353..285530d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java @@ -18,1288 +18,60 @@ */ package org.apache.hadoop.hbase.mapreduce; -import static java.lang.String.format; - -import org.apache.hadoop.hbase.shaded.com.google.common.collect.HashMultimap; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimaps; -import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder; - -import java.io.FileNotFoundException; import java.io.IOException; -import java.io.InterruptedIOException; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Deque; import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.TreeMap; -import java.util.UUID; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.lang.mutable.MutableInt; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.ClientServiceCallable; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.RegionLocator; -import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; -import org.apache.hadoop.hbase.client.SecureBulkLoadClient; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; -import org.apache.hadoop.hbase.io.HFileLink; -import org.apache.hadoop.hbase.io.HalfStoreFileReader; -import org.apache.hadoop.hbase.io.Reference; -import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.hfile.HFile; -import org.apache.hadoop.hbase.io.hfile.HFileContext; -import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; -import org.apache.hadoop.hbase.io.hfile.HFileScanner; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hadoop.hbase.regionserver.StoreFileWriter; -import org.apache.hadoop.hbase.security.UserProvider; -import org.apache.hadoop.hbase.security.token.FsDelegationToken; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSHDFSUtils; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.util.Tool; -import org.apache.hadoop.util.ToolRunner; /** * Tool to load the output of HFileOutputFormat into an existing table. + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use + * {@link org.apache.hadoop.hbase.tool.LoadIncrementalHFiles} instead. */ +@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS", +justification = "Temporary glue. To be removed") +@Deprecated @InterfaceAudience.Public -public class LoadIncrementalHFiles extends Configured implements Tool { - private static final Log LOG
[4/4] hbase git commit: HBASE-18699 Copy LoadIncrementalHFiles to another package and mark the old one as deprecated
HBASE-18699 Copy LoadIncrementalHFiles to another package and mark the old one as deprecated Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a37417c2 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a37417c2 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a37417c2 Branch: refs/heads/branch-2 Commit: a37417c25414e37cb719c69867fc8be11b0b94f4 Parents: 49986e9 Author: zhangduo <zhang...@apache.org> Authored: Fri Sep 1 20:27:16 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Sun Sep 3 19:49:47 2017 +0800 -- .../hbase/backup/impl/RestoreTablesClient.java |6 +- .../backup/mapreduce/MapReduceRestoreJob.java |2 +- .../hadoop/hbase/backup/util/BackupUtils.java |2 +- .../hadoop/hbase/backup/util/RestoreTool.java |2 +- .../hadoop/hbase/backup/TestBackupBase.java |2 +- .../TestIncrementalBackupWithBulkLoad.java |5 +- .../client/ColumnFamilyDescriptorBuilder.java | 10 +- .../hbase/coprocessor/TestSecureExport.java |2 +- ...ReplicationSyncUpToolWithBulkLoadedData.java |2 +- .../mapreduce/IntegrationTestBulkLoad.java | 23 +- .../mapreduce/IntegrationTestImportTsv.java |5 +- .../hadoop/hbase/mapreduce/CopyTable.java |1 + .../apache/hadoop/hbase/mapreduce/Driver.java |1 + .../hbase/mapreduce/HRegionPartitioner.java |2 +- ...opSecurityEnabledUserProviderForTesting.java | 41 - .../hbase/mapreduce/TestHFileOutputFormat2.java |6 +- .../TestLoadIncrementalHFilesSplitRecovery.java | 669 - .../TestSecureLoadIncrementalHFiles.java| 70 - ...ecureLoadIncrementalHFilesSplitRecovery.java | 69 - .../snapshot/TestMobSecureExportSnapshot.java |2 +- .../snapshot/TestSecureExportSnapshot.java |2 +- .../hbase/mapreduce/LoadIncrementalHFiles.java | 1284 +- .../compactions/PartitionedMobCompactor.java|2 +- .../regionserver/HFileReplicator.java |4 +- .../hbase/tool/LoadIncrementalHFiles.java | 1251 + .../org/apache/hadoop/hbase/util/HBaseFsck.java |2 +- .../TestRegionObserverInterface.java|5 +- .../mapreduce/TestLoadIncrementalHFiles.java| 763 --- .../regionserver/TestScannerWithBulkload.java |2 +- .../replication/TestMasterReplication.java |2 +- ...opSecurityEnabledUserProviderForTesting.java | 41 + .../security/access/TestAccessController.java | 19 +- .../hadoop/hbase/tool/MapreduceTestingShim.java | 171 +++ .../hbase/tool/TestLoadIncrementalHFiles.java | 723 ++ .../TestLoadIncrementalHFilesSplitRecovery.java | 628 + .../tool/TestSecureLoadIncrementalHFiles.java | 66 + ...ecureLoadIncrementalHFilesSplitRecovery.java | 66 + .../spark/IntegrationTestSparkBulkLoad.java |2 +- .../hbasecontext/JavaHBaseBulkLoadExample.java |2 +- .../hbase/spark/TestJavaHBaseContext.java |2 +- .../hadoop/hbase/spark/BulkLoadSuite.scala |2 +- src/main/asciidoc/_chapters/ops_mgt.adoc|2 +- 42 files changed, 3041 insertions(+), 2922 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/a37417c2/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java -- diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java index ea7a7b8..ff79533 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java @@ -44,8 +44,8 @@ import org.apache.hadoop.hbase.backup.util.RestoreTool; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles; -import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles.LoadQueueItem; +import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; +import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem; /** * Restore table implementation @@ -231,7 +231,7 @@ public class RestoreTablesClient { LoadIncrementalHFiles loader = BackupUtils.createLoader(conf); for (int i = 0; i < sTableList.size(); i++) { if (mapForSrc[i] != null && !mapForSrc[i].isEmpty()) { -loaderResult = loader.run(null, mapForSrc[i], tTableArray[i]); +loaderResult = loader.run(mapForSrc[i], tTableArray[i]); LOG.debug("bulk
[1/4] hbase git commit: HBASE-18699 Copy LoadIncrementalHFiles to another package and mark the old one as deprecated
Repository: hbase Updated Branches: refs/heads/master 7c51d3f2e -> 9e53f2927 http://git-wip-us.apache.org/repos/asf/hbase/blob/9e53f292/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java index 0fe79d1..199c2c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java @@ -60,13 +60,13 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.regionserver.TestSourceFSConfigurationProvider; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.HFileTestUtil; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; http://git-wip-us.apache.org/repos/asf/hbase/blob/9e53f292/hbase-server/src/test/java/org/apache/hadoop/hbase/security/HadoopSecurityEnabledUserProviderForTesting.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/HadoopSecurityEnabledUserProviderForTesting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/HadoopSecurityEnabledUserProviderForTesting.java new file mode 100644 index 000..3ebda29 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/HadoopSecurityEnabledUserProviderForTesting.java @@ -0,0 +1,41 @@ +/** + * 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 + * + * 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 org.apache.hadoop.hbase.security; + +import org.apache.hadoop.hbase.security.UserProvider; + +/** + * A {@link UserProvider} that always says hadoop security is enabled, regardless of the underlying + * configuration. HBase security is not enabled as this is used to determine if SASL is used + * to do the authentication, which requires a Kerberos ticket (which we currently don't have in + * tests). + * + * This should only be used for TESTING. + */ +public class HadoopSecurityEnabledUserProviderForTesting extends UserProvider { + + @Override + public boolean isHBaseSecurityEnabled() { +return false; + } + + @Override + public boolean isHadoopSecurityEnabled() { +return true; + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hbase/blob/9e53f292/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index 6583366..1e38179 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -25,6 +25,12 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.protobuf.BlockingRpcChannel; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; +import com.google.protobuf.ServiceException; + import java.io.IOException; import
[2/4] hbase git commit: HBASE-18699 Copy LoadIncrementalHFiles to another package and mark the old one as deprecated
http://git-wip-us.apache.org/repos/asf/hbase/blob/a37417c2/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java new file mode 100644 index 000..1f27d04 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java @@ -0,0 +1,1251 @@ +/** + * 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 + * + * 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 org.apache.hadoop.hbase.tool; + +import static java.lang.String.format; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Deque; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import org.apache.commons.lang.mutable.MutableInt; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ClientServiceCallable; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; +import org.apache.hadoop.hbase.client.SecureBulkLoadClient; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.io.HFileLink; +import org.apache.hadoop.hbase.io.HalfStoreFileReader; +import org.apache.hadoop.hbase.io.Reference; +import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFileContext; +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; +import org.apache.hadoop.hbase.io.hfile.HFileScanner; +import org.apache.hadoop.hbase.ipc.RpcControllerFactory; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; +import org.apache.hadoop.hbase.security.UserProvider; +import org.apache.hadoop.hbase.security.token.FsDelegationToken; +import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.HashMultimap; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap; +import
[3/4] hbase git commit: HBASE-18699 Copy LoadIncrementalHFiles to another package and mark the old one as deprecated
http://git-wip-us.apache.org/repos/asf/hbase/blob/a37417c2/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java index 7b4a353..285530d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java @@ -18,1288 +18,60 @@ */ package org.apache.hadoop.hbase.mapreduce; -import static java.lang.String.format; - -import org.apache.hadoop.hbase.shaded.com.google.common.collect.HashMultimap; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimaps; -import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder; - -import java.io.FileNotFoundException; import java.io.IOException; -import java.io.InterruptedIOException; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Deque; import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.TreeMap; -import java.util.UUID; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.lang.mutable.MutableInt; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.ClientServiceCallable; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.RegionLocator; -import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; -import org.apache.hadoop.hbase.client.SecureBulkLoadClient; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; -import org.apache.hadoop.hbase.io.HFileLink; -import org.apache.hadoop.hbase.io.HalfStoreFileReader; -import org.apache.hadoop.hbase.io.Reference; -import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.hfile.HFile; -import org.apache.hadoop.hbase.io.hfile.HFileContext; -import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; -import org.apache.hadoop.hbase.io.hfile.HFileScanner; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hadoop.hbase.regionserver.StoreFileWriter; -import org.apache.hadoop.hbase.security.UserProvider; -import org.apache.hadoop.hbase.security.token.FsDelegationToken; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSHDFSUtils; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.util.Tool; -import org.apache.hadoop.util.ToolRunner; /** * Tool to load the output of HFileOutputFormat into an existing table. + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use + * {@link org.apache.hadoop.hbase.tool.LoadIncrementalHFiles} instead. */ +@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS", +justification = "Temporary glue. To be removed") +@Deprecated @InterfaceAudience.Public -public class LoadIncrementalHFiles extends Configured implements Tool { - private static final Log LOG
[1/4] hbase git commit: HBASE-18699 Copy LoadIncrementalHFiles to another package and mark the old one as deprecated
Repository: hbase Updated Branches: refs/heads/branch-2 49986e9df -> a37417c25 http://git-wip-us.apache.org/repos/asf/hbase/blob/a37417c2/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java index 0fe79d1..199c2c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java @@ -60,13 +60,13 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.regionserver.TestSourceFSConfigurationProvider; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.HFileTestUtil; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; http://git-wip-us.apache.org/repos/asf/hbase/blob/a37417c2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/HadoopSecurityEnabledUserProviderForTesting.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/HadoopSecurityEnabledUserProviderForTesting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/HadoopSecurityEnabledUserProviderForTesting.java new file mode 100644 index 000..3ebda29 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/HadoopSecurityEnabledUserProviderForTesting.java @@ -0,0 +1,41 @@ +/** + * 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 + * + * 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 org.apache.hadoop.hbase.security; + +import org.apache.hadoop.hbase.security.UserProvider; + +/** + * A {@link UserProvider} that always says hadoop security is enabled, regardless of the underlying + * configuration. HBase security is not enabled as this is used to determine if SASL is used + * to do the authentication, which requires a Kerberos ticket (which we currently don't have in + * tests). + * + * This should only be used for TESTING. + */ +public class HadoopSecurityEnabledUserProviderForTesting extends UserProvider { + + @Override + public boolean isHBaseSecurityEnabled() { +return false; + } + + @Override + public boolean isHadoopSecurityEnabled() { +return true; + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hbase/blob/a37417c2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index 6583366..1e38179 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -25,6 +25,12 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.protobuf.BlockingRpcChannel; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; +import com.google.protobuf.ServiceException; + import java.io.IOException; import
hbase git commit: HBASE-18949 Remove the CompactionRequest parameter in preCompactSelection
Repository: hbase Updated Branches: refs/heads/master 54da4405d -> c3b3fd788 HBASE-18949 Remove the CompactionRequest parameter in preCompactSelection Signed-off-by: zhangduo <zhang...@apache.org> Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c3b3fd78 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c3b3fd78 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c3b3fd78 Branch: refs/heads/master Commit: c3b3fd7b8fd51f0dd1864a8cb618f88be16a Parents: 54da440 Author: Peter Somogyi <psomo...@cloudera.com> Authored: Thu Oct 5 09:59:28 2017 -0700 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 10 09:41:21 2017 +0800 -- .../org/apache/hadoop/hbase/coprocessor/RegionObserver.java| 6 ++ .../main/java/org/apache/hadoop/hbase/regionserver/HStore.java | 2 +- .../hadoop/hbase/regionserver/RegionCoprocessorHost.java | 6 ++ .../apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java | 3 +-- .../apache/hadoop/hbase/mob/compactions/TestMobCompactor.java | 4 +--- 5 files changed, 7 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/c3b3fd78/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index cbd6e4b..a1e4f0e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompareOperator; -import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; @@ -188,11 +187,10 @@ public interface RegionObserver { * @param store the store where compaction is being requested * @param candidates the store files currently available for compaction * @param tracker tracker used to track the life cycle of a compaction - * @param request the requested compaction */ default void preCompactSelection(ObserverContext c, Store store, - List candidates, CompactionLifeCycleTracker tracker, - CompactionRequest request) throws IOException {} + List candidates, CompactionLifeCycleTracker tracker) + throws IOException {} /** * Called after the {@link StoreFile}s to compact have been selected from the available http://git-wip-us.apache.org/repos/asf/hbase/blob/c3b3fd78/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index d8e82bb..d2009e3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1666,7 +1666,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat boolean override = false; //TODO: is it correct way to get CompactionRequest? override = getCoprocessorHost().preCompactSelection(this, candidatesForCoproc, -tracker, null, user); +tracker, user); if (override) { // Coprocessor is overriding normal file selection. compaction.forceSelect(new CompactionRequestImpl(candidatesForCoproc)); http://git-wip-us.apache.org/repos/asf/hbase/blob/c3b3fd78/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java index b76980d..035c8d1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java @@ -569,18 +569,16 @@ public class RegionCoprocessorHost * @param store The store where compaction is being requested * @param candidates The currently available store files * @param tracker used to track the life cycle of a compaction
[2/2] hbase git commit: HBASE-18951 Use Builder pattern to remove nullable parameters for checkAndXXX methods in RawAsyncTable/AsyncTable interface
HBASE-18951 Use Builder pattern to remove nullable parameters for checkAndXXX methods in RawAsyncTable/AsyncTable interface Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/d5b76547 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d5b76547 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d5b76547 Branch: refs/heads/branch-2 Commit: d5b76547f088783041212df559bcb08b6c641776 Parents: 294f6b7 Author: zhangduo <zhang...@apache.org> Authored: Tue Oct 10 11:11:19 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 10 14:41:27 2017 +0800 -- .../hadoop/hbase/client/AsyncTableBase.java | 158 --- .../hadoop/hbase/client/AsyncTableImpl.java | 57 +-- .../hadoop/hbase/client/RawAsyncTableImpl.java | 123 ++- .../hadoop/hbase/client/TestAsyncTable.java | 53 --- 4 files changed, 219 insertions(+), 172 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/d5b76547/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java index ed4c497..cc9f337 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java @@ -191,72 +191,75 @@ public interface AsyncTableBase { } /** - * Atomically checks if a row/family/qualifier value equals to the expected value. If it does, it - * adds the put. If the passed value is null, the check is for the lack of column (ie: - * non-existence) - * @param row to check - * @param family column family to check - * @param qualifier column qualifier to check - * @param value the expected value - * @param put data to put if check succeeds - * @return true if the new put was executed, false otherwise. The return value will be wrapped by - * a {@link CompletableFuture}. - */ - default CompletableFuture checkAndPut(byte[] row, byte[] family, byte[] qualifier, - byte[] value, Put put) { -return checkAndPut(row, family, qualifier, CompareOperator.EQUAL, value, put); - } - - /** * Atomically checks if a row/family/qualifier value matches the expected value. If it does, it - * adds the put. If the passed value is null, the check is for the lack of column (ie: - * non-existence) - * @param row to check - * @param family column family to check - * @param qualifier column qualifier to check - * @param compareOp comparison operator to use - * @param value the expected value - * @param put data to put if check succeeds - * @return true if the new put was executed, false otherwise. The return value will be wrapped by - * a {@link CompletableFuture}. - */ - CompletableFuture checkAndPut(byte[] row, byte[] family, byte[] qualifier, - CompareOperator compareOp, byte[] value, Put put); - - /** - * Atomically checks if a row/family/qualifier value equals to the expected value. If it does, it - * adds the delete. If the passed value is null, the check is for the lack of column (ie: - * non-existence) - * @param row to check - * @param family column family to check - * @param qualifier column qualifier to check - * @param value the expected value - * @param delete data to delete if check succeeds - * @return true if the new delete was executed, false otherwise. The return value will be wrapped - * by a {@link CompletableFuture}. + * adds the Put/Delete/RowMutations. + * + * Use the returned {@link CheckAndMutateBuilder} to construct your request and then execute it. + * This is a fluent style API, the code is like: + * + * + * + * table.checkAndMutate(row, family).qualifier(qualifier).ifNotExists().thenPut(put) + * .thenAccept(succ -> { + * if (succ) { + * System.out.println("Check and put succeeded"); + * } else { + * System.out.println("Check and put failed"); + * } + * }); + * + * */ - default CompletableFuture checkAndDelete(byte[] row, byte[] family, byte[] qualifier, - byte[] value, Delete delete) { -return checkAndDelete(row, family, qualifier, CompareOperator.EQUAL, value, delete); + CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family); + + /** + * A helper class for sending checkAndMutate request. + */ + interface CheckAndMutateBuilder { + +/** + * @param qualifier column qualifier to check. + */ +C
[1/2] hbase git commit: HBASE-18949 Remove the CompactionRequest parameter in preCompactSelection
Repository: hbase Updated Branches: refs/heads/branch-2 ca62f769b -> d5b76547f HBASE-18949 Remove the CompactionRequest parameter in preCompactSelection Signed-off-by: zhangduo <zhang...@apache.org> Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/294f6b78 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/294f6b78 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/294f6b78 Branch: refs/heads/branch-2 Commit: 294f6b786058edb93dd0327108845d2146437de0 Parents: ca62f76 Author: Peter Somogyi <psomo...@cloudera.com> Authored: Thu Oct 5 09:59:28 2017 -0700 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 10 09:42:03 2017 +0800 -- .../org/apache/hadoop/hbase/coprocessor/RegionObserver.java| 6 ++ .../main/java/org/apache/hadoop/hbase/regionserver/HStore.java | 2 +- .../hadoop/hbase/regionserver/RegionCoprocessorHost.java | 6 ++ .../apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java | 3 +-- .../apache/hadoop/hbase/mob/compactions/TestMobCompactor.java | 4 +--- 5 files changed, 7 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/294f6b78/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index cbd6e4b..a1e4f0e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompareOperator; -import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; @@ -188,11 +187,10 @@ public interface RegionObserver { * @param store the store where compaction is being requested * @param candidates the store files currently available for compaction * @param tracker tracker used to track the life cycle of a compaction - * @param request the requested compaction */ default void preCompactSelection(ObserverContext c, Store store, - List candidates, CompactionLifeCycleTracker tracker, - CompactionRequest request) throws IOException {} + List candidates, CompactionLifeCycleTracker tracker) + throws IOException {} /** * Called after the {@link StoreFile}s to compact have been selected from the available http://git-wip-us.apache.org/repos/asf/hbase/blob/294f6b78/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index d8e82bb..d2009e3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1666,7 +1666,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat boolean override = false; //TODO: is it correct way to get CompactionRequest? override = getCoprocessorHost().preCompactSelection(this, candidatesForCoproc, -tracker, null, user); +tracker, user); if (override) { // Coprocessor is overriding normal file selection. compaction.forceSelect(new CompactionRequestImpl(candidatesForCoproc)); http://git-wip-us.apache.org/repos/asf/hbase/blob/294f6b78/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java index b76980d..035c8d1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java @@ -569,18 +569,16 @@ public class RegionCoprocessorHost * @param store The store where compaction is being requested * @param candidates The currently available store files * @param tracker used to track the life cycle of a compaction
hbase git commit: HBASE-18951 Use Builder pattern to remove nullable parameters for checkAndXXX methods in RawAsyncTable/AsyncTable interface
Repository: hbase Updated Branches: refs/heads/master c3b3fd788 -> 8597b19b3 HBASE-18951 Use Builder pattern to remove nullable parameters for checkAndXXX methods in RawAsyncTable/AsyncTable interface Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/8597b19b Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/8597b19b Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/8597b19b Branch: refs/heads/master Commit: 8597b19b3d7bb2671a6afd7e902aaaea6690a105 Parents: c3b3fd7 Author: zhangduo <zhang...@apache.org> Authored: Tue Oct 10 11:11:19 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 10 14:41:23 2017 +0800 -- .../hadoop/hbase/client/AsyncTableBase.java | 158 --- .../hadoop/hbase/client/AsyncTableImpl.java | 57 +-- .../hadoop/hbase/client/RawAsyncTableImpl.java | 123 ++- .../hadoop/hbase/client/TestAsyncTable.java | 53 --- 4 files changed, 219 insertions(+), 172 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/8597b19b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java index ed4c497..cc9f337 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java @@ -191,72 +191,75 @@ public interface AsyncTableBase { } /** - * Atomically checks if a row/family/qualifier value equals to the expected value. If it does, it - * adds the put. If the passed value is null, the check is for the lack of column (ie: - * non-existence) - * @param row to check - * @param family column family to check - * @param qualifier column qualifier to check - * @param value the expected value - * @param put data to put if check succeeds - * @return true if the new put was executed, false otherwise. The return value will be wrapped by - * a {@link CompletableFuture}. - */ - default CompletableFuture checkAndPut(byte[] row, byte[] family, byte[] qualifier, - byte[] value, Put put) { -return checkAndPut(row, family, qualifier, CompareOperator.EQUAL, value, put); - } - - /** * Atomically checks if a row/family/qualifier value matches the expected value. If it does, it - * adds the put. If the passed value is null, the check is for the lack of column (ie: - * non-existence) - * @param row to check - * @param family column family to check - * @param qualifier column qualifier to check - * @param compareOp comparison operator to use - * @param value the expected value - * @param put data to put if check succeeds - * @return true if the new put was executed, false otherwise. The return value will be wrapped by - * a {@link CompletableFuture}. - */ - CompletableFuture checkAndPut(byte[] row, byte[] family, byte[] qualifier, - CompareOperator compareOp, byte[] value, Put put); - - /** - * Atomically checks if a row/family/qualifier value equals to the expected value. If it does, it - * adds the delete. If the passed value is null, the check is for the lack of column (ie: - * non-existence) - * @param row to check - * @param family column family to check - * @param qualifier column qualifier to check - * @param value the expected value - * @param delete data to delete if check succeeds - * @return true if the new delete was executed, false otherwise. The return value will be wrapped - * by a {@link CompletableFuture}. + * adds the Put/Delete/RowMutations. + * + * Use the returned {@link CheckAndMutateBuilder} to construct your request and then execute it. + * This is a fluent style API, the code is like: + * + * + * + * table.checkAndMutate(row, family).qualifier(qualifier).ifNotExists().thenPut(put) + * .thenAccept(succ -> { + * if (succ) { + * System.out.println("Check and put succeeded"); + * } else { + * System.out.println("Check and put failed"); + * } + * }); + * + * */ - default CompletableFuture checkAndDelete(byte[] row, byte[] family, byte[] qualifier, - byte[] value, Delete delete) { -return checkAndDelete(row, family, qualifier, CompareOperator.EQUAL, value, delete); + CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family); + + /** + * A helper class for sending checkAndMutate request. + */ + interface CheckAndMutateBuilder { + +
hbase git commit: HBASE-18411 Dividing FiterList into two separate sub-classes: FilterListWithOR , FilterListWithAND
Repository: hbase Updated Branches: refs/heads/HBASE-18410 5d0ee5f0e -> f67fe4da4 HBASE-18411 Dividing FiterList into two separate sub-classes: FilterListWithOR , FilterListWithAND Signed-off-by: zhangduo <zhang...@apache.org> Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/f67fe4da Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f67fe4da Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f67fe4da Branch: refs/heads/HBASE-18410 Commit: f67fe4da455c9cf3a6df6bea28241a1ae745b348 Parents: 5d0ee5f Author: huzheng <open...@gmail.com> Authored: Tue Oct 10 20:01:48 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Thu Oct 12 10:14:29 2017 +0800 -- .../apache/hadoop/hbase/filter/FilterList.java | 666 ++- .../hadoop/hbase/filter/FilterListBase.java | 159 + .../hadoop/hbase/filter/FilterListWithAND.java | 273 .../hadoop/hbase/filter/FilterListWithOR.java | 383 +++ .../hadoop/hbase/filter/TestFilterList.java | 90 +++ 5 files changed, 962 insertions(+), 609 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/f67fe4da/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index b94429f..a4fa74b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -22,101 +22,70 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashSet; import java.util.List; -import java.util.Set; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellComparator; -import org.apache.hadoop.hbase.CellUtil; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException; /** - * Implementation of {@link Filter} that represents an ordered List of Filters - * which will be evaluated with a specified boolean operator {@link Operator#MUST_PASS_ALL} - * (AND) or {@link Operator#MUST_PASS_ONE} (OR). - * Since you can use Filter Lists as children of Filter Lists, you can create a - * hierarchy of filters to be evaluated. - * - * - * {@link Operator#MUST_PASS_ALL} evaluates lazily: evaluation stops as soon as one filter does - * not include the KeyValue. - * - * - * {@link Operator#MUST_PASS_ONE} evaluates non-lazily: all filters are always evaluated. - * - * + * Implementation of {@link Filter} that represents an ordered List of Filters which will be + * evaluated with a specified boolean operator {@link Operator#MUST_PASS_ALL} (AND) or + * {@link Operator#MUST_PASS_ONE} (OR). Since you can use Filter Lists as children of + * Filter Lists, you can create a hierarchy of filters to be evaluated. + * {@link Operator#MUST_PASS_ALL} evaluates lazily: evaluation stops as soon as one filter does not + * include the KeyValue. + * {@link Operator#MUST_PASS_ONE} evaluates non-lazily: all filters are always evaluated. * Defaults to {@link Operator#MUST_PASS_ALL}. */ @InterfaceAudience.Public final public class FilterList extends FilterBase { + /** set operator */ @InterfaceAudience.Public - public static enum Operator { + public enum Operator { /** !AND */ MUST_PASS_ALL, /** !OR */ MUST_PASS_ONE } - private static final int MAX_LOG_FILTERS = 5; - private Operator operator = Operator.MUST_PASS_ALL; - private final List filters; - private Set seekHintFilter = new HashSet<>(); - - /** - * Save previous return code and previous cell for every filter in filter list. For MUST_PASS_ONE, - * we use the previous return code to decide whether we should pass current cell encountered to - * the filter. For MUST_PASS_ALL, the two list are meaningless. - */ - private List prevFilterRCList = null; - private List prevCellList = null; - - /** Reference Cell used by {@link #transformCell(Cell)} for validation purpose. */ - private Cell referenceCell = null; - - /** - * When filtering a given Cell in {@link #filterKeyValue(Cell)}, - * this stores the transformed Cell to be returned by {@link #transformCell(Cell)}. - * - * Individual filters transformation are applied onl
hbase git commit: HBASE-18552 Backport the server side change in HBASE-18489 to branch-1
Repository: hbase Updated Branches: refs/heads/branch-1.4 22e2539d0 -> 0fd4da998 HBASE-18552 Backport the server side change in HBASE-18489 to branch-1 Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/0fd4da99 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/0fd4da99 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/0fd4da99 Branch: refs/heads/branch-1.4 Commit: 0fd4da998e3d96f4df414f93e2db70879dc2 Parents: 22e2539 Author: zhangduo <zhang...@apache.org> Authored: Fri Aug 18 10:19:19 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Thu Oct 12 10:53:09 2017 +0800 -- .../apache/hadoop/hbase/regionserver/RSRpcServices.java | 6 +++--- .../hadoop/hbase/regionserver/ScannerContext.java | 12 +--- .../apache/hadoop/hbase/regionserver/StoreScanner.java | 4 ++-- 3 files changed, 10 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/0fd4da99/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 24d6d4c..531721a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -2857,9 +2857,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler, // Heartbeat messages occur when the time limit has been reached. builder.setHeartbeatMessage(timeLimitReached); if (timeLimitReached && rsh.needCursor) { -Cell readingCell = scannerContext.getPeekedCellInHeartbeat(); -if (readingCell != null) { - builder.setCursor(ProtobufUtil.toCursor(readingCell)); +Cell cursorCell = scannerContext.getLastPeekedCell(); +if (cursorCell != null) { + builder.setCursor(ProtobufUtil.toCursor(cursorCell)); } } } http://git-wip-us.apache.org/repos/asf/hbase/blob/0fd4da99/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java index ee53a95..6831c91 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java @@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.regionserver; import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -97,7 +95,7 @@ public class ScannerContext { boolean keepProgress; private static boolean DEFAULT_KEEP_PROGRESS = false; - private Cell peekedCellInHeartbeat = null; + private Cell lastPeekedCell = null; /** * Tracks the relevant server side metrics during scans. null when metrics should not be tracked @@ -324,12 +322,12 @@ public class ScannerContext { || checkTimeLimit(checkerScope); } - public Cell getPeekedCellInHeartbeat() { -return peekedCellInHeartbeat; + public Cell getLastPeekedCell() { +return lastPeekedCell; } - public void setPeekedCellInHeartbeat(Cell peekedCellInHeartbeat) { -this.peekedCellInHeartbeat = peekedCellInHeartbeat; + public void setLastPeekedCell(Cell lastPeekedCell) { +this.lastPeekedCell = lastPeekedCell; } @Override http://git-wip-us.apache.org/repos/asf/hbase/blob/0fd4da99/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java index 7c34b87..fbdf7b2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java @@ -46,11 +46,11 @@ import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope; import org.apache.hadoop.hbase.regionserver.S
hbase git commit: HBASE-18552 Backport the server side change in HBASE-18489 to branch-1
Repository: hbase Updated Branches: refs/heads/branch-1 3959b39b9 -> ff23e1576 HBASE-18552 Backport the server side change in HBASE-18489 to branch-1 Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/ff23e157 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/ff23e157 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/ff23e157 Branch: refs/heads/branch-1 Commit: ff23e15769013050814b9dc674c65a430f24af36 Parents: 3959b39 Author: zhangduo <zhang...@apache.org> Authored: Fri Aug 18 10:19:19 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Thu Oct 12 10:42:21 2017 +0800 -- .../apache/hadoop/hbase/regionserver/RSRpcServices.java | 6 +++--- .../hadoop/hbase/regionserver/ScannerContext.java | 12 +--- .../apache/hadoop/hbase/regionserver/StoreScanner.java | 4 ++-- 3 files changed, 10 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/ff23e157/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index d3c8772..266b4f3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -2855,9 +2855,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler, // Heartbeat messages occur when the time limit has been reached. builder.setHeartbeatMessage(timeLimitReached); if (timeLimitReached && rsh.needCursor) { -Cell readingCell = scannerContext.getPeekedCellInHeartbeat(); -if (readingCell != null) { - builder.setCursor(ProtobufUtil.toCursor(readingCell)); +Cell cursorCell = scannerContext.getLastPeekedCell(); +if (cursorCell != null) { + builder.setCursor(ProtobufUtil.toCursor(cursorCell)); } } } http://git-wip-us.apache.org/repos/asf/hbase/blob/ff23e157/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java index ee53a95..6831c91 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java @@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.regionserver; import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -97,7 +95,7 @@ public class ScannerContext { boolean keepProgress; private static boolean DEFAULT_KEEP_PROGRESS = false; - private Cell peekedCellInHeartbeat = null; + private Cell lastPeekedCell = null; /** * Tracks the relevant server side metrics during scans. null when metrics should not be tracked @@ -324,12 +322,12 @@ public class ScannerContext { || checkTimeLimit(checkerScope); } - public Cell getPeekedCellInHeartbeat() { -return peekedCellInHeartbeat; + public Cell getLastPeekedCell() { +return lastPeekedCell; } - public void setPeekedCellInHeartbeat(Cell peekedCellInHeartbeat) { -this.peekedCellInHeartbeat = peekedCellInHeartbeat; + public void setLastPeekedCell(Cell lastPeekedCell) { +this.lastPeekedCell = lastPeekedCell; } @Override http://git-wip-us.apache.org/repos/asf/hbase/blob/ff23e157/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java index 7c34b87..fbdf7b2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java @@ -46,11 +46,11 @@ import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope; import org.apache.hadoop.hbase.regionserver.S
hbase git commit: HBASE-19042 Oracle Java 8u144 downloader broken in precommit check
Repository: hbase Updated Branches: refs/heads/master 3acb08178 -> 9e688117b HBASE-19042 Oracle Java 8u144 downloader broken in precommit check Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/9e688117 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/9e688117 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/9e688117 Branch: refs/heads/master Commit: 9e688117bad3cb4826c7201bb359672676389620 Parents: 3acb081 Author: zhangduo <zhang...@apache.org> Authored: Thu Oct 19 14:49:09 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Thu Oct 19 15:32:48 2017 +0800 -- dev-support/docker/Dockerfile | 29 +++-- 1 file changed, 11 insertions(+), 18 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/9e688117/dev-support/docker/Dockerfile -- diff --git a/dev-support/docker/Dockerfile b/dev-support/docker/Dockerfile index 62c6030..c23c70d 100644 --- a/dev-support/docker/Dockerfile +++ b/dev-support/docker/Dockerfile @@ -65,18 +65,18 @@ RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ zlib1g-dev ### -# Oracle Java +# OpenJDK 8 ### RUN echo "dot_style = mega" > "/root/.wgetrc" RUN echo "quiet = on" >> "/root/.wgetrc" RUN apt-get -q update && apt-get -q install --no-install-recommends -y software-properties-common -RUN add-apt-repository -y ppa:webupd8team/java - -# Auto-accept the Oracle JDK license -RUN echo oracle-java8-installer shared/accepted-oracle-license-v1-1 select true | sudo /usr/bin/debconf-set-selections -RUN apt-get -q update && apt-get -q install --no-install-recommends -y oracle-java8-installer +RUN add-apt-repository -y ppa:openjdk-r/ppa +RUN apt-get -q update +RUN apt-get -q install --no-install-recommends -y openjdk-8-jdk +RUN update-alternatives --config java +RUN update-alternatives --config javac # Apps that require Java @@ -131,23 +131,16 @@ RUN pip install python-dateutil # Install Ruby 2, based on Yetus 0.4.0 dockerfile ### RUN echo 'gem: --no-rdoc --no-ri' >> /root/.gemrc -RUN apt-get -q install -y ruby2.0 -# -# on trusty, the above installs ruby2.0 and ruby (1.9.3) exes -# but update-alternatives is broken, so we need to do some work -# to make 2.0 actually the default without the system flipping out -# -# See https://bugs.launchpad.net/ubuntu/+source/ruby2.0/+bug/1310292 -# -RUN dpkg-divert --add --rename --divert /usr/bin/ruby.divert /usr/bin/ruby -RUN dpkg-divert --add --rename --divert /usr/bin/gem.divert /usr/bin/gemrc -RUN update-alternatives --install /usr/bin/ruby ruby /usr/bin/ruby2.0 1 -RUN update-alternatives --install /usr/bin/gem gem /usr/bin/gem2.0 1 +RUN apt-add-repository ppa:brightbox/ruby-ng +RUN apt-get -q update +RUN apt-get -q install --no-install-recommends -y ruby2.2 ruby-switch +RUN ruby-switch --set ruby2.2 # Install rubocop ### +RUN gem install rake RUN gem install rubocop
hbase git commit: Revert "HBASE-19042 Oracle Java 8u144 downloader broken in precommit check"
Repository: hbase Updated Branches: refs/heads/master 9e688117b -> 4a7b43039 Revert "HBASE-19042 Oracle Java 8u144 downloader broken in precommit check" This reverts commit 9e688117bad3cb4826c7201bb359672676389620. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/4a7b4303 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4a7b4303 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4a7b4303 Branch: refs/heads/master Commit: 4a7b4303979ffe9896811f633141681669e1c20d Parents: 9e68811 Author: zhangduo <zhang...@apache.org> Authored: Thu Oct 19 16:03:28 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Thu Oct 19 16:03:28 2017 +0800 -- dev-support/docker/Dockerfile | 29 ++--- 1 file changed, 18 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/4a7b4303/dev-support/docker/Dockerfile -- diff --git a/dev-support/docker/Dockerfile b/dev-support/docker/Dockerfile index c23c70d..62c6030 100644 --- a/dev-support/docker/Dockerfile +++ b/dev-support/docker/Dockerfile @@ -65,18 +65,18 @@ RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ zlib1g-dev ### -# OpenJDK 8 +# Oracle Java ### RUN echo "dot_style = mega" > "/root/.wgetrc" RUN echo "quiet = on" >> "/root/.wgetrc" RUN apt-get -q update && apt-get -q install --no-install-recommends -y software-properties-common -RUN add-apt-repository -y ppa:openjdk-r/ppa -RUN apt-get -q update -RUN apt-get -q install --no-install-recommends -y openjdk-8-jdk -RUN update-alternatives --config java -RUN update-alternatives --config javac +RUN add-apt-repository -y ppa:webupd8team/java + +# Auto-accept the Oracle JDK license +RUN echo oracle-java8-installer shared/accepted-oracle-license-v1-1 select true | sudo /usr/bin/debconf-set-selections +RUN apt-get -q update && apt-get -q install --no-install-recommends -y oracle-java8-installer # Apps that require Java @@ -131,16 +131,23 @@ RUN pip install python-dateutil # Install Ruby 2, based on Yetus 0.4.0 dockerfile ### RUN echo 'gem: --no-rdoc --no-ri' >> /root/.gemrc -RUN apt-add-repository ppa:brightbox/ruby-ng -RUN apt-get -q update +RUN apt-get -q install -y ruby2.0 +# +# on trusty, the above installs ruby2.0 and ruby (1.9.3) exes +# but update-alternatives is broken, so we need to do some work +# to make 2.0 actually the default without the system flipping out +# +# See https://bugs.launchpad.net/ubuntu/+source/ruby2.0/+bug/1310292 +# +RUN dpkg-divert --add --rename --divert /usr/bin/ruby.divert /usr/bin/ruby +RUN dpkg-divert --add --rename --divert /usr/bin/gem.divert /usr/bin/gemrc +RUN update-alternatives --install /usr/bin/ruby ruby /usr/bin/ruby2.0 1 +RUN update-alternatives --install /usr/bin/gem gem /usr/bin/gem2.0 1 -RUN apt-get -q install --no-install-recommends -y ruby2.2 ruby-switch -RUN ruby-switch --set ruby2.2 # Install rubocop ### -RUN gem install rake RUN gem install rubocop
[2/2] hbase git commit: HBASE-19001 Remove the hooks in RegionObserver which are designed to construct a StoreScanner which is marked as IA.Private
HBASE-19001 Remove the hooks in RegionObserver which are designed to construct a StoreScanner which is marked as IA.Private Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/e804dd0b Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e804dd0b Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e804dd0b Branch: refs/heads/master Commit: e804dd0b600f898f7519dee7134b68ad04c20a9a Parents: 5368fd5 Author: zhangduo <zhang...@apache.org> Authored: Tue Oct 17 21:27:05 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Wed Oct 18 11:06:39 2017 +0800 -- .../hbase/coprocessor/RegionObserver.java | 77 - .../hadoop/hbase/regionserver/HMobStore.java| 24 +- .../hadoop/hbase/regionserver/HRegion.java | 4 +- .../hadoop/hbase/regionserver/HStore.java | 18 +- .../hadoop/hbase/regionserver/Region.java | 3 - .../regionserver/RegionCoprocessorHost.java | 64 +--- .../regionserver/ReversedStoreScanner.java | 6 +- .../hadoop/hbase/regionserver/StoreFlusher.java | 12 +- .../regionserver/compactions/Compactor.java | 44 +-- ...estAvoidCellReferencesIntoShippedBlocks.java | 197 ++--- .../hadoop/hbase/client/TestFromClientSide.java | 156 -- .../client/TestFromClientSideScanExcpetion.java | 238 +++ ...mClientSideScanExcpetionWithCoprocessor.java | 43 +++ .../hbase/coprocessor/SimpleRegionObserver.java | 36 --- .../TestRegionObserverScannerOpenHook.java | 31 +- .../regionserver/DelegatingInternalScanner.java | 45 +++ .../regionserver/NoOpScanPolicyObserver.java| 60 +--- .../hbase/util/TestCoprocessorScanPolicy.java | 290 +++ 18 files changed, 647 insertions(+), 701 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/e804dd0b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index a1e4f0e..d03a9be 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -124,27 +124,6 @@ public interface RegionObserver { default void postLogReplay(ObserverContext c) {} /** - * Called before a memstore is flushed to disk and prior to creating the scanner to read from - * the memstore. To override or modify how a memstore is flushed, - * implementing classes can return a new scanner to provide the KeyValues to be - * stored into the new {@code StoreFile} or null to perform the default processing. - * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no - * effect in this hook. - * @param c the environment provided by the region server - * @param store the store being flushed - * @param scanners the scanners for the memstore that is flushed - * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain - * @param readPoint the readpoint to create scanner - * @return the scanner to use during the flush. {@code null} if the default implementation - * is to be used. - */ - default InternalScanner preFlushScannerOpen(ObserverContext c, - Store store, List scanners, InternalScanner s, long readPoint) - throws IOException { -return s; - } - - /** * Called before the memstore is flushed to disk. * @param c the environment provided by the region server */ @@ -236,33 +215,6 @@ public interface RegionObserver { } /** - * Called prior to writing the {@link StoreFile}s selected for compaction into a new - * {@code StoreFile} and prior to creating the scanner used to read the input files. To override - * or modify the compaction process, implementing classes can return a new scanner to provide the - * KeyValues to be stored into the new {@code StoreFile} or null to perform the default - * processing. Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no - * effect in this hook. - * @param c the environment provided by the region server - * @param store the store being compacted - * @param scanners the list of store file scanners to be read from - * @param scanType the {@link ScanType} indicating whether this is a major or minor compaction - * @param earliestPutTs timestamp of the earliest put that was found in any of the involved store - * files - * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain - * @param
[1/2] hbase git commit: HBASE-19001 Remove the hooks in RegionObserver which are designed to construct a StoreScanner which is marked as IA.Private
Repository: hbase Updated Branches: refs/heads/master 5368fd5bf -> e804dd0b6 http://git-wip-us.apache.org/repos/asf/hbase/blob/e804dd0b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java index ab9bfc59..c67d7bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java @@ -17,31 +17,25 @@ */ package org.apache.hadoop.hbase.util; // this is deliberately not in the o.a.h.h.regionserver package + // in order to make sure all required classes/method are available import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.Collection; -import java.util.HashMap; import java.util.List; -import java.util.Map; -import java.util.NavigableSet; import java.util.Optional; -import java.util.OptionalInt; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.function.Predicate; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -53,11 +47,12 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.DelegatingInternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.KeyValueScanner; -import org.apache.hadoop.hbase.regionserver.ScanInfo; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreScanner; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; @@ -73,7 +68,7 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -@Category({MiscTests.class, MediumTests.class}) +@Category({ MiscTests.class, MediumTests.class }) @RunWith(Parameterized.class) public class TestCoprocessorScanPolicy { protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -84,8 +79,7 @@ public class TestCoprocessorScanPolicy { @BeforeClass public static void setUpBeforeClass() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); -conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, -ScanObserver.class.getName()); +conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, ScanObserver.class.getName()); TEST_UTIL.startMiniCluster(); } @@ -106,49 +100,58 @@ public class TestCoprocessorScanPolicy { @Test public void testBaseCases() throws Exception { -TableName tableName = -TableName.valueOf("baseCases"); +TableName tableName = TableName.valueOf("baseCases"); if (TEST_UTIL.getAdmin().tableExists(tableName)) { TEST_UTIL.deleteTable(tableName); } -Table t = TEST_UTIL.createTable(tableName, F, 1); -// set the version override to 2 -Put p = new Put(R); -p.setAttribute("versions", new byte[]{}); -p.addColumn(F, tableName.getName(), Bytes.toBytes(2)); -t.put(p); - +Table t = TEST_UTIL.createTable(tableName, F, 10); +// insert 3 versions long now = EnvironmentEdgeManager.currentTime(); - -// insert 2 versions -p = new Put(R); +Put p = new Put(R); p.addColumn(F, Q, now, Q); t.put(p); p = new Put(R); p.addColumn(F, Q, now + 1, Q); t.put(p); +p = new Put(R); +p.addColumn(F, Q, now + 2, Q); +t.put(p); + Get g = new Get(R); -g.setMaxVersions(10); +g.readVersions(10); Result r = t.get(g); +
[2/2] hbase git commit: HBASE-19001 Remove the hooks in RegionObserver which are designed to construct a StoreScanner which is marked as IA.Private
HBASE-19001 Remove the hooks in RegionObserver which are designed to construct a StoreScanner which is marked as IA.Private Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/7a666858 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/7a666858 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/7a666858 Branch: refs/heads/branch-2 Commit: 7a666858019ed9af860763f615ce810e6b31a4d2 Parents: 998bc32 Author: zhangduo <zhang...@apache.org> Authored: Tue Oct 17 21:27:05 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Wed Oct 18 11:06:44 2017 +0800 -- .../hbase/coprocessor/RegionObserver.java | 77 - .../hadoop/hbase/regionserver/HMobStore.java| 24 +- .../hadoop/hbase/regionserver/HRegion.java | 4 +- .../hadoop/hbase/regionserver/HStore.java | 18 +- .../hadoop/hbase/regionserver/Region.java | 3 - .../regionserver/RegionCoprocessorHost.java | 64 +--- .../regionserver/ReversedStoreScanner.java | 6 +- .../hadoop/hbase/regionserver/StoreFlusher.java | 12 +- .../regionserver/compactions/Compactor.java | 44 +-- ...estAvoidCellReferencesIntoShippedBlocks.java | 197 ++--- .../hadoop/hbase/client/TestFromClientSide.java | 156 -- .../client/TestFromClientSideScanExcpetion.java | 238 +++ ...mClientSideScanExcpetionWithCoprocessor.java | 43 +++ .../hbase/coprocessor/SimpleRegionObserver.java | 36 --- .../TestRegionObserverScannerOpenHook.java | 31 +- .../regionserver/DelegatingInternalScanner.java | 45 +++ .../regionserver/NoOpScanPolicyObserver.java| 60 +--- .../hbase/util/TestCoprocessorScanPolicy.java | 290 +++ 18 files changed, 647 insertions(+), 701 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/7a666858/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index a1e4f0e..d03a9be 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -124,27 +124,6 @@ public interface RegionObserver { default void postLogReplay(ObserverContext c) {} /** - * Called before a memstore is flushed to disk and prior to creating the scanner to read from - * the memstore. To override or modify how a memstore is flushed, - * implementing classes can return a new scanner to provide the KeyValues to be - * stored into the new {@code StoreFile} or null to perform the default processing. - * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no - * effect in this hook. - * @param c the environment provided by the region server - * @param store the store being flushed - * @param scanners the scanners for the memstore that is flushed - * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain - * @param readPoint the readpoint to create scanner - * @return the scanner to use during the flush. {@code null} if the default implementation - * is to be used. - */ - default InternalScanner preFlushScannerOpen(ObserverContext c, - Store store, List scanners, InternalScanner s, long readPoint) - throws IOException { -return s; - } - - /** * Called before the memstore is flushed to disk. * @param c the environment provided by the region server */ @@ -236,33 +215,6 @@ public interface RegionObserver { } /** - * Called prior to writing the {@link StoreFile}s selected for compaction into a new - * {@code StoreFile} and prior to creating the scanner used to read the input files. To override - * or modify the compaction process, implementing classes can return a new scanner to provide the - * KeyValues to be stored into the new {@code StoreFile} or null to perform the default - * processing. Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no - * effect in this hook. - * @param c the environment provided by the region server - * @param store the store being compacted - * @param scanners the list of store file scanners to be read from - * @param scanType the {@link ScanType} indicating whether this is a major or minor compaction - * @param earliestPutTs timestamp of the earliest put that was found in any of the involved store - * files - * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain - * @param
[1/2] hbase git commit: HBASE-19001 Remove the hooks in RegionObserver which are designed to construct a StoreScanner which is marked as IA.Private
Repository: hbase Updated Branches: refs/heads/branch-2 998bc32ec -> 7a6668580 http://git-wip-us.apache.org/repos/asf/hbase/blob/7a666858/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java index ab9bfc59..c67d7bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java @@ -17,31 +17,25 @@ */ package org.apache.hadoop.hbase.util; // this is deliberately not in the o.a.h.h.regionserver package + // in order to make sure all required classes/method are available import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.Collection; -import java.util.HashMap; import java.util.List; -import java.util.Map; -import java.util.NavigableSet; import java.util.Optional; -import java.util.OptionalInt; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.function.Predicate; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -53,11 +47,12 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.DelegatingInternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.KeyValueScanner; -import org.apache.hadoop.hbase.regionserver.ScanInfo; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreScanner; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; @@ -73,7 +68,7 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -@Category({MiscTests.class, MediumTests.class}) +@Category({ MiscTests.class, MediumTests.class }) @RunWith(Parameterized.class) public class TestCoprocessorScanPolicy { protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -84,8 +79,7 @@ public class TestCoprocessorScanPolicy { @BeforeClass public static void setUpBeforeClass() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); -conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, -ScanObserver.class.getName()); +conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, ScanObserver.class.getName()); TEST_UTIL.startMiniCluster(); } @@ -106,49 +100,58 @@ public class TestCoprocessorScanPolicy { @Test public void testBaseCases() throws Exception { -TableName tableName = -TableName.valueOf("baseCases"); +TableName tableName = TableName.valueOf("baseCases"); if (TEST_UTIL.getAdmin().tableExists(tableName)) { TEST_UTIL.deleteTable(tableName); } -Table t = TEST_UTIL.createTable(tableName, F, 1); -// set the version override to 2 -Put p = new Put(R); -p.setAttribute("versions", new byte[]{}); -p.addColumn(F, tableName.getName(), Bytes.toBytes(2)); -t.put(p); - +Table t = TEST_UTIL.createTable(tableName, F, 10); +// insert 3 versions long now = EnvironmentEdgeManager.currentTime(); - -// insert 2 versions -p = new Put(R); +Put p = new Put(R); p.addColumn(F, Q, now, Q); t.put(p); p = new Put(R); p.addColumn(F, Q, now + 1, Q); t.put(p); +p = new Put(R); +p.addColumn(F, Q, now + 2, Q); +t.put(p); + Get g = new Get(R); -g.setMaxVersions(10); +g.readVersions(10); Result r = t.get(g); +
hbase git commit: HBASE-19069 Do not wrap the original CompactionLifeCycleTracker when calling CP hooks
Repository: hbase Updated Branches: refs/heads/master 81133f89f -> 37b29e909 HBASE-19069 Do not wrap the original CompactionLifeCycleTracker when calling CP hooks Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/37b29e90 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/37b29e90 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/37b29e90 Branch: refs/heads/master Commit: 37b29e909defecdc580112ce6cd306710d13e9e2 Parents: 81133f8 Author: zhangduo <zhang...@apache.org> Authored: Mon Oct 23 21:10:44 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 24 10:56:14 2017 +0800 -- .../hadoop/hbase/regionserver/CompactSplit.java | 135 ++- .../TestCompactionLifeCycleTracker.java | 9 +- 2 files changed, 80 insertions(+), 64 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/37b29e90/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java index b82b346..0749f85 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java @@ -237,80 +237,73 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati } } - // A compaction life cycle tracker to trace the execution of all the compactions triggered by one - // request and delegate to the source CompactionLifeCycleTracker. It will call completed method if - // all the compactions are finished. - private static final class AggregatingCompactionLifeCycleTracker - implements CompactionLifeCycleTracker { + private interface CompactionCompleteTracker { + +default void completed(Store store) { +} + } + + private static final CompactionCompleteTracker DUMMY_COMPLETE_TRACKER = + new CompactionCompleteTracker() { + }; + + private static final class AggregatingCompleteTracker implements CompactionCompleteTracker { private final CompactionLifeCycleTracker tracker; private final AtomicInteger remaining; -public AggregatingCompactionLifeCycleTracker(CompactionLifeCycleTracker tracker, -int numberOfStores) { +public AggregatingCompleteTracker(CompactionLifeCycleTracker tracker, int numberOfStores) { this.tracker = tracker; this.remaining = new AtomicInteger(numberOfStores); } -private void tryCompleted() { +@Override +public void completed(Store store) { if (remaining.decrementAndGet() == 0) { tracker.completed(); } } - -@Override -public void notExecuted(Store store, String reason) { - tracker.notExecuted(store, reason); - tryCompleted(); -} - -@Override -public void beforeExecution(Store store) { - tracker.beforeExecution(store); -} - -@Override -public void afterExecution(Store store) { - tracker.afterExecution(store); - tryCompleted(); -} } - private CompactionLifeCycleTracker wrap(CompactionLifeCycleTracker tracker, + private CompactionCompleteTracker getCompleteTracker(CompactionLifeCycleTracker tracker, IntSupplier numberOfStores) { if (tracker == CompactionLifeCycleTracker.DUMMY) { // a simple optimization to avoid creating unnecessary objects as usually we do not care about // the life cycle of a compaction. - return tracker; + return DUMMY_COMPLETE_TRACKER; } else { - return new AggregatingCompactionLifeCycleTracker(tracker, numberOfStores.getAsInt()); + return new AggregatingCompleteTracker(tracker, numberOfStores.getAsInt()); } } @Override public synchronized void requestCompaction(HRegion region, String why, int priority, CompactionLifeCycleTracker tracker, User user) throws IOException { -requestCompactionInternal(region, why, priority, true, - wrap(tracker, () -> region.getTableDescriptor().getColumnFamilyCount()), user); +requestCompactionInternal(region, why, priority, true, tracker, + getCompleteTracker(tracker, () -> region.getTableDescriptor().getColumnFamilyCount()), user); } @Override public synchronized void requestCompaction(HRegion region, HStore store, String why, int priority, CompactionLifeCycleTracker tracker, User user) throws IOException { -requestCompactionInternal(region, store, why, priority, true, wrap(tracker, () -> 1), user); +requestCompactionInternal(region, store, why,
hbase git commit: HBASE-19069 Do not wrap the original CompactionLifeCycleTracker when calling CP hooks
Repository: hbase Updated Branches: refs/heads/branch-2 3e0b90b94 -> a6f89f029 HBASE-19069 Do not wrap the original CompactionLifeCycleTracker when calling CP hooks Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a6f89f02 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a6f89f02 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a6f89f02 Branch: refs/heads/branch-2 Commit: a6f89f029a78a37eb84e4908b9975118e3050603 Parents: 3e0b90b Author: zhangduo <zhang...@apache.org> Authored: Mon Oct 23 21:10:44 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 24 10:56:19 2017 +0800 -- .../hadoop/hbase/regionserver/CompactSplit.java | 135 ++- .../TestCompactionLifeCycleTracker.java | 9 +- 2 files changed, 80 insertions(+), 64 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/a6f89f02/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java index b82b346..0749f85 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java @@ -237,80 +237,73 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati } } - // A compaction life cycle tracker to trace the execution of all the compactions triggered by one - // request and delegate to the source CompactionLifeCycleTracker. It will call completed method if - // all the compactions are finished. - private static final class AggregatingCompactionLifeCycleTracker - implements CompactionLifeCycleTracker { + private interface CompactionCompleteTracker { + +default void completed(Store store) { +} + } + + private static final CompactionCompleteTracker DUMMY_COMPLETE_TRACKER = + new CompactionCompleteTracker() { + }; + + private static final class AggregatingCompleteTracker implements CompactionCompleteTracker { private final CompactionLifeCycleTracker tracker; private final AtomicInteger remaining; -public AggregatingCompactionLifeCycleTracker(CompactionLifeCycleTracker tracker, -int numberOfStores) { +public AggregatingCompleteTracker(CompactionLifeCycleTracker tracker, int numberOfStores) { this.tracker = tracker; this.remaining = new AtomicInteger(numberOfStores); } -private void tryCompleted() { +@Override +public void completed(Store store) { if (remaining.decrementAndGet() == 0) { tracker.completed(); } } - -@Override -public void notExecuted(Store store, String reason) { - tracker.notExecuted(store, reason); - tryCompleted(); -} - -@Override -public void beforeExecution(Store store) { - tracker.beforeExecution(store); -} - -@Override -public void afterExecution(Store store) { - tracker.afterExecution(store); - tryCompleted(); -} } - private CompactionLifeCycleTracker wrap(CompactionLifeCycleTracker tracker, + private CompactionCompleteTracker getCompleteTracker(CompactionLifeCycleTracker tracker, IntSupplier numberOfStores) { if (tracker == CompactionLifeCycleTracker.DUMMY) { // a simple optimization to avoid creating unnecessary objects as usually we do not care about // the life cycle of a compaction. - return tracker; + return DUMMY_COMPLETE_TRACKER; } else { - return new AggregatingCompactionLifeCycleTracker(tracker, numberOfStores.getAsInt()); + return new AggregatingCompleteTracker(tracker, numberOfStores.getAsInt()); } } @Override public synchronized void requestCompaction(HRegion region, String why, int priority, CompactionLifeCycleTracker tracker, User user) throws IOException { -requestCompactionInternal(region, why, priority, true, - wrap(tracker, () -> region.getTableDescriptor().getColumnFamilyCount()), user); +requestCompactionInternal(region, why, priority, true, tracker, + getCompleteTracker(tracker, () -> region.getTableDescriptor().getColumnFamilyCount()), user); } @Override public synchronized void requestCompaction(HRegion region, HStore store, String why, int priority, CompactionLifeCycleTracker tracker, User user) throws IOException { -requestCompactionInternal(region, store, why, priority, true, wrap(tracker, () -> 1), user); +requestCompactionInternal(region, store, why,
[18/50] [abbrv] hbase git commit: HBASE-10367 RegionServer graceful stop / decommissioning
HBASE-10367 RegionServer graceful stop / decommissioning Signed-off-by: Jerry HeProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a43a00e8 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a43a00e8 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a43a00e8 Branch: refs/heads/HBASE-18410 Commit: a43a00e89c5c99968a205208ab9a5307c89730b3 Parents: af479c5 Author: Jerry He Authored: Thu Oct 19 21:44:38 2017 -0700 Committer: Jerry He Committed: Thu Oct 19 21:54:45 2017 -0700 -- bin/draining_servers.rb | 2 + .../org/apache/hadoop/hbase/client/Admin.java | 26 +++-- .../apache/hadoop/hbase/client/AsyncAdmin.java | 25 ++-- .../hadoop/hbase/client/AsyncHBaseAdmin.java| 14 ++- .../hbase/client/ConnectionImplementation.java | 30 ++--- .../apache/hadoop/hbase/client/HBaseAdmin.java | 23 ++-- .../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 66 +-- .../client/ShortCircuitMasterConnection.java| 30 ++--- .../hbase/shaded/protobuf/RequestConverter.java | 24 ++-- .../src/main/protobuf/Master.proto | 38 +++--- .../hbase/coprocessor/MasterObserver.java | 36 ++ .../org/apache/hadoop/hbase/master/HMaster.java | 117 +-- .../hbase/master/MasterCoprocessorHost.java | 56 + .../hadoop/hbase/master/MasterRpcServices.java | 71 ++- .../hadoop/hbase/master/MasterServices.java | 19 +-- .../hadoop/hbase/master/ServerManager.java | 14 ++- .../hbase/security/access/AccessController.java | 17 +++ .../hbase/zookeeper/DrainingServerTracker.java | 3 + .../apache/hadoop/hbase/client/TestAdmin2.java | 103 .../client/TestAsyncDecommissionAdminApi.java | 95 +++ .../hbase/client/TestAsyncDrainAdminApi.java| 101 .../hbase/master/MockNoopMasterServices.java| 15 --- .../hbase/zookeeper/TestZooKeeperACL.java | 18 +-- 23 files changed, 556 insertions(+), 387 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/a43a00e8/bin/draining_servers.rb -- diff --git a/bin/draining_servers.rb b/bin/draining_servers.rb index ea74c30..588bac4 100644 --- a/bin/draining_servers.rb +++ b/bin/draining_servers.rb @@ -17,6 +17,8 @@ # # Add or remove servers from draining mode via zookeeper +# Deprecated in 2.0, and will be removed in 3.0. Use Admin decommission +# API instead. require 'optparse' include Java http://git-wip-us.apache.org/repos/asf/hbase/blob/a43a00e8/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index 64d5e53..540b7c8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -2425,22 +2425,30 @@ public interface Admin extends Abortable, Closeable { } /** - * Mark a region server as draining to prevent additional regions from getting assigned to it. - * @param servers List of region servers to drain. + * Mark region server(s) as decommissioned to prevent additional regions from getting + * assigned to them. Optionally unload the regions on the servers. If there are multiple servers + * to be decommissioned, decommissioning them at the same time can prevent wasteful region + * movements. Region unloading is asynchronous. + * @param servers The list of servers to decommission. + * @param offload True to offload the regions from the decommissioned servers */ - void drainRegionServers(List servers) throws IOException; + void decommissionRegionServers(List servers, boolean offload) throws IOException; /** - * List region servers marked as draining to not get additional regions assigned to them. - * @return List of draining region servers. + * List region servers marked as decommissioned, which can not be assigned regions. + * @return List of decommissioned region servers. */ - List listDrainingRegionServers() throws IOException; + List listDecommissionedRegionServers() throws IOException; /** - * Remove drain from a region server to allow additional regions assignments. - * @param servers List of region servers to remove drain from. + * Remove decommission marker from a region server to allow regions assignments. + * Load regions onto the server if a list of regions is given. Region loading is + * asynchronous. + * @param server The server to
[26/50] [abbrv] hbase git commit: HBASE-19060 precommit plugin test 'hadoopcheck' should only run when java or maven files change.
HBASE-19060 precommit plugin test 'hadoopcheck' should only run when java or maven files change. Signed-off-by: Mike DrobProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/dd4dbae7 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/dd4dbae7 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/dd4dbae7 Branch: refs/heads/HBASE-18410 Commit: dd4dbae7643a7fb0beaa4e75a51d5a9c921c89b2 Parents: 89d3b0b Author: Sean Busbey Authored: Fri Oct 20 11:08:35 2017 -0500 Committer: Sean Busbey Committed: Fri Oct 20 19:35:14 2017 -0500 -- dev-support/hbase-personality.sh | 8 ++-- 1 file changed, 6 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/dd4dbae7/dev-support/hbase-personality.sh -- diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh index 27c2169..88e773e 100755 --- a/dev-support/hbase-personality.sh +++ b/dev-support/hbase-personality.sh @@ -218,7 +218,7 @@ function hadoopcheck_filefilter { local filename=$1 - if [[ ${filename} =~ \.java$ ]]; then + if [[ ${filename} =~ \.java$ ]] || [[ ${filename} =~ pom.xml$ ]]; then add_test hadoopcheck fi } @@ -241,6 +241,10 @@ function hadoopcheck_rebuild return 0 fi + if ! verify_needed_test hadoopcheck; then +return 0 + fi + big_console_header "Compiling against various Hadoop versions" # All supported Hadoop versions that we want to test the compilation with @@ -317,7 +321,7 @@ function hbaseprotoc_filefilter fi } -## @description hadoopcheck test +## @description check hbase proto compilation ## @audience private ## @stabilityevolving ## @paramrepostatus
[41/50] [abbrv] hbase git commit: HBASE-19069 Do not wrap the original CompactionLifeCycleTracker when calling CP hooks
HBASE-19069 Do not wrap the original CompactionLifeCycleTracker when calling CP hooks Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/37b29e90 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/37b29e90 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/37b29e90 Branch: refs/heads/HBASE-18410 Commit: 37b29e909defecdc580112ce6cd306710d13e9e2 Parents: 81133f8 Author: zhangduo <zhang...@apache.org> Authored: Mon Oct 23 21:10:44 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 24 10:56:14 2017 +0800 -- .../hadoop/hbase/regionserver/CompactSplit.java | 135 ++- .../TestCompactionLifeCycleTracker.java | 9 +- 2 files changed, 80 insertions(+), 64 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/37b29e90/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java index b82b346..0749f85 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java @@ -237,80 +237,73 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati } } - // A compaction life cycle tracker to trace the execution of all the compactions triggered by one - // request and delegate to the source CompactionLifeCycleTracker. It will call completed method if - // all the compactions are finished. - private static final class AggregatingCompactionLifeCycleTracker - implements CompactionLifeCycleTracker { + private interface CompactionCompleteTracker { + +default void completed(Store store) { +} + } + + private static final CompactionCompleteTracker DUMMY_COMPLETE_TRACKER = + new CompactionCompleteTracker() { + }; + + private static final class AggregatingCompleteTracker implements CompactionCompleteTracker { private final CompactionLifeCycleTracker tracker; private final AtomicInteger remaining; -public AggregatingCompactionLifeCycleTracker(CompactionLifeCycleTracker tracker, -int numberOfStores) { +public AggregatingCompleteTracker(CompactionLifeCycleTracker tracker, int numberOfStores) { this.tracker = tracker; this.remaining = new AtomicInteger(numberOfStores); } -private void tryCompleted() { +@Override +public void completed(Store store) { if (remaining.decrementAndGet() == 0) { tracker.completed(); } } - -@Override -public void notExecuted(Store store, String reason) { - tracker.notExecuted(store, reason); - tryCompleted(); -} - -@Override -public void beforeExecution(Store store) { - tracker.beforeExecution(store); -} - -@Override -public void afterExecution(Store store) { - tracker.afterExecution(store); - tryCompleted(); -} } - private CompactionLifeCycleTracker wrap(CompactionLifeCycleTracker tracker, + private CompactionCompleteTracker getCompleteTracker(CompactionLifeCycleTracker tracker, IntSupplier numberOfStores) { if (tracker == CompactionLifeCycleTracker.DUMMY) { // a simple optimization to avoid creating unnecessary objects as usually we do not care about // the life cycle of a compaction. - return tracker; + return DUMMY_COMPLETE_TRACKER; } else { - return new AggregatingCompactionLifeCycleTracker(tracker, numberOfStores.getAsInt()); + return new AggregatingCompleteTracker(tracker, numberOfStores.getAsInt()); } } @Override public synchronized void requestCompaction(HRegion region, String why, int priority, CompactionLifeCycleTracker tracker, User user) throws IOException { -requestCompactionInternal(region, why, priority, true, - wrap(tracker, () -> region.getTableDescriptor().getColumnFamilyCount()), user); +requestCompactionInternal(region, why, priority, true, tracker, + getCompleteTracker(tracker, () -> region.getTableDescriptor().getColumnFamilyCount()), user); } @Override public synchronized void requestCompaction(HRegion region, HStore store, String why, int priority, CompactionLifeCycleTracker tracker, User user) throws IOException { -requestCompactionInternal(region, store, why, priority, true, wrap(tracker, () -> 1), user); +requestCompactionInternal(region, store, why, priority, true, tracker, + getCompleteTracker(tracker, () -> 1), user)
[05/50] [abbrv] hbase git commit: HBSE-18945 Make a IA.LimitedPrivate interface for CellComparator (Ram)
HBSE-18945 Make a IA.LimitedPrivate interface for CellComparator (Ram) Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/70f4c5da Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/70f4c5da Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/70f4c5da Branch: refs/heads/HBASE-18410 Commit: 70f4c5da475a221b28e3516a23f35fc6098d4044 Parents: 9f61f8b Author: RamkrishnaAuthored: Tue Oct 17 23:17:07 2017 +0530 Committer: Ramkrishna Committed: Tue Oct 17 23:17:07 2017 +0530 -- .../hbase/backup/impl/BackupSystemTable.java| 15 +- .../hadoop/hbase/client/ConnectionUtils.java| 5 +- .../org/apache/hadoop/hbase/client/Put.java | 1 - .../org/apache/hadoop/hbase/client/Result.java | 11 +- .../hbase/filter/ColumnPaginationFilter.java| 3 +- .../hadoop/hbase/filter/ColumnRangeFilter.java | 5 +- .../hadoop/hbase/filter/CompareFilter.java | 18 +- .../apache/hadoop/hbase/filter/FilterList.java | 6 +- .../hadoop/hbase/filter/FuzzyRowFilter.java | 4 +- .../hbase/filter/InclusiveStopFilter.java | 4 +- .../hbase/filter/SingleColumnValueFilter.java | 3 +- .../hbase/client/TestClientNoCluster.java | 4 +- .../hadoop/hbase/client/TestOperation.java | 10 +- .../hadoop/hbase/filter/TestComparators.java| 38 +- .../hbase/shaded/protobuf/TestProtobufUtil.java | 4 +- .../org/apache/hadoop/hbase/CellComparator.java | 653 ++- .../apache/hadoop/hbase/CellComparatorImpl.java | 381 +++ .../java/org/apache/hadoop/hbase/CellUtil.java | 306 - .../java/org/apache/hadoop/hbase/KeyValue.java | 21 +- .../io/encoding/BufferedDataBlockEncoder.java | 5 +- .../hbase/io/encoding/DataBlockEncoder.java | 3 +- .../hbase/io/encoding/RowIndexCodecV1.java | 3 +- .../hbase/io/encoding/RowIndexEncoderV1.java| 3 +- .../hbase/io/encoding/RowIndexSeekerV1.java | 4 +- .../apache/hadoop/hbase/TestCellComparator.java | 28 +- .../org/apache/hadoop/hbase/TestKeyValue.java | 48 +- .../hadoop/hbase/util/RedundantKVGenerator.java | 6 +- .../mapreduce/IntegrationTestImportTsv.java | 6 +- .../hadoop/hbase/mapreduce/CellSortReducer.java | 4 +- .../hbase/mapreduce/HFileOutputFormat2.java | 6 +- .../apache/hadoop/hbase/mapreduce/Import.java | 4 +- .../hadoop/hbase/mapreduce/PutSortReducer.java | 4 +- .../hadoop/hbase/mapreduce/SyncTable.java | 9 +- .../hadoop/hbase/mapreduce/TextSortReducer.java | 4 +- .../hbase/codec/prefixtree/PrefixTreeCodec.java | 2 +- .../decode/PrefixTreeArrayScanner.java | 3 +- .../codec/prefixtree/decode/PrefixTreeCell.java | 3 +- .../row/data/TestRowDataNumberStrings.java | 4 +- .../hadoop/hbase/io/HalfStoreFileReader.java| 13 +- .../hadoop/hbase/io/hfile/FixedFileTrailer.java | 15 +- .../org/apache/hadoop/hbase/io/hfile/HFile.java | 3 +- .../hadoop/hbase/io/hfile/HFileBlockIndex.java | 3 +- .../hbase/io/hfile/HFilePrettyPrinter.java | 10 +- .../hadoop/hbase/io/hfile/HFileReaderImpl.java | 14 +- .../hadoop/hbase/io/hfile/HFileWriterImpl.java | 11 +- .../org/apache/hadoop/hbase/mob/MobUtils.java | 4 +- .../compactions/PartitionedMobCompactor.java| 4 +- .../regionserver/CellArrayImmutableSegment.java | 1 + .../regionserver/CellChunkImmutableSegment.java | 3 +- .../hbase/regionserver/DefaultMemStore.java | 3 +- .../hadoop/hbase/regionserver/HRegion.java | 10 +- .../hadoop/hbase/regionserver/HStore.java | 9 +- .../hbase/regionserver/ImmutableSegment.java| 2 +- .../apache/hadoop/hbase/regionserver/Store.java | 5 +- .../hadoop/hbase/regionserver/StoreFile.java| 4 +- .../hbase/regionserver/StoreFileReader.java | 5 +- .../hbase/regionserver/StoreFileWriter.java | 5 +- .../hadoop/hbase/regionserver/StoreScanner.java | 3 +- .../querymatcher/DeleteTracker.java | 7 + .../querymatcher/ExplicitColumnTracker.java | 5 +- .../querymatcher/NewVersionBehaviorTracker.java | 13 +- .../querymatcher/ScanDeleteTracker.java | 12 +- .../querymatcher/ScanQueryMatcher.java | 22 +- .../querymatcher/ScanWildcardColumnTracker.java | 4 +- .../hbase/regionserver/wal/FSWALEntry.java | 4 +- .../visibility/VisibilityController.java| 2 +- .../VisibilityNewVersionBehaivorTracker.java| 9 +- .../visibility/VisibilityScanDeleteTracker.java | 7 +- .../hadoop/hbase/util/BloomFilterFactory.java | 4 +- .../hbase/util/CollectionBackedScanner.java | 5 +- .../hadoop/hbase/util/CompressionTest.java | 4 +- .../hadoop/hbase/util/RowBloomContext.java | 1 + .../hadoop/hbase/HBaseTestingUtility.java | 2 +-
[19/50] [abbrv] hbase git commit: HBASE-16338 Remove Jackson1 deps
http://git-wip-us.apache.org/repos/asf/hbase/blob/5facaded/hbase-server/src/main/resources/hbase-webapps/regionserver/processRS.jsp -- diff --git a/hbase-server/src/main/resources/hbase-webapps/regionserver/processRS.jsp b/hbase-server/src/main/resources/hbase-webapps/regionserver/processRS.jsp index cc18d5b..f0df0c0 100644 --- a/hbase-server/src/main/resources/hbase-webapps/regionserver/processRS.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/regionserver/processRS.jsp @@ -29,7 +29,7 @@ import="java.lang.management.GarbageCollectorMXBean" import="org.apache.hadoop.hbase.util.JSONMetricUtil" import="org.apache.hadoop.hbase.procedure2.util.StringUtils" - import="org.codehaus.jackson.JsonNode" + import="com.fasterxml.jackson.databind.JsonNode" %> <% RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean(); http://git-wip-us.apache.org/repos/asf/hbase/blob/5facaded/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java index ee5a364..dab8673 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java @@ -23,6 +23,8 @@ import java.io.IOException; import java.util.Map; import java.util.NavigableSet; +import com.fasterxml.jackson.core.JsonGenerationException; +import com.fasterxml.jackson.databind.JsonMappingException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -32,8 +34,6 @@ import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.io.hfile.TestCacheConfig.DataCacheEntry; import org.apache.hadoop.hbase.io.hfile.TestCacheConfig.IndexCacheEntry; -import org.codehaus.jackson.JsonGenerationException; -import org.codehaus.jackson.map.JsonMappingException; import org.junit.After; import org.junit.Before; import org.junit.Test; http://git-wip-us.apache.org/repos/asf/hbase/blob/5facaded/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestJSONMetricUtil.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestJSONMetricUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestJSONMetricUtil.java index 30da26a..1135039 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestJSONMetricUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestJSONMetricUtil.java @@ -22,6 +22,7 @@ import java.lang.management.GarbageCollectorMXBean; import java.lang.management.ManagementFactory; import java.util.Hashtable; import java.util.List; +import java.util.Map; import javax.management.MalformedObjectNameException; import javax.management.ObjectName; @@ -29,13 +30,14 @@ import javax.management.openmbean.CompositeData; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.codehaus.jackson.JsonNode; -import org.codehaus.jackson.JsonProcessingException; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,17 +53,14 @@ public class TestJSONMetricUtil { String[] values = {"MemoryPool", "Par Eden Space"}; String[] values2 = {"MemoryPool", "Par Eden Space", "Test"}; String[] emptyValue = {}; -Hashtableproperties = JSONMetricUtil.buldKeyValueTable(keys, values); -Hashtable nullObject = JSONMetricUtil.buldKeyValueTable(keys, values2); -Hashtable nullObject1 = JSONMetricUtil.buldKeyValueTable(keys, emptyValue); -Hashtable nullObject2 = JSONMetricUtil.buldKeyValueTable(emptyKey, values2); -Hashtable nullObject3 = JSONMetricUtil.buldKeyValueTable(emptyKey, emptyValue); -assertEquals(properties.get("type"), values[0]); -assertEquals(properties.get("name"), values[1]); -assertEquals(nullObject, null); -assertEquals(nullObject1, null); -assertEquals(nullObject2, null); -assertEquals(nullObject3, null); +Map properties = JSONMetricUtil.buldKeyValueTable(keys,
[46/50] [abbrv] hbase git commit: HBASE-18904 Missing break in NEXT_ROW case of FilterList#mergeReturnCodeForOrOperator()
HBASE-18904 Missing break in NEXT_ROW case of FilterList#mergeReturnCodeForOrOperator() Signed-off-by: tedyu <yuzhih...@gmail.com> Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/9dd2ddae Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/9dd2ddae Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/9dd2ddae Branch: refs/heads/HBASE-18410 Commit: 9dd2ddaea2e9cba82d449a212f4f289d08aa4a7a Parents: 24a7ce8 Author: Biju Nair <gs.b...@gmail.com> Authored: Fri Sep 29 16:55:54 2017 -0400 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 24 11:37:45 2017 +0800 -- .../src/main/java/org/apache/hadoop/hbase/filter/FilterList.java | 2 ++ 1 file changed, 2 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/9dd2ddae/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index 3147ab0..b518645 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -566,6 +566,7 @@ final public class FilterList extends FilterBase { if (isInReturnCodes(rc, ReturnCode.NEXT_ROW)) { return ReturnCode.NEXT_ROW; } + break; case SEEK_NEXT_USING_HINT: if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.INCLUDE_AND_NEXT_COL, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) { @@ -577,6 +578,7 @@ final public class FilterList extends FilterBase { if (isInReturnCodes(rc, ReturnCode.SEEK_NEXT_USING_HINT)) { return ReturnCode.SEEK_NEXT_USING_HINT; } + break; } throw new IllegalStateException( "Received code is not valid. rc: " + rc + ", localRC: " + localRC);
[06/50] [abbrv] hbase git commit: HBASE-18960 A few bug fixes and minor improvements around batchMutate
HBASE-18960 A few bug fixes and minor improvements around batchMutate * batch validation and preparation is done before we start iterating over operations for writes * durability, familyCellMaps and observedExceptions are batch wide and are now sotred in BatchOperation, as a result durability is consistent across all operations in a batch * for all operations done by preBatchMutate() CP hook, operation status is updated to success * doWALAppend() is modified to habdle replay and is used from doMiniBatchMutate() * minor improvements Signed-off-by: Michael StackProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/e1941aa6 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e1941aa6 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e1941aa6 Branch: refs/heads/HBASE-18410 Commit: e1941aa6d14afd116a555fc93a3149f3e7c20af2 Parents: 70f4c5d Author: Umesh Agashe Authored: Fri Oct 6 15:40:05 2017 -0700 Committer: Michael Stack Committed: Tue Oct 17 13:57:00 2017 -0700 -- .../hadoop/hbase/regionserver/HRegion.java | 420 --- .../regionserver/TestHRegionReplayEvents.java | 21 + 2 files changed, 207 insertions(+), 234 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/e1941aa6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 0bef925..1cbb689 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -661,7 +661,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi private final MetricsRegion metricsRegion; private final MetricsRegionWrapperImpl metricsRegionWrapper; - private final Durability durability; + private final Durability regionDurability; private final boolean regionStatsEnabled; // Stores the replication scope of the various column families of the table // that has non-default scope @@ -787,9 +787,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi */ this.rowProcessorTimeout = conf.getLong( "hbase.hregion.row.processor.timeout", DEFAULT_ROW_PROCESSOR_TIMEOUT); -this.durability = htd.getDurability() == Durability.USE_DEFAULT -? DEFAULT_DURABILITY -: htd.getDurability(); +this.regionDurability = htd.getDurability() == Durability.USE_DEFAULT ? +DEFAULT_DURABILITY : htd.getDurability(); if (rsServices != null) { this.rsAccounting = this.rsServices.getRegionServerAccounting(); // don't initialize coprocessors if not running within a regionserver @@ -1945,13 +1944,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // upkeep. // /** - * @return returns size of largest HStore. - */ - public long getLargestHStoreSize() { -return stores.values().stream().mapToLong(HStore::getSize).max().orElse(0L); - } - - /** * Do preparation for pending compaction. * @throws IOException */ @@ -3018,21 +3010,28 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } /** - * Struct-like class that tracks the progress of a batch operation, - * accumulating status codes and tracking the index at which processing - * is proceeding. + * Struct-like class that tracks the progress of a batch operation, accumulating status codes + * and tracking the index at which processing is proceeding. These batch operations may get + * split into mini-batches for processing. */ private abstract static class BatchOperation { T[] operations; int nextIndexToProcess = 0; OperationStatus[] retCodeDetails; WALEdit[] walEditsFromCoprocessors; +// reference family cell maps directly so coprocessors can mutate them if desired +Map [] familyCellMaps; +ObservedExceptionsInBatch observedExceptions; +Durability durability; //Durability of the batch (highest durability of all operations) public BatchOperation(T[] operations) { this.operations = operations; this.retCodeDetails = new OperationStatus[operations.length]; this.walEditsFromCoprocessors = new WALEdit[operations.length]; Arrays.fill(this.retCodeDetails, OperationStatus.NOT_RUN); + familyCellMaps = new Map[operations.length]; +
[25/50] [abbrv] hbase git commit: HBASE-19061 update enforcer rules for NPE
HBASE-19061 update enforcer rules for NPE Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/89d3b0b0 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/89d3b0b0 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/89d3b0b0 Branch: refs/heads/HBASE-18410 Commit: 89d3b0b07f2ce7a84780e7088efaf9e3bce1ee5f Parents: 64d164b Author: Mike DrobAuthored: Fri Oct 20 16:04:16 2017 -0500 Committer: Mike Drob Committed: Fri Oct 20 16:04:16 2017 -0500 -- hbase-shaded/hbase-shaded-check-invariants/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/89d3b0b0/hbase-shaded/hbase-shaded-check-invariants/pom.xml -- diff --git a/hbase-shaded/hbase-shaded-check-invariants/pom.xml b/hbase-shaded/hbase-shaded-check-invariants/pom.xml index 69275a7..8592d71 100644 --- a/hbase-shaded/hbase-shaded-check-invariants/pom.xml +++ b/hbase-shaded/hbase-shaded-check-invariants/pom.xml @@ -76,7 +76,7 @@ org.codehaus.mojo extra-enforcer-rules -1.0-beta-3 +1.0-beta-6
[01/50] [abbrv] hbase git commit: HBSE-18945 Make a IA.LimitedPrivate interface for CellComparator (Ram) [Forced Update!]
Repository: hbase Updated Branches: refs/heads/HBASE-18410 a157c62de -> b5896b7a4 (forced update) http://git-wip-us.apache.org/repos/asf/hbase/blob/70f4c5da/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala -- diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala index d7e3f4f..0c51b28 100644 --- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala +++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala @@ -917,7 +917,7 @@ class HBaseContext(@transient sc: SparkContext, new WriterLength(0, new StoreFileWriter.Builder(conf, new CacheConfig(tempConf), new HFileSystem(fs)) .withBloomType(BloomType.valueOf(familyOptions.bloomType)) - .withComparator(CellComparator.COMPARATOR).withFileContext(hFileContext) + .withComparator(CellComparatorImpl.COMPARATOR).withFileContext(hFileContext) .withFilePath(new Path(familydir, "_" + UUID.randomUUID.toString.replaceAll("-", ""))) .withFavoredNodes(favoredNodes).build())
[22/50] [abbrv] hbase git commit: HBASE-19014 surefire fails; When writing xml report stdout/stderr ... No such file or directory
HBASE-19014 surefire fails; When writing xml report stdout/stderr ... No such file or directory Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/d59ed234 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d59ed234 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d59ed234 Branch: refs/heads/HBASE-18410 Commit: d59ed234ef0ba4f9c61961de306965ff39bec05f Parents: 8c6ddc1 Author: Chia-Ping TsaiAuthored: Sat Oct 21 01:22:19 2017 +0800 Committer: Chia-Ping Tsai Committed: Sat Oct 21 01:29:38 2017 +0800 -- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/d59ed234/pom.xml -- diff --git a/pom.xml b/pom.xml index 8e1d7c7..d7cbca2 100755 --- a/pom.xml +++ b/pom.xml @@ -1488,7 +1488,7 @@ hbase-rsgroup-${project.version}-tests.jar hbase-mapreduce-${project.version}-tests.jar bash -2.19.1 +2.20.1 surefire-junit47 false
[47/50] [abbrv] hbase git commit: HBASE-18160 Fix incorrect logic in FilterList.filterKeyValue
HBASE-18160 Fix incorrect logic in FilterList.filterKeyValue Signed-off-by: zhangduo <zhang...@apache.org> Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/24a7ce84 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/24a7ce84 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/24a7ce84 Branch: refs/heads/HBASE-18410 Commit: 24a7ce849f0e951ba0a84337681033c541d46276 Parents: 5c9523b Author: huzheng <open...@gmail.com> Authored: Thu Jun 8 15:58:42 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 24 11:37:45 2017 +0800 -- .../apache/hadoop/hbase/filter/FilterList.java | 541 --- .../hadoop/hbase/filter/TestFilterList.java | 148 - 2 files changed, 471 insertions(+), 218 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/24a7ce84/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index 3ff978d..3147ab0 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -90,62 +90,53 @@ final public class FilterList extends FilterBase { private Cell transformedCell = null; /** - * Constructor that takes a set of {@link Filter}s. The default operator - * MUST_PASS_ALL is assumed. + * Constructor that takes a set of {@link Filter}s and an operator. + * @param operator Operator to process filter set with. + * @param rowFilters Set of row filters. + */ + public FilterList(final Operator operator, final List rowFilters) { +reversed = checkAndGetReversed(rowFilters, reversed); +this.filters = new ArrayList<>(rowFilters); +this.operator = operator; +initPrevListForMustPassOne(rowFilters.size()); + } + + /** + * Constructor that takes a set of {@link Filter}s. The default operator MUST_PASS_ALL is assumed. * All filters are cloned to internal list. * @param rowFilters list of filters */ public FilterList(final List rowFilters) { -reversed = getReversed(rowFilters, reversed); -this.filters = new ArrayList<>(rowFilters); -initPrevListForMustPassOne(rowFilters.size()); +this(Operator.MUST_PASS_ALL, rowFilters); } /** - * Constructor that takes a var arg number of {@link Filter}s. The fefault operator - * MUST_PASS_ALL is assumed. + * Constructor that takes a var arg number of {@link Filter}s. The default operator MUST_PASS_ALL + * is assumed. * @param rowFilters */ public FilterList(final Filter... rowFilters) { -this(Arrays.asList(rowFilters)); +this(Operator.MUST_PASS_ALL, Arrays.asList(rowFilters)); } /** * Constructor that takes an operator. - * * @param operator Operator to process filter set with. */ public FilterList(final Operator operator) { -this.operator = operator; -this.filters = new ArrayList<>(); -initPrevListForMustPassOne(filters.size()); - } - - /** - * Constructor that takes a set of {@link Filter}s and an operator. - * - * @param operator Operator to process filter set with. - * @param rowFilters Set of row filters. - */ - public FilterList(final Operator operator, final List rowFilters) { -this(rowFilters); -this.operator = operator; -initPrevListForMustPassOne(rowFilters.size()); +this(operator, new ArrayList<>()); } /** * Constructor that takes a var arg number of {@link Filter}s and an operator. - * * @param operator Operator to process filter set with. * @param rowFilters Filters to use */ public FilterList(final Operator operator, final Filter... rowFilters) { -this(rowFilters); -this.operator = operator; -initPrevListForMustPassOne(rowFilters.length); +this(operator, Arrays.asList(rowFilters)); } - public void initPrevListForMustPassOne(int size) { + private void initPrevListForMustPassOne(int size) { if (operator == Operator.MUST_PASS_ONE) { if (this.prevFilterRCList == null) { prevFilterRCList = new ArrayList<>(Collections.nCopies(size, null)); @@ -156,10 +147,8 @@ final public class FilterList extends FilterBase { } } - /** * Get the operator. - * * @return operator */ public Operator getOperator() { @@ -168,7 +157,6 @@ final public class FilterList extends FilterBase { /** * Get the filters. - * * @return filters */ public List getFilters() { @@ -183,33 +171,22 @@ final public class FilterList extends FilterBase {
[44/50] [abbrv] hbase git commit: HBASE-17678 FilterList with MUST_PASS_ONE may lead to redundant cells returned
HBASE-17678 FilterList with MUST_PASS_ONE may lead to redundant cells returned Signed-off-by: tedyu <yuzhih...@gmail.com> Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/49a877db Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/49a877db Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/49a877db Branch: refs/heads/HBASE-18410 Commit: 49a877db30b8f5ea60af9340d83dec9c11a607d5 Parents: 2ebb7da Author: huzheng <open...@gmail.com> Authored: Sat May 27 16:58:00 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 24 11:30:34 2017 +0800 -- .../apache/hadoop/hbase/filter/FilterList.java | 74 +- .../hadoop/hbase/filter/TestFilterList.java | 136 +-- 2 files changed, 200 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/49a877db/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index 2f11472..3493082 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -67,6 +67,14 @@ final public class FilterList extends FilterBase { private final List filters; private Collection seekHintFilters = new ArrayList(); + /** + * Save previous return code and previous cell for every filter in filter list. For MUST_PASS_ONE, + * we use the previous return code to decide whether we should pass current cell encountered to + * the filter. For MUST_PASS_ALL, the two list are meaningless. + */ + private List prevFilterRCList = null; + private List prevCellList = null; + /** Reference Cell used by {@link #transformCell(Cell)} for validation purpose. */ private Cell referenceCell = null; @@ -88,6 +96,7 @@ final public class FilterList extends FilterBase { public FilterList(final List rowFilters) { reversed = getReversed(rowFilters, reversed); this.filters = new ArrayList<>(rowFilters); +initPrevListForMustPassOne(rowFilters.size()); } /** @@ -107,6 +116,7 @@ final public class FilterList extends FilterBase { public FilterList(final Operator operator) { this.operator = operator; this.filters = new ArrayList<>(); +initPrevListForMustPassOne(filters.size()); } /** @@ -118,6 +128,7 @@ final public class FilterList extends FilterBase { public FilterList(final Operator operator, final List rowFilters) { this(rowFilters); this.operator = operator; +initPrevListForMustPassOne(rowFilters.size()); } /** @@ -129,8 +140,21 @@ final public class FilterList extends FilterBase { public FilterList(final Operator operator, final Filter... rowFilters) { this(rowFilters); this.operator = operator; +initPrevListForMustPassOne(rowFilters.length); + } + + public void initPrevListForMustPassOne(int size) { +if (operator == Operator.MUST_PASS_ONE) { + if (this.prevCellList == null) { +prevFilterRCList = new ArrayList<>(Collections.nCopies(size, null)); + } + if (this.prevCellList == null) { +prevCellList = new ArrayList<>(Collections.nCopies(size, null)); + } +} } + /** * Get the operator. * @@ -185,6 +209,10 @@ final public class FilterList extends FilterBase { public void addFilter(List filters) { checkReversed(filters, isReversed()); this.filters.addAll(filters); +if (operator == Operator.MUST_PASS_ONE) { + this.prevFilterRCList.addAll(Collections.nCopies(filters.size(), null)); + this.prevCellList.addAll(Collections.nCopies(filters.size(), null)); +} } /** @@ -201,6 +229,10 @@ final public class FilterList extends FilterBase { int listize = filters.size(); for (int i = 0; i < listize; i++) { filters.get(i).reset(); + if (operator == Operator.MUST_PASS_ONE) { +prevFilterRCList.set(i, null); +prevCellList.set(i, null); + } } seekHintFilters.clear(); } @@ -283,6 +315,41 @@ final public class FilterList extends FilterBase { return this.transformedCell; } + /** + * For MUST_PASS_ONE, we cannot make sure that when filter-A in filter list return NEXT_COL then + * the next cell passing to filterList will be the first cell in next column, because if filter-B + * in filter list return SKIP, then the filter list will return SKIP. In this case, we should pass + * the cell following the previous cell, and it's possible that the next cell has the same column +
[27/50] [abbrv] hbase git commit: HBASE-19039 refactor shadedjars test to only run on java changes.
HBASE-19039 refactor shadedjars test to only run on java changes. Signed-off-by: Mike DrobProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b10ad9e9 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b10ad9e9 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b10ad9e9 Branch: refs/heads/HBASE-18410 Commit: b10ad9e97f67b462a4ab58ee1d449c9c319c4176 Parents: dd4dbae Author: Sean Busbey Authored: Fri Oct 20 14:39:03 2017 -0500 Committer: Sean Busbey Committed: Fri Oct 20 19:35:20 2017 -0500 -- dev-support/hbase-personality.sh | 17 ++--- 1 file changed, 14 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/b10ad9e9/dev-support/hbase-personality.sh -- diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh index 88e773e..dcf4f7a 100755 --- a/dev-support/hbase-personality.sh +++ b/dev-support/hbase-personality.sh @@ -171,12 +171,19 @@ function shadedjars_initialize { yetus_debug "initializing shaded client checks." maven_add_install shadedjars - add_test shadedjars } -function shadedjars_clean +## @description only run the test if java changes. +## @audience private +## @stabilityevolving +## @paramfilename +function shadedjars_filefilter { - "${MAVEN}" "${MAVEN_ARGS[@]}" clean -fae -pl hbase_shaded/hbase-shaded-check-invariants -am -Prelease + local filename=$1 + + if [[ ${filename} =~ \.java$ ]] || [[ ${filename} =~ pom.xml$ ]]; then +add_test shadedjars + fi } ## @description test the shaded client artifacts @@ -188,6 +195,10 @@ function shadedjars_rebuild local repostatus=$1 local logfile="${PATCH_DIR}/${repostatus}-shadedjars.txt" + if ! verify_needed_test shadedjars; then +return 0 + fi + big_console_header "Checking shaded client builds on ${repostatus}" echo_and_redirect "${logfile}" \
[34/50] [abbrv] hbase git commit: HBASE-19046 RegionObserver#postCompactSelection Avoid passing shaded ImmutableList param.
HBASE-19046 RegionObserver#postCompactSelection Avoid passing shaded ImmutableList param. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/4add40ca Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4add40ca Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4add40ca Branch: refs/heads/HBASE-18410 Commit: 4add40ca24405ca029739aaaf0b80cf5fff556f6 Parents: 2493104 Author: anoopsamjohnAuthored: Mon Oct 23 12:14:09 2017 +0530 Committer: anoopsamjohn Committed: Mon Oct 23 12:14:09 2017 +0530 -- .../org/apache/hadoop/hbase/coprocessor/RegionObserver.java | 5 + .../apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java | 3 +-- .../apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java | 3 +-- 3 files changed, 3 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/4add40ca/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index 076503f..94550df 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -22,7 +22,6 @@ package org.apache.hadoop.hbase.coprocessor; import java.io.IOException; import java.util.List; import java.util.Map; -import java.util.NavigableSet; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -44,7 +43,6 @@ import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; import org.apache.hadoop.hbase.regionserver.OperationStatus; import org.apache.hadoop.hbase.regionserver.Region; @@ -57,7 +55,6 @@ import org.apache.hadoop.hbase.regionserver.StoreFileReader; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKey; @@ -181,7 +178,7 @@ public interface RegionObserver { * @param request the requested compaction */ default void postCompactSelection(ObserverContext c, Store store, - ImmutableList selected, CompactionLifeCycleTracker tracker, + List selected, CompactionLifeCycleTracker tracker, CompactionRequest request) {} /** http://git-wip-us.apache.org/repos/asf/hbase/blob/4add40ca/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java index 8000a2f..735d7ba 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java @@ -87,7 +87,6 @@ import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKey; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; /** @@ -606,7 +605,7 @@ public class RegionCoprocessorHost * @param request the compaction request * @param user the user */ - public void postCompactSelection(final HStore store, final ImmutableList selected, + public void postCompactSelection(final HStore store, final List selected, final CompactionLifeCycleTracker tracker, final CompactionRequest request, final User user) throws IOException { execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperation(user) { http://git-wip-us.apache.org/repos/asf/hbase/blob/4add40ca/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java -- diff --git
[48/50] [abbrv] hbase git commit: HBASE-18411 Dividing FiterList into two separate sub-classes: FilterListWithOR , FilterListWithAND
HBASE-18411 Dividing FiterList into two separate sub-classes: FilterListWithOR , FilterListWithAND Signed-off-by: zhangduo <zhang...@apache.org> Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b32bff02 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b32bff02 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b32bff02 Branch: refs/heads/HBASE-18410 Commit: b32bff028d00fedee5fb6e1ae8c587fd9e5f3b1e Parents: 9dd2dda Author: huzheng <open...@gmail.com> Authored: Tue Oct 10 20:01:48 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 24 11:39:31 2017 +0800 -- .../apache/hadoop/hbase/filter/FilterList.java | 661 ++- .../hadoop/hbase/filter/FilterListBase.java | 159 + .../hadoop/hbase/filter/FilterListWithAND.java | 273 .../hadoop/hbase/filter/FilterListWithOR.java | 383 +++ .../hadoop/hbase/filter/TestFilterList.java | 89 +++ 5 files changed, 962 insertions(+), 603 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/b32bff02/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index b518645..97392d1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.yetus.audience.InterfaceAudience; @@ -37,86 +38,60 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; /** - * Implementation of {@link Filter} that represents an ordered List of Filters - * which will be evaluated with a specified boolean operator {@link Operator#MUST_PASS_ALL} - * (AND) or {@link Operator#MUST_PASS_ONE} (OR). - * Since you can use Filter Lists as children of Filter Lists, you can create a - * hierarchy of filters to be evaluated. - * - * - * {@link Operator#MUST_PASS_ALL} evaluates lazily: evaluation stops as soon as one filter does - * not include the KeyValue. - * - * - * {@link Operator#MUST_PASS_ONE} evaluates non-lazily: all filters are always evaluated. - * - * + * Implementation of {@link Filter} that represents an ordered List of Filters which will be + * evaluated with a specified boolean operator {@link Operator#MUST_PASS_ALL} (AND) or + * {@link Operator#MUST_PASS_ONE} (OR). Since you can use Filter Lists as children of + * Filter Lists, you can create a hierarchy of filters to be evaluated. + * {@link Operator#MUST_PASS_ALL} evaluates lazily: evaluation stops as soon as one filter does not + * include the KeyValue. + * {@link Operator#MUST_PASS_ONE} evaluates non-lazily: all filters are always evaluated. * Defaults to {@link Operator#MUST_PASS_ALL}. */ @InterfaceAudience.Public final public class FilterList extends FilterBase { + /** set operator */ @InterfaceAudience.Public - public static enum Operator { + public enum Operator { /** !AND */ MUST_PASS_ALL, /** !OR */ MUST_PASS_ONE } - private static final int MAX_LOG_FILTERS = 5; - private Operator operator = Operator.MUST_PASS_ALL; - private final List filters; - private Collection seekHintFilters = new ArrayList(); - - /** - * Save previous return code and previous cell for every filter in filter list. For MUST_PASS_ONE, - * we use the previous return code to decide whether we should pass current cell encountered to - * the filter. For MUST_PASS_ALL, the two list are meaningless. - */ - private List prevFilterRCList = null; - private List prevCellList = null; - - /** Reference Cell used by {@link #transformCell(Cell)} for validation purpose. */ - private Cell referenceCell = null; - - /** - * When filtering a given Cell in {@link #filterKeyValue(Cell)}, - * this stores the transformed Cell to be returned by {@link #transformCell(Cell)}. - * - * Individual filters transformation are applied only when the filter includes the Cell. - * Transformations are composed in the order specified by {@link #filters}. - */ - private Cell transformedCell = null; + private Operator operator; + private FilterListBase filterListBase; /** * Constructor that takes a set of {@link Filter
[16/50] [abbrv] hbase git commit: HBASE-19026 TestLockProcedure#testRemoteNamespaceLockRecovery fails
HBASE-19026 TestLockProcedure#testRemoteNamespaceLockRecovery fails Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/909e5f2f Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/909e5f2f Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/909e5f2f Branch: refs/heads/HBASE-18410 Commit: 909e5f2f14186709ceb4697f76103c83125c8b49 Parents: 4a7b430 Author: tedyuAuthored: Thu Oct 19 11:07:57 2017 -0700 Committer: tedyu Committed: Thu Oct 19 11:07:57 2017 -0700 -- .../org/apache/hadoop/hbase/master/locking/TestLockProcedure.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/909e5f2f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java index ce02395..a817bd5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java @@ -397,7 +397,7 @@ public class TestLockProcedure { sendHeartbeatAndCheckLocked(procId, true); Thread.sleep(HEARTBEAT_TIMEOUT/2); sendHeartbeatAndCheckLocked(procId, true); -Thread.sleep(2 * HEARTBEAT_TIMEOUT); +Thread.sleep(2 * HEARTBEAT_TIMEOUT + HEARTBEAT_TIMEOUT/2); sendHeartbeatAndCheckLocked(procId, false); ProcedureTestingUtility.waitProcedure(procExec, procId); ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
[15/50] [abbrv] hbase git commit: Revert "HBASE-19042 Oracle Java 8u144 downloader broken in precommit check"
Revert "HBASE-19042 Oracle Java 8u144 downloader broken in precommit check" This reverts commit 9e688117bad3cb4826c7201bb359672676389620. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/4a7b4303 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4a7b4303 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4a7b4303 Branch: refs/heads/HBASE-18410 Commit: 4a7b4303979ffe9896811f633141681669e1c20d Parents: 9e68811 Author: zhangduo <zhang...@apache.org> Authored: Thu Oct 19 16:03:28 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Thu Oct 19 16:03:28 2017 +0800 -- dev-support/docker/Dockerfile | 29 ++--- 1 file changed, 18 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/4a7b4303/dev-support/docker/Dockerfile -- diff --git a/dev-support/docker/Dockerfile b/dev-support/docker/Dockerfile index c23c70d..62c6030 100644 --- a/dev-support/docker/Dockerfile +++ b/dev-support/docker/Dockerfile @@ -65,18 +65,18 @@ RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ zlib1g-dev ### -# OpenJDK 8 +# Oracle Java ### RUN echo "dot_style = mega" > "/root/.wgetrc" RUN echo "quiet = on" >> "/root/.wgetrc" RUN apt-get -q update && apt-get -q install --no-install-recommends -y software-properties-common -RUN add-apt-repository -y ppa:openjdk-r/ppa -RUN apt-get -q update -RUN apt-get -q install --no-install-recommends -y openjdk-8-jdk -RUN update-alternatives --config java -RUN update-alternatives --config javac +RUN add-apt-repository -y ppa:webupd8team/java + +# Auto-accept the Oracle JDK license +RUN echo oracle-java8-installer shared/accepted-oracle-license-v1-1 select true | sudo /usr/bin/debconf-set-selections +RUN apt-get -q update && apt-get -q install --no-install-recommends -y oracle-java8-installer # Apps that require Java @@ -131,16 +131,23 @@ RUN pip install python-dateutil # Install Ruby 2, based on Yetus 0.4.0 dockerfile ### RUN echo 'gem: --no-rdoc --no-ri' >> /root/.gemrc -RUN apt-add-repository ppa:brightbox/ruby-ng -RUN apt-get -q update +RUN apt-get -q install -y ruby2.0 +# +# on trusty, the above installs ruby2.0 and ruby (1.9.3) exes +# but update-alternatives is broken, so we need to do some work +# to make 2.0 actually the default without the system flipping out +# +# See https://bugs.launchpad.net/ubuntu/+source/ruby2.0/+bug/1310292 +# +RUN dpkg-divert --add --rename --divert /usr/bin/ruby.divert /usr/bin/ruby +RUN dpkg-divert --add --rename --divert /usr/bin/gem.divert /usr/bin/gemrc +RUN update-alternatives --install /usr/bin/ruby ruby /usr/bin/ruby2.0 1 +RUN update-alternatives --install /usr/bin/gem gem /usr/bin/gem2.0 1 -RUN apt-get -q install --no-install-recommends -y ruby2.2 ruby-switch -RUN ruby-switch --set ruby2.2 # Install rubocop ### -RUN gem install rake RUN gem install rubocop
[39/50] [abbrv] hbase git commit: HBASE-19072 Missing beak in catch block of InterruptedException in HRegion#waitForFlushes()
HBASE-19072 Missing beak in catch block of InterruptedException in HRegion#waitForFlushes() Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b7db62c7 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b7db62c7 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b7db62c7 Branch: refs/heads/HBASE-18410 Commit: b7db62c702ef27b79365cfa62a8afee9042bcc6b Parents: a1bc20a Author: tedyuAuthored: Mon Oct 23 19:34:11 2017 -0700 Committer: tedyu Committed: Mon Oct 23 19:34:11 2017 -0700 -- .../src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java | 1 + 1 file changed, 1 insertion(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/b7db62c7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 9022e1f..99f5c35 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -1791,6 +1791,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // essentially ignore and propagate the interrupt back up LOG.warn("Interrupted while waiting"); interrupted = true; +break; } } } finally {
[45/50] [abbrv] hbase git commit: HBASE-15410 Utilize the max seek value when all Filters in MUST_PASS_ALL FilterList return SEEK_NEXT_USING_HINT
HBASE-15410 Utilize the max seek value when all Filters in MUST_PASS_ALL FilterList return SEEK_NEXT_USING_HINT Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/5c9523b7 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5c9523b7 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5c9523b7 Branch: refs/heads/HBASE-18410 Commit: 5c9523b757e5b0f6b8d5ef1829f9b199fc2f73ef Parents: 3f5f2a5 Author: tedyu <yuzhih...@gmail.com> Authored: Thu Sep 7 04:07:09 2017 -0700 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 24 11:35:24 2017 +0800 -- .../main/java/org/apache/hadoop/hbase/filter/FilterList.java| 5 +++-- .../java/org/apache/hadoop/hbase/filter/TestFilterList.java | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/5c9523b7/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index 83db1f2..3ff978d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -28,12 +28,13 @@ import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; -import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException; /** * Implementation of {@link Filter} that represents an ordered List of Filters http://git-wip-us.apache.org/repos/asf/hbase/blob/5c9523b7/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java index 46d44de..e414729 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java @@ -502,8 +502,8 @@ public class TestFilterList { // Should take the min if given two hints FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } )); -assertEquals(0, - CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), minKeyValue)); +assertEquals(0, CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), + minKeyValue)); // Should have no hint if any filter has no hint filterList = new FilterList(Operator.MUST_PASS_ONE,
[49/50] [abbrv] hbase git commit: HBASE-18879 HBase FilterList cause KeyOnlyFilter not work
HBASE-18879 HBase FilterList cause KeyOnlyFilter not work Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a17094f8 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a17094f8 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a17094f8 Branch: refs/heads/HBASE-18410 Commit: a17094f8e230e8350d97432243c9e69620e62619 Parents: b32bff0 Author: huzheng <open...@gmail.com> Authored: Wed Oct 11 21:17:03 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 24 11:39:31 2017 +0800 -- .../apache/hadoop/hbase/filter/FilterList.java | 6 +++ .../hadoop/hbase/filter/FilterListBase.java | 3 ++ .../hadoop/hbase/filter/FilterListWithAND.java | 22 + .../hadoop/hbase/filter/FilterListWithOR.java | 22 + .../hadoop/hbase/filter/TestFilterList.java | 48 5 files changed, 85 insertions(+), 16 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/a17094f8/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index 97392d1..e87f1b3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -72,6 +72,8 @@ final public class FilterList extends FilterBase { filterListBase = new FilterListWithAND(filters); } else if (operator == Operator.MUST_PASS_ONE) { filterListBase = new FilterListWithOR(filters); +} else { + throw new IllegalArgumentException("Invalid operator: " + operator); } this.operator = operator; } @@ -168,6 +170,10 @@ final public class FilterList extends FilterBase { return filterListBase.transformCell(c); } + ReturnCode internalFilterKeyValue(Cell c, Cell currentTransformedCell) throws IOException { +return this.filterListBase.internalFilterKeyValue(c, currentTransformedCell); + } + @Override public ReturnCode filterKeyValue(Cell c) throws IOException { return filterListBase.filterKeyValue(c); http://git-wip-us.apache.org/repos/asf/hbase/blob/a17094f8/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java index 7fa0245..60b0dc1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java @@ -107,6 +107,9 @@ public abstract class FilterListBase extends FilterBase { return cell; } + abstract ReturnCode internalFilterKeyValue(Cell c, Cell currentTransformedCell) + throws IOException; + /** * Filters that never filter by modifying the returned List of Cells can inherit this * implementation that does nothing. {@inheritDoc} http://git-wip-us.apache.org/repos/asf/hbase/blob/a17094f8/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java index fa979c0..4909dfd 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java @@ -147,16 +147,26 @@ public class FilterListWithAND extends FilterListBase { "Received code is not valid. rc: " + rc + ", localRC: " + localRC); } - private ReturnCode filterKeyValueWithMustPassAll(Cell c) throws IOException { + @Override + ReturnCode internalFilterKeyValue(Cell c, Cell currentTransformedCell) throws IOException { +if (isEmpty()) { + return ReturnCode.INCLUDE; +} ReturnCode rc = ReturnCode.INCLUDE; -Cell transformed = c; +Cell transformed = currentTransformedCell; +this.referenceCell = c; this.seekHintFilter.clear(); for (int i = 0, n = filters.size(); i < n; i++) { Filter filter = filters.get(i); if (filter.filterAllRemaining()) { return ReturnCode.NEXT_ROW; } - ReturnCode localRC = filter.filterKeyValue(c); + ReturnCode localRC; + if (filter instanceof FilterList) { +localRC = ((FilterList) filter).internal
[31/50] [abbrv] hbase git commit: HBASE-19007 Align Services Interfaces in Master and RegionServer
HBASE-19007 Align Services Interfaces in Master and RegionServer Purges Server, MasterServices, and RegionServerServices from CoprocessorEnvironments. Replaces removed functionality with a set of carefully curated methods on the *CoprocessorEnvironment implementations (Varies by CoprocessorEnvironment in that the MasterCoprocessorEnvironment has Master-type facility exposed, and so on). A few core Coprocessors that should long ago have been converted to be integral, violate their context; e.g. a RegionCoprocessor wants free access to a hosting RegionServer (which may or may not be present). Rather than let these violators make us corrupte the CP API, instead, we've made up a hacky system that allows core Coprocessors access to internals. A new CoreCoprocessor Annotation has been introduced. When loading Coprocessors, if the instance is annotated CoreCoprocessor, we pass it an Environment that has been padded w/ extra-stuff. On invocation, CoreCoprocessors know how to route their way to these extras in their environment. See the *CoprocessoHost for how the do the check for CoreCoprocessor and pass a fatter *Coprocessor, one that allows getting of either a RegionServerService or MasterService out of the environment via Marker Interfaces. Removed org.apache.hadoop.hbase.regionserver.CoprocessorRegionServerServices M hbase-endpoint/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java This Endpoint has been deprecated because its functionality has been moved to core. Marking it a CoreCoprocessor in the meantime to minimize change. M hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java This should be integral to hbase. Meantime, marking it CoreCoprocessor. M hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java Added doc on where it is used and added back a few methods we'd removed. A hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoreCoprocessor.java New annotation for core hbase coprocessors. They get richer environment on coprocessor loading. A hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/HasMasterServices.java A hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/HasRegionServerServices.java Marker Interface to access extras if present. M hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterCoprocessorEnvironment.java Purge MasterServices access. Allow CPs a Connection. M hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java Purge RegionServerServices access. Allow CPs a Connection. M hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerCoprocessorEnvironment.java Purge MasterServices access. Allow CPs a Connection. M hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterSpaceQuotaObserver.java M hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaCache.java We no longer have access to MasterServices. Don't need it actually. Use short-circuiting Admin instead. D hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CoprocessorRegionServerServices.java Removed. Not needed now we do CP Env differently. M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java No need to go via RSS to getOnlineTables; just use HRS. And so on. Adds tests to ensure we can only get at extra info if the CP has been properly marked. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/38879fb3 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/38879fb3 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/38879fb3 Branch: refs/heads/HBASE-18410 Commit: 38879fb3ffa88ca95b15c61656a92e72c0ed996f Parents: 592d541 Author: Guanghao ZhangAuthored: Mon Oct 16 17:12:37 2017 +0800 Committer: Michael Stack Committed: Sat Oct 21 11:06:30 2017 -0700 -- .../apache/hadoop/hbase/client/Connection.java | 2 +- .../hadoop/hbase/client/ConnectionUtils.java| 64 +++ .../apache/hadoop/hbase/client/HBaseAdmin.java | 2 +- .../security/access/SecureBulkLoadEndpoint.java | 6 +- .../hbase/rsgroup/RSGroupAdminEndpoint.java | 10 +- .../java/org/apache/hadoop/hbase/Server.java| 21 +++- .../hbase/coprocessor/BaseEnvironment.java | 3 +- .../hbase/coprocessor/CoreCoprocessor.java | 45 .../hbase/coprocessor/HasMasterServices.java| 37 ++ .../coprocessor/HasRegionServerServices.java| 37 ++ .../MasterCoprocessorEnvironment.java | 24 +++- .../RegionCoprocessorEnvironment.java | 26 - .../RegionServerCoprocessorEnvironment.java | 24 +++- .../org/apache/hadoop/hbase/ipc/RpcServer.java | 9 +- .../hbase/master/MasterCoprocessorHost.java | 48 ++--
[08/50] [abbrv] hbase git commit: HBASE-19032 Set mimetype for patches uploaded by submit-patch.py
HBASE-19032 Set mimetype for patches uploaded by submit-patch.py Change-Id: I38e64174e2525cd6a929922b2612c91d660d Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/5368fd5b Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5368fd5b Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5368fd5b Branch: refs/heads/HBASE-18410 Commit: 5368fd5bf0a281e67c4dde25816a1362d1f0a3f0 Parents: 41cc9a1 Author: Apekshit SharmaAuthored: Tue Oct 17 15:32:39 2017 -0700 Committer: Apekshit Sharma Committed: Tue Oct 17 15:43:07 2017 -0700 -- dev-support/submit-patch.py | 8 1 file changed, 4 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/5368fd5b/dev-support/submit-patch.py -- diff --git a/dev-support/submit-patch.py b/dev-support/submit-patch.py index 577be52..ad39495 100755 --- a/dev-support/submit-patch.py +++ b/dev-support/submit-patch.py @@ -205,12 +205,12 @@ def get_credentials(): return creds -def attach_patch_to_jira(issue_url, patch_filepath, creds): +def attach_patch_to_jira(issue_url, patch_filepath, patch_filename, creds): # Upload patch to jira using REST API. headers = {'X-Atlassian-Token': 'no-check'} -files = {'file': open(patch_filepath, 'rb')} +files = {'file': (patch_filename, open(patch_filepath, 'rb'), 'text/plain')} jira_auth = requests.auth.HTTPBasicAuth(creds['jira_username'], creds['jira_password']) -attachment_url = issue_url + "/attachments" +attachment_url = issue_url + "/attachments" r = requests.post(attachment_url, headers = headers, files = files, auth = jira_auth) assert_status_code(r, 200, "uploading patch to jira") @@ -256,7 +256,7 @@ if args.jira_id is not None: creds = get_credentials() issue_url = "https://issues.apache.org/jira/rest/api/2/issue/; + args.jira_id -attach_patch_to_jira(issue_url, patch_filepath, creds) +attach_patch_to_jira(issue_url, patch_filepath, patch_filename, creds) if not args.skip_review_board: rb_auth = requests.auth.HTTPBasicAuth(creds['rb_username'], creds['rb_password'])
[02/50] [abbrv] hbase git commit: HBSE-18945 Make a IA.LimitedPrivate interface for CellComparator (Ram)
http://git-wip-us.apache.org/repos/asf/hbase/blob/70f4c5da/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java index 13589fb..39419ca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java @@ -41,7 +41,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility; @@ -117,7 +117,7 @@ public class TestHFile { HFileContext meta = new HFileContextBuilder().withBlockSize(64 * 1024).build(); StoreFileWriter sfw = new StoreFileWriter.Builder(conf, fs).withOutputDir(storeFileParentDir) - .withComparator(CellComparator.COMPARATOR).withFileContext(meta).build(); + .withComparator(CellComparatorImpl.COMPARATOR).withFileContext(meta).build(); final int rowLen = 32; Random RNG = new Random(); @@ -319,7 +319,7 @@ public class TestHFile { Writer writer = HFile.getWriterFactory(conf, cacheConf) .withOutputStream(fout) .withFileContext(meta) -.withComparator(CellComparator.COMPARATOR) +.withComparator(CellComparatorImpl.COMPARATOR) .create(); LOG.info(writer); writeRecords(writer, useTags); @@ -486,72 +486,72 @@ public class TestHFile { 9, KeyValue.Type.Maximum.getCode(), HConstants.EMPTY_BYTE_ARRAY); -Cell mid = HFileWriterImpl.getMidpoint(CellComparator.COMPARATOR, left, right); -assertTrue(CellComparator.COMPARATOR.compareKeyIgnoresMvcc(left, mid) <= 0); -assertTrue(CellComparator.COMPARATOR.compareKeyIgnoresMvcc(mid, right) == 0); +Cell mid = HFileWriterImpl.getMidpoint(CellComparatorImpl.COMPARATOR, left, right); +assertTrue(CellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.COMPARATOR, left, mid) <= 0); +assertTrue(CellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.COMPARATOR, mid, right) == 0); } @Test public void testGetShortMidpoint() { Cell left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a")); Cell right = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a")); -Cell mid = HFileWriterImpl.getMidpoint(CellComparator.COMPARATOR, left, right); -assertTrue(CellComparator.COMPARATOR.compareKeyIgnoresMvcc(left, mid) <= 0); -assertTrue(CellComparator.COMPARATOR.compareKeyIgnoresMvcc(mid, right) <= 0); +Cell mid = HFileWriterImpl.getMidpoint(CellComparatorImpl.COMPARATOR, left, right); +assertTrue(CellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.COMPARATOR, left, mid) <= 0); +assertTrue(CellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.COMPARATOR, mid, right) <= 0); left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a")); right = CellUtil.createCell(Bytes.toBytes("b"), Bytes.toBytes("a"), Bytes.toBytes("a")); -mid = HFileWriterImpl.getMidpoint(CellComparator.COMPARATOR, left, right); -assertTrue(CellComparator.COMPARATOR.compareKeyIgnoresMvcc(left, mid) < 0); -assertTrue(CellComparator.COMPARATOR.compareKeyIgnoresMvcc(mid, right) <= 0); +mid = HFileWriterImpl.getMidpoint(CellComparatorImpl.COMPARATOR, left, right); +assertTrue(CellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.COMPARATOR, left, mid) < 0); +assertTrue(CellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.COMPARATOR, mid, right) <= 0); left = CellUtil.createCell(Bytes.toBytes("g"), Bytes.toBytes("a"), Bytes.toBytes("a")); right = CellUtil.createCell(Bytes.toBytes("i"), Bytes.toBytes("a"), Bytes.toBytes("a")); -mid = HFileWriterImpl.getMidpoint(CellComparator.COMPARATOR, left, right); -assertTrue(CellComparator.COMPARATOR.compareKeyIgnoresMvcc(left, mid) < 0); -assertTrue(CellComparator.COMPARATOR.compareKeyIgnoresMvcc(mid, right) <= 0); +mid = HFileWriterImpl.getMidpoint(CellComparatorImpl.COMPARATOR, left, right); +assertTrue(CellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.COMPARATOR, left, mid) < 0); +assertTrue(CellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.COMPARATOR, mid, right) <= 0); left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a")); right = CellUtil.createCell(Bytes.toBytes("bbb"), Bytes.toBytes("a"), Bytes.toBytes("a")); -mid =
[42/50] [abbrv] hbase git commit: HBASE-18410 disable the HBASE-18957 test until we can fix it on the feature branch.
HBASE-18410 disable the HBASE-18957 test until we can fix it on the feature branch. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/2ebb7da6 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/2ebb7da6 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/2ebb7da6 Branch: refs/heads/HBASE-18410 Commit: 2ebb7da688e214becbdcc56c311fdb84225aeef1 Parents: 37b29e9 Author: Sean Busbey <bus...@apache.org> Authored: Mon Oct 9 15:24:00 2017 -0500 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 24 11:16:32 2017 +0800 -- .../java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java | 1 + 1 file changed, 1 insertion(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/2ebb7da6/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java index dd2399f..590b26e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java @@ -58,6 +58,7 @@ public class TestFilterListOnMini { TEST_UTIL.shutdownMiniCluster(); } + @Ignore("HBASE-18410 Should not merge without this test running.") @Test public void testFiltersWithOR() throws Exception { TableName tn = TableName.valueOf(name.getMethodName());
[43/50] [abbrv] hbase git commit: HBASE-17678 FilterList with MUST_PASS_ONE lead to redundancy cells returned - addendum
HBASE-17678 FilterList with MUST_PASS_ONE lead to redundancy cells returned - addendum Signed-off-by: tedyu <yuzhih...@gmail.com> Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/3f5f2a54 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/3f5f2a54 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/3f5f2a54 Branch: refs/heads/HBASE-18410 Commit: 3f5f2a544691ba156705c87e3fd71c02ca1e7f5c Parents: 49a877d Author: huzheng <open...@gmail.com> Authored: Wed Jun 7 14:49:29 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 24 11:30:34 2017 +0800 -- .../java/org/apache/hadoop/hbase/filter/FilterList.java | 12 ++-- 1 file changed, 10 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/3f5f2a54/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index 3493082..83db1f2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; @@ -145,7 +146,7 @@ final public class FilterList extends FilterBase { public void initPrevListForMustPassOne(int size) { if (operator == Operator.MUST_PASS_ONE) { - if (this.prevCellList == null) { + if (this.prevFilterRCList == null) { prevFilterRCList = new ArrayList<>(Collections.nCopies(size, null)); } if (this.prevCellList == null) { @@ -407,7 +408,14 @@ final public class FilterList extends FilterBase { ReturnCode localRC = filter.filterKeyValue(c); // Update previous cell and return code we encountered. prevFilterRCList.set(i, localRC); -prevCellList.set(i, c); +if (c == null || localRC == ReturnCode.INCLUDE || localRC == ReturnCode.SKIP) { + // If previous return code is INCLUDE or SKIP, we should always pass the next cell to the + // corresponding sub-filter(need not test shouldPassCurrentCellToFilter() method), So we + // need not save current cell to prevCellList for saving heap memory. + prevCellList.set(i, null); +} else { + prevCellList.set(i, KeyValueUtil.toNewKeyCell(c)); +} if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) { seenNonHintReturnCode = true;
[13/50] [abbrv] hbase git commit: HBASE-18418 Remove apache_hbase_topology from dev-support
HBASE-18418 Remove apache_hbase_topology from dev-support Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/3acb0817 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/3acb0817 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/3acb0817 Branch: refs/heads/HBASE-18410 Commit: 3acb081787a4289d86d977db26bedaf6a42172ce Parents: c16eb78 Author: Dima SpivakAuthored: Thu Jul 20 10:08:11 2017 -0700 Committer: Dima Spivak Committed: Wed Oct 18 14:08:26 2017 -0700 -- dev-support/apache_hbase_topology/Dockerfile| 24 -- dev-support/apache_hbase_topology/README.md | 49 --- dev-support/apache_hbase_topology/__init__.py | 15 - dev-support/apache_hbase_topology/actions.py| 421 --- .../apache_hbase_topology/configurations.cfg| 80 dev-support/apache_hbase_topology/profile.cfg | 82 dev-support/apache_hbase_topology/ssh/id_rsa| 44 -- .../apache_hbase_topology/ssh/id_rsa.pub| 18 - 8 files changed, 733 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/3acb0817/dev-support/apache_hbase_topology/Dockerfile -- diff --git a/dev-support/apache_hbase_topology/Dockerfile b/dev-support/apache_hbase_topology/Dockerfile deleted file mode 100644 index 714a55c..000 --- a/dev-support/apache_hbase_topology/Dockerfile +++ /dev/null @@ -1,24 +0,0 @@ -# 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 -# -# 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. -FROM debian:wheezy - -ENV TOPOLOGY_NAME=apache_hbase -ADD . /root/clusterdock/clusterdock/topologies/${TOPOLOGY_NAME} - -RUN find /root -type f -name id_rsa -exec chmod 600 {} \; - -VOLUME /root/clusterdock/clusterdock/topologies/${TOPOLOGY_NAME} -CMD ["/true"] http://git-wip-us.apache.org/repos/asf/hbase/blob/3acb0817/dev-support/apache_hbase_topology/README.md -- diff --git a/dev-support/apache_hbase_topology/README.md b/dev-support/apache_hbase_topology/README.md deleted file mode 100644 index 018ee99..000 --- a/dev-support/apache_hbase_topology/README.md +++ /dev/null @@ -1,49 +0,0 @@ - -# apache_hbase clusterdock topology - -## Overview -*clusterdock* is a framework for creating Docker-based container clusters. Unlike regular Docker -containers, which tend to run single processes and then exit once the process terminates, these -container clusters are characterized by the execution of an init process in daemon mode. As such, -the containers act more like "fat containers" or "light VMs;" entities with accessible IP addresses -which emulate standalone hosts. - -*clusterdock* relies upon the notion of a topology to define how clusters should be built into -images and then what to do with those images to start Docker container clusters. - -## Usage -The *clusterdock* framework is designed to be run out of its own container while affecting -operations on the host. To avoid problems that might result from incorrectly -formatting this framework invocation, a Bash helper script (`clusterdock.sh`) can be sourced on a -host that has Docker installed. Afterwards, running any of the binaries intended to carry -out *clusterdock* actions can be done using the `clusterdock_run` command. -``` -wget https://raw.githubusercontent.com/cloudera/clusterdock/master/clusterdock.sh -# ALWAYS INSPECT SCRIPTS FROM THE INTERNET BEFORE SOURCING THEM. -source clusterdock.sh -``` - -Since the *clusterdock* framework itself lives outside of Apache HBase, an environmental variable -is used to let the helper script know where to find an image of the *apache_hbase* topology. To -start a four-node Apache HBase cluster with default versions, you would simply run -``` -CLUSTERDOCK_TOPOLOGY_IMAGE=apache_hbase_topology_location clusterdock_run \ -./bin/start_cluster apache_hbase --secondary-nodes='node-{2..4}' -``` http://git-wip-us.apache.org/repos/asf/hbase/blob/3acb0817/dev-support/apache_hbase_topology/__init__.py
[14/50] [abbrv] hbase git commit: HBASE-19042 Oracle Java 8u144 downloader broken in precommit check
HBASE-19042 Oracle Java 8u144 downloader broken in precommit check Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/9e688117 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/9e688117 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/9e688117 Branch: refs/heads/HBASE-18410 Commit: 9e688117bad3cb4826c7201bb359672676389620 Parents: 3acb081 Author: zhangduo <zhang...@apache.org> Authored: Thu Oct 19 14:49:09 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Thu Oct 19 15:32:48 2017 +0800 -- dev-support/docker/Dockerfile | 29 +++-- 1 file changed, 11 insertions(+), 18 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/9e688117/dev-support/docker/Dockerfile -- diff --git a/dev-support/docker/Dockerfile b/dev-support/docker/Dockerfile index 62c6030..c23c70d 100644 --- a/dev-support/docker/Dockerfile +++ b/dev-support/docker/Dockerfile @@ -65,18 +65,18 @@ RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ zlib1g-dev ### -# Oracle Java +# OpenJDK 8 ### RUN echo "dot_style = mega" > "/root/.wgetrc" RUN echo "quiet = on" >> "/root/.wgetrc" RUN apt-get -q update && apt-get -q install --no-install-recommends -y software-properties-common -RUN add-apt-repository -y ppa:webupd8team/java - -# Auto-accept the Oracle JDK license -RUN echo oracle-java8-installer shared/accepted-oracle-license-v1-1 select true | sudo /usr/bin/debconf-set-selections -RUN apt-get -q update && apt-get -q install --no-install-recommends -y oracle-java8-installer +RUN add-apt-repository -y ppa:openjdk-r/ppa +RUN apt-get -q update +RUN apt-get -q install --no-install-recommends -y openjdk-8-jdk +RUN update-alternatives --config java +RUN update-alternatives --config javac # Apps that require Java @@ -131,23 +131,16 @@ RUN pip install python-dateutil # Install Ruby 2, based on Yetus 0.4.0 dockerfile ### RUN echo 'gem: --no-rdoc --no-ri' >> /root/.gemrc -RUN apt-get -q install -y ruby2.0 -# -# on trusty, the above installs ruby2.0 and ruby (1.9.3) exes -# but update-alternatives is broken, so we need to do some work -# to make 2.0 actually the default without the system flipping out -# -# See https://bugs.launchpad.net/ubuntu/+source/ruby2.0/+bug/1310292 -# -RUN dpkg-divert --add --rename --divert /usr/bin/ruby.divert /usr/bin/ruby -RUN dpkg-divert --add --rename --divert /usr/bin/gem.divert /usr/bin/gemrc -RUN update-alternatives --install /usr/bin/ruby ruby /usr/bin/ruby2.0 1 -RUN update-alternatives --install /usr/bin/gem gem /usr/bin/gem2.0 1 +RUN apt-add-repository ppa:brightbox/ruby-ng +RUN apt-get -q update +RUN apt-get -q install --no-install-recommends -y ruby2.2 ruby-switch +RUN ruby-switch --set ruby2.2 # Install rubocop ### +RUN gem install rake RUN gem install rubocop
[12/50] [abbrv] hbase git commit: HBASE-19038 precommit mvn install should run from root on patch
HBASE-19038 precommit mvn install should run from root on patch Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c16eb788 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c16eb788 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c16eb788 Branch: refs/heads/HBASE-18410 Commit: c16eb7881fa530a2dd626c1e06e294c2d198af22 Parents: e320df5 Author: Mike DrobAuthored: Wed Oct 18 10:20:03 2017 -0500 Committer: Mike Drob Committed: Wed Oct 18 10:41:17 2017 -0500 -- dev-support/hbase-personality.sh | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/c16eb788/dev-support/hbase-personality.sh -- diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh index 43371f8..9b23e11 100755 --- a/dev-support/hbase-personality.sh +++ b/dev-support/hbase-personality.sh @@ -84,9 +84,7 @@ function personality_modules extra="-DHBasePatchProcess" - if [[ ${repostatus} == branch - && ${testtype} == mvninstall ]] || - [[ "${BUILDMODE}" == full ]];then + if [[ ${testtype} == mvninstall ]] || [[ "${BUILDMODE}" == full ]]; then personality_enqueue_module . ${extra} return fi
[38/50] [abbrv] hbase git commit: HBASE-18893 remove add/delete/modify column
HBASE-18893 remove add/delete/modify column Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a1bc20ab Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a1bc20ab Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a1bc20ab Branch: refs/heads/HBASE-18410 Commit: a1bc20ab5886acd65cc2b693eccf8e736d373b6b Parents: 880b26d Author: Mike DrobAuthored: Tue Oct 17 16:47:41 2017 -0500 Committer: Mike Drob Committed: Mon Oct 23 20:02:25 2017 -0500 -- .../src/main/protobuf/MasterProcedure.proto | 46 --- .../hbase/coprocessor/MasterObserver.java | 142 --- .../org/apache/hadoop/hbase/master/HMaster.java | 122 +++--- .../hbase/master/MasterCoprocessorHost.java | 133 --- .../procedure/AddColumnFamilyProcedure.java | 358 -- .../procedure/DeleteColumnFamilyProcedure.java | 371 --- .../procedure/ModifyColumnFamilyProcedure.java | 323 .../hbase/security/access/AccessController.java | 40 +- .../visibility/VisibilityController.java| 34 -- .../hbase/coprocessor/TestMasterObserver.java | 194 -- .../procedure/TestAddColumnFamilyProcedure.java | 190 -- .../TestDeleteColumnFamilyProcedure.java| 211 --- .../TestModifyColumnFamilyProcedure.java| 183 - .../security/access/TestAccessController.java | 51 --- .../access/TestWithDisabledAuthorization.java | 32 -- 15 files changed, 44 insertions(+), 2386 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/a1bc20ab/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto -- diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto index 626530f..af9caef 100644 --- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto @@ -148,52 +148,6 @@ message DeleteNamespaceStateData { optional NamespaceDescriptor namespace_descriptor = 2; } -enum AddColumnFamilyState { - ADD_COLUMN_FAMILY_PREPARE = 1; - ADD_COLUMN_FAMILY_PRE_OPERATION = 2; - ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3; - ADD_COLUMN_FAMILY_POST_OPERATION = 4; - ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5; -} - -message AddColumnFamilyStateData { - required UserInformation user_info = 1; - required TableName table_name = 2; - required ColumnFamilySchema columnfamily_schema = 3; - optional TableSchema unmodified_table_schema = 4; -} - -enum ModifyColumnFamilyState { - MODIFY_COLUMN_FAMILY_PREPARE = 1; - MODIFY_COLUMN_FAMILY_PRE_OPERATION = 2; - MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3; - MODIFY_COLUMN_FAMILY_POST_OPERATION = 4; - MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5; -} - -message ModifyColumnFamilyStateData { - required UserInformation user_info = 1; - required TableName table_name = 2; - required ColumnFamilySchema columnfamily_schema = 3; - optional TableSchema unmodified_table_schema = 4; -} - -enum DeleteColumnFamilyState { - DELETE_COLUMN_FAMILY_PREPARE = 1; - DELETE_COLUMN_FAMILY_PRE_OPERATION = 2; - DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3; - DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT = 4; - DELETE_COLUMN_FAMILY_POST_OPERATION = 5; - DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 6; -} - -message DeleteColumnFamilyStateData { - required UserInformation user_info = 1; - required TableName table_name = 2; - required bytes columnfamily_name = 3; - optional TableSchema unmodified_table_schema = 4; -} - enum EnableTableState { ENABLE_TABLE_PREPARE = 1; ENABLE_TABLE_PRE_OPERATION = 2; http://git-wip-us.apache.org/repos/asf/hbase/blob/a1bc20ab/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java index 29f0f9f..397ec8a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java @@ -271,148 +271,6 @@ public interface MasterObserver { final TableDescriptor htd) throws IOException {} /** - * Called prior to adding a new column family to the table. Called as part of - * add column RPC call. - * - * @param ctx the environment to interact with the framework and master - * @param tableName the name of the table - * @param columnFamily the ColumnFamilyDescriptor - */ - default void preAddColumnFamily(final
[09/50] [abbrv] hbase git commit: HBASE-19001 Remove the hooks in RegionObserver which are designed to construct a StoreScanner which is marked as IA.Private
http://git-wip-us.apache.org/repos/asf/hbase/blob/e804dd0b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java index ab9bfc59..c67d7bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java @@ -17,31 +17,25 @@ */ package org.apache.hadoop.hbase.util; // this is deliberately not in the o.a.h.h.regionserver package + // in order to make sure all required classes/method are available import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.Collection; -import java.util.HashMap; import java.util.List; -import java.util.Map; -import java.util.NavigableSet; import java.util.Optional; -import java.util.OptionalInt; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.function.Predicate; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -53,11 +47,12 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.DelegatingInternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.KeyValueScanner; -import org.apache.hadoop.hbase.regionserver.ScanInfo; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreScanner; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; @@ -73,7 +68,7 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -@Category({MiscTests.class, MediumTests.class}) +@Category({ MiscTests.class, MediumTests.class }) @RunWith(Parameterized.class) public class TestCoprocessorScanPolicy { protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -84,8 +79,7 @@ public class TestCoprocessorScanPolicy { @BeforeClass public static void setUpBeforeClass() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); -conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, -ScanObserver.class.getName()); +conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, ScanObserver.class.getName()); TEST_UTIL.startMiniCluster(); } @@ -106,49 +100,58 @@ public class TestCoprocessorScanPolicy { @Test public void testBaseCases() throws Exception { -TableName tableName = -TableName.valueOf("baseCases"); +TableName tableName = TableName.valueOf("baseCases"); if (TEST_UTIL.getAdmin().tableExists(tableName)) { TEST_UTIL.deleteTable(tableName); } -Table t = TEST_UTIL.createTable(tableName, F, 1); -// set the version override to 2 -Put p = new Put(R); -p.setAttribute("versions", new byte[]{}); -p.addColumn(F, tableName.getName(), Bytes.toBytes(2)); -t.put(p); - +Table t = TEST_UTIL.createTable(tableName, F, 10); +// insert 3 versions long now = EnvironmentEdgeManager.currentTime(); - -// insert 2 versions -p = new Put(R); +Put p = new Put(R); p.addColumn(F, Q, now, Q); t.put(p); p = new Put(R); p.addColumn(F, Q, now + 1, Q); t.put(p); +p = new Put(R); +p.addColumn(F, Q, now + 2, Q); +t.put(p); + Get g = new Get(R); -g.setMaxVersions(10); +g.readVersions(10); Result r = t.get(g); +assertEquals(3, r.size()); + +TEST_UTIL.flush(tableName); +
[20/50] [abbrv] hbase git commit: HBASE-16338 Remove Jackson1 deps
HBASE-16338 Remove Jackson1 deps * Change imports from org.codehaus to com.fasterxml * Exclude transitive jackson1 from hadoop and others * Minor test cleanup to add assert messages, fix some parameter order * Add anti-pattern check for using jackson 1 imports * Add explicit non-null serialization directive to ScannerModel Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/5facaded Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5facaded Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5facaded Branch: refs/heads/HBASE-18410 Commit: 5facaded902a13556952b1f9d26b768cb86e6599 Parents: a43a00e Author: Mike DrobAuthored: Mon Oct 2 16:31:48 2017 -0500 Committer: Mike Drob Committed: Fri Oct 20 09:20:12 2017 -0500 -- dev-support/hbase-personality.sh| 6 ++ hbase-client/pom.xml| 8 +- .../apache/hadoop/hbase/util/JsonMapper.java| 2 +- .../hadoop/hbase/client/TestOperation.java | 2 +- hbase-it/pom.xml| 4 + .../hadoop/hbase/RESTApiClusterManager.java | 18 ++-- hbase-mapreduce/pom.xml | 12 +-- .../hadoop/hbase/PerformanceEvaluation.java | 10 +- .../hadoop/hbase/TestPerformanceEvaluation.java | 6 +- .../src/main/resources/supplemental-models.xml | 13 --- hbase-rest/pom.xml | 21 ++-- .../hbase/rest/ProtobufStreamingOutput.java | 105 ++ .../hbase/rest/ProtobufStreamingUtil.java | 106 --- .../apache/hadoop/hbase/rest/RESTServer.java| 4 +- .../hadoop/hbase/rest/TableScanResource.java| 26 ++--- .../hadoop/hbase/rest/model/CellModel.java | 2 +- .../hbase/rest/model/ColumnSchemaModel.java | 5 +- .../hbase/rest/model/NamespacesModel.java | 3 +- .../hadoop/hbase/rest/model/RowModel.java | 2 +- .../hadoop/hbase/rest/model/ScannerModel.java | 6 +- .../rest/model/StorageClusterStatusModel.java | 6 ++ .../rest/model/StorageClusterVersionModel.java | 3 - .../hbase/rest/model/TableSchemaModel.java | 7 +- .../hbase/rest/HBaseRESTTestingUtility.java | 5 +- .../hadoop/hbase/rest/RowResourceBase.java | 4 +- .../apache/hadoop/hbase/rest/TestDeleteRow.java | 2 +- .../hadoop/hbase/rest/TestMultiRowResource.java | 9 +- .../rest/TestNamespacesInstanceResource.java| 9 +- .../hadoop/hbase/rest/TestSchemaResource.java | 52 ++--- .../apache/hadoop/hbase/rest/TestTableScan.java | 60 +++ .../hadoop/hbase/rest/TestVersionResource.java | 21 ++-- .../hbase/rest/model/TestColumnSchemaModel.java | 16 +-- .../hadoop/hbase/rest/model/TestModelBase.java | 6 +- .../hbase/rest/model/TestTableSchemaModel.java | 3 + hbase-server/pom.xml| 16 +-- .../hadoop/hbase/io/hfile/AgeSnapshot.java | 2 +- .../hadoop/hbase/io/hfile/BlockCacheUtil.java | 17 ++- .../hadoop/hbase/io/hfile/LruBlockCache.java| 5 +- .../hbase/io/hfile/bucket/BucketAllocator.java | 2 +- .../org/apache/hadoop/hbase/ipc/RpcServer.java | 2 +- .../hbase/monitoring/MonitoredTaskImpl.java | 2 +- .../org/apache/hadoop/hbase/util/JSONBean.java | 6 +- .../hadoop/hbase/util/JSONMetricUtil.java | 10 +- .../hadoop/hbase/wal/WALPrettyPrinter.java | 2 +- .../hbase-webapps/master/processMaster.jsp | 2 +- .../hbase-webapps/master/processRS.jsp | 2 +- .../hbase-webapps/regionserver/processRS.jsp| 2 +- .../hbase/io/hfile/TestBlockCacheReporting.java | 4 +- .../hadoop/hbase/util/TestJSONMetricUtil.java | 33 +++--- hbase-shaded/hbase-shaded-mapreduce/pom.xml | 4 - hbase-shaded/pom.xml| 4 + hbase-shell/src/main/ruby/hbase/taskmonitor.rb | 2 +- hbase-spark/pom.xml | 20 pom.xml | 97 - 54 files changed, 417 insertions(+), 381 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/5facaded/dev-support/hbase-personality.sh -- diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh index 9b23e11..27c2169 100755 --- a/dev-support/hbase-personality.sh +++ b/dev-support/hbase-personality.sh @@ -428,6 +428,12 @@ function hbaseanti_patchfile ((result=result+1)) fi + warnings=$(${GREP} 'import org.codehaus.jackson' "${patchfile}") + if [[ ${warnings} -gt 0 ]]; then +add_vote_table -1 hbaseanti "" "The patch appears use Jackson 1 classes/annotations: ${warnings}." +((result=result+1)) + fi + if [[ ${result} -gt 0 ]]; then return 1 fi
[10/50] [abbrv] hbase git commit: HBASE-19001 Remove the hooks in RegionObserver which are designed to construct a StoreScanner which is marked as IA.Private
HBASE-19001 Remove the hooks in RegionObserver which are designed to construct a StoreScanner which is marked as IA.Private Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/e804dd0b Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e804dd0b Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e804dd0b Branch: refs/heads/HBASE-18410 Commit: e804dd0b600f898f7519dee7134b68ad04c20a9a Parents: 5368fd5 Author: zhangduo <zhang...@apache.org> Authored: Tue Oct 17 21:27:05 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Wed Oct 18 11:06:39 2017 +0800 -- .../hbase/coprocessor/RegionObserver.java | 77 - .../hadoop/hbase/regionserver/HMobStore.java| 24 +- .../hadoop/hbase/regionserver/HRegion.java | 4 +- .../hadoop/hbase/regionserver/HStore.java | 18 +- .../hadoop/hbase/regionserver/Region.java | 3 - .../regionserver/RegionCoprocessorHost.java | 64 +--- .../regionserver/ReversedStoreScanner.java | 6 +- .../hadoop/hbase/regionserver/StoreFlusher.java | 12 +- .../regionserver/compactions/Compactor.java | 44 +-- ...estAvoidCellReferencesIntoShippedBlocks.java | 197 ++--- .../hadoop/hbase/client/TestFromClientSide.java | 156 -- .../client/TestFromClientSideScanExcpetion.java | 238 +++ ...mClientSideScanExcpetionWithCoprocessor.java | 43 +++ .../hbase/coprocessor/SimpleRegionObserver.java | 36 --- .../TestRegionObserverScannerOpenHook.java | 31 +- .../regionserver/DelegatingInternalScanner.java | 45 +++ .../regionserver/NoOpScanPolicyObserver.java| 60 +--- .../hbase/util/TestCoprocessorScanPolicy.java | 290 +++ 18 files changed, 647 insertions(+), 701 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/e804dd0b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index a1e4f0e..d03a9be 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -124,27 +124,6 @@ public interface RegionObserver { default void postLogReplay(ObserverContext c) {} /** - * Called before a memstore is flushed to disk and prior to creating the scanner to read from - * the memstore. To override or modify how a memstore is flushed, - * implementing classes can return a new scanner to provide the KeyValues to be - * stored into the new {@code StoreFile} or null to perform the default processing. - * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no - * effect in this hook. - * @param c the environment provided by the region server - * @param store the store being flushed - * @param scanners the scanners for the memstore that is flushed - * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain - * @param readPoint the readpoint to create scanner - * @return the scanner to use during the flush. {@code null} if the default implementation - * is to be used. - */ - default InternalScanner preFlushScannerOpen(ObserverContext c, - Store store, List scanners, InternalScanner s, long readPoint) - throws IOException { -return s; - } - - /** * Called before the memstore is flushed to disk. * @param c the environment provided by the region server */ @@ -236,33 +215,6 @@ public interface RegionObserver { } /** - * Called prior to writing the {@link StoreFile}s selected for compaction into a new - * {@code StoreFile} and prior to creating the scanner used to read the input files. To override - * or modify the compaction process, implementing classes can return a new scanner to provide the - * KeyValues to be stored into the new {@code StoreFile} or null to perform the default - * processing. Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no - * effect in this hook. - * @param c the environment provided by the region server - * @param store the store being compacted - * @param scanners the list of store file scanners to be read from - * @param scanType the {@link ScanType} indicating whether this is a major or minor compaction - * @param earliestPutTs timestamp of the earliest put that was found in any of the involved store - * files - * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain - * @p
[24/50] [abbrv] hbase git commit: HBASE-19045 Deprecate RegionObserver#postInstantiateDeleteTracker.
HBASE-19045 Deprecate RegionObserver#postInstantiateDeleteTracker. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/64d164b8 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/64d164b8 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/64d164b8 Branch: refs/heads/HBASE-18410 Commit: 64d164b86d32f6d6e987722bf223a809743f9f47 Parents: d798541 Author: anoopsamjohnAuthored: Fri Oct 20 23:57:40 2017 +0530 Committer: anoopsamjohn Committed: Fri Oct 20 23:57:40 2017 +0530 -- .../org/apache/hadoop/hbase/coprocessor/RegionObserver.java | 5 - 1 file changed, 4 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/64d164b8/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index d03a9be..076503f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -1016,11 +1016,14 @@ public interface RegionObserver { * Called after the ScanQueryMatcher creates ScanDeleteTracker. Implementing * this hook would help in creating customised DeleteTracker and returning * the newly created DeleteTracker - * + * + * Warn: This is used by internal coprocessors. Should not be implemented by user coprocessors * @param ctx the environment provided by the region server * @param delTracker the deleteTracker that is created by the QueryMatcher * @return the Delete Tracker + * @deprecated Since 2.0 with out any replacement and will be removed in 3.0 */ + @Deprecated default DeleteTracker postInstantiateDeleteTracker( ObserverContext ctx, DeleteTracker delTracker) throws IOException {
[03/50] [abbrv] hbase git commit: HBSE-18945 Make a IA.LimitedPrivate interface for CellComparator (Ram)
http://git-wip-us.apache.org/repos/asf/hbase/blob/70f4c5da/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java index 039f499..7068fe1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java @@ -34,6 +34,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -104,7 +105,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { private int avgValueLen = -1; /** Key comparator */ - private CellComparator comparator = CellComparator.COMPARATOR; + private CellComparator comparator = CellComparatorImpl.COMPARATOR; /** Size of this file. */ private final long fileSize; @@ -727,7 +728,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { offsetFromPos += Bytes.SIZEOF_LONG; blockBuffer.asSubByteBuffer(blockBuffer.position() + offsetFromPos, klen, pair); bufBackedKeyOnlyKv.setKey(pair.getFirst(), pair.getSecond(), klen); -int comp = reader.getComparator().compareKeyIgnoresMvcc(key, bufBackedKeyOnlyKv); +int comp = CellUtil.compareKeyIgnoresMvcc(reader.getComparator(), key, bufBackedKeyOnlyKv); offsetFromPos += klen + vlen; if (this.reader.getFileContext().isIncludesTags()) { // Read short as unsigned, high byte first @@ -810,8 +811,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { } else { // The comparison with no_next_index_key has to be checked if (this.nextIndexedKey != null && - (this.nextIndexedKey == KeyValueScanner.NO_NEXT_INDEXED_KEY || reader - .getComparator().compareKeyIgnoresMvcc(key, nextIndexedKey) < 0)) { + (this.nextIndexedKey == KeyValueScanner.NO_NEXT_INDEXED_KEY || CellUtil + .compareKeyIgnoresMvcc(reader.getComparator(), key, nextIndexedKey) < 0)) { // The reader shall continue to scan the current data block instead // of querying the // block index as long as it knows the target key is strictly @@ -864,8 +865,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { return false; } Cell firstKey = getFirstKeyCellInBlock(seekToBlock); - if (reader.getComparator() - .compareKeyIgnoresMvcc(firstKey, key) >= 0) { + if (CellUtil.compareKeyIgnoresMvcc(reader.getComparator(), firstKey, key) >= 0) { long previousBlockOffset = seekToBlock.getPrevBlockOffset(); // The key we are interested in if (previousBlockOffset == -1) { @@ -1229,7 +1229,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { public int compareKey(CellComparator comparator, Cell key) { blockBuffer.asSubByteBuffer(blockBuffer.position() + KEY_VALUE_LEN_SIZE, currKeyLen, pair); this.bufBackedKeyOnlyKv.setKey(pair.getFirst(), pair.getSecond(), currKeyLen); - return comparator.compareKeyIgnoresMvcc(key, this.bufBackedKeyOnlyKv); + return CellUtil.compareKeyIgnoresMvcc(comparator, key, this.bufBackedKeyOnlyKv); } @Override http://git-wip-us.apache.org/repos/asf/hbase/blob/70f4c5da/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java index 5b25bed..33cfa1d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java @@ -36,10 +36,11 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hbase.ByteBufferCell; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.hadoop.hbase.CellComparator.MetaCellComparator; +import org.apache.hadoop.hbase.CellComparatorImpl.MetaCellComparator; import org.apache.yetus.audience.InterfaceAudience; import
[37/50] [abbrv] hbase git commit: HBASE-18893 remove add/delete/modify column
http://git-wip-us.apache.org/repos/asf/hbase/blob/a1bc20ab/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java deleted file mode 100644 index 01de512..000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java +++ /dev/null @@ -1,190 +0,0 @@ -/** - * 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 - * - * 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 org.apache.hadoop.hbase.master.procedure; - -import static org.junit.Assert.assertTrue; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.CategoryBasedTimeout; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.InvalidFamilyOperationException; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.procedure2.Procedure; -import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; -import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; -import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; -import org.junit.rules.TestRule; - -@Category({MasterTests.class, MediumTests.class}) -public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase { - private static final Log LOG = LogFactory.getLog(TestAddColumnFamilyProcedure.class); - @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). - withLookingForStuckThread(true).build(); - - @Rule public TestName name = new TestName(); - - @Test(timeout = 6) - public void testAddColumnFamily() throws Exception { -final TableName tableName = TableName.valueOf(name.getMethodName()); -final String cf1 = "cf1"; -final String cf2 = "cf2"; -final HColumnDescriptor columnDescriptor1 = new HColumnDescriptor(cf1); -final HColumnDescriptor columnDescriptor2 = new HColumnDescriptor(cf2); -final ProcedureExecutor procExec = getMasterProcedureExecutor(); - -MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f3"); - -// Test 1: Add a column family online -long procId1 = procExec.submitProcedure( - new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor1)); -// Wait the completion -ProcedureTestingUtility.waitProcedure(procExec, procId1); -ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); - -MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf1); - -// Test 2: Add a column family offline -UTIL.getAdmin().disableTable(tableName); -long procId2 = procExec.submitProcedure( - new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor2)); -// Wait the completion -ProcedureTestingUtility.waitProcedure(procExec, procId2); -ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); -MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf2); - } - - @Test(timeout=6) - public void testAddSameColumnFamilyTwice() throws Exception { -final TableName tableName = TableName.valueOf(name.getMethodName()); -final String cf2 = "cf2"; -final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf2); - -final ProcedureExecutor procExec = getMasterProcedureExecutor(); - -MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1"); - -// add the column family -long procId1 = procExec.submitProcedure( - new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor)); -// Wait the completion -ProcedureTestingUtility.waitProcedure(procExec, procId1); -ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
[11/50] [abbrv] hbase git commit: HBASE-19020 HBase Rest test for xml parsing external entities should not rely on implementation of java XML APIs.
HBASE-19020 HBase Rest test for xml parsing external entities should not rely on implementation of java XML APIs. Signed-off-by: Chia-Ping TsaiProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/e320df5a Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e320df5a Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e320df5a Branch: refs/heads/HBASE-18410 Commit: e320df5a0c267258c03909da8d0eee4c0e287532 Parents: e804dd0 Author: Sean Busbey Authored: Mon Oct 16 16:11:39 2017 -0500 Committer: Sean Busbey Committed: Wed Oct 18 09:39:55 2017 -0500 -- .../apache/hadoop/hbase/rest/client/TestXmlParsing.java | 10 +- 1 file changed, 9 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/e320df5a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestXmlParsing.java -- diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestXmlParsing.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestXmlParsing.java index 5e259f2..586e33c 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestXmlParsing.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestXmlParsing.java @@ -23,7 +23,10 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import java.io.IOException; +import javax.xml.bind.UnmarshalException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.rest.Constants; import org.apache.hadoop.hbase.rest.model.StorageClusterVersionModel; @@ -37,6 +40,7 @@ import org.junit.experimental.categories.Category; */ @Category(SmallTests.class) public class TestXmlParsing { + private static final Log LOG = LogFactory.getLog(TestXmlParsing.class); @Test public void testParsingClusterVersion() throws Exception { @@ -68,8 +72,12 @@ public class TestXmlParsing { admin.getClusterVersion(); fail("Expected getClusterVersion() to throw an exception"); } catch (IOException e) { + assertEquals("Cause of exception ought to be a failure to parse the stream due to our " + + "invalid external entity. Make sure this isn't just a false positive due to " + + "implementation. see HBASE-19020.", UnmarshalException.class, e.getCause().getClass()); final String exceptionText = StringUtils.stringifyException(e); - final String expectedText = "The entity \"xee\" was referenced, but not declared."; + final String expectedText = "\"xee\""; + LOG.debug("exception text: '" + exceptionText + "'", e); assertTrue("Exception does not contain expected text", exceptionText.contains(expectedText)); } }
[04/50] [abbrv] hbase git commit: HBSE-18945 Make a IA.LimitedPrivate interface for CellComparator (Ram)
http://git-wip-us.apache.org/repos/asf/hbase/blob/70f4c5da/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java -- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java new file mode 100644 index 000..264984a --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java @@ -0,0 +1,381 @@ +/* + * 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 + * + * 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 org.apache.hadoop.hbase; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.KeyValue.Type; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; +import org.apache.hadoop.hbase.util.ByteBufferUtils; +import org.apache.hadoop.hbase.util.Bytes; + +import org.apache.hadoop.hbase.shaded.com.google.common.primitives.Longs; + +/** + * Compare two HBase cells. Do not use this method comparing -ROOT- or + * hbase:meta cells. Cells from these tables need a specialized comparator, one that + * takes account of the special formatting of the row where we have commas to delimit table from + * regionname, from row. See KeyValue for how it has a special comparator to do hbase:meta cells + * and yet another for -ROOT-. + * While using this comparator for {{@link #compareRows(Cell, Cell)} et al, the hbase:meta cells + * format should be taken into consideration, for which the instance of this comparator + * should be used. In all other cases the static APIs in this comparator would be enough + */ +@edu.umd.cs.findbugs.annotations.SuppressWarnings( +value="UNKNOWN", +justification="Findbugs doesn't like the way we are negating the result of a compare in below") +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class CellComparatorImpl implements CellComparator { + static final Log LOG = LogFactory.getLog(CellComparatorImpl.class); + /** + * Comparator for plain key/values; i.e. non-catalog table key/values. Works on Key portion + * of KeyValue only. + */ + public static final CellComparatorImpl COMPARATOR = new CellComparatorImpl(); + /** + * A {@link CellComparatorImpl} for hbase:meta catalog table + * {@link KeyValue}s. + */ + public static final CellComparatorImpl META_COMPARATOR = new MetaCellComparator(); + + @Override + public int compare(Cell a, Cell b) { +return compare(a, b, false); + } + + /** + * Compare cells. + * @param a + * @param b + * @param ignoreSequenceid True if we are to compare the key portion only and ignore + * the sequenceid. Set to false to compare key and consider sequenceid. + * @return 0 if equal, -1 if a b, and +1 if a b. + */ + public final int compare(final Cell a, final Cell b, boolean ignoreSequenceid) { +// row +int c = compareRows(a, b); +if (c != 0) return c; + +c = compareWithoutRow(a, b); +if(c != 0) return c; + +if (!ignoreSequenceid) { + // Negate following comparisons so later edits show up first + // mvccVersion: later sorts first + return Longs.compare(b.getSequenceId(), a.getSequenceId()); +} else { + return c; +} + } + + /** + * Compares the family and qualifier part of the cell + * @param left the left cell + * @param right the right cell + * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise + */ + public final int compareColumns(final Cell left, final Cell right) { +int diff = compareFamilies(left, right); +if (diff != 0) { + return diff; +} +return compareQualifiers(left, right); + } + + /** + * Compare the families of left and right cell + * @param left + * @param right + * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise + */ + @Override + public final int compareFamilies(Cell left, Cell right) { +if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) { + return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(), +
[23/50] [abbrv] hbase git commit: HBASE-19043 Purge TableWrapper and CoprocessorHConnnection Also purge Coprocessor#getTable... Let Coprocessors manage their Table Connections in hbase2.0.0.
HBASE-19043 Purge TableWrapper and CoprocessorHConnnection Also purge Coprocessor#getTable... Let Coprocessors manage their Table Connections in hbase2.0.0. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/d7985412 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d7985412 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d7985412 Branch: refs/heads/HBASE-18410 Commit: d7985412610b612c09cf377ab87963e897c72afa Parents: d59ed23 Author: Michael StackAuthored: Wed Oct 18 21:45:39 2017 -0700 Committer: Michael Stack Committed: Fri Oct 20 11:06:10 2017 -0700 -- .../hadoop/hbase/CoprocessorEnvironment.java| 15 - .../hbase/client/CoprocessorHConnection.java| 105 -- .../hadoop/hbase/client/HTableWrapper.java | 346 -- .../hbase/coprocessor/BaseEnvironment.java | 44 --- .../hbase/security/access/AccessController.java | 82 +++-- .../hbase/coprocessor/TestCoprocessorHost.java | 13 - .../hbase/coprocessor/TestHTableWrapper.java| 362 --- .../coprocessor/TestOpenTableInCoprocessor.java | 28 +- .../security/token/TestTokenAuthentication.java | 15 - 9 files changed, 70 insertions(+), 940 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/d7985412/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java index aabf3b5..4022b4b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java @@ -20,11 +20,9 @@ package org.apache.hadoop.hbase; import java.io.IOException; -import java.util.concurrent.ExecutorService; import org.apache.hadoop.conf.Configuration; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.client.Table; /** * Coprocessor environment state. @@ -51,19 +49,6 @@ public interface CoprocessorEnvironment { Configuration getConfiguration(); /** - * @return an interface for accessing the given table - * @throws IOException - */ - Table getTable(TableName tableName) throws IOException; - - /** - * @return an interface for accessing the given table using the passed executor to run batch - * operations - * @throws IOException - */ - Table getTable(TableName tableName, ExecutorService service) throws IOException; - - /** * @return the classloader for the loaded coprocessor instance */ ClassLoader getClassLoader(); http://git-wip-us.apache.org/repos/asf/hbase/blob/d7985412/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java deleted file mode 100644 index c87c56e..000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java +++ /dev/null @@ -1,105 +0,0 @@ -/** - * 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 - * - * 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 org.apache.hadoop.hbase.client; - -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CoprocessorEnvironment; -import org.apache.hadoop.hbase.ServerName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.yetus.audience.InterfaceStability; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.regionserver.CoprocessorRegionServerServices; -import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.security.UserProvider; - -/** - * Connection to an
[07/50] [abbrv] hbase git commit: HBASE-18350 RSGroups are broken under AMv2
HBASE-18350 RSGroups are broken under AMv2 - Table moving to RSG was buggy, because it left the table unassigned. Now it is fixed we immediately assign to an appropriate RS (MoveRegionProcedure). - Table was locked while moving, but unassign operation hung, because locked table queues are not scheduled while locked. Fixed. - ProcedureSyncWait was buggy, because it searched the procId in executor, but executor does not store the return values of internal operations (they are stored, but immediately removed by the cleaner). - list_rsgroups in the shell show also the assigned tables and servers. Signed-off-by: Michael StackProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/41cc9a12 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/41cc9a12 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/41cc9a12 Branch: refs/heads/HBASE-18410 Commit: 41cc9a125f0074bdb9633d873f5bc2219ca1fb73 Parents: e1941aa Author: Balazs Meszaros Authored: Tue Oct 10 09:24:51 2017 +0200 Committer: Michael Stack Committed: Tue Oct 17 13:58:36 2017 -0700 -- .../hbase/rsgroup/RSGroupAdminServer.java | 156 +-- .../hbase/rsgroup/RSGroupBasedLoadBalancer.java | 16 +- .../hadoop/hbase/rsgroup/TestRSGroups.java | 16 +- .../hbase/rsgroup/TestRSGroupsOfflineMode.java | 3 +- .../master/assignment/AssignmentManager.java| 25 +-- .../master/procedure/ProcedureSyncWait.java | 50 +++--- .../resources/hbase-webapps/master/table.jsp| 6 +- .../src/main/ruby/hbase/rsgroup_admin.rb| 29 +--- .../src/main/ruby/shell/commands/get_rsgroup.rb | 14 +- .../main/ruby/shell/commands/list_rsgroups.rb | 37 - .../ruby/shell/commands/move_servers_rsgroup.rb | 3 +- 11 files changed, 183 insertions(+), 172 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/41cc9a12/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java -- diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java index b13dafd..3c82d76 100644 --- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java +++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java @@ -44,13 +44,10 @@ import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.ServerManager; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode; -import org.apache.hadoop.hbase.master.locking.LockManager; import org.apache.hadoop.hbase.net.Address; -import org.apache.hadoop.hbase.procedure2.LockType; -import org.apache.yetus.audience.InterfaceAudience; - import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps; +import org.apache.yetus.audience.InterfaceAudience; /** * Service to support Region Server Grouping (HBase-6721). @@ -88,10 +85,10 @@ public class RSGroupAdminServer implements RSGroupAdmin { for(ServerName server: master.getServerManager().getOnlineServers().keySet()) { onlineServers.add(server.getAddress()); } -for (Address el: servers) { - if (!onlineServers.contains(el)) { +for (Address address: servers) { + if (!onlineServers.contains(address)) { throw new ConstraintException( -"Server " + el + " is not an online server in 'default' RSGroup."); +"Server " + address + " is not an online server in 'default' RSGroup."); } } } @@ -192,18 +189,20 @@ public class RSGroupAdminServer implements RSGroupAdmin { } /** + * Moves every region from servers which are currently located on these servers, + * but should not be located there. * @param servers the servers that will move to new group + * @param tables these tables will be kept on the servers, others will be moved * @param targetGroupName the target group name - * @param tables The regions of tables assigned to these servers will not unassign * @throws IOException */ - private void unassignRegionFromServers(Set servers, String targetGroupName, - Set tables) throws IOException { -boolean foundRegionsToUnassign; + private void moveRegionsFromServers(Set servers, Set tables, + String targetGroupName) throws IOException { +boolean foundRegionsToMove; RSGroupInfo targetGrp = getRSGroupInfo(targetGroupName); Set allSevers = new
[28/50] [abbrv] hbase git commit: HBASE-19058. The wget isn't installed in building docker image
HBASE-19058. The wget isn't installed in building docker image Signed-off-by: Sean BusbeyProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/cb5c4776 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/cb5c4776 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/cb5c4776 Branch: refs/heads/HBASE-18410 Commit: cb5c4776deee270ea21afc52d4ba70d9474d8a8a Parents: b10ad9e Author: Chia-Ping Tsai Authored: Fri Oct 20 20:32:27 2017 -0500 Committer: Sean Busbey Committed: Fri Oct 20 20:35:33 2017 -0500 -- dev-support/docker/Dockerfile | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/cb5c4776/dev-support/docker/Dockerfile -- diff --git a/dev-support/docker/Dockerfile b/dev-support/docker/Dockerfile index c23c70d..da5f32e 100644 --- a/dev-support/docker/Dockerfile +++ b/dev-support/docker/Dockerfile @@ -62,7 +62,8 @@ RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ python-pip \ rsync \ snappy \ -zlib1g-dev +zlib1g-dev \ +wget ### # OpenJDK 8
[33/50] [abbrv] hbase git commit: Add Zheng Hu to pom.xml
Add Zheng Hu to pom.xml Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/24931044 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/24931044 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/24931044 Branch: refs/heads/HBASE-18410 Commit: 24931044d6ec1a6eda4513102a99c453fe128bd9 Parents: 2ee8690 Author: huzhengAuthored: Mon Oct 23 13:41:45 2017 +0800 Committer: huzheng Committed: Mon Oct 23 13:41:45 2017 +0800 -- pom.xml | 6 ++ 1 file changed, 6 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/24931044/pom.xml -- diff --git a/pom.xml b/pom.xml index d7cbca2..0a55b64 100755 --- a/pom.xml +++ b/pom.xml @@ -458,6 +458,12 @@ 0 + openinx + Zheng Hu + open...@apache.org + +8 + + rajeshbabu Rajeshbabu Chintaguntla rajeshb...@apache.org
[50/50] [abbrv] hbase git commit: HBASE-18368 FilterList with multiple FamilyFilters concatenated by OR does not work
HBASE-18368 FilterList with multiple FamilyFilters concatenated by OR does not work Signed-off-by: Guanghao Zhang <zg...@apache.org> Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b5896b7a Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b5896b7a Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b5896b7a Branch: refs/heads/HBASE-18410 Commit: b5896b7a45b5e2324dc5d3e5fbd775c3e784caf5 Parents: a17094f Author: huzheng <open...@gmail.com> Authored: Tue Oct 17 19:25:23 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Oct 24 11:39:31 2017 +0800 -- .../org/apache/hadoop/hbase/filter/Filter.java | 10 +--- .../hadoop/hbase/filter/FilterListWithOR.java | 10 ++-- .../hadoop/hbase/filter/TestFilterList.java | 26 .../hbase/filter/TestFilterListOnMini.java | 7 +++--- 4 files changed, 44 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/b5896b7a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java index 70c68b6..a92ea0b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java @@ -172,8 +172,12 @@ public abstract class Filter { */ NEXT_COL, /** - * Done with columns, skip to next row. Note that filterRow() will - * still be called. + * Seek to next row in current family. It may still pass a cell whose family is different but + * row is the same as previous cell to {@link #filterKeyValue(Cell)} , even if we get a NEXT_ROW + * returned for previous cell. For more details see HBASE-18368. + * Once reset() method was invoked, then we switch to the next row for all family, and you can + * catch the event by invoking CellUtils.matchingRows(previousCell, currentCell). + * Note that filterRow() will still be called. */ NEXT_ROW, /** @@ -181,7 +185,7 @@ public abstract class Filter { */ SEEK_NEXT_USING_HINT, /** - * Include KeyValue and done with row, seek to next. + * Include KeyValue and done with row, seek to next. See NEXT_ROW. */ INCLUDE_AND_SEEK_NEXT_ROW, } http://git-wip-us.apache.org/repos/asf/hbase/blob/b5896b7a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java index bac9023..31e2a55 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java @@ -74,7 +74,12 @@ public class FilterListWithOR extends FilterListBase { * as the previous cell even if filter-A has NEXT_COL returned for the previous cell. So we should * save the previous cell and the return code list when checking previous cell for every filter in * filter list, and verify if currentCell fit the previous return code, if fit then pass the - * currentCell to the corresponding filter. (HBASE-17678) + * currentCell to the corresponding filter. (HBASE-17678) + * Note that: In StoreScanner level, NEXT_ROW will skip to the next row in current family, and in + * RegionScanner level, NEXT_ROW will skip to the next row in current family and switch to the + * next family for RegionScanner, INCLUDE_AND_NEXT_ROW is the same. so we should pass current cell + * to the filter, if row mismatch or row match but column family mismatch. (HBASE-18368) + * @see org.apache.hadoop.hbase.filter.Filter.ReturnCode */ private boolean shouldPassCurrentCellToFilter(Cell prevCell, Cell currentCell, int filterIdx) throws IOException { @@ -94,7 +99,8 @@ public class FilterListWithOR extends FilterListBase { return !CellUtil.matchingRowColumn(prevCell, currentCell); case NEXT_ROW: case INCLUDE_AND_SEEK_NEXT_ROW: - return !CellUtil.matchingRows(prevCell, currentCell); + return !CellUtil.matchingRows(prevCell, currentCell) + || !CellUtil.matchingFamily(prevCell, currentCell); default: throw new IllegalStateException("Received code is not valid."); } http://git-wip-us.apache.org/repos/asf/hbase/blob/b5896b7a/hbase-server/src/test/java/org/apache/hadoop/hbas
[29/50] [abbrv] hbase git commit: HBASE-19010 Reimplement getMasterInfoPort for Admin
HBASE-19010 Reimplement getMasterInfoPort for Admin Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/592d541f Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/592d541f Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/592d541f Branch: refs/heads/HBASE-18410 Commit: 592d541f5d5e5fea5668915e0400f048fa3f65e3 Parents: cb5c477 Author: Guanghao ZhangAuthored: Tue Oct 17 19:12:54 2017 +0800 Committer: Guanghao Zhang Committed: Sat Oct 21 18:19:22 2017 +0800 -- .../org/apache/hadoop/hbase/ClusterStatus.java | 28 +++- .../org/apache/hadoop/hbase/client/Admin.java | 4 ++- .../apache/hadoop/hbase/client/AsyncAdmin.java | 9 +++ .../apache/hadoop/hbase/client/HBaseAdmin.java | 12 - .../hbase/shaded/protobuf/ProtobufUtil.java | 7 + .../src/main/protobuf/ClusterStatus.proto | 2 ++ .../org/apache/hadoop/hbase/master/HMaster.java | 6 + .../apache/hadoop/hbase/TestInfoServers.java| 10 +++ .../hbase/client/TestAsyncClusterAdminApi.java | 19 + .../hbase/client/TestClientClusterStatus.java | 2 ++ 10 files changed, 80 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/592d541f/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java index 0655b18..13a1358 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java @@ -81,6 +81,7 @@ public class ClusterStatus { private String clusterId; private String[] masterCoprocessors; private Boolean balancerOn; + private int masterInfoPort; /** * Use {@link ClusterStatus.Builder} to construct a ClusterStatus instead. @@ -95,7 +96,8 @@ public class ClusterStatus { final Collection backupMasters, final List rit, final String[] masterCoprocessors, - final Boolean balancerOn) { + final Boolean balancerOn, + final int masterInfoPort) { // TODO: make this constructor private this.hbaseVersion = hbaseVersion; this.liveServers = servers; @@ -106,6 +108,7 @@ public class ClusterStatus { this.clusterId = clusterid; this.masterCoprocessors = masterCoprocessors; this.balancerOn = balancerOn; +this.masterInfoPort = masterInfoPort; } /** @@ -202,15 +205,17 @@ public class ClusterStatus { getDeadServerNames().containsAll(other.getDeadServerNames()) && Arrays.equals(getMasterCoprocessors(), other.getMasterCoprocessors()) && Objects.equal(getMaster(), other.getMaster()) && - getBackupMasters().containsAll(other.getBackupMasters()); + getBackupMasters().containsAll(other.getBackupMasters()) && + Objects.equal(getClusterId(), other.getClusterId()) && + getMasterInfoPort() == other.getMasterInfoPort(); } /** * @see java.lang.Object#hashCode() */ public int hashCode() { -return Objects.hashCode(hbaseVersion, liveServers, deadServers, - master, backupMasters); +return Objects.hashCode(hbaseVersion, liveServers, deadServers, master, backupMasters, + clusterId, masterInfoPort); } /** @@ -312,6 +317,10 @@ public class ClusterStatus { return balancerOn; } + public int getMasterInfoPort() { +return masterInfoPort; + } + public String toString() { StringBuilder sb = new StringBuilder(1024); sb.append("Master: " + master); @@ -372,6 +381,7 @@ public class ClusterStatus { private String clusterId = null; private String[] masterCoprocessors = null; private Boolean balancerOn = null; +private int masterInfoPort = -1; private Builder() {} @@ -420,10 +430,15 @@ public class ClusterStatus { return this; } +public Builder setMasterInfoPort(int masterInfoPort) { + this.masterInfoPort = masterInfoPort; + return this; +} + public ClusterStatus build() { return new ClusterStatus(hbaseVersion, clusterId, liveServers, deadServers, master, backupMasters, intransition, masterCoprocessors, - balancerOn); + balancerOn, masterInfoPort); } } @@ -439,6 +454,7 @@ public class ClusterStatus { MASTER, /** status about master */ BACKUP_MASTERS, /** status about backup masters */ MASTER_COPROCESSORS, /** status about master coprocessors */ -REGIONS_IN_TRANSITION; /** status about regions in transition */ +REGIONS_IN_TRANSITION, /** status about regions in transition */ +
[21/50] [abbrv] hbase git commit: HBASE-19051 Add new split algorithm for num string
HBASE-19051 Add new split algorithm for num string Signed-off-by: tedyuSigned-off-by: Mike Drob Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/8c6ddc1a Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/8c6ddc1a Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/8c6ddc1a Branch: refs/heads/HBASE-18410 Commit: 8c6ddc1aa5497a38018fdcf100bd33b385ca2c84 Parents: 5facade Author: xiaowen147 Authored: Fri Oct 20 02:18:17 2017 +0800 Committer: tedyu Committed: Fri Oct 20 09:49:57 2017 -0700 -- .../hadoop/hbase/util/RegionSplitter.java | 80 +++- .../hadoop/hbase/util/TestRegionSplitter.java | 74 -- 2 files changed, 131 insertions(+), 23 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/8c6ddc1a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java index 3ee593a..06bccd1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java @@ -274,6 +274,12 @@ public class RegionSplitter { * bin/hbase org.apache.hadoop.hbase.util.RegionSplitter -c 60 -f test:rs * myTable HexStringSplit * + * create a table named 'myTable' with 50 pre-split regions, + * assuming the keys are decimal-encoded ASCII: + * + * bin/hbase org.apache.hadoop.hbase.util.RegionSplitter -c 50 + * myTable DecimalStringSplit + * * perform a rolling split of 'myTable' (i.e. 60 = 120 regions), # 2 * outstanding splits at a time, assuming keys are uniformly distributed * bytes: @@ -283,9 +289,9 @@ public class RegionSplitter { * * * - * There are two SplitAlgorithms built into RegionSplitter, HexStringSplit - * and UniformSplit. These are different strategies for choosing region - * boundaries. See their source code for details. + * There are three SplitAlgorithms built into RegionSplitter, HexStringSplit, + * DecimalStringSplit, and UniformSplit. These are different strategies for + * choosing region boundaries. See their source code for details. * * @param args * Usage: RegionSplitter TABLE SPLITALGORITHM @@ -353,9 +359,10 @@ public class RegionSplitter { if (2 != cmd.getArgList().size() || !oneOperOnly || cmd.hasOption("h")) { new HelpFormatter().printHelp("RegionSplitter \n"+ "SPLITALGORITHM is a java class name of a class implementing " + - "SplitAlgorithm, or one of the special strings HexStringSplit " + - "or UniformSplit, which are built-in split algorithms. " + + "SplitAlgorithm, or one of the special strings HexStringSplit or " + + "DecimalStringSplit or UniformSplit, which are built-in split algorithms. " + "HexStringSplit treats keys as hexadecimal ASCII, and " + + "DecimalStringSplit treats keys as decimal ASCII, and " + "UniformSplit treats keys as arbitrary bytes.", opt); return; } @@ -660,6 +667,8 @@ public class RegionSplitter { // their simple class name instead of a fully qualified class name. if(splitClassName.equals(HexStringSplit.class.getSimpleName())) { splitClass = HexStringSplit.class; +} else if (splitClassName.equals(DecimalStringSplit.class.getSimpleName())) { + splitClass = DecimalStringSplit.class; } else if (splitClassName.equals(UniformSplit.class.getSimpleName())) { splitClass = UniformSplit.class; } else { @@ -893,15 +902,52 @@ public class RegionSplitter { * Since this split algorithm uses hex strings as keys, it is easy to read * write in the shell but takes up more space and may be non-intuitive. */ - public static class HexStringSplit implements SplitAlgorithm { + public static class HexStringSplit extends NumberStringSplit { final static String DEFAULT_MIN_HEX = ""; final static String DEFAULT_MAX_HEX = ""; +final static int RADIX_HEX = 16; + +public HexStringSplit() { + super(DEFAULT_MIN_HEX, DEFAULT_MAX_HEX, RADIX_HEX); +} -String firstRow = DEFAULT_MIN_HEX; -BigInteger firstRowInt = BigInteger.ZERO; -String lastRow = DEFAULT_MAX_HEX; -BigInteger lastRowInt = new BigInteger(lastRow, 16); -int rowComparisonLength = lastRow.length(); + } + + /** + * The format of a DecimalStringSplit region boundary is the ASCII representation of + * reversed sequential number, or any
[17/50] [abbrv] hbase git commit: HBASE-19042 Oracle Java 8u144 downloader broken in precommit check
HBASE-19042 Oracle Java 8u144 downloader broken in precommit check Signed-off-by: Mike Drob <md...@apache.org> Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/af479c58 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/af479c58 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/af479c58 Branch: refs/heads/HBASE-18410 Commit: af479c580c24a78b34052dc4ad16dacd3dd988cd Parents: 909e5f2 Author: zhangduo <zhang...@apache.org> Authored: Thu Oct 19 14:49:09 2017 +0800 Committer: Mike Drob <md...@apache.org> Committed: Thu Oct 19 15:53:52 2017 -0500 -- dev-support/docker/Dockerfile | 29 +++-- 1 file changed, 11 insertions(+), 18 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/af479c58/dev-support/docker/Dockerfile -- diff --git a/dev-support/docker/Dockerfile b/dev-support/docker/Dockerfile index 62c6030..c23c70d 100644 --- a/dev-support/docker/Dockerfile +++ b/dev-support/docker/Dockerfile @@ -65,18 +65,18 @@ RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ zlib1g-dev ### -# Oracle Java +# OpenJDK 8 ### RUN echo "dot_style = mega" > "/root/.wgetrc" RUN echo "quiet = on" >> "/root/.wgetrc" RUN apt-get -q update && apt-get -q install --no-install-recommends -y software-properties-common -RUN add-apt-repository -y ppa:webupd8team/java - -# Auto-accept the Oracle JDK license -RUN echo oracle-java8-installer shared/accepted-oracle-license-v1-1 select true | sudo /usr/bin/debconf-set-selections -RUN apt-get -q update && apt-get -q install --no-install-recommends -y oracle-java8-installer +RUN add-apt-repository -y ppa:openjdk-r/ppa +RUN apt-get -q update +RUN apt-get -q install --no-install-recommends -y openjdk-8-jdk +RUN update-alternatives --config java +RUN update-alternatives --config javac # Apps that require Java @@ -131,23 +131,16 @@ RUN pip install python-dateutil # Install Ruby 2, based on Yetus 0.4.0 dockerfile ### RUN echo 'gem: --no-rdoc --no-ri' >> /root/.gemrc -RUN apt-get -q install -y ruby2.0 -# -# on trusty, the above installs ruby2.0 and ruby (1.9.3) exes -# but update-alternatives is broken, so we need to do some work -# to make 2.0 actually the default without the system flipping out -# -# See https://bugs.launchpad.net/ubuntu/+source/ruby2.0/+bug/1310292 -# -RUN dpkg-divert --add --rename --divert /usr/bin/ruby.divert /usr/bin/ruby -RUN dpkg-divert --add --rename --divert /usr/bin/gem.divert /usr/bin/gemrc -RUN update-alternatives --install /usr/bin/ruby ruby /usr/bin/ruby2.0 1 -RUN update-alternatives --install /usr/bin/gem gem /usr/bin/gem2.0 1 +RUN apt-add-repository ppa:brightbox/ruby-ng +RUN apt-get -q update +RUN apt-get -q install --no-install-recommends -y ruby2.2 ruby-switch +RUN ruby-switch --set ruby2.2 # Install rubocop ### +RUN gem install rake RUN gem install rubocop
[35/50] [abbrv] hbase git commit: HBASE-18989 Polish the compaction related CP hooks
HBASE-18989 Polish the compaction related CP hooks Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c9fdbec7 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c9fdbec7 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c9fdbec7 Branch: refs/heads/HBASE-18410 Commit: c9fdbec772fe7dea06644d86e2854b98047ac9da Parents: 4add40c Author: zhangduo <zhang...@apache.org> Authored: Mon Oct 23 16:44:54 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Mon Oct 23 16:44:54 2017 +0800 -- .../hbase/coprocessor/RegionObserver.java | 23 +- .../hadoop/hbase/regionserver/CompactSplit.java | 101 +-- .../hadoop/hbase/regionserver/HRegion.java | 34 ++- .../hbase/regionserver/HRegionServer.java | 10 +- .../hadoop/hbase/regionserver/HStore.java | 5 +- .../hbase/regionserver/RSRpcServices.java | 47 +--- .../hadoop/hbase/regionserver/Region.java | 19 +- .../regionserver/RegionServerServices.java | 10 +- .../apache/hadoop/hbase/regionserver/Store.java | 7 - .../compactions/CompactionLifeCycleTracker.java | 19 +- .../compactions/CompactionRequester.java| 46 .../hadoop/hbase/MockRegionServerServices.java | 6 + .../hadoop/hbase/master/MockRegionServer.java | 36 +-- .../hbase/regionserver/TestCompaction.java | 2 +- .../TestCompactionLifeCycleTracker.java | 267 +++ 15 files changed, 487 insertions(+), 145 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/c9fdbec7/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index 94550df..ba96a5b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -158,7 +158,7 @@ public interface RegionObserver { /** * Called prior to selecting the {@link StoreFile StoreFiles} to compact from the list of * available candidates. To alter the files used for compaction, you may mutate the passed in list - * of candidates. + * of candidates. If you remove all the candidates then the compaction will be canceled. * @param c the environment provided by the region server * @param store the store where compaction is being requested * @param candidates the store files currently available for compaction @@ -183,18 +183,12 @@ public interface RegionObserver { /** * Called prior to writing the {@link StoreFile}s selected for compaction into a new - * {@code StoreFile}. To override or modify the compaction process, implementing classes have two - * options: - * - * Wrap the provided {@link InternalScanner} with a custom implementation that is returned - * from this method. The custom scanner can then inspect - * {@link org.apache.hadoop.hbase.KeyValue}s from the wrapped scanner, applying its own - * policy to what gets written. - * Call {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} and provide a - * custom implementation for writing of new {@link StoreFile}s. Note: any implementations - * bypassing core compaction using this approach must write out new store files themselves or the - * existing data will no longer be available after compaction. - * + * {@code StoreFile}. + * + * To override or modify the compaction process, implementing classes can wrap the provided + * {@link InternalScanner} with a custom implementation that is returned from this method. The + * custom scanner can then inspect {@link org.apache.hadoop.hbase.Cell}s from the wrapped scanner, + * applying its own policy to what gets written. * @param c the environment provided by the region server * @param store the store being compacted * @param scanner the scanner over existing data used in the store file rewriting @@ -206,8 +200,7 @@ public interface RegionObserver { */ default InternalScanner preCompact(ObserverContext c, Store store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, - CompactionRequest request) - throws IOException { + CompactionRequest request) throws IOException { return scanner; } http://git-wip-us.apache.org/repos/asf/hbase/blob/c9fdbec7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java -- diff --git a/hbase-server/src/main/java/org/apache/
[30/50] [abbrv] hbase git commit: HBASE-19007 Align Services Interfaces in Master and RegionServer
http://git-wip-us.apache.org/repos/asf/hbase/blob/38879fb3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenProvider.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenProvider.java index 0588138..e355752 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenProvider.java @@ -27,6 +27,8 @@ import java.util.Collections; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor; +import org.apache.hadoop.hbase.coprocessor.HasRegionServerServices; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils; @@ -46,6 +48,7 @@ import org.apache.yetus.audience.InterfaceAudience; * Provides a service for obtaining authentication tokens via the * {@link AuthenticationProtos} AuthenticationService coprocessor service. */ +@CoreCoprocessor @InterfaceAudience.Private public class TokenProvider implements AuthenticationProtos.AuthenticationService.Interface, RegionCoprocessor { @@ -59,11 +62,13 @@ public class TokenProvider implements AuthenticationProtos.AuthenticationService public void start(CoprocessorEnvironment env) { // if running at region if (env instanceof RegionCoprocessorEnvironment) { - RegionCoprocessorEnvironment regionEnv = - (RegionCoprocessorEnvironment)env; - assert regionEnv.getCoprocessorRegionServerServices() instanceof RegionServerServices; - RpcServerInterface server = ((RegionServerServices) regionEnv - .getCoprocessorRegionServerServices()).getRpcServer(); + RegionCoprocessorEnvironment regionEnv = (RegionCoprocessorEnvironment)env; + /* Getting the RpcServer from a RegionCE is wrong. There cannot be an expectation that Region + is hosted inside a RegionServer. If you need RpcServer, then pass in a RegionServerCE. + TODO: FIX. + */ + RegionServerServices rss = ((HasRegionServerServices)regionEnv).getRegionServerServices(); + RpcServerInterface server = rss.getRpcServer(); SecretManager mgr = ((RpcServer)server).getSecretManager(); if (mgr instanceof AuthenticationTokenSecretManager) { secretManager = (AuthenticationTokenSecretManager)mgr; http://git-wip-us.apache.org/repos/asf/hbase/blob/38879fb3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java index 8a5265d..5bd7c3f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -46,11 +46,11 @@ import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.AuthUtil; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants.OperationStatusCode; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.TagUtil; +import org.apache.hadoop.hbase.coprocessor.HasRegionServerServices; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Mutation; @@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.io.util.StreamUtils; import org.apache.hadoop.hbase.regionserver.OperationStatus; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; -import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; @@ -112,9 +111,15 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService @Override public void init(RegionCoprocessorEnvironment e) throws IOException { -assert
[32/50] [abbrv] hbase git commit: HBASE-18824 Add meaningful comment to HConstants.LATEST_TIMESTAMP to explain why it is MAX_VALUE
HBASE-18824 Add meaningful comment to HConstants.LATEST_TIMESTAMP to explain why it is MAX_VALUE Signed-off-by: Chia-Ping TsaiProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/2ee8690b Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/2ee8690b Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/2ee8690b Branch: refs/heads/HBASE-18410 Commit: 2ee8690b47763fd0ed97d47713b1c516633f597b Parents: 38879fb Author: Xiang Li Authored: Tue Sep 19 23:10:31 2017 +0800 Committer: Chia-Ping Tsai Committed: Sun Oct 22 04:47:00 2017 +0800 -- .../org/apache/hadoop/hbase/HConstants.java | 21 ++-- 1 file changed, 19 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/2ee8690b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java -- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 7577644..a272fc8 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -549,8 +549,25 @@ public final class HConstants { /** * Timestamp to use when we want to refer to the latest cell. - * This is the timestamp sent by clients when no timestamp is specified on - * commit. + * + * On client side, this is the timestamp set by default when no timestamp is specified, to refer to the latest. + * On server side, this acts as a notation. + * (1) For a cell of Put, which has this notation, + * its timestamp will be replaced with server's current time. + * (2) For a cell of Delete, which has this notation, + * A. If the cell is of {@link KeyValue.Type#Delete}, HBase issues a Get operation firstly. + *a. When the count of cell it gets is less than the count of cell to delete, + * the timestamp of Delete cell will be replaced with server's current time. + *b. When the count of cell it gets is equal to the count of cell to delete, + * the timestamp of Delete cell will be replaced with the latest timestamp of cell it gets. + * (c. It is invalid and an exception will be thrown, + * if the count of cell it gets is greater than the count of cell to delete, + * as the max version of Get is set to the count of cell to delete.) + * B. If the cell is of other Delete types, like {@link KeyValue.Type#DeleteFamilyVersion}, + *{@link KeyValue.Type#DeleteColumn}, or {@link KeyValue.Type#DeleteFamily}, + *the timestamp of Delete cell will be replaced with server's current time. + * + * So that is why it is named as "latest" but assigned as the max value of Long. */ public static final long LATEST_TIMESTAMP = Long.MAX_VALUE;
[40/50] [abbrv] hbase git commit: HBASE-18873 Move protobufs to private implementation on GlobalQuotaSettings
HBASE-18873 Move protobufs to private implementation on GlobalQuotaSettings A hack to "hide" the protobufs, but it's not going to be a trivial change to remove use of protobufs entirely as they're serialized into the hbase:quota table. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/81133f89 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/81133f89 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/81133f89 Branch: refs/heads/HBASE-18410 Commit: 81133f89fc9a80fbd03aff5a3b51184eeb90f130 Parents: b7db62c Author: Josh ElserAuthored: Wed Oct 11 18:37:42 2017 -0400 Committer: Josh Elser Committed: Mon Oct 23 22:37:10 2017 -0400 -- .../hbase/quotas/QuotaSettingsFactory.java | 2 +- .../hbase/quotas/GlobalQuotaSettings.java | 290 +--- .../hbase/quotas/GlobalQuotaSettingsImpl.java | 332 +++ .../hadoop/hbase/quotas/MasterQuotaManager.java | 72 ++-- .../hbase/quotas/TestGlobalQuotaSettings.java | 122 --- .../quotas/TestGlobalQuotaSettingsImpl.java | 122 +++ 6 files changed, 501 insertions(+), 439 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/81133f89/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java index 185365b..2a20c51 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java @@ -116,7 +116,7 @@ public class QuotaSettingsFactory { return settings; } - private static List fromThrottle(final String userName, final TableName tableName, + protected static List fromThrottle(final String userName, final TableName tableName, final String namespace, final QuotaProtos.Throttle throttle) { List settings = new ArrayList<>(); if (throttle.hasReqNum()) { http://git-wip-us.apache.org/repos/asf/hbase/blob/81133f89/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java index 079edf0..107523b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java @@ -16,23 +16,12 @@ */ package org.apache.hadoop.hbase.quotas; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Objects; +import java.util.List; -import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.quotas.QuotaSettingsFactory.QuotaGlobalsSettingsBypass; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest.Builder; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; @@ -43,28 +32,19 @@ import org.apache.yetus.audience.InterfaceStability; */ @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC}) @InterfaceStability.Evolving -public class GlobalQuotaSettings extends QuotaSettings { - private final QuotaProtos.Throttle throttleProto; - private final Boolean bypassGlobals; - private final QuotaProtos.SpaceQuota spaceProto; +public abstract class GlobalQuotaSettings extends QuotaSettings { - protected GlobalQuotaSettings( - String username, TableName tableName, String namespace, QuotaProtos.Quotas quotas) { -this(username, tableName, namespace, -(quotas != null && quotas.hasThrottle() ? quotas.getThrottle() : null), -(quotas != null && quotas.hasBypassGlobals() ? quotas.getBypassGlobals() : null), -(quotas != null && quotas.hasSpace() ? quotas.getSpace() : null)); - } - - protected
[36/50] [abbrv] hbase git commit: HBASE-19067 Do not expose getHDFSBlockDistribution in StoreFile.
HBASE-19067 Do not expose getHDFSBlockDistribution in StoreFile. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/880b26d7 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/880b26d7 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/880b26d7 Branch: refs/heads/HBASE-18410 Commit: 880b26d7d8678c688d741d991f55bd2245bee345 Parents: c9fdbec Author: anoopsamjohnAuthored: Mon Oct 23 17:04:05 2017 +0530 Committer: anoopsamjohn Committed: Mon Oct 23 17:04:05 2017 +0530 -- .../org/apache/hadoop/hbase/coprocessor/RegionObserver.java | 4 .../java/org/apache/hadoop/hbase/regionserver/HStoreFile.java | 5 - .../java/org/apache/hadoop/hbase/regionserver/StoreFile.java | 7 --- 3 files changed, 8 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/880b26d7/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index ba96a5b..815daf1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -958,6 +958,8 @@ public interface RegionObserver { * @deprecated For Phoenix only, StoreFileReader is not a stable interface. */ @Deprecated + // Passing InterfaceAudience.Private args FSDataInputStreamWrapper, CacheConfig and Reference. + // This is fine as the hook is deprecated any way. default StoreFileReader preStoreFileReaderOpen(ObserverContext ctx, FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r, StoreFileReader reader) throws IOException { @@ -979,6 +981,8 @@ public interface RegionObserver { * @deprecated For Phoenix only, StoreFileReader is not a stable interface. */ @Deprecated + // Passing InterfaceAudience.Private args FSDataInputStreamWrapper, CacheConfig and Reference. + // This is fine as the hook is deprecated any way. default StoreFileReader postStoreFileReaderOpen(ObserverContext ctx, FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r, StoreFileReader reader) throws IOException { http://git-wip-us.apache.org/repos/asf/hbase/blob/880b26d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java index 5301922..0ca01a5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java @@ -331,7 +331,10 @@ public class HStoreFile implements StoreFile { : OptionalLong.of(Bytes.toLong(bulkLoadTimestamp)); } - @Override + /** + * @return the cached value of HDFS blocks distribution. The cached value is calculated when store + * file is opened. + */ public HDFSBlocksDistribution getHDFSBlockDistribution() { return this.fileInfo.getHDFSBlockDistribution(); } http://git-wip-us.apache.org/repos/asf/hbase/blob/880b26d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index 9e318cd..4f4cfcc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -25,7 +25,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; @@ -118,12 +117,6 @@ public interface StoreFile { OptionalLong getBulkLoadTimestamp(); /** - * @return the cached value of HDFS blocks distribution. The cached value is calculated when store - * file is opened. - */ - HDFSBlocksDistribution getHDFSBlockDistribution(); - - /** * @return a length description of
hbase git commit: HBASE-18989 Polish the compaction related CP hooks
Repository: hbase Updated Branches: refs/heads/branch-2 fad75f07a -> 4c43ef268 HBASE-18989 Polish the compaction related CP hooks Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/4c43ef26 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4c43ef26 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4c43ef26 Branch: refs/heads/branch-2 Commit: 4c43ef2683e8e410c28fda9f96d912f2edf9fe29 Parents: fad75f0 Author: zhangduo <zhang...@apache.org> Authored: Mon Oct 23 16:44:54 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Mon Oct 23 16:52:02 2017 +0800 -- .../hbase/coprocessor/RegionObserver.java | 23 +- .../hadoop/hbase/regionserver/CompactSplit.java | 101 +-- .../hadoop/hbase/regionserver/HRegion.java | 34 ++- .../hbase/regionserver/HRegionServer.java | 10 +- .../hadoop/hbase/regionserver/HStore.java | 5 +- .../hbase/regionserver/RSRpcServices.java | 47 +--- .../hadoop/hbase/regionserver/Region.java | 19 +- .../regionserver/RegionServerServices.java | 10 +- .../apache/hadoop/hbase/regionserver/Store.java | 7 - .../compactions/CompactionLifeCycleTracker.java | 19 +- .../compactions/CompactionRequester.java| 46 .../hadoop/hbase/MockRegionServerServices.java | 6 + .../hadoop/hbase/master/MockRegionServer.java | 36 +-- .../hbase/regionserver/TestCompaction.java | 2 +- .../TestCompactionLifeCycleTracker.java | 267 +++ 15 files changed, 487 insertions(+), 145 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/4c43ef26/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index 94550df..ba96a5b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -158,7 +158,7 @@ public interface RegionObserver { /** * Called prior to selecting the {@link StoreFile StoreFiles} to compact from the list of * available candidates. To alter the files used for compaction, you may mutate the passed in list - * of candidates. + * of candidates. If you remove all the candidates then the compaction will be canceled. * @param c the environment provided by the region server * @param store the store where compaction is being requested * @param candidates the store files currently available for compaction @@ -183,18 +183,12 @@ public interface RegionObserver { /** * Called prior to writing the {@link StoreFile}s selected for compaction into a new - * {@code StoreFile}. To override or modify the compaction process, implementing classes have two - * options: - * - * Wrap the provided {@link InternalScanner} with a custom implementation that is returned - * from this method. The custom scanner can then inspect - * {@link org.apache.hadoop.hbase.KeyValue}s from the wrapped scanner, applying its own - * policy to what gets written. - * Call {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} and provide a - * custom implementation for writing of new {@link StoreFile}s. Note: any implementations - * bypassing core compaction using this approach must write out new store files themselves or the - * existing data will no longer be available after compaction. - * + * {@code StoreFile}. + * + * To override or modify the compaction process, implementing classes can wrap the provided + * {@link InternalScanner} with a custom implementation that is returned from this method. The + * custom scanner can then inspect {@link org.apache.hadoop.hbase.Cell}s from the wrapped scanner, + * applying its own policy to what gets written. * @param c the environment provided by the region server * @param store the store being compacted * @param scanner the scanner over existing data used in the store file rewriting @@ -206,8 +200,7 @@ public interface RegionObserver { */ default InternalScanner preCompact(ObserverContext c, Store store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, - CompactionRequest request) - throws IOException { + CompactionRequest request) throws IOException { return scanner; } http://git-wip-us.apache.org/repos/asf/hbase/blob/4c43ef26/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Co
hbase git commit: HBASE-18989 Polish the compaction related CP hooks
Repository: hbase Updated Branches: refs/heads/master 4add40ca2 -> c9fdbec77 HBASE-18989 Polish the compaction related CP hooks Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c9fdbec7 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c9fdbec7 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c9fdbec7 Branch: refs/heads/master Commit: c9fdbec772fe7dea06644d86e2854b98047ac9da Parents: 4add40c Author: zhangduo <zhang...@apache.org> Authored: Mon Oct 23 16:44:54 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Mon Oct 23 16:44:54 2017 +0800 -- .../hbase/coprocessor/RegionObserver.java | 23 +- .../hadoop/hbase/regionserver/CompactSplit.java | 101 +-- .../hadoop/hbase/regionserver/HRegion.java | 34 ++- .../hbase/regionserver/HRegionServer.java | 10 +- .../hadoop/hbase/regionserver/HStore.java | 5 +- .../hbase/regionserver/RSRpcServices.java | 47 +--- .../hadoop/hbase/regionserver/Region.java | 19 +- .../regionserver/RegionServerServices.java | 10 +- .../apache/hadoop/hbase/regionserver/Store.java | 7 - .../compactions/CompactionLifeCycleTracker.java | 19 +- .../compactions/CompactionRequester.java| 46 .../hadoop/hbase/MockRegionServerServices.java | 6 + .../hadoop/hbase/master/MockRegionServer.java | 36 +-- .../hbase/regionserver/TestCompaction.java | 2 +- .../TestCompactionLifeCycleTracker.java | 267 +++ 15 files changed, 487 insertions(+), 145 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/c9fdbec7/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index 94550df..ba96a5b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -158,7 +158,7 @@ public interface RegionObserver { /** * Called prior to selecting the {@link StoreFile StoreFiles} to compact from the list of * available candidates. To alter the files used for compaction, you may mutate the passed in list - * of candidates. + * of candidates. If you remove all the candidates then the compaction will be canceled. * @param c the environment provided by the region server * @param store the store where compaction is being requested * @param candidates the store files currently available for compaction @@ -183,18 +183,12 @@ public interface RegionObserver { /** * Called prior to writing the {@link StoreFile}s selected for compaction into a new - * {@code StoreFile}. To override or modify the compaction process, implementing classes have two - * options: - * - * Wrap the provided {@link InternalScanner} with a custom implementation that is returned - * from this method. The custom scanner can then inspect - * {@link org.apache.hadoop.hbase.KeyValue}s from the wrapped scanner, applying its own - * policy to what gets written. - * Call {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} and provide a - * custom implementation for writing of new {@link StoreFile}s. Note: any implementations - * bypassing core compaction using this approach must write out new store files themselves or the - * existing data will no longer be available after compaction. - * + * {@code StoreFile}. + * + * To override or modify the compaction process, implementing classes can wrap the provided + * {@link InternalScanner} with a custom implementation that is returned from this method. The + * custom scanner can then inspect {@link org.apache.hadoop.hbase.Cell}s from the wrapped scanner, + * applying its own policy to what gets written. * @param c the environment provided by the region server * @param store the store being compacted * @param scanner the scanner over existing data used in the store file rewriting @@ -206,8 +200,7 @@ public interface RegionObserver { */ default InternalScanner preCompact(ObserverContext c, Store store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, - CompactionRequest request) - throws IOException { + CompactionRequest request) throws IOException { return scanner; } http://git-wip-us.apache.org/repos/asf/hbase/blob/c9fdbec7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Co
hbase git commit: HBASE-19241 Improve javadoc for AsyncAdmin and cleanup warnings for the implementation classes
Repository: hbase Updated Branches: refs/heads/master 1ba7cc216 -> 5c312667e HBASE-19241 Improve javadoc for AsyncAdmin and cleanup warnings for the implementation classes Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/5c312667 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5c312667 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5c312667 Branch: refs/heads/master Commit: 5c312667ed9b9b3bbd870c7fcf26591479da000d Parents: 1ba7cc2 Author: zhangduo <zhang...@apache.org> Authored: Mon Nov 13 16:40:10 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Mon Nov 13 16:40:10 2017 +0800 -- .../apache/hadoop/hbase/client/AsyncAdmin.java | 3 - .../hbase/client/AsyncConnectionImpl.java | 5 +- .../hadoop/hbase/client/AsyncHBaseAdmin.java| 16 +- .../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 165 ++- .../apache/hadoop/hbase/GenericTestUtils.java | 1 - 5 files changed, 95 insertions(+), 95 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/5c312667/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java index baae6cf..f251a8f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java @@ -46,9 +46,6 @@ import com.google.protobuf.RpcChannel; /** * The asynchronous administrative API for HBase. - * - * This feature is still under development, so marked as IA.Private. Will change to public when - * done. Use it with caution. * @since 2.0.0 */ @InterfaceAudience.Public http://git-wip-us.apache.org/repos/asf/hbase/blob/5c312667/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java index 320d970..d5df785 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java @@ -282,7 +282,7 @@ class AsyncConnectionImpl implements AsyncConnection { return new AsyncAdminBuilderBase(connConf) { @Override public AsyncAdmin build() { -return new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, this); +return new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this); } }; } @@ -292,7 +292,8 @@ class AsyncConnectionImpl implements AsyncConnection { return new AsyncAdminBuilderBase(connConf) { @Override public AsyncAdmin build() { -RawAsyncHBaseAdmin rawAdmin = new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, this); +RawAsyncHBaseAdmin rawAdmin = +new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this); return new AsyncHBaseAdmin(rawAdmin, pool); } }; http://git-wip-us.apache.org/repos/asf/hbase/blob/5c312667/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java index 04005eb..250a38c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.client; +import com.google.protobuf.RpcChannel; + import java.util.Collection; import java.util.EnumSet; import java.util.List; @@ -28,8 +30,6 @@ import java.util.concurrent.ExecutorService; import java.util.function.Function; import java.util.regex.Pattern; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.ClusterStatus; import org.apache.hadoop.hbase.ClusterStatus.Option; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -45,17 +45,19 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; import org.apache.yetus.audience.InterfaceAudience; -import com.google.protobuf.RpcChannel; - /** - * The implementation of AsyncAdmin. + * Just a wrapper of {@link Raw
hbase git commit: HBASE-19241 Improve javadoc for AsyncAdmin and cleanup warnings for the implementation classes
Repository: hbase Updated Branches: refs/heads/branch-2 cd681f26b -> 8a5273f38 HBASE-19241 Improve javadoc for AsyncAdmin and cleanup warnings for the implementation classes Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/8a5273f3 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/8a5273f3 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/8a5273f3 Branch: refs/heads/branch-2 Commit: 8a5273f38c430873e3abf28f740088c1c36a322d Parents: cd681f2 Author: zhangduo <zhang...@apache.org> Authored: Mon Nov 13 16:40:10 2017 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Mon Nov 13 17:00:14 2017 +0800 -- .../apache/hadoop/hbase/client/AsyncAdmin.java | 3 - .../hbase/client/AsyncConnectionImpl.java | 5 +- .../hadoop/hbase/client/AsyncHBaseAdmin.java| 16 +- .../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 165 ++- .../apache/hadoop/hbase/GenericTestUtils.java | 1 - 5 files changed, 95 insertions(+), 95 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/8a5273f3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java index baae6cf..f251a8f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java @@ -46,9 +46,6 @@ import com.google.protobuf.RpcChannel; /** * The asynchronous administrative API for HBase. - * - * This feature is still under development, so marked as IA.Private. Will change to public when - * done. Use it with caution. * @since 2.0.0 */ @InterfaceAudience.Public http://git-wip-us.apache.org/repos/asf/hbase/blob/8a5273f3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java index 320d970..d5df785 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java @@ -282,7 +282,7 @@ class AsyncConnectionImpl implements AsyncConnection { return new AsyncAdminBuilderBase(connConf) { @Override public AsyncAdmin build() { -return new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, this); +return new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this); } }; } @@ -292,7 +292,8 @@ class AsyncConnectionImpl implements AsyncConnection { return new AsyncAdminBuilderBase(connConf) { @Override public AsyncAdmin build() { -RawAsyncHBaseAdmin rawAdmin = new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, this); +RawAsyncHBaseAdmin rawAdmin = +new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this); return new AsyncHBaseAdmin(rawAdmin, pool); } }; http://git-wip-us.apache.org/repos/asf/hbase/blob/8a5273f3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java index 04005eb..250a38c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.client; +import com.google.protobuf.RpcChannel; + import java.util.Collection; import java.util.EnumSet; import java.util.List; @@ -28,8 +30,6 @@ import java.util.concurrent.ExecutorService; import java.util.function.Function; import java.util.regex.Pattern; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.ClusterStatus; import org.apache.hadoop.hbase.ClusterStatus.Option; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -45,17 +45,19 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; import org.apache.yetus.audience.InterfaceAudience; -import com.google.protobuf.RpcChannel; - /** - * The implementation of AsyncAdmin. + * Just a wrapper of {@link Raw