You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2022/09/01 14:55:46 UTC

[GitHub] [accumulo] jmark99 opened a new pull request, #2910: Add ability to retrieve TimeType for a table

jmark99 opened a new pull request, #2910:
URL: https://github.com/apache/accumulo/pull/2910

   Provide a way to programmatically retrieve a table's TimeType value.
   
   Added client method 'gettimetype' and corresponding shell command that retrieves the TimeType for a supplied table.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#discussion_r961829449


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -2011,6 +2013,16 @@ public ImportDestinationArguments importDirectory(String directory) {
     return new BulkImport(directory, context);
   }
 
+  @Override
+  public TimeType getTimeType(final String tableName) throws TableNotFoundException {
+    TableId tableId = context.getTableId(tableName);
+    Optional<TabletMetadata> tabletMetadata = context.getAmple().readTablets().forTable(tableId)
+        .fetch(TabletMetadata.ColumnType.TIME).checkConsistency().build().stream().findFirst();
+    TabletMetadata timeData =
+        tabletMetadata.orElseThrow(() -> new RuntimeException("Failed to retrieve TimeType"));

Review Comment:
   Prefer a more specific RTE type. Perhaps IllegalStateException here?



##########
shell/src/main/java/org/apache/accumulo/shell/commands/GetTimeTypeCommand.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.shell.commands;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.shell.Shell;
+import org.apache.accumulo.shell.Shell.Command;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+
+public class GetTimeTypeCommand extends Command {

Review Comment:
   Instead of a very narrow "gettimetype" command, a more generic "tableinfo" command would be better, so that we don't need a thousand separate narrowly-scoped commands to get basic information about a table.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#discussion_r960898970


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -2011,6 +2015,34 @@ public ImportDestinationArguments importDirectory(String directory) {
     return new BulkImport(directory, context);
   }
 
+  @Override
+  public TimeType getTimeType(final String tableName) throws TableNotFoundException {
+    if (tableName.equals(RootTable.NAME)) {
+      throw new IllegalArgumentException("accumulo.root table has no TimeType");
+    }
+    String systemTableToCheck =
+        MetadataTable.NAME.equals(tableName) ? RootTable.NAME : MetadataTable.NAME;
+    final Scanner scanner = context.createScanner(systemTableToCheck, Authorizations.EMPTY);
+    String tableId = tableIdMap().get(tableName);
+    if (tableId == null) {
+      throw new TableNotFoundException(null, tableName, "specified table does not exist");
+    }
+    final Text start = new Text(tableId);
+    final Text end = new Text(start);
+    start.append(new byte[] {'<'}, 0, 1);
+    end.append(new byte[] {'<'}, 0, 1);
+    scanner.setRange(new Range(start, end));
+    MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.fetch(scanner);
+    Entry<Key,Value> next = scanner.iterator().next();
+    Value val = next.getValue();
+    if (val.toString().startsWith("L")) {
+      return TimeType.LOGICAL;
+    } else if (val.toString().startsWith("M")) {
+      return TimeType.MILLIS;
+    }
+    throw new RuntimeException("Failed to retrieve TimeType");

Review Comment:
   May be able to use ample for this.  Also may not need a special case for root tablet w/ ample, seems like it will return the time type logical.  But not sure, this is the [code that initializes the root tablet metadata](https://github.com/apache/accumulo/blob/114704bb58a37073a0401c3b6f479761089bd1fb/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java#L189-L206) and makes me think it would return logical.
   
   ```suggestion
        String tableId = tableIdMap().get(tableName);
       if (tableId == null) {
         throw new TableNotFoundException(null, tableName, "specified table does not exist");
       }
           Optional<TabletMetadata> tabletMetadata = context.getAmple().readTablets().forTable(tableId)
               .fetch(ColumnType.TIME).checkConsistency().build().stream().findFirst();
       return tabletMetadata.orElseThrow(() -> new RuntimeException("Failed to retrieve TimeType"));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#discussion_r960780534


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -2011,6 +2015,34 @@ public ImportDestinationArguments importDirectory(String directory) {
     return new BulkImport(directory, context);
   }
 
+  @Override
+  public TimeType getTimeType(final String tableName) throws TableNotFoundException {
+    if (tableName.equals(RootTable.NAME)) {
+      throw new IllegalArgumentException("accumulo.root table has no TimeType");
+    }
+    String systemTableToCheck =
+        MetadataTable.NAME.equals(tableName) ? RootTable.NAME : MetadataTable.NAME;
+    final Scanner scanner = context.createScanner(systemTableToCheck, Authorizations.EMPTY);

Review Comment:
   suggest moving this line to right before scanner is used and put in try-with-resources block.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#discussion_r961507295


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -2011,6 +2013,20 @@ public ImportDestinationArguments importDirectory(String directory) {
     return new BulkImport(directory, context);
   }
 
+  @Override
+  public TimeType getTimeType(final String tableName) throws TableNotFoundException {
+    String tableId = tableIdMap().get(tableName);
+    if (tableId == null) {
+      throw new TableNotFoundException(null, tableName, "specified table does not exist");
+    }
+    Optional<TabletMetadata> tabletMetadata =
+        context.getAmple().readTablets().forTable(TableId.of(tableId))

Review Comment:
   May be able to use this function to get the table id and it throws a table not found exception.
   
   ```suggestion
       TableId tableId = context.getTableId(tableName);
       Optional<TabletMetadata> tabletMetadata =
           context.getAmple().readTablets().forTable(tableId)
   ```



##########
test/src/main/java/org/apache/accumulo/test/shell/ShellIT.java:
##########
@@ -614,4 +615,15 @@ public void testMaxSplitsOption() throws Exception {
         "0\n1\n2\n3\n4\n5\n6\n7\n8\n9\na\nb\nc\nd\ne\nf\ng\nh\ni\nj\nk\nl\nm\nn\no\np\nq\nr\ns\nt\n");
   }
 
+  @Test
+  public void testGetTimeType() throws Exception {
+    Shell.log.debug("Starting testGetTimeType test -----------------");
+    exec("createtable tmtype", true);
+    exec("gettimetype", true, TimeType.MILLIS.toString());
+    exec("gettimetype -t tmtype", true, TimeType.MILLIS.toString());

Review Comment:
   Could also try a user table with logical time to make the test cover a bit more.
   
   ```suggestion
       exec("gettimetype -t tmtype", true, TimeType.MILLIS.toString());
       exec("createtable -tl logicaltt", true);
        exec("gettimetype -t logicaltt", true, TimeType.LOGICAL.toString());
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] DomGarguilo commented on a diff in pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
DomGarguilo commented on code in PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#discussion_r960850160


##########
test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java:
##########
@@ -292,4 +295,59 @@ public void testCompactEmptyTablesWithBadIterator_FailsAndCancel() throws TableE
     }
   }
 
+  @Test
+  public void getTimeTypeTest() throws TableNotFoundException, AccumuloException,
+      TableExistsException, AccumuloSecurityException {
+    String[] tableNames = getUniqueNames(4);
+
+    // Create table with default MILLIS TimeType
+    accumuloClient.tableOperations().create(tableNames[0]);
+    TimeType timeType = accumuloClient.tableOperations().getTimeType(tableNames[0]);
+    assertEquals(TimeType.MILLIS, timeType);
+
+    // Create table with LOGICAL TimeType.
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    ntc.setTimeType(TimeType.LOGICAL);
+    accumuloClient.tableOperations().create(tableNames[1], ntc);
+    timeType = accumuloClient.tableOperations().getTimeType(tableNames[1]);
+    assertEquals(TimeType.LOGICAL, timeType);
+
+    // Create some split points
+    SortedSet<Text> splits = new TreeSet<>();
+    splits.add(new Text("F"));
+    splits.add(new Text("M"));
+    splits.add(new Text("S"));
+
+    // Create table with MILLIS TimeType. Use splits to create multiple tablets
+    ntc = new NewTableConfiguration();
+    ntc.withSplits(splits);
+    accumuloClient.tableOperations().create(tableNames[2], ntc);

Review Comment:
   Maybe a note could be added to the comment about `TimeType.MILLIS` being the default for ntc and/or another test case could be added which explicitly calls `ntc.setTimeType(TimeType.MILLIS);`



##########
test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java:
##########
@@ -292,4 +295,59 @@ public void testCompactEmptyTablesWithBadIterator_FailsAndCancel() throws TableE
     }
   }
 
+  @Test
+  public void getTimeTypeTest() throws TableNotFoundException, AccumuloException,
+      TableExistsException, AccumuloSecurityException {
+    String[] tableNames = getUniqueNames(4);
+
+    // Create table with default MILLIS TimeType
+    accumuloClient.tableOperations().create(tableNames[0]);
+    TimeType timeType = accumuloClient.tableOperations().getTimeType(tableNames[0]);
+    assertEquals(TimeType.MILLIS, timeType);
+
+    // Create table with LOGICAL TimeType.
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    ntc.setTimeType(TimeType.LOGICAL);
+    accumuloClient.tableOperations().create(tableNames[1], ntc);
+    timeType = accumuloClient.tableOperations().getTimeType(tableNames[1]);
+    assertEquals(TimeType.LOGICAL, timeType);
+
+    // Create some split points
+    SortedSet<Text> splits = new TreeSet<>();
+    splits.add(new Text("F"));
+    splits.add(new Text("M"));
+    splits.add(new Text("S"));
+
+    // Create table with MILLIS TimeType. Use splits to create multiple tablets
+    ntc = new NewTableConfiguration();
+    ntc.withSplits(splits);
+    accumuloClient.tableOperations().create(tableNames[2], ntc);
+    timeType = accumuloClient.tableOperations().getTimeType(tableNames[2]);
+    assertEquals(TimeType.MILLIS, timeType);
+
+    // Create table with LOGICAL TimeType. Use splits to create multiple tablets
+    ntc = new NewTableConfiguration();
+    ntc.setTimeType(TimeType.LOGICAL).withSplits(splits);
+    accumuloClient.tableOperations().create(tableNames[3], ntc);
+    timeType = accumuloClient.tableOperations().getTimeType(tableNames[3]);
+    assertEquals(TimeType.LOGICAL, timeType);
+
+    // check system tables
+    timeType = accumuloClient.tableOperations().getTimeType("accumulo.metadata");
+    assertEquals(TimeType.LOGICAL, timeType);
+
+    timeType = accumuloClient.tableOperations().getTimeType("accumulo.replication");
+    assertEquals(TimeType.LOGICAL, timeType);
+
+    // test non-existent table
+    assertThrows(TableNotFoundException.class,
+        () -> accumuloClient.tableOperations().getTimeType("notatable"),
+        "specified table that doesn't exist");
+
+    // cannot get TimeType for root table
+    assertThrows(RuntimeException.class,

Review Comment:
   ```suggestion
       assertThrows(IllegalArgumentException.class,
   ```
   This could be narrowed down I think



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] jmark99 commented on a diff in pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
jmark99 commented on code in PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#discussion_r975633723


##########
shell/src/main/java/org/apache/accumulo/shell/commands/GetTimeTypeCommand.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.shell.commands;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.shell.Shell;
+import org.apache.accumulo.shell.Shell.Command;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+
+public class GetTimeTypeCommand extends Command {

Review Comment:
   Removed shell related timetype code, leaving API changes behind.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] jmark99 commented on a diff in pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
jmark99 commented on code in PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#discussion_r961075476


##########
test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java:
##########
@@ -292,4 +295,59 @@ public void testCompactEmptyTablesWithBadIterator_FailsAndCancel() throws TableE
     }
   }
 
+  @Test
+  public void getTimeTypeTest() throws TableNotFoundException, AccumuloException,
+      TableExistsException, AccumuloSecurityException {
+    String[] tableNames = getUniqueNames(4);
+
+    // Create table with default MILLIS TimeType
+    accumuloClient.tableOperations().create(tableNames[0]);
+    TimeType timeType = accumuloClient.tableOperations().getTimeType(tableNames[0]);
+    assertEquals(TimeType.MILLIS, timeType);
+
+    // Create table with LOGICAL TimeType.
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    ntc.setTimeType(TimeType.LOGICAL);
+    accumuloClient.tableOperations().create(tableNames[1], ntc);
+    timeType = accumuloClient.tableOperations().getTimeType(tableNames[1]);
+    assertEquals(TimeType.LOGICAL, timeType);
+
+    // Create some split points
+    SortedSet<Text> splits = new TreeSet<>();
+    splits.add(new Text("F"));
+    splits.add(new Text("M"));
+    splits.add(new Text("S"));
+
+    // Create table with MILLIS TimeType. Use splits to create multiple tablets
+    ntc = new NewTableConfiguration();
+    ntc.withSplits(splits);
+    accumuloClient.tableOperations().create(tableNames[2], ntc);
+    timeType = accumuloClient.tableOperations().getTimeType(tableNames[2]);
+    assertEquals(TimeType.MILLIS, timeType);
+
+    // Create table with LOGICAL TimeType. Use splits to create multiple tablets
+    ntc = new NewTableConfiguration();
+    ntc.setTimeType(TimeType.LOGICAL).withSplits(splits);
+    accumuloClient.tableOperations().create(tableNames[3], ntc);
+    timeType = accumuloClient.tableOperations().getTimeType(tableNames[3]);
+    assertEquals(TimeType.LOGICAL, timeType);
+
+    // check system tables
+    timeType = accumuloClient.tableOperations().getTimeType("accumulo.metadata");
+    assertEquals(TimeType.LOGICAL, timeType);
+
+    timeType = accumuloClient.tableOperations().getTimeType("accumulo.replication");
+    assertEquals(TimeType.LOGICAL, timeType);
+
+    // test non-existent table
+    assertThrows(TableNotFoundException.class,
+        () -> accumuloClient.tableOperations().getTimeType("notatable"),
+        "specified table that doesn't exist");
+
+    // cannot get TimeType for root table
+    assertThrows(RuntimeException.class,

Review Comment:
   With the changes suggested by Keith below this will no longer be necessary. All tables should return a valid TimeType and a runtime exception will be thrown if this is not the case.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] jmark99 commented on a diff in pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
jmark99 commented on code in PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#discussion_r961073453


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -2011,6 +2015,34 @@ public ImportDestinationArguments importDirectory(String directory) {
     return new BulkImport(directory, context);
   }
 
+  @Override
+  public TimeType getTimeType(final String tableName) throws TableNotFoundException {
+    if (tableName.equals(RootTable.NAME)) {
+      throw new IllegalArgumentException("accumulo.root table has no TimeType");
+    }
+    String systemTableToCheck =
+        MetadataTable.NAME.equals(tableName) ? RootTable.NAME : MetadataTable.NAME;
+    final Scanner scanner = context.createScanner(systemTableToCheck, Authorizations.EMPTY);

Review Comment:
   With the changes to be made with Keith's suggestion below this change will no longer be necessary.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] jmark99 commented on a diff in pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
jmark99 commented on code in PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#discussion_r961077160


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -2011,6 +2015,34 @@ public ImportDestinationArguments importDirectory(String directory) {
     return new BulkImport(directory, context);
   }
 
+  @Override
+  public TimeType getTimeType(final String tableName) throws TableNotFoundException {
+    if (tableName.equals(RootTable.NAME)) {
+      throw new IllegalArgumentException("accumulo.root table has no TimeType");
+    }
+    String systemTableToCheck =
+        MetadataTable.NAME.equals(tableName) ? RootTable.NAME : MetadataTable.NAME;
+    final Scanner scanner = context.createScanner(systemTableToCheck, Authorizations.EMPTY);
+    String tableId = tableIdMap().get(tableName);
+    if (tableId == null) {
+      throw new TableNotFoundException(null, tableName, "specified table does not exist");
+    }
+    final Text start = new Text(tableId);
+    final Text end = new Text(start);
+    start.append(new byte[] {'<'}, 0, 1);
+    end.append(new byte[] {'<'}, 0, 1);
+    scanner.setRange(new Range(start, end));
+    MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.fetch(scanner);
+    Entry<Key,Value> next = scanner.iterator().next();
+    Value val = next.getValue();
+    if (val.toString().startsWith("L")) {
+      return TimeType.LOGICAL;
+    } else if (val.toString().startsWith("M")) {
+      return TimeType.MILLIS;
+    }
+    throw new RuntimeException("Failed to retrieve TimeType");

Review Comment:
   @keith-turner thanks for the suggestion. I forgot about the use of ample. I made a few slight tweaks. 1) forTable requires a TableID vs a String, and 2) I updated to return a TimeType rather than a TableMetadata object.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] jmark99 commented on a diff in pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
jmark99 commented on code in PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#discussion_r961074258


##########
test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java:
##########
@@ -292,4 +295,59 @@ public void testCompactEmptyTablesWithBadIterator_FailsAndCancel() throws TableE
     }
   }
 
+  @Test
+  public void getTimeTypeTest() throws TableNotFoundException, AccumuloException,
+      TableExistsException, AccumuloSecurityException {
+    String[] tableNames = getUniqueNames(4);
+
+    // Create table with default MILLIS TimeType
+    accumuloClient.tableOperations().create(tableNames[0]);
+    TimeType timeType = accumuloClient.tableOperations().getTimeType(tableNames[0]);
+    assertEquals(TimeType.MILLIS, timeType);
+
+    // Create table with LOGICAL TimeType.
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    ntc.setTimeType(TimeType.LOGICAL);
+    accumuloClient.tableOperations().create(tableNames[1], ntc);
+    timeType = accumuloClient.tableOperations().getTimeType(tableNames[1]);
+    assertEquals(TimeType.LOGICAL, timeType);
+
+    // Create some split points
+    SortedSet<Text> splits = new TreeSet<>();
+    splits.add(new Text("F"));
+    splits.add(new Text("M"));
+    splits.add(new Text("S"));
+
+    // Create table with MILLIS TimeType. Use splits to create multiple tablets
+    ntc = new NewTableConfiguration();
+    ntc.withSplits(splits);
+    accumuloClient.tableOperations().create(tableNames[2], ntc);

Review Comment:
   @DomGarguilo I added a comment concerning the default TimeType at table creation and also created a separate case for explicitly setting the TimeType to MILLIS.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] jmark99 commented on a diff in pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
jmark99 commented on code in PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#discussion_r975633236


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -2011,6 +2013,16 @@ public ImportDestinationArguments importDirectory(String directory) {
     return new BulkImport(directory, context);
   }
 
+  @Override
+  public TimeType getTimeType(final String tableName) throws TableNotFoundException {
+    TableId tableId = context.getTableId(tableName);
+    Optional<TabletMetadata> tabletMetadata = context.getAmple().readTablets().forTable(tableId)
+        .fetch(TabletMetadata.ColumnType.TIME).checkConsistency().build().stream().findFirst();
+    TabletMetadata timeData =
+        tabletMetadata.orElseThrow(() -> new RuntimeException("Failed to retrieve TimeType"));

Review Comment:
   Updated to use IllegalStateException.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#discussion_r960898970


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -2011,6 +2015,34 @@ public ImportDestinationArguments importDirectory(String directory) {
     return new BulkImport(directory, context);
   }
 
+  @Override
+  public TimeType getTimeType(final String tableName) throws TableNotFoundException {
+    if (tableName.equals(RootTable.NAME)) {
+      throw new IllegalArgumentException("accumulo.root table has no TimeType");
+    }
+    String systemTableToCheck =
+        MetadataTable.NAME.equals(tableName) ? RootTable.NAME : MetadataTable.NAME;
+    final Scanner scanner = context.createScanner(systemTableToCheck, Authorizations.EMPTY);
+    String tableId = tableIdMap().get(tableName);
+    if (tableId == null) {
+      throw new TableNotFoundException(null, tableName, "specified table does not exist");
+    }
+    final Text start = new Text(tableId);
+    final Text end = new Text(start);
+    start.append(new byte[] {'<'}, 0, 1);
+    end.append(new byte[] {'<'}, 0, 1);
+    scanner.setRange(new Range(start, end));
+    MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.fetch(scanner);
+    Entry<Key,Value> next = scanner.iterator().next();
+    Value val = next.getValue();
+    if (val.toString().startsWith("L")) {
+      return TimeType.LOGICAL;
+    } else if (val.toString().startsWith("M")) {
+      return TimeType.MILLIS;
+    }
+    throw new RuntimeException("Failed to retrieve TimeType");

Review Comment:
   May be able to use ample for this.  Also may not need a special case for root tablet w/ ample, seems like it will return the time type logical.  But not sure, this is the [code that initializes the root tablet metadata](https://github.com/apache/accumulo/blob/114704bb58a37073a0401c3b6f479761089bd1fb/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java#L189-L206) and makes me think it would return logical.
   
   ```suggestion
        String tableId = tableIdMap().get(tableName);
       if (tableId == null) {
         throw new TableNotFoundException(null, tableName, "specified table does not exist");
       }
           Optional<TabletMetadata> tabletMetadata = context.getAmple().readTablets().forTable(tableId)
               .fetch(ColumnType.TIME).checkConsistency().build().stream().findFirst();
       return tabletMetadata.orElseThrow(() -> new RuntimeException(tabletMetadata.orElseThrow(() -> new RuntimeException())));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#discussion_r960898970


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -2011,6 +2015,34 @@ public ImportDestinationArguments importDirectory(String directory) {
     return new BulkImport(directory, context);
   }
 
+  @Override
+  public TimeType getTimeType(final String tableName) throws TableNotFoundException {
+    if (tableName.equals(RootTable.NAME)) {
+      throw new IllegalArgumentException("accumulo.root table has no TimeType");
+    }
+    String systemTableToCheck =
+        MetadataTable.NAME.equals(tableName) ? RootTable.NAME : MetadataTable.NAME;
+    final Scanner scanner = context.createScanner(systemTableToCheck, Authorizations.EMPTY);
+    String tableId = tableIdMap().get(tableName);
+    if (tableId == null) {
+      throw new TableNotFoundException(null, tableName, "specified table does not exist");
+    }
+    final Text start = new Text(tableId);
+    final Text end = new Text(start);
+    start.append(new byte[] {'<'}, 0, 1);
+    end.append(new byte[] {'<'}, 0, 1);
+    scanner.setRange(new Range(start, end));
+    MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.fetch(scanner);
+    Entry<Key,Value> next = scanner.iterator().next();
+    Value val = next.getValue();
+    if (val.toString().startsWith("L")) {
+      return TimeType.LOGICAL;
+    } else if (val.toString().startsWith("M")) {
+      return TimeType.MILLIS;
+    }
+    throw new RuntimeException("Failed to retrieve TimeType");

Review Comment:
   May be able to use ample for this.  Also may not need a special case for root tablet w/ ample, seems like it will return the time type logical.  But not sure, this is the [code that initializes the root tablet metadata](https://github.com/apache/accumulo/blob/114704bb58a37073a0401c3b6f479761089bd1fb/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java#L189-L206) and makes me think it would return logical.
   
   ```suggestion
        String tableId = tableIdMap().get(tableName);
       if (tableId == null) {
         throw new TableNotFoundException(null, tableName, "specified table does not exist");
       }
           Optional<TabletMetadata> tabletMetadata = context.getAmple().readTablets().forTable(tableId)
               .fetch(ColumnType.TIME).checkConsistency().build().stream().findFirst();
       if(tabletMetadata.isPresent()) {
         return tabletMetadata.get().getTime().getType();
       } else {
         throw new RuntimeException("Failed to retrieve TimeType"))
       }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] jmark99 commented on pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
jmark99 commented on PR #2910:
URL: https://github.com/apache/accumulo/pull/2910#issuecomment-1234413246

   Ticket replaces Jira issue https://issues.apache.org/jira/browse/ACCUMULO-3447


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii merged pull request #2910: Add ability to retrieve TimeType for a table

Posted by GitBox <gi...@apache.org>.
ctubbsii merged PR #2910:
URL: https://github.com/apache/accumulo/pull/2910


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org