Return-Path: X-Original-To: apmail-accumulo-notifications-archive@minotaur.apache.org Delivered-To: apmail-accumulo-notifications-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 4BFFB183BA for ; Sun, 24 Apr 2016 23:18:13 +0000 (UTC) Received: (qmail 34647 invoked by uid 500); 24 Apr 2016 23:18:13 -0000 Delivered-To: apmail-accumulo-notifications-archive@accumulo.apache.org Received: (qmail 34607 invoked by uid 500); 24 Apr 2016 23:18:13 -0000 Mailing-List: contact notifications-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: jira@apache.org Delivered-To: mailing list notifications@accumulo.apache.org Received: (qmail 34592 invoked by uid 99); 24 Apr 2016 23:18:13 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 24 Apr 2016 23:18:13 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id F3F232C1F5D for ; Sun, 24 Apr 2016 23:18:12 +0000 (UTC) Date: Sun, 24 Apr 2016 23:18:12 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: notifications@accumulo.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (ACCUMULO-4229) BatchWriter Locator cache out-of-sync when shared with tserver MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/ACCUMULO-4229?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15255742#comment-15255742 ] ASF GitHub Bot commented on ACCUMULO-4229: ------------------------------------------ Github user joshelser commented on a diff in the pull request: https://github.com/apache/accumulo/pull/96#discussion_r60853711 --- Diff: test/src/test/java/org/apache/accumulo/test/BatchWriterInTabletServerIT.java --- @@ -0,0 +1,120 @@ +/* + * 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 com.google.common.collect.Iterators; +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.Scanner; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.data.PartialKey; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.iterators.LongCombiner; +import org.apache.accumulo.core.iterators.user.SummingCombiner; +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; + +import java.util.Map; + +/** + * Test writing to another table from inside an iterator. + * + * @see BatchWriterIterator + */ +public class BatchWriterInTabletServerIT extends AccumuloClusterIT { + + /** + * This test should succeed. + */ + @Test + public void testNormalWrite() throws Exception { + String[] uniqueNames = getUniqueNames(2); + String t1 = uniqueNames[0], t2 = uniqueNames[1]; + Connector c = getConnector(); + int numEntriesToWritePerEntry = 50; + IteratorSetting itset = BatchWriterIterator.iteratorSetting(6, 0, 15, 1000, numEntriesToWritePerEntry, t2, c, getToken(), false, false); + test(t1, t2, c, itset, numEntriesToWritePerEntry); + } + + /** + * Fixed by ACCUMULO-4229. + *

+ * This tests a situation that a client which shares a LocatorCache with the tablet server may fall into. + * Before the problem was fixed, adding a split after the Locator cache falls out of sync caused the + * BatchWriter to continuously attempt to write to an old, closed tablet. + * It would do so for 15 seconds until a timeout on the BatchWriter. + */ + @Test + public void testClearLocatorAndSplitWrite() throws Exception { + String[] uniqueNames = getUniqueNames(2); + String t1 = uniqueNames[0], t2 = uniqueNames[1]; + Connector c = getConnector(); + int numEntriesToWritePerEntry = 50; + IteratorSetting itset = BatchWriterIterator.iteratorSetting(6, 0, 15, 1000, numEntriesToWritePerEntry, t2, c, getToken(), true, true); + test(t1, t2, c, itset, numEntriesToWritePerEntry); + } + + private void test(String t1, String t2, Connector c, IteratorSetting itset, int numEntriesToWritePerEntry) throws Exception { + // Write an entry to t1 + c.tableOperations().create(t1); + Key k = new Key(new Text("row"), new Text("cf"), new Text("cq")); + Value v = new Value("1".getBytes()); + { + BatchWriterConfig config = new BatchWriterConfig(); + config.setMaxMemory(0); + BatchWriter writer = c.createBatchWriter(t1, config); + Mutation m = new Mutation(k.getRow()); + m.put(k.getColumnFamily(), k.getColumnQualifier(), v); + writer.addMutation(m); + writer.close(); + } + + // Create t2 with a combiner to count entries written to it + c.tableOperations().create(t2); + IteratorSetting summer = new IteratorSetting(2, "summer", SummingCombiner.class); + LongCombiner.setEncodingType(summer, LongCombiner.Type.STRING); + LongCombiner.setCombineAllColumns(summer, true); + c.tableOperations().attachIterator(t2, summer); + + Map.Entry actual; + // Scan t1 with an iterator that writes to table t2 + Scanner scanner = c.createScanner(t1, Authorizations.EMPTY); + scanner.addScanIterator(itset); + actual = Iterators.getOnlyElement(scanner.iterator()); + Assert.assertTrue(actual.getKey().equals(k, PartialKey.ROW_COLFAM_COLQUAL)); + Assert.assertEquals(BatchWriterIterator.SUCCESS_VALUE, actual.getValue()); + scanner.close(); + + // ensure entries correctly wrote to table t2 + scanner = c.createScanner(t2, Authorizations.EMPTY); + actual = Iterators.getOnlyElement(scanner.iterator()); + // System.out.println("t2 entry is " + actual.getKey().toStringNoTime() + " -> " + actual.getValue()); --- End diff -- If this is going to help debug the test on someone else's system, uncomment it and make it a logger debug. Otherwise, remove it. Commented debug statements are useless. > BatchWriter Locator cache out-of-sync when shared with tserver > -------------------------------------------------------------- > > Key: ACCUMULO-4229 > URL: https://issues.apache.org/jira/browse/ACCUMULO-4229 > Project: Accumulo > Issue Type: Bug > Components: client > Affects Versions: 1.6.5, 1.7.1 > Reporter: Dylan Hutchison > Assignee: Dylan Hutchison > > BatchWriters that run a long time have write rates that sometimes mysteriously decrease after the table it is writing to goes through a major compaction or a split. The decrease can be as bad as reducing throughput to 0. > This was first first mentioned in this [email thread|https://mail-archives.apache.org/mod_mbox/accumulo-user/201406.mbox/%3CCAMz+DuvmmHegOn9EJeHR9H_rRpP50L2QZ53BbdruVO0pirArQw@mail.gmail.com%3E] for major compactions. > I discovered this in this [email thread|https://mail-archives.apache.org/mod_mbox/accumulo-dev/201604.mbox/%3CCAPx%3DJkaY7fVh-U0O%2Bysx2d98LOGMcA4oEQOYgoPxR-0em4hdvg%40mail.gmail.com%3E] for splits. See the thread for some log messages. > I turned on TRACE logs and I think I pinned it down: the TabletLocator cached by a BatchWriter gets out of sync with the static cache of TabletLocators. > # The TabletServerBatchWriter caches a TabletLocator from the static collection of TabletLocators when it starts writing. Suppose it is writing to tablet T1. > # The TabletServerBatchWriter uses its locally cached TabletLocator inside its `binMutations` method for its entire lifespan; this cache is never refreshed or updated to sync up with the static collection of TabletLocators. > # Every hour, the static collection of TabletLocators clears itself. The next call to get a TabletLocator from the static collection allocates a new TabletLocator. Unfortunately, the TabletServerBatchWriter does not reflect this change and continues to use the old, locally cached TabletLocator. > # Tablet T1 splits into T2 and T3, which closes T1. As such, it no longer exists and the tablet server that receives the entries meant to go to T1 all fail to write because T1 is closed. > # The TabletServerBatchWriter receives the response from the tablet server that all entries failed to write. It invalidates the cache of the *new* TabletLocator obtained from the static collection of TabletLocators. The old TabletLocator that is cached locally does not get invalidated. > # The TabletServerBatchWriter re-queues the failed entries and tries to write them to the same closed tablet T1, because it is still looking up tablets using the old TabletLocator. > This behavior subsumes the circumstances William wrote about in the thread he mentioned. The problem would occur as a result of either splits or major compactions. It would only stop the BatchWriter if its entire memory filled up with writes to the same tablet that was closed as a result of a majc or split; otherwise it would just slow down the BatchWriter by failing to write some number of entries with every RPC. > There are a few solutions we can think of. > # Not have the MutationWriter inside the TabletServerBatchWriter locally cache TabletLocators. I suspect this was done for performance reasons, so it's probably not a good solution. > # Have all the MutationWriters clear their cache at the same time the static TabletLocator cache clears. We could store a reference to the Map that each MutationWriter has inside a static synchronized WeakHashMap. The only time the weak map needs to be accessed is: > ## When a MutationWriter is constructed (from constructing a TabletServerBatchWriter), add its new local TabletLocator cache to the weak map. > ## When the static TabletLocator cache is cleared, also clear every map in the weak map. > # Another solution is to make the invalidate calls on the local TabletLocator cache rather than the global static one. If we go this route we should double check the idea to make sure it does not impact the correctness of any other pieces of code that use the cache. > # Perhaps the simplest solution is to put an extra Boolean variable inside the Locators indicating whether they are valid. When they are cleared, their Boolean variables set to false. Before a client uses a locator from cache, it checks its Boolean indicator. > The TimeoutTabletLocator does not help when no timeout is set on the BatchWriter (the default behavior). -- This message was sent by Atlassian JIRA (v6.3.4#6332)