/* * Copyright (C) 2015 SoftIndex LLC. * * Licensed 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 io.datakernel.cube; import io.datakernel.aggregation.AggregationChunkStorage; import io.datakernel.aggregation.LocalFsChunkStorage; import io.datakernel.aggregation.fieldtype.FieldTypes; import io.datakernel.async.IgnoreCompletionCallback; import io.datakernel.async.ResultCallbackFuture; import io.datakernel.codegen.DefiningClassLoader; import io.datakernel.eventloop.Eventloop; import io.datakernel.logfs.LogManager; import io.datakernel.logfs.LogToCubeMetadataStorage; import io.datakernel.logfs.LogToCubeRunner; import io.datakernel.stream.StreamConsumers; import io.datakernel.stream.StreamProducers; import org.jooq.Configuration; import org.jooq.SQLDialect; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import java.io.File; import java.nio.file.Path; import java.util.*; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import static io.datakernel.aggregation.AggregationPredicates.alwaysTrue; import static io.datakernel.aggregation.AggregationPredicates.has; import static io.datakernel.aggregation.fieldtype.FieldTypes.ofDouble; import static io.datakernel.aggregation.fieldtype.FieldTypes.ofLong; import static io.datakernel.aggregation.measure.Measures.sum; import static io.datakernel.cube.Cube.AggregationConfig.id; import static io.datakernel.cube.CubeTestUtils.*; import static io.datakernel.eventloop.FatalErrorHandlers.rethrowOnAnyError; import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; @SuppressWarnings("ArraysAsListWithZeroOrOneArgument") public class CubeIntegrationTest { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); private static final String DATABASE_PROPERTIES_PATH = "test.properties"; private static final SQLDialect DATABASE_DIALECT = SQLDialect.MYSQL; private static final String LOG_PARTITION_NAME = "partitionA"; private static final List<String> LOG_PARTITIONS = asList(LOG_PARTITION_NAME); private static final String LOG_NAME = "testlog"; private static Cube getCube(Eventloop eventloop, ExecutorService executorService, DefiningClassLoader classLoader, CubeMetadataStorage cubeMetadataStorage, AggregationChunkStorage aggregationChunkStorage) { return Cube.create(eventloop, executorService, classLoader, cubeMetadataStorage, aggregationChunkStorage) .withDimension("date", FieldTypes.ofLocalDate()) .withDimension("advertiser", FieldTypes.ofInt()) .withDimension("campaign", FieldTypes.ofInt()) .withDimension("banner", FieldTypes.ofInt()) .withRelation("campaign", "advertiser") .withRelation("banner", "campaign") .withMeasure("impressions", sum(ofLong())) .withMeasure("clicks", sum(ofLong())) .withMeasure("conversions", sum(ofLong())) .withMeasure("revenue", sum(ofDouble())) .withAggregation(id("detailed") .withDimensions("date", "advertiser", "campaign", "banner") .withMeasures("impressions", "clicks", "conversions", "revenue")) .withAggregation(id("date") .withDimensions("date") .withMeasures("impressions", "clicks", "conversions", "revenue")) .withAggregation(id("advertiser") .withDimensions("advertiser") .withMeasures("impressions", "clicks", "conversions", "revenue")); } @SuppressWarnings("ConstantConditions") @Test public void test() throws Exception { ExecutorService executor = Executors.newCachedThreadPool(); DefiningClassLoader classLoader = DefiningClassLoader.create(); Eventloop eventloop = Eventloop.create().withFatalErrorHandler(rethrowOnAnyError()); Path aggregationsDir = temporaryFolder.newFolder().toPath(); Path logsDir = temporaryFolder.newFolder().toPath(); Configuration jooqConfiguration = getJooqConfiguration(DATABASE_PROPERTIES_PATH, DATABASE_DIALECT); AggregationChunkStorage aggregationChunkStorage = LocalFsChunkStorage.create(eventloop, executor, aggregationsDir); CubeMetadataStorageSql cubeMetadataStorageSql = CubeMetadataStorageSql.create(eventloop, executor, jooqConfiguration, "processId"); LogToCubeMetadataStorage logToCubeMetadataStorage = getLogToCubeMetadataStorage(eventloop, executor, jooqConfiguration, cubeMetadataStorageSql); Cube cube = getCube(eventloop, executor, classLoader, cubeMetadataStorageSql, aggregationChunkStorage); LogManager<LogItem> logManager = getLogManager(LogItem.class, eventloop, executor, classLoader, logsDir); LogToCubeRunner<LogItem> logToCubeRunner = LogToCubeRunner.create(eventloop, cube, logManager, LogItemSplitter.factory(), LOG_NAME, LOG_PARTITIONS, logToCubeMetadataStorage); // Save and aggregate logs List<LogItem> listOfRandomLogItems = LogItem.getListOfRandomLogItems(100); StreamProducers.OfIterator<LogItem> producerOfRandomLogItems = new StreamProducers.OfIterator<>(eventloop, listOfRandomLogItems.iterator()); producerOfRandomLogItems.streamTo(logManager.consumer(LOG_PARTITION_NAME)); eventloop.run(); logToCubeRunner.processLog(IgnoreCompletionCallback.create()); eventloop.run(); List<LogItem> listOfRandomLogItems2 = LogItem.getListOfRandomLogItems(300); producerOfRandomLogItems = new StreamProducers.OfIterator<>(eventloop, listOfRandomLogItems2.iterator()); producerOfRandomLogItems.streamTo(logManager.consumer(LOG_PARTITION_NAME)); eventloop.run(); logToCubeRunner.processLog(IgnoreCompletionCallback.create()); eventloop.run(); List<LogItem> listOfRandomLogItems3 = LogItem.getListOfRandomLogItems(50); producerOfRandomLogItems = new StreamProducers.OfIterator<>(eventloop, listOfRandomLogItems3.iterator()); producerOfRandomLogItems.streamTo(logManager.consumer(LOG_PARTITION_NAME)); eventloop.run(); logToCubeRunner.processLog(IgnoreCompletionCallback.create()); eventloop.run(); // Load metadata cube.loadChunks(IgnoreCompletionCallback.create()); eventloop.run(); StreamConsumers.ToList<LogItem> queryResultConsumer = new StreamConsumers.ToList<>(eventloop); cube.queryRawStream(asList("date"), asList("clicks"), alwaysTrue(), LogItem.class, DefiningClassLoader.create(classLoader)).streamTo(queryResultConsumer); eventloop.run(); // Aggregate manually Map<Integer, Long> map = new HashMap<>(); aggregateToMap(map, listOfRandomLogItems); aggregateToMap(map, listOfRandomLogItems2); aggregateToMap(map, listOfRandomLogItems3); // Check query results for (LogItem logItem : queryResultConsumer.getList()) { assertEquals(logItem.clicks, map.get(logItem.date).longValue()); } // Consolidate ResultCallbackFuture<Boolean> callback = ResultCallbackFuture.create(); cube.consolidate(callback); eventloop.run(); boolean consolidated = callback.isDone() ? callback.get() : false; assertEquals(true, consolidated); // Load metadata cube.loadChunks(IgnoreCompletionCallback.create()); eventloop.run(); // Query queryResultConsumer = new StreamConsumers.ToList<>(eventloop); cube.queryRawStream(asList("date"), asList("clicks"), alwaysTrue(), LogItem.class, DefiningClassLoader.create(classLoader)).streamTo(queryResultConsumer); eventloop.run(); // Check query results for (LogItem logItem : queryResultConsumer.getList()) { assertEquals(logItem.clicks, map.get(logItem.date).longValue()); } // Check files in aggregations directory Set<String> actualChunkFileNames = new TreeSet<>(); for (File file : aggregationsDir.toFile().listFiles()) { actualChunkFileNames.add(file.getName()); } Set<String> expectedChunkFileNames = new TreeSet<>(); for (int i = 1; i <= 12; ++i) { expectedChunkFileNames.add(i + ".log"); } assertEquals(expectedChunkFileNames, actualChunkFileNames); } private void aggregateToMap(Map<Integer, Long> map, List<LogItem> logItems) { for (LogItem logItem : logItems) { int date = logItem.date; long clicks = logItem.clicks; if (map.get(date) == null) { map.put(date, clicks); } else { Long clicksForDate = map.get(date); map.put(date, clicksForDate + clicks); } } } }