Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id D56AB200CC6 for ; Tue, 18 Jul 2017 22:48:16 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id D3F71167962; Tue, 18 Jul 2017 20:48:16 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 93FDA167958 for ; Tue, 18 Jul 2017 22:48:14 +0200 (CEST) Received: (qmail 87058 invoked by uid 500); 18 Jul 2017 20:48:07 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 86809 invoked by uid 99); 18 Jul 2017 20:48:06 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 18 Jul 2017 20:48:06 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 30CECDF97C; Tue, 18 Jul 2017 20:48:06 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: mmiller@apache.org To: commits@accumulo.apache.org Date: Tue, 18 Jul 2017 20:48:07 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [03/10] accumulo git commit: ACCUMULO-3238 Table.ID Namespace.ID Refactor archived-at: Tue, 18 Jul 2017 20:48:17 -0000 http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java index 64e3bcd..7e6188c 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java @@ -69,7 +69,9 @@ import org.apache.accumulo.core.client.SampleNotPresentException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.impl.CompressedIterators; import org.apache.accumulo.core.client.impl.DurabilityImpl; +import org.apache.accumulo.core.client.impl.Namespace; import org.apache.accumulo.core.client.impl.ScannerImpl; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.client.impl.Tables; import org.apache.accumulo.core.client.impl.TabletLocator; import org.apache.accumulo.core.client.impl.TabletType; @@ -480,8 +482,8 @@ public class TabletServer extends AccumuloServerContext implements Runnable { long readaheadThreshold, TSamplerConfiguration tSamplerConfig, long batchTimeOut, String context) throws NotServingTabletException, ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException, TSampleNotPresentException { - String tableId = new String(textent.getTable(), UTF_8); - String namespaceId; + Table.ID tableId = new Table.ID(new String(textent.getTable(), UTF_8)); + Namespace.ID namespaceId; try { namespaceId = Tables.getNamespaceId(getInstance(), tableId); } catch (TableNotFoundException e1) { @@ -637,17 +639,17 @@ public class TabletServer extends AccumuloServerContext implements Runnable { List ssiList, Map> ssio, List authorizations, boolean waitForWrites, TSamplerConfiguration tSamplerConfig, long batchTimeOut, String context) throws ThriftSecurityException, TSampleNotPresentException { // find all of the tables that need to be scanned - final HashSet tables = new HashSet<>(); + final HashSet tables = new HashSet<>(); for (TKeyExtent keyExtent : tbatch.keySet()) { - tables.add(new String(keyExtent.getTable(), UTF_8)); + tables.add(new Table.ID(new String(keyExtent.getTable(), UTF_8))); } if (tables.size() != 1) throw new IllegalArgumentException("Cannot batch scan over multiple tables"); // check if user has permission to the tables - for (String tableId : tables) { - String namespaceId; + for (Table.ID tableId : tables) { + Namespace.ID namespaceId; try { namespaceId = Tables.getNamespaceId(getInstance(), tableId); } catch (TableNotFoundException e1) { @@ -781,12 +783,12 @@ public class TabletServer extends AccumuloServerContext implements Runnable { return; } - String tableId = ""; + Table.ID tableId = null; try { // if user has no permission to write to this table, add it to // the failures list boolean sameTable = us.currentTablet != null && (us.currentTablet.getExtent().getTableId().equals(keyExtent.getTableId())); - tableId = keyExtent.getTableId().toString(); + tableId = keyExtent.getTableId(); if (sameTable || security.canWrite(us.getCredentials(), tableId, Tables.getNamespaceId(getInstance(), tableId))) { long t2 = System.currentTimeMillis(); us.authTimes.addStat(t2 - t1); @@ -1074,8 +1076,8 @@ public class TabletServer extends AccumuloServerContext implements Runnable { public void update(TInfo tinfo, TCredentials credentials, TKeyExtent tkeyExtent, TMutation tmutation, TDurability tdurability) throws NotServingTabletException, ConstraintViolationException, ThriftSecurityException { - final String tableId = new String(tkeyExtent.getTable(), UTF_8); - String namespaceId; + final Table.ID tableId = new Table.ID(new String(tkeyExtent.getTable(), UTF_8)); + Namespace.ID namespaceId; try { namespaceId = Tables.getNamespaceId(getInstance(), tableId); } catch (TableNotFoundException e1) { @@ -1310,11 +1312,12 @@ public class TabletServer extends AccumuloServerContext implements Runnable { } @Override - public TConditionalSession startConditionalUpdate(TInfo tinfo, TCredentials credentials, List authorizations, String tableId, + public TConditionalSession startConditionalUpdate(TInfo tinfo, TCredentials credentials, List authorizations, String tableIdStr, TDurability tdurabilty, String classLoaderContext) throws ThriftSecurityException, TException { + Table.ID tableId = new Table.ID(tableIdStr); Authorizations userauths = null; - String namespaceId; + Namespace.ID namespaceId; try { namespaceId = Tables.getNamespaceId(getInstance(), tableId); } catch (TableNotFoundException e) { @@ -1354,7 +1357,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable { } } - String tid = cs.tableId; + Table.ID tid = cs.tableId; long opid = writeTracker.startWrite(TabletType.type(new KeyExtent(tid, null, null))); try { @@ -1405,8 +1408,8 @@ public class TabletServer extends AccumuloServerContext implements Runnable { public void splitTablet(TInfo tinfo, TCredentials credentials, TKeyExtent tkeyExtent, ByteBuffer splitPoint) throws NotServingTabletException, ThriftSecurityException { - String tableId = new String(ByteBufferUtil.toBytes(tkeyExtent.table)); - String namespaceId; + Table.ID tableId = new Table.ID(new String(ByteBufferUtil.toBytes(tkeyExtent.table))); + Namespace.ID namespaceId; try { namespaceId = Tables.getNamespaceId(getInstance(), tableId); } catch (TableNotFoundException ex) { @@ -1448,7 +1451,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable { onlineTabletsCopy = new TreeMap<>(onlineTablets); } List result = new ArrayList<>(); - String text = tableId; + Table.ID text = new Table.ID(tableId); KeyExtent start = new KeyExtent(text, new Text(), null); for (Entry entry : onlineTabletsCopy.tailMap(start).entrySet()) { KeyExtent ke = entry.getKey(); @@ -1623,7 +1626,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable { ArrayList tabletsToFlush = new ArrayList<>(); - KeyExtent ke = new KeyExtent(tableId, ByteBufferUtil.toText(endRow), ByteBufferUtil.toText(startRow)); + KeyExtent ke = new KeyExtent(new Table.ID(tableId), ByteBufferUtil.toText(endRow), ByteBufferUtil.toText(startRow)); synchronized (onlineTablets) { for (Tablet tablet : onlineTablets.values()) @@ -1741,7 +1744,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable { throw new RuntimeException(e); } - KeyExtent ke = new KeyExtent(tableId, ByteBufferUtil.toText(endRow), ByteBufferUtil.toText(startRow)); + KeyExtent ke = new KeyExtent(new Table.ID(tableId), ByteBufferUtil.toText(endRow), ByteBufferUtil.toText(startRow)); ArrayList tabletsToCompact = new ArrayList<>(); synchronized (onlineTablets) { @@ -1836,20 +1839,21 @@ public class TabletServer extends AccumuloServerContext implements Runnable { @Override public TSummaries startGetSummaries(TInfo tinfo, TCredentials credentials, TSummaryRequest request) throws ThriftSecurityException, ThriftTableOperationException, NoSuchScanIDException, TException { - String namespaceId; + Namespace.ID namespaceId; + Table.ID tableId = new Table.ID(request.getTableId()); try { - namespaceId = Tables.getNamespaceId(TabletServer.this.getInstance(), request.getTableId()); + namespaceId = Tables.getNamespaceId(TabletServer.this.getInstance(), tableId); } catch (TableNotFoundException e1) { - throw new ThriftTableOperationException(request.getTableId(), null, null, TableOperationExceptionType.NOTFOUND, null); + throw new ThriftTableOperationException(tableId.canonicalID(), null, null, TableOperationExceptionType.NOTFOUND, null); } - if (!security.canGetSummaries(credentials, request.getTableId(), namespaceId)) { + if (!security.canGetSummaries(credentials, tableId, namespaceId)) { throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED).asThriftException(); } ServerConfigurationFactory factory = TabletServer.this.getServerConfigurationFactory(); ExecutorService es = resourceManager.getSummaryPartitionExecutor(); - Future future = new Gatherer(TabletServer.this, request, factory.getTableConfiguration(request.getTableId())).gather(es); + Future future = new Gatherer(TabletServer.this, request, factory.getTableConfiguration(tableId)).gather(es); return startSummaryOperation(credentials, future); } @@ -1864,8 +1868,8 @@ public class TabletServer extends AccumuloServerContext implements Runnable { ServerConfigurationFactory factory = TabletServer.this.getServerConfigurationFactory(); ExecutorService spe = resourceManager.getSummaryRemoteExecutor(); - Future future = new Gatherer(TabletServer.this, request, factory.getTableConfiguration(request.getTableId())).processPartition(spe, - modulus, remainder); + Future future = new Gatherer(TabletServer.this, request, factory.getTableConfiguration(new Table.ID(request.getTableId()))) + .processPartition(spe, modulus, remainder); return startSummaryOperation(credentials, future); } @@ -1879,7 +1883,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable { } ExecutorService srp = resourceManager.getSummaryRetrievalExecutor(); - TableConfiguration tableCfg = confFactory.getTableConfiguration(request.getTableId()); + TableConfiguration tableCfg = confFactory.getTableConfiguration(new Table.ID(request.getTableId())); BlockCache summaryCache = resourceManager.getSummaryCache(); BlockCache indexCache = resourceManager.getIndexCache(); FileSystemResolver volMgr = p -> fs.getVolumeByPath(p).getFileSystem(); @@ -2324,7 +2328,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable { log.warn("{}", e.getMessage()); } - String tableId = extent.getTableId(); + Table.ID tableId = extent.getTableId(); ProblemReports.getInstance(TabletServer.this).report(new ProblemReport(tableId, TABLET_LOAD, extent.getUUID().toString(), getClientAddressString(), e)); } finally { releaseRecoveryMemory(extent); @@ -2747,7 +2751,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable { if (extent.isRootTablet()) { return verifyRootTablet(extent, instance); } - String tableToVerify = MetadataTable.ID; + Table.ID tableToVerify = MetadataTable.ID; if (extent.isMeta()) tableToVerify = RootTable.ID; @@ -2965,7 +2969,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable { SimpleTimer.getInstance(aconf).schedule(constraintTask, 0, 1000); } - public TabletServerStatus getStats(Map> scanCounts) { + public TabletServerStatus getStats(Map> scanCounts) { TabletServerStatus result = new TabletServerStatus(); Map onlineTabletsCopy; @@ -2975,7 +2979,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable { Map tables = new HashMap<>(); for (Entry entry : onlineTabletsCopy.entrySet()) { - String tableId = entry.getKey().getTableId(); + String tableId = entry.getKey().getTableId().canonicalID(); TableInfo table = tables.get(tableId); if (table == null) { table = new TableInfo(); @@ -3005,11 +3009,11 @@ public class TabletServer extends AccumuloServerContext implements Runnable { table.majors.queued++; } - for (Entry> entry : scanCounts.entrySet()) { - TableInfo table = tables.get(entry.getKey()); + for (Entry> entry : scanCounts.entrySet()) { + TableInfo table = tables.get(entry.getKey().canonicalID()); if (table == null) { table = new TableInfo(); - tables.put(entry.getKey(), table); + tables.put(entry.getKey().canonicalID(), table); } if (table.scans == null) @@ -3028,7 +3032,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable { } for (KeyExtent extent : offlineTabletsCopy) { - String tableId = extent.getTableId(); + String tableId = extent.getTableId().canonicalID(); TableInfo table = tables.get(tableId); if (table == null) { table = new TableInfo(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java index d5a4db9..0bdcf19 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java @@ -29,6 +29,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.accumulo.core.cli.Help; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.conf.SiteConfiguration; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.impl.KeyExtent; @@ -84,7 +85,7 @@ public class LogReader { row = new Text(opts.row); if (opts.extent != null) { String sa[] = opts.extent.split(";"); - ke = new KeyExtent(sa[0], new Text(sa[1]), new Text(sa[2])); + ke = new KeyExtent(new Table.ID(sa[0]), new Text(sa[1]), new Text(sa[2])); } if (opts.regexp != null) { Pattern pattern = Pattern.compile(opts.regexp); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java index cc79f31..0dd45c7 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java @@ -21,6 +21,7 @@ import java.util.Map; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.client.impl.Tables; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; @@ -49,7 +50,8 @@ public class ReplicationServicerHandler implements Iface { } @Override - public long replicateLog(String tableId, WalEdits data, TCredentials tcreds) throws RemoteReplicationException, TException { + public long replicateLog(String tableIdStr, WalEdits data, TCredentials tcreds) throws RemoteReplicationException, TException { + Table.ID tableId = new Table.ID(tableIdStr); log.debug("Got replication request to tableID {} with {} edits", tableId, data.getEditsSize()); tabletServer.getSecurityOperation().authenticateUser(tabletServer.rpcCreds(), tcreds); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java index dc62312..37940d9 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java @@ -19,18 +19,19 @@ package org.apache.accumulo.tserver.session; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.accumulo.core.client.Durability; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.security.thrift.TCredentials; public class ConditionalSession extends Session { public final TCredentials credentials; public final Authorizations auths; - public final String tableId; + public final Table.ID tableId; public final AtomicBoolean interruptFlag = new AtomicBoolean(); public final Durability durability; public final String classLoaderContext; - public ConditionalSession(TCredentials credentials, Authorizations authorizations, String tableId, Durability durability, String classLoaderContext) { + public ConditionalSession(TCredentials credentials, Authorizations authorizations, Table.ID tableId, Durability durability, String classLoaderContext) { super(credentials); this.credentials = credentials; this.auths = authorizations; http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java index bf37855..d4785e8 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java @@ -27,6 +27,7 @@ import java.util.Map.Entry; import java.util.Set; import java.util.TimerTask; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.client.impl.Translator; import org.apache.accumulo.core.client.impl.Translators; import org.apache.accumulo.core.conf.AccumuloConfiguration; @@ -230,8 +231,8 @@ public class SessionManager { } } - public synchronized Map> getActiveScansPerTable() { - Map> counts = new HashMap<>(); + public synchronized Map> getActiveScansPerTable() { + Map> counts = new HashMap<>(); Set> copiedIdleSessions = new HashSet<>(); synchronized (idleSessions) { @@ -248,7 +249,7 @@ public class SessionManager { Session session = entry.getValue(); @SuppressWarnings("rawtypes") ScanTask nbt = null; - String tableID = null; + Table.ID tableID = null; if (session instanceof ScanSession) { ScanSession ss = (ScanSession) session; @@ -320,8 +321,9 @@ public class SessionManager { } } - ActiveScan activeScan = new ActiveScan(ss.client, ss.getUser(), ss.extent.getTableId(), ct - ss.startTime, ct - ss.lastAccessTime, ScanType.SINGLE, - state, ss.extent.toThrift(), Translator.translate(ss.columnSet, Translators.CT), ss.ssiList, ss.ssio, ss.auths.getAuthorizationsBB(), ss.context); + ActiveScan activeScan = new ActiveScan(ss.client, ss.getUser(), ss.extent.getTableId().canonicalID(), ct - ss.startTime, ct - ss.lastAccessTime, + ScanType.SINGLE, state, ss.extent.toThrift(), Translator.translate(ss.columnSet, Translators.CT), ss.ssiList, ss.ssio, + ss.auths.getAuthorizationsBB(), ss.context); // scanId added by ACCUMULO-2641 is an optional thrift argument and not available in ActiveScan constructor activeScan.setScanId(entry.getKey()); @@ -350,7 +352,7 @@ public class SessionManager { } } - activeScans.add(new ActiveScan(mss.client, mss.getUser(), mss.threadPoolExtent.getTableId(), ct - mss.startTime, ct - mss.lastAccessTime, + activeScans.add(new ActiveScan(mss.client, mss.getUser(), mss.threadPoolExtent.getTableId().canonicalID(), ct - mss.startTime, ct - mss.lastAccessTime, ScanType.BATCH, state, mss.threadPoolExtent.toThrift(), Translator.translate(mss.columnSet, Translators.CT), mss.ssiList, mss.ssio, mss.auths .getAuthorizationsBB(), mss.context)); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java index 9fac722..3867120 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java @@ -220,7 +220,7 @@ class DatafileManager { boolean inTheRightDirectory = false; Path parent = tpath.path().getParent().getParent(); for (String tablesDir : ServerConstants.getTablesDirs()) { - if (parent.equals(new Path(tablesDir, tablet.getExtent().getTableId()))) { + if (parent.equals(new Path(tablesDir, tablet.getExtent().getTableId().canonicalID()))) { inTheRightDirectory = true; break; } http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java index ce064db..0b3cc57 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java @@ -54,6 +54,7 @@ import org.apache.accumulo.core.client.Durability; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.admin.CompactionStrategyConfig; import org.apache.accumulo.core.client.impl.DurabilityImpl; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.client.impl.Tables; import org.apache.accumulo.core.client.sample.SamplerConfiguration; import org.apache.accumulo.core.conf.AccumuloConfiguration; @@ -2131,8 +2132,8 @@ public class Tablet implements TabletCommitter { SortedMap highDatafileSizes = new TreeMap<>(); List highDatafilesToRemove = new ArrayList<>(); - MetadataTableUtil.splitDatafiles(extent.getTableId(), midRow, splitRatio, firstAndLastRows, getDatafileManager().getDatafileSizes(), lowDatafileSizes, - highDatafileSizes, highDatafilesToRemove); + MetadataTableUtil.splitDatafiles(midRow, splitRatio, firstAndLastRows, getDatafileManager().getDatafileSizes(), lowDatafileSizes, highDatafileSizes, + highDatafilesToRemove); log.debug("Files for low split " + low + " " + lowDatafileSizes.keySet()); log.debug("Files for high split " + high + " " + highDatafileSizes.keySet()); @@ -2617,7 +2618,7 @@ public class Tablet implements TabletCommitter { return scannedCount; } - private static String createTabletDirectory(VolumeManager fs, String tableId, Text endRow) { + private static String createTabletDirectory(VolumeManager fs, Table.ID tableId, Text endRow) { String lowDirectory; UniqueNameAllocator namer = UniqueNameAllocator.getInstance(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/test/java/org/apache/accumulo/tserver/AssignmentWatcherTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/AssignmentWatcherTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/AssignmentWatcherTest.java index 5dd3c19..c264aae 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/AssignmentWatcherTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/AssignmentWatcherTest.java @@ -19,6 +19,7 @@ package org.apache.accumulo.tserver; import java.util.HashMap; import java.util.Map; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.impl.KeyExtent; @@ -49,7 +50,7 @@ public class AssignmentWatcherTest { RunnableStartedAt run = new RunnableStartedAt(task, System.currentTimeMillis()); EasyMock.expect(conf.getTimeInMillis(Property.TSERV_ASSIGNMENT_DURATION_WARNING)).andReturn(0l); - assignments.put(new KeyExtent("1", null, null), run); + assignments.put(new KeyExtent(new Table.ID("1"), null, null), run); EasyMock.expect(task.getException()).andReturn(new Exception("Assignment warning happened")); EasyMock.expect(timer.schedule(watcher, 5000l)).andReturn(null); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/test/java/org/apache/accumulo/tserver/CheckTabletMetadataTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/CheckTabletMetadataTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/CheckTabletMetadataTest.java index 221e921..f97607e 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/CheckTabletMetadataTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/CheckTabletMetadataTest.java @@ -19,6 +19,7 @@ package org.apache.accumulo.tserver; import java.util.TreeMap; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.data.impl.KeyExtent; @@ -70,7 +71,7 @@ public class CheckTabletMetadataTest { @Test public void testBadTabletMetadata() throws Exception { - KeyExtent ke = new KeyExtent("1", null, null); + KeyExtent ke = new KeyExtent(new Table.ID("1"), null, null); TreeMap tabletMeta = new TreeMap<>(); @@ -89,9 +90,9 @@ public class CheckTabletMetadataTest { assertFail(tabletMeta, ke, new TServerInstance("127.0.0.2:9997", 4)); assertFail(tabletMeta, ke, new TServerInstance("127.0.0.2:9997", 5)); - assertFail(tabletMeta, new KeyExtent("1", null, new Text("m")), tsi); + assertFail(tabletMeta, new KeyExtent(new Table.ID("1"), null, new Text("m")), tsi); - assertFail(tabletMeta, new KeyExtent("1", new Text("r"), new Text("m")), tsi); + assertFail(tabletMeta, new KeyExtent(new Table.ID("1"), new Text("r"), new Text("m")), tsi); assertFail(tabletMeta, ke, tsi, newKey("1<", TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN)); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/test/java/org/apache/accumulo/tserver/LargestFirstMemoryManagerTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/LargestFirstMemoryManagerTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/LargestFirstMemoryManagerTest.java index 1c68e82..27045c3 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/LargestFirstMemoryManagerTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/LargestFirstMemoryManagerTest.java @@ -23,6 +23,8 @@ import java.util.List; import java.util.function.Function; import org.apache.accumulo.core.client.Instance; +import org.apache.accumulo.core.client.impl.Namespace; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.conf.Property; @@ -70,12 +72,12 @@ public class LargestFirstMemoryManagerTest { } @Override - public TableConfiguration getTableConfiguration(String tableId) { + public TableConfiguration getTableConfiguration(Table.ID tableId) { return delegate.getTableConfiguration(tableId); } @Override - public NamespaceConfiguration getNamespaceConfiguration(String namespaceId) { + public NamespaceConfiguration getNamespaceConfiguration(Namespace.ID namespaceId) { return delegate.getNamespaceConfiguration(namespaceId); } @@ -170,7 +172,7 @@ public class LargestFirstMemoryManagerTest { @Test public void testDeletedTable() throws Exception { final String deletedTableId = "1"; - Function existenceCheck = tableId -> !deletedTableId.equals(tableId); + Function existenceCheck = tableId -> !deletedTableId.equals(tableId); LargestFirstMemoryManagerWithExistenceCheck mgr = new LargestFirstMemoryManagerWithExistenceCheck(existenceCheck); ServerConfiguration config = new ServerConfiguration() { ServerConfigurationFactory delegate = new ServerConfigurationFactory(inst); @@ -181,12 +183,12 @@ public class LargestFirstMemoryManagerTest { } @Override - public TableConfiguration getTableConfiguration(String tableId) { + public TableConfiguration getTableConfiguration(Table.ID tableId) { return delegate.getTableConfiguration(tableId); } @Override - public NamespaceConfiguration getNamespaceConfiguration(String namespaceId) { + public NamespaceConfiguration getNamespaceConfiguration(Namespace.ID namespaceId) { return delegate.getNamespaceConfiguration(namespaceId); } @@ -194,7 +196,7 @@ public class LargestFirstMemoryManagerTest { mgr.init(config); MemoryManagementActions result; // one tablet is really big and the other is for a nonexistent table - KeyExtent extent = new KeyExtent("2", new Text("j"), null); + KeyExtent extent = new KeyExtent(new Table.ID("2"), new Text("j"), null); result = mgr.getMemoryManagementActions(tablets(t(extent, ZERO, ONE_GIG, 0), t(k("j"), ZERO, ONE_GIG, 0))); assertEquals(1, result.tabletsToMinorCompact.size()); assertEquals(extent, result.tabletsToMinorCompact.get(0)); @@ -215,28 +217,28 @@ public class LargestFirstMemoryManagerTest { } @Override - protected boolean tableExists(Instance instance, String tableId) { + protected boolean tableExists(Instance instance, Table.ID tableId) { return true; } } private static class LargestFirstMemoryManagerWithExistenceCheck extends LargestFirstMemoryManagerUnderTest { - Function existenceCheck; + Function existenceCheck; - public LargestFirstMemoryManagerWithExistenceCheck(Function existenceCheck) { + public LargestFirstMemoryManagerWithExistenceCheck(Function existenceCheck) { super(); this.existenceCheck = existenceCheck; } @Override - protected boolean tableExists(Instance instance, String tableId) { + protected boolean tableExists(Instance instance, Table.ID tableId) { return existenceCheck.apply(tableId); } } private static KeyExtent k(String endRow) { - return new KeyExtent("1", new Text(endRow), null); + return new KeyExtent(new Table.ID("1"), new Text(endRow), null); } private static class TestTabletState implements TabletState { http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletServerSyncCheckTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletServerSyncCheckTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletServerSyncCheckTest.java index d584581..515aa67 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletServerSyncCheckTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletServerSyncCheckTest.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Collections; import java.util.Map; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.conf.ConfigurationCopy; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.volume.Volume; @@ -188,7 +189,7 @@ public class TabletServerSyncCheckTest { } @Override - public Path getFullPath(String tableId, String path) { + public Path getFullPath(Table.ID tableId, String path) { return null; } http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java index f6a7c73..0c36d3a 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java @@ -29,6 +29,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; @@ -157,7 +158,7 @@ public class DefaultCompactionStrategyTest { } private MajorCompactionRequest createRequest(MajorCompactionReason reason, Object... objs) throws IOException { - return createRequest(new KeyExtent("0", null, null), reason, objs); + return createRequest(new KeyExtent(new Table.ID("0"), null, null), reason, objs); } private MajorCompactionRequest createRequest(KeyExtent extent, MajorCompactionReason reason, Object... objs) throws IOException { http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategyTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategyTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategyTest.java index 2c5f136..982d2b6 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategyTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategyTest.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.conf.ConfigurationTypeHelper; import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.data.impl.KeyExtent; @@ -51,7 +52,7 @@ public class SizeLimitCompactionStrategyTest { slcs.init(opts); - KeyExtent ke = new KeyExtent("0", null, null); + KeyExtent ke = new KeyExtent(new Table.ID("0"), null, null); MajorCompactionRequest mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, DefaultConfiguration.getInstance()); mcr.setFiles(nfl("f1", "2G", "f2", "2G", "f3", "2G", "f4", "2G")); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/TwoTierCompactionStrategyTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/TwoTierCompactionStrategyTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/TwoTierCompactionStrategyTest.java index 60145a7..80668bc 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/TwoTierCompactionStrategyTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/TwoTierCompactionStrategyTest.java @@ -22,6 +22,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.ConfigurationTypeHelper; import org.apache.accumulo.core.conf.DefaultConfiguration; @@ -63,7 +64,7 @@ public class TwoTierCompactionStrategyTest { public void testDefaultCompaction() throws IOException { ttcs.init(opts); conf = DefaultConfiguration.getInstance(); - KeyExtent ke = new KeyExtent("0", null, null); + KeyExtent ke = new KeyExtent(new Table.ID("0"), null, null); mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, conf); Map fileMap = createFileMap("f1", "10M", "f2", "10M", "f3", "10M", "f4", "10M", "f5", "100M", "f6", "100M", "f7", "100M", "f8", "100M"); @@ -82,7 +83,7 @@ public class TwoTierCompactionStrategyTest { public void testLargeCompaction() throws IOException { ttcs.init(opts); conf = DefaultConfiguration.getInstance(); - KeyExtent ke = new KeyExtent("0", null, null); + KeyExtent ke = new KeyExtent(new Table.ID("0"), null, null); mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, conf); Map fileMap = createFileMap("f1", "2G", "f2", "2G", "f3", "2G", "f4", "2G"); mcr.setFiles(fileMap); @@ -111,7 +112,7 @@ public class TwoTierCompactionStrategyTest { public void testFileSubsetCompaction() throws IOException { ttcs.init(opts); conf = DefaultConfiguration.getInstance(); - KeyExtent ke = new KeyExtent("0", null, null); + KeyExtent ke = new KeyExtent(new Table.ID("0"), null, null); mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, conf); Map fileMap = createFileMap("f1", "1G", "f2", "10M", "f3", "10M", "f4", "10M", "f5", "10M", "f6", "10M", "f7", "10M"); Map filesToCompactMap = createFileMap("f2", "10M", "f3", "10M", "f4", "10M", "f5", "10M", "f6", "10M", "f7", "10M"); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategyTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategyTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategyTest.java index 4f5dea3..c991ced 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategyTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategyTest.java @@ -20,6 +20,7 @@ package org.apache.accumulo.tserver.compaction.strategies; import java.util.HashMap; import java.util.Map; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.compaction.CompactionSettings; import org.apache.accumulo.core.conf.ConfigurationTypeHelper; import org.apache.accumulo.core.data.impl.KeyExtent; @@ -37,7 +38,7 @@ public class ConfigurableCompactionStrategyTest { @Test public void testOutputOptions() throws Exception { - MajorCompactionRequest mcr = new MajorCompactionRequest(new KeyExtent("1", null, null), MajorCompactionReason.USER, null); + MajorCompactionRequest mcr = new MajorCompactionRequest(new KeyExtent(new Table.ID("1"), null, null), MajorCompactionReason.USER, null); Map files = new HashMap<>(); files.put(new FileRef("hdfs://nn1/accumulo/tables/1/t-009/F00001.rf"), new DataFileValue(50000, 400)); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java index 2edcaa9..7b6474f 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java @@ -18,6 +18,7 @@ package org.apache.accumulo.tserver.log; import static org.junit.Assert.assertEquals; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.data.impl.KeyExtent; @@ -29,7 +30,7 @@ public class LogEntryTest { @Test public void test() throws Exception { - KeyExtent extent = new KeyExtent("1", null, new Text("")); + KeyExtent extent = new KeyExtent(new Table.ID("1"), null, new Text("")); long ts = 12345678L; String server = "localhost:1234"; String filename = "default/foo"; http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java index b65d5ce..e8da141 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java @@ -35,6 +35,7 @@ import java.util.Objects; import java.util.Set; import java.util.TreeMap; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.data.impl.KeyExtent; @@ -56,7 +57,7 @@ import org.junit.rules.TemporaryFolder; public class SortedLogRecoveryTest { - static final KeyExtent extent = new KeyExtent("table", null, null); + static final KeyExtent extent = new KeyExtent(new Table.ID("table"), null, null); static final Text cf = new Text("cf"); static final Text cq = new Text("cq"); static final Value value = new Value("value".getBytes()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/test/java/org/apache/accumulo/tserver/logger/LogFileTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/logger/LogFileTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/logger/LogFileTest.java index 582d6df..51205ac 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/logger/LogFileTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/logger/LogFileTest.java @@ -28,6 +28,7 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Arrays; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.data.impl.KeyExtent; @@ -80,7 +81,7 @@ public class LogFileTest { assertEquals(key.seq, 3); assertEquals(key.tid, 4); assertEquals(key.filename, "some file"); - KeyExtent tablet = new KeyExtent("table", new Text("bbbb"), new Text("aaaa")); + KeyExtent tablet = new KeyExtent(new Table.ID("table"), new Text("bbbb"), new Text("aaaa")); readWrite(DEFINE_TABLET, 5, 6, null, tablet, null, key, value); assertEquals(key.event, DEFINE_TABLET); assertEquals(key.seq, 5); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystemTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystemTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystemTest.java index 71d8c50..d05e849 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystemTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystemTest.java @@ -36,6 +36,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.ConfigurationCopy; import org.apache.accumulo.core.conf.Property; @@ -77,7 +78,7 @@ public class AccumuloReplicaSystemTest { * look like in a WAL. They are solely for testing that each LogEvents is handled, order is not important. */ key.event = LogEvents.DEFINE_TABLET; - key.tablet = new KeyExtent("1", null, null); + key.tablet = new KeyExtent(new Table.ID("1"), null, null); key.tid = 1; key.write(dos); @@ -92,7 +93,7 @@ public class AccumuloReplicaSystemTest { value.write(dos); key.event = LogEvents.DEFINE_TABLET; - key.tablet = new KeyExtent("2", null, null); + key.tablet = new KeyExtent(new Table.ID("2"), null, null); key.tid = 2; value.mutations = Collections.emptyList(); @@ -123,7 +124,7 @@ public class AccumuloReplicaSystemTest { value.write(dos); key.event = LogEvents.DEFINE_TABLET; - key.tablet = new KeyExtent("1", null, null); + key.tablet = new KeyExtent(new Table.ID("1"), null, null); key.tid = 3; value.mutations = Collections.emptyList(); @@ -157,8 +158,8 @@ public class AccumuloReplicaSystemTest { Status status = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(false).build(); DataInputStream dis = new DataInputStream(new ByteArrayInputStream(baos.toByteArray())); - WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", "1"), dis, new Path("/accumulo/wals/tserver+port/wal"), status, Long.MAX_VALUE, - new HashSet()); + WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", new Table.ID("1")), dis, new Path("/accumulo/wals/tserver+port/wal"), status, + Long.MAX_VALUE, new HashSet()); // We stopped because we got to the end of the file Assert.assertEquals(9, repl.entriesConsumed); @@ -183,7 +184,7 @@ public class AccumuloReplicaSystemTest { * look like in a WAL. They are solely for testing that each LogEvents is handled, order is not important. */ key.event = LogEvents.DEFINE_TABLET; - key.tablet = new KeyExtent("1", null, null); + key.tablet = new KeyExtent(new Table.ID("1"), null, null); key.tid = 1; key.write(dos); @@ -198,7 +199,7 @@ public class AccumuloReplicaSystemTest { value.write(dos); key.event = LogEvents.DEFINE_TABLET; - key.tablet = new KeyExtent("2", null, null); + key.tablet = new KeyExtent(new Table.ID("2"), null, null); key.tid = 2; value.mutations = Collections.emptyList(); @@ -229,7 +230,7 @@ public class AccumuloReplicaSystemTest { value.write(dos); key.event = LogEvents.DEFINE_TABLET; - key.tablet = new KeyExtent("1", null, null); + key.tablet = new KeyExtent(new Table.ID("1"), null, null); key.tid = 3; value.mutations = Collections.emptyList(); @@ -265,8 +266,8 @@ public class AccumuloReplicaSystemTest { // If it were still open, more data could be appended that we need to process Status status = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true).build(); DataInputStream dis = new DataInputStream(new ByteArrayInputStream(baos.toByteArray())); - WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", "1"), dis, new Path("/accumulo/wals/tserver+port/wal"), status, Long.MAX_VALUE, - new HashSet()); + WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", new Table.ID("1")), dis, new Path("/accumulo/wals/tserver+port/wal"), status, + Long.MAX_VALUE, new HashSet()); // We stopped because we got to the end of the file Assert.assertEquals(Long.MAX_VALUE, repl.entriesConsumed); @@ -300,7 +301,7 @@ public class AccumuloReplicaSystemTest { DataOutputStream out = new DataOutputStream(baos); // Replicate our 2 mutations to "peer", from tableid 1 to tableid 1 - ars.writeValueAvoidingReplicationCycles(out, value, new ReplicationTarget("peer", "1", "1")); + ars.writeValueAvoidingReplicationCycles(out, value, new ReplicationTarget("peer", "1", new Table.ID("1"))); out.close(); @@ -331,8 +332,8 @@ public class AccumuloReplicaSystemTest { // If it were still open, more data could be appended that we need to process Status status = Status.newBuilder().setBegin(100).setEnd(0).setInfiniteEnd(true).setClosed(true).build(); DataInputStream dis = new DataInputStream(new ByteArrayInputStream(new byte[0])); - WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", "1"), dis, new Path("/accumulo/wals/tserver+port/wal"), status, Long.MAX_VALUE, - new HashSet()); + WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", new Table.ID("1")), dis, new Path("/accumulo/wals/tserver+port/wal"), status, + Long.MAX_VALUE, new HashSet()); // We stopped because we got to the end of the file Assert.assertEquals(Long.MAX_VALUE, repl.entriesConsumed); @@ -354,8 +355,8 @@ public class AccumuloReplicaSystemTest { // If it were still open, more data could be appended that we need to process Status status = Status.newBuilder().setBegin(100).setEnd(0).setInfiniteEnd(true).setClosed(false).build(); DataInputStream dis = new DataInputStream(new ByteArrayInputStream(new byte[0])); - WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", "1"), dis, new Path("/accumulo/wals/tserver+port/wal"), status, Long.MAX_VALUE, - new HashSet()); + WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", new Table.ID("1")), dis, new Path("/accumulo/wals/tserver+port/wal"), status, + Long.MAX_VALUE, new HashSet()); // We stopped because we got to the end of the file Assert.assertEquals(0, repl.entriesConsumed); @@ -380,7 +381,7 @@ public class AccumuloReplicaSystemTest { * look like in a WAL. They are solely for testing that each LogEvents is handled, order is not important. */ key.event = LogEvents.DEFINE_TABLET; - key.tablet = new KeyExtent("1", null, null); + key.tablet = new KeyExtent(new Table.ID("1"), null, null); key.tid = 1; key.write(dos); @@ -418,7 +419,8 @@ public class AccumuloReplicaSystemTest { HashSet tids = new HashSet<>(); // Only consume the first mutation, not the second - WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", "1"), dis, new Path("/accumulo/wals/tserver+port/wal"), status, 1l, tids); + WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", new Table.ID("1")), dis, new Path("/accumulo/wals/tserver+port/wal"), status, 1l, + tids); // We stopped because we got to the end of the file Assert.assertEquals(2, repl.entriesConsumed); @@ -429,7 +431,7 @@ public class AccumuloReplicaSystemTest { status = Status.newBuilder(status).setBegin(2).build(); // Consume the rest of the mutations - repl = ars.getWalEdits(new ReplicationTarget("peer", "1", "1"), dis, new Path("/accumulo/wals/tserver+port/wal"), status, 1l, tids); + repl = ars.getWalEdits(new ReplicationTarget("peer", "1", new Table.ID("1")), dis, new Path("/accumulo/wals/tserver+port/wal"), status, 1l, tids); // We stopped because we got to the end of the file Assert.assertEquals(1, repl.entriesConsumed); @@ -445,7 +447,7 @@ public class AccumuloReplicaSystemTest { WalEdits edits = new WalEdits(Collections. emptyList()); WalReplication walReplication = new WalReplication(edits, 0, 0, 0); - ReplicationTarget target = new ReplicationTarget("peer", "2", "1"); + ReplicationTarget target = new ReplicationTarget("peer", "2", new Table.ID("1")); DataInputStream input = null; Path p = new Path("/accumulo/wals/tserver+port/" + UUID.randomUUID().toString()); Status status = null; @@ -474,7 +476,7 @@ public class AccumuloReplicaSystemTest { WalEdits edits = new WalEdits(Collections. emptyList()); WalReplication walReplication = new WalReplication(edits, 0, 5, 0); - ReplicationTarget target = new ReplicationTarget("peer", "2", "1"); + ReplicationTarget target = new ReplicationTarget("peer", "2", new Table.ID("1")); DataInputStream input = null; Path p = new Path("/accumulo/wals/tserver+port/" + UUID.randomUUID().toString()); Status status = null; @@ -499,7 +501,7 @@ public class AccumuloReplicaSystemTest { @Test public void testUserPassword() throws Exception { AccumuloReplicaSystem ars = new AccumuloReplicaSystem(); - ReplicationTarget target = new ReplicationTarget("peer", "peer_table", "1"); + ReplicationTarget target = new ReplicationTarget("peer", "peer_table", new Table.ID("1")); String user = "user", password = "password"; Map confMap = new HashMap<>(); @@ -514,7 +516,7 @@ public class AccumuloReplicaSystemTest { @Test public void testUserKeytab() throws Exception { AccumuloReplicaSystem ars = new AccumuloReplicaSystem(); - ReplicationTarget target = new ReplicationTarget("peer", "peer_table", "1"); + ReplicationTarget target = new ReplicationTarget("peer", "peer_table", new Table.ID("1")); String user = "user", keytab = "/etc/security/keytabs/replication.keytab"; Map confMap = new HashMap<>(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/ReplicationProcessorTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/ReplicationProcessorTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/ReplicationProcessorTest.java index 3ac5f2f..e82e96e 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/ReplicationProcessorTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/ReplicationProcessorTest.java @@ -25,6 +25,7 @@ import org.apache.accumulo.core.client.ClientConfiguration; import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.client.impl.ClientContext; import org.apache.accumulo.core.client.impl.Credentials; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.client.security.tokens.PasswordToken; import org.apache.accumulo.core.conf.ConfigurationCopy; import org.apache.accumulo.core.conf.Property; @@ -82,7 +83,7 @@ public class ReplicationProcessorTest { ReplicationProcessor proc = EasyMock.createMockBuilder(ReplicationProcessor.class) .addMockedMethods("getReplicaSystem", "doesFileExist", "getStatus", "getHelper").createMock(); - ReplicationTarget target = new ReplicationTarget("peer", "1", "1"); + ReplicationTarget target = new ReplicationTarget("peer", "1", new Table.ID("1")); Status status = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true).build(); Path path = new Path("/accumulo"); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/shell/src/main/java/org/apache/accumulo/shell/commands/DUCommand.java ---------------------------------------------------------------------- diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/DUCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/DUCommand.java index 4d312fc..38b5bf0 100644 --- a/shell/src/main/java/org/apache/accumulo/shell/commands/DUCommand.java +++ b/shell/src/main/java/org/apache/accumulo/shell/commands/DUCommand.java @@ -25,6 +25,7 @@ import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.client.NamespaceNotFoundException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.admin.DiskUsage; +import org.apache.accumulo.core.client.impl.Namespace; import org.apache.accumulo.core.client.impl.Namespaces; import org.apache.accumulo.core.util.NumUtil; import org.apache.accumulo.shell.Shell; @@ -50,7 +51,7 @@ public class DUCommand extends Command { if (cl.hasOption(optNamespace.getOpt())) { Instance instance = shellState.getInstance(); - String namespaceId = Namespaces.getNamespaceId(instance, cl.getOptionValue(optNamespace.getOpt())); + Namespace.ID namespaceId = Namespaces.getNamespaceId(instance, cl.getOptionValue(optNamespace.getOpt())); tables.addAll(Namespaces.getTableNames(instance, namespaceId)); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteNamespaceCommand.java ---------------------------------------------------------------------- diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteNamespaceCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteNamespaceCommand.java index b749f06..e27d1a9 100644 --- a/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteNamespaceCommand.java +++ b/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteNamespaceCommand.java @@ -20,7 +20,7 @@ import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.accumulo.core.client.NamespaceNotFoundException; +import org.apache.accumulo.core.client.impl.Namespace; import org.apache.accumulo.core.client.impl.Namespaces; import org.apache.accumulo.shell.Shell; import org.apache.accumulo.shell.Shell.Command; @@ -60,11 +60,8 @@ public class DeleteNamespaceCommand extends Command { protected void doTableOp(final Shell shellState, final String namespace, boolean force) throws Exception { boolean resetContext = false; String currentTable = shellState.getTableName(); - if (!Namespaces.getNameToIdMap(shellState.getInstance()).containsKey(namespace)) { - throw new NamespaceNotFoundException(null, namespace, null); - } - String namespaceId = Namespaces.getNamespaceId(shellState.getInstance(), namespace); + Namespace.ID namespaceId = Namespaces.getNamespaceId(shellState.getInstance(), namespace); List tables = Namespaces.getTableNames(shellState.getInstance(), namespaceId); resetContext = tables.contains(currentTable); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/shell/src/main/java/org/apache/accumulo/shell/commands/RenameNamespaceCommand.java ---------------------------------------------------------------------- diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/RenameNamespaceCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/RenameNamespaceCommand.java index 6d0cdd3..268b7db 100644 --- a/shell/src/main/java/org/apache/accumulo/shell/commands/RenameNamespaceCommand.java +++ b/shell/src/main/java/org/apache/accumulo/shell/commands/RenameNamespaceCommand.java @@ -26,7 +26,9 @@ import org.apache.accumulo.core.client.NamespaceExistsException; import org.apache.accumulo.core.client.NamespaceNotFoundException; import org.apache.accumulo.core.client.TableExistsException; import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.impl.Namespace; import org.apache.accumulo.core.client.impl.Namespaces; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.client.impl.Tables; import org.apache.accumulo.shell.Shell; import org.apache.accumulo.shell.Shell.Command; @@ -40,10 +42,10 @@ public class RenameNamespaceCommand extends Command { String old = cl.getArgs()[0]; String newer = cl.getArgs()[1]; boolean resetContext = false; - String currentTableId = ""; + Table.ID currentTableId = null; if (!(shellState.getTableName() == null) && !shellState.getTableName().isEmpty()) { - String namespaceId = Namespaces.getNamespaceId(shellState.getInstance(), old); - List tableIds = Namespaces.getTableIds(shellState.getInstance(), namespaceId); + Namespace.ID namespaceId = Namespaces.getNamespaceId(shellState.getInstance(), old); + List tableIds = Namespaces.getTableIds(shellState.getInstance(), namespaceId); currentTableId = Tables.getTableId(shellState.getInstance(), shellState.getTableName()); resetContext = tableIds.contains(currentTableId); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/shell/src/main/java/org/apache/accumulo/shell/commands/TableOperation.java ---------------------------------------------------------------------- diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/TableOperation.java b/shell/src/main/java/org/apache/accumulo/shell/commands/TableOperation.java index 6ad1246..45cd124 100644 --- a/shell/src/main/java/org/apache/accumulo/shell/commands/TableOperation.java +++ b/shell/src/main/java/org/apache/accumulo/shell/commands/TableOperation.java @@ -24,6 +24,8 @@ import java.util.TreeSet; import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.impl.Namespaces; +import org.apache.accumulo.core.client.impl.Namespace; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.client.impl.Tables; import org.apache.accumulo.shell.Shell; import org.apache.accumulo.shell.Shell.Command; @@ -55,8 +57,8 @@ public abstract class TableOperation extends Command { tableSet.add(cl.getOptionValue(optTableName.getOpt())); } else if (cl.hasOption(optNamespace.getOpt())) { Instance instance = shellState.getInstance(); - String namespaceId = Namespaces.getNamespaceId(instance, cl.getOptionValue(optNamespace.getOpt())); - for (String tableId : Namespaces.getTableIds(instance, namespaceId)) { + Namespace.ID namespaceId = Namespaces.getNamespaceId(instance, cl.getOptionValue(optNamespace.getOpt())); + for (Table.ID tableId : Namespaces.getTableIds(instance, namespaceId)) { tableSet.add(Tables.getTableName(instance, tableId)); } } else if (useCommandLine && cl.getArgs().length > 0) { http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/test/src/main/java/org/apache/accumulo/test/CloneIT.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/CloneIT.java b/test/src/main/java/org/apache/accumulo/test/CloneIT.java index 713be3b..d6f614e 100644 --- a/test/src/main/java/org/apache/accumulo/test/CloneIT.java +++ b/test/src/main/java/org/apache/accumulo/test/CloneIT.java @@ -26,6 +26,7 @@ import org.apache.accumulo.core.client.BatchWriter; import org.apache.accumulo.core.client.BatchWriterConfig; import org.apache.accumulo.core.client.Connector; import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Value; @@ -48,7 +49,7 @@ public class CloneIT extends AccumuloClusterHarness { String tableName = getUniqueNames(1)[0]; conn.tableOperations().create(tableName); - KeyExtent ke = new KeyExtent("0", null, null); + KeyExtent ke = new KeyExtent(new Table.ID("0"), null, null); Mutation mut = ke.getPrevRowUpdateMutation(); TabletsSection.ServerColumnFamily.TIME_COLUMN.put(mut, new Value("M0".getBytes())); @@ -62,9 +63,9 @@ public class CloneIT extends AccumuloClusterHarness { BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig()); - MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2); + MetadataTableUtil.initializeClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); - int rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2); + int rc = MetadataTableUtil.checkClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); assertEquals(0, rc); @@ -78,7 +79,7 @@ public class CloneIT extends AccumuloClusterHarness { String tableName = getUniqueNames(1)[0]; conn.tableOperations().create(tableName); - KeyExtent ke = new KeyExtent("0", null, null); + KeyExtent ke = new KeyExtent(new Table.ID("0"), null, null); Mutation mut = ke.getPrevRowUpdateMutation(); TabletsSection.ServerColumnFamily.TIME_COLUMN.put(mut, new Value("M0".getBytes())); @@ -93,7 +94,7 @@ public class CloneIT extends AccumuloClusterHarness { BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig()); - MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2); + MetadataTableUtil.initializeClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); Mutation mut2 = new Mutation(ke.getMetadataEntry()); mut2.putDelete(DataFileColumnFamily.NAME.toString(), "/default_tablet/0_0.rf"); @@ -102,16 +103,16 @@ public class CloneIT extends AccumuloClusterHarness { bw1.addMutation(mut2); bw1.flush(); - int rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2); + int rc = MetadataTableUtil.checkClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); assertEquals(1, rc); - rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2); + rc = MetadataTableUtil.checkClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); assertEquals(0, rc); Scanner scanner = conn.createScanner(tableName, Authorizations.EMPTY); - scanner.setRange(new KeyExtent("1", null, null).toMetadataRange()); + scanner.setRange(new KeyExtent(new Table.ID("1"), null, null).toMetadataRange()); HashSet files = new HashSet<>(); @@ -140,19 +141,19 @@ public class CloneIT extends AccumuloClusterHarness { BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig()); - MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2); + MetadataTableUtil.initializeClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); bw1.addMutation(createTablet("0", "m", null, "/default_tablet", "/default_tablet/0_0.rf")); bw1.addMutation(createTablet("0", null, "m", "/t-1", "/default_tablet/0_0.rf")); bw1.flush(); - int rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2); + int rc = MetadataTableUtil.checkClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); assertEquals(0, rc); Scanner scanner = conn.createScanner(tableName, Authorizations.EMPTY); - scanner.setRange(new KeyExtent("1", null, null).toMetadataRange()); + scanner.setRange(new KeyExtent(new Table.ID("1"), null, null).toMetadataRange()); HashSet files = new HashSet<>(); @@ -184,7 +185,7 @@ public class CloneIT extends AccumuloClusterHarness { BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig()); - MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2); + MetadataTableUtil.initializeClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); bw1.addMutation(createTablet("0", "m", null, "/default_tablet", "/default_tablet/1_0.rf")); Mutation mut3 = createTablet("0", null, "m", "/t-1", "/default_tablet/1_0.rf"); @@ -193,16 +194,16 @@ public class CloneIT extends AccumuloClusterHarness { bw1.flush(); - int rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2); + int rc = MetadataTableUtil.checkClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); assertEquals(1, rc); - rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2); + rc = MetadataTableUtil.checkClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); assertEquals(0, rc); Scanner scanner = conn.createScanner(tableName, Authorizations.EMPTY); - scanner.setRange(new KeyExtent("1", null, null).toMetadataRange()); + scanner.setRange(new KeyExtent(new Table.ID("1"), null, null).toMetadataRange()); HashSet files = new HashSet<>(); @@ -221,7 +222,7 @@ public class CloneIT extends AccumuloClusterHarness { } private static Mutation deleteTablet(String tid, String endRow, String prevRow, String dir, String file) throws Exception { - KeyExtent ke = new KeyExtent(tid, endRow == null ? null : new Text(endRow), prevRow == null ? null : new Text(prevRow)); + KeyExtent ke = new KeyExtent(new Table.ID(tid), endRow == null ? null : new Text(endRow), prevRow == null ? null : new Text(prevRow)); Mutation mut = new Mutation(ke.getMetadataEntry()); TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.putDelete(mut); TabletsSection.ServerColumnFamily.TIME_COLUMN.putDelete(mut); @@ -232,7 +233,7 @@ public class CloneIT extends AccumuloClusterHarness { } private static Mutation createTablet(String tid, String endRow, String prevRow, String dir, String file) throws Exception { - KeyExtent ke = new KeyExtent(tid, endRow == null ? null : new Text(endRow), prevRow == null ? null : new Text(prevRow)); + KeyExtent ke = new KeyExtent(new Table.ID(tid), endRow == null ? null : new Text(endRow), prevRow == null ? null : new Text(prevRow)); Mutation mut = ke.getPrevRowUpdateMutation(); TabletsSection.ServerColumnFamily.TIME_COLUMN.put(mut, new Value("M0".getBytes())); @@ -258,7 +259,7 @@ public class CloneIT extends AccumuloClusterHarness { BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig()); - MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2); + MetadataTableUtil.initializeClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); bw1.addMutation(createTablet("0", "f", null, "/d1", "/d1/file3")); bw1.addMutation(createTablet("0", "m", "f", "/d3", "/d1/file1")); @@ -267,12 +268,12 @@ public class CloneIT extends AccumuloClusterHarness { bw1.flush(); - int rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2); + int rc = MetadataTableUtil.checkClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); assertEquals(0, rc); Scanner scanner = conn.createScanner(tableName, Authorizations.EMPTY); - scanner.setRange(new KeyExtent("1", null, null).toMetadataRange()); + scanner.setRange(new KeyExtent(new Table.ID("1"), null, null).toMetadataRange()); HashSet files = new HashSet<>(); @@ -306,7 +307,7 @@ public class CloneIT extends AccumuloClusterHarness { BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig()); - MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2); + MetadataTableUtil.initializeClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); bw1.addMutation(deleteTablet("0", "m", null, "/d1", "/d1/file1")); bw1.addMutation(deleteTablet("0", null, "m", "/d2", "/d2/file2")); @@ -320,7 +321,7 @@ public class CloneIT extends AccumuloClusterHarness { bw1.flush(); - int rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2); + int rc = MetadataTableUtil.checkClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); assertEquals(1, rc); @@ -332,12 +333,12 @@ public class CloneIT extends AccumuloClusterHarness { bw1.flush(); - rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2); + rc = MetadataTableUtil.checkClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); assertEquals(0, rc); Scanner scanner = conn.createScanner(tableName, Authorizations.EMPTY); - scanner.setRange(new KeyExtent("1", null, null).toMetadataRange()); + scanner.setRange(new KeyExtent(new Table.ID("1"), null, null).toMetadataRange()); HashSet files = new HashSet<>(); @@ -372,7 +373,7 @@ public class CloneIT extends AccumuloClusterHarness { BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig()); - MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2); + MetadataTableUtil.initializeClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); bw1.addMutation(deleteTablet("0", "m", null, "/d1", "/d1/file1")); Mutation mut = createTablet("0", null, null, "/d2", "/d2/file2"); @@ -382,7 +383,7 @@ public class CloneIT extends AccumuloClusterHarness { bw1.flush(); try { - MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2); + MetadataTableUtil.checkClone(tableName, new Table.ID("0"), new Table.ID("1"), conn, bw2); assertTrue(false); } catch (TabletIterator.TabletDeletedException tde) {} http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/test/src/main/java/org/apache/accumulo/test/FileArchiveIT.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/FileArchiveIT.java b/test/src/main/java/org/apache/accumulo/test/FileArchiveIT.java index 8e51984..f3f3edd 100644 --- a/test/src/main/java/org/apache/accumulo/test/FileArchiveIT.java +++ b/test/src/main/java/org/apache/accumulo/test/FileArchiveIT.java @@ -22,6 +22,7 @@ import org.apache.accumulo.core.client.BatchWriter; import org.apache.accumulo.core.client.BatchWriterConfig; import org.apache.accumulo.core.client.Connector; import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; @@ -64,7 +65,7 @@ public class FileArchiveIT extends ConfigurableMacBase { conn.tableOperations().create(tableName); - final String tableId = conn.tableOperations().tableIdMap().get(tableName); + final Table.ID tableId = new Table.ID(conn.tableOperations().tableIdMap().get(tableName)); Assert.assertNotNull("Could not get table ID", tableId); BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig()); @@ -122,7 +123,7 @@ public class FileArchiveIT extends ConfigurableMacBase { conn.tableOperations().create(tableName); - final String tableId = conn.tableOperations().tableIdMap().get(tableName); + final Table.ID tableId = new Table.ID(conn.tableOperations().tableIdMap().get(tableName)); Assert.assertNotNull("Could not get table ID", tableId); BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig()); @@ -179,7 +180,7 @@ public class FileArchiveIT extends ConfigurableMacBase { conn.tableOperations().create(tableName); - final String tableId = conn.tableOperations().tableIdMap().get(tableName); + final Table.ID tableId = new Table.ID(conn.tableOperations().tableIdMap().get(tableName)); Assert.assertNotNull("Could not get table ID", tableId); BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/test/src/main/java/org/apache/accumulo/test/ImportExportIT.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/ImportExportIT.java b/test/src/main/java/org/apache/accumulo/test/ImportExportIT.java index ddc9404..41be4ba 100644 --- a/test/src/main/java/org/apache/accumulo/test/ImportExportIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ImportExportIT.java @@ -140,7 +140,7 @@ public class ImportExportIT extends AccumuloClusterHarness { // Get all `file` colfams from the metadata table for the new table log.info("Imported into table with ID: {}", tableId); Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY); - s.setRange(MetadataSchema.TabletsSection.getRange(tableId)); + s.setRange(MetadataSchema.TabletsSection.getRange(new org.apache.accumulo.core.client.impl.Table.ID(tableId))); s.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME); MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.fetch(s); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/test/src/main/java/org/apache/accumulo/test/ListTables.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/ListTables.java b/test/src/main/java/org/apache/accumulo/test/ListTables.java index be8a7d3..4aa1de0 100644 --- a/test/src/main/java/org/apache/accumulo/test/ListTables.java +++ b/test/src/main/java/org/apache/accumulo/test/ListTables.java @@ -18,6 +18,7 @@ package org.apache.accumulo.test; import java.util.Map.Entry; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.client.impl.Tables; import org.apache.accumulo.server.cli.ClientOpts; @@ -28,7 +29,7 @@ public class ListTables { public static void main(String[] args) throws Exception { ClientOpts opts = new ClientOpts(); opts.parseArgs(ListTables.class.getName(), args); - for (Entry table : Tables.getNameToIdMap(opts.getInstance()).entrySet()) + for (Entry table : Tables.getNameToIdMap(opts.getInstance()).entrySet()) System.out.println(table.getKey() + " => " + table.getValue()); } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/test/src/main/java/org/apache/accumulo/test/LocatorIT.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/LocatorIT.java b/test/src/main/java/org/apache/accumulo/test/LocatorIT.java index f5caf3c..b3c36a9 100644 --- a/test/src/main/java/org/apache/accumulo/test/LocatorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/LocatorIT.java @@ -30,6 +30,7 @@ import org.apache.accumulo.core.client.Connector; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.TableOfflineException; import org.apache.accumulo.core.client.admin.Locations; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.TabletId; import org.apache.accumulo.core.data.impl.KeyExtent; @@ -70,7 +71,7 @@ public class LocatorIT extends AccumuloClusterHarness { } private static TabletId newTabletId(String tableId, String endRow, String prevRow) { - return new TabletIdImpl(new KeyExtent(tableId, endRow == null ? null : new Text(endRow), prevRow == null ? null : new Text(prevRow))); + return new TabletIdImpl(new KeyExtent(new Table.ID(tableId), endRow == null ? null : new Text(endRow), prevRow == null ? null : new Text(prevRow))); } @Test http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/test/src/main/java/org/apache/accumulo/test/MetaConstraintRetryIT.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/MetaConstraintRetryIT.java b/test/src/main/java/org/apache/accumulo/test/MetaConstraintRetryIT.java index 468b41a..d5ac807 100644 --- a/test/src/main/java/org/apache/accumulo/test/MetaConstraintRetryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/MetaConstraintRetryIT.java @@ -19,6 +19,7 @@ package org.apache.accumulo.test; import org.apache.accumulo.core.client.impl.ClientContext; import org.apache.accumulo.core.client.impl.Credentials; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.client.impl.Writer; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.impl.KeyExtent; @@ -45,7 +46,7 @@ public class MetaConstraintRetryIT extends AccumuloClusterHarness { Credentials credentials = new Credentials(getAdminPrincipal(), getAdminToken()); ClientContext context = new ClientContext(getConnector().getInstance(), credentials, cluster.getClientConfig()); Writer w = new Writer(context, MetadataTable.ID); - KeyExtent extent = new KeyExtent("5", null, null); + KeyExtent extent = new KeyExtent(new Table.ID("5"), null, null); Mutation m = new Mutation(extent.getMetadataEntry()); // unknown columns should cause contraint violation http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java index c04201d..99db049 100644 --- a/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java @@ -25,6 +25,7 @@ import org.apache.accumulo.core.client.BatchWriter; import org.apache.accumulo.core.client.BatchWriterConfig; import org.apache.accumulo.core.client.Connector; import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.impl.KeyExtent; @@ -124,7 +125,7 @@ public class MissingWalHeaderCompletesRecoveryIT extends ConfigurableMacBase { String tableName = getUniqueNames(1)[0]; conn.tableOperations().create(tableName); - String tableId = conn.tableOperations().tableIdMap().get(tableName); + Table.ID tableId = new Table.ID(conn.tableOperations().tableIdMap().get(tableName)); Assert.assertNotNull("Table ID was null", tableId); LogEntry logEntry = new LogEntry(new KeyExtent(tableId, null, null), 0, "127.0.0.1:12345", emptyWalog.toURI().toString()); @@ -179,7 +180,7 @@ public class MissingWalHeaderCompletesRecoveryIT extends ConfigurableMacBase { String tableName = getUniqueNames(1)[0]; conn.tableOperations().create(tableName); - String tableId = conn.tableOperations().tableIdMap().get(tableName); + Table.ID tableId = new Table.ID(conn.tableOperations().tableIdMap().get(tableName)); Assert.assertNotNull("Table ID was null", tableId); LogEntry logEntry = new LogEntry(null, 0, "127.0.0.1:12345", partialHeaderWalog.toURI().toString()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java b/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java index 18c275a..6e8594b 100644 --- a/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java +++ b/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java @@ -910,8 +910,8 @@ public class NamespacesIT extends AccumuloClusterHarness { assertTrue(namespaces.contains(Namespaces.ACCUMULO_NAMESPACE)); assertTrue(namespaces.contains(Namespaces.DEFAULT_NAMESPACE)); assertFalse(namespaces.contains(namespace)); - assertEquals(Namespaces.ACCUMULO_NAMESPACE_ID, map.get(Namespaces.ACCUMULO_NAMESPACE)); - assertEquals(Namespaces.DEFAULT_NAMESPACE_ID, map.get(Namespaces.DEFAULT_NAMESPACE)); + assertEquals(Namespaces.ACCUMULO_NAMESPACE_ID.canonicalID(), map.get(Namespaces.ACCUMULO_NAMESPACE)); + assertEquals(Namespaces.DEFAULT_NAMESPACE_ID.canonicalID(), map.get(Namespaces.DEFAULT_NAMESPACE)); assertNull(map.get(namespace)); c.namespaceOperations().create(namespace); @@ -922,8 +922,8 @@ public class NamespacesIT extends AccumuloClusterHarness { assertTrue(namespaces.contains(Namespaces.ACCUMULO_NAMESPACE)); assertTrue(namespaces.contains(Namespaces.DEFAULT_NAMESPACE)); assertTrue(namespaces.contains(namespace)); - assertEquals(Namespaces.ACCUMULO_NAMESPACE_ID, map.get(Namespaces.ACCUMULO_NAMESPACE)); - assertEquals(Namespaces.DEFAULT_NAMESPACE_ID, map.get(Namespaces.DEFAULT_NAMESPACE)); + assertEquals(Namespaces.ACCUMULO_NAMESPACE_ID.canonicalID(), map.get(Namespaces.ACCUMULO_NAMESPACE)); + assertEquals(Namespaces.DEFAULT_NAMESPACE_ID.canonicalID(), map.get(Namespaces.DEFAULT_NAMESPACE)); assertNotNull(map.get(namespace)); c.namespaceOperations().delete(namespace); @@ -934,8 +934,8 @@ public class NamespacesIT extends AccumuloClusterHarness { assertTrue(namespaces.contains(Namespaces.ACCUMULO_NAMESPACE)); assertTrue(namespaces.contains(Namespaces.DEFAULT_NAMESPACE)); assertFalse(namespaces.contains(namespace)); - assertEquals(Namespaces.ACCUMULO_NAMESPACE_ID, map.get(Namespaces.ACCUMULO_NAMESPACE)); - assertEquals(Namespaces.DEFAULT_NAMESPACE_ID, map.get(Namespaces.DEFAULT_NAMESPACE)); + assertEquals(Namespaces.ACCUMULO_NAMESPACE_ID.canonicalID(), map.get(Namespaces.ACCUMULO_NAMESPACE)); + assertEquals(Namespaces.DEFAULT_NAMESPACE_ID.canonicalID(), map.get(Namespaces.DEFAULT_NAMESPACE)); assertNull(map.get(namespace)); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java b/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java index 4dcd39d..e80989b 100644 --- a/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java +++ b/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java @@ -120,7 +120,7 @@ public class QueryMetadataTable { location = entry.getValue().toString(); } - if (!entry.getKey().getRow().toString().startsWith(MetadataTable.ID)) + if (!entry.getKey().getRow().toString().startsWith(MetadataTable.ID.canonicalID())) rowSet.add(entry.getKey().getRow()); count++; } http://git-wip-us.apache.org/repos/asf/accumulo/blob/d6ea49e0/test/src/main/java/org/apache/accumulo/test/RewriteTabletDirectoriesIT.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/RewriteTabletDirectoriesIT.java b/test/src/main/java/org/apache/accumulo/test/RewriteTabletDirectoriesIT.java index 3317558..17b3ee4 100644 --- a/test/src/main/java/org/apache/accumulo/test/RewriteTabletDirectoriesIT.java +++ b/test/src/main/java/org/apache/accumulo/test/RewriteTabletDirectoriesIT.java @@ -32,6 +32,7 @@ import java.util.TreeSet; import org.apache.accumulo.core.client.BatchScanner; import org.apache.accumulo.core.client.BatchWriter; import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.impl.Table; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; @@ -99,7 +100,7 @@ public class RewriteTabletDirectoriesIT extends ConfigurableMacBase { BatchScanner scanner = c.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 1); DIRECTORY_COLUMN.fetch(scanner); - String tableId = c.tableOperations().tableIdMap().get(tableName); + Table.ID tableId = new Table.ID(c.tableOperations().tableIdMap().get(tableName)); assertNotNull("TableID for " + tableName + " was null", tableId); scanner.setRanges(Collections.singletonList(TabletsSection.getRange(tableId))); // verify the directory entries are all on v1, make a few entries relative