accumulo-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From keith-turner <...@git.apache.org>
Subject [GitHub] accumulo pull request #134: ACCUMULO-4391 Added synchronization around the a...
Date Thu, 28 Jul 2016 16:27:14 GMT
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/134#discussion_r72655136
  
    --- Diff: core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java
---
    @@ -0,0 +1,387 @@
    +/*
    + * 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.core.file.rfile;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.PrintWriter;
    +import java.io.StringWriter;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.ThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.data.ArrayByteSequence;
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +import org.apache.accumulo.core.file.FileSKVIterator;
    +import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
    +import org.apache.accumulo.core.file.rfile.RFile.Reader;
    +import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
    +import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
    +import org.apache.accumulo.core.util.CachedConfiguration;
    +import org.apache.accumulo.core.util.NamingThreadFactory;
    +import org.apache.commons.lang3.mutable.MutableInt;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FSDataOutputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertTrue;
    +
    +public class MultiThreadedRFileTest {
    +
    +  private static final Collection<ByteSequence> EMPTY_COL_FAMS = new ArrayList<ByteSequence>();
    +
    +  @Rule
    +  public TemporaryFolder tempFolder = new TemporaryFolder(new File(System.getProperty("user.dir")
+ "/target"));
    +
    +  static {
    +    Logger.getLogger(org.apache.hadoop.io.compress.CodecPool.class).setLevel(Level.OFF);
    +    Logger.getLogger(org.apache.hadoop.util.NativeCodeLoader.class).setLevel(Level.OFF);
    +  }
    +
    +  private static void checkIndex(Reader reader) throws IOException {
    +    FileSKVIterator indexIter = reader.getIndex();
    +
    +    if (indexIter.hasTop()) {
    +      Key lastKey = new Key(indexIter.getTopKey());
    +
    +      if (reader.getFirstKey().compareTo(lastKey) > 0)
    +        throw new RuntimeException("First key out of order " + reader.getFirstKey() +
" " + lastKey);
    +
    +      indexIter.next();
    +
    +      while (indexIter.hasTop()) {
    +        if (lastKey.compareTo(indexIter.getTopKey()) > 0)
    +          throw new RuntimeException("Indext out of order " + lastKey + " " + indexIter.getTopKey());
    +
    +        lastKey = new Key(indexIter.getTopKey());
    +        indexIter.next();
    +
    +      }
    +
    +      if (!reader.getLastKey().equals(lastKey)) {
    +        throw new RuntimeException("Last key out of order " + reader.getLastKey() + "
" + lastKey);
    +      }
    +    }
    +  }
    +
    +  public static class TestRFile {
    +
    +    private Configuration conf = CachedConfiguration.getInstance();
    +    public RFile.Writer writer;
    +    private FSDataOutputStream dos;
    +    private FSDataInputStream in;
    +    private AccumuloConfiguration accumuloConfiguration;
    +    public Reader reader;
    +    public SortedKeyValueIterator<Key,Value> iter;
    +    public File rfile = null;
    +    public boolean deepCopy = false;
    +
    +    public TestRFile(AccumuloConfiguration accumuloConfiguration) {
    +      this.accumuloConfiguration = accumuloConfiguration;
    +      if (this.accumuloConfiguration == null)
    +        this.accumuloConfiguration = AccumuloConfiguration.getDefaultConfiguration();
    +    }
    +
    +    public void close() throws IOException {
    +      if (rfile != null) {
    +        FileSystem fs = FileSystem.newInstance(conf);
    +        Path path = new Path("file://" + rfile.toString());
    +        fs.delete(path, false);
    +      }
    +    }
    +
    +    public TestRFile deepCopy() throws IOException {
    +      TestRFile copy = new TestRFile(accumuloConfiguration);
    +      // does not copy any writer resources. This would be for read only.
    +      copy.reader = (Reader) reader.deepCopy(null);
    +      copy.rfile = rfile;
    +      copy.iter = new ColumnFamilySkippingIterator(copy.reader);
    +      copy.deepCopy = true;
    +
    +      checkIndex(copy.reader);
    +      return copy;
    +    }
    +
    +    public void openWriter(boolean startDLG) throws IOException {
    +      if (deepCopy) {
    +        throw new IOException("Cannot open writer on a deep copy");
    +      }
    +      if (rfile == null) {
    +        rfile = File.createTempFile("TestRFile", ".rf");
    +      }
    +      FileSystem fs = FileSystem.newInstance(conf);
    +      Path path = new Path("file://" + rfile.toString());
    +      dos = fs.create(path, true);
    +      CachableBlockFile.Writer _cbw = new CachableBlockFile.Writer(dos, "gz", conf, accumuloConfiguration);
    +      writer = new RFile.Writer(_cbw, 1000, 1000);
    +
    +      if (startDLG)
    +        writer.startDefaultLocalityGroup();
    +    }
    +
    +    public void openWriter() throws IOException {
    +      openWriter(true);
    +    }
    +
    +    public void closeWriter() throws IOException {
    +      if (deepCopy) {
    +        throw new IOException("Cannot open writer on a deepcopy");
    +      }
    +      dos.flush();
    +      writer.close();
    +      dos.flush();
    +      dos.close();
    +    }
    +
    +    public void openReader() throws IOException {
    +      FileSystem fs = FileSystem.newInstance(conf);
    +      Path path = new Path("file://" + rfile.toString());
    +
    +      // the caches used to obfuscate the multithreaded issues
    +      CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(fs, path, conf, null,
null, AccumuloConfiguration.getDefaultConfiguration());
    +      reader = new RFile.Reader(_cbr);
    +      iter = new ColumnFamilySkippingIterator(reader);
    +
    +      checkIndex(reader);
    +    }
    +
    +    public void closeReader() throws IOException {
    +      reader.close();
    +    }
    +
    +    public void seek(Key nk) throws IOException {
    +      iter.seek(new Range(nk, null), EMPTY_COL_FAMS, false);
    +    }
    +  }
    +
    +  static Key nk(String row, String cf, String cq, String cv, long ts) {
    +    return new Key(row.getBytes(), cf.getBytes(), cq.getBytes(), cv.getBytes(), ts);
    +  }
    +
    +  static Value nv(String val) {
    +    return new Value(val.getBytes());
    +  }
    +
    +  public AccumuloConfiguration conf = null;
    +
    +  @Test
    +  public void testMultipleReaders() throws IOException {
    +    final List<Throwable> threadExceptions = Collections.synchronizedList(new ArrayList<Throwable>());
    +    Map<String,MutableInt> messages = new HashMap<String,MutableInt>();
    +    Map<String,String> stackTrace = new HashMap<String,String>();
    +
    +    final TestRFile trfBase = new TestRFile(conf);
    +
    +    writeData(trfBase);
    +
    +    trfBase.openReader();
    +
    +    try {
    +
    +      validate(trfBase);
    +
    +      final TestRFile trfBaseCopy = trfBase.deepCopy();
    +
    +      validate(trfBaseCopy);
    +
    +      // now start up multiple RFile deepcopies
    +      int maxThreads = 10;
    +      String name = "MultiThreadedRFileTestThread";
    +      ThreadPoolExecutor pool = new ThreadPoolExecutor(maxThreads + 1, maxThreads + 1,
5 * 60, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
    +          new NamingThreadFactory(name));
    +      pool.allowCoreThreadTimeOut(true);
    +      try {
    +        Runnable runnable = new Runnable() {
    +          @Override
    +          public void run() {
    +            try {
    +              TestRFile trf = trfBase;
    +              synchronized (trfBaseCopy) {
    +                trf = trfBaseCopy.deepCopy();
    +              }
    +              validate(trf);
    +            } catch (Throwable t) {
    +              threadExceptions.add(t);
    +            }
    +          }
    +        };
    +        for (int i = 0; i < maxThreads; i++) {
    +          pool.submit(runnable);
    +        }
    +      } finally {
    +        pool.shutdown();
    +        try {
    +          pool.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
    +        } catch (InterruptedException e) {
    +          e.printStackTrace();
    +        }
    +      }
    +
    +      for (Throwable t : threadExceptions) {
    +        String msg = t.getClass() + " : " + t.getMessage();
    +        if (!messages.containsKey(msg)) {
    +          messages.put(msg, new MutableInt(1));
    +        } else {
    +          messages.get(msg).increment();
    +        }
    +        StringWriter string = new StringWriter();
    +        PrintWriter writer = new PrintWriter(string);
    +        t.printStackTrace(writer);
    +        writer.flush();
    +        stackTrace.put(msg, string.getBuffer().toString());
    +      }
    +    } finally {
    +      trfBase.closeReader();
    +      trfBase.close();
    +    }
    +
    +    for (String message : messages.keySet()) {
    +      System.out.println(messages.get(message) + ": " + message);
    +      System.out.println(stackTrace.get(message));
    +    }
    +
    +    assertTrue(threadExceptions.isEmpty());
    +  }
    +
    +  private void validate(TestRFile trf) throws IOException {
    +    Random random = new Random();
    +    for (int iteration = 0; iteration < 10; iteration++) {
    +      int part = random.nextInt(4);
    --- End diff --
    
    My thinking here may not be completely correct. I think it depends on if caching layer
is used.  when the caching layer is used, it forces entire blocks to be read into memory which
would avoid concurrent access to input.  However w/o caching there may still be lots of concurrent
access if the input stream when scanning lots of data, not 100% sure though.


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

Mime
View raw message