Skip to content

Commit

Permalink
tests
Browse files Browse the repository at this point in the history
  • Loading branch information
clintropolis committed Aug 16, 2023
1 parent 799c544 commit f76b07d
Show file tree
Hide file tree
Showing 6 changed files with 167 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,7 @@
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executor;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;

Expand Down Expand Up @@ -1062,6 +1063,29 @@ public void testStaleOffsetsNegativeLagNotEmitted() throws Exception
Assert.assertEquals(0, emitter.getEvents().size());
}

@Test
public void testScheduleReporting()
{
EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes();
DruidMonitorSchedulerConfig config = new DruidMonitorSchedulerConfig();
EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(config).times(2);
ScheduledExecutorService executorService = EasyMock.createMock(ScheduledExecutorService.class);
EasyMock.expect(executorService.scheduleWithFixedDelay(EasyMock.anyObject(), EasyMock.eq(86415000L), EasyMock.eq(300000L), EasyMock.eq(TimeUnit.MILLISECONDS))).andReturn(EasyMock.createMock(ScheduledFuture.class)).once();
EasyMock.expect(executorService.scheduleAtFixedRate(EasyMock.anyObject(), EasyMock.eq(86425000L), EasyMock.eq(config.getEmissionDuration().getMillis()), EasyMock.eq(TimeUnit.MILLISECONDS))).andReturn(EasyMock.createMock(ScheduledFuture.class)).times(2);

EasyMock.replay(executorService, spec);
final BaseTestSeekableStreamSupervisor supervisor = new BaseTestSeekableStreamSupervisor()
{
@Override
public LagStats computeLagStats()
{
return new LagStats(0, 0, 0);
}
};
supervisor.scheduleReporting(executorService);
EasyMock.verify(executorService, spec);
}

private List<Event> filterMetrics(List<Event> events, List<String> whitelist)
{
List<Event> result = events.stream()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,7 @@
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
Expand Down Expand Up @@ -276,6 +277,10 @@ private static void validateRowValues(
throw notEqualValidationException(dim1Name, vals1, vals2);
}
}
} else if (vals1 instanceof Object[]) {
if (!Arrays.deepEquals((Object[]) vals1, (Object[]) vals2)) {
throw notEqualValidationException(dim1Name, vals1, vals2);
}
} else {
if (!Objects.equals(vals1, vals2)) {
throw notEqualValidationException(dim1Name, vals1, vals2);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -326,6 +326,13 @@ public int getParallelism()
// fixed so same behavior across all test environments
return 4;
}

@Override
public int getDefaultMaxQueryParallelism()
{
// fixed so same behavior across all test environments
return 4;
}
},
ForkJoinPool.commonPool(),
QueryStackTests.DEFAULT_NOOP_SCHEDULER,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2837,6 +2837,13 @@ public int getParallelism()
// fixed so same behavior across all test environments
return 4;
}

@Override
public int getDefaultMaxQueryParallelism()
{
// fixed so same behavior across all test environments
return 4;
}
},
ForkJoinPool.commonPool(),
new QueryScheduler(
Expand Down
14 changes: 14 additions & 0 deletions services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,10 @@
import org.apache.druid.collections.bitmap.ImmutableBitmap;
import org.apache.druid.collections.bitmap.RoaringBitmapFactory;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.guice.DruidProcessingModule;
import org.apache.druid.guice.NestedDataModule;
import org.apache.druid.guice.QueryRunnerFactoryModule;
import org.apache.druid.guice.QueryableModule;
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.StringUtils;
Expand Down Expand Up @@ -215,6 +218,17 @@ public void testDumpNestedColumnPath() throws Exception
}
}

@Test
public void testGetModules()
{
DumpSegment dumpSegment = new DumpSegment();
List<?> modules = dumpSegment.getModules();
Assert.assertTrue(modules.stream().anyMatch(x -> x instanceof DruidProcessingModule));
Assert.assertTrue(modules.stream().anyMatch(x -> x instanceof QueryableModule));
Assert.assertTrue(modules.stream().anyMatch(x -> x instanceof QueryRunnerFactoryModule));
}


public static List<Segment> createSegments(
AggregationTestHelper helper,
TemporaryFolder tempFolder,
Expand Down
110 changes: 110 additions & 0 deletions services/src/test/java/org/apache/druid/cli/ValidateSegmentsTest.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
/*
* 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.druid.cli;

import com.google.inject.Injector;
import org.apache.druid.data.input.ResourceInputSource;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.guice.DruidProcessingModule;
import org.apache.druid.guice.QueryRunnerFactoryModule;
import org.apache.druid.guice.QueryableModule;
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
import org.apache.druid.query.NestedDataTestUtils;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.segment.IndexBuilder;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.mockito.Mockito;

import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;

public class ValidateSegmentsTest extends InitializedNullHandlingTest
{
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();

@Test
public void testValidateSegments() throws IOException
{

JsonInputFormat inputFormat = new JsonInputFormat(
JSONPathSpec.DEFAULT,
null,
null,
null,
null
);
IndexBuilder bob = IndexBuilder.create()
.tmpDir(temporaryFolder.newFolder())
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.schema(
new IncrementalIndexSchema.Builder()
.withTimestampSpec(NestedDataTestUtils.AUTO_SCHEMA.getTimestampSpec())
.withDimensionsSpec(NestedDataTestUtils.AUTO_SCHEMA.getDimensionsSpec())
.withMetrics(
new CountAggregatorFactory("cnt")
)
.withRollup(false)
.build()
)
.inputSource(
ResourceInputSource.of(
NestedDataTestUtils.class.getClassLoader(),
NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE
)
)
.inputFormat(inputFormat)
.inputTmpDir(temporaryFolder.newFolder());

final File segment1 = bob.buildMMappedIndexFile();
final File segment2 = bob.buildMMappedIndexFile();
final Injector injector = Mockito.mock(Injector.class);
Mockito.when(injector.getInstance(IndexIO.class)).thenReturn(bob.getIndexIO());
ValidateSegments validator = new ValidateSegments() {
@Override
public Injector makeInjector()
{
return injector;
}
};
validator.directories = Arrays.asList(segment1.getAbsolutePath(), segment2.getAbsolutePath());
// if this doesn't pass, it throws a runtime exception, which would fail the test
validator.run();
}

@Test
public void testGetModules()
{
ValidateSegments validator = new ValidateSegments();
List<?> modules = validator.getModules();
Assert.assertTrue(modules.stream().anyMatch(x -> x instanceof DruidProcessingModule));
Assert.assertTrue(modules.stream().anyMatch(x -> x instanceof QueryableModule));
Assert.assertTrue(modules.stream().anyMatch(x -> x instanceof QueryRunnerFactoryModule));
}
}

0 comments on commit f76b07d

Please sign in to comment.