You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2015/08/27 23:40:50 UTC

[1/3] accumulo git commit: ACCUMULO-3977 fix issue with deep copy and isolation

Repository: accumulo
Updated Branches:
  refs/heads/1.7 162acf2f2 -> d77863e04


ACCUMULO-3977 fix issue with deep copy and isolation


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/562b6ad6
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/562b6ad6
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/562b6ad6

Branch: refs/heads/1.7
Commit: 562b6ad606d3211a957852baf5d598237631ddde
Parents: 2906895
Author: Keith Turner <ke...@deenlo.com>
Authored: Thu Aug 27 16:27:29 2015 -0400
Committer: Keith Turner <ke...@deenlo.com>
Committed: Thu Aug 27 16:40:27 2015 -0400

----------------------------------------------------------------------
 .../apache/accumulo/tserver/FileManager.java    |  8 +-
 .../accumulo/test/IsolationAndDeepCopyIT.java   | 85 ++++++++++++++++++++
 2 files changed, 89 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/562b6ad6/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
index 4b9664a..3a4d512 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
@@ -425,9 +425,7 @@ public class FileManager {
     }
 
     @Override
-    public SortedKeyValueIterator<Key,Value> iterator() throws IOException {
-      if (iflag != null)
-        ((InterruptibleIterator) this.iter).setInterruptFlag(iflag);
+    public SortedKeyValueIterator<Key,Value> iterator() {
       return iter;
     }
 
@@ -444,8 +442,9 @@ public class FileManager {
       current = false;
       this.iter = iter;
 
-      if (iflag != null)
+      if (iflag != null) {
         ((InterruptibleIterator) this.iter).setInterruptFlag(iflag);
+      }
 
       for (FileDataSource fds : deepCopies) {
         fds.current = false;
@@ -456,6 +455,7 @@ public class FileManager {
     @Override
     public void setInterruptFlag(AtomicBoolean flag) {
       this.iflag = flag;
+      ((InterruptibleIterator) this.iter).setInterruptFlag(iflag);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/562b6ad6/test/src/test/java/org/apache/accumulo/test/IsolationAndDeepCopyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/IsolationAndDeepCopyIT.java b/test/src/test/java/org/apache/accumulo/test/IsolationAndDeepCopyIT.java
new file mode 100644
index 0000000..6af1fdf
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/IsolationAndDeepCopyIT.java
@@ -0,0 +1,85 @@
+/*
+ * 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.accumulo.test;
+
+import java.util.Iterator;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.user.IntersectingIterator;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.harness.AccumuloClusterIT;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class IsolationAndDeepCopyIT extends AccumuloClusterIT {
+
+  @Test
+  public void testBugFix() throws Exception {
+    // test bug fox for ACCUMULO-3977
+
+    String table = super.getUniqueNames(1)[0];
+    Connector conn = getConnector();
+
+    conn.tableOperations().create(table);
+
+    BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
+
+    addDocument(bw, "000A", "dog", "cat", "hamster", "iguana", "the");
+    addDocument(bw, "000B", "java", "perl", "C++", "pascal", "the");
+    addDocument(bw, "000C", "chrome", "firefox", "safari", "opera", "the");
+    addDocument(bw, "000D", "logarithmic", "quadratic", "linear", "exponential", "the");
+
+    bw.close();
+
+    // its a bug when using rfiles, so flush
+    conn.tableOperations().flush(table, null, null, true);
+
+    IteratorSetting iterCfg = new IteratorSetting(30, "ayeaye", IntersectingIterator.class.getName());
+    IntersectingIterator.setColumnFamilies(iterCfg, new Text[] {new Text("the"), new Text("hamster")});
+
+    Scanner scanner = conn.createScanner(table, Authorizations.EMPTY);
+    scanner.enableIsolation();
+    scanner.addScanIterator(iterCfg);
+
+    for (int i = 0; i < 100; i++) {
+      Iterator<Entry<Key,Value>> iter = scanner.iterator();
+      Assert.assertTrue(iter.hasNext());
+      Assert.assertEquals("000A", iter.next().getKey().getColumnQualifierData().toString());
+      Assert.assertFalse(iter.hasNext());
+    }
+  }
+
+  private void addDocument(BatchWriter bw, String docId, String... terms) throws MutationsRejectedException {
+    Mutation m = new Mutation(String.format("%04d", docId.hashCode() % 10));
+    for (String term : terms) {
+      m.put(term, docId, "");
+    }
+
+    bw.addMutation(m);
+  }
+}


[3/3] accumulo git commit: Merge branch '1.6' into 1.7

Posted by kt...@apache.org.
Merge branch '1.6' into 1.7


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/d77863e0
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/d77863e0
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/d77863e0

Branch: refs/heads/1.7
Commit: d77863e044f36107fcf84449f56c818b55a15ed4
Parents: 162acf2 a1497d1
Author: Keith Turner <ke...@deenlo.com>
Authored: Thu Aug 27 17:15:52 2015 -0400
Committer: Keith Turner <ke...@deenlo.com>
Committed: Thu Aug 27 17:15:52 2015 -0400

----------------------------------------------------------------------
 .../apache/accumulo/tserver/FileManager.java    |  8 +-
 .../shell/commands/SetShellIterCommand.java     | 15 +---
 .../accumulo/test/IsolationAndDeepCopyIT.java   | 85 ++++++++++++++++++++
 3 files changed, 93 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/d77863e0/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d77863e0/shell/src/main/java/org/apache/accumulo/shell/commands/SetShellIterCommand.java
----------------------------------------------------------------------
diff --cc shell/src/main/java/org/apache/accumulo/shell/commands/SetShellIterCommand.java
index 10c67be,0000000..6b1da3b
mode 100644,000000..100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/SetShellIterCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/SetShellIterCommand.java
@@@ -1,130 -1,0 +1,123 @@@
 +/*
 + * 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.accumulo.shell.commands;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 +import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 +import org.apache.accumulo.shell.Shell;
 +import org.apache.accumulo.shell.ShellCommandException;
 +import org.apache.accumulo.shell.ShellCommandException.ErrorCode;
 +import org.apache.commons.cli.CommandLine;
 +import org.apache.commons.cli.Option;
 +import org.apache.commons.cli.OptionGroup;
 +import org.apache.commons.cli.Options;
 +
 +public class SetShellIterCommand extends SetIterCommand {
 +  private Option profileOpt;
 +
 +  @Override
 +  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
 +      TableNotFoundException, IOException, ShellCommandException {
 +    return super.execute(fullCommand, cl, shellState);
 +  }
 +
 +  @Override
 +  protected void setTableProperties(final CommandLine cl, final Shell shellState, final int priority, final Map<String,String> options, final String classname,
 +      final String name) throws AccumuloException, AccumuloSecurityException, ShellCommandException, TableNotFoundException {
 +    // instead of setting table properties, just put the options in a list to use at scan time
 +
 +    String profile = cl.getOptionValue(profileOpt.getOpt());
 +
-     // instead of setting table properties, just put the options in a list to use at scan time
-     Class<?> loadClass;
-     try {
-       loadClass = getClass().getClassLoader().loadClass(classname);
-     } catch (ClassNotFoundException e) {
-       throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Unable to load " + classname);
-     }
-     try {
-       loadClass.asSubclass(SortedKeyValueIterator.class);
-     } catch (ClassCastException ex) {
-       throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "xUnable to load " + classname + " as type " + SortedKeyValueIterator.class.getName());
++    if (!shellState.getConnector().instanceOperations().testClassLoad(classname, SortedKeyValueIterator.class.getName())) {
++      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + classname + " as type "
++          + SortedKeyValueIterator.class.getName());
 +    }
 +
++    // instead of setting table properties, just put the options in a list to use at scan time
 +    for (Iterator<Entry<String,String>> i = options.entrySet().iterator(); i.hasNext();) {
 +      final Entry<String,String> entry = i.next();
 +      if (entry.getValue() == null || entry.getValue().isEmpty()) {
 +        i.remove();
 +      }
 +    }
 +
 +    List<IteratorSetting> tableScanIterators = shellState.iteratorProfiles.get(profile);
 +    if (tableScanIterators == null) {
 +      tableScanIterators = new ArrayList<IteratorSetting>();
 +      shellState.iteratorProfiles.put(profile, tableScanIterators);
 +    }
 +    final IteratorSetting setting = new IteratorSetting(priority, name, classname);
 +    setting.addOptions(options);
 +
 +    Iterator<IteratorSetting> iter = tableScanIterators.iterator();
 +    while (iter.hasNext()) {
 +      if (iter.next().getName().equals(name)) {
 +        iter.remove();
 +      }
 +    }
 +
 +    tableScanIterators.add(setting);
 +  }
 +
 +  @Override
 +  public String description() {
 +    return "adds an iterator to a profile for this shell session";
 +  }
 +
 +  @Override
 +  public Options getOptions() {
 +    // Remove the options that specify which type of iterator this is, since
 +    // they are all scan iterators with this command.
 +    final HashSet<OptionGroup> groups = new HashSet<OptionGroup>();
 +    final Options parentOptions = super.getOptions();
 +    final Options modifiedOptions = new Options();
 +    for (Iterator<?> it = parentOptions.getOptions().iterator(); it.hasNext();) {
 +      Option o = (Option) it.next();
 +      if (!IteratorScope.majc.name().equals(o.getOpt()) && !IteratorScope.minc.name().equals(o.getOpt()) && !IteratorScope.scan.name().equals(o.getOpt())
 +          && !"table".equals(o.getLongOpt())) {
 +        modifiedOptions.addOption(o);
 +        OptionGroup group = parentOptions.getOptionGroup(o);
 +        if (group != null)
 +          groups.add(group);
 +      }
 +    }
 +    for (OptionGroup group : groups) {
 +      modifiedOptions.addOptionGroup(group);
 +    }
 +
 +    profileOpt = new Option("pn", "profile", true, "iterator profile name");
 +    profileOpt.setRequired(true);
 +    profileOpt.setArgName("profile");
 +
 +    modifiedOptions.addOption(profileOpt);
 +
 +    return modifiedOptions;
 +  }
 +
 +}


[2/3] accumulo git commit: ACCUMULO-3972 fix how setshelliter checks class can load

Posted by kt...@apache.org.
ACCUMULO-3972 fix how setshelliter checks class can load


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/a1497d14
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/a1497d14
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/a1497d14

Branch: refs/heads/1.7
Commit: a1497d14afd205b5a365edca3ede4c11624c4f57
Parents: 562b6ad
Author: Keith Turner <ke...@deenlo.com>
Authored: Thu Aug 27 17:15:14 2015 -0400
Committer: Keith Turner <ke...@deenlo.com>
Committed: Thu Aug 27 17:15:14 2015 -0400

----------------------------------------------------------------------
 .../util/shell/commands/SetShellIterCommand.java     | 15 ++++-----------
 1 file changed, 4 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a1497d14/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetShellIterCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetShellIterCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetShellIterCommand.java
index d5d2174..a06c412 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetShellIterCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetShellIterCommand.java
@@ -54,19 +54,12 @@ public class SetShellIterCommand extends SetIterCommand {
 
     String profile = cl.getOptionValue(profileOpt.getOpt());
 
-    // instead of setting table properties, just put the options in a list to use at scan time
-    Class<?> loadClass;
-    try {
-      loadClass = getClass().getClassLoader().loadClass(classname);
-    } catch (ClassNotFoundException e) {
-      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Unable to load " + classname);
-    }
-    try {
-      loadClass.asSubclass(SortedKeyValueIterator.class);
-    } catch (ClassCastException ex) {
-      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "xUnable to load " + classname + " as type " + SortedKeyValueIterator.class.getName());
+    if (!shellState.getConnector().instanceOperations().testClassLoad(classname, SortedKeyValueIterator.class.getName())) {
+      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + classname + " as type "
+          + SortedKeyValueIterator.class.getName());
     }
 
+    // instead of setting table properties, just put the options in a list to use at scan time
     for (Iterator<Entry<String,String>> i = options.entrySet().iterator(); i.hasNext();) {
       final Entry<String,String> entry = i.next();
       if (entry.getValue() == null || entry.getValue().isEmpty()) {