未验证 提交 04be7b61 编写于 作者: X Xiangwei Wei 提交者: GitHub

Fix group by month bug that use calendar month rather than use 30 days (#2964)

* Seperate IOTDBGroupByMonthIT from IoTDBGroupByIT
上级 fd7e68f1
......@@ -31,6 +31,8 @@ import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.query.control.QueryResourceManager;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
import org.apache.iotdb.tsfile.read.filter.GroupByFilter;
import org.junit.Test;
......@@ -68,6 +70,7 @@ public class ClusterGroupByNoVFilterDataSetTest extends BaseQueryTest {
groupByPlan.setEndTime(20);
groupByPlan.setSlidingStep(5);
groupByPlan.setInterval(5);
groupByPlan.setExpression(new GlobalTimeExpression(new GroupByFilter(5, 5, 0, 20)));
ClusterGroupByNoVFilterDataSet dataSet =
new ClusterGroupByNoVFilterDataSet(queryContext, groupByPlan, testMetaMember);
......
......@@ -85,6 +85,9 @@ public class GroupByWithoutValueFilterDataSet extends GroupByEngineDataSet {
if (expression != null) {
timeFilter = ((GlobalTimeExpression) expression).getFilter();
}
if (timeFilter == null) {
throw new QueryProcessException("TimeFilter cannot be null in GroupBy query.");
}
List<StorageGroupProcessor> list =
StorageEngine.getInstance()
......@@ -102,7 +105,7 @@ public class GroupByWithoutValueFilterDataSet extends GroupByEngineDataSet {
groupByTimePlan.getAllMeasurementsInDevice(path.getDevice()),
dataTypes.get(i),
context,
timeFilter,
timeFilter.copy(),
null,
groupByTimePlan.isAscending()));
resultIndexes.put(path, new ArrayList<>());
......
......@@ -44,6 +44,7 @@ import org.apache.iotdb.tsfile.read.expression.impl.BinaryExpression;
import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
import org.apache.iotdb.tsfile.read.expression.util.ExpressionOptimizer;
import org.apache.iotdb.tsfile.read.filter.GroupByFilter;
import org.apache.iotdb.tsfile.read.filter.GroupByMonthFilter;
import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
import org.slf4j.Logger;
......@@ -154,16 +155,9 @@ public class QueryRouter implements IQueryRouter {
}
GroupByEngineDataSet dataSet = null;
long unit = groupByTimePlan.getInterval();
long slidingStep = groupByTimePlan.getSlidingStep();
long startTime = groupByTimePlan.getStartTime();
long endTime = groupByTimePlan.getEndTime();
IExpression expression = groupByTimePlan.getExpression();
List<PartialPath> selectedSeries = groupByTimePlan.getDeduplicatedPaths();
GlobalTimeExpression timeExpression =
new GlobalTimeExpression(new GroupByFilter(unit, slidingStep, startTime, endTime));
GlobalTimeExpression timeExpression = getTimeExpression(groupByTimePlan);
if (expression == null) {
expression = timeExpression;
......@@ -191,6 +185,23 @@ public class QueryRouter implements IQueryRouter {
return dataSet;
}
private GlobalTimeExpression getTimeExpression(GroupByTimePlan plan) {
if (plan.isSlidingStepByMonth() || plan.isIntervalByMonth()) {
return new GlobalTimeExpression(
(new GroupByMonthFilter(
plan.getInterval(),
plan.getSlidingStep(),
plan.getStartTime(),
plan.getEndTime(),
plan.isSlidingStepByMonth(),
plan.isIntervalByMonth())));
} else {
return new GlobalTimeExpression(
new GroupByFilter(
plan.getInterval(), plan.getSlidingStep(), plan.getStartTime(), plan.getEndTime()));
}
}
protected GroupByWithoutValueFilterDataSet getGroupByWithoutValueFilterDataSet(
QueryContext context, GroupByTimePlan plan)
throws StorageEngineException, QueryProcessException {
......
......@@ -29,14 +29,20 @@ import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.sql.*;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.List;
import java.util.TimeZone;
import static org.apache.iotdb.db.constant.TestConstant.*;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.Statement;
import static org.apache.iotdb.db.constant.TestConstant.avg;
import static org.apache.iotdb.db.constant.TestConstant.count;
import static org.apache.iotdb.db.constant.TestConstant.first_value;
import static org.apache.iotdb.db.constant.TestConstant.last_value;
import static org.apache.iotdb.db.constant.TestConstant.max_time;
import static org.apache.iotdb.db.constant.TestConstant.max_value;
import static org.apache.iotdb.db.constant.TestConstant.min_time;
import static org.apache.iotdb.db.constant.TestConstant.min_value;
import static org.apache.iotdb.db.constant.TestConstant.sum;
import static org.junit.Assert.fail;
public class IOTDBGroupByIT {
......@@ -899,157 +905,6 @@ public class IOTDBGroupByIT {
}
}
@Test
public void groupByNaturalMonth1() {
try (Connection connection =
DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
Statement statement = connection.createStatement()) {
// 10/31/2019:19:57:18
long startTime = 1572523038000L;
// 04/01/2020:19:57:18
long endTime = 1585742238000L;
String[] retArray1 = {
"10/31/2019:19:57:18",
"30.0",
"11/30/2019:19:57:18",
"31.0",
"12/31/2019:19:57:18",
"31.0",
"01/31/2020:19:57:18",
"29.0",
"02/29/2020:19:57:18",
"31.0",
"03/31/2020:19:57:18",
"1.0"
};
for (long i = startTime; i <= endTime; i += 86400_000L) {
statement.execute("insert into root.sg1.d1(timestamp, temperature) values (" + i + ", 1)");
}
DateFormat df = new SimpleDateFormat("MM/dd/yyyy:HH:mm:ss");
df.setTimeZone(TimeZone.getTimeZone("GMT+8:00"));
boolean hasResultSet =
statement.execute(
"select sum(temperature) from "
+ "root.sg1.d1 "
+ "GROUP BY ([1572523038000, 1585742238000), 1mo, 1mo)");
Assert.assertTrue(hasResultSet);
int cnt;
try (ResultSet resultSet = statement.getResultSet()) {
cnt = 0;
while (resultSet.next()) {
String time = resultSet.getString(TIMESTAMP_STR);
String ans = resultSet.getString(sum("root.sg1.d1.temperature"));
Assert.assertEquals(retArray1[cnt++], df.format(Long.parseLong(time)));
Assert.assertEquals(retArray1[cnt++], ans);
}
Assert.assertEquals(retArray1.length, cnt);
}
} catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
}
@Test
public void groupByNaturalMonth2() {
try (Connection connection =
DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
Statement statement = connection.createStatement()) {
// 10/31/2019:19:57:18
long startTime = 1572523038000L;
// 04/01/2020:19:57:18
long endTime = 1585742238000L;
String[] retArray1 = {
"10/31/2019:19:57:18",
"10.0",
"11/30/2019:19:57:18",
"10.0",
"12/31/2019:19:57:18",
"9.0",
"01/31/2020:19:57:18",
"8.0",
"02/29/2020:19:57:18",
"9.0",
"03/31/2020:19:57:18",
"1.0"
};
List<String> start = new ArrayList<>();
for (long i = startTime; i <= endTime; i += 86400_000L) {
statement.execute("insert into root.sg1.d1(timestamp, temperature) values (" + i + ", 1)");
}
DateFormat df = new SimpleDateFormat("MM/dd/yyyy:HH:mm:ss");
df.setTimeZone(TimeZone.getTimeZone("GMT+8:00"));
boolean hasResultSet =
statement.execute(
"select sum(temperature) from "
+ "root.sg1.d1 "
+ "GROUP BY ([1572523038000, 1585742238000), 10d, 1mo)");
Assert.assertTrue(hasResultSet);
int cnt;
try (ResultSet resultSet = statement.getResultSet()) {
cnt = 0;
while (resultSet.next()) {
String time = resultSet.getString(TIMESTAMP_STR);
String ans = resultSet.getString(sum("root.sg1.d1.temperature"));
Assert.assertEquals(retArray1[cnt++], df.format(Long.parseLong(time)));
Assert.assertEquals(retArray1[cnt++], ans);
}
Assert.assertEquals(retArray1.length, cnt);
}
} catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
}
@Test
public void groupByNaturalMonth3() {
// test when endtime - starttime = interval
try (Connection connection =
DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
Statement statement = connection.createStatement()) {
// 2/1/2021:15:20
long startTime = 1612164041000L;
// 3/1/2021:15:20
long endTime = 1614583241000L;
for (long i = startTime; i <= endTime; i += 86400_000L) {
statement.execute("insert into root.sg1.d1(timestamp, temperature) values (" + i + ", 1)");
}
DateFormat df = new SimpleDateFormat("MM/dd/yyyy:HH:mm:ss");
df.setTimeZone(TimeZone.getTimeZone("GMT+8:00"));
boolean hasResultSet =
statement.execute(
"select sum(temperature) from root.sg1.d1 "
+ "GROUP BY ([1612164041000, 1614583241000), 1mo)");
Assert.assertTrue(hasResultSet);
int cnt = 0;
try (ResultSet resultSet = statement.getResultSet()) {
cnt++;
}
Assert.assertEquals(1, cnt);
} catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
}
@Test
public void usingNowFunction() {
System.out.println("usingNowFunction");
......
/*
* 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.integration;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
import org.apache.iotdb.db.utils.EnvironmentUtils;
import org.apache.iotdb.jdbc.Config;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.Statement;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.TimeZone;
import static org.apache.iotdb.db.constant.TestConstant.sum;
import static org.junit.Assert.fail;
public class IoTDBGroupByMonthIT {
private static final String TIMESTAMP_STR = "Time";
private final DateFormat df = new SimpleDateFormat("MM/dd/yyyy:HH:mm:ss");
@Before
public void setUp() throws Exception {
IoTDBDescriptor.getInstance()
.getConfig()
.setCompactionStrategy(CompactionStrategy.NO_COMPACTION);
df.setTimeZone(TimeZone.getTimeZone("GMT+00:00"));
EnvironmentUtils.envSetUp();
Class.forName(Config.JDBC_DRIVER_NAME);
prepareData();
}
@After
public void tearDown() throws Exception {
EnvironmentUtils.cleanEnv();
IoTDBDescriptor.getInstance()
.getConfig()
.setCompactionStrategy(CompactionStrategy.LEVEL_COMPACTION);
}
@Test
public void groupByNaturalMonth1() {
try (Connection connection =
DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
Statement statement = connection.createStatement()) {
String[] retArray1 = {
"10/31/2020:00:00:00", "30.0",
"11/30/2020:00:00:00", "31.0",
"12/31/2020:00:00:00", "31.0",
"01/31/2021:00:00:00", "28.0",
"02/28/2021:00:00:00", "1.0"
};
boolean hasResultSet =
statement.execute(
"select sum(temperature) from root.sg1.d1 "
+ "GROUP BY ([1604102400000, 1614556800000), 1mo, 1mo)");
Assert.assertTrue(hasResultSet);
int cnt = 0;
try (ResultSet resultSet = statement.getResultSet()) {
while (resultSet.next()) {
String time = resultSet.getString(TIMESTAMP_STR);
String ans = resultSet.getString(sum("root.sg1.d1.temperature"));
Assert.assertEquals(retArray1[cnt++], df.format(Long.parseLong(time)));
Assert.assertEquals(retArray1[cnt++], ans);
}
Assert.assertEquals(retArray1.length, cnt);
}
} catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
}
@Test
public void groupByNaturalMonth2() {
try (Connection connection =
DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
Statement statement = connection.createStatement()) {
String[] retArray1 = {
"10/31/2020:00:00:00", "10.0",
"11/30/2020:00:00:00", "10.0",
"12/31/2020:00:00:00", "10.0",
"01/31/2021:00:00:00", "10.0",
"02/28/2021:00:00:00", "1.0"
};
boolean hasResultSet =
statement.execute(
"select sum(temperature) from root.sg1.d1 "
+ "GROUP BY ([1604102400000, 1614556800000), 10d, 1mo)");
Assert.assertTrue(hasResultSet);
int cnt = 0;
try (ResultSet resultSet = statement.getResultSet()) {
while (resultSet.next()) {
String time = resultSet.getString(TIMESTAMP_STR);
String ans = resultSet.getString(sum("root.sg1.d1.temperature"));
Assert.assertEquals(retArray1[cnt++], df.format(Long.parseLong(time)));
Assert.assertEquals(retArray1[cnt++], ans);
}
Assert.assertEquals(retArray1.length, cnt);
}
} catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
}
/** Test when endTime - startTime = interval */
@Test
public void groupByNaturalMonth3() {
try (Connection connection =
DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
Statement statement = connection.createStatement()) {
boolean hasResultSet =
statement.execute(
"select sum(temperature) from root.sg1.d1 "
+ "GROUP BY ([1604102400000, 1606694400000), 1mo)");
Assert.assertTrue(hasResultSet);
int cnt = 0;
ResultSet resultSet = statement.getResultSet();
while (resultSet.next()) {
cnt++;
}
Assert.assertEquals(1, cnt);
} catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
}
private void prepareData() {
try (Connection connection =
DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
Statement statement = connection.createStatement()) {
// 2020-10-31 08:00:00
long startTime = 1604102400000L;
// 2021-03-01 08:00:00
long endTime = 1614556800000L;
for (long i = startTime; i <= endTime; i += 86400_000L) {
statement.execute("insert into root.sg1.d1(timestamp, temperature) values (" + i + ", 1)");
}
} catch (Exception e) {
e.printStackTrace();
}
}
}
......@@ -31,10 +31,10 @@ import java.util.Objects;
public class GroupByFilter implements Filter, Serializable {
private static final long serialVersionUID = -1211805021419281440L;
private long interval;
private long slidingStep;
private long startTime;
private long endTime;
protected long interval;
protected long slidingStep;
protected long startTime;
protected long endTime;
public GroupByFilter(long interval, long slidingStep, long startTime, long endTime) {
this.interval = interval;
......@@ -58,9 +58,8 @@ public class GroupByFilter implements Filter, Serializable {
@Override
public boolean satisfyStartEndTime(long startTime, long endTime) {
if (endTime < this.startTime) return false;
if (endTime < this.startTime || startTime >= this.endTime) return false;
else if (startTime <= this.startTime) return true;
else if (startTime >= this.endTime) return false;
else {
long minTime = startTime - this.startTime;
long count = minTime / slidingStep;
......
/*
* 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.filter;
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
import java.util.Calendar;
import java.util.Objects;
/**
* GroupByMonthFilter is used to handle natural month slidingStep and interval by generating
* dynamically. Attention: it's only supported to access in ascending order now.
*/
public class GroupByMonthFilter extends GroupByFilter {
private final boolean isSlidingStepByMonth;
private final boolean isIntervalByMonth;
private int slidingStepsInMo;
private int intervalInMo;
private final Calendar calendar = Calendar.getInstance();
private static final long MS_TO_MONTH = 30 * 86400_000L;
private int intervalCnt = 0;
/** 10.31 -> 11.30 -> 12.31, not 10.31 -> 11.30 -> 12.30 */
private final long initialStartTime;
public GroupByMonthFilter(
long interval,
long slidingStep,
long startTime,
long endTime,
boolean isSlidingStepByMonth,
boolean isIntervalByMonth) {
super(interval, slidingStep, startTime, endTime);
initialStartTime = startTime;
calendar.setTimeInMillis(startTime);
this.isIntervalByMonth = isIntervalByMonth;
this.isSlidingStepByMonth = isSlidingStepByMonth;
if (isIntervalByMonth) {
// TODO: 1mo1d
intervalInMo = (int) (interval / MS_TO_MONTH);
}
if (isSlidingStepByMonth) {
slidingStepsInMo = (int) (slidingStep / MS_TO_MONTH);
}
getNextIntervalAndSlidingStep();
}
// TODO: time descending order
@Override
public boolean satisfy(long time, Object value) {
if (time < startTime || time >= endTime) {
return false;
} else if (time - startTime < interval) {
return true;
} else {
this.startTime = calendar.getTimeInMillis();
getNextIntervalAndSlidingStep();
return satisfy(time, value);
}
}
@Override
public boolean satisfyStartEndTime(long startTime, long endTime) {
boolean isSatisfy = satisfyCurrentInterval(startTime, endTime);
if (isSatisfy) {
return true;
} else {
long beforeStartTime = this.startTime;
int beforeIntervalCnt = this.intervalCnt;
// TODO: optimize to jump but not one by one
while (endTime >= this.startTime && !isSatisfy) {
this.startTime = calendar.getTimeInMillis();
getNextIntervalAndSlidingStep();
isSatisfy = satisfyCurrentInterval(startTime, endTime);
}
// recover the initial state
this.intervalCnt = beforeIntervalCnt - 1;
this.startTime = beforeStartTime;
getNextIntervalAndSlidingStep();
return isSatisfy;
}
}
@Override
public Filter copy() {
return new GroupByMonthFilter(
interval, slidingStep, startTime, endTime, isSlidingStepByMonth, isIntervalByMonth);
}
private boolean satisfyCurrentInterval(long startTime, long endTime) {
if (endTime < this.startTime || startTime >= this.endTime) {
return false;
} else {
return startTime <= this.startTime || startTime - this.startTime < interval;
}
}
@Override
public boolean containStartEndTime(long startTime, long endTime) {
boolean isContained = isContainedByCurrentInterval(startTime, endTime);
if (isContained) {
return true;
} else {
long beforeStartTime = this.startTime;
int beforeIntervalCnt = this.intervalCnt;
while (!isContained && startTime >= this.startTime) {
this.startTime = calendar.getTimeInMillis();
getNextIntervalAndSlidingStep();
isContained = isContainedByCurrentInterval(startTime, endTime);
}
// recover the initial state
this.intervalCnt = beforeIntervalCnt - 1;
this.startTime = beforeStartTime;
getNextIntervalAndSlidingStep();
return isContained;
}
}
private boolean isContainedByCurrentInterval(long startTime, long endTime) {
if (startTime < this.startTime || endTime > this.endTime) {
return false;
} else {
return startTime - this.startTime < interval && endTime - this.startTime < interval;
}
}
@Override
public boolean equals(Object obj) {
if (!(obj instanceof GroupByMonthFilter)) {
return false;
}
GroupByMonthFilter other = (GroupByMonthFilter) obj;
return this.interval == other.interval
&& this.slidingStep == other.slidingStep
&& this.startTime == other.startTime
&& this.endTime == other.endTime
&& this.isSlidingStepByMonth == other.isSlidingStepByMonth
&& this.isIntervalByMonth == other.isIntervalByMonth;
}
@Override
public int hashCode() {
return Objects.hash(
interval, slidingStep, startTime, endTime, isSlidingStepByMonth, isIntervalByMonth);
}
private void getNextIntervalAndSlidingStep() {
intervalCnt++;
if (isIntervalByMonth) {
calendar.setTimeInMillis(initialStartTime);
calendar.add(Calendar.MONTH, slidingStepsInMo * (intervalCnt - 1) + intervalInMo);
this.interval = calendar.getTimeInMillis() - startTime;
}
if (isSlidingStepByMonth) {
calendar.setTimeInMillis(initialStartTime);
calendar.add(Calendar.MONTH, slidingStepsInMo * intervalCnt);
this.slidingStep = calendar.getTimeInMillis() - startTime;
}
}
}
/*
* 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.filter;
import org.apache.iotdb.tsfile.file.metadata.statistics.LongStatistics;
import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue;
public class GroupByMonthFilterTest {
// The number of milliseconds in 30 days
private final long MS_TO_DAY = 86400_000L;
private final long MS_TO_MONTH = 30 * MS_TO_DAY;
// 1970-12-31 23:59:59
private final long END_TIME = 31507199000L;
/** Test filter with slidingStep = 2 month, and timeInterval = 1 month */
@Test
public void TestSatisfy1() {
GroupByMonthFilter filter =
new GroupByMonthFilter(MS_TO_MONTH, 2 * MS_TO_MONTH, 0, END_TIME, true, true);
// 1970-01-01 08:00:00, timezone = GMT+08:00
assertTrue(filter.satisfy(0, null));
// 1970-02-01 07:59:59
assertTrue(filter.satisfy(2678399000L, null));
// 1970-02-01 08:00:00
assertFalse(filter.satisfy(2678400000L, null));
// 1970-03-01 07:59:59
assertFalse(filter.satisfy(5097599000L, null));
// 1970-03-01 08:00:00
assertTrue(filter.satisfy(5097600000L, null));
// 1970-04-05 00:00:00
assertFalse(filter.satisfy(8092800000L, null));
// 1970-07-01 07:59:59
assertFalse(filter.satisfy(15638399000L, null));
// 1970-11-30 23:59:59
assertTrue(filter.satisfy(28828799000L, null));
// 1970-12-31 23:59:59
assertFalse(filter.satisfy(31507199000L, null));
}
/** Test filter with slidingStep = 1 month, and timeInterval = 1 month */
@Test
public void TestSatisfy2() {
GroupByMonthFilter filter =
new GroupByMonthFilter(MS_TO_MONTH, MS_TO_MONTH, 0, END_TIME, true, true);
// 1970-01-01 08:00:00, timezone = GMT+08:00
assertTrue(filter.satisfy(0, null));
// 1970-02-01 07:59:59
assertTrue(filter.satisfy(2678399000L, null));
// 1970-02-01 08:00:00
assertTrue(filter.satisfy(2678400000L, null));
// 1970-03-01 07:59:59
assertTrue(filter.satisfy(5097599000L, null));
// 1970-03-01 08:00:00
assertTrue(filter.satisfy(5097600000L, null));
// 1970-12-31 23:59:58
assertTrue(filter.satisfy(31507198000L, null));
// 1970-12-31 23:59:59
assertFalse(filter.satisfy(31507199000L, null));
}
/** Test filter with slidingStep = 1 month, and timeInterval = 1 day */
@Test
public void TestSatisfy3() {
GroupByMonthFilter filter =
new GroupByMonthFilter(MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false);
// 1970-01-01 08:00:00, timezone = GMT+08:00
assertTrue(filter.satisfy(0, null));
// 1970-01-02 07:59:59
assertTrue(filter.satisfy(86399000L, null));
// 1970-01-02 08:00:00
assertFalse(filter.satisfy(86400000L, null));
// 1970-02-01 07:59:59
assertFalse(filter.satisfy(2678399000L, null));
// 1970-02-01 08:00:00
assertTrue(filter.satisfy(2678400000L, null));
// 1970-03-01 08:00:00
assertTrue(filter.satisfy(5097600000L, null));
// 1970-12-01 08:00:00
assertTrue(filter.satisfy(28857600000L, null));
// 1970-12-31 23:59:59
assertFalse(filter.satisfy(31507199000L, null));
}
/** Test filter with slidingStep = 1 month, and timeInterval = 1 day */
@Test
public void TestSatisfyStartEndTime() {
GroupByMonthFilter filter =
new GroupByMonthFilter(MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false);
// 1970-01-01 08:00:00 - 1970-01-02 08:00:00, timezone = GMT+08:00
Statistics statistics = new LongStatistics();
statistics.setStartTime(0);
statistics.setEndTime(MS_TO_DAY);
assertTrue(filter.satisfy(statistics));
// 1970-01-01 20:00:00 - 1970-01-02 08:00:00
statistics.setStartTime(MS_TO_DAY / 2);
statistics.setEndTime(MS_TO_DAY);
assertTrue(filter.satisfy(statistics));
// 1970-01-01 20:00:00 - 1970-01-03 08:00:00
statistics.setStartTime(MS_TO_DAY / 2);
statistics.setEndTime(MS_TO_DAY * 2);
assertTrue(filter.satisfy(statistics));
// 1970-01-02 08:00:00 - 1970-01-03 08:00:00
statistics.setStartTime(MS_TO_DAY);
statistics.setEndTime(MS_TO_DAY * 2);
assertFalse(filter.satisfy(statistics));
// 1970-02-28 08:00:00 - 1970-03-01 07:59:59
statistics.setStartTime(5011200000L);
statistics.setEndTime(5097599000L);
assertFalse(filter.satisfy(statistics));
// 1970-03-01 09:00:00 - 1970-03-01 10:00:00
statistics.setStartTime(5101200000L);
statistics.setEndTime(5104800000L);
assertTrue(filter.satisfy(statistics));
// 1970-05-01 07:00:00 - 1970-05-01 08:00:00
statistics.setStartTime(10364400000L);
statistics.setEndTime(10368000000L);
assertTrue(filter.satisfy(statistics));
// 1970-01-02 07:59:59
assertTrue(filter.satisfy(86399000L, null));
}
/** Test filter with slidingStep = 1 month, and timeInterval = 1 day */
@Test
public void TestContainStartEndTime() {
GroupByMonthFilter filter =
new GroupByMonthFilter(MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false);
// 1970-01-01 08:00:00 - 1970-01-02 08:00:00, timezone = GMT+08:00
assertFalse(filter.containStartEndTime(0, MS_TO_DAY));
// 1970-01-01 08:00:00 - 1970-01-02 07:59:59
assertTrue(filter.containStartEndTime(0, MS_TO_DAY - 1000));
// 1970-02-01 07:59:59 - 1970-02-02 07:59:59
assertFalse(filter.containStartEndTime(2678399000L, 2764799000L));
// 1970-02-01 08:00:00 - 1970-02-02 07:59:59
assertTrue(filter.containStartEndTime(2678400000L, 2764799000L));
// 1970-02-01 08:00:00 - 1970-02-02 08:00:00
assertFalse(filter.containStartEndTime(2678400000L, 2764800000L));
// 1970-02-10 08:00:00 - 1970-02-11 08:00:00
assertFalse(filter.containStartEndTime(3456000000L, 3542400000L));
// 1970-10-01 10:00:00 - 1970-10-01 12:00:00
assertTrue(filter.containStartEndTime(23594400000L, 23601600000L));
// 1970-05-01 08:00:00 - 1970-05-01 10:00:00
assertTrue(filter.containStartEndTime(10368000000L, 10375200000L));
// 1970-03-01 08:00:00 - 1970-05-01 10:00:00
assertFalse(filter.containStartEndTime(5097600000L, 10375200000L));
// 1970-01-02 07:59:59
assertTrue(filter.satisfy(86399000L, null));
}
@Test
public void TestEquals() {
GroupByMonthFilter filter =
new GroupByMonthFilter(MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false);
Filter filter2 = filter.copy();
assertEquals(filter, filter2);
GroupByMonthFilter filter3 =
new GroupByMonthFilter(MS_TO_MONTH, MS_TO_MONTH, 0, END_TIME, true, true);
assertNotEquals(filter, filter3);
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册