[
https://issues.apache.org/jira/browse/CASSANDRA-10825?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15511827#comment-15511827
]
Edward Capriolo edited comment on CASSANDRA-10825 at 9/22/16 1:58 AM:
----------------------------------------------------------------------
Looking at this I thinkthere is a serious problem here:
In StorageProxy i see hintsInProgress to be a system that always returns 0,
unless I really do not understand the idiom. I am Working on a fix that uses a
codahale meter.
java.lang.AssertionError: expected:<2> but was:<0>
{noformat}
package org.apache.cassandra.service;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.cassandra.OrderedJUnit4ClassRunner;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.locator.IEndpointSnitch;
import org.apache.cassandra.locator.PropertyFileSnitch;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.runner.RunWith;
import com.google.common.cache.CacheLoader;
@RunWith(OrderedJUnit4ClassRunner.class)
public class StorageProxyOverloadTests
{
@BeforeClass
public static void setUp() throws ConfigurationException
{
DatabaseDescriptor.daemonInitialization();
IEndpointSnitch snitch = new PropertyFileSnitch();
DatabaseDescriptor.setEndpointSnitch(snitch);
Keyspace.setInitialized();
}
@Test
public void abc() throws UnknownHostException{
InetAddress me = InetAddress.getLocalHost();
StorageProxy.instance.setMaxHintsInProgress(-1);
Assert.assertEquals(0, StorageProxy.getHintsInProgressFor(me).get());
StorageProxy.getHintsInProgressFor(me).incrementAndGet();
StorageProxy.getHintsInProgressFor(me).incrementAndGet();
StorageProxy.getHintsInProgressFor(me).decrementAndGet();
StorageProxy.getHintsInProgressFor(me).incrementAndGet();
Assert.assertEquals(0, StorageProxy.getHintsInProgressFor(me).get());
Assert.assertTrue(StorageProxy.shouldHint(me));
StorageProxy.checkHintOverload(me);
}
@Test
public void def() throws Exception{
CacheLoader<Integer, AtomicInteger> hintsInProgress = new
CacheLoader<Integer, AtomicInteger>()
{
public AtomicInteger load(Integer i)
{
return new AtomicInteger(0);
}
};
hintsInProgress.load(1).incrementAndGet();
hintsInProgress.load(1).incrementAndGet();
Assert.assertEquals(2, hintsInProgress.load(1));
}
}
{noformat}
{noformat}
public void def() throws Exception{
CacheLoader<Integer, AtomicInteger> hintsInProgress = new
CacheLoader<Integer, AtomicInteger>()
{
public AtomicInteger load(Integer i)
{
return new AtomicInteger(0);
}
};
hintsInProgress.load(1).incrementAndGet();
hintsInProgress.load(1).incrementAndGet();
Assert.assertEquals(2, hintsInProgress.load(1));
}
{noformat}
If someone smart is looking make sure I am not on crazy pills.
was (Author: appodictic):
Looking at this I thinkthere is a serious problem here:
In StorageProxy i see hintsInProgress to be a system that always returns 0,
unless I really do not understand the idiom. I am Working on a fix that uses a
codahale meter.
java.lang.AssertionError: expected:<2> but was:<0>
{noformat}
package org.apache.cassandra.service;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.cassandra.OrderedJUnit4ClassRunner;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.locator.IEndpointSnitch;
import org.apache.cassandra.locator.PropertyFileSnitch;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.runner.RunWith;
import com.google.common.cache.CacheLoader;
@RunWith(OrderedJUnit4ClassRunner.class)
public class StorageProxyOverloadTests
{
@BeforeClass
public static void setUp() throws ConfigurationException
{
DatabaseDescriptor.daemonInitialization();
IEndpointSnitch snitch = new PropertyFileSnitch();
DatabaseDescriptor.setEndpointSnitch(snitch);
Keyspace.setInitialized();
}
@Test
public void abc() throws UnknownHostException{
InetAddress me = InetAddress.getLocalHost();
StorageProxy.instance.setMaxHintsInProgress(-1);
Assert.assertEquals(0, StorageProxy.getHintsInProgressFor(me).get());
StorageProxy.getHintsInProgressFor(me).incrementAndGet();
StorageProxy.getHintsInProgressFor(me).incrementAndGet();
StorageProxy.getHintsInProgressFor(me).decrementAndGet();
StorageProxy.getHintsInProgressFor(me).incrementAndGet();
Assert.assertEquals(0, StorageProxy.getHintsInProgressFor(me).get());
Assert.assertTrue(StorageProxy.shouldHint(me));
StorageProxy.checkHintOverload(me);
}
@Test
public void def() throws Exception{
CacheLoader<Integer, AtomicInteger> hintsInProgress = new
CacheLoader<Integer, AtomicInteger>()
{
public AtomicInteger load(Integer i)
{
return new AtomicInteger(0);
}
};
hintsInProgress.load(1).incrementAndGet();
hintsInProgress.load(1).incrementAndGet();
Assert.assertEquals(2, hintsInProgress.load(1));
}
}
{noformat}
{noformat}
public void def() throws Exception{
CacheLoader<Integer, AtomicInteger> hintsInProgress = new
CacheLoader<Integer, AtomicInteger>()
{
public AtomicInteger load(Integer i)
{
return new AtomicInteger(0);
}
};
hintsInProgress.load(1).incrementAndGet();
hintsInProgress.load(1).incrementAndGet();
Assert.assertEquals(2, hintsInProgress.load(1));
}
{noformat}
If someone smart is looking make sure I am not on crazy pills.
> OverloadedException is untested
> -------------------------------
>
> Key: CASSANDRA-10825
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10825
> Project: Cassandra
> Issue Type: Bug
> Components: Local Write-Read Paths
> Reporter: Ariel Weisberg
> Assignee: Edward Capriolo
>
> If you grep test/src and cassandra-dtest you will find that the string
> OverloadedException doesn't appear anywhere.
> In CASSANDRA-10477 it was found that there were cases where Paxos should
> back-pressure and throw OverloadedException but didn't.
> If OverloadedException is used for functional purposes then we should test
> that it is thrown under expected conditions. If there are behaviors driven by
> catching or tracking OverloadedException we should test those as well.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)