You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by ilooner <gi...@git.apache.org> on 2017/08/26 00:51:56 UTC

[GitHub] drill pull request #923: Drill 5723

GitHub user ilooner opened a pull request:

    https://github.com/apache/drill/pull/923

    Drill 5723

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ilooner/drill DRILL-5723

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/drill/pull/923.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #923
    
----
commit 5c9cd42032df3fe357003046fff658fd4cabf5f3
Author: Jyothsna Donapati <jy...@skatkam-598.local>
Date:   2017-06-07T18:15:08Z

    Testing config linkage

commit fde6e569b72876e750164381168f3c850975dfd8
Author: dvjyothsna <jy...@skatkam-598.local>
Date:   2017-06-16T18:07:25Z

    Added config linkage

commit 9218ff0aef9f41a19d61c24c69b1187b55134afd
Author: dvjyothsna <jy...@skatkam-598.local>
Date:   2017-06-21T17:08:28Z

    Added new query result structure

commit 96ed451de8308fed764895287b0aa3f3590f83e8
Author: dvjyothsna <jy...@skatkam-598.local>
Date:   2017-06-21T17:36:06Z

    Added comments

commit 2dc4ce40385f2dbfcf3faefd36fba89888525b5b
Author: dvjyothsna <jy...@skatkam-598.local>
Date:   2017-06-26T17:47:08Z

    Added comments

commit 9fb3db77a087bf536b0a090640460b5f72e3593c
Author: dvjyothsna <jy...@skatkam-598.local>
Date:   2017-06-28T22:25:30Z

    modified extended option iterator

commit 394c03c3a8d3b8e737c48ebbace71549e8c95908
Author: dvjyothsna <jy...@skatkam-598.local>
Date:   2017-06-29T05:10:18Z

    Modified default to system

commit c9b2b0949382c6844f1256908ffbc83a56604c25
Author: dvjyothsna <jy...@skatkam-598.local>
Date:   2017-06-30T16:51:59Z

    modified test cases

commit fcdca4d04a093f63423231718603e7ed0713134d
Author: dvjyothsna <jy...@skatkam-598.local>
Date:   2017-06-30T17:27:28Z

    Modified test cases

commit 450c1fb35df45f541c6918357c1105ea3929b354
Author: dvjyothsna <jy...@skatkam-598.local>
Date:   2017-06-30T18:05:52Z

    removed old test

commit 82d12a45d899b9ee3750114f4c1ede99799d55f0
Author: Timothy Farkas <ti...@apache.org>
Date:   2017-08-17T23:49:51Z

     - Added option metadata

commit ae15e928b79e2d1c7747887e518d007af2ce0a46
Author: Timothy Farkas <ti...@apache.org>
Date:   2017-08-21T22:29:04Z

     - Added methods to the option managers to retrieve internal and external options
     - Made sys.options and sys.options2 only list external options, and added sys.internal_options and sys.internal_options2 to list internal options

commit 1b1ed1e362be2b3ff87d50fcb371c429204a410c
Author: Timothy Farkas <ti...@apache.org>
Date:   2017-08-22T23:10:19Z

     - Updated rest api to differentiate between internal and external options

commit e63920d2fd99b333d1e2352b9b03f3d0884e7fb0
Author: Timothy Farkas <ti...@apache.org>
Date:   2017-08-23T19:52:30Z

     - Made the option validators configurable

commit 818cc73d2c974497fafcb0b5c31b2dc67b6a35b1
Author: Timothy Farkas <ti...@apache.org>
Date:   2017-08-24T18:23:09Z

     - Added rest client fixture for testing.
     - Added rest api unit tests for options.

commit 44f1f5e912dd21c407a940903dc3c1a3f89db889
Author: Timothy Farkas <ti...@apache.org>
Date:   2017-08-26T00:49:59Z

     - Added javadoc

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138152185
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java ---
    @@ -346,17 +347,63 @@ public void deleteAllOptions(OptionType type) {
         }
       }
     
    -  public void populateDefaultValues() {
    -
    +  public static CaseInsensitiveMap<OptionValue> populateDefualtValues(Map<String, OptionDefinition> definitions, DrillConfig bootConfig) {
    --- End diff --
    
    Done


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138205470
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java ---
    @@ -56,7 +56,7 @@ public void checkChangedColumn() throws Exception {
         test("ALTER session SET `%s` = %d;", SLICE_TARGET,
           ExecConstants.SLICE_TARGET_DEFAULT);
         testBuilder()
    -        .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND type = 'SESSION'", SLICE_TARGET)
    +        .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND optionScope = 'SESSION'", SLICE_TARGET)
    --- End diff --
    
    Taking a note of what we discussed offline. 
    
    Course of action is to just change the name for **type** since it is a bad name, and the semantics of **type** have been ill-defined. So no one could have relied on the values returned to them in the past anyway. I am changing the name to **accessibleScopes**.


---

[GitHub] drill issue #923: DRILL-5723: Added System Internal Options That can be Modi...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/923
  
    @paul-rogers Finished applying comments, and cleanup. It's ready for review again now.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137122952
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java ---
    @@ -83,18 +86,20 @@
       @VisibleForTesting
       public Drillbit(
           final DrillConfig config,
    +      final CaseInsensitiveMap<OptionValidator> validators,
           final RemoteServiceSet serviceSet) throws Exception {
    -    this(config, serviceSet, ClassPathScanner.fromPrescan(config));
    +    this(config, validators, serviceSet, ClassPathScanner.fromPrescan(config));
    --- End diff --
    
    Currently there are no internal validators defined, so in order to test the logic for internal validators I had to create a set of mock validators which included internal validators, and use those for testing. I have changed this so that the Drillbit constructors which take a set of validators are annotated to be visible for testing.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937333
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java ---
    @@ -17,49 +17,97 @@
      */
     package org.apache.drill.exec.server.options;
     
    -import org.apache.drill.exec.server.options.OptionValue.OptionType;
    +import javax.validation.constraints.NotNull;
     
     /**
      * Manager for Drill {@link OptionValue options}. Implementations must be case-insensitive to the name of an option.
      */
     public interface OptionManager extends OptionSet, Iterable<OptionValue> {
     
       /**
    -   * Sets an option value.
    -   *
    -   * @param value option value
    -   * @throws org.apache.drill.common.exceptions.UserException message to describe error with value
    +   * Sets a boolean option on the {@link OptionManager}.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
        */
    -  void setOption(OptionValue value);
    +  void setLocalOption(String name, boolean value);
     
       /**
    -   * Deletes the option. Unfortunately, the type is required given the fallback structure of option managers.
    -   * See {@link FallbackOptionManager}.
    +   * Sets a long option on the {@link OptionManager}.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
    +   */
    +  void setLocalOption(String name, long value);
    +
    +  /**
    +   * Sets a double option on the {@link OptionManager}.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
    +   */
    +  void setLocalOption(String name, double value);
    +
    +  /**
    +   * Sets a String option on the {@link OptionManager}.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
    +   */
    +  void setLocalOption(String name, String value);
    +
    +  /**
    +   * Sets an option of the specified {@link OptionValue.Kind} on the {@link OptionManager}.
    +   * @param kind The kind of the option.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
    +   */
    +  void setLocalOption(OptionValue.Kind kind, String name, String value);
    +
    +  /**
    +   * Deletes the option.
        *
    -   * If the option name is valid (exists in {@link SystemOptionManager#VALIDATORS}),
    +   * If the option name is valid (exists in the set of validators produced by {@link SystemOptionManager#createDefaultOptionDefinitions()}),
        * but the option was not set within this manager, calling this method should be a no-op.
        *
        * @param name option name
    -   * @param type option type
        * @throws org.apache.drill.common.exceptions.UserException message to describe error with value
        */
    -  void deleteOption(String name, OptionType type);
    +  void deleteLocalOption(String name);
     
       /**
    -   * Deletes all options. Unfortunately, the type is required given the fallback structure of option managers.
    -   * See {@link FallbackOptionManager}.
    +   * Deletes all options.
        *
        * If no options are set, calling this method should be no-op.
        *
    -   * @param type option type
        * @throws org.apache.drill.common.exceptions.UserException message to describe error with value
        */
    -  void deleteAllOptions(OptionType type);
    +  void deleteAllLocalOptions();
    +
    +  /**
    +   * Get the option definition corresponding to the given option name.
    +   * @param name The name of the option to retrieve a validator for.
    +   * @return The option validator corresponding to the given option name.
    +   */
    +  @NotNull
    +  OptionDefinition getOptionDefinition(String name);
     
       /**
        * Gets the list of options managed this manager.
        *
        * @return the list of options
        */
       OptionList getOptionList();
    +
    +  /**
    +   * Returns all the internal options contained in this option manager.
    +   *
    +   * @return All the internal options contained in this option manager.
    +   */
    +  @NotNull
    +  OptionList getInternalOptionList();
    --- End diff --
    
    `Internal` --> `Local` to be consistent with the other methods?


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937945
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java ---
    @@ -150,66 +150,61 @@ public void pauseOnSpecificBit() {
         final ZookeeperHelper zkHelper = new ZookeeperHelper();
         zkHelper.startZookeeper(1);
     
    -    // Creating two drillbits
    -    final Drillbit drillbit1, drillbit2;
    -    final DrillConfig drillConfig = zkHelper.getConfig();
         try {
    -      drillbit1 = Drillbit.start(drillConfig, remoteServiceSet);
    -      drillbit2 = Drillbit.start(drillConfig, remoteServiceSet);
    -    } catch (final DrillbitStartupException e) {
    -      throw new RuntimeException("Failed to start two drillbits.", e);
    -    }
    -
    -    final DrillbitContext drillbitContext1 = drillbit1.getContext();
    -    final DrillbitContext drillbitContext2 = drillbit2.getContext();
    -
    -    final UserSession session = UserSession.Builder.newBuilder()
    -      .withCredentials(UserCredentials.newBuilder()
    -        .setUserName("foo")
    -        .build())
    -      .withUserProperties(UserProperties.getDefaultInstance())
    -      .withOptionManager(drillbitContext1.getOptionManager())
    -      .build();
    -
    -    final DrillbitEndpoint drillbitEndpoint1 = drillbitContext1.getEndpoint();
    -    final String controls = Controls.newBuilder()
    -      .addPauseOnBit(DummyClass.class, DummyClass.PAUSES, drillbitEndpoint1)
    -      .build();
    -
    -    ControlsInjectionUtil.setControls(session, controls);
    -
    -    {
    -      final long expectedDuration = 1000L;
    -      final ExtendedLatch trigger = new ExtendedLatch(1);
    -      final Pointer<Exception> ex = new Pointer<>();
    -      final QueryContext queryContext = new QueryContext(session, drillbitContext1, QueryId.getDefaultInstance());
    -      (new ResumingThread(queryContext, trigger, ex, expectedDuration)).start();
    -
    -      // test that the pause happens
    -      final DummyClass dummyClass = new DummyClass(queryContext, trigger);
    -      final long actualDuration = dummyClass.pauses();
    -      assertTrue(String.format("Test should stop for at least %d milliseconds.", expectedDuration),
    -        expectedDuration <= actualDuration);
    -      assertTrue("No exception should be thrown.", ex.value == null);
    +      // Creating two drillbits
    +      final Drillbit drillbit1, drillbit2;
    +      final DrillConfig drillConfig = zkHelper.getConfig();
           try {
    -        queryContext.close();
    -      } catch (final Exception e) {
    -        fail("Failed to close query context: " + e);
    +        drillbit1 = Drillbit.start(drillConfig, remoteServiceSet);
    +        drillbit2 = Drillbit.start(drillConfig, remoteServiceSet);
    +      } catch (final DrillbitStartupException e) {
    +        throw new RuntimeException("Failed to start two drillbits.", e);
           }
    -    }
     
    -    {
    -      final ExtendedLatch trigger = new ExtendedLatch(1);
    -      final QueryContext queryContext = new QueryContext(session, drillbitContext2, QueryId.getDefaultInstance());
    +      final DrillbitContext drillbitContext1 = drillbit1.getContext();
    +      final DrillbitContext drillbitContext2 = drillbit2.getContext();
    +
    +      final UserSession session = UserSession.Builder.newBuilder().withCredentials(UserCredentials.newBuilder().setUserName("foo").build()).withUserProperties(UserProperties.getDefaultInstance()).withOptionManager(drillbitContext1.getOptionManager()).build();
    --- End diff --
    
    Same issue of one big line vs. multi-line.


---

[GitHub] drill issue #923: DRILL-5723: Added System Internal Options That can be Modi...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/923
  
    Thanks @paul-rogers squashed and ready to commit. Please merge this change before my other PR https://github.com/apache/drill/pull/940


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136661867
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/StatusResourcesTest.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.exec.server.rest;
    +
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.server.options.OptionValidator;
    +import org.apache.drill.exec.server.options.TypeValidators;
    +import org.apache.drill.test.ClientFixture;
    +import org.apache.drill.test.ClusterFixture;
    +import org.apache.drill.test.FixtureBuilder;
    +import org.apache.drill.test.RestClientFixture;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import static org.apache.drill.test.TestConfigLinkage.MOCK_PROPERTY;
    +import static org.apache.drill.test.TestConfigLinkage.createMockPropValidator;
    +
    +public class StatusResourcesTest {
    +  @Test
    +  public void testRetrieveInternalOption() throws Exception {
    +    TypeValidators.StringValidator stringValidator = createMockPropValidator();
    +
    +    FixtureBuilder builder = ClusterFixture.builder().
    +      configProperty(ExecConstants.HTTP_ENABLE, true).
    +      configProperty(OptionValidator.OPTION_DEFAULTS_ROOT + MOCK_PROPERTY, "a").
    +      putValidator(MOCK_PROPERTY, stringValidator);
    +
    +    try (ClusterFixture cluster = builder.build();
    +         ClientFixture client = cluster.clientFixture()) {
    +      RestClientFixture restClientFixture = cluster.restClientFixture();
    +
    +      Assert.assertNull(restClientFixture.getStatusOption(MOCK_PROPERTY));
    +      StatusResources.OptionWrapper option = restClientFixture.getStatusInternalOption(MOCK_PROPERTY);
    +      Assert.assertEquals("a", option.getValueAsString());
    +
    +      client.alterSystem(MOCK_PROPERTY, "c");
    --- End diff --
    
    This test shows the value in allowing "extended" options defined outside of the usual big table in `SystemOptionManager`.
    
    In fact, we can take a step back and say that having a global table is always a bad idea: it introduces tight coupling. Tight coupling makes it much harder to do unit testing. For an open source tool such as Drill, which should allow extensions, tight coupling is counter-productive to the goal of extensibility.
    
    So, I wonder, can we add a mechanism that allows other modules (and tests) to add options? Just brainstorming a bit...
    
    * For code modules, provide a config setting that names a class that builds options. The Drillbit, during initialization, scans for these classes (the way it scans for, say, storage plugins), instantiates the class, and calls a method to define the options.
    * The easiest way to define options is to add an `OptionBuilder` class, created by the Drillbit in the constructor, that is available up to some point in time, such as once the Drillbit starts running.
    
    The API might be:
    
    ```
    Drillbit bit = new Drillbit();
    bit.optionBuilder().add(new OptionDefinition("my.option", ...));
    bit.run();
    ```
    
    The `add()` method can be called from tests (perhaps with added `ClusterFixture` support) and from the module-specific option builder.
    
    Two issues in this model:
    
    * Is there any reason to add options after the Drillbit starts? Doing so adds complexity.
    * Do we have code in Drillbit startup that uses the option manager? If so, some careful sequencing would be needed to call this code only *after* the options are built.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136659109
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -421,13 +422,13 @@
        * such as changing system options.
        */
       String ADMIN_USERS_KEY = "security.admin.users";
    -  StringValidator ADMIN_USERS_VALIDATOR = new StringValidator(ADMIN_USERS_KEY, true);
    +  StringValidator ADMIN_USERS_VALIDATOR = new StringValidator(ADMIN_USERS_KEY, new OptionValue.MetaData(true, false));
    --- End diff --
    
    We are adding a distinction between runtime options that the user can set vs. those that the user cannot set? This is a very bizarre distinction as runtime options are only for the user...
    
    Actually, these are options that are "public" vs. those that exist, but are generally "hidden", right?
    
    Another very useful addition, while we're in the code, is between the allowed "scope" of a setting. Today, all options can be set at the system and session (per user) level. But some options, such as the memory per query, or enabling queues, really only make sense at the system level. So, part of the new metadata should be to specify "session-only" options and forbid setting such options at the session level.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937112
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java ---
    @@ -17,44 +17,84 @@
      */
     package org.apache.drill.exec.server.options;
     
    -import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
    -
    -public abstract class BaseOptionManager implements OptionSet {
    -//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
    -
    -  /**
    -   * Gets the current option value given a validator.
    -   *
    -   * @param validator the validator
    -   * @return option value
    -   * @throws IllegalArgumentException - if the validator is not found
    -   */
    -  private OptionValue getOptionSafe(OptionValidator validator)  {
    -    OptionValue value = getOption(validator.getOptionName());
    -    return value == null ? validator.getDefault() : value;
    +import org.apache.drill.common.exceptions.UserException;
    +
    +import java.util.Iterator;
    +
    +/**
    + * This {@link OptionManager} implements some the basic methods and should be extended by concrete implementations.
    + */
    +public abstract class BaseOptionManager extends BaseOptionSet implements OptionManager {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
    +
    +  @Override
    +  public OptionList getInternalOptionList() {
    +    return getAllOptionList(true);
       }
     
       @Override
    -  public boolean getOption(BooleanValidator validator) {
    -    return getOptionSafe(validator).bool_val;
    +  public OptionList getPublicOptionList() {
    +    return getAllOptionList(false);
       }
     
       @Override
    -  public double getOption(DoubleValidator validator) {
    -    return getOptionSafe(validator).float_val;
    +  public void setLocalOption(String name, boolean value) {
    +    setLocalOption(OptionValue.Kind.BOOLEAN, name, Boolean.toString(value));
       }
     
       @Override
    -  public long getOption(LongValidator validator) {
    -    return getOptionSafe(validator).num_val;
    +  public void setLocalOption(String name, long value) {
    +    setLocalOption(OptionValue.Kind.LONG, name, Long.toString(value));
       }
     
       @Override
    -  public String getOption(StringValidator validator) {
    -    return getOptionSafe(validator).string_val;
    +  public void setLocalOption(String name, double value) {
    +    setLocalOption(OptionValue.Kind.DOUBLE, name, Double.toString(value));
       }
     
    +  @Override
    +  public void setLocalOption(String name, String value) {
    +    setLocalOption(OptionValue.Kind.STRING, name, value);
    +  }
    +
    +  @Override
    +  public void setLocalOption(OptionValue.Kind kind, String name, String value) {
    +    final OptionDefinition definition = getOptionDefinition(name);
    --- End diff --
    
    Presumably the `getOptionDefinition()` method checks that the option is defined and throws an exception if not?


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136669290
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java ---
    @@ -70,11 +69,13 @@
      *  </p>
      */
     
    -public class SystemOptionManager extends BaseOptionManager implements OptionManager, AutoCloseable {
    +public class SystemOptionManager extends BaseOptionManager implements AutoCloseable {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SystemOptionManager.class);
     
    -  private CaseInsensitiveMap<OptionValidator> VALIDATORS;
    -  public void populateValidators() {
    +  public static final CaseInsensitiveMap<OptionValidator> DEFAULT_VALIDATORS =
    --- End diff --
    
    Agreed we should not be using global variables. The intended flow was the following:
    
    - DEFAULT_VALIDATORS holds the default set of validators
    - The system option manager is passed a set of validators when it is initialized
    - The validators map is copied and stored in the validators variable on line 230. So each option manager has a separate copy of the validator map.
    
    Looking at this again I agree that this is not ideal. The DEFAULT_VALIDATORS constant should be deleted and usages should be replaced with createDefaultValidators.
    
    I also see another issue that was present before where the validators themselves are static, they should be explicitly copied, so we should probably add a clone method to the validators to do that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138694085
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java ---
    @@ -56,7 +56,7 @@ public void checkChangedColumn() throws Exception {
         test("ALTER session SET `%s` = %d;", SLICE_TARGET,
           ExecConstants.SLICE_TARGET_DEFAULT);
         testBuilder()
    -        .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND type = 'SESSION'", SLICE_TARGET)
    +        .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND optionScope = 'SESSION'", SLICE_TARGET)
    --- End diff --
    
    Sounds good. Thanks.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136670944
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/StatusResourcesTest.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.exec.server.rest;
    +
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.server.options.OptionValidator;
    +import org.apache.drill.exec.server.options.TypeValidators;
    +import org.apache.drill.test.ClientFixture;
    +import org.apache.drill.test.ClusterFixture;
    +import org.apache.drill.test.FixtureBuilder;
    +import org.apache.drill.test.RestClientFixture;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import static org.apache.drill.test.TestConfigLinkage.MOCK_PROPERTY;
    +import static org.apache.drill.test.TestConfigLinkage.createMockPropValidator;
    +
    +public class StatusResourcesTest {
    +  @Test
    +  public void testRetrieveInternalOption() throws Exception {
    +    TypeValidators.StringValidator stringValidator = createMockPropValidator();
    +
    +    FixtureBuilder builder = ClusterFixture.builder().
    +      configProperty(ExecConstants.HTTP_ENABLE, true).
    +      configProperty(OptionValidator.OPTION_DEFAULTS_ROOT + MOCK_PROPERTY, "a").
    +      putValidator(MOCK_PROPERTY, stringValidator);
    +
    +    try (ClusterFixture cluster = builder.build();
    +         ClientFixture client = cluster.clientFixture()) {
    +      RestClientFixture restClientFixture = cluster.restClientFixture();
    +
    +      Assert.assertNull(restClientFixture.getStatusOption(MOCK_PROPERTY));
    +      StatusResources.OptionWrapper option = restClientFixture.getStatusInternalOption(MOCK_PROPERTY);
    +      Assert.assertEquals("a", option.getValueAsString());
    +
    +      client.alterSystem(MOCK_PROPERTY, "c");
    --- End diff --
    
    I added a method that allows the addition of option validators at startup, and those validators are passed to the SystemOption manager. I think this could be enhanced using the OptionBuilder mechanism you described.
    
    - I don't think it is useful to add options after starting a Drillbit, but we could apply the OptionBuilder to add options at boot time.
    - That should be possible. It was possible to add option validators to the SystemOptionManager at boot time.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136662888
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/test/RestClientFixture.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.test;
    +
    +import com.google.common.base.Preconditions;
    +import io.restassured.RestAssured;
    +import io.restassured.http.ContentType;
    +import org.apache.drill.exec.server.rest.StatusResources;
    +
    +import javax.annotation.Nullable;
    +
    +import static io.restassured.RestAssured.when;
    +
    +/**
    + * Represents a client for the Drill Rest API.
    + */
    +public class RestClientFixture {
    --- End diff --
    
    Very nice addition!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937991
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/test/RestClientFixture.java ---
    @@ -0,0 +1,117 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.test;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.server.rest.StatusResources;
    +import org.glassfish.jersey.client.ClientConfig;
    +import org.glassfish.jersey.client.JerseyClientBuilder;
    +
    +import javax.annotation.Nullable;
    +import javax.ws.rs.client.Client;
    +import javax.ws.rs.client.WebTarget;
    +import javax.ws.rs.core.GenericType;
    +import javax.ws.rs.core.MediaType;
    +
    +import java.util.List;
    +
    +/**
    + * Represents a client for the Drill Rest API.
    + */
    +public class RestClientFixture implements AutoCloseable {
    +  /**
    +   * A builder for the rest client.
    +   */
    +  public static class Builder {
    +    private ClusterFixture cluster;
    +
    +    public Builder(ClusterFixture cluster) {
    +      this.cluster = Preconditions.checkNotNull(cluster);
    +    }
    +
    +    public RestClientFixture build() {
    +      return new RestClientFixture(cluster);
    +    }
    +  }
    +
    +  private WebTarget baseTarget;
    +  private Client client;
    --- End diff --
    
    `final` here and above?


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937210
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/DrillConfigIterator.java ---
    @@ -58,17 +58,17 @@ public OptionValue next() {
           OptionValue optionValue = null;
           switch(cv.valueType()) {
           case BOOLEAN:
    -        optionValue = OptionValue.createBoolean(OptionType.BOOT, name, (Boolean) cv.unwrapped(), OptionScope.BOOT);
    +        optionValue = OptionValue.create(OptionType.BOOT, name, (Boolean) cv.unwrapped(), OptionScope.BOOT);
    --- End diff --
    
    I suppose this has worked, but it is a bit of a muddle. Config settings are not runtime options. It is a hack to map them into the same structures. Since this iterator is probably only used to create a table, at some point we probably should create a new object to handle config options. We might want: 1) the full name, 2) the source (system options, command line, drill-override.conf, etc.), 3) the TypeSafe type (which is a superset of the runtime option types.
    
    OK to leave this for now, but we should consider fixing this in the future.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138173050
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java ---
    @@ -17,44 +17,84 @@
      */
     package org.apache.drill.exec.server.options;
     
    -import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
    -
    -public abstract class BaseOptionManager implements OptionSet {
    -//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
    -
    -  /**
    -   * Gets the current option value given a validator.
    -   *
    -   * @param validator the validator
    -   * @return option value
    -   * @throws IllegalArgumentException - if the validator is not found
    -   */
    -  private OptionValue getOptionSafe(OptionValidator validator)  {
    -    OptionValue value = getOption(validator.getOptionName());
    -    return value == null ? validator.getDefault() : value;
    +import org.apache.drill.common.exceptions.UserException;
    +
    +import java.util.Iterator;
    +
    +/**
    + * This {@link OptionManager} implements some the basic methods and should be extended by concrete implementations.
    + */
    +public abstract class BaseOptionManager extends BaseOptionSet implements OptionManager {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
    +
    +  @Override
    +  public OptionList getInternalOptionList() {
    +    return getAllOptionList(true);
       }
     
       @Override
    -  public boolean getOption(BooleanValidator validator) {
    -    return getOptionSafe(validator).bool_val;
    +  public OptionList getPublicOptionList() {
    +    return getAllOptionList(false);
       }
     
       @Override
    -  public double getOption(DoubleValidator validator) {
    -    return getOptionSafe(validator).float_val;
    +  public void setLocalOption(String name, boolean value) {
    +    setLocalOption(OptionValue.Kind.BOOLEAN, name, Boolean.toString(value));
       }
     
       @Override
    -  public long getOption(LongValidator validator) {
    -    return getOptionSafe(validator).num_val;
    +  public void setLocalOption(String name, long value) {
    +    setLocalOption(OptionValue.Kind.LONG, name, Long.toString(value));
       }
     
       @Override
    -  public String getOption(StringValidator validator) {
    -    return getOptionSafe(validator).string_val;
    +  public void setLocalOption(String name, double value) {
    +    setLocalOption(OptionValue.Kind.DOUBLE, name, Double.toString(value));
       }
     
    +  @Override
    +  public void setLocalOption(String name, String value) {
    +    setLocalOption(OptionValue.Kind.STRING, name, value);
    +  }
    +
    +  @Override
    +  public void setLocalOption(OptionValue.Kind kind, String name, String value) {
    +    final OptionDefinition definition = getOptionDefinition(name);
    --- End diff --
    
    Yeah it does. I've updated the javadoc to make this explicit.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138168189
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionMetaData.java ---
    @@ -0,0 +1,68 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.exec.server.options;
    +
    +/**
    + * Contains information about the scopes in which an option can be set, and an option's visibility.
    + */
    +public class OptionMetaData {
    +  public static final OptionMetaData DEFAULT = new OptionMetaData(OptionValue.OptionType.ALL, false, false);
    +
    +  private OptionValue.OptionType type;
    --- End diff --
    
    Done


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138148176
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java ---
    @@ -102,10 +103,11 @@ public void testFix2967() throws Exception {
           test("select * from dfs_test.`%s/join/j1` j1 left outer join dfs_test.`%s/join/j2` j2 on (j1.c_varchar = j2.c_varchar)",
             TEST_RES_PATH, TEST_RES_PATH);
         } finally {
    -      setSessionOption(PlannerSettings.BROADCAST.getOptionName(), String.valueOf(PlannerSettings.BROADCAST.getDefault().bool_val));
    -      setSessionOption(PlannerSettings.HASHJOIN.getOptionName(), String.valueOf(PlannerSettings.HASHJOIN.getDefault().bool_val));
    +      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
    +      setSessionOption(PlannerSettings.BROADCAST.getOptionName(), String.valueOf(testOptionSet.getDefault(PlannerSettings.BROADCAST.getOptionName()).bool_val));
    --- End diff --
    
    The test changed some of the options for the session. Since the same drillbit cluster is used for multiple tests, the changes need to be undone at the end of the test to avoid impacting other tests, which reuse the drill bit cluster. I basically kept things as they were before, but looking at it again I agree this is a pretty messy way of doing it. I think this could be cleaned up by doing:
    
    ```
    ALTER SESSION RESET ALL;
    ```
    
    To clear all the changes at the end of the test. Instead of passing an option value I'll also add methods for each type: 
    
    ```
    setSessionOption(final String option, final boolean value)
    setSessionOption(final String option, final long value)
    setSessionOption(final String option, final double value)
    setSessionOption(final String option, final String value)
    ```


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138171404
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/DrillConfigIterator.java ---
    @@ -58,17 +58,17 @@ public OptionValue next() {
           OptionValue optionValue = null;
           switch(cv.valueType()) {
           case BOOLEAN:
    -        optionValue = OptionValue.createBoolean(OptionType.BOOT, name, (Boolean) cv.unwrapped(), OptionScope.BOOT);
    +        optionValue = OptionValue.create(OptionType.BOOT, name, (Boolean) cv.unwrapped(), OptionScope.BOOT);
    --- End diff --
    
    Agreed


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937347
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionMetaData.java ---
    @@ -0,0 +1,68 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.exec.server.options;
    +
    +/**
    + * Contains information about the scopes in which an option can be set, and an option's visibility.
    + */
    +public class OptionMetaData {
    +  public static final OptionMetaData DEFAULT = new OptionMetaData(OptionValue.OptionType.ALL, false, false);
    +
    +  private OptionValue.OptionType type;
    +  private boolean adminOption;
    --- End diff --
    
    `adminOnly` (we know it is an option...)


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/drill/pull/923


---

[GitHub] drill issue #923: DRILL-5723: Added System Internal Options That can be Modi...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/923
  
    So it looks like Drill's treatment of OptionType and OptionScope is inconsistent at present. To correct this I am doing the following:
    
    - OptionType seems to be I'll defined. I think the intention is that it defines the levels in which an option can be configured. This should be a static property tied to an option (much like a validator). In reality it looks like the OptionType of an option can be changed depending on where it is set, which is incorrect behavior. I think this came about because it is the user's responsibility to define the OptionType for an option value which is in correct.
    - OptionScope will define the level at which the option was set
    
    To solidify the semantics I am doing the following:
    
    - OptionManagers no longer allow an option value to be set. Only the following methods are exposed: ```
    void setLocalOption(String name, boolean value);
    void setLocalOption(String name, long value);
    void setLocalOption(String name, double value);
    void setLocalOption(String name, String value);
    void setLocalOption(OptionValue.Kind kind, String name, String value);
    ``` Doing this prevents the possibility of defining an incorrect scope or type. The option type is now determined by the static option definition. The option scope is determined by the option manager on which the option was set.
    
    - I have also cleaned up the OptionManager interface so that it does not require the option type to be passed to the delete methods. This helps clean up the semantics further because OptionType was being overloaded to define both where an option can be set and which option manager can set it. This is unclear because some options can be set exclusively at one level (e.g. only the session level) whereas other options can be set at multiple levels (e.g. system and session levels). It also introduces the possibility for error by allowing the user to provide the wrong type to the wrong option manager.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136663308
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/test/RestClientFixture.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.test;
    +
    +import com.google.common.base.Preconditions;
    +import io.restassured.RestAssured;
    +import io.restassured.http.ContentType;
    +import org.apache.drill.exec.server.rest.StatusResources;
    +
    +import javax.annotation.Nullable;
    +
    +import static io.restassured.RestAssured.when;
    +
    +/**
    + * Represents a client for the Drill Rest API.
    + */
    +public class RestClientFixture {
    +  public static final String REST_API_PORT = "drill.exec.http.port";
    +
    +  /**
    +   * A builder for the rest client.
    +   */
    +  public static class Builder {
    +    private ClusterFixture cluster;
    +
    +    public Builder(ClusterFixture cluster) {
    +      this.cluster = Preconditions.checkNotNull(cluster);
    +    }
    +
    +    public RestClientFixture build() {
    +      return new RestClientFixture(cluster);
    +    }
    +  }
    +
    +
    +  private RestClientFixture(ClusterFixture cluster) {
    +    int port = cluster.config.getInt(REST_API_PORT);
    +    String address = cluster.drillbits().iterator().next().getContext().getEndpoint().getAddress();
    +
    +    RestAssured.port = port;
    +    RestAssured.basePath = "/";
    +    RestAssured.baseURI = "http://" + address;
    --- End diff --
    
    Are these global variables? How will this work in Surefire, when multiple tests run concurrently, if someone uses this mechanism to, say, test HTTPS support in one test while this runs in another test?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136662766
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java ---
    @@ -90,13 +93,13 @@
       public static class OperatorFixtureBuilder
       {
         ConfigBuilder configBuilder = new ConfigBuilder();
    -    TestOptionSet options = new TestOptionSet();
    +    TestOptionManager options = new TestOptionManager();
    --- End diff --
    
    This is a bit tricky. Some background.
    
    Most of our existing "unit" (really, system) tests start a Drillbit in order to test anything.
    
    We are trying (slowly!) to move to true unit tests. To do so, we need to introduce lose coupling between modules. The full option managers have dependencies on the Drillbit which sucks in all of Drill.
    
    The goal of the `OptionSet` is to provide a read-only view of the options since accessing options is pretty simple; it does not have dependencies on the Drillbit.
    
    We need to preserve this view to allow unit testing. We can do that either by ensuring that the option managers are stand-alone (have no runtime dependencies on the Drillbit or the DrillbitContext), or by continuing to do what this code does: create a test-time, read-only mock version that implements the option reader (`OptionSet`) interface.
    
    Either are valid approaches; which do you think would work best?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138168825
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java ---
    @@ -17,49 +17,97 @@
      */
     package org.apache.drill.exec.server.options;
     
    -import org.apache.drill.exec.server.options.OptionValue.OptionType;
    +import javax.validation.constraints.NotNull;
     
     /**
      * Manager for Drill {@link OptionValue options}. Implementations must be case-insensitive to the name of an option.
      */
     public interface OptionManager extends OptionSet, Iterable<OptionValue> {
     
       /**
    -   * Sets an option value.
    -   *
    -   * @param value option value
    -   * @throws org.apache.drill.common.exceptions.UserException message to describe error with value
    +   * Sets a boolean option on the {@link OptionManager}.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
        */
    -  void setOption(OptionValue value);
    +  void setLocalOption(String name, boolean value);
     
       /**
    -   * Deletes the option. Unfortunately, the type is required given the fallback structure of option managers.
    -   * See {@link FallbackOptionManager}.
    +   * Sets a long option on the {@link OptionManager}.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
    +   */
    +  void setLocalOption(String name, long value);
    +
    +  /**
    +   * Sets a double option on the {@link OptionManager}.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
    +   */
    +  void setLocalOption(String name, double value);
    +
    +  /**
    +   * Sets a String option on the {@link OptionManager}.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
    +   */
    +  void setLocalOption(String name, String value);
    +
    +  /**
    +   * Sets an option of the specified {@link OptionValue.Kind} on the {@link OptionManager}.
    +   * @param kind The kind of the option.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
    +   */
    +  void setLocalOption(OptionValue.Kind kind, String name, String value);
    +
    +  /**
    +   * Deletes the option.
        *
    -   * If the option name is valid (exists in {@link SystemOptionManager#VALIDATORS}),
    +   * If the option name is valid (exists in the set of validators produced by {@link SystemOptionManager#createDefaultOptionDefinitions()}),
        * but the option was not set within this manager, calling this method should be a no-op.
        *
        * @param name option name
    -   * @param type option type
        * @throws org.apache.drill.common.exceptions.UserException message to describe error with value
        */
    -  void deleteOption(String name, OptionType type);
    +  void deleteLocalOption(String name);
     
       /**
    -   * Deletes all options. Unfortunately, the type is required given the fallback structure of option managers.
    -   * See {@link FallbackOptionManager}.
    +   * Deletes all options.
        *
        * If no options are set, calling this method should be no-op.
        *
    -   * @param type option type
        * @throws org.apache.drill.common.exceptions.UserException message to describe error with value
        */
    -  void deleteAllOptions(OptionType type);
    +  void deleteAllLocalOptions();
    +
    +  /**
    +   * Get the option definition corresponding to the given option name.
    +   * @param name The name of the option to retrieve a validator for.
    +   * @return The option validator corresponding to the given option name.
    +   */
    +  @NotNull
    +  OptionDefinition getOptionDefinition(String name);
     
       /**
        * Gets the list of options managed this manager.
        *
        * @return the list of options
        */
       OptionList getOptionList();
    +
    +  /**
    +   * Returns all the internal options contained in this option manager.
    +   *
    +   * @return All the internal options contained in this option manager.
    +   */
    +  @NotNull
    +  OptionList getInternalOptionList();
    --- End diff --
    
    Internal and Local are not the same thing. Local means that the value for the option is stored in this option manager. Internal means that it is not visible with the rest of the options unless you look in the special sys.internal_options table. I will add this distinction to the javadoc.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138152163
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java ---
    @@ -102,10 +103,11 @@ public void testFix2967() throws Exception {
           test("select * from dfs_test.`%s/join/j1` j1 left outer join dfs_test.`%s/join/j2` j2 on (j1.c_varchar = j2.c_varchar)",
             TEST_RES_PATH, TEST_RES_PATH);
         } finally {
    -      setSessionOption(PlannerSettings.BROADCAST.getOptionName(), String.valueOf(PlannerSettings.BROADCAST.getDefault().bool_val));
    -      setSessionOption(PlannerSettings.HASHJOIN.getOptionName(), String.valueOf(PlannerSettings.HASHJOIN.getDefault().bool_val));
    +      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
    +      setSessionOption(PlannerSettings.BROADCAST.getOptionName(), String.valueOf(testOptionSet.getDefault(PlannerSettings.BROADCAST.getOptionName()).bool_val));
    --- End diff --
    
    Done


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937777
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java ---
    @@ -346,17 +347,63 @@ public void deleteAllOptions(OptionType type) {
         }
       }
     
    -  public void populateDefaultValues() {
    -
    +  public static CaseInsensitiveMap<OptionValue> populateDefualtValues(Map<String, OptionDefinition> definitions, DrillConfig bootConfig) {
    --- End diff --
    
    `populateDefualtValues` --> `populateDefaultValues` (that is "ua" --> "au").


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937392
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValue.java ---
    @@ -63,32 +88,32 @@
       public final Double float_val;
       public final OptionScope scope;
     
    -  public static OptionValue createLong(OptionType type, String name, long val, OptionScope scope) {
    +  public static OptionValue create(OptionType type, String name, long val, OptionScope scope) {
         return new OptionValue(Kind.LONG, type, name, val, null, null, null, scope);
       }
     
    -  public static OptionValue createBoolean(OptionType type, String name, boolean bool, OptionScope scope) {
    +  public static OptionValue create(OptionType type, String name, boolean bool, OptionScope scope) {
         return new OptionValue(Kind.BOOLEAN, type, name, null, null, bool, null, scope);
       }
     
    -  public static OptionValue createString(OptionType type, String name, String val, OptionScope scope) {
    +  public static OptionValue create(OptionType type, String name, String val, OptionScope scope) {
         return new OptionValue(Kind.STRING, type, name, null, val, null, null, scope);
       }
     
    -  public static OptionValue createDouble(OptionType type, String name, double val, OptionScope scope) {
    +  public static OptionValue create(OptionType type, String name, double val, OptionScope scope) {
         return new OptionValue(Kind.DOUBLE, type, name, null, null, null, val, scope);
       }
     
    -  public static OptionValue createOption(Kind kind, OptionType type, String name, String val, OptionScope scope) {
    +  public static OptionValue create(Kind kind, OptionType type, String name, String val, OptionScope scope) {
    --- End diff --
    
    Per earlier comment, `String val` --> `Object val`, with casts rather than string conversions below?


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937100
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java ---
    @@ -17,44 +17,84 @@
      */
     package org.apache.drill.exec.server.options;
     
    -import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
    -
    -public abstract class BaseOptionManager implements OptionSet {
    -//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
    -
    -  /**
    -   * Gets the current option value given a validator.
    -   *
    -   * @param validator the validator
    -   * @return option value
    -   * @throws IllegalArgumentException - if the validator is not found
    -   */
    -  private OptionValue getOptionSafe(OptionValidator validator)  {
    -    OptionValue value = getOption(validator.getOptionName());
    -    return value == null ? validator.getDefault() : value;
    +import org.apache.drill.common.exceptions.UserException;
    +
    +import java.util.Iterator;
    +
    +/**
    + * This {@link OptionManager} implements some the basic methods and should be extended by concrete implementations.
    + */
    +public abstract class BaseOptionManager extends BaseOptionSet implements OptionManager {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
    +
    +  @Override
    +  public OptionList getInternalOptionList() {
    +    return getAllOptionList(true);
       }
     
       @Override
    -  public boolean getOption(BooleanValidator validator) {
    -    return getOptionSafe(validator).bool_val;
    +  public OptionList getPublicOptionList() {
    +    return getAllOptionList(false);
       }
     
       @Override
    -  public double getOption(DoubleValidator validator) {
    -    return getOptionSafe(validator).float_val;
    +  public void setLocalOption(String name, boolean value) {
    +    setLocalOption(OptionValue.Kind.BOOLEAN, name, Boolean.toString(value));
    --- End diff --
    
    Very nice improvement to the API! I wonder, however, if we should pass the value as an `Object` rather than as a `String`? Seems more natural to do it that way. Or, maybe here just create the option value since we know the type?
    
    ```
    setLocalOption(String name, OptionValue.fromBoolean(value));
    ```
    
    This means that the other fields would be filled in later, so they couldn't be final.
    
    Hence, the alternative, use an `Object`
    
    ```
    setLocalOption(String name, value);
    ...
    
    private void setLocalOption(String name, Object value) {
    ```
    
    Then, when creating an option, validate that the type of the `Object` matches the type of the option. (In fact, the `Object` for would be handy for testing...)



---

[GitHub] drill issue #923: DRILL-5723: Added System Internal Options That can be Modi...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/923
  
    @paul-rogers Finished apply review comments. PR is ready for review again.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136663098
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/test/RestClientFixture.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.test;
    +
    +import com.google.common.base.Preconditions;
    +import io.restassured.RestAssured;
    +import io.restassured.http.ContentType;
    +import org.apache.drill.exec.server.rest.StatusResources;
    +
    +import javax.annotation.Nullable;
    +
    +import static io.restassured.RestAssured.when;
    +
    +/**
    + * Represents a client for the Drill Rest API.
    + */
    +public class RestClientFixture {
    +  public static final String REST_API_PORT = "drill.exec.http.port";
    --- End diff --
    
    Isn't there already an `ExecConstants` value for this option? See `ExecConstants.HTTP_PORT`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138133824
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitWithExchanges.java ---
    @@ -71,7 +72,8 @@ public void testPushLimitPastUnionExchange() throws Exception {
           final String[] expectedPlan5 = {"(?s)Limit\\(fetch=\\[1\\].*UnionExchange.*Limit\\(fetch=\\[1\\]\\).*Join"};
           testLimitHelper(sql5, expectedPlan5, excludedPlan, 1);
         } finally {
    -      test("alter session set `planner.slice_target` = " + ExecConstants.SLICE_TARGET_OPTION.getDefault().getValue());
    +      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
    +      test("alter session set `planner.slice_target` = " + testOptionSet.getDefault(ExecConstants.SLICE_TARGET).getValue());
    --- End diff --
    
    done


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937125
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java ---
    @@ -17,44 +17,84 @@
      */
     package org.apache.drill.exec.server.options;
     
    -import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
    -
    -public abstract class BaseOptionManager implements OptionSet {
    -//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
    -
    -  /**
    -   * Gets the current option value given a validator.
    -   *
    -   * @param validator the validator
    -   * @return option value
    -   * @throws IllegalArgumentException - if the validator is not found
    -   */
    -  private OptionValue getOptionSafe(OptionValidator validator)  {
    -    OptionValue value = getOption(validator.getOptionName());
    -    return value == null ? validator.getDefault() : value;
    +import org.apache.drill.common.exceptions.UserException;
    +
    +import java.util.Iterator;
    +
    +/**
    + * This {@link OptionManager} implements some the basic methods and should be extended by concrete implementations.
    + */
    +public abstract class BaseOptionManager extends BaseOptionSet implements OptionManager {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
    +
    +  @Override
    +  public OptionList getInternalOptionList() {
    +    return getAllOptionList(true);
       }
     
       @Override
    -  public boolean getOption(BooleanValidator validator) {
    -    return getOptionSafe(validator).bool_val;
    +  public OptionList getPublicOptionList() {
    +    return getAllOptionList(false);
       }
     
       @Override
    -  public double getOption(DoubleValidator validator) {
    -    return getOptionSafe(validator).float_val;
    +  public void setLocalOption(String name, boolean value) {
    +    setLocalOption(OptionValue.Kind.BOOLEAN, name, Boolean.toString(value));
       }
     
       @Override
    -  public long getOption(LongValidator validator) {
    -    return getOptionSafe(validator).num_val;
    +  public void setLocalOption(String name, long value) {
    +    setLocalOption(OptionValue.Kind.LONG, name, Long.toString(value));
       }
     
       @Override
    -  public String getOption(StringValidator validator) {
    -    return getOptionSafe(validator).string_val;
    +  public void setLocalOption(String name, double value) {
    +    setLocalOption(OptionValue.Kind.DOUBLE, name, Double.toString(value));
       }
     
    +  @Override
    +  public void setLocalOption(String name, String value) {
    +    setLocalOption(OptionValue.Kind.STRING, name, value);
    +  }
    +
    +  @Override
    +  public void setLocalOption(OptionValue.Kind kind, String name, String value) {
    +    final OptionDefinition definition = getOptionDefinition(name);
    +    final OptionValidator validator = definition.getValidator();
    +    final OptionMetaData metaData = definition.getMetaData();
    +    final OptionValue.OptionType type = definition.getMetaData().getType();
    +    final OptionValue.OptionScope scope = getScope();
    +    checkOptionPermissions(name, type, scope);
    +    final OptionValue optionValue = OptionValue.create(kind, type, name, value, scope);
    +    validator.validate(optionValue, metaData, this);
    +    setLocalOptionHelper(optionValue);
    --- End diff --
    
    Very nice and clean, does all the necessary steps in one place. Great improvement!


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138694479
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/StatusResourcesTest.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.exec.server.rest;
    +
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.server.options.OptionValidator;
    +import org.apache.drill.exec.server.options.TypeValidators;
    +import org.apache.drill.test.ClientFixture;
    +import org.apache.drill.test.ClusterFixture;
    +import org.apache.drill.test.FixtureBuilder;
    +import org.apache.drill.test.RestClientFixture;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import static org.apache.drill.test.TestConfigLinkage.MOCK_PROPERTY;
    +import static org.apache.drill.test.TestConfigLinkage.createMockPropValidator;
    +
    +public class StatusResourcesTest {
    +  @Test
    +  public void testRetrieveInternalOption() throws Exception {
    +    TypeValidators.StringValidator stringValidator = createMockPropValidator();
    +
    +    FixtureBuilder builder = ClusterFixture.builder().
    +      configProperty(ExecConstants.HTTP_ENABLE, true).
    +      configProperty(OptionValidator.OPTION_DEFAULTS_ROOT + MOCK_PROPERTY, "a").
    +      putValidator(MOCK_PROPERTY, stringValidator);
    +
    +    try (ClusterFixture cluster = builder.build();
    +         ClientFixture client = cluster.clientFixture()) {
    +      RestClientFixture restClientFixture = cluster.restClientFixture();
    +
    +      Assert.assertNull(restClientFixture.getStatusOption(MOCK_PROPERTY));
    +      StatusResources.OptionWrapper option = restClientFixture.getStatusInternalOption(MOCK_PROPERTY);
    +      Assert.assertEquals("a", option.getValueAsString());
    +
    +      client.alterSystem(MOCK_PROPERTY, "c");
    --- End diff --
    
    Thanks.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937864
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitWithExchanges.java ---
    @@ -71,7 +72,8 @@ public void testPushLimitPastUnionExchange() throws Exception {
           final String[] expectedPlan5 = {"(?s)Limit\\(fetch=\\[1\\].*UnionExchange.*Limit\\(fetch=\\[1\\]\\).*Join"};
           testLimitHelper(sql5, expectedPlan5, excludedPlan, 1);
         } finally {
    -      test("alter session set `planner.slice_target` = " + ExecConstants.SLICE_TARGET_OPTION.getDefault().getValue());
    +      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
    +      test("alter session set `planner.slice_target` = " + testOptionSet.getDefault(ExecConstants.SLICE_TARGET).getValue());
    --- End diff --
    
    Here, seems it would be better to refer to the constant defining the option name rather than hard-coding it. (Yes, this is existing code, so OK to leave as is...)


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137136257
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/test/RestClientFixture.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.test;
    +
    +import com.google.common.base.Preconditions;
    +import io.restassured.RestAssured;
    +import io.restassured.http.ContentType;
    +import org.apache.drill.exec.server.rest.StatusResources;
    +
    +import javax.annotation.Nullable;
    +
    +import static io.restassured.RestAssured.when;
    +
    +/**
    + * Represents a client for the Drill Rest API.
    + */
    +public class RestClientFixture {
    +  public static final String REST_API_PORT = "drill.exec.http.port";
    +
    +  /**
    +   * A builder for the rest client.
    +   */
    +  public static class Builder {
    +    private ClusterFixture cluster;
    +
    +    public Builder(ClusterFixture cluster) {
    +      this.cluster = Preconditions.checkNotNull(cluster);
    +    }
    +
    +    public RestClientFixture build() {
    +      return new RestClientFixture(cluster);
    +    }
    +  }
    +
    +
    +  private RestClientFixture(ClusterFixture cluster) {
    +    int port = cluster.config.getInt(REST_API_PORT);
    +    String address = cluster.drillbits().iterator().next().getContext().getEndpoint().getAddress();
    +
    +    RestAssured.port = port;
    +    RestAssured.basePath = "/";
    +    RestAssured.baseURI = "http://" + address;
    --- End diff --
    
    I reverted to using the jersey client so global variables aren't used anymore.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138689935
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java ---
    @@ -17,49 +17,97 @@
      */
     package org.apache.drill.exec.server.options;
     
    -import org.apache.drill.exec.server.options.OptionValue.OptionType;
    +import javax.validation.constraints.NotNull;
     
     /**
      * Manager for Drill {@link OptionValue options}. Implementations must be case-insensitive to the name of an option.
      */
     public interface OptionManager extends OptionSet, Iterable<OptionValue> {
     
       /**
    -   * Sets an option value.
    -   *
    -   * @param value option value
    -   * @throws org.apache.drill.common.exceptions.UserException message to describe error with value
    +   * Sets a boolean option on the {@link OptionManager}.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
        */
    -  void setOption(OptionValue value);
    +  void setLocalOption(String name, boolean value);
     
       /**
    -   * Deletes the option. Unfortunately, the type is required given the fallback structure of option managers.
    -   * See {@link FallbackOptionManager}.
    +   * Sets a long option on the {@link OptionManager}.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
    +   */
    +  void setLocalOption(String name, long value);
    +
    +  /**
    +   * Sets a double option on the {@link OptionManager}.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
    +   */
    +  void setLocalOption(String name, double value);
    +
    +  /**
    +   * Sets a String option on the {@link OptionManager}.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
    +   */
    +  void setLocalOption(String name, String value);
    +
    +  /**
    +   * Sets an option of the specified {@link OptionValue.Kind} on the {@link OptionManager}.
    +   * @param kind The kind of the option.
    +   * @param name The name of the option.
    +   * @param value The value of the option.
    +   */
    +  void setLocalOption(OptionValue.Kind kind, String name, String value);
    +
    +  /**
    +   * Deletes the option.
        *
    -   * If the option name is valid (exists in {@link SystemOptionManager#VALIDATORS}),
    +   * If the option name is valid (exists in the set of validators produced by {@link SystemOptionManager#createDefaultOptionDefinitions()}),
        * but the option was not set within this manager, calling this method should be a no-op.
        *
        * @param name option name
    -   * @param type option type
        * @throws org.apache.drill.common.exceptions.UserException message to describe error with value
        */
    -  void deleteOption(String name, OptionType type);
    +  void deleteLocalOption(String name);
     
       /**
    -   * Deletes all options. Unfortunately, the type is required given the fallback structure of option managers.
    -   * See {@link FallbackOptionManager}.
    +   * Deletes all options.
        *
        * If no options are set, calling this method should be no-op.
        *
    -   * @param type option type
        * @throws org.apache.drill.common.exceptions.UserException message to describe error with value
        */
    -  void deleteAllOptions(OptionType type);
    +  void deleteAllLocalOptions();
    +
    +  /**
    +   * Get the option definition corresponding to the given option name.
    +   * @param name The name of the option to retrieve a validator for.
    +   * @return The option validator corresponding to the given option name.
    +   */
    +  @NotNull
    +  OptionDefinition getOptionDefinition(String name);
     
       /**
        * Gets the list of options managed this manager.
        *
        * @return the list of options
        */
       OptionList getOptionList();
    +
    +  /**
    +   * Returns all the internal options contained in this option manager.
    +   *
    +   * @return All the internal options contained in this option manager.
    +   */
    +  @NotNull
    +  OptionList getInternalOptionList();
    --- End diff --
    
    Thanks for the explanation.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937350
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionMetaData.java ---
    @@ -0,0 +1,68 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.exec.server.options;
    +
    +/**
    + * Contains information about the scopes in which an option can be set, and an option's visibility.
    + */
    +public class OptionMetaData {
    +  public static final OptionMetaData DEFAULT = new OptionMetaData(OptionValue.OptionType.ALL, false, false);
    +
    +  private OptionValue.OptionType type;
    --- End diff --
    
    `final` here and others?


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136669723
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java ---
    @@ -70,11 +69,13 @@
      *  </p>
      */
     
    -public class SystemOptionManager extends BaseOptionManager implements OptionManager, AutoCloseable {
    +public class SystemOptionManager extends BaseOptionManager implements AutoCloseable {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SystemOptionManager.class);
     
    -  private CaseInsensitiveMap<OptionValidator> VALIDATORS;
    -  public void populateValidators() {
    +  public static final CaseInsensitiveMap<OptionValidator> DEFAULT_VALIDATORS =
    --- End diff --
    
    I don't completely follow this. Let's talk about this offline next week so that I can understand :). I can post the conclusion of the discussion on the PR for the community then.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136660399
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java ---
    @@ -83,18 +86,20 @@
       @VisibleForTesting
       public Drillbit(
           final DrillConfig config,
    +      final CaseInsensitiveMap<OptionValidator> validators,
           final RemoteServiceSet serviceSet) throws Exception {
    -    this(config, serviceSet, ClassPathScanner.fromPrescan(config));
    +    this(config, validators, serviceSet, ClassPathScanner.fromPrescan(config));
    --- End diff --
    
    Interesting... Why was this change needed?
    
    To be honest, I'm not convinced that the set of inputs is an *input* to the Drillbit rather than an internal *attribute* of the Drillbit. That is, the Drillbit should be responsible for defining its set of valid options, not the code that houses the Drillbit.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138132972
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/test/RestClientFixture.java ---
    @@ -0,0 +1,117 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.test;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.server.rest.StatusResources;
    +import org.glassfish.jersey.client.ClientConfig;
    +import org.glassfish.jersey.client.JerseyClientBuilder;
    +
    +import javax.annotation.Nullable;
    +import javax.ws.rs.client.Client;
    +import javax.ws.rs.client.WebTarget;
    +import javax.ws.rs.core.GenericType;
    +import javax.ws.rs.core.MediaType;
    +
    +import java.util.List;
    +
    +/**
    + * Represents a client for the Drill Rest API.
    + */
    +public class RestClientFixture implements AutoCloseable {
    +  /**
    +   * A builder for the rest client.
    +   */
    +  public static class Builder {
    +    private ClusterFixture cluster;
    +
    +    public Builder(ClusterFixture cluster) {
    +      this.cluster = Preconditions.checkNotNull(cluster);
    +    }
    +
    +    public RestClientFixture build() {
    +      return new RestClientFixture(cluster);
    +    }
    +  }
    +
    +  private WebTarget baseTarget;
    +  private Client client;
    --- End diff --
    
    done


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937938
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java ---
    @@ -216,79 +216,77 @@ public void injectionOnSpecificBit() {
         final ZookeeperHelper zkHelper = new ZookeeperHelper();
         zkHelper.startZookeeper(1);
     
    -    // Creating two drillbits
    -    final Drillbit drillbit1, drillbit2;
    -    final DrillConfig drillConfig = zkHelper.getConfig();
         try {
    -      drillbit1 = Drillbit.start(drillConfig, remoteServiceSet);
    -      drillbit2 = Drillbit.start(drillConfig, remoteServiceSet);
    -    } catch (DrillbitStartupException e) {
    -      throw new RuntimeException("Failed to start drillbits.", e);
    -    }
    +      // Creating two drillbits
    +      final Drillbit drillbit1, drillbit2;
    +      final DrillConfig drillConfig = zkHelper.getConfig();
    +      try {
    +        drillbit1 = Drillbit.start(drillConfig, remoteServiceSet);
    +        drillbit2 = Drillbit.start(drillConfig, remoteServiceSet);
    +      } catch (DrillbitStartupException e) {
    +        throw new RuntimeException("Failed to start drillbits.", e);
    +      }
     
    -    final DrillbitContext drillbitContext1 = drillbit1.getContext();
    -    final DrillbitContext drillbitContext2 = drillbit2.getContext();
    +      final DrillbitContext drillbitContext1 = drillbit1.getContext();
    +      final DrillbitContext drillbitContext2 = drillbit2.getContext();
     
    -    final UserSession session = UserSession.Builder.newBuilder()
    -        .withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build())
    -        .withUserProperties(UserProperties.getDefaultInstance())
    -        .withOptionManager(drillbitContext1.getOptionManager())
    -        .build();
    +      final UserSession session = UserSession.Builder.newBuilder().withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build()).withUserProperties(UserProperties.getDefaultInstance()).withOptionManager(drillbitContext1.getOptionManager()).build();
    --- End diff --
    
    Did we want to combine all the method calls onto one line rather than the fluent, multi-line style originally in the code? An artifact of a code formatter?


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138133008
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java ---
    @@ -150,66 +150,61 @@ public void pauseOnSpecificBit() {
         final ZookeeperHelper zkHelper = new ZookeeperHelper();
         zkHelper.startZookeeper(1);
     
    -    // Creating two drillbits
    -    final Drillbit drillbit1, drillbit2;
    -    final DrillConfig drillConfig = zkHelper.getConfig();
         try {
    -      drillbit1 = Drillbit.start(drillConfig, remoteServiceSet);
    -      drillbit2 = Drillbit.start(drillConfig, remoteServiceSet);
    -    } catch (final DrillbitStartupException e) {
    -      throw new RuntimeException("Failed to start two drillbits.", e);
    -    }
    -
    -    final DrillbitContext drillbitContext1 = drillbit1.getContext();
    -    final DrillbitContext drillbitContext2 = drillbit2.getContext();
    -
    -    final UserSession session = UserSession.Builder.newBuilder()
    -      .withCredentials(UserCredentials.newBuilder()
    -        .setUserName("foo")
    -        .build())
    -      .withUserProperties(UserProperties.getDefaultInstance())
    -      .withOptionManager(drillbitContext1.getOptionManager())
    -      .build();
    -
    -    final DrillbitEndpoint drillbitEndpoint1 = drillbitContext1.getEndpoint();
    -    final String controls = Controls.newBuilder()
    -      .addPauseOnBit(DummyClass.class, DummyClass.PAUSES, drillbitEndpoint1)
    -      .build();
    -
    -    ControlsInjectionUtil.setControls(session, controls);
    -
    -    {
    -      final long expectedDuration = 1000L;
    -      final ExtendedLatch trigger = new ExtendedLatch(1);
    -      final Pointer<Exception> ex = new Pointer<>();
    -      final QueryContext queryContext = new QueryContext(session, drillbitContext1, QueryId.getDefaultInstance());
    -      (new ResumingThread(queryContext, trigger, ex, expectedDuration)).start();
    -
    -      // test that the pause happens
    -      final DummyClass dummyClass = new DummyClass(queryContext, trigger);
    -      final long actualDuration = dummyClass.pauses();
    -      assertTrue(String.format("Test should stop for at least %d milliseconds.", expectedDuration),
    -        expectedDuration <= actualDuration);
    -      assertTrue("No exception should be thrown.", ex.value == null);
    +      // Creating two drillbits
    +      final Drillbit drillbit1, drillbit2;
    +      final DrillConfig drillConfig = zkHelper.getConfig();
           try {
    -        queryContext.close();
    -      } catch (final Exception e) {
    -        fail("Failed to close query context: " + e);
    +        drillbit1 = Drillbit.start(drillConfig, remoteServiceSet);
    +        drillbit2 = Drillbit.start(drillConfig, remoteServiceSet);
    +      } catch (final DrillbitStartupException e) {
    +        throw new RuntimeException("Failed to start two drillbits.", e);
           }
    -    }
     
    -    {
    -      final ExtendedLatch trigger = new ExtendedLatch(1);
    -      final QueryContext queryContext = new QueryContext(session, drillbitContext2, QueryId.getDefaultInstance());
    +      final DrillbitContext drillbitContext1 = drillbit1.getContext();
    +      final DrillbitContext drillbitContext2 = drillbit2.getContext();
    +
    +      final UserSession session = UserSession.Builder.newBuilder().withCredentials(UserCredentials.newBuilder().setUserName("foo").build()).withUserProperties(UserProperties.getDefaultInstance()).withOptionManager(drillbitContext1.getOptionManager()).build();
    --- End diff --
    
    done


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138167788
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValue.java ---
    @@ -63,32 +88,32 @@
       public final Double float_val;
       public final OptionScope scope;
     
    -  public static OptionValue createLong(OptionType type, String name, long val, OptionScope scope) {
    +  public static OptionValue create(OptionType type, String name, long val, OptionScope scope) {
         return new OptionValue(Kind.LONG, type, name, val, null, null, null, scope);
       }
     
    -  public static OptionValue createBoolean(OptionType type, String name, boolean bool, OptionScope scope) {
    +  public static OptionValue create(OptionType type, String name, boolean bool, OptionScope scope) {
         return new OptionValue(Kind.BOOLEAN, type, name, null, null, bool, null, scope);
       }
     
    -  public static OptionValue createString(OptionType type, String name, String val, OptionScope scope) {
    +  public static OptionValue create(OptionType type, String name, String val, OptionScope scope) {
         return new OptionValue(Kind.STRING, type, name, null, val, null, null, scope);
       }
     
    -  public static OptionValue createDouble(OptionType type, String name, double val, OptionScope scope) {
    +  public static OptionValue create(OptionType type, String name, double val, OptionScope scope) {
         return new OptionValue(Kind.DOUBLE, type, name, null, null, null, val, scope);
       }
     
    -  public static OptionValue createOption(Kind kind, OptionType type, String name, String val, OptionScope scope) {
    +  public static OptionValue create(Kind kind, OptionType type, String name, String val, OptionScope scope) {
    --- End diff --
    
    Done


---

[GitHub] drill issue #923: DRILL-5723: Added System Internal Options That can be Modi...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/923
  
    @paul-rogers Applied your comments and cleaned up the PR, it should be ready for review now. If additional work needs to be done to make this easier to review please let me know.
    
    Thanks,
    Tim


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #923: DRILL-5723

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on the issue:

    https://github.com/apache/drill/pull/923
  
    Please add a brief description in the PR title.
    
    This PR has commits from other PRs, and has merge conflicts. If this PR depends on others, then please identify the commits to be reviewed, and which commits will be removed once this one is rebased onto master with those other commits.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #923: DRILL-5723: Added System Internal Options That can be Modi...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/923
  
    Thanks @paul-rogers I'll work on addressing the comments and talk to you next week about some questions I have.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937977
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java ---
    @@ -196,9 +200,12 @@ public void tablesWithSystemTableFilter() throws Exception {
         verifyTable("sys", "boot", tables);
         verifyTable("sys", "drillbits", tables);
         verifyTable("sys", "memory", tables);
    -    verifyTable("sys", "options", tables);
    +    verifyTable("sys", SystemTable.OPTION.getTableName(), tables);
    +    verifyTable("sys", SystemTable.OPTION_VAL.getTableName(), tables);
         verifyTable("sys", "threads", tables);
         verifyTable("sys", "version", tables);
    +    verifyTable("sys", SystemTable.INTERNAL_OPTIONS.getTableName(), tables);
    +    verifyTable("sys", SystemTable.INTERNAL_OPTIONS_VAL.getTableName(), tables);
    --- End diff --
    
    Thanks for providing the constants. Nice improvement.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138182296
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java ---
    @@ -17,44 +17,84 @@
      */
     package org.apache.drill.exec.server.options;
     
    -import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
    -import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
    -
    -public abstract class BaseOptionManager implements OptionSet {
    -//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
    -
    -  /**
    -   * Gets the current option value given a validator.
    -   *
    -   * @param validator the validator
    -   * @return option value
    -   * @throws IllegalArgumentException - if the validator is not found
    -   */
    -  private OptionValue getOptionSafe(OptionValidator validator)  {
    -    OptionValue value = getOption(validator.getOptionName());
    -    return value == null ? validator.getDefault() : value;
    +import org.apache.drill.common.exceptions.UserException;
    +
    +import java.util.Iterator;
    +
    +/**
    + * This {@link OptionManager} implements some the basic methods and should be extended by concrete implementations.
    + */
    +public abstract class BaseOptionManager extends BaseOptionSet implements OptionManager {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
    +
    +  @Override
    +  public OptionList getInternalOptionList() {
    +    return getAllOptionList(true);
       }
     
       @Override
    -  public boolean getOption(BooleanValidator validator) {
    -    return getOptionSafe(validator).bool_val;
    +  public OptionList getPublicOptionList() {
    +    return getAllOptionList(false);
       }
     
       @Override
    -  public double getOption(DoubleValidator validator) {
    -    return getOptionSafe(validator).float_val;
    +  public void setLocalOption(String name, boolean value) {
    +    setLocalOption(OptionValue.Kind.BOOLEAN, name, Boolean.toString(value));
    --- End diff --
    
    Done.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937901
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java ---
    @@ -56,7 +56,7 @@ public void checkChangedColumn() throws Exception {
         test("ALTER session SET `%s` = %d;", SLICE_TARGET,
           ExecConstants.SLICE_TARGET_DEFAULT);
         testBuilder()
    -        .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND type = 'SESSION'", SLICE_TARGET)
    +        .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND optionScope = 'SESSION'", SLICE_TARGET)
    --- End diff --
    
    Hmmm... I wonder about this. Aside from our own tests, might users have existing code or scripts that know about the existing schema? It is for this reason that Jyothsna added a new table for the enhanced option data, going out of her way to preserve the public schema of the existing tables. I wonder, should we do that here? Or, are we fortunate and no one depends on the current system table schema?


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136671911
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/test/RestClientFixture.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.test;
    +
    +import com.google.common.base.Preconditions;
    +import io.restassured.RestAssured;
    +import io.restassured.http.ContentType;
    +import org.apache.drill.exec.server.rest.StatusResources;
    +
    +import javax.annotation.Nullable;
    +
    +import static io.restassured.RestAssured.when;
    +
    +/**
    + * Represents a client for the Drill Rest API.
    + */
    +public class RestClientFixture {
    +  public static final String REST_API_PORT = "drill.exec.http.port";
    +
    +  /**
    +   * A builder for the rest client.
    +   */
    +  public static class Builder {
    +    private ClusterFixture cluster;
    +
    +    public Builder(ClusterFixture cluster) {
    +      this.cluster = Preconditions.checkNotNull(cluster);
    +    }
    +
    +    public RestClientFixture build() {
    +      return new RestClientFixture(cluster);
    +    }
    +  }
    +
    +
    +  private RestClientFixture(ClusterFixture cluster) {
    +    int port = cluster.config.getInt(REST_API_PORT);
    +    String address = cluster.drillbits().iterator().next().getContext().getEndpoint().getAddress();
    +
    +    RestAssured.port = port;
    +    RestAssured.basePath = "/";
    +    RestAssured.baseURI = "http://" + address;
    --- End diff --
    
    Yeah the rest-assured library uses global variables, which I don't like either. But it gives some nice additionally functionality for testing. The global configuration shouldn't impact us though because we run tests concurrently by forking separate jvms, and running tests sequentially within each jvm. If you see this being a larger issue later when we potentially run tests concurrently within a jvm, I can flip back to using the jersey-client which does not have this problem.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138132892
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java ---
    @@ -56,7 +56,7 @@ public void checkChangedColumn() throws Exception {
         test("ALTER session SET `%s` = %d;", SLICE_TARGET,
           ExecConstants.SLICE_TARGET_DEFAULT);
         testBuilder()
    -        .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND type = 'SESSION'", SLICE_TARGET)
    +        .sqlQuery("SELECT status FROM sys.options WHERE name = '%s' AND optionScope = 'SESSION'", SLICE_TARGET)
    --- End diff --
    
    Yeah we are in a tough spot because **type** was not well defined previously, so the tests implicitly assumed **type** represented where an option was set. Now that we have settled on a definition for **type**, which is the set of scopes where an option can be set, we have deviated from the meaning **type** was given in the tests. One possible way out of this situation is to change the definition of **type** and **optionScope** again by swapping their meanings:
    
    * **type**: Would become where an option was set.
    * **optionScope**: Would become the set of scopes where an option could be set.
    
    This would minimize the changes required to the unit tests. It's hard to say how it would impact other user's scripts though because **type** was treated inconsistently in the code base, so I'm not sure how someone could have used the **type** information productively except to write unit tests, which verified incorrect behavior.
    
    Long story short, I'll swap the definitions of **type** and **optionScope**. I think that would minimize the impact on unit tests and users, but we cannot provide any guarantees for users who depended on type when it was inconsistently defined.



---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138168177
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionMetaData.java ---
    @@ -0,0 +1,68 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.exec.server.options;
    +
    +/**
    + * Contains information about the scopes in which an option can be set, and an option's visibility.
    + */
    +public class OptionMetaData {
    +  public static final OptionMetaData DEFAULT = new OptionMetaData(OptionValue.OptionType.ALL, false, false);
    +
    +  private OptionValue.OptionType type;
    +  private boolean adminOption;
    --- End diff --
    
    Done


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136669314
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java ---
    @@ -70,11 +69,13 @@
      *  </p>
      */
     
    -public class SystemOptionManager extends BaseOptionManager implements OptionManager, AutoCloseable {
    +public class SystemOptionManager extends BaseOptionManager implements AutoCloseable {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SystemOptionManager.class);
     
    -  private CaseInsensitiveMap<OptionValidator> VALIDATORS;
    -  public void populateValidators() {
    +  public static final CaseInsensitiveMap<OptionValidator> DEFAULT_VALIDATORS =
    --- End diff --
    
    Not a regression see comment above.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136662040
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java ---
    @@ -211,12 +217,15 @@ public void tablesWithTableNameFilter() throws Exception {
     
         assertEquals(RequestStatus.OK, resp.getStatus());
         List<TableMetadata> tables = resp.getTablesList();
    -    assertEquals(5, tables.size());
    +    assertEquals(7, tables.size());
     
         verifyTable("sys", "boot", tables);
         verifyTable("sys", "memory", tables);
         verifyTable("sys", "options", tables);
    +    verifyTable("sys", "options2", tables);
         verifyTable("sys", "version", tables);
    +    verifyTable("sys", "internal_options", tables);
    +    verifyTable("sys", "internal_options2", tables);
    --- End diff --
    
    Better name? Nobody is going to know what "options2" means... I certainly can't guess...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136658551
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/OptionIterator.java ---
    @@ -36,7 +36,7 @@
     //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OptionIterator.class);
     
       enum Mode {
    -    BOOT, SYS_SESS, BOTH
    +    BOOT, SYS_SESS_EXTERNAL, SYS_SESS_INTERNAL, BOTH
    --- End diff --
    
    Can we improve these names? Maybe `RUNTIME` for system/session option.
    
    I'm not clear on the distinction between external and internal. Do we mean something like public and internal?
    
    Does the external mean only public, while internal means public and private?
    
    And, given that we have three options, `BOTH` seems a bit of a misnomer...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136659980
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java ---
    @@ -70,11 +69,13 @@
      *  </p>
      */
     
    -public class SystemOptionManager extends BaseOptionManager implements OptionManager, AutoCloseable {
    +public class SystemOptionManager extends BaseOptionManager implements AutoCloseable {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SystemOptionManager.class);
     
    -  private CaseInsensitiveMap<OptionValidator> VALIDATORS;
    -  public void populateValidators() {
    +  public static final CaseInsensitiveMap<OptionValidator> DEFAULT_VALIDATORS =
    --- End diff --
    
    A compromise is to introduce a two-level table.
    
    * Global table of option metadata. Perhaps evolve the validator to be that metadata. Or, better, define a new `OptionDefinition` class, an attribute of which is the validator.
    * Per-Drillbit table of defaults resolved from the config system using the information from the definitions.
    
    If we do this, we should clean up all the messy enums in this system. The current ones confuse everyone. We want:
    
    * Definition: The allowed scope: session-only or both session and system.
    * Visibility: public or hidden (this change)
    * Data type: (long, double, string, etc.)
    
    For a value:
    
    * Scope: default, session or system.
    
    Sometimes it is easier to fix the original problems than to keep working around an outdated design...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136657343
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StatusResources.java ---
    @@ -50,75 +53,124 @@
     public class StatusResources {
       static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatusResources.class);
     
    +  public static final String PATH_STATUS_JSON = "/status.json";
    +  public static final String PATH_STATUS = "/status";
    +  public static final String PATH_OPTIONS_JSON = "/options.json";
    +  public static final String PATH_INTERNAL_OPTIONS_JSON = "/internal_options.json";
    +  public static final String PATH_OPTIONS = "/options";
    +  public static final String PATH_INTERNAL_OPTIONS = "/internal_options";
    --- End diff --
    
    Nice improvement to externalize constants. Perhaps the constants can show the relationships between paths. For example:
    
    ```
    public static final String REST_API_SUFFIX = ".json";
    public static final String PATH_STATUS_UI = "/status";
    public static final String PATH_STATUS_API = PATH_STATUS_UI + REST_API_SUFFIX;
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136657555
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StatusResources.java ---
    @@ -50,75 +53,124 @@
     public class StatusResources {
       static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatusResources.class);
     
    +  public static final String PATH_STATUS_JSON = "/status.json";
    +  public static final String PATH_STATUS = "/status";
    +  public static final String PATH_OPTIONS_JSON = "/options.json";
    +  public static final String PATH_INTERNAL_OPTIONS_JSON = "/internal_options.json";
    +  public static final String PATH_OPTIONS = "/options";
    +  public static final String PATH_INTERNAL_OPTIONS = "/internal_options";
    +
       @Inject UserAuthEnabled authEnabled;
       @Inject WorkManager work;
       @Inject SecurityContext sc;
     
       @GET
    -  @Path("/status.json")
    +  @Path(StatusResources.PATH_STATUS_JSON)
       @Produces(MediaType.APPLICATION_JSON)
       public Pair<String, String> getStatusJSON() {
         return new ImmutablePair<>("status", "Running!");
       }
     
       @GET
    -  @Path("/status")
    +  @Path(StatusResources.PATH_STATUS)
       @Produces(MediaType.TEXT_HTML)
       public Viewable getStatus() {
         return ViewableWithPermissions.create(authEnabled.get(), "/rest/status.ftl", sc, getStatusJSON());
       }
     
    -  @GET
    -  @Path("/options.json")
    -  @RolesAllowed(DrillUserPrincipal.AUTHENTICATED_ROLE)
    -  @Produces(MediaType.APPLICATION_JSON)
    -  public List<OptionWrapper> getSystemOptionsJSON() {
    +  private List<OptionWrapper> getSystemOptionsJSONHelper(boolean internal)
    +  {
         List<OptionWrapper> options = new LinkedList<>();
    -    for (OptionValue option : work.getContext().getOptionManager()) {
    +    OptionManager optionManager = work.getContext().getOptionManager();
    +    OptionList optionList = internal ? optionManager.getInternalOptionList(): optionManager.getExternalOptionList();
    +
    +    for (OptionValue option : optionList) {
           options.add(new OptionWrapper(option.name, option.getValue(), option.type, option.kind));
         }
    +
    --- End diff --
    
    FWIW, DRILL-5716 introduces name sorting to this page. (I can never find anything in the original unsorted list...)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r137937858
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java ---
    @@ -102,10 +103,11 @@ public void testFix2967() throws Exception {
           test("select * from dfs_test.`%s/join/j1` j1 left outer join dfs_test.`%s/join/j2` j2 on (j1.c_varchar = j2.c_varchar)",
             TEST_RES_PATH, TEST_RES_PATH);
         } finally {
    -      setSessionOption(PlannerSettings.BROADCAST.getOptionName(), String.valueOf(PlannerSettings.BROADCAST.getDefault().bool_val));
    -      setSessionOption(PlannerSettings.HASHJOIN.getOptionName(), String.valueOf(PlannerSettings.HASHJOIN.getDefault().bool_val));
    +      final OperatorFixture.TestOptionSet testOptionSet = new OperatorFixture.TestOptionSet();
    +      setSessionOption(PlannerSettings.BROADCAST.getOptionName(), String.valueOf(testOptionSet.getDefault(PlannerSettings.BROADCAST.getOptionName()).bool_val));
    --- End diff --
    
    No need to fix now but I wonder:
    
    * The `ClusterFixture` class has a method of form `setSystemOption(String name, Option value)` to avoid this kind of silly to/from String overhead. We might want to do the same for these "legacy" `BaseTestQuery` style tests.
    * But, here we are setting system options to their default value. Why? Isn't that already the value? If not, wouldn't clearing (deleting) the option value be better?
    * Or, has the test previously set a system option to other than the default value, and we are undoing that here? Seems it would be better to allow a test to say not to muck with the defaults. (Hard to do with `BaseTestQuery`, but there is a built-in provision for this in the `ClusterFixture`.)


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r138133105
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java ---
    @@ -216,79 +216,77 @@ public void injectionOnSpecificBit() {
         final ZookeeperHelper zkHelper = new ZookeeperHelper();
         zkHelper.startZookeeper(1);
     
    -    // Creating two drillbits
    -    final Drillbit drillbit1, drillbit2;
    -    final DrillConfig drillConfig = zkHelper.getConfig();
         try {
    -      drillbit1 = Drillbit.start(drillConfig, remoteServiceSet);
    -      drillbit2 = Drillbit.start(drillConfig, remoteServiceSet);
    -    } catch (DrillbitStartupException e) {
    -      throw new RuntimeException("Failed to start drillbits.", e);
    -    }
    +      // Creating two drillbits
    +      final Drillbit drillbit1, drillbit2;
    +      final DrillConfig drillConfig = zkHelper.getConfig();
    +      try {
    +        drillbit1 = Drillbit.start(drillConfig, remoteServiceSet);
    +        drillbit2 = Drillbit.start(drillConfig, remoteServiceSet);
    +      } catch (DrillbitStartupException e) {
    +        throw new RuntimeException("Failed to start drillbits.", e);
    +      }
     
    -    final DrillbitContext drillbitContext1 = drillbit1.getContext();
    -    final DrillbitContext drillbitContext2 = drillbit2.getContext();
    +      final DrillbitContext drillbitContext1 = drillbit1.getContext();
    +      final DrillbitContext drillbitContext2 = drillbit2.getContext();
     
    -    final UserSession session = UserSession.Builder.newBuilder()
    -        .withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build())
    -        .withUserProperties(UserProperties.getDefaultInstance())
    -        .withOptionManager(drillbitContext1.getOptionManager())
    -        .build();
    +      final UserSession session = UserSession.Builder.newBuilder().withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build()).withUserProperties(UserProperties.getDefaultInstance()).withOptionManager(drillbitContext1.getOptionManager()).build();
    --- End diff --
    
    I'm a victim of autoformat. Fixed.


---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136656853
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java ---
    @@ -70,11 +69,13 @@
      *  </p>
      */
     
    -public class SystemOptionManager extends BaseOptionManager implements OptionManager, AutoCloseable {
    +public class SystemOptionManager extends BaseOptionManager implements AutoCloseable {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SystemOptionManager.class);
     
    -  private CaseInsensitiveMap<OptionValidator> VALIDATORS;
    -  public void populateValidators() {
    +  public static final CaseInsensitiveMap<OptionValidator> DEFAULT_VALIDATORS =
    --- End diff --
    
    I wonder, was this an intentional regression, or has your code not been rebased onto the latest master (with Jythosna's changes) and so your commit reintroduces old code?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136656530
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java ---
    @@ -70,11 +69,13 @@
      *  </p>
      */
     
    -public class SystemOptionManager extends BaseOptionManager implements OptionManager, AutoCloseable {
    +public class SystemOptionManager extends BaseOptionManager implements AutoCloseable {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SystemOptionManager.class);
     
    -  private CaseInsensitiveMap<OptionValidator> VALIDATORS;
    -  public void populateValidators() {
    +  public static final CaseInsensitiveMap<OptionValidator> DEFAULT_VALIDATORS =
    --- End diff --
    
    This change reverses a change made by Jyothsna. While it seems reasonable to create a static table of validators, there is a subtle problem which Jyothsna's fix addressed.
    
    Under normal operation, we have exactly one Drillbit per process and thus exactly one system option manager.
    
    But, we have unit tests that create multiple Drillbits per process. This is the most convenient way to test concurrency issues in unit tests. In general, each Drillbit could define a different default value for the system/session option; perhaps to check forward/backward compatibility when the time comes to support rolling upgrades.
    
    In this model, each Drillbit will have a slightly different set of defaults. Thus, the option validator table must be a member variable of the system option manager; it cannot be global.
    
    Note also, in the existing tests with multiple Drillbits, a global table runs into a race condition during reading defaults from the config file if we use a single table. But, if we have a separate table per Drillbit, the race condition disappears.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136672454
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java ---
    @@ -90,13 +93,13 @@
       public static class OperatorFixtureBuilder
       {
         ConfigBuilder configBuilder = new ConfigBuilder();
    -    TestOptionSet options = new TestOptionSet();
    +    TestOptionManager options = new TestOptionManager();
    --- End diff --
    
    I'll revert back to OptionSet interface


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #923: DRILL-5723: Added System Internal Options That can ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/923#discussion_r136651457
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValue.java ---
    @@ -218,4 +218,44 @@ public int compareTo(OptionValue o) {
       public String toString() {
         return "OptionValue [ type=" + type + ", name=" + name + ", value=" + getValue() + " ]";
       }
    +
    +  public static class MetaData {
    --- End diff --
    
    Should the metadata be part of the *value* or part of the *definition* (the validator)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---