Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,14 @@
package org.apache.cassandra.cql3.statements;

import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
import java.util.regex.Pattern;

import javax.annotation.Nullable;

import com.google.common.annotations.VisibleForTesting;

import com.codahale.metrics.Clock;
import org.apache.cassandra.cql3.QualifiedName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -33,10 +37,24 @@ public class PropertyDefinitions
public static final String MULTIPLE_DEFINITIONS_ERROR = "Multiple definitions for property '%s'";
private static final Pattern POSITIVE_PATTERN = Pattern.compile("(1|true|yes)");
private static final Pattern NEGATIVE_PATTERN = Pattern.compile("(0|false|no)");

private static final Map<String, Long> OBSOLETE_PROPERTY_LAST_LOG_TIMES = new ConcurrentHashMap<>();
private static final long OBSOLETE_PROPERTY_LOG_INTERVAL_MS = 30_000;

protected static final Logger logger = LoggerFactory.getLogger(PropertyDefinitions.class);

protected final Map<String, Object> properties = new HashMap<>();
// Wrapper around System.currentTimeMillis() to simplify unit testing.
private final Clock clock;

@VisibleForTesting
PropertyDefinitions(Clock clock)
{
this.clock = clock;
}

public PropertyDefinitions() {
this.clock = Clock.defaultClock();
}

public void addProperty(String name, String value) throws SyntaxException
{
Expand Down Expand Up @@ -64,7 +82,15 @@ public void validate(Set<String> keywords, Set<String> obsolete) throws SyntaxEx
continue;

if (obsolete.contains(name))
logger.warn("Ignoring obsolete property {}", name);
{
long now = clock.getTime();
Long lastLogged = OBSOLETE_PROPERTY_LAST_LOG_TIMES.putIfAbsent(name, now);

if (lastLogged == null || (now - lastLogged) >= OBSOLETE_PROPERTY_LOG_INTERVAL_MS)
{
logger.warn("Ignoring obsolete property {}", name);
}
}
else
throw new SyntaxException(String.format("Unknown property '%s'", name));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,24 +20,62 @@
*/
package org.apache.cassandra.cql3.statements;

import java.lang.reflect.Field;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.BiConsumer;
import java.util.stream.Collectors;

import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.LoggerFactory;

import ch.qos.logback.classic.Level;
import ch.qos.logback.classic.Logger;
import ch.qos.logback.classic.spi.ILoggingEvent;
import ch.qos.logback.core.read.ListAppender;
import com.codahale.metrics.Clock;
import org.apache.cassandra.cql3.QualifiedName;
import org.apache.cassandra.exceptions.SyntaxException;
import org.assertj.core.api.Assertions;

import static org.junit.Assert.assertEquals;

import static org.apache.cassandra.cql3.statements.PropertyDefinitions.parseBoolean;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;

public class PropertyDefinitionsTest
{

private ListAppender<ILoggingEvent> logAppender;
private Logger logger;
private Field lastLoggedTimeField;

@Before
public void setup() throws Exception
{
logger = (Logger) LoggerFactory.getLogger(PropertyDefinitions.class);
logAppender = new ListAppender<>();
logAppender.start();
logger.addAppender(logAppender);

lastLoggedTimeField = PropertyDefinitions.class.getDeclaredField("OBSOLETE_PROPERTY_LAST_LOG_TIMES");
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think you need this any longer, since you are using constructors to get the same level of access as before.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This field needs to be static, so creating it by constructor is not an option I think

lastLoggedTimeField.setAccessible(true);
}

@After
public void cleanup() throws Exception
{
logger.detachAppender(logAppender);

// Clear map
Map<String, Long> lastLoggedTime = (Map<String, Long>) lastLoggedTimeField.get(null);
lastLoggedTime.clear();
}

@Test
public void testPostiveBooleanParsing()
{
Expand Down Expand Up @@ -110,10 +148,108 @@ private <V> void testAddProperty(V oldValue, V newValue, BiConsumer<PropertyDefi
String key = "k";
PropertyDefinitions pd = new PropertyDefinitions();
adder.accept(pd, oldValue);

Assertions.assertThat(pd.getProperty(key)).isEqualTo(oldValue);
Assertions.assertThatThrownBy(() -> adder.accept(pd, newValue))
.isInstanceOf(SyntaxException.class)
.hasMessageContaining(String.format(PropertyDefinitions.MULTIPLE_DEFINITIONS_ERROR, key));
Assertions.assertThat(pd.getProperty(key)).isEqualTo(oldValue);
}

@Test
public void testObsoletePropertyWarningRateLimiting() throws Exception
{
String obsoleteProperty = "old_prop";
TestClock testClock = new TestClock();

// First call - should log
testClock.setTime(0);
PropertyDefinitions pd1 = new PropertyDefinitions(new TestClock());
pd1.addProperty(obsoleteProperty, "value1");
pd1.validate(Collections.emptySet(), Collections.singleton(obsoleteProperty));

List<ILoggingEvent> logs1 = getWarningLogs(obsoleteProperty);
assertEquals("First call should log warning", 1, logs1.size());
assertTrue(logs1.get(0).getFormattedMessage().contains("Ignoring obsolete property"));

logAppender.list.clear();

// Second call immediately - should NOT log (within 30 seconds)
testClock.setTime(100); // Only 100ms passed
PropertyDefinitions pd2 = new PropertyDefinitions(testClock);
pd2.addProperty(obsoleteProperty, "value2");
pd2.validate(Collections.emptySet(), Collections.singleton(obsoleteProperty));

List<ILoggingEvent> logs2 = getWarningLogs(obsoleteProperty);
assertEquals("Second call within 100ms should not log", 0, logs2.size());

logAppender.list.clear();

// Advance time by 30 seconds and try again - should log
testClock.setTime(30_100); // 30 seconds + 100ms from start

PropertyDefinitions pd3 = new PropertyDefinitions(testClock);
pd3.addProperty(obsoleteProperty, "value3");
pd3.validate(Collections.emptySet(), Collections.singleton(obsoleteProperty));

List<ILoggingEvent> logs3 = getWarningLogs(obsoleteProperty);
assertEquals("Third call after 30s should log again", 1, logs3.size());
}

@Test
public void testObsoletePropertyWarningPerProperty()
{
String obsoleteProperty1 = "old_prop1";
String obsoleteProperty2 = "old_prop2";

// First property - should log
PropertyDefinitions pd1 = new PropertyDefinitions();
pd1.addProperty(obsoleteProperty1, "value1");
pd1.validate(Collections.emptySet(), Collections.singleton(obsoleteProperty1));

List<ILoggingEvent> logs1 = getWarningLogs(obsoleteProperty1);
assertEquals("First property should log", 1, logs1.size());

logAppender.list.clear();

// Second property immediately - should ALSO log (different property)
PropertyDefinitions pd2 = new PropertyDefinitions();
pd2.addProperty(obsoleteProperty2, "value2");
pd2.validate(Collections.emptySet(), Collections.singleton(obsoleteProperty2));

List<ILoggingEvent> logs2 = getWarningLogs(obsoleteProperty2);
assertEquals("Different property should log independently", 1, logs2.size());
}

private List<ILoggingEvent> getWarningLogs(String propertyName)
{
return logAppender.list.stream()
.filter(event -> event.getLevel() == Level.WARN)
.filter(event -> event.getFormattedMessage().contains(propertyName))
.collect(Collectors.toList());
}


// Custom Clock implementation for testing
private static class TestClock extends Clock
{
private long currentTime = 0;

@Override
public long getTick()
{
return currentTime * 1_000_000; // Convert ms to ns
}

@Override
public long getTime()
{
return currentTime;
}

public void setTime(long timeMs)
{
this.currentTime = timeMs;
}
}
}