...
 
Commits (8)
    https://gitcode.net/apache/iotdb/-/commit/cbc7c3fb7f51669e842e299a354db836f76bd4b5 Revert "[To rel/1.1][IOTDB-6106] Fixed the timeout parameter not working in t... 2023-08-16T14:14:22+08:00 Potato tanxinyu@apache.org https://gitcode.net/apache/iotdb/-/commit/16e96283dd7d4ffc67fb6f4f5f9225ffb3a89c8e [To rel/1.1] Override existing jar instead of deleting it when registering UDF 2023-08-19T11:08:20+08:00 Liao Lanyu 1435078631@qq.com https://gitcode.net/apache/iotdb/-/commit/780875b1ad8346e7cb5a035163d77f1254efc2bf [To rel/1.1][IOTDB-6087] Implement stream interface of Mods read 2023-08-23T09:57:22+08:00 Weihao Li 60659567+Wei-hao-Li@users.noreply.github.com https://gitcode.net/apache/iotdb/-/commit/5c7b113f6eeee38d45609d72eecda1611a45553b [To rel/1.1] cp commits from master to solve the mods file OOM (#10928) 2023-08-23T17:54:02+08:00 Zhijia Cao caozhijia@126.com https://gitcode.net/apache/iotdb/-/commit/517d4d50fef9ab1a8b80b457140f15732867c699 [Metric] Fix flush point statistics (#10915) (cherry picked from commit... 2023-08-25T14:50:06+08:00 ZhangHongYin 46039728+SpriCoder@users.noreply.github.com [Metric] Fix flush point statistics (#10915) (cherry picked from commit <a href="/apache/iotdb/-/commit/1dce787c39e14973918ee390c470b901b68be73e" data-original="1dce787c39e14973918ee390c470b901b68be73e" data-link="false" data-link-reference="false" data-project="48792" data-commit="1dce787c39e14973918ee390c470b901b68be73e" data-reference-type="commit" data-container="body" data-placement="top" title="[Metric] Fix flush point statistics (#10915)" class="gfm gfm-commit has-tooltip">1dce787c</a>) (#10946) https://gitcode.net/apache/iotdb/-/commit/c1bec1bb82bccc8d786e16b77443c04a24115a1c [To rel/1.1] Rewrite compaction memory estimator (#10965) 2023-08-29T11:49:07+08:00 shuwenwei 55970239+shuwenwei@users.noreply.github.com https://gitcode.net/apache/iotdb/-/commit/43f8ceae2afc834df871e3bdf91a543c23ee6927 [To rel/1.1] Fix point priority reader index out of bound (#10933) 2023-08-29T11:49:53+08:00 shuwenwei 55970239+shuwenwei@users.noreply.github.com https://gitcode.net/apache/iotdb/-/commit/e585544c909092bc72fe6d7723ff07d870f521d5 [To rel/1.1] add lazy page reader for aligned page reader to avoid huge memor... 2023-08-29T11:50:56+08:00 Zhang.Jinrui xingtanzjr@gmail.com [To rel/1.1] add lazy page reader for aligned page reader to avoid huge memory cost when reading rows of aligned timeseries (#10969)
......@@ -56,7 +56,6 @@ public class AsyncIoTConsensusServiceClient extends IoTConsensusIService.AsyncCl
protocolFactory,
tClientManager,
TNonblockingSocketWrapper.wrap(endpoint.getIp(), endpoint.getPort(), connectionTimeout));
setTimeout(connectionTimeout);
this.endpoint = endpoint;
this.clientManager = clientManager;
}
......
......@@ -32,6 +32,7 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import java.io.File;
import java.sql.Connection;
import java.sql.ResultSet;
import java.sql.SQLException;
......@@ -54,6 +55,16 @@ public class IoTDBUDFManagementIT {
private static final String FUNCTION_TYPE_BUILTIN_UDTF = "built-in UDTF";
private static final String FUNCTION_TYPE_EXTERNAL_UDTF = "external UDTF";
private static final String UDF_LIB_PREFIX =
System.getProperty("user.dir")
+ File.separator
+ "target"
+ File.separator
+ "test-classes"
+ File.separator;
private static final String UDF_JAR_PREFIX = new File(UDF_LIB_PREFIX).toURI().toString();
@Before
public void setUp() throws Exception {
EnvFactory.getEnv().initClusterEnvironment();
......@@ -208,6 +219,35 @@ public class IoTDBUDFManagementIT {
}
}
@Test
public void testCreateFunctionWithURI() throws SQLException {
try (Connection connection = EnvFactory.getEnv().getConnection();
Statement statement = connection.createStatement()) {
statement.execute(
String.format(
"create function udf as 'org.apache.iotdb.db.query.udf.example.Adder' using URI '%s'",
UDF_JAR_PREFIX + "udf-example.jar"));
statement.execute(
String.format(
"create function udf1 as 'org.apache.iotdb.db.query.udf.example.Adder' using URI '%s'",
UDF_JAR_PREFIX + "udf-example.jar"));
try (ResultSet resultSet = statement.executeQuery("show functions")) {
int count = 0;
while (resultSet.next()) {
++count;
}
Assert.assertEquals(2 + NATIVE_FUNCTIONS_COUNT + BUILTIN_FUNCTIONS_COUNT, count);
assertEquals(3, resultSet.getMetaData().getColumnCount());
statement.execute("drop function udf");
statement.execute("drop function udf1");
} catch (Exception e) {
fail();
}
}
}
@Test
public void testCreateFunctionWithInvalidURI() {
try (Connection connection = EnvFactory.getEnv().getConnection();
......@@ -215,8 +255,8 @@ public class IoTDBUDFManagementIT {
try {
statement.execute(
String.format(
"create stateless trigger %s before insert on root.test.stateless.* as '%s' using URI '%s' with (\"name\"=\"%s\")",
"a", "org.apache.iotdb.test", "", "test"));
"create function udf as 'org.apache.iotdb.db.query.udf.example.Adder' using URI '%s'",
""));
fail();
} catch (Exception e) {
assertTrue(e.getMessage().contains("URI"));
......@@ -225,8 +265,8 @@ public class IoTDBUDFManagementIT {
try {
statement.execute(
String.format(
"create stateless trigger %s before insert on root.test.stateless.* as '%s' using URI '%s' with (\"name\"=\"%s\")",
"a", "org.apache.iotdb.test", "file:///data/udf/upload-test.jar", "test"));
"create function udf as 'org.apache.iotdb.db.query.udf.example.Adder' using URI '%s'",
"file:///data/udf/upload-test.jar"));
fail();
} catch (Exception e) {
assertTrue(e.getMessage().contains("URI"));
......
......@@ -271,9 +271,9 @@ public abstract class AbstractMetricService {
/** GetOrCreateCounter with internal report. */
public Counter getOrCreateCounterWithInternalReport(
String metric, MetricLevel metricLevel, String... tags) {
String metric, MetricLevel metricLevel, long time, String... tags) {
Counter counter = metricManager.getOrCreateCounter(metric, metricLevel, tags);
internalReporter.writeMetricToIoTDB(counter, metric, tags);
internalReporter.writeMetricToIoTDB(counter, metric, time, tags);
return counter;
}
......@@ -287,69 +287,74 @@ public abstract class AbstractMetricService {
/** GetOrCreateGauge with internal report. */
public Gauge getOrCreateGaugeWithInternalReport(
String metric, MetricLevel metricLevel, String... tags) {
String metric, MetricLevel metricLevel, long time, String... tags) {
Gauge gauge = metricManager.getOrCreateGauge(metric, metricLevel, tags);
internalReporter.writeMetricToIoTDB(gauge, metric, tags);
internalReporter.writeMetricToIoTDB(gauge, metric, time, tags);
return gauge;
}
/** GetOrCreateRate with internal report. */
public Rate getOrCreateRateWithInternalReport(
String metric, MetricLevel metricLevel, String... tags) {
String metric, MetricLevel metricLevel, long time, String... tags) {
Rate rate = metricManager.getOrCreateRate(metric, metricLevel, tags);
internalReporter.writeMetricToIoTDB(rate, metric, tags);
internalReporter.writeMetricToIoTDB(rate, metric, time, tags);
return rate;
}
/** GetOrCreateHistogram with internal report. */
public Histogram getOrCreateHistogramWithInternalReport(
String metric, MetricLevel metricLevel, String... tags) {
String metric, MetricLevel metricLevel, long time, String... tags) {
Histogram histogram = metricManager.getOrCreateHistogram(metric, metricLevel, tags);
internalReporter.writeMetricToIoTDB(histogram, metric, tags);
internalReporter.writeMetricToIoTDB(histogram, metric, time, tags);
return histogram;
}
/** GetOrCreateTimer with internal report. */
public Timer getOrCreateTimerWithInternalReport(
String metric, MetricLevel metricLevel, String... tags) {
String metric, MetricLevel metricLevel, long time, String... tags) {
Timer timer = metricManager.getOrCreateTimer(metric, metricLevel, tags);
internalReporter.writeMetricToIoTDB(timer, metric, tags);
internalReporter.writeMetricToIoTDB(timer, metric, time, tags);
return timer;
}
/** Count with internal report. */
public void countWithInternalReportAsync(
long delta, String metric, MetricLevel metricLevel, String... tags) {
long delta, String metric, MetricLevel metricLevel, long time, String... tags) {
internalReporter.writeMetricToIoTDB(
metricManager.count(delta, metric, metricLevel, tags), metric, tags);
metricManager.count(delta, metric, metricLevel, tags), metric, time, tags);
}
/** Gauge value with internal report */
public void gaugeWithInternalReportAsync(
long value, String metric, MetricLevel metricLevel, String... tags) {
long value, String metric, MetricLevel metricLevel, long time, String... tags) {
internalReporter.writeMetricToIoTDB(
metricManager.gauge(value, metric, metricLevel, tags), metric, tags);
metricManager.gauge(value, metric, metricLevel, tags), metric, time, tags);
}
/** Rate with internal report. */
public void rateWithInternalReportAsync(
long value, String metric, MetricLevel metricLevel, String... tags) {
long value, String metric, MetricLevel metricLevel, long time, String... tags) {
internalReporter.writeMetricToIoTDB(
metricManager.rate(value, metric, metricLevel, tags), metric, tags);
metricManager.rate(value, metric, metricLevel, tags), metric, time, tags);
}
/** Histogram with internal report. */
public void histogramWithInternalReportAsync(
long value, String metric, MetricLevel metricLevel, String... tags) {
long value, String metric, MetricLevel metricLevel, long time, String... tags) {
internalReporter.writeMetricToIoTDB(
metricManager.histogram(value, metric, metricLevel, tags), metric, tags);
metricManager.histogram(value, metric, metricLevel, tags), metric, time, tags);
}
/** Timer with internal report. */
public void timerWithInternalReportAsync(
long delta, TimeUnit timeUnit, String metric, MetricLevel metricLevel, String... tags) {
long delta,
TimeUnit timeUnit,
String metric,
MetricLevel metricLevel,
long time,
String... tags) {
internalReporter.writeMetricToIoTDB(
metricManager.timer(delta, timeUnit, metric, metricLevel, tags), metric, tags);
metricManager.timer(delta, timeUnit, metric, metricLevel, tags), metric, time, tags);
}
public List<Pair<String, String[]>> getAllMetricKeys() {
......
......@@ -36,14 +36,14 @@ public abstract class IoTDBReporter implements Reporter {
*
* @param metric the target metric
* @param name the name of metric
* @param time the target time of metric
* @param tags the tags of metric
*/
public void writeMetricToIoTDB(IMetric metric, String name, String... tags) {
public void writeMetricToIoTDB(IMetric metric, String name, long time, String... tags) {
if (!(metric instanceof DoNothingMetric)) {
Map<String, Object> values = new HashMap<>();
metric.constructValueMap(values);
writeMetricToIoTDB(
values, IoTDBMetricsUtils.generatePath(name, tags), System.currentTimeMillis());
writeMetricToIoTDB(values, IoTDBMetricsUtils.generatePath(name, tags), time);
}
}
......
......@@ -51,7 +51,6 @@ public class AsyncConfigNodeHeartbeatServiceClient extends IConfigNodeRPCService
protocolFactory,
tClientManager,
TNonblockingSocketWrapper.wrap(endpoint.getIp(), endpoint.getPort(), connectionTimeout));
setTimeout(connectionTimeout);
this.endpoint = endpoint;
this.clientManager = clientManager;
}
......
......@@ -55,7 +55,6 @@ public class AsyncConfigNodeIServiceClient extends IConfigNodeRPCService.AsyncCl
protocolFactory,
tClientManager,
TNonblockingSocketWrapper.wrap(endpoint.getIp(), endpoint.getPort(), connectionTimeout));
setTimeout(connectionTimeout);
this.endpoint = endpoint;
this.clientManager = clientManager;
}
......
......@@ -51,7 +51,6 @@ public class AsyncDataNodeHeartbeatServiceClient extends IDataNodeRPCService.Asy
protocolFactory,
tClientManager,
TNonblockingSocketWrapper.wrap(endpoint.getIp(), endpoint.getPort(), connectionTimeout));
setTimeout(connectionTimeout);
this.endpoint = endpoint;
this.clientManager = clientManager;
}
......
......@@ -57,7 +57,6 @@ public class AsyncDataNodeInternalServiceClient extends IDataNodeRPCService.Asyn
protocolFactory,
tClientManager,
TNonblockingSocketWrapper.wrap(endpoint.getIp(), endpoint.getPort(), connectionTimeout));
setTimeout(connectionTimeout);
this.endpoint = endpoint;
this.clientManager = clientManager;
}
......
......@@ -56,7 +56,6 @@ public class AsyncDataNodeMPPDataExchangeServiceClient extends MPPDataExchangeSe
protocolFactory,
tClientManager,
TNonblockingSocketWrapper.wrap(endpoint.getIp(), endpoint.getPort(), connectionTimeout));
setTimeout(connectionTimeout);
this.endpoint = endpoint;
this.clientManager = clientManager;
}
......
......@@ -257,6 +257,10 @@ public class IoTDBConstant {
// compaction mods of previous version (<0.13)
public static final String COMPACTION_MODIFICATION_FILE_NAME_FROM_OLD = "merge.mods";
public static final String SETTLE_SUFFIX = ".settle";
public static final String MODS_SETTLE_FILE_SUFFIX = ".mods.settle";
public static final String BLANK = "";
// write ahead log
public static final String WAL_FILE_PREFIX = "_";
public static final String WAL_FILE_SUFFIX = ".wal";
......
......@@ -223,11 +223,18 @@ public class ExecutableManager {
}
}
/**
* Create and save the file if the specified file does not exist, or this method will override the
* existing file.
*/
protected void saveToDir(ByteBuffer byteBuffer, String destination) throws IOException {
try {
Path path = Paths.get(destination);
Files.deleteIfExists(path);
Files.createFile(path);
if (!Files.exists(path)) {
Files.createFile(path);
}
// FileOutPutStream is not in append mode by default, so the file will be overridden if it
// already exists.
try (FileOutputStream outputStream = new FileOutputStream(destination)) {
outputStream.getChannel().write(byteBuffer);
}
......
......@@ -20,7 +20,6 @@
package org.apache.iotdb.commons.client;
import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
import org.apache.iotdb.commons.client.async.AsyncDataNodeInternalServiceClient;
import org.apache.iotdb.commons.client.exception.BorrowNullClientManagerException;
import org.apache.iotdb.commons.client.exception.ClientManagerException;
......@@ -34,8 +33,6 @@ import org.apache.iotdb.mpp.rpc.thrift.IDataNodeRPCService;
import org.apache.commons.pool2.KeyedObjectPool;
import org.apache.commons.pool2.impl.GenericKeyedObjectPool;
import org.apache.thrift.TException;
import org.apache.thrift.async.AsyncMethodCallback;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
......@@ -43,41 +40,20 @@ import org.junit.Test;
import java.io.IOException;
import java.util.NoSuchElementException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class ClientManagerTest {
private final TEndPoint endPoint = new TEndPoint("localhost", 10730);
private static final int CONNECTION_TIMEOUT = 5_000;
private MockInternalRPCService service;
@SuppressWarnings("java:S2925")
@Before
public void setUp() throws StartupException, TException {
public void setUp() throws StartupException {
service = new MockInternalRPCService(endPoint);
IDataNodeRPCService.Iface processor = mock(IDataNodeRPCService.Iface.class);
// timeout method
when(processor.clearCache())
.thenAnswer(
invocation -> {
Thread.sleep(CONNECTION_TIMEOUT + 1000);
return new TSStatus();
});
// normal method
when(processor.merge())
.thenAnswer(
invocation -> {
Thread.sleep(1000);
return new TSStatus();
});
service.initSyncedServiceImpl(processor);
service.initSyncedServiceImpl(mock(IDataNodeRPCService.Iface.class));
service.start();
}
......@@ -102,8 +78,6 @@ public class ClientManagerTest {
invalidSyncClientReturnTest();
invalidAsyncClientReturnTest();
borrowNullTest();
syncClientTimeoutTest();
asyncClientTimeoutTest();
}
public void normalSyncTest() throws Exception {
......@@ -487,97 +461,6 @@ public class ClientManagerTest {
Assert.assertEquals(0, asyncClusterManager.getPool().getNumIdle(endPoint));
}
public void syncClientTimeoutTest() throws Exception {
// init syncClientManager
ClientManager<TEndPoint, SyncDataNodeInternalServiceClient> syncClusterManager =
(ClientManager<TEndPoint, SyncDataNodeInternalServiceClient>)
new IClientManager.Factory<TEndPoint, SyncDataNodeInternalServiceClient>()
.createClientManager(new TestSyncDataNodeInternalServiceClientPoolFactory());
// normal RPC
try (SyncDataNodeInternalServiceClient syncClient = syncClusterManager.borrowClient(endPoint)) {
syncClient.merge();
} catch (Exception e) {
Assert.fail("There should be no timeout here");
}
Assert.assertEquals(0, syncClusterManager.getPool().getNumActive(endPoint));
Assert.assertEquals(1, syncClusterManager.getPool().getNumIdle(endPoint));
// timeout RPC
try (SyncDataNodeInternalServiceClient syncClient = syncClusterManager.borrowClient(endPoint)) {
syncClient.clearCache();
Assert.fail("A timeout exception should occur here");
} catch (Exception ignored) {
// no handling
}
Assert.assertEquals(0, syncClusterManager.getPool().getNumActive(endPoint));
Assert.assertEquals(0, syncClusterManager.getPool().getNumIdle(endPoint));
syncClusterManager.close();
}
public void asyncClientTimeoutTest() throws Exception {
// init asyncClientManager
ClientManager<TEndPoint, AsyncDataNodeInternalServiceClient> asyncClusterManager =
(ClientManager<TEndPoint, AsyncDataNodeInternalServiceClient>)
new IClientManager.Factory<TEndPoint, AsyncDataNodeInternalServiceClient>()
.createClientManager(new TestAsyncDataNodeInternalServiceClientPoolFactory());
// normal RPC
AsyncDataNodeInternalServiceClient asyncClient = asyncClusterManager.borrowClient(endPoint);
CountDownLatch latch = new CountDownLatch(1);
AtomicBoolean failed = new AtomicBoolean(false);
CountDownLatch finalLatch = latch;
AtomicBoolean finalFailed = failed;
asyncClient.merge(
new AsyncMethodCallback<TSStatus>() {
@Override
public void onComplete(TSStatus response) {
finalLatch.countDown();
}
@Override
public void onError(Exception exception) {
finalFailed.set(true);
finalLatch.countDown();
}
});
latch.await();
if (failed.get()) {
Assert.fail("There should be no timeout here");
}
Assert.assertEquals(0, asyncClusterManager.getPool().getNumActive(endPoint));
Assert.assertEquals(1, asyncClusterManager.getPool().getNumIdle(endPoint));
// timeout RPC
asyncClient = asyncClusterManager.borrowClient(endPoint);
latch = new CountDownLatch(1);
failed = new AtomicBoolean(false);
AtomicBoolean finalFailed1 = failed;
CountDownLatch finalLatch1 = latch;
asyncClient.clearCache(
new AsyncMethodCallback<TSStatus>() {
@Override
public void onComplete(TSStatus response) {
finalFailed1.set(true);
finalLatch1.countDown();
}
@Override
public void onError(Exception exception) {
finalLatch1.countDown();
}
});
latch.await();
if (failed.get()) {
Assert.fail("A timeout exception should occur here");
}
Assert.assertEquals(0, asyncClusterManager.getPool().getNumActive(endPoint));
Assert.assertEquals(0, asyncClusterManager.getPool().getNumIdle(endPoint));
asyncClusterManager.close();
}
public static class TestSyncDataNodeInternalServiceClientPoolFactory
implements IClientPoolFactory<TEndPoint, SyncDataNodeInternalServiceClient> {
......@@ -586,10 +469,7 @@ public class ClientManagerTest {
ClientManager<TEndPoint, SyncDataNodeInternalServiceClient> manager) {
return new GenericKeyedObjectPool<>(
new SyncDataNodeInternalServiceClient.Factory(
manager,
new ThriftClientProperty.Builder()
.setConnectionTimeoutMs(CONNECTION_TIMEOUT)
.build()),
manager, new ThriftClientProperty.Builder().build()),
new ClientPoolProperty.Builder<SyncDataNodeInternalServiceClient>().build().getConfig());
}
}
......@@ -603,7 +483,7 @@ public class ClientManagerTest {
return new GenericKeyedObjectPool<>(
new AsyncDataNodeInternalServiceClient.Factory(
manager,
new ThriftClientProperty.Builder().setConnectionTimeoutMs(CONNECTION_TIMEOUT).build(),
new ThriftClientProperty.Builder().build(),
ThreadName.ASYNC_DATANODE_CLIENT_POOL.getName()),
new ClientPoolProperty.Builder<AsyncDataNodeInternalServiceClient>().build().getConfig());
}
......
......@@ -28,9 +28,16 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.List;
import java.util.regex.Pattern;
import java.util.stream.Stream;
import static org.apache.iotdb.commons.conf.IoTDBConstant.BLANK;
import static org.apache.iotdb.commons.conf.IoTDBConstant.MODS_SETTLE_FILE_SUFFIX;
import static org.apache.iotdb.commons.conf.IoTDBConstant.SETTLE_SUFFIX;
import static org.apache.iotdb.db.engine.compaction.execute.utils.log.CompactionLogger.CROSS_COMPACTION_LOG_NAME_FROM_OLD;
import static org.apache.iotdb.db.engine.compaction.execute.utils.log.CompactionLogger.INNER_COMPACTION_LOG_NAME_SUFFIX_FROM_OLD;
......@@ -87,18 +94,51 @@ public class CompactionRecoverManager {
|| !Pattern.compile("[0-9]*").matcher(timePartitionDir.getName()).matches()) {
continue;
}
File[] compactionLogs =
CompactionLogger.findCompactionLogs(isInnerSpace, timePartitionDir.getPath());
for (File compactionLog : compactionLogs) {
logger.info("Calling compaction recover task.");
new CompactionRecoverTask(
logicalStorageGroupName, dataRegionId, tsFileManager, compactionLog, isInnerSpace)
.doCompaction();
}
// recover temporary files generated during compacted
recoverCompaction(isInnerSpace, timePartitionDir);
// recover temporary files generated during .mods file settled
recoverModSettleFile(timePartitionDir.toPath());
}
}
}
public void recoverModSettleFile(Path timePartitionDir) {
try (Stream<Path> settlesStream = Files.list(timePartitionDir)) {
settlesStream
.filter(path -> path.toString().endsWith(MODS_SETTLE_FILE_SUFFIX))
.forEach(
modsSettle -> {
Path originModFile =
modsSettle.resolveSibling(
modsSettle.getFileName().toString().replace(SETTLE_SUFFIX, BLANK));
try {
if (Files.exists(originModFile)) {
Files.deleteIfExists(modsSettle);
} else {
Files.move(modsSettle, originModFile);
}
} catch (IOException e) {
logger.error(
"recover mods file error on delete origin file or rename mods settle,", e);
}
});
} catch (IOException e) {
logger.error("recover mods file error on list files:{}", timePartitionDir, e);
}
}
public void recoverCompaction(boolean isInnerSpace, File timePartitionDir) {
File[] compactionLogs =
CompactionLogger.findCompactionLogs(isInnerSpace, timePartitionDir.getPath());
for (File compactionLog : compactionLogs) {
logger.info("Calling compaction recover task.");
new CompactionRecoverTask(
logicalStorageGroupName, dataRegionId, tsFileManager, compactionLog, isInnerSpace)
.doCompaction();
}
}
/** Check whether there is old compaction log from previous version (<0.13) and recover it. */
private void recoverCompactionBefore013(boolean isInnerSpace) {
String oldLogName =
......
......@@ -55,6 +55,8 @@ public abstract class AbstractCompactionTask {
protected boolean crossTask;
protected boolean innerSeqTask;
protected long memoryCost = 0L;
public AbstractCompactionTask(
String storageGroupName,
String dataRegionId,
......
......@@ -64,7 +64,6 @@ public class CrossSpaceCompactionTask extends AbstractCompactionTask {
protected List<TsFileResource> holdWriteLockList = new ArrayList<>();
protected double selectedSeqFileSize = 0;
protected double selectedUnseqFileSize = 0;
protected long memoryCost = 0L;
public CrossSpaceCompactionTask(
long timePartition,
......
......@@ -23,16 +23,21 @@ import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.TsFileMetricManager;
import org.apache.iotdb.db.engine.compaction.execute.exception.CompactionExceptionHandler;
import org.apache.iotdb.db.engine.compaction.execute.exception.CompactionMemoryNotEnoughException;
import org.apache.iotdb.db.engine.compaction.execute.performer.ICompactionPerformer;
import org.apache.iotdb.db.engine.compaction.execute.performer.impl.FastCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.execute.task.subtask.FastCompactionTaskSummary;
import org.apache.iotdb.db.engine.compaction.execute.utils.CompactionUtils;
import org.apache.iotdb.db.engine.compaction.execute.utils.log.CompactionLogger;
import org.apache.iotdb.db.engine.compaction.selector.estimator.AbstractInnerSpaceEstimator;
import org.apache.iotdb.db.engine.compaction.selector.estimator.FastCompactionInnerCompactionEstimator;
import org.apache.iotdb.db.engine.compaction.selector.estimator.ReadChunkInnerCompactionEstimator;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.engine.storagegroup.TsFileResourceList;
import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
import org.apache.iotdb.db.rescon.SystemInfo;
import org.apache.iotdb.db.service.metrics.recorder.CompactionMetricsManager;
import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
import org.apache.iotdb.tsfile.exception.write.TsFileNotCompleteException;
......@@ -46,6 +51,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.atomic.AtomicInteger;
public class InnerSpaceCompactionTask extends AbstractCompactionTask {
......@@ -66,6 +72,10 @@ public class InnerSpaceCompactionTask extends AbstractCompactionTask {
protected boolean[] isHoldingReadLock;
protected boolean[] isHoldingWriteLock;
protected long maxModsFileSize;
protected AbstractInnerSpaceEstimator innerSpaceEstimator;
public InnerSpaceCompactionTask(
long timePartition,
TsFileManager tsFileManager,
......@@ -84,6 +94,13 @@ public class InnerSpaceCompactionTask extends AbstractCompactionTask {
this.selectedTsFileResourceList = selectedTsFileResourceList;
this.sequence = sequence;
this.performer = performer;
if (IoTDBDescriptor.getInstance().getConfig().isEnableCompactionMemControl()) {
if (this.performer instanceof ReadChunkInnerCompactionEstimator) {
innerSpaceEstimator = new ReadChunkInnerCompactionEstimator();
} else if (!sequence && this.performer instanceof FastCompactionInnerCompactionEstimator) {
innerSpaceEstimator = new FastCompactionInnerCompactionEstimator();
}
}
isHoldingReadLock = new boolean[selectedTsFileResourceList.size()];
isHoldingWriteLock = new boolean[selectedTsFileResourceList.size()];
for (int i = 0; i < selectedTsFileResourceList.size(); ++i) {
......@@ -314,6 +331,7 @@ public class InnerSpaceCompactionTask extends AbstractCompactionTask {
isSequence());
}
} finally {
SystemInfo.getInstance().resetCompactionMemoryCost(memoryCost);
releaseAllLocksAndResetStatus();
return isSuccess;
}
......@@ -339,6 +357,7 @@ public class InnerSpaceCompactionTask extends AbstractCompactionTask {
sumOfCompactionCount = 0;
maxFileVersion = -1L;
maxCompactionCount = -1;
maxModsFileSize = 0;
if (selectedTsFileResourceList == null) {
return;
}
......@@ -354,6 +373,10 @@ public class InnerSpaceCompactionTask extends AbstractCompactionTask {
if (fileName.getVersion() > maxFileVersion) {
maxFileVersion = fileName.getVersion();
}
if (!Objects.isNull(resource.getModFile())) {
long modsFileSize = resource.getModFile().getSize();
maxModsFileSize = Math.max(maxModsFileSize, modsFileSize);
}
} catch (IOException e) {
LOGGER.warn("Fail to get the tsfile name of {}", resource.getTsFile(), e);
}
......@@ -380,6 +403,10 @@ public class InnerSpaceCompactionTask extends AbstractCompactionTask {
return maxFileVersion;
}
public long getMaxModsFileSize() {
return maxModsFileSize;
}
@Override
public String toString() {
return storageGroupName
......@@ -446,9 +473,27 @@ public class InnerSpaceCompactionTask extends AbstractCompactionTask {
return false;
}
}
if (innerSpaceEstimator != null) {
memoryCost = innerSpaceEstimator.estimateInnerCompactionMemory(selectedTsFileResourceList);
}
SystemInfo.getInstance().addCompactionMemoryCost(memoryCost, 60);
} catch (Throwable e) {
if (e instanceof InterruptedException) {
LOGGER.warn("Interrupted when allocating memory for compaction", e);
Thread.currentThread().interrupt();
} else if (e instanceof CompactionMemoryNotEnoughException) {
LOGGER.warn("No enough memory for current compaction task {}", this, e);
}
releaseAllLocksAndResetStatus();
throw e;
return false;
} finally {
try {
if (innerSpaceEstimator != null) {
innerSpaceEstimator.close();
}
} catch (IOException e) {
LOGGER.warn("Failed to close InnerSpaceCompactionMemoryEstimator");
}
}
return true;
}
......
......@@ -236,8 +236,9 @@ public abstract class SeriesCompactionExecutor {
|| firstPageElement.needForceDecoding) {
// has overlap or modified pages, then deserialize it
summary.PAGE_OVERLAP_OR_MODIFIED += 1;
pointPriorityReader.addNewPage(firstPageElement);
compactWithOverlapPages();
if (pointPriorityReader.addNewPageIfPageNotEmpty(firstPageElement)) {
compactWithOverlapPages();
}
} else {
// has none overlap or modified pages, flush it to chunk writer directly
summary.PAGE_NONE_OVERLAP += 1;
......@@ -268,7 +269,9 @@ public abstract class SeriesCompactionExecutor {
} else {
// unsealed page is not large enough or page.endTime > file.endTime, then deserialze it
summary.PAGE_NONE_OVERLAP_BUT_DESERIALIZE += 1;
pointPriorityReader.addNewPage(pageElement);
if (!pointPriorityReader.addNewPageIfPageNotEmpty(pageElement)) {
return;
}
// write data points of the current page into chunk writer
TimeValuePair point;
......@@ -344,7 +347,7 @@ public abstract class SeriesCompactionExecutor {
|| nextPageElement.needForceDecoding) {
// next page is overlapped or modified, then deserialize it
summary.PAGE_OVERLAP_OR_MODIFIED++;
pointPriorityReader.addNewPage(nextPageElement);
pointPriorityReader.addNewPageIfPageNotEmpty(nextPageElement);
} else {
// has none overlap or modified pages, flush it to chunk writer directly
summary.PAGE_FAKE_OVERLAP += 1;
......
......@@ -21,6 +21,7 @@ package org.apache.iotdb.db.engine.compaction.execute.utils.executor.fast.elemen
import org.apache.iotdb.tsfile.file.header.PageHeader;
import org.apache.iotdb.tsfile.read.common.block.TsBlock;
import org.apache.iotdb.tsfile.read.reader.IChunkReader;
import org.apache.iotdb.tsfile.read.reader.IPointReader;
import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader;
import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
......@@ -36,6 +37,10 @@ public class PageElement {
public TsBlock batchData;
// pointReader is used to replace batchData to get rid of huge memory cost by loading data point
// in a lazy way
public IPointReader pointReader;
// compressed page data
public ByteBuffer pageData;
......@@ -95,9 +100,9 @@ public class PageElement {
public void deserializePage() throws IOException {
if (iChunkReader instanceof AlignedChunkReader) {
this.batchData =
this.pointReader =
((AlignedChunkReader) iChunkReader)
.readPageData(pageHeader, valuePageHeaders, pageData, valuePageDatas);
.getPagePointReader(pageHeader, valuePageHeaders, pageData, valuePageDatas);
} else {
this.batchData = ((ChunkReader) iChunkReader).readPageData(pageHeader, pageData);
}
......
......@@ -37,7 +37,12 @@ public class PointElement {
if (pageElement.iChunkReader instanceof ChunkReader) {
this.pointReader = pageElement.batchData.getTsBlockSingleColumnIterator();
} else {
this.pointReader = pageElement.batchData.getTsBlockAlignedRowIterator();
// For aligned page, we use pointReader rather than deserialize all data point to get rid of
// huge memory cost
this.pointReader = pageElement.pointReader;
}
if (!pointReader.hasNextTimeValuePair()) {
return;
}
this.timeValuePair = pointReader.nextTimeValuePair();
this.timestamp = timeValuePair.getTimestamp();
......
......@@ -164,7 +164,8 @@ public class PointPriorityReader {
}
/** Add a new overlapped page. */
public void addNewPage(PageElement pageElement) throws IOException {
public boolean addNewPageIfPageNotEmpty(PageElement pageElement)
throws IOException, IllegalPathException, WriteProcessException {
if (currentPointElement != null) {
nextPointInOtherPage = Math.min(nextPointInOtherPage, pageElement.startTime);
if (currentPoint.getTimestamp() >= nextPointInOtherPage) {
......@@ -172,6 +173,14 @@ public class PointPriorityReader {
currentPointElement = null;
}
}
pointQueue.add(new PointElement(pageElement));
PointElement pointElement = new PointElement(pageElement);
boolean pageIsNotEmpty = pointElement.timeValuePair != null;
if (pageIsNotEmpty) {
pointQueue.add(pointElement);
} else {
removePage.call(pageElement);
}
return pageIsNotEmpty;
}
}
......@@ -62,6 +62,13 @@ public class DefaultCompactionTaskComparatorImpl implements ICompactionTaskCompa
public int compareInnerSpaceCompactionTask(
InnerSpaceCompactionTask o1, InnerSpaceCompactionTask o2) {
// if max mods file size of o1 and o2 are different
// we prefer to execute task with greater mods file
if (o1.getMaxModsFileSize() != o2.getMaxModsFileSize()) {
return o2.getMaxModsFileSize() > o1.getMaxModsFileSize() ? 1 : -1;
}
// if the sum of compaction count of the selected files are different
// we prefer to execute task with smaller compaction count
// this can reduce write amplification
......
......@@ -21,7 +21,7 @@ package org.apache.iotdb.db.engine.compaction.selector;
import org.apache.iotdb.db.engine.compaction.execute.performer.constant.CrossCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.schedule.CompactionTaskManager;
import org.apache.iotdb.db.engine.compaction.selector.estimator.AbstractCompactionEstimator;
import org.apache.iotdb.db.engine.compaction.selector.estimator.ReadPointCrossCompactionEstimator;
import org.apache.iotdb.db.engine.compaction.selector.estimator.FastCrossSpaceCompactionEstimator;
import org.apache.iotdb.db.engine.compaction.selector.utils.CrossCompactionTaskResource;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
......@@ -66,7 +66,7 @@ public interface ICompactionSelector {
case READ_POINT:
case FAST:
if (!isInnerSpace) {
return new ReadPointCrossCompactionEstimator();
return new FastCrossSpaceCompactionEstimator();
}
default:
throw new RuntimeException(
......
......@@ -16,55 +16,128 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.iotdb.db.engine.compaction.selector.estimator;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.conf.adapter.CompressionRatio;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.engine.storagegroup.timeindex.DeviceTimeIndex;
import org.apache.iotdb.db.engine.storagegroup.timeindex.FileTimeIndex;
import org.apache.iotdb.db.engine.storagegroup.timeindex.ITimeIndex;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
/**
* Estimate the memory cost of one compaction task with specific source files based on its
* corresponding implementation.
*/
public abstract class AbstractCompactionEstimator {
public abstract class AbstractCompactionEstimator implements Closeable {
protected Map<TsFileResource, TsFileSequenceReader> fileReaderCache = new HashMap<>();
protected Map<TsFileResource, FileInfo> fileInfoCache = new HashMap<>();
protected Map<TsFileResource, DeviceTimeIndex> deviceTimeIndexCache = new HashMap<>();
protected IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
protected long compressionRatio = (long) CompressionRatio.getInstance().getRatio() + 1;
/**
* Estimate the memory cost of compacting the unseq file and its corresponding overlapped seq
* files in cross space compaction task.
*/
public abstract long estimateCrossCompactionMemory(
List<TsFileResource> seqResources, TsFileResource unseqResource) throws IOException;
protected abstract long calculatingMetadataMemoryCost(CompactionTaskInfo taskInfo);
/** Estimate the memory cost of compacting the source files in inner space compaction task. */
public abstract long estimateInnerCompactionMemory(List<TsFileResource> resources);
protected abstract long calculatingDataMemoryCost(CompactionTaskInfo taskInfo) throws IOException;
/** Construct a new or get an existing TsFileSequenceReader of a TsFile. */
protected TsFileSequenceReader getFileReader(TsFileResource tsFileResource) throws IOException {
TsFileSequenceReader reader = fileReaderCache.get(tsFileResource);
if (reader == null) {
reader = new TsFileSequenceReader(tsFileResource.getTsFilePath(), true, false);
fileReaderCache.put(tsFileResource, reader);
protected CompactionTaskInfo calculatingCompactionTaskInfo(List<TsFileResource> resources)
throws IOException {
List<FileInfo> fileInfoList = new ArrayList<>();
for (TsFileResource resource : resources) {
FileInfo fileInfo = getFileInfoFromCache(resource);
fileInfoList.add(fileInfo);
}
return reader;
return new CompactionTaskInfo(resources, fileInfoList);
}
public void close() throws IOException {
for (TsFileSequenceReader reader : fileReaderCache.values()) {
reader.close();
private FileInfo getFileInfoFromCache(TsFileResource resource) throws IOException {
if (fileInfoCache.containsKey(resource)) {
return fileInfoCache.get(resource);
}
try (TsFileSequenceReader reader =
new TsFileSequenceReader(resource.getTsFilePath(), true, false)) {
FileInfo fileInfo = CompactionEstimateUtils.calculateFileInfo(reader);
fileInfoCache.put(resource, fileInfo);
return fileInfo;
}
}
protected int calculatingMaxOverlapFileNumInSubCompactionTask(List<TsFileResource> resources)
throws IOException {
Set<String> devices = new HashSet<>();
List<DeviceTimeIndex> resourceDevices = new ArrayList<>(resources.size());
for (TsFileResource resource : resources) {
DeviceTimeIndex deviceTimeIndex = getDeviceTimeIndexFromCache(resource);
devices.addAll(deviceTimeIndex.getDevices());
resourceDevices.add(deviceTimeIndex);
}
int maxOverlapFileNumInSubCompactionTask = 1;
for (String device : devices) {
List<DeviceTimeIndex> resourcesContainsCurrentDevice =
resourceDevices.stream()
.filter(resource -> !resource.definitelyNotContains(device))
.sorted(Comparator.comparingLong(resource -> resource.getStartTime(device)))
.collect(Collectors.toList());
if (resourcesContainsCurrentDevice.size() < maxOverlapFileNumInSubCompactionTask) {
continue;
}
long maxEndTimeOfCurrentDevice = Long.MIN_VALUE;
int overlapFileNumOfCurrentDevice = 0;
for (DeviceTimeIndex resource : resourcesContainsCurrentDevice) {
long deviceStartTimeInCurrentFile = resource.getStartTime(device);
long deviceEndTimeInCurrentFile = resource.getEndTime(device);
if (deviceStartTimeInCurrentFile <= maxEndTimeOfCurrentDevice) {
// has overlap, update max end time
maxEndTimeOfCurrentDevice =
Math.max(maxEndTimeOfCurrentDevice, deviceEndTimeInCurrentFile);
overlapFileNumOfCurrentDevice++;
maxOverlapFileNumInSubCompactionTask =
Math.max(maxOverlapFileNumInSubCompactionTask, overlapFileNumOfCurrentDevice);
} else {
// reset max end time and overlap file num of current device
maxEndTimeOfCurrentDevice = deviceEndTimeInCurrentFile;
overlapFileNumOfCurrentDevice = 1;
}
}
// already reach the max value
if (maxOverlapFileNumInSubCompactionTask == resources.size()) {
return maxOverlapFileNumInSubCompactionTask;
}
}
fileReaderCache.clear();
return maxOverlapFileNumInSubCompactionTask;
}
private DeviceTimeIndex getDeviceTimeIndexFromCache(TsFileResource resource) throws IOException {
if (deviceTimeIndexCache.containsKey(resource)) {
return deviceTimeIndexCache.get(resource);
}
ITimeIndex timeIndex = resource.getTimeIndex();
if (timeIndex instanceof FileTimeIndex) {
timeIndex = resource.buildDeviceTimeIndex();
}
deviceTimeIndexCache.put(resource, (DeviceTimeIndex) timeIndex);
return (DeviceTimeIndex) timeIndex;
}
public void close() throws IOException {
deviceTimeIndexCache.clear();
fileInfoCache.clear();
}
}
......@@ -16,11 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.iotdb.db.engine.compaction.selector.estimator;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
/**
......@@ -28,11 +30,27 @@ import java.util.List;
* its corresponding implementation.
*/
public abstract class AbstractCrossSpaceEstimator extends AbstractCompactionEstimator {
public abstract long estimateCrossCompactionMemory(
List<TsFileResource> seqResources, TsFileResource unseqResource) throws IOException;
public long estimateInnerCompactionMemory(List<TsFileResource> resources) {
throw new RuntimeException(
"This kind of estimator cannot be used to estimate inner space compaction task");
public long estimateCrossCompactionMemory(
List<TsFileResource> seqResources, List<TsFileResource> unseqResources) throws IOException {
if (!config.isEnableCompactionMemControl()) {
return 0;
}
List<TsFileResource> resources = new ArrayList<>(seqResources.size() + unseqResources.size());
resources.addAll(seqResources);
resources.addAll(unseqResources);
if (!CompactionEstimateUtils.addReadLock(resources)) {
return -1L;
}
long cost = 0;
try {
CompactionTaskInfo taskInfo = calculatingCompactionTaskInfo(resources);
cost += calculatingMetadataMemoryCost(taskInfo);
cost += calculatingDataMemoryCost(taskInfo);
} finally {
CompactionEstimateUtils.releaseReadLock(resources);
}
return cost;
}
}
......@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.iotdb.db.engine.compaction.selector.estimator;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
......@@ -28,11 +29,14 @@ import java.util.List;
* its corresponding implementation.
*/
public abstract class AbstractInnerSpaceEstimator extends AbstractCompactionEstimator {
public abstract long estimateInnerCompactionMemory(List<TsFileResource> resources);
public long estimateCrossCompactionMemory(
List<TsFileResource> seqResources, TsFileResource unseqResource) throws IOException {
throw new RuntimeException(
"This kind of estimator cannot be used to estimate cross space compaction task");
public long estimateInnerCompactionMemory(List<TsFileResource> resources) throws IOException {
if (!config.isEnableCompactionMemControl()) {
return 0;
}
CompactionTaskInfo taskInfo = calculatingCompactionTaskInfo(resources);
long cost = calculatingMetadataMemoryCost(taskInfo);
cost += calculatingDataMemoryCost(taskInfo);
return cost;
}
}
/*
* 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.iotdb.db.engine.compaction.selector.estimator;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class CompactionEstimateUtils {
/**
* Get the details of the tsfile, the returned array contains the following elements in sequence:
*
* <p>total chunk num in this tsfile
*
* <p>max chunk num of one timeseries in this tsfile
*
* <p>max aligned series num in one device. If there is no aligned series in this file, then it
* turns to be -1.
*
* <p>max chunk num of one device in this tsfile
*
* @throws IOException if io errors occurred
*/
public static FileInfo calculateFileInfo(TsFileSequenceReader reader) throws IOException {
int totalChunkNum = 0;
int maxChunkNum = 0;
int maxAlignedSeriesNumInDevice = -1;
int maxDeviceChunkNum = 0;
Map<String, List<TimeseriesMetadata>> deviceMetadata = reader.getAllTimeseriesMetadata(true);
for (Map.Entry<String, List<TimeseriesMetadata>> entry : deviceMetadata.entrySet()) {
int deviceChunkNum = 0;
List<TimeseriesMetadata> deviceTimeseriesMetadata = entry.getValue();
if (deviceTimeseriesMetadata.get(0).getMeasurementId().equals("")) {
// aligned device
maxAlignedSeriesNumInDevice =
Math.max(maxAlignedSeriesNumInDevice, deviceTimeseriesMetadata.size());
}
for (TimeseriesMetadata timeseriesMetadata : deviceTimeseriesMetadata) {
deviceChunkNum += timeseriesMetadata.getChunkMetadataList().size();
totalChunkNum += timeseriesMetadata.getChunkMetadataList().size();
maxChunkNum = Math.max(maxChunkNum, timeseriesMetadata.getChunkMetadataList().size());
}
maxDeviceChunkNum = Math.max(maxDeviceChunkNum, deviceChunkNum);
}
long averageChunkMetadataSize =
totalChunkNum == 0 ? 0 : reader.getAllMetadataSize() / totalChunkNum;
return new FileInfo(
totalChunkNum,
maxChunkNum,
maxAlignedSeriesNumInDevice,
maxDeviceChunkNum,
averageChunkMetadataSize);
}
public static boolean addReadLock(List<TsFileResource> resources) {
for (int i = 0; i < resources.size(); i++) {
TsFileResource resource = resources.get(i);
resource.readLock();
if (resource.isDeleted()) {
// release read lock
for (int j = 0; j <= i; j++) {
resources.get(j).readUnlock();
}
return false;
}
}
return true;
}
public static void releaseReadLock(List<TsFileResource> resources) {
resources.forEach(TsFileResource::readUnlock);
}
}
/*
* 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.iotdb.db.engine.compaction.selector.estimator;
import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import java.util.List;
public class CompactionTaskInfo {
private final List<FileInfo> fileInfoList;
private final List<TsFileResource> resources;
private int maxConcurrentSeriesNum = 1;
private long maxChunkMetadataSize = 0;
private int maxChunkMetadataNumInDevice = 0;
private int maxChunkMetadataNumInSeries = 0;
private long modificationFileSize = 0;
private long totalFileSize = 0;
private long totalChunkNum = 0;
private long totalChunkMetadataSize = 0;
protected CompactionTaskInfo(List<TsFileResource> resources, List<FileInfo> fileInfoList) {
this.fileInfoList = fileInfoList;
this.resources = resources;
for (TsFileResource resource : resources) {
ModificationFile modificationFile = resource.getModFile();
if (modificationFile.exists()) {
modificationFileSize += modificationFile.getSize();
}
this.totalFileSize += resource.getTsFileSize();
}
for (FileInfo fileInfo : fileInfoList) {
maxConcurrentSeriesNum =
Math.max(maxConcurrentSeriesNum, fileInfo.maxAlignedSeriesNumInDevice);
maxChunkMetadataNumInSeries =
Math.max(maxChunkMetadataNumInSeries, fileInfo.maxSeriesChunkNum);
maxChunkMetadataNumInDevice =
Math.max(maxChunkMetadataNumInDevice, fileInfo.maxDeviceChunkNum);
maxChunkMetadataSize = Math.max(maxChunkMetadataSize, fileInfo.averageChunkMetadataSize);
totalChunkNum += fileInfo.totalChunkNum;
totalChunkMetadataSize += fileInfo.totalChunkNum * fileInfo.averageChunkMetadataSize;
}
}
public int getMaxChunkMetadataNumInDevice() {
return maxChunkMetadataNumInDevice;
}
public int getMaxChunkMetadataNumInSeries() {
return maxChunkMetadataNumInSeries;
}
public long getMaxChunkMetadataSize() {
return maxChunkMetadataSize;
}
public List<FileInfo> getFileInfoList() {
return fileInfoList;
}
public int getMaxConcurrentSeriesNum() {
return maxConcurrentSeriesNum;
}
public long getModificationFileSize() {
return modificationFileSize;
}
public long getTotalFileSize() {
return totalFileSize;
}
public long getTotalChunkNum() {
return totalChunkNum;
}
public List<TsFileResource> getResources() {
return resources;
}
public long getTotalChunkMetadataSize() {
return totalChunkMetadataSize;
}
}
/*
* 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.iotdb.db.engine.compaction.selector.estimator;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.rescon.SystemInfo;
import java.io.IOException;
public class FastCompactionInnerCompactionEstimator extends AbstractInnerSpaceEstimator {
@Override
public long calculatingMetadataMemoryCost(CompactionTaskInfo taskInfo) {
long cost = 0;
// add ChunkMetadata size of MultiTsFileDeviceIterator
cost +=
Math.min(
taskInfo.getTotalChunkMetadataSize(),
taskInfo.getFileInfoList().size()
* taskInfo.getMaxChunkMetadataNumInSeries()
* taskInfo.getMaxChunkMetadataSize());
// add ChunkMetadata size of targetFileWriter
long sizeForFileWriter =
(long)
((double) SystemInfo.getInstance().getMemorySizeForCompaction()
/ IoTDBDescriptor.getInstance().getConfig().getCompactionThreadCount()
* IoTDBDescriptor.getInstance().getConfig().getChunkMetadataSizeProportion());
cost += sizeForFileWriter;
return cost;
}
/**
* The data algorithm is: targetChunkWriterSize + mods file size + read chunk size
*
* @return estimate data memory cost
*/
@Override
public long calculatingDataMemoryCost(CompactionTaskInfo taskInfo) throws IOException {
if (taskInfo.getTotalChunkNum() == 0) {
return taskInfo.getModificationFileSize();
}
long maxConcurrentSeriesNum =
Math.max(config.getSubCompactionTaskNum(), taskInfo.getMaxConcurrentSeriesNum());
long averageUncompressedChunkSize =
taskInfo.getTotalFileSize() * compressionRatio / taskInfo.getTotalChunkNum();
long maxConcurrentSeriesSizeOfTotalFiles =
averageUncompressedChunkSize
* taskInfo.getFileInfoList().size()
* maxConcurrentSeriesNum
* taskInfo.getMaxChunkMetadataNumInSeries()
/ compressionRatio;
long maxTargetChunkWriterSize = config.getTargetChunkSize() * maxConcurrentSeriesNum;
long targetChunkWriterSize =
Math.min(maxConcurrentSeriesSizeOfTotalFiles, maxTargetChunkWriterSize);
long maxConcurrentChunkSizeFromSourceFile =
averageUncompressedChunkSize
* maxConcurrentSeriesNum
* calculatingMaxOverlapFileNumInSubCompactionTask(taskInfo.getResources());
return targetChunkWriterSize
+ maxConcurrentChunkSizeFromSourceFile
+ taskInfo.getModificationFileSize();
}
}
/*
* 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.iotdb.db.engine.compaction.selector.estimator;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.rescon.SystemInfo;
import java.io.IOException;
public class FastCrossSpaceCompactionEstimator extends AbstractCrossSpaceEstimator {
@Override
protected long calculatingMetadataMemoryCost(CompactionTaskInfo taskInfo) {
long cost = 0;
// add ChunkMetadata size of MultiTsFileDeviceIterator
cost +=
Math.min(
taskInfo.getTotalChunkMetadataSize(),
taskInfo.getFileInfoList().size()
* taskInfo.getMaxChunkMetadataNumInDevice()
* taskInfo.getMaxChunkMetadataSize());
// add ChunkMetadata size of targetFileWriter
long sizeForFileWriter =
(long)
((double) SystemInfo.getInstance().getMemorySizeForCompaction()
/ IoTDBDescriptor.getInstance().getConfig().getCompactionThreadCount()
* IoTDBDescriptor.getInstance().getConfig().getChunkMetadataSizeProportion());
cost += sizeForFileWriter;
return cost;
}
@Override
protected long calculatingDataMemoryCost(CompactionTaskInfo taskInfo) throws IOException {
if (taskInfo.getTotalChunkNum() == 0) {
return taskInfo.getModificationFileSize();
}
long maxConcurrentSeriesNum =
Math.max(config.getSubCompactionTaskNum(), taskInfo.getMaxConcurrentSeriesNum());
long averageUncompressedChunkSize =
taskInfo.getTotalFileSize() * compressionRatio / taskInfo.getTotalChunkNum();
long maxConcurrentSeriesSizeOfTotalFiles =
averageUncompressedChunkSize
* taskInfo.getFileInfoList().size()
* maxConcurrentSeriesNum
* taskInfo.getMaxChunkMetadataNumInSeries()
/ compressionRatio;
long maxTargetChunkWriterSize = config.getTargetChunkSize() * maxConcurrentSeriesNum;
long targetChunkWriterSize =
Math.min(maxConcurrentSeriesSizeOfTotalFiles, maxTargetChunkWriterSize);
long maxConcurrentChunkSizeFromSourceFile =
averageUncompressedChunkSize
* maxConcurrentSeriesNum
* calculatingMaxOverlapFileNumInSubCompactionTask(taskInfo.getResources());
return targetChunkWriterSize
+ maxConcurrentChunkSizeFromSourceFile
+ taskInfo.getModificationFileSize();
}
}
/*
* 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.iotdb.db.engine.compaction.selector.estimator;
public class FileInfo {
// total chunk num in this tsfile
int totalChunkNum = 0;
// max chunk num of one timeseries in this tsfile
int maxSeriesChunkNum = 0;
// max aligned series num in one device. If there is no aligned series in this file, then it
// turns to be -1.
int maxAlignedSeriesNumInDevice = -1;
// max chunk num of one device in this tsfile
@SuppressWarnings("squid:S1068")
int maxDeviceChunkNum = 0;
long averageChunkMetadataSize = 0;
public FileInfo(
int totalChunkNum,
int maxSeriesChunkNum,
int maxAlignedSeriesNumInDevice,
int maxDeviceChunkNum,
long averageChunkMetadataSize) {
this.totalChunkNum = totalChunkNum;
this.maxSeriesChunkNum = maxSeriesChunkNum;
this.maxAlignedSeriesNumInDevice = maxAlignedSeriesNumInDevice;
this.maxDeviceChunkNum = maxDeviceChunkNum;
this.averageChunkMetadataSize = averageChunkMetadataSize;
}
}
/*
* 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.iotdb.db.engine.compaction.selector.estimator;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class InplaceCompactionEstimator extends AbstractCrossSpaceEstimator {
private static final Logger logger = LoggerFactory.getLogger(InplaceCompactionEstimator.class);
private static final String LOG_FILE_COST = "Memory cost of file {} is {}";
private boolean tightEstimate;
private long maxSeqFileCost;
// the number of timeseries being compacted at the same time
private final int concurrentSeriesNum =
IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum();
/** Total metadata size of each file. */
private final Map<TsFileResource, Long> fileMetaSizeMap = new HashMap<>();
/** Maximum memory cost of querying a timeseries in each file. */
private final Map<TsFileResource, Long> maxSeriesQueryCostMap = new HashMap<>();
public InplaceCompactionEstimator() {
this.tightEstimate = false;
this.maxSeqFileCost = 0;
}
@Override
public long estimateCrossCompactionMemory(
List<TsFileResource> seqResources, TsFileResource unseqResource) throws IOException {
if (tightEstimate) {
return calculateTightMemoryCost(unseqResource, seqResources);
} else {
return calculateLooseMemoryCost(unseqResource, seqResources);
}
}
private long calculateMemoryCost(
TsFileResource unseqResource,
List<TsFileResource> seqResources,
IFileQueryMemMeasurement unseqMeasurement,
IFileQueryMemMeasurement seqMeasurement)
throws IOException {
long cost = 0;
Long fileCost = unseqMeasurement.measure(unseqResource);
cost += fileCost;
for (TsFileResource seqFile : seqResources) {
fileCost = seqMeasurement.measure(seqFile);
if (fileCost > maxSeqFileCost) {
// only one file will be read at the same time, so only the largest one is recorded here
cost -= maxSeqFileCost;
cost += fileCost;
maxSeqFileCost = fileCost;
}
// but writing data into a new file may generate the same amount of metadata in memory
cost += calculateMetadataSize(seqFile);
}
return cost;
}
private long calculateLooseMemoryCost(
TsFileResource unseqResource, List<TsFileResource> seqResources) throws IOException {
return calculateMemoryCost(
unseqResource, seqResources, TsFileResource::getTsFileSize, this::calculateMetadataSize);
}
private long calculateTightMemoryCost(
TsFileResource unseqResource, List<TsFileResource> seqResources) throws IOException {
return calculateMemoryCost(
unseqResource,
seqResources,
this::calculateTightUnseqMemoryCost,
this::calculateTightSeqMemoryCost);
}
private long calculateMetadataSize(TsFileResource seqFile) throws IOException {
Long cost = fileMetaSizeMap.get(seqFile);
if (cost == null) {
cost = getFileReader(seqFile).getFileMetadataSize();
fileMetaSizeMap.put(seqFile, cost);
logger.debug(LOG_FILE_COST, seqFile, cost);
}
return cost;
}
private long calculateTightFileMemoryCost(
TsFileResource seqFile, IFileQueryMemMeasurement measurement) throws IOException {
Long cost = maxSeriesQueryCostMap.get(seqFile);
if (cost == null) {
long[] chunkNums = findTotalAndLargestSeriesChunkNum(seqFile, getFileReader(seqFile));
long totalChunkNum = chunkNums[0];
long maxChunkNum = chunkNums[1];
cost = measurement.measure(seqFile) * maxChunkNum / totalChunkNum;
maxSeriesQueryCostMap.put(seqFile, cost);
logger.debug(LOG_FILE_COST, seqFile, cost);
}
return cost;
}
// this method traverses all ChunkMetadata to find out which series has the most chunks and uses
// its proportion to all series to get a maximum estimation
private long calculateTightSeqMemoryCost(TsFileResource seqFile) throws IOException {
long singleSeriesCost = calculateTightFileMemoryCost(seqFile, this::calculateMetadataSize);
long multiSeriesCost = concurrentSeriesNum * singleSeriesCost;
long maxCost = calculateMetadataSize(seqFile);
return Math.min(multiSeriesCost, maxCost);
}
// this method traverses all ChunkMetadata to find out which series has the most chunks and uses
// its proportion among all series to get a maximum estimation
private long calculateTightUnseqMemoryCost(TsFileResource unseqFile) throws IOException {
long singleSeriesCost = calculateTightFileMemoryCost(unseqFile, TsFileResource::getTsFileSize);
long multiSeriesCost = concurrentSeriesNum * singleSeriesCost;
long maxCost = unseqFile.getTsFileSize();
return Math.min(multiSeriesCost, maxCost);
}
// returns totalChunkNum of a file and the max number of chunks of a series
private long[] findTotalAndLargestSeriesChunkNum(
TsFileResource tsFileResource, TsFileSequenceReader sequenceReader) throws IOException {
long totalChunkNum = 0;
long maxChunkNum = Long.MIN_VALUE;
List<Path> paths = sequenceReader.getAllPaths();
for (Path path : paths) {
List<ChunkMetadata> chunkMetadataList = sequenceReader.getChunkMetadataList(path, true);
totalChunkNum += chunkMetadataList.size();
maxChunkNum = chunkMetadataList.size() > maxChunkNum ? chunkMetadataList.size() : maxChunkNum;
}
logger.debug(
"In file {}, total chunk num {}, series max chunk num {}",
tsFileResource,
totalChunkNum,
maxChunkNum);
return new long[] {totalChunkNum, maxChunkNum};
}
public void setTightEstimate(boolean tightEstimate) {
this.tightEstimate = tightEstimate;
}
}
/*
* 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.iotdb.db.engine.compaction.selector.estimator;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.rescon.SystemInfo;
public class ReadChunkInnerCompactionEstimator extends AbstractInnerSpaceEstimator {
@Override
public long calculatingMetadataMemoryCost(CompactionTaskInfo taskInfo) {
long cost = 0;
// add ChunkMetadata size of MultiTsFileDeviceIterator
cost +=
Math.min(
taskInfo.getTotalChunkMetadataSize(),
taskInfo.getFileInfoList().size()
* taskInfo.getMaxChunkMetadataNumInDevice()
* taskInfo.getMaxChunkMetadataSize());
// add ChunkMetadata size of targetFileWriter
long sizeForFileWriter =
(long)
((double) SystemInfo.getInstance().getMemorySizeForCompaction()
/ IoTDBDescriptor.getInstance().getConfig().getCompactionThreadCount()
* IoTDBDescriptor.getInstance().getConfig().getChunkMetadataSizeProportion());
cost += sizeForFileWriter;
return cost;
}
@Override
public long calculatingDataMemoryCost(CompactionTaskInfo taskInfo) {
if (taskInfo.getTotalChunkNum() == 0) {
return taskInfo.getModificationFileSize();
}
long averageUncompressedChunkSize =
taskInfo.getTotalFileSize() * compressionRatio / taskInfo.getTotalChunkNum();
long maxConcurrentSeriesSizeOfTotalFiles =
averageUncompressedChunkSize
* taskInfo.getFileInfoList().size()
* taskInfo.getMaxConcurrentSeriesNum()
* taskInfo.getMaxChunkMetadataNumInSeries()
/ compressionRatio;
long maxTargetChunkWriterSize =
config.getTargetChunkSize() * taskInfo.getMaxConcurrentSeriesNum();
long targetChunkWriterSize =
Math.min(maxConcurrentSeriesSizeOfTotalFiles, maxTargetChunkWriterSize);
long chunkSizeFromSourceFile =
averageUncompressedChunkSize * taskInfo.getMaxConcurrentSeriesNum();
return targetChunkWriterSize + chunkSizeFromSourceFile + taskInfo.getModificationFileSize();
}
}
/*
* 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.iotdb.db.engine.compaction.selector.estimator;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
public class ReadPointCrossCompactionEstimator extends AbstractCrossSpaceEstimator {
private static final Logger logger =
LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME);
// the max cost of reading source seq file among all source seq files of this cross compaction
// task
private long maxCostOfReadingSeqFile;
// the max cost of writing target file
private long maxCostOfWritingTargetFile;
private int maxConcurrentSeriesNum = 1;
// the number of timeseries being compacted at the same time
private final int subCompactionTaskNum =
IoTDBDescriptor.getInstance().getConfig().getSubCompactionTaskNum();
public ReadPointCrossCompactionEstimator() {
this.maxCostOfReadingSeqFile = 0;
this.maxCostOfWritingTargetFile = 0;
}
@Override
public long estimateCrossCompactionMemory(
List<TsFileResource> seqResources, TsFileResource unseqResource) throws IOException {
if (!addReadLock(seqResources, unseqResource)) {
// there is file been deleted during selection, return -1
return -1L;
}
try {
long cost = 0;
cost += calculateReadingUnseqFile(unseqResource);
cost += calculateReadingSeqFiles(seqResources);
cost += calculatingWritingTargetFiles(seqResources, unseqResource);
return cost;
} finally {
releaseReadLock(seqResources, unseqResource);
}
}
/** Add read lock. Return false if any of the file were deleted. */
private boolean addReadLock(List<TsFileResource> seqResources, TsFileResource unseqResource) {
List<TsFileResource> allResources = new ArrayList<>(seqResources);
allResources.add(unseqResource);
for (int i = 0; i < allResources.size(); i++) {
TsFileResource resource = allResources.get(i);
resource.readLock();
if (resource.isDeleted()) {
// release read lock
for (int j = 0; j <= i; j++) {
allResources.get(j).readUnlock();
}
return false;
}
}
return true;
}
private void releaseReadLock(List<TsFileResource> seqResources, TsFileResource unseqResource) {
seqResources.forEach(TsFileResource::readUnlock);
unseqResource.readUnlock();
}
/**
* Calculate memory cost of reading source unseq files in the cross space compaction. Double the
* total size of the timeseries to be compacted at the same time in all unseq files.
*/
private long calculateReadingUnseqFile(TsFileResource unseqResource) throws IOException {
TsFileSequenceReader reader = getFileReader(unseqResource);
FileInfo fileInfo = getSeriesAndDeviceChunkNum(reader);
// it is max aligned series num of one device when tsfile contains aligned series,
// else is sub compaction task num.
int concurrentSeriesNum =
fileInfo.maxAlignedSeriesNumInDevice == -1
? subCompactionTaskNum
: fileInfo.maxAlignedSeriesNumInDevice;
maxConcurrentSeriesNum = Math.max(maxConcurrentSeriesNum, concurrentSeriesNum);
if (fileInfo.totalChunkNum == 0) { // If totalChunkNum ==0, i.e. this unSeq tsFile has no chunk.
logger.warn(
"calculateReadingUnseqFile(), find 1 empty unSeq tsFile: {}.",
unseqResource.getTsFilePath());
return 0;
}
// it means the max size of a timeseries in this file when reading all of its chunk into memory.
return compressionRatio
* concurrentSeriesNum
* (unseqResource.getTsFileSize() * fileInfo.maxSeriesChunkNum / fileInfo.totalChunkNum);
}
/**
* Calculate memory cost of reading source seq files in the cross space compaction. Select the
* maximun size of the timeseries to be compacted at the same time in one seq file, because only
* one seq file will be queried at the same time.
*/
private long calculateReadingSeqFiles(List<TsFileResource> seqResources) throws IOException {
long cost = 0;
for (TsFileResource seqResource : seqResources) {
TsFileSequenceReader reader = getFileReader(seqResource);
FileInfo fileInfo = getSeriesAndDeviceChunkNum(reader);
// it is max aligned series num of one device when tsfile contains aligned series,
// else is sub compaction task num.
int concurrentSeriesNum =
fileInfo.maxAlignedSeriesNumInDevice == -1
? subCompactionTaskNum
: fileInfo.maxAlignedSeriesNumInDevice;
maxConcurrentSeriesNum = Math.max(maxConcurrentSeriesNum, concurrentSeriesNum);
long seqFileCost;
if (fileInfo.totalChunkNum == 0) { // If totalChunkNum ==0, i.e. this seq tsFile has no chunk.
logger.warn(
"calculateReadingSeqFiles(), find 1 empty seq tsFile: {}.",
seqResource.getTsFilePath());
seqFileCost = 0;
} else {
// We need to multiply the compression ratio here.
seqFileCost =
compressionRatio
* seqResource.getTsFileSize()
* concurrentSeriesNum
/ fileInfo.totalChunkNum;
}
if (seqFileCost > maxCostOfReadingSeqFile) {
// Only one seq file will be read at the same time.
// not only reading chunk into chunk cache, but also need to deserialize data point into
// merge reader. We have to add the cost in merge reader here and the cost of chunk cache is
// unnecessary.
cost -= maxCostOfReadingSeqFile;
cost += seqFileCost;
maxCostOfReadingSeqFile = seqFileCost;
}
}
return cost;
}
/**
* Calculate memory cost of writing target files in the cross space compaction. Including metadata
* size of all source files and size of concurrent target chunks.
*/
private long calculatingWritingTargetFiles(
List<TsFileResource> seqResources, TsFileResource unseqResource) throws IOException {
long cost = 0;
for (TsFileResource seqResource : seqResources) {
TsFileSequenceReader reader = getFileReader(seqResource);
// add seq file metadata size
cost += reader.getFileMetadataSize();
}
// add unseq file metadata size
cost += getFileReader(unseqResource).getFileMetadataSize();
// concurrent series chunk size
long writingTargetCost = maxConcurrentSeriesNum * config.getTargetChunkSize();
if (writingTargetCost > maxCostOfWritingTargetFile) {
cost -= maxCostOfWritingTargetFile;
cost += writingTargetCost;
maxCostOfWritingTargetFile = writingTargetCost;
}
return cost;
}
/**
* Get the details of the tsfile, the returned array contains the following elements in sequence:
*
* <p>total chunk num in this tsfile
*
* <p>max chunk num of one timeseries in this tsfile
*
* <p>max aligned series num in one device. If there is no aligned series in this file, then it
* turns to be -1.
*
* <p>max chunk num of one device in this tsfile
*/
private FileInfo getSeriesAndDeviceChunkNum(TsFileSequenceReader reader) throws IOException {
int totalChunkNum = 0;
int maxChunkNum = 0;
int maxAlignedSeriesNumInDevice = -1;
int maxDeviceChunkNum = 0;
Map<String, List<TimeseriesMetadata>> deviceMetadata = reader.getAllTimeseriesMetadata(true);
for (Map.Entry<String, List<TimeseriesMetadata>> entry : deviceMetadata.entrySet()) {
int deviceChunkNum = 0;
List<TimeseriesMetadata> deviceTimeseriesMetadata = entry.getValue();
if (deviceTimeseriesMetadata.get(0).getMeasurementId().equals("")) {
// aligned device
maxAlignedSeriesNumInDevice =
Math.max(maxAlignedSeriesNumInDevice, deviceTimeseriesMetadata.size());
}
for (TimeseriesMetadata timeseriesMetadata : deviceTimeseriesMetadata) {
deviceChunkNum += timeseriesMetadata.getChunkMetadataList().size();
totalChunkNum += timeseriesMetadata.getChunkMetadataList().size();
maxChunkNum = Math.max(maxChunkNum, timeseriesMetadata.getChunkMetadataList().size());
}
maxDeviceChunkNum = Math.max(maxDeviceChunkNum, deviceChunkNum);
}
return new FileInfo(totalChunkNum, maxChunkNum, maxAlignedSeriesNumInDevice, maxDeviceChunkNum);
}
private class FileInfo {
// total chunk num in this tsfile
public int totalChunkNum = 0;
// max chunk num of one timeseries in this tsfile
public int maxSeriesChunkNum = 0;
// max aligned series num in one device. If there is no aligned series in this file, then it
// turns to be -1.
public int maxAlignedSeriesNumInDevice = -1;
// max chunk num of one device in this tsfile
public int maxDeviceChunkNum = 0;
public FileInfo(
int totalChunkNum,
int maxSeriesChunkNum,
int maxAlignedSeriesNumInDevice,
int maxDeviceChunkNum) {
this.totalChunkNum = totalChunkNum;
this.maxSeriesChunkNum = maxSeriesChunkNum;
this.maxAlignedSeriesNumInDevice = maxAlignedSeriesNumInDevice;
this.maxDeviceChunkNum = maxDeviceChunkNum;
}
}
}
......@@ -24,7 +24,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.schedule.CompactionTaskManager;
import org.apache.iotdb.db.engine.compaction.selector.ICompactionSelector;
import org.apache.iotdb.db.engine.compaction.selector.ICrossSpaceSelector;
import org.apache.iotdb.db.engine.compaction.selector.estimator.AbstractCompactionEstimator;
import org.apache.iotdb.db.engine.compaction.selector.estimator.AbstractCrossSpaceEstimator;
import org.apache.iotdb.db.engine.compaction.selector.utils.CrossCompactionTaskResource;
import org.apache.iotdb.db.engine.compaction.selector.utils.CrossSpaceCompactionCandidate;
import org.apache.iotdb.db.engine.compaction.selector.utils.CrossSpaceCompactionCandidate.CrossCompactionTaskResourceSplit;
......@@ -39,6 +39,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
......@@ -59,7 +60,7 @@ public class RewriteCrossSpaceCompactionSelector implements ICrossSpaceSelector
private final int maxCrossCompactionFileNum;
private final long maxCrossCompactionFileSize;
private AbstractCompactionEstimator compactionEstimator;
private final AbstractCrossSpaceEstimator compactionEstimator;
public RewriteCrossSpaceCompactionSelector(
String logicalStorageGroupName,
......@@ -81,8 +82,9 @@ public class RewriteCrossSpaceCompactionSelector implements ICrossSpaceSelector
IoTDBDescriptor.getInstance().getConfig().getMaxCrossCompactionCandidateFileSize();
this.compactionEstimator =
ICompactionSelector.getCompactionEstimator(
IoTDBDescriptor.getInstance().getConfig().getCrossCompactionPerformer(), false);
(AbstractCrossSpaceEstimator)
ICompactionSelector.getCompactionEstimator(
IoTDBDescriptor.getInstance().getConfig().getCrossCompactionPerformer(), false);
}
/**
......@@ -168,8 +170,15 @@ public class RewriteCrossSpaceCompactionSelector implements ICrossSpaceSelector
}
}
List<TsFileResource> newSelectedSeqResources = new ArrayList<>(taskResource.getSeqFiles());
newSelectedSeqResources.addAll(targetSeqFiles);
List<TsFileResource> newSelectedUnseqResources =
new ArrayList<>(taskResource.getUnseqFiles());
newSelectedUnseqResources.add(unseqFile);
long memoryCost =
compactionEstimator.estimateCrossCompactionMemory(targetSeqFiles, unseqFile);
compactionEstimator.estimateCrossCompactionMemory(
newSelectedSeqResources, newSelectedUnseqResources);
if (!canAddToTaskResource(taskResource, unseqFile, targetSeqFiles, memoryCost)) {
break;
}
......@@ -247,7 +256,7 @@ public class RewriteCrossSpaceCompactionSelector implements ICrossSpaceSelector
if (taskResource.getTotalFileNums() + 1 + seqFiles.size() <= maxCrossCompactionFileNum
&& taskResource.getTotalFileSize() + totalFileSize <= maxCrossCompactionFileSize
&& taskResource.getTotalMemoryCost() + memoryCost < memoryBudget) {
&& memoryCost < memoryBudget) {
return true;
}
return false;
......
......@@ -25,6 +25,7 @@ import org.apache.iotdb.db.engine.compaction.schedule.CompactionTaskManager;
import org.apache.iotdb.db.engine.compaction.schedule.comparator.ICompactionTaskComparator;
import org.apache.iotdb.db.engine.compaction.selector.IInnerSeqSpaceSelector;
import org.apache.iotdb.db.engine.compaction.selector.IInnerUnseqSpaceSelector;
import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
......@@ -40,6 +41,7 @@ import java.util.Collections;
import java.util.Comparator;
import java.util.LinkedList;
import java.util.List;
import java.util.Objects;
import java.util.PriorityQueue;
/**
......@@ -64,6 +66,7 @@ public class SizeTieredCompactionSelector
protected boolean sequence;
protected TsFileManager tsFileManager;
protected boolean hasNextTimePartition;
private static final long MODS_FILE_SIZE_THRESHOLD = 1024 * 1024 * 50L;
public SizeTieredCompactionSelector(
String storageGroupName,
......@@ -88,27 +91,23 @@ public class SizeTieredCompactionSelector
* longer search for higher layers), otherwise it will return true.
*
* @param level the level to be searched
* @param taskPriorityQueue it stores the batches of files to be compacted and the total size of
* each batch
* @return return whether to continue the search to higher levels
* @throws IOException
*/
private boolean selectLevelTask(
int level, PriorityQueue<Pair<List<TsFileResource>, Long>> taskPriorityQueue)
throws IOException {
boolean shouldContinueToSearch = true;
@SuppressWarnings({"squid:S3776", "squid:S135"})
private List<Pair<List<TsFileResource>, Long>> selectSingleLevel(int level) throws IOException {
List<TsFileResource> selectedFileList = new ArrayList<>();
long selectedFileSize = 0L;
long targetCompactionFileSize = config.getTargetCompactionFileSize();
List<Pair<List<TsFileResource>, Long>> taskList = new ArrayList<>();
for (TsFileResource currentFile : tsFileResources) {
TsFileNameGenerator.TsFileName currentName =
TsFileNameGenerator.getTsFileName(currentFile.getTsFile().getName());
if (currentName.getInnerCompactionCnt() != level) {
// meet files of another level
if (selectedFileList.size() > 1) {
taskPriorityQueue.add(new Pair<>(new ArrayList<>(selectedFileList), selectedFileSize));
shouldContinueToSearch = false;
taskList.add(new Pair<>(new ArrayList<>(selectedFileList), selectedFileSize));
}
selectedFileList = new ArrayList<>();
selectedFileSize = 0L;
......@@ -132,8 +131,7 @@ public class SizeTieredCompactionSelector
|| selectedFileList.size() >= config.getMaxInnerCompactionCandidateFileNum()) {
// submit the task
if (selectedFileList.size() > 1) {
taskPriorityQueue.add(new Pair<>(new ArrayList<>(selectedFileList), selectedFileSize));
shouldContinueToSearch = false;
taskList.add(new Pair<>(new ArrayList<>(selectedFileList), selectedFileSize));
}
selectedFileList = new ArrayList<>();
selectedFileSize = 0L;
......@@ -143,17 +141,19 @@ public class SizeTieredCompactionSelector
// if next time partition exists
// submit a merge task even it does not meet the requirement for file num or file size
if (hasNextTimePartition && selectedFileList.size() > 1) {
taskPriorityQueue.add(new Pair<>(new ArrayList<>(selectedFileList), selectedFileSize));
shouldContinueToSearch = false;
taskList.add(new Pair<>(new ArrayList<>(selectedFileList), selectedFileSize));
}
return shouldContinueToSearch;
return taskList;
}
/**
* This method searches for a batch of files to be compacted from layer 0 to the highest layer. If
* there are more than a batch of files to be merged on a certain layer, it does not search to
* higher layers. It creates a compaction thread for each batch of files and put it into the
* candidateCompactionTaskQueue of the {@link CompactionTaskManager}.
* This method is used to select a batch of files to be merged. There are two ways to select
* files.If the first method selects the appropriate file, the second method is not executed. The
* first one is based on the mods file corresponding to the file. We will preferentially select
* file with mods file larger than 50M. The second way is based on the file layer from layer 0 to
* the highest layer. If there are more than a batch of files to be merged on a certain layer, it
* does not search to higher layers. It creates a compaction thread for each batch of files and
* put it into the candidateCompactionTaskQueue of the {@link CompactionTaskManager}.
*
* @return Returns whether the file was found and submits the merge task
*/
......@@ -163,12 +163,14 @@ public class SizeTieredCompactionSelector
PriorityQueue<Pair<List<TsFileResource>, Long>> taskPriorityQueue =
new PriorityQueue<>(new SizeTieredCompactionTaskComparator());
try {
int maxLevel = searchMaxFileLevel();
for (int currentLevel = 0; currentLevel <= maxLevel; currentLevel++) {
if (!selectLevelTask(currentLevel, taskPriorityQueue)) {
break;
}
// preferentially select files based on mods file size
taskPriorityQueue.addAll(selectMaxModsFileTask());
// if a suitable file is not selected in the first step, select the file at the tsfile level
if (taskPriorityQueue.isEmpty()) {
taskPriorityQueue.addAll(selectLevelTask());
}
List<List<TsFileResource>> taskList = new LinkedList<>();
while (taskPriorityQueue.size() > 0) {
List<TsFileResource> resources = taskPriorityQueue.poll().left;
......@@ -181,6 +183,32 @@ public class SizeTieredCompactionSelector
return Collections.emptyList();
}
private List<Pair<List<TsFileResource>, Long>> selectLevelTask() throws IOException {
List<Pair<List<TsFileResource>, Long>> taskList = new ArrayList<>();
int maxLevel = searchMaxFileLevel();
for (int currentLevel = 0; currentLevel <= maxLevel; currentLevel++) {
List<Pair<List<TsFileResource>, Long>> singleLevelTask = selectSingleLevel(currentLevel);
if (!singleLevelTask.isEmpty()) {
taskList.addAll(singleLevelTask);
break;
}
}
return taskList;
}
private List<Pair<List<TsFileResource>, Long>> selectMaxModsFileTask() {
List<Pair<List<TsFileResource>, Long>> taskList = new ArrayList<>();
for (TsFileResource tsFileResource : tsFileResources) {
ModificationFile modFile = tsFileResource.getModFile();
if (!Objects.isNull(modFile) && modFile.getSize() > MODS_FILE_SIZE_THRESHOLD) {
taskList.add(
new Pair<>(Collections.singletonList(tsFileResource), tsFileResource.getTsFileSize()));
LOGGER.debug("select tsfile {},the mod file size is {}", tsFileResource, modFile.getSize());
}
}
return taskList;
}
private int searchMaxFileLevel() throws IOException {
int maxLevel = -1;
for (TsFileResource currentFile : tsFileResources) {
......
......@@ -69,7 +69,7 @@ public class CrossCompactionTaskResource {
TsFileResource unseqFile, List<TsFileResource> seqFiles, long memoryCost) {
addUnseqFile(unseqFile);
addTargetSeqFiles(seqFiles);
increaseMemoryCost(memoryCost);
updateMemoryCost(memoryCost);
}
private void addUnseqFile(TsFileResource file) {
......@@ -88,8 +88,8 @@ public class CrossCompactionTaskResource {
countStatistic(file);
}
private void increaseMemoryCost(long newMemoryCost) {
this.totalMemoryCost += newMemoryCost;
private void updateMemoryCost(long newMemoryCost) {
this.totalMemoryCost = Math.max(totalMemoryCost, newMemoryCost);
}
private void countStatistic(TsFileResource file) {
......
......@@ -30,6 +30,7 @@ import org.apache.iotdb.db.engine.memtable.IWritableMemChunkGroup;
import org.apache.iotdb.db.exception.runtime.FlushRunTimeException;
import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
import org.apache.iotdb.db.rescon.SystemInfo;
import org.apache.iotdb.db.utils.DateTimeUtils;
import org.apache.iotdb.metrics.utils.IoTDBMetricsUtils;
import org.apache.iotdb.metrics.utils.MetricLevel;
import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
......@@ -43,6 +44,7 @@ import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
......@@ -58,6 +60,8 @@ public class MemTableFlushTask {
private static final FlushSubTaskPoolManager SUB_TASK_POOL_MANAGER =
FlushSubTaskPoolManager.getInstance();
private static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
/* storage group name -> last time */
private static final Map<String, Long> flushPointsCache = new ConcurrentHashMap<>();
private final Future<?> encodingTaskFuture;
private final Future<?> ioTaskFuture;
private RestorableTsFileIOWriter writer;
......@@ -257,23 +261,7 @@ public class MemTableFlushTask {
Thread.currentThread().interrupt();
}
if (!storageGroup.startsWith(IoTDBMetricsUtils.DATABASE)) {
int lastIndex = storageGroup.lastIndexOf("-");
if (lastIndex == -1) {
lastIndex = storageGroup.length();
}
MetricService.getInstance()
.gaugeWithInternalReportAsync(
memTable.getTotalPointsNum(),
Metric.POINTS.toString(),
MetricLevel.CORE,
Tag.DATABASE.toString(),
storageGroup.substring(0, lastIndex),
Tag.TYPE.toString(),
"flush",
Tag.REGION.toString(),
dataRegionId);
}
recordFlushPointsMetric();
LOGGER.info(
"Database {}, flushing memtable {} into disk: Encoding data cost " + "{} ms.",
......@@ -283,6 +271,42 @@ public class MemTableFlushTask {
}
};
private void recordFlushPointsMetric() {
if (storageGroup.startsWith(IoTDBMetricsUtils.DATABASE)) {
return;
}
int lastIndex = storageGroup.lastIndexOf("-");
if (lastIndex == -1) {
lastIndex = storageGroup.length();
}
String storageGroupName = storageGroup.substring(0, lastIndex);
long currentTime = DateTimeUtils.currentTime();
// compute the flush points
long writeTime =
flushPointsCache.compute(
storageGroupName,
(storageGroup, lastTime) -> {
if (lastTime == null || lastTime != currentTime) {
return currentTime;
} else {
return currentTime + 1;
}
});
// record the flush points
MetricService.getInstance()
.gaugeWithInternalReportAsync(
memTable.getTotalPointsNum(),
Metric.POINTS.toString(),
MetricLevel.CORE,
writeTime,
Tag.DATABASE.toString(),
storageGroup.substring(0, lastIndex),
Tag.TYPE.toString(),
"flush",
Tag.REGION.toString(),
dataRegionId);
}
/** io task (third task of pipeline) */
@SuppressWarnings("squid:S135")
private Runnable ioTask =
......
......@@ -44,6 +44,10 @@ public class Deletion extends Modification implements Cloneable {
public Deletion(PartialPath path, long fileOffset, long endTime) {
super(Type.DELETION, path, fileOffset);
this.timeRange = new TimeRange(Long.MIN_VALUE, endTime);
this.timeRange.setLeftClose(false);
if (endTime == Long.MAX_VALUE) {
this.timeRange.setRightClose(false);
}
}
/**
......@@ -56,6 +60,12 @@ public class Deletion extends Modification implements Cloneable {
public Deletion(PartialPath path, long fileOffset, long startTime, long endTime) {
super(Type.DELETION, path, fileOffset);
this.timeRange = new TimeRange(startTime, endTime);
if (startTime == Long.MIN_VALUE) {
this.timeRange.setLeftClose(false);
}
if (endTime == Long.MAX_VALUE) {
this.timeRange.setRightClose(false);
}
}
public long getStartTime() {
......
......@@ -29,6 +29,8 @@ import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.concurrent.GuardedBy;
import java.io.File;
import java.io.IOException;
import java.nio.file.FileAlreadyExistsException;
......@@ -37,7 +39,10 @@ import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.stream.Collectors;
/**
* ModificationFile stores the Modifications of a TsFile or unseq file in another file in the same
......@@ -47,15 +52,20 @@ public class ModificationFile implements AutoCloseable {
private static final Logger logger = LoggerFactory.getLogger(ModificationFile.class);
public static final String FILE_SUFFIX = ".mods";
public static final String COMPACT_SUFFIX = ".settle";
public static final String COMPACTION_FILE_SUFFIX = ".compaction.mods";
// lazy loaded, set null when closed
private List<Modification> modifications;
private ModificationWriter writer;
private ModificationReader reader;
// whether to verify the last line, it may be incomplete in extreme cases
private boolean needVerify = true;
private final ModificationWriter writer;
private final ModificationReader reader;
private String filePath;
private Random random = new Random();
private static final long COMPACT_THRESHOLD = 1024 * 1024;
private boolean hasCompacted = false;
/**
* Construct a ModificationFile using a file as its storage.
*
......@@ -68,33 +78,11 @@ public class ModificationFile implements AutoCloseable {
this.filePath = filePath;
}
private void init() {
synchronized (this) {
modifications = (List<Modification>) reader.read();
}
}
private void checkInit() {
if (modifications == null) {
init();
}
}
/** Release resources such as streams and caches. */
@Override
public void close() throws IOException {
synchronized (this) {
writer.close();
modifications = null;
}
}
public void abort() throws IOException {
synchronized (this) {
writer.abort();
if (modifications != null && !modifications.isEmpty()) {
modifications.remove(modifications.size() - 1);
}
}
}
......@@ -107,13 +95,19 @@ public class ModificationFile implements AutoCloseable {
*/
public void write(Modification mod) throws IOException {
synchronized (this) {
writer.write(mod);
if (modifications != null) {
modifications.add(mod);
if (needVerify && new File(filePath).exists()) {
writer.mayTruncateLastLine();
needVerify = false;
}
writer.write(mod);
}
}
@GuardedBy("TsFileResource-WriteLock")
public void truncate(long size) {
writer.truncate(size);
}
/**
* Get all modifications stored in this file.
*
......@@ -121,11 +115,14 @@ public class ModificationFile implements AutoCloseable {
*/
public Collection<Modification> getModifications() {
synchronized (this) {
checkInit();
return new ArrayList<>(modifications);
return reader.read();
}
}
public Iterable<Modification> getModificationsIter() {
return reader::getModificationIterator;
}
public String getFilePath() {
return filePath;
}
......@@ -136,7 +133,10 @@ public class ModificationFile implements AutoCloseable {
public void remove() throws IOException {
close();
FSFactoryProducer.getFSFactory().getFile(filePath).delete();
boolean deleted = FSFactoryProducer.getFSFactory().getFile(filePath).delete();
if (!deleted) {
logger.warn("Delete ModificationFile {} failed.", filePath);
}
}
public boolean exists() {
......@@ -189,4 +189,78 @@ public class ModificationFile implements AutoCloseable {
return 0;
}
}
public void compact() {
long originFileSize = getSize();
if (originFileSize > COMPACT_THRESHOLD && !hasCompacted) {
Map<String, List<Modification>> pathModificationMap =
getModifications().stream().collect(Collectors.groupingBy(Modification::getPathString));
String newModsFileName = filePath + COMPACT_SUFFIX;
try (ModificationFile compactedModificationFile = new ModificationFile(newModsFileName)) {
Set<Map.Entry<String, List<Modification>>> modificationsEntrySet =
pathModificationMap.entrySet();
for (Map.Entry<String, List<Modification>> modificationEntry : modificationsEntrySet) {
List<Modification> settledModifications = sortAndMerge(modificationEntry.getValue());
for (Modification settledModification : settledModifications) {
compactedModificationFile.write(settledModification);
}
}
} catch (IOException e) {
logger.error("compact mods file exception of {}", filePath, e);
}
try {
// remove origin mods file
this.remove();
// rename new mods file to origin name
Files.move(new File(newModsFileName).toPath(), new File(filePath).toPath());
logger.info("{} settle successful", filePath);
if (getSize() > COMPACT_THRESHOLD) {
logger.warn(
"After the mod file is settled, the file size is still greater than 1M,the size of the file before settle is {},after settled the file size is {}",
originFileSize,
getSize());
}
} catch (IOException e) {
logger.error("remove origin file or rename new mods file error.", e);
}
hasCompacted = true;
}
}
public static List<Modification> sortAndMerge(List<Modification> modifications) {
modifications.sort(
(o1, o2) -> {
if (!o1.getType().equals(o2.getType())) {
return o1.getType().compareTo(o2.getType());
} else if (!o1.getPath().equals(o2.getPath())) {
return o1.getPath().compareTo(o2.getPath());
} else if (o1.getFileOffset() != o2.getFileOffset()) {
return (int) (o1.getFileOffset() - o2.getFileOffset());
} else {
if (o1.getType() == Modification.Type.DELETION) {
Deletion del1 = (Deletion) o1;
Deletion del2 = (Deletion) o2;
return del1.getTimeRange().compareTo(del2.getTimeRange());
}
throw new IllegalArgumentException();
}
});
List<Modification> result = new ArrayList<>();
if (!modifications.isEmpty()) {
Deletion current = ((Deletion) modifications.get(0)).clone();
for (int i = 1; i < modifications.size(); i++) {
Deletion del = (Deletion) modifications.get(i);
if (current.intersects(del)) {
current.merge(del);
} else {
result.add(current);
current = del.clone();
}
}
result.add(current);
}
return result;
}
}
......@@ -21,24 +21,28 @@ package org.apache.iotdb.db.engine.modification.io;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.utils.TestOnly;
import org.apache.iotdb.db.engine.modification.Deletion;
import org.apache.iotdb.db.engine.modification.Modification;
import org.apache.iotdb.db.engine.modification.utils.TracedBufferedReader;
import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.BufferedReader;
import java.io.BufferedWriter;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.FileReader;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
/**
* LocalTextModificationAccessor uses a file on local file system to store the modifications in text
......@@ -49,9 +53,10 @@ public class LocalTextModificationAccessor
private static final Logger logger = LoggerFactory.getLogger(LocalTextModificationAccessor.class);
private static final String SEPARATOR = ",";
private static final String ABORT_MARK = "aborted";
private static final String NO_MODIFICATION_MSG =
"No modification has been written to this file[{}]";
private String filePath;
private final String filePath;
private BufferedWriter writer;
/**
......@@ -65,44 +70,82 @@ public class LocalTextModificationAccessor
@Override
public Collection<Modification> read() {
List<Modification> result = new ArrayList<>();
Iterator<Modification> iterator = getModificationIterator();
while (iterator.hasNext()) {
result.add(iterator.next());
}
return result;
}
// we need to hold the reader for the Iterator, cannot use auto close or close in finally block
@SuppressWarnings("java:S2095")
@Override
public Iterator<Modification> getModificationIterator() {
File file = FSFactoryProducer.getFSFactory().getFile(filePath);
if (!file.exists()) {
logger.debug("No modification has been written to this file");
return new ArrayList<>();
}
String line;
long truncatedSize = 0;
boolean crashed = false;
List<Modification> modificationList = new ArrayList<>();
try (TracedBufferedReader reader = new TracedBufferedReader(new FileReader(file))) {
while ((line = reader.readLine()) != null) {
if (line.equals(ABORT_MARK) && !modificationList.isEmpty()) {
modificationList.remove(modificationList.size() - 1);
} else {
modificationList.add(decodeModification(line));
final BufferedReader reader;
try {
reader = new BufferedReader(new FileReader(file));
} catch (FileNotFoundException e) {
logger.debug(NO_MODIFICATION_MSG, file);
// return empty iterator
return new Iterator<Modification>() {
@Override
public boolean hasNext() {
return false;
}
truncatedSize = reader.position();
@Override
public Modification next() {
throw new NoSuchElementException();
}
};
}
final Modification[] cachedModification = new Modification[1];
return new Iterator<Modification>() {
@Override
public boolean hasNext() {
try {
if (cachedModification[0] == null) {
String line = reader.readLine();
if (line == null) {
reader.close();
return false;
} else {
return decodeModificationAndCache(reader, cachedModification, line);
}
}
} catch (IOException e) {
logger.warn("An error occurred when reading modifications", e);
}
return true;
}
} catch (IOException e) {
crashed = true;
logger.error(
"An error occurred when reading modifications, and the remaining modifications will be truncated to size {}.",
truncatedSize,
e);
}
if (crashed) {
try (FileOutputStream outputStream = new FileOutputStream(file, true)) {
outputStream.getChannel().truncate(truncatedSize);
} catch (FileNotFoundException e) {
logger.debug("No modification has been written to this file");
} catch (IOException e) {
logger.error(
"An error occurred when truncating modifications to size {}.", truncatedSize, e);
@Override
public Modification next() {
if (cachedModification[0] == null) {
throw new NoSuchElementException();
}
Modification result = cachedModification[0];
cachedModification[0] = null;
return result;
}
};
}
private boolean decodeModificationAndCache(
BufferedReader reader, Modification[] cachedModification, String line) throws IOException {
try {
cachedModification[0] = decodeModification(line);
return true;
} catch (IOException e) {
logger.warn("An error occurred when decode line-[{}] to modification", line);
cachedModification[0] = null;
reader.close();
return false;
}
return modificationList;
}
@Override
......@@ -114,23 +157,71 @@ public class LocalTextModificationAccessor
}
@Override
public void abort() throws IOException {
public void write(Modification mod) throws IOException {
if (writer == null) {
writer = FSFactoryProducer.getFSFactory().getBufferedWriter(filePath, true);
}
writer.write(ABORT_MARK);
writer.write(encodeModification(mod));
writer.newLine();
writer.flush();
}
@Override
public void write(Modification mod) throws IOException {
@TestOnly
public void writeInComplete(Modification mod) throws IOException {
if (writer == null) {
writer = FSFactoryProducer.getFSFactory().getBufferedWriter(filePath, true);
}
writer.write(encodeModification(mod));
writer.newLine();
writer.flush();
String line = encodeModification(mod);
if (line != null) {
writer.write(line.substring(0, 2));
}
}
@TestOnly
public void writeMeetException(Modification mod) throws IOException {
if (writer == null) {
writer = FSFactoryProducer.getFSFactory().getBufferedWriter(filePath, true);
}
writeInComplete(mod);
throw new IOException();
}
@Override
public void truncate(long size) {
try (FileOutputStream outputStream =
new FileOutputStream(FSFactoryProducer.getFSFactory().getFile(filePath), true)) {
outputStream.getChannel().truncate(size);
logger.warn("The modifications[{}] will be truncated to size {}.", filePath, size);
} catch (FileNotFoundException e) {
logger.debug(NO_MODIFICATION_MSG, filePath);
} catch (IOException e) {
logger.error(
"An error occurred when truncating modifications[{}] to size {}.", filePath, size, e);
}
}
@Override
public void mayTruncateLastLine() {
try (RandomAccessFile file = new RandomAccessFile(filePath, "r")) {
long filePointer = file.length() - 1;
if (filePointer == 0) {
return;
}
file.seek(filePointer);
byte lastChar = file.readByte();
if (lastChar != '\n') {
while (filePointer > -1 && lastChar != '\n') {
file.seek(filePointer);
filePointer--;
lastChar = file.readByte();
}
logger.warn("The last line of Mods is incomplete, will be truncated");
truncate(filePointer + 2);
}
} catch (IOException e) {
logger.error("An error occurred when reading modifications", e);
}
}
private static String encodeModification(Modification mod) {
......
......@@ -23,6 +23,7 @@ import org.apache.iotdb.db.engine.modification.Modification;
import java.io.IOException;
import java.util.Collection;
import java.util.Iterator;
/** ModificationReader reads all modifications from a persistent medium like file system. */
public interface ModificationReader {
......@@ -35,6 +36,14 @@ public interface ModificationReader {
*/
Collection<Modification> read();
/**
* Get an iterator over this mod file, others keep consistence with {@link #read()}. Please ensure
* you have called hasNext() with return of {@code true} before calling next().
*
* @return the modification iterator.
*/
Iterator<Modification> getModificationIterator();
/** Release resources like streams. */
void close() throws IOException;
}
......@@ -37,9 +37,10 @@ public interface ModificationWriter {
*/
void write(Modification mod) throws IOException;
void truncate(long size);
void mayTruncateLastLine();
/** Release resources like streams. */
void close() throws IOException;
/** Abort last modification. Notice that after calling abort(), a fileWriter is opened. */
void abort() throws IOException;
}
/*
* 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.iotdb.db.engine.modification.utils;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.Reader;
import java.io.UncheckedIOException;
import java.util.Iterator;
import java.util.NoSuchElementException;
import java.util.Spliterator;
import java.util.Spliterators;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
/**
* Copied from {@link java.io.BufferedReader}, trace the read position by modifying the fill()
* method.
*/
public class TracedBufferedReader extends Reader {
private Reader in;
private char cb[];
private int nChars, nextChar;
private static final int INVALIDATED = -2;
private static final int UNMARKED = -1;
private int markedChar = UNMARKED;
private int readAheadLimit = 0; /* Valid only when markedChar > 0 */
/** If the next character is a line feed, skip it */
private boolean skipLF = false;
/** The skipLF flag when the mark was set */
private boolean markedSkipLF = false;
private static int defaultCharBufferSize = 8192;
private static int defaultExpectedLineLength = 80;
/** the total bytes number already filled into cb */
private long totalFilledBytesNum = 0;
/**
* Creates a buffering character-input stream that uses an input buffer of the specified size.
*
* @param in A Reader
* @param sz Input-buffer size
* @exception IllegalArgumentException If {@code sz <= 0}
*/
public TracedBufferedReader(Reader in, int sz) {
super(in);
if (sz <= 0) {
throw new IllegalArgumentException("Buffer size <= 0");
}
this.in = in;
cb = new char[sz];
nextChar = nChars = 0;
}
/**
* Creates a buffering character-input stream that uses a default-sized input buffer.
*
* @param in A Reader
*/
public TracedBufferedReader(Reader in) {
this(in, defaultCharBufferSize);
}
/** Checks to make sure that the stream has not been closed */
private void ensureOpen() throws IOException {
if (in == null) {
throw new IOException("Stream closed");
}
}
/** {@link BufferedReader#fill()} */
private void fill() throws IOException {
int dst;
if (markedChar <= UNMARKED) {
/* No mark */
dst = 0;
} else {
/* Marked */
int delta = nextChar - markedChar;
if (delta >= readAheadLimit) {
/* Gone past read-ahead limit: Invalidate mark */
markedChar = INVALIDATED;
readAheadLimit = 0;
dst = 0;
} else {
if (readAheadLimit <= cb.length) {
/* Shuffle in the current buffer */
System.arraycopy(cb, markedChar, cb, 0, delta);
markedChar = 0;
dst = delta;
} else {
/* Reallocate buffer to accommodate read-ahead limit */
char ncb[] = new char[readAheadLimit];
System.arraycopy(cb, markedChar, ncb, 0, delta);
cb = ncb;
markedChar = 0;
dst = delta;
}
nextChar = nChars = delta;
}
}
int n;
do {
n = in.read(cb, dst, cb.length - dst);
} while (n == 0);
if (n > 0) {
nChars = dst + n;
nextChar = dst;
totalFilledBytesNum = totalFilledBytesNum + n;
}
}
/** {@link BufferedReader#read()} */
@Override
public int read() throws IOException {
synchronized (lock) {
ensureOpen();
for (; ; ) {
if (nextChar >= nChars) {
fill();
if (nextChar >= nChars) {
return -1;
}
}
if (skipLF) {
skipLF = false;
if (cb[nextChar] == '\n') {
nextChar++;
continue;
}
}
return cb[nextChar++];
}
}
}
/** {@link BufferedReader#read1(char[], int, int)} */
private int read1(char[] cbuf, int off, int len) throws IOException {
if (nextChar >= nChars) {
/* If the requested length is at least as large as the buffer, and
if there is no mark/reset activity, and if line feeds are not
being skipped, do not bother to copy the characters into the
local buffer. In this way buffered streams will cascade
harmlessly. */
if (len >= cb.length && markedChar <= UNMARKED && !skipLF) {
return in.read(cbuf, off, len);
}
fill();
}
if (nextChar >= nChars) {
return -1;
}
if (skipLF) {
skipLF = false;
if (cb[nextChar] == '\n') {
nextChar++;
if (nextChar >= nChars) {
fill();
}
if (nextChar >= nChars) {
return -1;
}
}
}
int n = Math.min(len, nChars - nextChar);
System.arraycopy(cb, nextChar, cbuf, off, n);
nextChar += n;
return n;
}
/** {@link BufferedReader#read(char[], int, int)} */
@Override
public int read(char cbuf[], int off, int len) throws IOException {
synchronized (lock) {
ensureOpen();
if ((off < 0)
|| (off > cbuf.length)
|| (len < 0)
|| ((off + len) > cbuf.length)
|| ((off + len) < 0)) {
throw new IndexOutOfBoundsException();
} else if (len == 0) {
return 0;
}
int n = read1(cbuf, off, len);
if (n <= 0) {
return n;
}
while ((n < len) && in.ready()) {
int n1 = read1(cbuf, off + n, len - n);
if (n1 <= 0) {
break;
}
n += n1;
}
return n;
}
}
/** {@link BufferedReader#readLine(boolean)} */
String readLine(boolean ignoreLF) throws IOException {
StringBuilder s = null;
int startChar;
synchronized (lock) {
ensureOpen();
boolean omitLF = ignoreLF || skipLF;
bufferLoop:
for (; ; ) {
if (nextChar >= nChars) {
fill();
}
if (nextChar >= nChars) {
/* EOF */
if (s != null && s.length() > 0) {
return s.toString();
} else {
return null;
}
}
boolean eol = false;
char c = 0;
int i;
/* Skip a leftover '\n', if necessary */
if (omitLF && (cb[nextChar] == '\n')) {
nextChar++;
}
skipLF = false;
omitLF = false;
charLoop:
for (i = nextChar; i < nChars; i++) {
c = cb[i];
if ((c == '\n') || (c == '\r')) {
eol = true;
break charLoop;
}
}
startChar = nextChar;
nextChar = i;
if (eol) {
String str;
if (s == null) {
str = new String(cb, startChar, i - startChar);
} else {
s.append(cb, startChar, i - startChar);
str = s.toString();
}
nextChar++;
if (c == '\r') {
skipLF = true;
if (read() != -1) {
nextChar--;
}
}
return str;
}
if (s == null) {
s = new StringBuilder(defaultExpectedLineLength);
}
s.append(cb, startChar, i - startChar);
}
}
}
/** {@link BufferedReader#readLine()} */
public String readLine() throws IOException {
return readLine(false);
}
/** {@link BufferedReader#skip(long)} */
@Override
public long skip(long n) throws IOException {
if (n < 0L) {
throw new IllegalArgumentException("skip value is negative");
}
synchronized (lock) {
ensureOpen();
long r = n;
while (r > 0) {
if (nextChar >= nChars) {
fill();
}
if (nextChar >= nChars) {
/* EOF */
break;
}
if (skipLF) {
skipLF = false;
if (cb[nextChar] == '\n') {
nextChar++;
}
}
long d = (long) nChars - nextChar;
if (r <= d) {
nextChar += r;
r = 0;
break;
} else {
r -= d;
nextChar = nChars;
}
}
return n - r;
}
}
/** {@link BufferedReader#ready()} */
@Override
public boolean ready() throws IOException {
synchronized (lock) {
ensureOpen();
/*
* If newline needs to be skipped and the next char to be read
* is a newline character, then just skip it right away.
*/
if (skipLF) {
/* Note that in.ready() will return true if and only if the next
* read on the stream will not block.
*/
if (nextChar >= nChars && in.ready()) {
fill();
}
if (nextChar < nChars) {
if (cb[nextChar] == '\n') {
nextChar++;
}
skipLF = false;
}
}
return (nextChar < nChars) || in.ready();
}
}
/** {@link BufferedReader#markSupported()} */
@Override
public boolean markSupported() {
return true;
}
/** {@link BufferedReader#mark(int)} */
@Override
public void mark(int readAheadLimit) throws IOException {
if (readAheadLimit < 0) {
throw new IllegalArgumentException("Read-ahead limit < 0");
}
synchronized (lock) {
ensureOpen();
this.readAheadLimit = readAheadLimit;
markedChar = nextChar;
markedSkipLF = skipLF;
}
}
/** {@link BufferedReader#reset()} */
@Override
public void reset() throws IOException {
synchronized (lock) {
ensureOpen();
if (markedChar < 0) {
throw new IOException((markedChar == INVALIDATED) ? "Mark invalid" : "Stream not marked");
}
nextChar = markedChar;
skipLF = markedSkipLF;
}
}
/** {@link BufferedReader#close()} */
@Override
public void close() throws IOException {
synchronized (lock) {
if (in == null) {
return;
}
try {
in.close();
} finally {
in = null;
cb = null;
}
}
}
/** {@link BufferedReader#lines()} */
public Stream<String> lines() {
Iterator<String> iter =
new Iterator<String>() {
String nextLine = null;
@Override
public boolean hasNext() {
if (nextLine != null) {
return true;
} else {
try {
nextLine = readLine();
return (nextLine != null);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}
}
@Override
public String next() {
if (nextLine != null || hasNext()) {
String line = nextLine;
nextLine = null;
return line;
} else {
throw new NoSuchElementException();
}
}
};
return StreamSupport.stream(
Spliterators.spliteratorUnknownSize(iter, Spliterator.ORDERED | Spliterator.NONNULL),
false);
}
/**
* Returns this reader's file position.
*
* @return This reader's file position, a non-negative integer counting the number of bytes from
* the beginning of the file to the current position
*/
public long position() {
// position = totalFilledBytesNum - lastFilledBytesNum + readOffsetInLastFilledBytes
// lastFilledBytesNum = nChars - dst, readOffsetInLastFilledBytes = nextChar - dst
return totalFilledBytesNum - nChars + nextChar;
}
}
......@@ -1888,8 +1888,6 @@ public class DataRegion implements IDataRegionForQuery {
// mod files in mergingModification, sequenceFileList, and unsequenceFileList
writeLock("delete");
// record files which are updated so that we can roll back them in case of exception
List<ModificationFile> updatedModFiles = new ArrayList<>();
boolean hasReleasedLock = false;
try {
......@@ -1917,21 +1915,13 @@ public class DataRegion implements IDataRegionForQuery {
List<TsFileResource> unsealedTsFileResource = new ArrayList<>();
separateTsFile(sealedTsFileResource, unsealedTsFileResource);
deleteDataInFiles(
unsealedTsFileResource, deletion, devicePaths, updatedModFiles, timePartitionFilter);
deleteDataInFiles(unsealedTsFileResource, deletion, devicePaths, timePartitionFilter);
writeUnlock();
hasReleasedLock = true;
deleteDataInFiles(
sealedTsFileResource, deletion, devicePaths, updatedModFiles, timePartitionFilter);
deleteDataInFiles(sealedTsFileResource, deletion, devicePaths, timePartitionFilter);
} catch (Exception e) {
// roll back
for (ModificationFile modFile : updatedModFiles) {
modFile.abort();
// remember to close mod file
modFile.close();
}
throw new IOException(e);
} finally {
if (!hasReleasedLock) {
......@@ -2022,11 +2012,12 @@ public class DataRegion implements IDataRegionForQuery {
return true;
}
// suppress warn of Throwable catch
@SuppressWarnings("java:S1181")
private void deleteDataInFiles(
Collection<TsFileResource> tsFileResourceList,
Deletion deletion,
Set<PartialPath> devicePaths,
List<ModificationFile> updatedModFiles,
TimePartitionFilter timePartitionFilter)
throws IOException {
for (TsFileResource tsFileResource : tsFileResourceList) {
......@@ -2039,39 +2030,52 @@ public class DataRegion implements IDataRegionForQuery {
continue;
}
ModificationFile modFile = tsFileResource.getModFile();
if (tsFileResource.isClosed()) {
// delete data in sealed file
if (tsFileResource.isCompacting()) {
// we have to set modification offset to MAX_VALUE, as the offset of source chunk may
// change after compaction
deletion.setFileOffset(Long.MAX_VALUE);
// write deletion into compaction modification file
tsFileResource.getCompactionModFile().write(deletion);
// write deletion into modification file to enable query during compaction
tsFileResource.getModFile().write(deletion);
// remember to close mod file
tsFileResource.getCompactionModFile().close();
tsFileResource.getModFile().close();
} else {
deletion.setFileOffset(tsFileResource.getTsFileSize());
// write deletion into modification file
boolean modFileExists = tsFileResource.getModFile().exists();
long originSize = tsFileResource.getModFile().getSize();
tsFileResource.getModFile().write(deletion);
// remember to close mod file
tsFileResource.getModFile().close();
if (!modFileExists) {
TsFileMetricManager.getInstance().increaseModFileNum(1);
long originSize = -1;
synchronized (modFile) {
try {
originSize = modFile.getSize();
// delete data in sealed file
if (tsFileResource.isCompacting()) {
// we have to set modification offset to MAX_VALUE, as the offset of source chunk may
// change after compaction
deletion.setFileOffset(Long.MAX_VALUE);
// write deletion into compaction modification file
tsFileResource.getCompactionModFile().write(deletion);
// write deletion into modification file to enable read during compaction
modFile.write(deletion);
// remember to close mod file
tsFileResource.getCompactionModFile().close();
modFile.close();
} else {
deletion.setFileOffset(tsFileResource.getTsFileSize());
// write deletion into modification file
boolean modFileExists = tsFileResource.getModFile().exists();
tsFileResource.getModFile().write(deletion);
// remember to close mod file
tsFileResource.getModFile().close();
// if file length greater than 1M,execute compact.
modFile.compact();
if (!modFileExists) {
TsFileMetricManager.getInstance().increaseModFileNum(1);
}
TsFileMetricManager.getInstance()
.increaseModFileSize(tsFileResource.getModFile().getSize() - originSize);
}
} catch (Throwable t) {
if (originSize != -1) {
modFile.truncate(originSize);
}
throw t;
}
TsFileMetricManager.getInstance()
.increaseModFileSize(tsFileResource.getModFile().getSize() - originSize);
logger.info(
"[Deletion] Deletion with path:{}, time:{}-{} written into mods file:{}.",
deletion.getPath(),
deletion.getStartTime(),
deletion.getEndTime(),
modFile.getFilePath());
}
logger.info(
"[Deletion] Deletion with path:{}, time:{}-{} written into mods file:{}.",
deletion.getPath(),
deletion.getStartTime(),
deletion.getEndTime(),
tsFileResource.getModFile().getFilePath());
} else {
// delete data in memory of unsealed file
tsFileResource.getProcessor().deleteDataInMemory(deletion, devicePaths);
......@@ -2081,9 +2085,6 @@ public class DataRegion implements IDataRegionForQuery {
SyncService.getInstance().getOrCreateSyncManager(dataRegionId)) {
syncManager.syncRealTimeDeletion(deletion);
}
// add a record in case of rollback
updatedModFiles.add(tsFileResource.getModFile());
}
}
......
......@@ -22,7 +22,6 @@ package org.apache.iotdb.db.query.context;
import org.apache.iotdb.commons.path.AlignedPath;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.path.PatternTreeMap;
import org.apache.iotdb.db.engine.modification.Deletion;
import org.apache.iotdb.db.engine.modification.Modification;
import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.metadata.path.PatternTreeMapFactory;
......@@ -108,50 +107,15 @@ public class QueryContext {
fileModCache.get(modFile.getFilePath());
if (allModifications == null) {
allModifications = PatternTreeMapFactory.getModsPatternTreeMap();
for (Modification modification : modFile.getModifications()) {
for (Modification modification : modFile.getModificationsIter()) {
allModifications.append(modification.getPath(), modification);
}
fileModCache.put(modFile.getFilePath(), allModifications);
}
return sortAndMerge(allModifications.getOverlapped(path));
return ModificationFile.sortAndMerge(allModifications.getOverlapped(path));
});
}
private List<Modification> sortAndMerge(List<Modification> modifications) {
modifications.sort(
(o1, o2) -> {
if (!o1.getType().equals(o2.getType())) {
return o1.getType().compareTo(o2.getType());
} else if (!o1.getPath().equals(o2.getPath())) {
return o1.getPath().compareTo(o2.getPath());
} else if (o1.getFileOffset() != o2.getFileOffset()) {
return (int) (o1.getFileOffset() - o2.getFileOffset());
} else {
if (o1.getType() == Modification.Type.DELETION) {
Deletion del1 = (Deletion) o1;
Deletion del2 = (Deletion) o2;
return del1.getTimeRange().compareTo(del2.getTimeRange());
}
throw new IllegalArgumentException();
}
});
List<Modification> result = new ArrayList<>();
if (!modifications.isEmpty()) {
Deletion current = ((Deletion) modifications.get(0)).clone();
for (int i = 1; i < modifications.size(); i++) {
Deletion del = (Deletion) modifications.get(i);
if (current.intersects(del)) {
current.merge(del);
} else {
result.add(current);
current = del.clone();
}
}
result.add(current);
}
return result;
}
/**
* Find the modifications of all aligned 'paths' in 'modFile'. If they are not in the cache, read
* them from 'modFile' and put then into the cache.
......
......@@ -212,6 +212,9 @@ public class SystemInfo {
}
public synchronized void resetCompactionMemoryCost(long compactionMemoryCost) {
if (!config.isEnableCompactionMemControl()) {
return;
}
this.compactionMemoryCost.addAndGet(-compactionMemoryCost);
}
......
......@@ -18,6 +18,8 @@
*/
package org.apache.iotdb.db.engine.compaction;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.execute.performer.impl.FastCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.execute.performer.impl.ReadPointCompactionPerformer;
......@@ -28,6 +30,7 @@ import org.apache.iotdb.db.engine.compaction.schedule.CompactionTaskManager;
import org.apache.iotdb.db.engine.compaction.schedule.comparator.DefaultCompactionTaskComparatorImpl;
import org.apache.iotdb.db.engine.compaction.schedule.constant.CompactionPriority;
import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
import org.apache.iotdb.db.engine.modification.Deletion;
import org.apache.iotdb.db.engine.storagegroup.TsFileManager;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.utils.datastructure.FixedPriorityBlockingQueue;
......@@ -41,7 +44,9 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
......@@ -374,6 +379,40 @@ public class CompactionTaskComparatorTest {
}
}
@Test
public void testCompareByMaxModsFileSize()
throws InterruptedException, IllegalPathException, IOException {
for (int i = 0; i < 100; ++i) {
List<TsFileResource> resources = new ArrayList<>();
for (int j = i; j < 100; ++j) {
resources.add(
new FakedTsFileResource(new File(String.format("%d-%d-0-0.tsfile", i + j, i + j)), j));
}
FakedInnerSpaceCompactionTask innerTask =
new FakedInnerSpaceCompactionTask(
"fakeSg", 0, tsFileManager, taskNum, true, resources, 0);
compactionTaskQueue.put(innerTask);
}
String targetFileName = "101-101-0-0.tsfile";
FakedTsFileResource fakedTsFileResource =
new FakedTsFileResource(new File(targetFileName), 100);
fakedTsFileResource.getModFile().write(new Deletion(new PartialPath("root.test.d1"), 1, 1));
compactionTaskQueue.put(
new FakedInnerSpaceCompactionTask(
"fakeSg",
0,
tsFileManager,
taskNum,
true,
Collections.singletonList(fakedTsFileResource),
0));
FakedInnerSpaceCompactionTask task = (FakedInnerSpaceCompactionTask) compactionTaskQueue.take();
Assert.assertEquals(
targetFileName, task.getSelectedTsFileResourceList().get(0).getTsFile().getName());
fakedTsFileResource.getModFile().remove();
}
private static class FakedInnerSpaceCompactionTask extends InnerSpaceCompactionTask {
public FakedInnerSpaceCompactionTask(
......
/*
* 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.iotdb.db.engine.compaction;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.engine.compaction.execute.performer.impl.FastCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.execute.task.CrossSpaceCompactionTask;
import org.apache.iotdb.db.engine.compaction.utils.CompactionTestFileWriter;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.exception.StorageEngineException;
import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.TimeRange;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
public class FastCompactionPerformerWithEmptyPageTest extends AbstractCompactionTest {
@Before
public void setUp()
throws IOException, WriteProcessException, MetadataException, InterruptedException {
super.setUp();
}
@After
public void tearDown() throws IOException, StorageEngineException {
super.tearDown();
}
@Test
public void test1() throws IOException, IllegalPathException {
String device = "root.testsg.d1";
TsFileResource seqFile1 = createEmptyFileAndResource(true);
try (CompactionTestFileWriter writer = new CompactionTestFileWriter(seqFile1)) {
writer.startChunkGroup("d1");
writer.generateSimpleAlignedSeriesToCurrentDeviceWithNullValue(
Arrays.asList("s1", "s2", "s3"),
new TimeRange[][] {new TimeRange[] {new TimeRange(10, 30)}},
TSEncoding.RLE,
CompressionType.UNCOMPRESSED,
Arrays.asList(false, true, true));
writer.generateSimpleAlignedSeriesToCurrentDeviceWithNullValue(
Arrays.asList("s1", "s2", "s3"),
new TimeRange[][] {new TimeRange[] {new TimeRange(40, 50)}},
TSEncoding.RLE,
CompressionType.UNCOMPRESSED,
Arrays.asList(false, false, false));
writer.endChunkGroup();
writer.endFile();
}
seqFile1.updateStartTime(device, 10);
seqFile1.updateEndTime(device, 50);
seqFile1.serialize();
generateModsFile(Arrays.asList(new PartialPath("root.testsg.d1.s1")), seqFile1, 0, 31);
TsFileResource unseqFile1 = createEmptyFileAndResource(false);
try (CompactionTestFileWriter writer = new CompactionTestFileWriter(unseqFile1)) {
writer.startChunkGroup("d1");
writer.generateSimpleAlignedSeriesToCurrentDevice(
Arrays.asList("s1", "s2", "s3"),
new TimeRange[][] {new TimeRange[] {new TimeRange(20, 34)}},
TSEncoding.RLE,
CompressionType.UNCOMPRESSED);
writer.endChunkGroup();
writer.endFile();
}
unseqFile1.updateStartTime(device, 20);
unseqFile1.updateEndTime(device, 34);
unseqFile1.serialize();
CrossSpaceCompactionTask task =
new CrossSpaceCompactionTask(
0,
tsFileManager,
Arrays.asList(seqFile1),
Arrays.asList(unseqFile1),
new FastCompactionPerformer(true),
new AtomicInteger(0),
0,
0);
try {
Assert.assertTrue(task.start());
} catch (Exception e) {
Assert.fail();
}
TsFileResource result = tsFileManager.getTsFileList(true).get(0);
result.buildDeviceTimeIndex();
Assert.assertEquals(20, result.getStartTime(device));
Assert.assertEquals(50, result.getEndTime(device));
validateSeqFiles(true);
try (TsFileSequenceReader reader = new TsFileSequenceReader(result.getTsFilePath())) {
Map<String, List<ChunkMetadata>> chunkMetadataInDevice =
reader.readChunkMetadataInDevice(device);
long startTime = Long.MAX_VALUE, endTime = Long.MIN_VALUE;
List<ChunkMetadata> chunkMetadataList = chunkMetadataInDevice.get("s1");
for (ChunkMetadata chunkMetadata : chunkMetadataList) {
startTime = Math.min(startTime, chunkMetadata.getStartTime());
endTime = Math.max(endTime, chunkMetadata.getEndTime());
}
Assert.assertEquals(20, startTime);
Assert.assertEquals(50, endTime);
}
}
}
......@@ -19,12 +19,15 @@
package org.apache.iotdb.db.engine.compaction.inner;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.AbstractCompactionTest;
import org.apache.iotdb.db.engine.compaction.execute.performer.impl.FastCompactionPerformer;
import org.apache.iotdb.db.engine.compaction.execute.task.InnerSpaceCompactionTask;
import org.apache.iotdb.db.engine.compaction.schedule.CompactionTaskManager;
import org.apache.iotdb.db.engine.compaction.selector.impl.SizeTieredCompactionSelector;
import org.apache.iotdb.db.engine.modification.Deletion;
import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
import org.apache.iotdb.db.exception.StorageEngineException;
......@@ -41,6 +44,8 @@ import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
public class InnerSpaceCompactionSelectorTest extends AbstractCompactionTest {
@Before
public void setUp()
......@@ -623,4 +628,32 @@ public class InnerSpaceCompactionSelectorTest extends AbstractCompactionTest {
Assert.fail();
}
}
@Test
public void testSelectWhenModsFileGreaterThan50M()
throws IOException, MetadataException, WriteProcessException {
createFiles(6, 2, 3, 50, 0, 10000, 50, 50, false, true);
tsFileManager.addAll(seqResources, true);
tsFileManager.addAll(unseqResources, false);
TsFileResource tsFileResource = seqResources.get(0);
ModificationFile modFile = tsFileResource.getModFile();
while (modFile.getSize() < 1024 * 1024 * 50) {
modFile.write(
new Deletion(
new PartialPath(COMPACTION_TEST_SG + PATH_SEPARATOR + "**"),
Long.MIN_VALUE,
Long.MAX_VALUE));
}
SizeTieredCompactionSelector selector =
new SizeTieredCompactionSelector("", "", 0, true, tsFileManager);
// copy candidate source file list
List<TsFileResource> resources = tsFileManager.getOrCreateSequenceListByTimePartition(0);
List<List<TsFileResource>> taskResource = selector.selectInnerSpaceTask(resources);
Assert.assertEquals(1, taskResource.size());
modFile.remove();
}
}
/*
* 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.iotdb.db.engine.compaction.utils;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.db.engine.compaction.AbstractCompactionTest;
import org.apache.iotdb.db.engine.compaction.selector.estimator.FastCompactionInnerCompactionEstimator;
import org.apache.iotdb.db.engine.compaction.selector.estimator.FastCrossSpaceCompactionEstimator;
import org.apache.iotdb.db.engine.compaction.selector.estimator.ReadChunkInnerCompactionEstimator;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.exception.StorageEngineException;
import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.util.List;
public class CompactionTaskMemCostEstimatorTest extends AbstractCompactionTest {
@Before
public void setUp()
throws IOException, WriteProcessException, MetadataException, InterruptedException {
super.setUp();
}
@After
public void tearDown() throws IOException, StorageEngineException {
super.tearDown();
}
@Test
public void testEstimateReadChunkInnerSpaceCompactionTaskMemCost()
throws IOException, MetadataException, WriteProcessException {
createFiles(3, 10, 5, 100000, 0, 0, 50, 50, true, true);
tsFileManager.addAll(seqResources, true);
List<TsFileResource> tsFileList = tsFileManager.getTsFileList(true);
System.out.println(tsFileList.get(0).getTsFile().getAbsolutePath());
long cost = new ReadChunkInnerCompactionEstimator().estimateInnerCompactionMemory(tsFileList);
Assert.assertTrue(cost > 0);
}
@Test
public void testEstimateReadChunkInnerSpaceCompactionTaskMemCost2()
throws IOException, MetadataException, WriteProcessException {
createFiles(3, 10, 5, 100, 0, 0, 50, 50, false, true);
tsFileManager.addAll(seqResources, true);
List<TsFileResource> tsFileList = tsFileManager.getTsFileList(true);
long cost = new ReadChunkInnerCompactionEstimator().estimateInnerCompactionMemory(tsFileList);
Assert.assertTrue(cost > 0);
}
@Test
public void testEstimateFastCompactionInnerSpaceCompactionTaskMemCost()
throws IOException, MetadataException, WriteProcessException {
createFiles(5, 10, 5, 10000, 0, 0, 50, 50, true, false);
createFiles(10, 4, 5, 10000, 1000, 0, 30, 90, true, false);
tsFileManager.addAll(unseqResources, false);
long cost =
new FastCompactionInnerCompactionEstimator().estimateInnerCompactionMemory(unseqResources);
Assert.assertTrue(cost > 0);
}
@Test
public void testEstimateFastCompactionInnerSpaceCompactionTaskMemCost2()
throws IOException, MetadataException, WriteProcessException {
createFiles(3, 10, 5, 100, 0, 0, 50, 50, false, true);
tsFileManager.addAll(seqResources, true);
List<TsFileResource> tsFileList = tsFileManager.getTsFileList(true);
long cost =
new FastCompactionInnerCompactionEstimator().estimateInnerCompactionMemory(tsFileList);
Assert.assertTrue(cost > 0);
}
@Test
public void testEstimateFastCompactionCrossSpaceCompactionTaskMemCost1()
throws IOException, MetadataException, WriteProcessException {
createFiles(3, 10, 5, 100, 0, 0, 50, 50, false, true);
createFiles(4, 10, 5, 400, 0, 0, 30, 50, false, false);
long cost =
new FastCrossSpaceCompactionEstimator()
.estimateCrossCompactionMemory(seqResources, unseqResources);
Assert.assertTrue(cost > 0);
}
}
......@@ -31,12 +31,13 @@ import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
public class CompactionTestFileWriter {
public class CompactionTestFileWriter implements Closeable {
private TsFileResource resource;
private TsFileIOWriter fileWriter;
......@@ -170,6 +171,34 @@ public class CompactionTestFileWriter {
}
}
public void generateSimpleAlignedSeriesToCurrentDeviceWithNullValue(
List<String> measurementNames,
TimeRange[] toGenerateChunkTimeRanges,
TSEncoding encoding,
CompressionType compressionType,
List<Boolean> nullMeasurements)
throws IOException {
List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
for (String measurementName : measurementNames) {
measurementSchemas.add(
new MeasurementSchema(measurementName, TSDataType.INT32, encoding, compressionType));
}
for (TimeRange toGenerateChunk : toGenerateChunkTimeRanges) {
AlignedChunkWriterImpl alignedChunkWriter = new AlignedChunkWriterImpl(measurementSchemas);
currentDeviceStartTime = Math.min(toGenerateChunk.getMin(), currentDeviceStartTime);
currentDeviceEndTime = Math.max(toGenerateChunk.getMax(), currentDeviceEndTime);
for (long time = toGenerateChunk.getMin(); time <= toGenerateChunk.getMax(); time++) {
alignedChunkWriter.getTimeChunkWriter().write(time);
for (int i = 0; i < measurementNames.size(); i++) {
alignedChunkWriter
.getValueChunkWriterByIndex(i)
.write(time, new Random().nextInt(), nullMeasurements.get(i));
}
}
alignedChunkWriter.writeToFileWriter(fileWriter);
}
}
public void generateSimpleAlignedSeriesToCurrentDevice(
List<String> measurementNames,
TimeRange[][] toGenerateChunkPageTimeRanges,
......@@ -203,6 +232,40 @@ public class CompactionTestFileWriter {
}
}
public void generateSimpleAlignedSeriesToCurrentDeviceWithNullValue(
List<String> measurementNames,
TimeRange[][] toGenerateChunkPageTimeRanges,
TSEncoding encoding,
CompressionType compressionType,
List<Boolean> nullMeasurement)
throws IOException {
List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
for (String measurementName : measurementNames) {
measurementSchemas.add(
new MeasurementSchema(measurementName, TSDataType.INT32, encoding, compressionType));
}
for (TimeRange[] toGenerateChunk : toGenerateChunkPageTimeRanges) {
AlignedChunkWriterImpl alignedChunkWriter = new AlignedChunkWriterImpl(measurementSchemas);
for (TimeRange toGeneratePageTimeRange : toGenerateChunk) {
currentDeviceStartTime = Math.min(toGeneratePageTimeRange.getMin(), currentDeviceStartTime);
currentDeviceEndTime = Math.max(toGeneratePageTimeRange.getMax(), currentDeviceEndTime);
for (long time = toGeneratePageTimeRange.getMin();
time <= toGeneratePageTimeRange.getMax();
time++) {
alignedChunkWriter.write(time);
for (int i = 0; i < measurementNames.size(); i++) {
alignedChunkWriter
.getValueChunkWriterByIndex(i)
.getPageWriter()
.write(time, new Random().nextInt(), nullMeasurement.get(i));
}
}
alignedChunkWriter.sealCurrentPage();
}
alignedChunkWriter.writeToFileWriter(fileWriter);
}
}
public void generateSimpleAlignedSeriesToCurrentDevice(
List<String> measurementNames,
TimeRange[][][] toGenerateChunkPageTimeRanges,
......@@ -235,4 +298,38 @@ public class CompactionTestFileWriter {
alignedChunkWriter.writeToFileWriter(fileWriter);
}
}
public void generateSimpleAlignedSeriesToCurrentDeviceWithNullValue(
List<String> measurementNames,
TimeRange[][][] toGenerateChunkPageTimeRanges,
TSEncoding encoding,
CompressionType compressionType,
List<Boolean> nullMeasurements)
throws IOException {
List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
for (String measurementName : measurementNames) {
measurementSchemas.add(
new MeasurementSchema(measurementName, TSDataType.INT32, encoding, compressionType));
}
for (TimeRange[][] toGenerateChunk : toGenerateChunkPageTimeRanges) {
AlignedChunkWriterImpl alignedChunkWriter = new AlignedChunkWriterImpl(measurementSchemas);
for (TimeRange[] toGeneratePageTimeRanges : toGenerateChunk) {
for (TimeRange pointsTimeRange : toGeneratePageTimeRanges) {
currentDeviceStartTime = Math.min(pointsTimeRange.getMin(), currentDeviceStartTime);
currentDeviceEndTime = Math.max(pointsTimeRange.getMax(), currentDeviceEndTime);
for (long time = pointsTimeRange.getMin(); time <= pointsTimeRange.getMax(); time++) {
alignedChunkWriter.write(time);
for (int i = 0; i < measurementNames.size(); i++) {
alignedChunkWriter
.getValueChunkWriterByIndex(i)
.getPageWriter()
.write(time, new Random().nextInt(), nullMeasurements.get(i));
}
}
}
alignedChunkWriter.sealCurrentPage();
}
alignedChunkWriter.writeToFileWriter(fileWriter);
}
}
}
......@@ -21,14 +21,19 @@ package org.apache.iotdb.db.engine.modification;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.constant.TestConstant;
import org.apache.iotdb.db.engine.compaction.execute.recover.CompactionRecoverManager;
import org.junit.Assert;
import org.junit.Test;
import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.util.ArrayList;
import java.util.List;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class ModificationFileTest {
......@@ -89,7 +94,6 @@ public class ModificationFileTest {
mFile.write(modifications[i]);
}
modificationList = (List<Modification>) mFile.getModifications();
mFile.abort();
for (int i = 0; i < 3; i++) {
assertEquals(modifications[i], modificationList.get(i));
......@@ -100,4 +104,181 @@ public class ModificationFileTest {
new File(tempFileName).delete();
}
}
// test if file size greater than 1M.
@Test
public void testCompact01() {
String tempFileName = TestConstant.BASE_OUTPUT_PATH.concat("compact01.mods");
long time = 1000;
try (ModificationFile modificationFile = new ModificationFile(tempFileName)) {
while (modificationFile.getSize() < 1024 * 1024) {
modificationFile.write(
new Deletion(
new PartialPath(new String[] {"root", "sg", "d1"}),
1000,
Long.MIN_VALUE,
time += 5000));
}
modificationFile.compact();
List<Modification> modificationList = new ArrayList<>(modificationFile.getModifications());
assertEquals(1, modificationList.size());
Deletion deletion = (Deletion) modificationList.get(0);
assertEquals(time, deletion.getEndTime());
assertEquals(Long.MIN_VALUE, deletion.getStartTime());
} catch (IOException e) {
fail(e.getMessage());
} finally {
new File(tempFileName).delete();
}
}
// test if file size less than 1M.
@Test
public void testCompact02() {
String tempFileName = TestConstant.BASE_OUTPUT_PATH.concat("compact02.mods");
long time = 1000;
try (ModificationFile modificationFile = new ModificationFile(tempFileName)) {
while (modificationFile.getSize() < 1024 * 100) {
modificationFile.write(
new Deletion(
new PartialPath(new String[] {"root", "sg", "d1"}),
1000,
Long.MIN_VALUE,
time += 5000));
}
modificationFile.compact();
List<Modification> modificationList = new ArrayList<>(modificationFile.getModifications());
assertTrue(modificationList.size() > 1);
} catch (IOException e) {
fail(e.getMessage());
} finally {
new File(tempFileName).delete();
}
}
// test if file size greater than 1M.
@Test
public void testCompact03() {
String tempFileName = TestConstant.BASE_OUTPUT_PATH.concat("compact03.mods");
try (ModificationFile modificationFile = new ModificationFile(tempFileName)) {
while (modificationFile.getSize() < 1024 * 1024) {
modificationFile.write(
new Deletion(
new PartialPath(new String[] {"root", "sg", "d1"}),
1000,
Long.MIN_VALUE,
Long.MAX_VALUE));
}
modificationFile.compact();
List<Modification> modificationList = new ArrayList<>(modificationFile.getModifications());
assertEquals(1, modificationList.size());
Deletion deletion = (Deletion) modificationList.get(0);
assertEquals(Long.MAX_VALUE, deletion.getEndTime());
assertEquals(Long.MIN_VALUE, deletion.getStartTime());
} catch (IOException e) {
fail(e.getMessage());
} finally {
new File(tempFileName).delete();
}
}
@Test
public void testCompact04() {
String tempFileName = TestConstant.BASE_OUTPUT_PATH.concat("compact04.mods");
try (ModificationFile modificationFile = new ModificationFile(tempFileName)) {
long time = 0;
while (modificationFile.getSize() < 1024 * 1024) {
for (int i = 0; i < 5; i++) {
modificationFile.write(
new Deletion(
new PartialPath(new String[] {"root", "sg", "d1"}),
1000,
Long.MIN_VALUE,
time += 5000));
modificationFile.write(
new Deletion(
new PartialPath(new String[] {"root", "sg", "*"}),
1000,
Long.MIN_VALUE,
time += 5000));
}
}
modificationFile.compact();
List<Modification> modificationList = new ArrayList<>(modificationFile.getModifications());
assertEquals(2, modificationList.size());
} catch (IOException e) {
fail(e.getMessage());
} finally {
new File(tempFileName).delete();
}
}
// test mods file and mods settle file both exists
@Test
public void testRecover01() {
String modsFileName = TestConstant.BASE_OUTPUT_PATH.concat("compact01.mods");
String modsSettleFileName = TestConstant.BASE_OUTPUT_PATH.concat("compact01.mods.settle");
try (ModificationFile modsFile = new ModificationFile(modsFileName);
ModificationFile modsSettleFile = new ModificationFile(modsSettleFileName)) {
modsFile.write(
new Deletion(
new PartialPath(new String[] {"root", "sg", "d1"}),
1000,
Long.MIN_VALUE,
Long.MAX_VALUE));
modsSettleFile.write(
new Deletion(
new PartialPath(new String[] {"root", "sg", "d1"}),
1000,
Long.MIN_VALUE,
Long.MAX_VALUE));
modsFile.close();
modsSettleFile.close();
new CompactionRecoverManager(null, null, null)
.recoverModSettleFile(new File(TestConstant.BASE_OUTPUT_PATH).toPath());
Assert.assertTrue(modsFile.exists());
Assert.assertFalse(modsSettleFile.exists());
} catch (IOException e) {
throw new RuntimeException(e);
} finally {
try {
Files.delete(new File(modsFileName).toPath());
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
// test only mods settle file exists
@Test
public void testRecover02() {
String modsSettleFileName = TestConstant.BASE_OUTPUT_PATH.concat("compact02.mods.settle");
String originModsFileName = TestConstant.BASE_OUTPUT_PATH.concat("compact02.mods");
try (ModificationFile modsSettleFile = new ModificationFile(modsSettleFileName)) {
modsSettleFile.write(
new Deletion(
new PartialPath(new String[] {"root", "sg", "d1"}),
1000,
Long.MIN_VALUE,
Long.MAX_VALUE));
modsSettleFile.close();
new CompactionRecoverManager(null, null, null)
.recoverModSettleFile(new File(TestConstant.BASE_OUTPUT_PATH).toPath());
Assert.assertFalse(modsSettleFile.exists());
Assert.assertTrue(new File(originModsFileName).exists());
} catch (IOException e) {
throw new RuntimeException(e);
} finally {
try {
Files.delete(new File(originModsFileName).toPath());
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
}
......@@ -24,11 +24,10 @@ import org.apache.iotdb.db.constant.TestConstant;
import org.apache.iotdb.db.engine.modification.Deletion;
import org.apache.iotdb.db.engine.modification.Modification;
import org.junit.AfterClass;
import org.junit.Test;
import java.io.BufferedWriter;
import java.io.File;
import java.io.FileWriter;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
......@@ -38,17 +37,53 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
public class LocalTextModificationAccessorTest {
private static Modification[] modifications =
new Modification[] {
new Deletion(new PartialPath(new String[] {"d1", "s1"}), 1, 1),
new Deletion(new PartialPath(new String[] {"d1", "s2"}), 2, 2),
new Deletion(new PartialPath(new String[] {"d1", "s3"}), 3, 3),
new Deletion(new PartialPath(new String[] {"d1", "s4"}), 4, 4),
};
@AfterClass
public static void tearDown() {
modifications = null;
}
@Test
public void writeMeetException() throws IOException {
String tempFileName = TestConstant.BASE_OUTPUT_PATH.concat("mod.temp");
long length = 0;
LocalTextModificationAccessor accessor = null;
try {
accessor = new LocalTextModificationAccessor(tempFileName);
for (int i = 0; i < 2; i++) {
accessor.write(modifications[i]);
}
length = new File(tempFileName).length();
// the current line should be truncated when meet exception
accessor.writeMeetException(modifications[2]);
for (int i = 2; i < 4; i++) {
accessor.write(modifications[i]);
}
List<Modification> modificationList = (List<Modification>) accessor.read();
assertEquals(4, modificationList.size());
for (int i = 0; i < 4; i++) {
assertEquals(modifications[i], modificationList.get(i));
}
} catch (IOException e) {
accessor.truncate(length);
} finally {
if (accessor != null) {
accessor.close();
}
new File(tempFileName).delete();
}
}
@Test
public void readMyWrite() {
String tempFileName = TestConstant.BASE_OUTPUT_PATH.concat("mod.temp");
Modification[] modifications =
new Modification[] {
new Deletion(new PartialPath(new String[] {"d1", "s1"}), 1, 1),
new Deletion(new PartialPath(new String[] {"d1", "s2"}), 2, 2),
new Deletion(new PartialPath(new String[] {"d1", "s3"}), 3, 3),
new Deletion(new PartialPath(new String[] {"d1", "s4"}), 4, 4),
};
try (LocalTextModificationAccessor accessor = new LocalTextModificationAccessor(tempFileName)) {
for (int i = 0; i < 2; i++) {
accessor.write(modifications[i]);
......@@ -75,52 +110,41 @@ public class LocalTextModificationAccessorTest {
@Test
public void readNull() {
String tempFileName = TestConstant.BASE_OUTPUT_PATH.concat("mod.temp");
LocalTextModificationAccessor accessor = new LocalTextModificationAccessor(tempFileName);
new File(tempFileName).delete();
Collection<Modification> modifications = accessor.read();
assertEquals(new ArrayList<>(), modifications);
try (LocalTextModificationAccessor accessor = new LocalTextModificationAccessor(tempFileName)) {
new File(tempFileName).delete();
Collection<Modification> modifications = accessor.read();
assertEquals(new ArrayList<>(), modifications);
} catch (Exception e) {
fail(e.getMessage());
}
}
@Test
public void readAndTruncate() {
public void readMeetError() {
String tempFileName = TestConstant.BASE_OUTPUT_PATH.concat("mod.temp");
File file = new File(tempFileName);
if (file.exists()) {
file.delete();
}
Modification[] modifications =
new Modification[] {
new Deletion(new PartialPath(new String[] {"d1", "s1"}), 1, 1),
new Deletion(new PartialPath(new String[] {"d1", "s2"}), 2, 2),
new Deletion(new PartialPath(new String[] {"d1", "s3"}), 3, 3),
new Deletion(new PartialPath(new String[] {"d1", "s4"}), 4, 4),
};
try (LocalTextModificationAccessor accessor = new LocalTextModificationAccessor(tempFileName);
BufferedWriter writer = new BufferedWriter(new FileWriter(tempFileName, true))) {
try (LocalTextModificationAccessor accessor = new LocalTextModificationAccessor(tempFileName)) {
// write normal message
for (int i = 0; i < 2; i++) {
for (int i = 0; i < 4; i++) {
accessor.write(modifications[i]);
}
List<Modification> modificationList = (List<Modification>) accessor.read();
for (int i = 0; i < 2; i++) {
for (int i = 0; i < 4; i++) {
assertEquals(modifications[i], modificationList.get(i));
}
// write error message
long length = file.length();
writer.write("error");
writer.newLine();
writer.flush();
// write normal message & read
for (int i = 2; i < 4; i++) {
accessor.write(modifications[i]);
}
accessor.writeInComplete(modifications[0]);
modificationList = (List<Modification>) accessor.read();
for (int i = 0; i < 2; i++) {
// the error line is ignored
assertEquals(4, modificationList.size());
for (int i = 0; i < 4; i++) {
System.out.println(modificationList);
assertEquals(modifications[i], modificationList.get(i));
}
// check truncated file
assertEquals(length, file.length());
} catch (IOException e) {
fail(e.getMessage());
} finally {
......
......@@ -33,6 +33,7 @@ public class FakedTsFileResource extends TsFileResource {
private String fakeTsfileName;
public FakedTsFileResource(long tsFileSize, String name) {
super(new File(name));
this.timeIndex = new FileTimeIndex();
this.tsFileSize = tsFileSize;
fakeTsfileName = name;
......
......@@ -223,12 +223,20 @@ public class TsFileSequenceReader implements AutoCloseable {
return fileMetadataSize;
}
/** Return the tsfile meta data size of this tsfile. */
public long getFileMetadataSize() throws IOException {
return tsFileInput.size() - getFileMetadataPos();
}
/**
* Return the whole meta data size of this tsfile, including ChunkMetadata, TimeseriesMetadata and
* etc.
*/
public long getFileMetadataSize() throws IOException {
return tsFileInput.size() - getFileMetadataPos();
public long getAllMetadataSize() throws IOException {
if (tsFileMetaData == null) {
readFileMetadata();
}
return tsFileInput.size() - tsFileMetaData.getMetaOffset();
}
/** this function does not modify the position of the file reader. */
......
......@@ -30,10 +30,10 @@ import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
import org.apache.iotdb.tsfile.read.common.BatchData;
import org.apache.iotdb.tsfile.read.common.Chunk;
import org.apache.iotdb.tsfile.read.common.TimeRange;
import org.apache.iotdb.tsfile.read.common.block.TsBlock;
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
import org.apache.iotdb.tsfile.read.reader.IChunkReader;
import org.apache.iotdb.tsfile.read.reader.IPageReader;
import org.apache.iotdb.tsfile.read.reader.IPointReader;
import org.apache.iotdb.tsfile.read.reader.page.AlignedPageReader;
import java.io.IOException;
......@@ -272,7 +272,7 @@ public class AlignedChunkReader implements IChunkReader {
}
/** Read data from compressed page data. Uncompress the page and decode it to tsblock data. */
public TsBlock readPageData(
public IPointReader getPagePointReader(
PageHeader timePageHeader,
List<PageHeader> valuePageHeaders,
ByteBuffer compressedTimePageData,
......@@ -317,7 +317,7 @@ public class AlignedChunkReader implements IChunkReader {
null);
alignedPageReader.initTsBlockBuilder(valueTypes);
alignedPageReader.setDeleteIntervalList(valueDeleteIntervalList);
return alignedPageReader.getAllSatisfiedData();
return alignedPageReader.getLazyPointReader();
}
private ByteBuffer uncompressPageData(
......
......@@ -31,6 +31,7 @@ import org.apache.iotdb.tsfile.read.filter.basic.Filter;
import org.apache.iotdb.tsfile.read.filter.operator.AndFilter;
import org.apache.iotdb.tsfile.read.reader.IAlignedPageReader;
import org.apache.iotdb.tsfile.read.reader.IPageReader;
import org.apache.iotdb.tsfile.read.reader.IPointReader;
import org.apache.iotdb.tsfile.read.reader.series.PaginationController;
import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
......@@ -140,6 +141,10 @@ public class AlignedPageReader implements IPageReader, IAlignedPageReader {
return true;
}
public IPointReader getLazyPointReader() throws IOException {
return new LazyLoadAlignedPagePointReader(timePageReader, valuePageReaderList);
}
@Override
public TsBlock getAllSatisfiedData() throws IOException {
builder.reset();
......
/*
* 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.iotdb.tsfile.read.reader.page;
import org.apache.iotdb.tsfile.read.TimeValuePair;
import org.apache.iotdb.tsfile.read.reader.IPointReader;
import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
import java.io.IOException;
import java.util.List;
/**
* This class is used to read data of aligned-series row by row. It won't deserialize all data point
* of one page in memory. In contrast, it constructs row one by one as needed
*/
public class LazyLoadAlignedPagePointReader implements IPointReader {
private TimePageReader timeReader;
private List<ValuePageReader> valueReaders;
private boolean hasNextRow = false;
private int timeIndex;
private long currentTime;
private TsPrimitiveType currentRow;
public LazyLoadAlignedPagePointReader(
TimePageReader timeReader, List<ValuePageReader> valueReaders) throws IOException {
this.timeIndex = -1;
this.timeReader = timeReader;
this.valueReaders = valueReaders;
prepareNextRow();
}
private void prepareNextRow() throws IOException {
while (true) {
if (!timeReader.hasNextTime()) {
hasNextRow = false;
return;
}
currentTime = timeReader.nextTime();
timeIndex++;
boolean someValueNotNull = false;
TsPrimitiveType[] valuesInThisRow = new TsPrimitiveType[valueReaders.size()];
for (int i = 0; i < valueReaders.size(); i++) {
TsPrimitiveType value =
valueReaders.get(i) == null
? null
: valueReaders.get(i).nextValue(currentTime, timeIndex);
someValueNotNull = someValueNotNull || (value != null);
valuesInThisRow[i] = value;
}
if (someValueNotNull) {
currentRow = new TsPrimitiveType.TsVector(valuesInThisRow);
hasNextRow = true;
break;
}
}
}
@Override
public boolean hasNextTimeValuePair() throws IOException {
return hasNextRow;
}
@Override
public TimeValuePair nextTimeValuePair() throws IOException {
TimeValuePair ret = currentTimeValuePair();
prepareNextRow();
return ret;
}
@Override
public TimeValuePair currentTimeValuePair() throws IOException {
return new TimeValuePair(currentTime, currentRow);
}
@Override
public void close() throws IOException {}
}
......@@ -448,4 +448,45 @@ public class TimeRangeTest {
Assert.assertTrue(new TimeRange(0, 3).compareTo(new TimeRange(1, 2)) < 0);
Assert.assertTrue(new TimeRange(5, 6).compareTo(new TimeRange(5, 6)) == 0);
}
@Test
/*
* test min is Long.MIN_VALUE
*/
public void intersect8() {
TimeRange r1 = new TimeRange(Long.MIN_VALUE, 3);
r1.setLeftClose(false);
TimeRange r2 = new TimeRange(Long.MIN_VALUE, 5);
r2.setLeftClose(false);
assertTrue(r1.intersects(r2));
assertTrue(r2.intersects(r1));
}
@Test
/*
* test max is Long.MAX_VALUE
*/
public void intersect9() {
TimeRange r1 = new TimeRange(1, Long.MAX_VALUE);
r1.setRightClose(false);
TimeRange r2 = new TimeRange(3, Long.MAX_VALUE);
r2.setRightClose(false);
assertTrue(r1.intersects(r2));
assertTrue(r2.intersects(r1));
}
@Test
/*
* test min is Long.MIN_VALUE and max is Long.MAX_VALUE
*/
public void intersect10() {
TimeRange r1 = new TimeRange(Long.MIN_VALUE, Long.MAX_VALUE);
r1.setLeftClose(false);
r1.setRightClose(false);
TimeRange r2 = new TimeRange(Long.MIN_VALUE, Long.MAX_VALUE);
r2.setLeftClose(false);
r2.setRightClose(false);
assertTrue(r1.intersects(r2));
assertTrue(r2.intersects(r1));
}
}
/*
* 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.iotdb.tsfile.read.reader;
import org.apache.iotdb.tsfile.read.TimeValuePair;
import org.apache.iotdb.tsfile.read.reader.page.LazyLoadAlignedPagePointReader;
import org.apache.iotdb.tsfile.read.reader.page.TimePageReader;
import org.apache.iotdb.tsfile.read.reader.page.ValuePageReader;
import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
import org.powermock.api.mockito.PowerMockito;
import java.io.IOException;
import java.util.LinkedList;
import java.util.List;
public class LazyLoadAlignedPagePointReaderTest {
@Test
public void testTimeNoData() throws IOException {
int columnCount = 2;
TimePageReader timePageReader = PowerMockito.mock(TimePageReader.class);
List<ValuePageReader> valuePageReaders = new LinkedList<>();
for (int i = 0; i < columnCount; i++) {
valuePageReaders.add(PowerMockito.mock(ValuePageReader.class));
}
PowerMockito.when(timePageReader.hasNextTime()).thenReturn(false);
PowerMockito.when(valuePageReaders.get(0).nextValue(Mockito.anyLong(), Mockito.anyInt()))
.thenReturn(null);
PowerMockito.when(valuePageReaders.get(1).nextValue(Mockito.anyLong(), Mockito.anyInt()))
.thenReturn(null);
LazyLoadAlignedPagePointReader reader =
new LazyLoadAlignedPagePointReader(timePageReader, valuePageReaders);
boolean hasNextValue = reader.hasNextTimeValuePair();
Assert.assertFalse(hasNextValue);
}
@Test
public void testValueNoData() throws IOException {
int columnCount = 2;
TimePageReader timePageReader = PowerMockito.mock(TimePageReader.class);
List<ValuePageReader> valuePageReaders = new LinkedList<>();
for (int i = 0; i < columnCount; i++) {
valuePageReaders.add(PowerMockito.mock(ValuePageReader.class));
}
PowerMockito.when(timePageReader.hasNextTime()).thenReturn(true).thenReturn(false);
PowerMockito.when(valuePageReaders.get(0).nextValue(Mockito.anyLong(), Mockito.anyInt()))
.thenReturn(null);
PowerMockito.when(valuePageReaders.get(1).nextValue(Mockito.anyLong(), Mockito.anyInt()))
.thenReturn(null);
LazyLoadAlignedPagePointReader reader =
new LazyLoadAlignedPagePointReader(timePageReader, valuePageReaders);
boolean hasNextValue = reader.hasNextTimeValuePair();
Assert.assertFalse(hasNextValue);
}
@Test
public void testOneRow() throws IOException {
int columnCount = 2;
TimePageReader timePageReader = PowerMockito.mock(TimePageReader.class);
List<ValuePageReader> valuePageReaders = new LinkedList<>();
for (int i = 0; i < columnCount; i++) {
valuePageReaders.add(PowerMockito.mock(ValuePageReader.class));
}
PowerMockito.when(timePageReader.hasNextTime()).thenReturn(true).thenReturn(false);
PowerMockito.when(timePageReader.nextTime()).thenReturn(1L);
PowerMockito.when(valuePageReaders.get(0).nextValue(Mockito.anyLong(), Mockito.anyInt()))
.thenReturn(new TsPrimitiveType.TsInt(1));
PowerMockito.when(valuePageReaders.get(1).nextValue(Mockito.anyLong(), Mockito.anyInt()))
.thenReturn(new TsPrimitiveType.TsInt(2));
LazyLoadAlignedPagePointReader reader =
new LazyLoadAlignedPagePointReader(timePageReader, valuePageReaders);
boolean hasNextValue = reader.hasNextTimeValuePair();
Assert.assertTrue(hasNextValue);
TimeValuePair row = reader.nextTimeValuePair();
Assert.assertEquals(1L, row.getTimestamp());
Assert.assertEquals(
new TsPrimitiveType.TsVector(
new TsPrimitiveType.TsInt[] {
new TsPrimitiveType.TsInt(1), new TsPrimitiveType.TsInt(2)
}),
row.getValue());
Assert.assertFalse(reader.hasNextTimeValuePair());
}
@Test
public void testSomeColumnNull() throws IOException {
int columnCount = 2;
TimePageReader timePageReader = PowerMockito.mock(TimePageReader.class);
List<ValuePageReader> valuePageReaders = new LinkedList<>();
for (int i = 0; i < columnCount; i++) {
valuePageReaders.add(PowerMockito.mock(ValuePageReader.class));
}
PowerMockito.when(timePageReader.hasNextTime())
.thenReturn(true)
.thenReturn(true)
.thenReturn(false);
PowerMockito.when(timePageReader.nextTime()).thenReturn(1L).thenReturn(2L);
PowerMockito.when(valuePageReaders.get(0).nextValue(Mockito.anyLong(), Mockito.anyInt()))
.thenReturn(new TsPrimitiveType.TsInt(1))
.thenReturn(null);
PowerMockito.when(valuePageReaders.get(1).nextValue(Mockito.anyLong(), Mockito.anyInt()))
.thenReturn(null)
.thenReturn(null);
LazyLoadAlignedPagePointReader reader =
new LazyLoadAlignedPagePointReader(timePageReader, valuePageReaders);
boolean hasNextValue = reader.hasNextTimeValuePair();
Assert.assertTrue(hasNextValue);
TimeValuePair row = reader.nextTimeValuePair();
Assert.assertEquals(1L, row.getTimestamp());
Assert.assertEquals("[1, null]", row.getValue().toString());
Assert.assertFalse(reader.hasNextTimeValuePair());
}
@Test
public void testMultiRow() throws IOException {
int columnCount = 2;
TimePageReader timePageReader = PowerMockito.mock(TimePageReader.class);
List<ValuePageReader> valuePageReaders = new LinkedList<>();
for (int i = 0; i < columnCount; i++) {
valuePageReaders.add(PowerMockito.mock(ValuePageReader.class));
}
PowerMockito.when(timePageReader.hasNextTime())
.thenReturn(true)
.thenReturn(true)
.thenReturn(false);
PowerMockito.when(timePageReader.nextTime()).thenReturn(1L).thenReturn(2L);
PowerMockito.when(valuePageReaders.get(0).nextValue(Mockito.anyLong(), Mockito.anyInt()))
.thenReturn(new TsPrimitiveType.TsInt(1))
.thenReturn(new TsPrimitiveType.TsInt(1));
PowerMockito.when(valuePageReaders.get(1).nextValue(Mockito.anyLong(), Mockito.anyInt()))
.thenReturn(null)
.thenReturn(new TsPrimitiveType.TsInt(2));
LazyLoadAlignedPagePointReader reader =
new LazyLoadAlignedPagePointReader(timePageReader, valuePageReaders);
boolean hasNextValue = reader.hasNextTimeValuePair();
Assert.assertTrue(hasNextValue);
TimeValuePair row1 = reader.nextTimeValuePair();
Assert.assertEquals(1L, row1.getTimestamp());
Assert.assertEquals("[1, null]", row1.getValue().toString());
Assert.assertTrue(reader.hasNextTimeValuePair());
TimeValuePair row2 = reader.nextTimeValuePair();
Assert.assertEquals(2L, row2.getTimestamp());
Assert.assertEquals("[1, 2]", row2.getValue().toString());
Assert.assertFalse(reader.hasNextTimeValuePair());
}
}