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/08/29 18:22:49 UTC

[GitHub] [accumulo] keith-turner commented on a diff in pull request #2896: ScanServerSelector - Break nested interfaces into their own files

keith-turner commented on code in PR #2896:
URL: https://github.com/apache/accumulo/pull/2896#discussion_r957663554


##########
core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java:
##########
@@ -44,139 +37,23 @@ public interface ScanServerSelector {
   /**
    * The scan server group name that will be used when one is not specified.
    */
-  public static final String DEFAULT_SCAN_SERVER_GROUP_NAME = "default";
-
-  /**
-   * Information about a scan server.
-   *
-   * @since 2.1.0
-   */
-  public interface ScanServer {
-    /**
-     * @return the address in the form of {@code <host>:<port>} where the scan server is running.
-     */
-    String getAddress();
-
-    /**
-     * @return the group name set when the scan server was started. If a group name was not set for
-     *         the scan server, then the string {@value #DEFAULT_SCAN_SERVER_GROUP_NAME} is
-     *         returned.
-     */
-    String getGroup();
-  }
+  String DEFAULT_SCAN_SERVER_GROUP_NAME = "default";
 
   /**
-   * This interface exists so that is easier to evolve what is passed to
-   * {@link #init(InitParameters)} without having to make breaking changes.
-   *
-   * @since 2.1.0
+   * This method is called once after a {@link ScanServerSelector} is instantiated.
    */
-  public interface InitParameters {
-
-    /**
-     * @return Options that were set in the client config using the prefix
-     *         {@code scan.server.selector.opts.}. The prefix will be stripped. For example if
-     *         {@code scan.server.selector.opts.k1=v1} is set in client config, then the returned
-     *         map will contain {@code k1=v1}.
-     */
-    Map<String,String> getOptions();
-
-    ServiceEnvironment getServiceEnv();
-
-    /**
-     * @return the set of live ScanServers. Each time the supplier is called it may return something
-     *         different. A good practice would be to call this no more than once per a call to
-     *         {@link #determineActions(SelectorParameters)} so that decisions are made using a
-     *         consistent set of scan servers.
-     */
-    Supplier<Collection<ScanServer>> getScanServers();
-  }
-
-  /**
-   * This method is called once after a ScanSelector is instantiated.
-   */
-  default void init(InitParameters params) {
+  default void init(ScanServerSelectorInitParameters params) {
     Preconditions.checkArgument(params.getOptions().isEmpty(), "No options expected");
   }
 
   /**
-   * this object is used to communicate what the previous actions were attempted, when they were
-   * attempted, and the result of the attempt
-   *
-   * @since 2.1.0
-   */
-  interface ScanAttempt {
-
-    // represents reasons that previous attempts to scan failed
-    enum Result {
-      BUSY, ERROR
-    }
-
-    String getServer();
-
-    long getEndTime();
-
-    Result getResult();
-  }
-
-  /**
-   * This interface exists so that is easier to evolve what is passed to
-   * {@link #determineActions(SelectorParameters)} without having to make breaking changes.
-   *
-   * @since 2.1.0
-   */
-  public interface SelectorParameters {
-
-    /**
-     * @return the set of tablets to be scanned
-     */
-    Collection<TabletId> getTablets();
-
-    /**
-     * @return scan attempt information for the tablet
-     */
-    Collection<? extends ScanAttempt> getAttempts(TabletId tabletId);
-
-    /**
-     * @return any hints set on a scanner using
-     *         {@link org.apache.accumulo.core.client.ScannerBase#setExecutionHints(Map)}. If none
-     *         were set an empty map is returned.
-     */
-    Map<String,String> getHints();
-  }
-
-  public interface Actions {
-
-    /**
-     * @return what scan server to use for a given tablet. Returning null indicates the tablet
-     *         server should be used for this tablet.
-     */
-    String getScanServer(TabletId tabletId);
-
-    /**
-     * @return The amount of time to wait on the client side before starting to contact servers.
-     *         Return {@link Duration#ZERO} if no client side wait is desired.
-     */
-    public Duration getDelay();
-
-    /**
-     * @return The amount of time to wait for a scan to start on the server side before reporting
-     *         busy. For example if a scan request is sent to scan server with a busy timeout of
-     *         50ms and the scan has not started running within that time then the scan server will
-     *         not ever run the scan and it will report back busy. If the scan starts running, then
-     *         it will never report back busy. Setting a busy timeout that is &le; 0 means that it
-     *         will wait indefinitely on the server side for the task to start.
-     */
-    public Duration getBusyTimeout();
-  }
-
-  /**
-   * Uses the SelectorParameters to determine which, if any, ScanServer should be used for scanning
-   * a tablet.
+   * Uses the {@link ScanServerSelectorParameters} to determine which, if any, ScanServer should be
+   * used for scanning a tablet.
    *
    * @param params
    *          parameters for the calculation
    * @return results
    */
-  Actions determineActions(SelectorParameters params);
+  ScanServerSelectorActions determineActions(ScanServerSelectorParameters params);

Review Comment:
   In addition to the init parameters, I think it would a be nice to keep the parameters interface for this method nested.
   
   Unrelated to the PR, but looking at this after not having looked at it for a while I think this method needs a different name.  Maybe something like `selectServers` instead of `determineActions` and the return type could be named `ScanServerSelections`.
   
   



##########
core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java:
##########
@@ -44,139 +37,23 @@ public interface ScanServerSelector {
   /**
    * The scan server group name that will be used when one is not specified.
    */
-  public static final String DEFAULT_SCAN_SERVER_GROUP_NAME = "default";
-
-  /**
-   * Information about a scan server.
-   *
-   * @since 2.1.0
-   */
-  public interface ScanServer {
-    /**
-     * @return the address in the form of {@code <host>:<port>} where the scan server is running.
-     */
-    String getAddress();
-
-    /**
-     * @return the group name set when the scan server was started. If a group name was not set for
-     *         the scan server, then the string {@value #DEFAULT_SCAN_SERVER_GROUP_NAME} is
-     *         returned.
-     */
-    String getGroup();
-  }
+  String DEFAULT_SCAN_SERVER_GROUP_NAME = "default";
 
   /**
-   * This interface exists so that is easier to evolve what is passed to
-   * {@link #init(InitParameters)} without having to make breaking changes.
-   *
-   * @since 2.1.0
+   * This method is called once after a {@link ScanServerSelector} is instantiated.
    */
-  public interface InitParameters {
-
-    /**
-     * @return Options that were set in the client config using the prefix
-     *         {@code scan.server.selector.opts.}. The prefix will be stripped. For example if
-     *         {@code scan.server.selector.opts.k1=v1} is set in client config, then the returned
-     *         map will contain {@code k1=v1}.
-     */
-    Map<String,String> getOptions();
-
-    ServiceEnvironment getServiceEnv();
-
-    /**
-     * @return the set of live ScanServers. Each time the supplier is called it may return something
-     *         different. A good practice would be to call this no more than once per a call to
-     *         {@link #determineActions(SelectorParameters)} so that decisions are made using a
-     *         consistent set of scan servers.
-     */
-    Supplier<Collection<ScanServer>> getScanServers();
-  }
-
-  /**
-   * This method is called once after a ScanSelector is instantiated.
-   */
-  default void init(InitParameters params) {
+  default void init(ScanServerSelectorInitParameters params) {

Review Comment:
   I think its nice to keep InitParameters as a nested interface for the following reasons.
   
    * Its nice to have the method parameters in the file right above the only method that it goes with.  There is a 1:1 relationship between the method and the parameter interface, will not use the same interface for multiple methods.
    * The interface name can be shorter when its nested (`ScanServerSelectorInitParameters` vs `InitParameters`)
    * The pattern is consistent with other SPI interfaces that have parameter interfaces.



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