drill-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (DRILL-3171) Storage Plugins : Two processes tried to update the storage plugin at the same time
Date Tue, 17 Nov 2015 00:32:11 GMT

    [ https://issues.apache.org/jira/browse/DRILL-3171?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15007693#comment-15007693
] 

ASF GitHub Bot commented on DRILL-3171:
---------------------------------------

Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/260#discussion_r45006978
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/zk/ZkAbstractStore.java
---
    @@ -136,7 +138,81 @@ public boolean putIfAbsent(String key, V value) {
         }
       }
     
    -  public abstract void createNodeInZK (String key, V value);
    +  /**
    +   * Default {@link CreateMode create mode} that will be used in create operations referred
in the see also section.
    +   *
    +   * @see #createOrUpdate(String, Object)
    +   * @see #createWithPrefix(String, Object)
    +   */
    +  protected abstract CreateMode getCreateMode();
    +
    +
    +  /**
    +   * Creates a node in zookeeper with the {@link #getCreateMode() default create mode}
and sets its value if supplied.
    +   *
    +   * @param path    target path
    +   * @param value   value to set, null if none available
    +   *
    +   * @see #getCreateMode()
    +   * @see #createOrUpdate(String, Object)
    +   * @see #withPrefix(String)
    +   */
    +  protected void createWithPrefix(String path, V value) {
    +    createOrUpdate(withPrefix(path), value);
    +  }
    +
    +  /**
    +   * Creates a node in zookeeper with the {@link #getCreateMode() default create mode}
and sets its value if supplied
    +   * or updates its value if the node already exists.
    +   *
    +   * Note that if node exists, its mode will not be changed.
    +   *
    +   * @param path    target path
    +   * @param value   value to set, null if none available
    +   *
    +   * @see #getCreateMode()
    +   * @see #createOrUpdate(String, Object, CreateMode)
    +   */
    +  protected void createOrUpdate(String path, V value) {
    +    createOrUpdate(path, value, getCreateMode());
    +  }
    +
    +  /**
    +   * Creates a node in zookeeper with the given mode and sets its value if supplied or
updates its value if the node
    +   * already exists.
    +   *
    +   * Note that if the node exists, its mode will not be changed.
    +   *
    +   * Internally, the method suppresses {@link org.apache.zookeeper.KeeperException.NodeExistsException}.
It is
    +   * safe to do so since the implementation is idempotent.
    +   *
    +   * @param path    target path
    +   * @param value   value to set, null if none available
    +   * @param mode    creation mode
    +   * @throws RuntimeException  throws a {@link RuntimeException} wrapping the root cause.
    +   */
    +  protected void createOrUpdate(String path, V value, CreateMode mode) {
    +    try {
    +      final boolean isUpdate = value != null;
    +      final byte[] valueInBytes = isUpdate ? config.getSerializer().serialize(value)
: null;
    +      final boolean nodeExists = framework.checkExists().forPath(path) != null;
    +      if (!nodeExists) {
    +        final ACLBackgroundPathAndBytesable<String> creator = framework.create().withMode(mode);
    +        if (isUpdate) {
    +          creator.forPath(path, valueInBytes);
    +        } else {
    +          creator.forPath(path);
    +        }
    +      } else if (isUpdate) {
    +        framework.setData().forPath(path, valueInBytes);
    +      }
    +    } catch (KeeperException.NodeExistsException ex) {
    +      logger.warn("Node already exists in Zookeeper. Skipping... -- [path: {}, mode:
{}]", path, mode);
    --- End diff --
    
    If isUpdate == true, should we try to update?


> Storage Plugins : Two processes tried to update the storage plugin at the same time
> -----------------------------------------------------------------------------------
>
>                 Key: DRILL-3171
>                 URL: https://issues.apache.org/jira/browse/DRILL-3171
>             Project: Apache Drill
>          Issue Type: Bug
>          Components: Storage - Information Schema
>    Affects Versions: 1.0.0
>            Reporter: Rahul Challapalli
>            Assignee: Deneche A. Hakim
>              Labels: test
>             Fix For: Future
>
>
> Commit Id# : bd8ac4fca03ad5043bca27fbc7e0dec5a35ac474
> We have seen this issue happen with the below steps
>    1. Clear out the zookeeper
>    2. Update the storage plugin using the rest API on one of the node
>    3. Submit 10 queries concurrently
> With randomized foreman node selection, the node executing the query might not have the
updated storage plugins info. This could be causing the issue.
> - Rahul



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message