未验证 提交 db3d1f75 编写于 作者: X Xiangdong Huang 提交者: GitHub

[IOTDB-31] control the float precision of data in memtable (#83)

* fix #31. add props of a time series declarison into FileNodeProcessor; pick Compressor out of props

* fix bug and optimize checkMetadata parameters logic

* add float precision it

* add NotImplementedException

* control the float precision of data in memtable when query

* modify a UNCOMPRESSOR to UNCOMPRESSED

* fix max_point_number in show timeseries as lowercase
上级 b9563938
......@@ -191,8 +191,12 @@ A column of data contains all values belonging to a time series and the timestam
## Data Type
IoTDB supports six data types in total: BOOLEAN (Boolean), INT32 (Integer), INT64 (Long Integer), FLOAT (Single Precision Floating Point), DOUBLE (Double Precision Floating Point), TEXT (String).
The time series of FLOAT and DOUBLE type can specify (MAX\_POINT\_NUMBER, see [this page](#iotdb-query-statement) for more information on how to specify), which is the number of digits after the decimal point of the floating point number, if the encoding method is [RLE](#encoding) or [TS\_2DIFF](#encoding) (Refer to [Create Timeseries Statement](#chapter-5-iotdb-sql-documentation) for more information on how to specify). If MAX\_POINT\_NUMBER is not specified, the system will use [float\_precision](#encoding) in the configuration file "tsfile-format.properties" for configuration for the configuration method.
* For Float data value, The data range is (-Integer.MAX_VALUE, Integer.MAX_VALUE), rather than Float.MAX_VALUE, and the max_point_number is 19, it is because of the limition of function Math.round(float) in Java.
* For Double data value, The data range is (-Long.MAX_VALUE, Long.MAX_VALUE), rather than Double.MAX_VALUE, and the max_point_number is 19, it is because of the limition of function Math.round(double) in Java (Long.MAX_VALUE=9.22E18).
When the data type of data input by the user in the system does not correspond to the data type of the time series, the system will report type errors. As shown below, the second-order difference encoding does not support the Boolean type:
```
......
......@@ -20,6 +20,7 @@ package org.apache.iotdb.db.engine.bufferwrite;
import java.io.File;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
......@@ -219,16 +220,18 @@ public class BufferWriteProcessor extends Processor {
* @return corresponding chunk data and chunk metadata in memory
*/
public Pair<ReadOnlyMemChunk, List<ChunkMetaData>> queryBufferWriteData(String deviceId,
String measurementId,
TSDataType dataType) {
String measurementId, TSDataType dataType, Map<String, String> props) {
flushQueryLock.lock();
try {
MemSeriesLazyMerger memSeriesLazyMerger = new MemSeriesLazyMerger();
if (flushMemTable != null) {
memSeriesLazyMerger.addMemSeries(flushMemTable.query(deviceId, measurementId, dataType));
memSeriesLazyMerger.addMemSeries(flushMemTable.query(deviceId, measurementId, dataType, props));
}
memSeriesLazyMerger.addMemSeries(workMemTable.query(deviceId, measurementId, dataType));
ReadOnlyMemChunk timeValuePairSorter = new ReadOnlyMemChunk(dataType, memSeriesLazyMerger);
memSeriesLazyMerger.addMemSeries(workMemTable.query(deviceId, measurementId, dataType, props));
// memSeriesLazyMerger has handled the props,
// so we do not need to handle it again in the following readOnlyMemChunk
ReadOnlyMemChunk timeValuePairSorter = new ReadOnlyMemChunk(dataType, memSeriesLazyMerger,
Collections.emptyMap());
return new Pair<>(timeValuePairSorter,
writer.getMetadatas(deviceId, measurementId, dataType));
} finally {
......
......@@ -751,19 +751,19 @@ public class FileNodeProcessor extends Processor implements IStatistic {
* query data.
*/
public <T extends Comparable<T>> QueryDataSource query(String deviceId, String measurementId,
QueryContext context)
throws FileNodeProcessorException {
QueryContext context) throws FileNodeProcessorException {
// query overflow data
MeasurementSchema mSchema;
TSDataType dataType;
try {
dataType = mManager.getSeriesType(deviceId + "." + measurementId);
} catch (PathErrorException e) {
throw new FileNodeProcessorException(e);
}
//mSchema = mManager.getSchemaForOnePath(deviceId + "." + measurementId);
mSchema = fileSchema.getMeasurementSchema(measurementId);
dataType = mSchema.getType();
OverflowSeriesDataSource overflowSeriesDataSource;
try {
overflowSeriesDataSource = overflowProcessor.query(deviceId, measurementId, dataType,
context);
mSchema.getProps(), context);
} catch (IOException e) {
throw new FileNodeProcessorException(e);
}
......@@ -794,7 +794,7 @@ public class FileNodeProcessor extends Processor implements IStatistic {
newFileNodes.get(newFileNodes.size() - 1).getRelativePath(), getProcessorName()));
}
bufferwritedata = bufferWriteProcessor
.queryBufferWriteData(deviceId, measurementId, dataType);
.queryBufferWriteData(deviceId, measurementId, dataType, mSchema.getProps());
try {
List<Modification> pathModifications = context.getPathModifications(
......
......@@ -21,6 +21,7 @@ package org.apache.iotdb.db.engine.memtable;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
import org.apache.iotdb.db.utils.TimeValuePair;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
......@@ -93,7 +94,12 @@ public abstract class AbstractMemTable implements IMemTable {
}
@Override
public TimeValuePairSorter query(String deviceId, String measurement, TSDataType dataType) {
public ReadOnlyMemChunk query(String deviceId, String measurement, TSDataType dataType,
Map<String, String> props) {
return new ReadOnlyMemChunk(dataType, getSeriesData(deviceId, measurement, dataType), props);
}
private TimeValuePairSorter getSeriesData(String deviceId, String measurement, TSDataType dataType) {
if (!checkPath(deviceId, measurement)) {
return new WritableMemChunk(dataType);
}
......@@ -156,5 +162,4 @@ public abstract class AbstractMemTable implements IMemTable {
}
return null;
}
}
......@@ -19,6 +19,7 @@
package org.apache.iotdb.db.engine.memtable;
import java.util.Map;
import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
/**
......@@ -37,7 +38,8 @@ public interface IMemTable {
int size();
TimeValuePairSorter query(String deviceId, String measurement, TSDataType dataType);
ReadOnlyMemChunk query(String deviceId, String measurement, TSDataType dataType,
Map<String, String> props);
/**
* release all the memory resources.
......
......@@ -21,12 +21,12 @@ package org.apache.iotdb.db.engine.memtable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
import org.apache.iotdb.db.utils.TimeValuePair;
public class MemSeriesLazyMerger implements TimeValuePairSorter {
private List<TimeValuePairSorter> memSeriesList;
private List<ReadOnlyMemChunk> memSeriesList;
public MemSeriesLazyMerger() {
memSeriesList = new ArrayList<>();
......@@ -37,7 +37,7 @@ public class MemSeriesLazyMerger implements TimeValuePairSorter {
*
* @param memSerieses Please ensure that the memSerieses are in ascending order by timestamp.
*/
public MemSeriesLazyMerger(TimeValuePairSorter... memSerieses) {
public MemSeriesLazyMerger(ReadOnlyMemChunk... memSerieses) {
this();
Collections.addAll(memSeriesList, memSerieses);
}
......@@ -46,20 +46,16 @@ public class MemSeriesLazyMerger implements TimeValuePairSorter {
* IMPORTANT: Please ensure that the minimum timestamp of added {@link IWritableMemChunk} is
* larger than any timestamps of the IWritableMemChunk already added in.
*/
public void addMemSeries(TimeValuePairSorter series) {
public void addMemSeries(ReadOnlyMemChunk series) {
memSeriesList.add(series);
}
@Override
public List<TimeValuePair> getSortedTimeValuePairList() {
if (memSeriesList.isEmpty()) {
return new ArrayList<>();
} else {
List<TimeValuePair> ret = memSeriesList.get(0).getSortedTimeValuePairList();
for (int i = 1; i < memSeriesList.size(); i++) {
ret.addAll(memSeriesList.get(i).getSortedTimeValuePairList());
}
return ret;
List<TimeValuePair> res = new ArrayList<>();
for (int i = 0; i < memSeriesList.size(); i++) {
res.addAll(memSeriesList.get(i).getSortedTimeValuePairList());
}
return res;
}
}
......@@ -24,6 +24,7 @@ import java.time.Instant;
import java.time.ZonedDateTime;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
......@@ -40,18 +41,17 @@ import org.apache.iotdb.db.engine.bufferwrite.FileNodeConstants;
import org.apache.iotdb.db.engine.filenode.FileNodeManager;
import org.apache.iotdb.db.engine.memcontrol.BasicMemController;
import org.apache.iotdb.db.engine.memtable.MemSeriesLazyMerger;
import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.engine.pool.FlushManager;
import org.apache.iotdb.db.engine.querycontext.MergeSeriesDataSource;
import org.apache.iotdb.db.engine.querycontext.OverflowInsertFile;
import org.apache.iotdb.db.engine.querycontext.OverflowSeriesDataSource;
import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
import org.apache.iotdb.db.engine.version.VersionController;
import org.apache.iotdb.db.exception.OverflowProcessorException;
import org.apache.iotdb.db.qp.constant.DatetimeUtils;
import org.apache.iotdb.db.utils.ImmediateFuture;
import org.apache.iotdb.db.engine.version.VersionController;
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.utils.ImmediateFuture;
import org.apache.iotdb.db.utils.MemUtils;
import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager;
import org.apache.iotdb.db.writelog.node.WriteLogNode;
......@@ -255,14 +255,14 @@ public class OverflowProcessor extends Processor {
* @return OverflowSeriesDataSource
*/
public OverflowSeriesDataSource query(String deviceId, String measurementId,
TSDataType dataType, QueryContext context)
TSDataType dataType, Map<String, String> props, QueryContext context)
throws IOException {
queryFlushLock.lock();
try {
// query insert data in memory and unseqTsFiles
// memory
TimeValuePairSorter insertInMem = queryOverflowInsertInMemory(deviceId, measurementId,
dataType);
ReadOnlyMemChunk insertInMem = queryOverflowInsertInMemory(deviceId, measurementId,
dataType, props);
List<OverflowInsertFile> overflowInsertFileList = new ArrayList<>();
// work file
Pair<String, List<ChunkMetaData>> insertInDiskWork = queryWorkDataInOverflowInsert(deviceId,
......@@ -295,8 +295,8 @@ public class OverflowProcessor extends Processor {
*
* @return insert data in SeriesChunkInMemTable
*/
private TimeValuePairSorter queryOverflowInsertInMemory(String deviceId, String measurementId,
TSDataType dataType) {
private ReadOnlyMemChunk queryOverflowInsertInMemory(String deviceId, String measurementId,
TSDataType dataType, Map<String, String> props) {
MemSeriesLazyMerger memSeriesLazyMerger = new MemSeriesLazyMerger();
queryFlushLock.lock();
......@@ -304,12 +304,14 @@ public class OverflowProcessor extends Processor {
if (flushSupport != null && isFlush()) {
memSeriesLazyMerger
.addMemSeries(
flushSupport.queryOverflowInsertInMemory(deviceId, measurementId, dataType));
flushSupport.queryOverflowInsertInMemory(deviceId, measurementId, dataType, props));
}
memSeriesLazyMerger
.addMemSeries(workSupport.queryOverflowInsertInMemory(deviceId, measurementId,
dataType));
return new ReadOnlyMemChunk(dataType, memSeriesLazyMerger);
dataType, props));
// memSeriesLazyMerger has handled the props,
// so we do not need to handle it again in the following readOnlyMemChunk
return new ReadOnlyMemChunk(dataType, memSeriesLazyMerger, Collections.emptyMap());
} finally {
queryFlushLock.unlock();
}
......
......@@ -20,10 +20,9 @@ package org.apache.iotdb.db.engine.overflow.io;
import java.util.HashMap;
import java.util.Map;
import org.apache.iotdb.db.engine.memtable.IMemTable;
import org.apache.iotdb.db.engine.memtable.PrimitiveMemTable;
import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.BatchData;
import org.apache.iotdb.tsfile.write.record.TSRecord;
......@@ -83,13 +82,13 @@ public class OverflowSupport {
}
}
public TimeValuePairSorter queryOverflowInsertInMemory(String deviceId, String measurementId,
TSDataType dataType) {
return memTable.query(deviceId, measurementId, dataType);
public ReadOnlyMemChunk queryOverflowInsertInMemory(String deviceId, String measurementId,
TSDataType dataType, Map<String, String> props) {
return memTable.query(deviceId, measurementId, dataType, props);
}
public BatchData queryOverflowUpdateInMemory(String deviceId, String measurementId,
TSDataType dataType) {
TSDataType dataType) {
if (indexTrees.containsKey(deviceId) && indexTrees.get(deviceId).containsKey(measurementId)
&& indexTrees.get(deviceId).get(measurementId).getDataType().equals(dataType)) {
return indexTrees.get(deviceId).get(measurementId).query();
......
......@@ -34,11 +34,11 @@ public class GlobalSortedSeriesDataSource {
private UnsealedTsFile unsealedTsFile;
// seq mem-table
private TimeValuePairSorter readableChunk;
private ReadOnlyMemChunk readableChunk;
public GlobalSortedSeriesDataSource(Path seriesPath, List<IntervalFileNode> sealedTsFiles,
UnsealedTsFile unsealedTsFile,
TimeValuePairSorter readableChunk) {
ReadOnlyMemChunk readableChunk) {
this.seriesPath = seriesPath;
this.sealedTsFiles = sealedTsFiles;
this.unsealedTsFile = unsealedTsFile;
......@@ -74,11 +74,11 @@ public class GlobalSortedSeriesDataSource {
return readableChunk != null;
}
public TimeValuePairSorter getReadableChunk() {
public ReadOnlyMemChunk getReadableChunk() {
return readableChunk;
}
public void setReadableChunk(TimeValuePairSorter readableChunk) {
public void setReadableChunk(ReadOnlyMemChunk readableChunk) {
this.readableChunk = readableChunk;
}
......
......@@ -30,14 +30,14 @@ public class OverflowSeriesDataSource {
// overflow tsfile
private List<OverflowInsertFile> overflowInsertFileList;
// unSeq mem-table
private TimeValuePairSorter readableMemChunk;
private ReadOnlyMemChunk readableMemChunk;
public OverflowSeriesDataSource(Path seriesPath) {
this.seriesPath = seriesPath;
}
public OverflowSeriesDataSource(Path seriesPath, TSDataType dataType,
List<OverflowInsertFile> overflowInsertFileList, TimeValuePairSorter readableMemChunk) {
List<OverflowInsertFile> overflowInsertFileList, ReadOnlyMemChunk readableMemChunk) {
this.seriesPath = seriesPath;
this.dataType = dataType;
this.overflowInsertFileList = overflowInsertFileList;
......@@ -52,11 +52,11 @@ public class OverflowSeriesDataSource {
this.overflowInsertFileList = overflowInsertFileList;
}
public TimeValuePairSorter getReadableMemChunk() {
public ReadOnlyMemChunk getReadableMemChunk() {
return readableMemChunk;
}
public void setReadableMemChunk(TimeValuePairSorter rawChunk) {
public void setReadableMemChunk(ReadOnlyMemChunk rawChunk) {
this.readableMemChunk = rawChunk;
}
......
......@@ -21,9 +21,16 @@ package org.apache.iotdb.db.engine.querycontext;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.iotdb.db.engine.memtable.MemSeriesLazyMerger;
import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
import org.apache.iotdb.db.utils.MathUtils;
import org.apache.iotdb.db.utils.TimeValuePair;
import org.apache.iotdb.db.utils.TsPrimitiveType;
import org.apache.iotdb.db.utils.TsPrimitiveType.TsDouble;
import org.apache.iotdb.db.utils.TsPrimitiveType.TsFloat;
import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
//TODO: merge ReadOnlyMemChunk and WritableMemChunk and IWritableMemChunk
......@@ -35,13 +42,27 @@ public class ReadOnlyMemChunk implements TimeValuePairSorter {
private TimeValuePairSorter memSeries;
private List<TimeValuePair> sortedTimeValuePairList;
Map<String, String> props;
private int floatPrecision = TSFileConfig.floatPrecision;
/**
* init by TSDataType and TimeValuePairSorter.
*/
public ReadOnlyMemChunk(TSDataType dataType, TimeValuePairSorter memSeries) {
// public ReadOnlyMemChunk(TSDataType dataType, TimeValuePairSorter memSeries) {
// this(dataType, memSeries, Collections.emptyMap());
// }
/**
* init by TSDataType and TimeValuePairSorter.
*/
public ReadOnlyMemChunk(TSDataType dataType, TimeValuePairSorter memSeries, Map<String, String> props) {
this.dataType = dataType;
this.memSeries = memSeries;
this.initialized = false;
this.props = props;
if (props.containsKey(Encoder.MAX_POINT_NUMBER)) {
this.floatPrecision = Integer.valueOf(props.get(Encoder.MAX_POINT_NUMBER));
}
}
private void checkInitialized() {
......@@ -52,6 +73,22 @@ public class ReadOnlyMemChunk implements TimeValuePairSorter {
private void init() {
sortedTimeValuePairList = memSeries.getSortedTimeValuePairList();
if (!(memSeries instanceof MemSeriesLazyMerger)) {
switch (dataType) {
case FLOAT:
sortedTimeValuePairList.replaceAll(x -> new TimeValuePair(x.getTimestamp(),
new TsFloat(
MathUtils.roundWithGivenPrecision(x.getValue().getFloat(), floatPrecision))));
break;
case DOUBLE:
sortedTimeValuePairList.replaceAll(x -> new TimeValuePair(x.getTimestamp(),
new TsDouble(
MathUtils.roundWithGivenPrecision(x.getValue().getDouble(), floatPrecision))));
break;
default:
break;
}
}
initialized = true;
}
......
......@@ -20,6 +20,7 @@ package org.apache.iotdb.db.query.reader.mem;
import java.util.Iterator;
import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
import org.apache.iotdb.db.query.reader.IReader;
import org.apache.iotdb.db.utils.TimeValuePair;
import org.apache.iotdb.tsfile.read.common.BatchData;
......@@ -32,7 +33,7 @@ public class MemChunkReaderWithFilter implements IReader {
private boolean hasCachedTimeValuePair;
private TimeValuePair cachedTimeValuePair;
public MemChunkReaderWithFilter(TimeValuePairSorter readableChunk, Filter filter) {
public MemChunkReaderWithFilter(ReadOnlyMemChunk readableChunk, Filter filter) {
timeValuePairIterator = readableChunk.getIterator();
this.filter = filter;
}
......
/**
* 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.utils;
import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
public class MathUtils {
private MathUtils() {
throw new IllegalStateException("Utility class");
}
/**
* @param data data should be less than Long.MAX_VALUE. otherwise Math.round() will return wrong
* value.
*/
public static float roundWithGivenPrecision(float data, int size) {
if (size == 0) {
return Math.round(data);
}
return Math.round(data)
+ Math.round(((data - Math.round(data)) * Math.pow(10, size)))
/ (float) Math.pow(10, size);
}
public static float roundWithGivenPrecision(float data) {
if (TSFileConfig.floatPrecision == 0) {
return Math.round(data);
}
return Math.round(data)
+ Math.round(((data - Math.round(data)) * (float) Math.pow(10, TSFileConfig.floatPrecision)))
/ (float) Math.pow(10, TSFileConfig.floatPrecision);
}
/**
* @param data data should be less than Long.MAX_VALUE. otherwise Math.round() will return wrong
* value.
*/
public static double roundWithGivenPrecision(double data, int size) {
if (size == 0) {
return Math.round(data);
}
return Math.round(data)
+ Math.round(((data - Math.round(data)) * Math.pow(10, size)))
/ Math.pow(10, size);
}
/**
* @param data data should be less than Long.MAX_VALUE. otherwise Math.round() will return wrong
* value.
*/
public static double roundWithGivenPrecision(double data) {
if (TSFileConfig.floatPrecision == 0) {
return Math.round(data);
}
return Math.round(data)
+ Math.round(((data - Math.round(data)) * Math.pow(10, TSFileConfig.floatPrecision)))
/ Math.pow(10, TSFileConfig.floatPrecision);
}
}
......@@ -25,13 +25,12 @@ import static org.junit.Assert.assertFalse;
import java.io.File;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.iotdb.db.conf.directories.Directories;
import org.apache.iotdb.db.engine.MetadataManagerHelper;
import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
......@@ -52,6 +51,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class BufferWriteProcessorNewTest {
private static final Logger LOGGER = LoggerFactory.getLogger(BufferWriteProcessorNewTest.class);
Action bfflushaction = new Action() {
......@@ -79,6 +79,7 @@ public class BufferWriteProcessorNewTest {
private String processorName = "root.vehicle.d0";
private String measurementId = "s0";
private TSDataType dataType = TSDataType.INT32;
private Map<String, String> props = Collections.emptyMap();
private BufferWriteProcessor bufferwrite;
private String filename = "tsfile";
......@@ -111,7 +112,7 @@ public class BufferWriteProcessorNewTest {
assertFalse(bufferwrite.isNewProcessor());
Pair<ReadOnlyMemChunk, List<ChunkMetaData>> pair = bufferwrite
.queryBufferWriteData(processorName,
measurementId, dataType);
measurementId, dataType, props);
ReadOnlyMemChunk left = pair.left;
List<ChunkMetaData> right = pair.right;
assertTrue(left.isEmpty());
......@@ -120,7 +121,7 @@ public class BufferWriteProcessorNewTest {
bufferwrite.write(processorName, measurementId, i, dataType, String.valueOf(i));
}
// query data in memory
pair = bufferwrite.queryBufferWriteData(processorName, measurementId, dataType);
pair = bufferwrite.queryBufferWriteData(processorName, measurementId, dataType, props);
left = pair.left;
right = pair.right;
assertFalse(left.isEmpty());
......@@ -143,11 +144,11 @@ public class BufferWriteProcessorNewTest {
bufferwrite.getFlushFuture().get(10, TimeUnit.SECONDS);
} catch (Exception e) {
//because UT uses a mock flush operation, 10 seconds should be enough.
LOGGER.error(e.getMessage(),e);
LOGGER.error(e.getMessage(), e);
Assert.fail("mock flush spends more than 10 seconds... "
+ "Please modify the value or change a better test environment");
}
pair = bufferwrite.queryBufferWriteData(processorName, measurementId, dataType);
pair = bufferwrite.queryBufferWriteData(processorName, measurementId, dataType, props);
left = pair.left;
right = pair.right;
assertTrue(left.isEmpty());
......@@ -160,7 +161,7 @@ public class BufferWriteProcessorNewTest {
Directories.getInstance().getFolderForTest(), processorName, filename, parameters,
SysTimeVersionController.INSTANCE,
FileSchemaUtils.constructFileSchema(processorName));
pair = bufferWriteProcessor.queryBufferWriteData(processorName, measurementId, dataType);
pair = bufferWriteProcessor.queryBufferWriteData(processorName, measurementId, dataType, props);
left = pair.left;
right = pair.right;
assertTrue(left.isEmpty());
......
......@@ -24,17 +24,15 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue;
import ch.qos.logback.core.util.TimeUtil;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.iotdb.db.conf.directories.Directories;
import org.apache.iotdb.db.engine.MetadataManagerHelper;
import org.apache.iotdb.db.engine.PathUtils;
......@@ -153,8 +151,7 @@ public class BufferWriteProcessorTest {
assertTrue(insertFile.exists());
assertEquals(insertFileLength, insertFile.length());
Pair<ReadOnlyMemChunk, List<ChunkMetaData>> pair = bufferWriteProcessor
.queryBufferWriteData(deviceId,
measurementId, dataType);
.queryBufferWriteData(deviceId, measurementId, dataType, Collections.emptyMap());
assertTrue(pair.left.isEmpty());
assertEquals(1, pair.right.size());
ChunkMetaData chunkMetaData = pair.right.get(0);
......@@ -185,8 +182,7 @@ public class BufferWriteProcessorTest {
insertPath, parameters, SysTimeVersionController.INSTANCE,
FileSchemaUtils.constructFileSchema(deviceId));
Pair<ReadOnlyMemChunk, List<ChunkMetaData>> pair = bufferWriteProcessor
.queryBufferWriteData(deviceId,
measurementId, dataType);
.queryBufferWriteData(deviceId, measurementId, dataType, Collections.emptyMap());
assertTrue(pair.left.isEmpty());
assertEquals(1, pair.right.size());
ChunkMetaData chunkMetaData = pair.right.get(0);
......@@ -231,7 +227,7 @@ public class BufferWriteProcessorTest {
// query result
Pair<ReadOnlyMemChunk, List<ChunkMetaData>> pair = bufferwrite
.queryBufferWriteData(deviceId, measurementId,
dataType);
dataType, Collections.emptyMap());
assertTrue(pair.left.isEmpty());
assertEquals(1, pair.right.size());
ChunkMetaData chunkMetaData = pair.right.get(0);
......@@ -241,7 +237,8 @@ public class BufferWriteProcessorTest {
bufferwrite.write(deviceId, measurementId, i, dataType, String.valueOf(i));
assertEquals((i - 86) * 12, bufferwrite.memoryUsage());
}
pair = bufferwrite.queryBufferWriteData(deviceId, measurementId, dataType);
pair = bufferwrite
.queryBufferWriteData(deviceId, measurementId, dataType, Collections.emptyMap());
ReadOnlyMemChunk rawSeriesChunk = (ReadOnlyMemChunk) pair.left;
assertFalse(rawSeriesChunk.isEmpty());
assertEquals(87, rawSeriesChunk.getMinTimestamp());
......
......@@ -19,8 +19,10 @@
package org.apache.iotdb.db.engine.memtable;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.Random;
import org.apache.iotdb.db.utils.MathUtils;
import org.apache.iotdb.db.utils.TimeValuePair;
import org.apache.iotdb.db.utils.TsPrimitiveType;
import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
......@@ -66,7 +68,8 @@ public class PrimitiveMemTableTest {
for (int i = 0; i < dataSize; i++) {
memTable.write(deviceId, measurementId[0], TSDataType.INT32, i, String.valueOf(i));
}
Iterator<TimeValuePair> tvPair = memTable.query(deviceId, measurementId[0], TSDataType.INT32)
Iterator<TimeValuePair> tvPair = memTable
.query(deviceId, measurementId[0], TSDataType.INT32, Collections.emptyMap())
.getSortedTimeValuePairList().iterator();
for (int i = 0; i < dataSize; i++) {
TimeValuePair timeValuePair = tvPair.next();
......@@ -84,7 +87,8 @@ public class PrimitiveMemTableTest {
memTable.write(deviceId, sensorId, dataType, ret[i].getTimestamp(),
ret[i].getValue().getStringValue());
}
Iterator<TimeValuePair> tvPair = memTable.query(deviceId, sensorId, dataType)
Iterator<TimeValuePair> tvPair = memTable
.query(deviceId, sensorId, dataType, Collections.emptyMap())
.getSortedTimeValuePairList()
.iterator();
Arrays.sort(ret);
......@@ -100,7 +104,15 @@ public class PrimitiveMemTableTest {
last = pair;
TimeValuePair next = tvPair.next();
Assert.assertEquals(pair.getTimestamp(), next.getTimestamp());
Assert.assertEquals(pair.getValue(), next.getValue());
if (dataType == TSDataType.DOUBLE) {
Assert.assertEquals(MathUtils.roundWithGivenPrecision(pair.getValue().getDouble()),
next.getValue().getDouble(), 0.0001);
} else if (dataType == TSDataType.FLOAT) {
Assert.assertEquals(MathUtils.roundWithGivenPrecision(pair.getValue().getFloat()),
next.getValue().getFloat(), 0.0001);
} else {
Assert.assertEquals(pair.getValue(), next.getValue());
}
}
}
......
......@@ -22,11 +22,11 @@ import static org.junit.Assert.assertEquals;
import java.io.File;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.iotdb.db.engine.PathUtils;
import org.apache.iotdb.db.engine.bufferwrite.Action;
import org.apache.iotdb.db.engine.bufferwrite.ActionException;
......@@ -80,15 +80,16 @@ public class OverflowProcessorTest {
public void testInsertUpdate()
throws IOException, OverflowProcessorException, InterruptedException {
processor = new OverflowProcessor(processorName, parameters, OverflowTestUtils.getFileSchema(),
SysTimeVersionController.INSTANCE);
SysTimeVersionController.INSTANCE);
assertEquals(true, new File(PathUtils.getOverflowWriteDir(processorName),
"0").exists());
"0").exists());
assertEquals(false, processor.isFlush());
assertEquals(false, processor.isMerge());
QueryContext context = new QueryContext();
// write update data
OverflowSeriesDataSource overflowSeriesDataSource = processor.query(OverflowTestUtils.deviceId1,
OverflowTestUtils.measurementId1, OverflowTestUtils.dataType1, context);
OverflowTestUtils.measurementId1, OverflowTestUtils.dataType1, Collections.emptyMap(),
context);
assertEquals(OverflowTestUtils.dataType1, overflowSeriesDataSource.getDataType());
Assert.assertEquals(true, overflowSeriesDataSource.getReadableMemChunk().isEmpty());
assertEquals(1, overflowSeriesDataSource.getOverflowInsertFileList().size());
......@@ -102,7 +103,7 @@ public class OverflowProcessorTest {
assertEquals(false, processor.isFlush());
overflowSeriesDataSource = processor
.query(OverflowTestUtils.deviceId1, OverflowTestUtils.measurementId1,
OverflowTestUtils.dataType1, context);
OverflowTestUtils.dataType1, Collections.emptyMap(), context);
assertEquals(OverflowTestUtils.dataType1, overflowSeriesDataSource.getDataType());
Assert.assertEquals(false, overflowSeriesDataSource.getReadableMemChunk().isEmpty());
assertEquals(1, overflowSeriesDataSource.getOverflowInsertFileList().size());
......@@ -117,7 +118,7 @@ public class OverflowProcessorTest {
processor.close();
overflowSeriesDataSource = processor
.query(OverflowTestUtils.deviceId1, OverflowTestUtils.measurementId1,
OverflowTestUtils.dataType1, context);
OverflowTestUtils.dataType1, Collections.emptyMap(), context);
Assert.assertEquals(true, overflowSeriesDataSource.getReadableMemChunk().isEmpty());
assertEquals(1, overflowSeriesDataSource.getOverflowInsertFileList().size());
assertEquals(1,
......@@ -125,7 +126,7 @@ public class OverflowProcessorTest {
processor.switchWorkToMerge();
overflowSeriesDataSource = processor
.query(OverflowTestUtils.deviceId1, OverflowTestUtils.measurementId1,
OverflowTestUtils.dataType1, context);
OverflowTestUtils.dataType1, Collections.emptyMap(), context);
assertEquals(2, overflowSeriesDataSource.getOverflowInsertFileList().size());
assertEquals(1,
overflowSeriesDataSource.getOverflowInsertFileList().get(0).getChunkMetaDataList().size());
......@@ -139,7 +140,7 @@ public class OverflowProcessorTest {
processor.switchMergeToWork();
overflowSeriesDataSource = processor
.query(OverflowTestUtils.deviceId1, OverflowTestUtils.measurementId1,
OverflowTestUtils.dataType1, context);
OverflowTestUtils.dataType1, Collections.emptyMap(), context);
processor.close();
processor.clear();
}
......@@ -147,13 +148,14 @@ public class OverflowProcessorTest {
@Test
public void testWriteMemoryAndQuery() throws IOException, OverflowProcessorException {
processor = new OverflowProcessor(processorName, parameters, OverflowTestUtils.getFileSchema(),
SysTimeVersionController.INSTANCE);
SysTimeVersionController.INSTANCE);
OverflowTestUtils.produceInsertData(processor);
processor.close();
QueryContext context = new QueryContext();
// test query
OverflowSeriesDataSource overflowSeriesDataSource = processor.query(OverflowTestUtils.deviceId1,
OverflowTestUtils.measurementId1, OverflowTestUtils.dataType2, context);
OverflowTestUtils.measurementId1, OverflowTestUtils.dataType2, Collections.emptyMap(),
context);
Assert.assertTrue(overflowSeriesDataSource.getReadableMemChunk().isEmpty());
assertEquals(0,
overflowSeriesDataSource.getOverflowInsertFileList().get(0).getChunkMetaDataList().size());
......@@ -163,7 +165,7 @@ public class OverflowProcessorTest {
@Test
public void testFlushAndQuery() throws IOException, OverflowProcessorException {
processor = new OverflowProcessor(processorName, parameters, OverflowTestUtils.getFileSchema(),
SysTimeVersionController.INSTANCE);
SysTimeVersionController.INSTANCE);
processor.flush();
// waiting for the end of flush.
try {
......@@ -172,10 +174,10 @@ public class OverflowProcessorTest {
}
QueryContext context = new QueryContext();
processor.query(OverflowTestUtils.deviceId1, OverflowTestUtils.measurementId1,
OverflowTestUtils.dataType1, context);
OverflowTestUtils.dataType1, Collections.emptyMap(), context);
OverflowTestUtils.produceInsertData(processor);
processor.query(OverflowTestUtils.deviceId1, OverflowTestUtils.measurementId1,
OverflowTestUtils.dataType2, context);
OverflowTestUtils.dataType2, Collections.emptyMap(), context);
processor.close();
processor.clear();
}
......@@ -183,7 +185,7 @@ public class OverflowProcessorTest {
@Test
public void testRecovery() throws OverflowProcessorException, IOException {
processor = new OverflowProcessor(processorName, parameters, OverflowTestUtils.getFileSchema(),
SysTimeVersionController.INSTANCE);
SysTimeVersionController.INSTANCE);
processor.close();
processor.switchWorkToMerge();
assertEquals(true, processor.isMerge());
......@@ -196,7 +198,8 @@ public class OverflowProcessorTest {
QueryContext context = new QueryContext();
OverflowSeriesDataSource overflowSeriesDataSource = overflowProcessor
.query(OverflowTestUtils.deviceId1,
OverflowTestUtils.measurementId1, OverflowTestUtils.dataType1, context);
OverflowTestUtils.measurementId1, OverflowTestUtils.dataType1, Collections.emptyMap(),
context);
Assert.assertEquals(true, overflowSeriesDataSource.getReadableMemChunk().isEmpty());
assertEquals(2, overflowSeriesDataSource.getOverflowInsertFileList().size());
overflowProcessor.switchMergeToWork();
......
......@@ -20,6 +20,7 @@ package org.apache.iotdb.db.engine.overflow.io;
import static org.junit.Assert.assertEquals;
import java.util.Collections;
import org.apache.iotdb.db.utils.TimeValuePair;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.utils.BytesUtils;
......@@ -76,7 +77,7 @@ public class OverflowSupportTest {
int num = 1;
for (TimeValuePair pair : support
.queryOverflowInsertInMemory(deviceId1, measurementId1, dataType1)
.queryOverflowInsertInMemory(deviceId1, measurementId1, dataType1, Collections.emptyMap())
.getSortedTimeValuePairList()) {
assertEquals(num, pair.getTimestamp());
assertEquals(num, pair.getValue().getInt());
......@@ -84,7 +85,7 @@ public class OverflowSupportTest {
}
num = 1;
for (TimeValuePair pair : support
.queryOverflowInsertInMemory(deviceId2, measurementId2, dataType2)
.queryOverflowInsertInMemory(deviceId2, measurementId2, dataType2, Collections.emptyMap())
.getSortedTimeValuePairList()) {
assertEquals(num, pair.getTimestamp());
if (num == 2) {
......
......@@ -100,7 +100,7 @@ public class IoTDBCompleteIT {
"CREATE TIMESERIES root.vehicle.d3.s7 WITH DATATYPE=INT32,ENCODING=RLE,compressor=SNAPPY",
"CREATE TIMESERIES root.vehicle.d4.s8 WITH DATATYPE=INT32,ENCODING=RLE,MAX_POINT_NUMBER=100",
"CREATE TIMESERIES root.vehicle.d5.s9 WITH DATATYPE=FLOAT,ENCODING=PLAIN,compressor=SNAPPY,MAX_POINT_NUMBER=10",
"CREATE TIMESERIES root.vehicle.d6.s10 WITH DATATYPE=DOUBLE,ENCODING=RLE,compressor=UNCOMPRESSOR,MAX_POINT_NUMBER=10",
"CREATE TIMESERIES root.vehicle.d6.s10 WITH DATATYPE=DOUBLE,ENCODING=RLE,compressor=UNCOMPRESSED,MAX_POINT_NUMBER=10",
"DELETE TIMESERIES root.vehicle.d0.*",
"SHOW TIMESERIES",
"=== Timeseries Tree ===\n"
......@@ -139,7 +139,7 @@ public class IoTDBCompleteIT {
+ " DataType: INT32,\n"
+ " Encoding: RLE,\n"
+ " Compressor: UNCOMPRESSED,\n"
+ " args: {MAX_POINT_NUMBER=100},\n"
+ " args: {max_point_number=100},\n"
+ " StorageGroup: root.vehicle\n"
+ " }\n"
+ " },\n"
......@@ -148,7 +148,16 @@ public class IoTDBCompleteIT {
+ " DataType: FLOAT,\n"
+ " Encoding: PLAIN,\n"
+ " Compressor: SNAPPY,\n"
+ " args: {MAX_POINT_NUMBER=10},\n"
+ " args: {max_point_number=10},\n"
+ " StorageGroup: root.vehicle\n"
+ " }\n"
+ " },\n"
+ " d6:{\n"
+ " s10:{\n"
+ " DataType: DOUBLE,\n"
+ " Encoding: RLE,\n"
+ " Compressor: UNCOMPRESSED,\n"
+ " args: {max_point_number=10},\n"
+ " StorageGroup: root.vehicle\n"
+ " }\n"
+ " }\n"
......
......@@ -18,24 +18,28 @@
*/
package org.apache.iotdb.db.integration;
import static org.apache.iotdb.db.integration.Constant.TIMESTAMP_STR;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import java.math.BigDecimal;
import java.math.RoundingMode;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.ArrayList;
import java.util.List;
import org.apache.iotdb.db.service.IoTDB;
import org.apache.iotdb.db.utils.EnvironmentUtils;
import org.apache.iotdb.db.utils.MathUtils;
import org.apache.iotdb.jdbc.Config;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import java.math.BigDecimal;
import java.math.RoundingMode;
import java.sql.*;
import java.util.ArrayList;
import java.util.List;
import static org.apache.iotdb.db.integration.Constant.*;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
/**
* Notice that, all test begins with "IoTDB" is integration test. All test which will start the IoTDB server should be
* defined as integration test.
......@@ -58,9 +62,6 @@ public class IoTDBFloatPrecisionIT {
EnvironmentUtils.closeStatMonitor();
EnvironmentUtils.closeMemControl();
initCreateSQLStatement();
for(String sql : sqls){
System.out.println(sql);
}
daemon = IoTDB.getInstance();
daemon.active();
EnvironmentUtils.envSetUp();
......@@ -124,12 +125,20 @@ public class IoTDBFloatPrecisionIT {
ResultSet resultSet = statement.getResultSet();
int cnt = 0;
while (resultSet.next()) {
assertEquals(TIMESTAMP+"", resultSet.getString(TIMESTAMP_STR));
for(int i = 0; i < 10; i++){
Assert.assertEquals(Float.parseFloat(VALUE), resultSet.getFloat(String.format("root.vehicle.%s.%s", "f0", "s"+i+"rle")), DELTA_FLOAT);
Assert.assertEquals(Float.parseFloat(VALUE), resultSet.getFloat(String.format("root.vehicle.%s.%s", "f0", "s"+i+"2f")), DELTA_FLOAT);
Assert.assertEquals(Double.parseDouble(VALUE), resultSet.getDouble(String.format("root.vehicle.%s.%s", "d0", "s"+i+"rle")), DELTA_FLOAT);
Assert.assertEquals(Double.parseDouble(VALUE), resultSet.getDouble(String.format("root.vehicle.%s.%s", "d0", "s"+i+"2f")), DELTA_DOUBLE);
assertEquals(TIMESTAMP + "", resultSet.getString(TIMESTAMP_STR));
for (int i = 0; i < 10; i++) {
Assert.assertEquals(MathUtils.roundWithGivenPrecision(Float.parseFloat(VALUE), i),
resultSet.getFloat(String.format("root.vehicle.%s.%s", "f0", "s" + i + "rle")),
DELTA_FLOAT);
Assert.assertEquals(MathUtils.roundWithGivenPrecision(Float.parseFloat(VALUE), i),
resultSet.getFloat(String.format("root.vehicle.%s.%s", "f0", "s" + i + "2f")),
DELTA_FLOAT);
Assert.assertEquals(MathUtils.roundWithGivenPrecision(Double.parseDouble(VALUE), i),
resultSet.getDouble(String.format("root.vehicle.%s.%s", "d0", "s" + i + "rle")),
DELTA_DOUBLE);
Assert.assertEquals(MathUtils.roundWithGivenPrecision(Double.parseDouble(VALUE), i),
resultSet.getDouble(String.format("root.vehicle.%s.%s", "d0", "s" + i + "2f")),
DELTA_DOUBLE);
}
cnt++;
}
......@@ -141,13 +150,21 @@ public class IoTDBFloatPrecisionIT {
resultSet = statement.executeQuery("select * from root");
cnt = 0;
while (resultSet.next()) {
assertEquals(TIMESTAMP+"", resultSet.getString(TIMESTAMP_STR));
for(int i = 0; i < 10; i++){
assertEquals(TIMESTAMP + "", resultSet.getString(TIMESTAMP_STR));
for (int i = 0; i < 10; i++) {
BigDecimal b = new BigDecimal(VALUE);
Assert.assertEquals(b.setScale(i, RoundingMode.HALF_UP).floatValue(), resultSet.getFloat(String.format("root.vehicle.%s.%s", "f0", "s"+i+"rle")), DELTA_FLOAT);
Assert.assertEquals(b.setScale(i, RoundingMode.HALF_UP).floatValue(), resultSet.getFloat(String.format("root.vehicle.%s.%s", "f0", "s"+i+"2f")), DELTA_FLOAT);
Assert.assertEquals(b.setScale(i, RoundingMode.HALF_UP).doubleValue(), resultSet.getDouble(String.format("root.vehicle.%s.%s", "d0", "s"+i+"rle")), DELTA_DOUBLE);
Assert.assertEquals(b.setScale(i, RoundingMode.HALF_UP).doubleValue(), resultSet.getDouble(String.format("root.vehicle.%s.%s", "d0", "s"+i+"2f")), DELTA_DOUBLE);
Assert.assertEquals(b.setScale(i, RoundingMode.HALF_UP).floatValue(),
resultSet.getFloat(String.format("root.vehicle.%s.%s", "f0", "s" + i + "rle")),
DELTA_FLOAT);
Assert.assertEquals(b.setScale(i, RoundingMode.HALF_UP).floatValue(),
resultSet.getFloat(String.format("root.vehicle.%s.%s", "f0", "s" + i + "2f")),
DELTA_FLOAT);
Assert.assertEquals(b.setScale(i, RoundingMode.HALF_UP).doubleValue(),
resultSet.getDouble(String.format("root.vehicle.%s.%s", "d0", "s" + i + "rle")),
DELTA_DOUBLE);
Assert.assertEquals(b.setScale(i, RoundingMode.HALF_UP).doubleValue(),
resultSet.getDouble(String.format("root.vehicle.%s.%s", "d0", "s" + i + "2f")),
DELTA_DOUBLE);
}
cnt++;
}
......
package org.apache.iotdb.db.utils;
import static org.junit.Assert.*;
import org.junit.Test;
public class MathUtilsTest {
@Test
public void roundFloatWithGivenPrecision() {
int[] size = new int[]{0, 4, 19};
float[] data = new float[]{0.0f, (float)Math.PI, (float)(10*Math.PI), Integer.MAX_VALUE, -Integer.MAX_VALUE};
float[][] expData = new float[][] {
{0.0f, 3.0f, 31f, Integer.MAX_VALUE, -Integer.MAX_VALUE},
{0.0f, 3.1416f, 31.4159f, Integer.MAX_VALUE, -Integer.MAX_VALUE},
{0.0f, (float)Math.PI, (float)(10*Math.PI), Integer.MAX_VALUE, -Integer.MAX_VALUE},
};
for (int j =0; j<size.length; j++) {
for (int i = 0; i < data.length; i++) {
float res = MathUtils.roundWithGivenPrecision(data[i], size[j]);
assertEquals(expData[j][i], res, Float.MIN_NORMAL);
}
}
}
@Test
public void roundDoubleWithGivenPrecision() {
int[] size = new int[]{0, 4, 19};
double[] data = new double[]{0.0, Math.PI, 10*Math.PI, Long.MAX_VALUE, -Long.MAX_VALUE};
double[][] expData = new double[][] {
{0.0, 3.0, 31, Long.MAX_VALUE, -Long.MAX_VALUE},
{0.0, 3.1416, 31.4159, Long.MAX_VALUE, -Long.MAX_VALUE},
{0.0, Math.PI, 10*Math.PI, Long.MAX_VALUE, -Long.MAX_VALUE},
};
for (int j =0; j<size.length; j++) {
for (int i = 0; i < data.length; i++) {
double res = MathUtils.roundWithGivenPrecision(data[i], size[j]);
assertEquals(expData[j][i], res, Double.MIN_NORMAL);
}
}
}
}
\ No newline at end of file
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册