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/06/16 06:19:50 UTC

[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2780: New FateOperations API

ctubbsii commented on code in PR #2780:
URL: https://github.com/apache/accumulo/pull/2780#discussion_r898721597


##########
core/src/main/java/org/apache/accumulo/core/client/admin/FateOperations.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.client.admin;
+
+import java.util.List;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.AccumuloException;
+
+public interface FateOperations {
+
+  /**
+   * Fails a fate transaction based on the given txID. At least one txID must be provided.
+   *
+   * @param txids
+   *          Transaction IDs to fail.
+   * @since 2.1.0
+   */
+  void fateFail(Set<String> txids) throws AccumuloException;
+
+  /**
+   * Deletes a fate transaction based on the given txID. At least one txID must be provided.
+   *
+   * @param txids
+   *          Transaction IDs to delete.
+   * @since 2.1.0
+   */
+  void fateDelete(Set<String> txids) throws AccumuloException;
+
+  /**
+   * Gathers Transaction status information for either all fate transactions or requested txIDs.
+   *
+   * @param txids
+   *          Transaction IDs to use as a filter. Optional.
+   * @param tStatus
+   *          Parsed TStatus for print filter. Optional.
+   * @return A set of TransactionStatues for corresponding txids
+   * @since 2.1.0
+   */
+  List<FateTransactionStatus> fateStatus(Set<String> txids, List<String> tStatus)
+      throws AccumuloException;

Review Comment:
   If you had methods, `fail()` and `delete()` on `FateTransactionStatus`, this list (or set, really.. since the items should be unique) is all you'd really need, and you wouldn't need the separate `FateOperations` interface. It could also be a stream, so you can do things like:
   
   ```java
      client.instanceOperations().fateTransactions()
        .filter(tx -> tx.getCreatedTime().before(threeDaysAgo))
        .peek(FateTransaction::fail)
        .filter(tx -> tx.getCreatedTime().before(fiveDaysAgo))
        .forEach(FateTransaction::delete);
   ```
   
   Returning it as a set and calling `.stream()` on the set would work too. The main point is that `FateTransactionImpl` should have a reference to the client, so you can act on it with `fail()` and `delete()` operations.



##########
core/src/main/java/org/apache/accumulo/core/client/admin/FateTransactionStatus.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.client.admin;
+
+import java.util.List;
+
+/**
+ * FATE transaction status, including lock information.
+ */
+public interface FateTransactionStatus {
+
+  /**
+   * @return This fate operations transaction id, formatted in the same way as FATE transactions are
+   *         in the Accumulo logs.
+   */
+  String getTxid();
+
+  /**
+   * @return This fate operations transaction id, in its original long form.
+   */
+  long getTxidLong();

Review Comment:
   I think these might be a bit backwards. The transaction ID (what should be returned from a getter of the transaction ID) is a long. The String is merely an prettyPrinted encoding. The method that returns the transaction ID should just be called `getTxid`, and the method that returns the String should be the one that is called something else.
   
   Alternatively, we don't provide any String version, and just add a javadoc that points to Long.toHexString (I think that's what we're using). Or, we could just stop encoding the long in the logs, and just use the number in decimal in the logs.



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

Review Comment:
   Could just call this `FateTransaction`, since it is a representation of that. I'm not sure adding `Status` to the name adds much value, other than to disambiguate it from internal types that might represent the transaction. The internal types could be renamed, if necessary.



##########
core/src/main/java/org/apache/accumulo/core/client/admin/FateTransactionStatus.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.client.admin;
+
+import java.util.List;
+
+/**
+ * FATE transaction status, including lock information.
+ */
+public interface FateTransactionStatus {
+
+  /**
+   * @return This fate operations transaction id, formatted in the same way as FATE transactions are
+   *         in the Accumulo logs.
+   */
+  String getTxid();
+
+  /**
+   * @return This fate operations transaction id, in its original long form.
+   */
+  long getTxidLong();
+
+  /**
+   * @return The transaction's operation status code.
+   */
+  String getStatus();
+
+  /**
+   * @return The debug info for the operation on the top of the stack for this Fate operation.
+   */
+  String getDebug();
+
+  /**
+   * @return list of namespace and table ids locked
+   */
+  List<String> getHeldLocks();
+
+  /**
+   * @return list of namespace and table ids locked
+   */
+  List<String> getWaitingLocks();

Review Comment:
   Are these lists or sets?



##########
core/src/main/java/org/apache/accumulo/core/client/admin/FateTransactionStatus.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.client.admin;
+
+import java.util.List;
+
+/**
+ * FATE transaction status, including lock information.
+ */
+public interface FateTransactionStatus {
+
+  /**
+   * @return This fate operations transaction id, formatted in the same way as FATE transactions are
+   *         in the Accumulo logs.
+   */
+  String getTxid();
+
+  /**
+   * @return This fate operations transaction id, in its original long form.
+   */
+  long getTxidLong();
+
+  /**
+   * @return The transaction's operation status code.
+   */
+  String getStatus();

Review Comment:
   This line makes the current class name very confusing. This object is a status object, with a getStatus method. So, it's returning the status of a status. Also if this is a status code, it could be a type other than String.



##########
core/src/main/java/org/apache/accumulo/core/client/admin/FateTransactionStatus.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.client.admin;
+
+import java.util.List;
+
+/**
+ * FATE transaction status, including lock information.
+ */
+public interface FateTransactionStatus {
+
+  /**
+   * @return This fate operations transaction id, formatted in the same way as FATE transactions are
+   *         in the Accumulo logs.
+   */
+  String getTxid();
+
+  /**
+   * @return This fate operations transaction id, in its original long form.
+   */
+  long getTxidLong();
+
+  /**
+   * @return The transaction's operation status code.
+   */
+  String getStatus();
+
+  /**
+   * @return The debug info for the operation on the top of the stack for this Fate operation.
+   */
+  String getDebug();
+
+  /**
+   * @return list of namespace and table ids locked
+   */
+  List<String> getHeldLocks();
+
+  /**
+   * @return list of namespace and table ids locked
+   */
+  List<String> getWaitingLocks();
+
+  /**
+   * @return The operation on the top of the stack for this Fate operation.
+   */
+  String getTop();
+
+  /**
+   * @return The timestamp of when the operation was created in ISO format with UTC timezone.
+   */
+  String getTimeCreatedFormatted();
+
+  /**
+   * @return The unformatted form of the timestamp.
+   */
+  long getTimeCreated();

Review Comment:
   These should just return a `Date` object or similar. The consumer of this API can format it if they like, using the normal APIs for formatting dates. Only one method is needed to return the date type.



##########
core/src/main/java/org/apache/accumulo/core/client/admin/FateTransactionStatus.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.client.admin;
+
+import java.util.List;
+
+/**
+ * FATE transaction status, including lock information.
+ */
+public interface FateTransactionStatus {
+
+  /**
+   * @return This fate operations transaction id, formatted in the same way as FATE transactions are
+   *         in the Accumulo logs.
+   */
+  String getTxid();
+
+  /**
+   * @return This fate operations transaction id, in its original long form.
+   */
+  long getTxidLong();
+
+  /**
+   * @return The transaction's operation status code.
+   */
+  String getStatus();
+
+  /**
+   * @return The debug info for the operation on the top of the stack for this Fate operation.
+   */
+  String getDebug();
+
+  /**
+   * @return list of namespace and table ids locked
+   */
+  List<String> getHeldLocks();
+
+  /**
+   * @return list of namespace and table ids locked
+   */
+  List<String> getWaitingLocks();
+
+  /**
+   * @return The operation on the top of the stack for this Fate operation.
+   */
+  String getTop();

Review Comment:
   This method could be collapsed with `getStackInfo` into a method that returns an actual stack type (a List or Queue might work in Java), whose top could easily be viewed.



##########
core/src/main/java/org/apache/accumulo/core/client/admin/FateTransactionStatus.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.client.admin;
+
+import java.util.List;
+
+/**
+ * FATE transaction status, including lock information.
+ */
+public interface FateTransactionStatus {
+
+  /**
+   * @return This fate operations transaction id, formatted in the same way as FATE transactions are
+   *         in the Accumulo logs.
+   */
+  String getTxid();
+
+  /**
+   * @return This fate operations transaction id, in its original long form.
+   */
+  long getTxidLong();
+
+  /**
+   * @return The transaction's operation status code.
+   */
+  String getStatus();
+
+  /**
+   * @return The debug info for the operation on the top of the stack for this Fate operation.
+   */
+  String getDebug();

Review Comment:
   This method name is confusing, because it's not clear what a "debug" is as a noun to "get". Expanding it could help: `getDebugInfo` or `getDebuggingInformation`



-- 
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