Improve Cassandra resultSet pagination
This commit is contained in:
		
							parent
							
								
									34da0fe811
								
							
						
					
					
						commit
						2d235cf9eb
					
				@ -176,9 +176,9 @@ cassandra:
 | 
			
		||||
  # Enable/disable secure connection
 | 
			
		||||
  ssl: "${CASSANDRA_USE_SSL:false}"
 | 
			
		||||
  # Enable/disable JMX
 | 
			
		||||
  jmx: "${CASSANDRA_USE_JMX:true}"
 | 
			
		||||
  jmx: "${CASSANDRA_USE_JMX:false}"
 | 
			
		||||
  # Enable/disable metrics collection.
 | 
			
		||||
  metrics: "${CASSANDRA_USE_METRICS:true}"
 | 
			
		||||
  metrics: "${CASSANDRA_USE_METRICS:false}"
 | 
			
		||||
  # NONE SNAPPY LZ4
 | 
			
		||||
  compression: "${CASSANDRA_COMPRESSION:none}"
 | 
			
		||||
  # Specify cassandra cluster initialization timeout in milliseconds (if no hosts available during startup)
 | 
			
		||||
 | 
			
		||||
@ -16,11 +16,16 @@
 | 
			
		||||
package org.thingsboard.server.dao.nosql;
 | 
			
		||||
 | 
			
		||||
import com.datastax.oss.driver.api.core.cql.Statement;
 | 
			
		||||
import com.google.common.util.concurrent.Futures;
 | 
			
		||||
import com.google.common.util.concurrent.ListenableFuture;
 | 
			
		||||
import com.google.common.util.concurrent.MoreExecutors;
 | 
			
		||||
import lombok.Data;
 | 
			
		||||
import org.thingsboard.server.common.data.id.TenantId;
 | 
			
		||||
import org.thingsboard.server.dao.cassandra.guava.GuavaSession;
 | 
			
		||||
import org.thingsboard.server.dao.util.AsyncTask;
 | 
			
		||||
 | 
			
		||||
import java.util.function.Function;
 | 
			
		||||
 | 
			
		||||
/**
 | 
			
		||||
 * Created by ashvayka on 24.10.18.
 | 
			
		||||
 */
 | 
			
		||||
@ -31,4 +36,11 @@ public class CassandraStatementTask implements AsyncTask {
 | 
			
		||||
    private final GuavaSession session;
 | 
			
		||||
    private final Statement statement;
 | 
			
		||||
 | 
			
		||||
    public ListenableFuture<TbResultSet> executeAsync(Function<Statement, TbResultSetFuture> executeAsyncFunction) {
 | 
			
		||||
        return Futures.transform(session.executeAsync(statement),
 | 
			
		||||
                result -> new TbResultSet(statement, result, executeAsyncFunction),
 | 
			
		||||
                MoreExecutors.directExecutor()
 | 
			
		||||
        );
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@ -0,0 +1,131 @@
 | 
			
		||||
/**
 | 
			
		||||
 * Copyright © 2016-2020 The Thingsboard Authors
 | 
			
		||||
 *
 | 
			
		||||
 * Licensed under the Apache License, Version 2.0 (the "License");
 | 
			
		||||
 * you may not use this file except in compliance with the License.
 | 
			
		||||
 * You may obtain a copy of the License at
 | 
			
		||||
 *
 | 
			
		||||
 *     http://www.apache.org/licenses/LICENSE-2.0
 | 
			
		||||
 *
 | 
			
		||||
 * Unless required by applicable law or agreed to in writing, software
 | 
			
		||||
 * distributed under the License is distributed on an "AS IS" BASIS,
 | 
			
		||||
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 | 
			
		||||
 * See the License for the specific language governing permissions and
 | 
			
		||||
 * limitations under the License.
 | 
			
		||||
 */
 | 
			
		||||
package org.thingsboard.server.dao.nosql;
 | 
			
		||||
 | 
			
		||||
import com.datastax.oss.driver.api.core.cql.AsyncResultSet;
 | 
			
		||||
import com.datastax.oss.driver.api.core.cql.ColumnDefinitions;
 | 
			
		||||
import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
 | 
			
		||||
import com.datastax.oss.driver.api.core.cql.Row;
 | 
			
		||||
import com.datastax.oss.driver.api.core.cql.Statement;
 | 
			
		||||
import com.google.common.collect.Lists;
 | 
			
		||||
import com.google.common.util.concurrent.FutureCallback;
 | 
			
		||||
import com.google.common.util.concurrent.Futures;
 | 
			
		||||
import com.google.common.util.concurrent.ListenableFuture;
 | 
			
		||||
import com.google.common.util.concurrent.MoreExecutors;
 | 
			
		||||
import com.google.common.util.concurrent.SettableFuture;
 | 
			
		||||
import edu.umd.cs.findbugs.annotations.NonNull;
 | 
			
		||||
import org.checkerframework.checker.nullness.qual.Nullable;
 | 
			
		||||
 | 
			
		||||
import java.nio.ByteBuffer;
 | 
			
		||||
import java.util.ArrayList;
 | 
			
		||||
import java.util.List;
 | 
			
		||||
import java.util.concurrent.CompletionStage;
 | 
			
		||||
import java.util.concurrent.Executor;
 | 
			
		||||
import java.util.function.Function;
 | 
			
		||||
 | 
			
		||||
public class TbResultSet implements AsyncResultSet {
 | 
			
		||||
 | 
			
		||||
    private final Statement originalStatement;
 | 
			
		||||
    private final AsyncResultSet delegate;
 | 
			
		||||
    private final Function<Statement, TbResultSetFuture> executeAsyncFunction;
 | 
			
		||||
 | 
			
		||||
    public TbResultSet(Statement originalStatement, AsyncResultSet delegate,
 | 
			
		||||
                       Function<Statement, TbResultSetFuture> executeAsyncFunction) {
 | 
			
		||||
        this.originalStatement = originalStatement;
 | 
			
		||||
        this.delegate = delegate;
 | 
			
		||||
        this.executeAsyncFunction = executeAsyncFunction;
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @NonNull
 | 
			
		||||
    @Override
 | 
			
		||||
    public ColumnDefinitions getColumnDefinitions() {
 | 
			
		||||
        return delegate.getColumnDefinitions();
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @NonNull
 | 
			
		||||
    @Override
 | 
			
		||||
    public ExecutionInfo getExecutionInfo() {
 | 
			
		||||
        return delegate.getExecutionInfo();
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @Override
 | 
			
		||||
    public int remaining() {
 | 
			
		||||
        return delegate.remaining();
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @NonNull
 | 
			
		||||
    @Override
 | 
			
		||||
    public Iterable<Row> currentPage() {
 | 
			
		||||
        return delegate.currentPage();
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @Override
 | 
			
		||||
    public boolean hasMorePages() {
 | 
			
		||||
        return delegate.hasMorePages();
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @NonNull
 | 
			
		||||
    @Override
 | 
			
		||||
    public CompletionStage<AsyncResultSet> fetchNextPage() throws IllegalStateException {
 | 
			
		||||
        return delegate.fetchNextPage();
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @Override
 | 
			
		||||
    public boolean wasApplied() {
 | 
			
		||||
        return delegate.wasApplied();
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    public ListenableFuture<List<Row>> allRows(Executor executor) {
 | 
			
		||||
        List<Row> allRows = new ArrayList<>();
 | 
			
		||||
        SettableFuture<List<Row>> resultFuture = SettableFuture.create();
 | 
			
		||||
        this.processRows(originalStatement, delegate, allRows, resultFuture, executor);
 | 
			
		||||
        return resultFuture;
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    private void processRows(Statement statement,
 | 
			
		||||
                             AsyncResultSet resultSet,
 | 
			
		||||
                             List<Row> allRows,
 | 
			
		||||
                             SettableFuture<List<Row>> resultFuture,
 | 
			
		||||
                             Executor executor) {
 | 
			
		||||
        allRows.addAll(loadRows(resultSet));
 | 
			
		||||
        if (resultSet.hasMorePages()) {
 | 
			
		||||
            ByteBuffer nextPagingState = resultSet.getExecutionInfo().getPagingState();
 | 
			
		||||
            Statement<?> nextStatement = statement.setPagingState(nextPagingState);
 | 
			
		||||
            TbResultSetFuture resultSetFuture = executeAsyncFunction.apply(nextStatement);
 | 
			
		||||
            Futures.addCallback(resultSetFuture,
 | 
			
		||||
                    new FutureCallback<TbResultSet>() {
 | 
			
		||||
                        @Override
 | 
			
		||||
                        public void onSuccess(@Nullable TbResultSet result) {
 | 
			
		||||
                            processRows(nextStatement, result,
 | 
			
		||||
                                    allRows, resultFuture, executor);
 | 
			
		||||
                        }
 | 
			
		||||
 | 
			
		||||
                        @Override
 | 
			
		||||
                        public void onFailure(Throwable t) {
 | 
			
		||||
                            resultFuture.setException(t);
 | 
			
		||||
                        }
 | 
			
		||||
                    }, executor != null ? executor : MoreExecutors.directExecutor()
 | 
			
		||||
            );
 | 
			
		||||
        } else {
 | 
			
		||||
            resultFuture.set(allRows);
 | 
			
		||||
        }
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    List<Row> loadRows(AsyncResultSet resultSet) {
 | 
			
		||||
        return Lists.newArrayList(resultSet.currentPage());
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
}
 | 
			
		||||
@ -27,19 +27,19 @@ import java.util.concurrent.TimeoutException;
 | 
			
		||||
/**
 | 
			
		||||
 * Created by ashvayka on 24.10.18.
 | 
			
		||||
 */
 | 
			
		||||
public class TbResultSetFuture implements ListenableFuture<AsyncResultSet> {
 | 
			
		||||
public class TbResultSetFuture implements ListenableFuture<TbResultSet> {
 | 
			
		||||
 | 
			
		||||
    private final SettableFuture<AsyncResultSet> mainFuture;
 | 
			
		||||
    private final SettableFuture<TbResultSet> mainFuture;
 | 
			
		||||
 | 
			
		||||
    public TbResultSetFuture(SettableFuture<AsyncResultSet> mainFuture) {
 | 
			
		||||
    public TbResultSetFuture(SettableFuture<TbResultSet> mainFuture) {
 | 
			
		||||
        this.mainFuture = mainFuture;
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    public AsyncResultSet getUninterruptibly() {
 | 
			
		||||
    public TbResultSet getUninterruptibly() {
 | 
			
		||||
        return getSafe();
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    public AsyncResultSet getUninterruptibly(long timeout, TimeUnit unit) throws TimeoutException {
 | 
			
		||||
    public TbResultSet getUninterruptibly(long timeout, TimeUnit unit) throws TimeoutException {
 | 
			
		||||
        return getSafe(timeout, unit);
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
@ -59,12 +59,12 @@ public class TbResultSetFuture implements ListenableFuture<AsyncResultSet> {
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @Override
 | 
			
		||||
    public AsyncResultSet get() throws InterruptedException, ExecutionException {
 | 
			
		||||
    public TbResultSet get() throws InterruptedException, ExecutionException {
 | 
			
		||||
        return mainFuture.get();
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @Override
 | 
			
		||||
    public AsyncResultSet get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
 | 
			
		||||
    public TbResultSet get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
 | 
			
		||||
        return mainFuture.get(timeout, unit);
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
@ -73,7 +73,7 @@ public class TbResultSetFuture implements ListenableFuture<AsyncResultSet> {
 | 
			
		||||
        mainFuture.addListener(listener, executor);
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    private AsyncResultSet getSafe() {
 | 
			
		||||
    private TbResultSet getSafe() {
 | 
			
		||||
        try {
 | 
			
		||||
            return mainFuture.get();
 | 
			
		||||
        } catch (InterruptedException | ExecutionException e) {
 | 
			
		||||
@ -81,7 +81,7 @@ public class TbResultSetFuture implements ListenableFuture<AsyncResultSet> {
 | 
			
		||||
        }
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    private AsyncResultSet getSafe(long timeout, TimeUnit unit) throws TimeoutException {
 | 
			
		||||
    private TbResultSet getSafe(long timeout, TimeUnit unit) throws TimeoutException {
 | 
			
		||||
        try {
 | 
			
		||||
            return mainFuture.get(timeout, unit);
 | 
			
		||||
        } catch (InterruptedException | ExecutionException e) {
 | 
			
		||||
 | 
			
		||||
@ -52,21 +52,21 @@ public abstract class CassandraAbstractAsyncDao extends CassandraAbstractDao {
 | 
			
		||||
        }
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    protected <T> ListenableFuture<T> getFuture(TbResultSetFuture future, java.util.function.Function<AsyncResultSet, T> transformer) {
 | 
			
		||||
        return Futures.transform(future, new Function<AsyncResultSet, T>() {
 | 
			
		||||
    protected <T> ListenableFuture<T> getFuture(TbResultSetFuture future, java.util.function.Function<TbResultSet, T> transformer) {
 | 
			
		||||
        return Futures.transform(future, new Function<TbResultSet, T>() {
 | 
			
		||||
            @Nullable
 | 
			
		||||
            @Override
 | 
			
		||||
            public T apply(@Nullable AsyncResultSet input) {
 | 
			
		||||
            public T apply(@Nullable TbResultSet input) {
 | 
			
		||||
                return transformer.apply(input);
 | 
			
		||||
            }
 | 
			
		||||
        }, readResultsProcessingExecutor);
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    protected <T> ListenableFuture<T> getFutureAsync(TbResultSetFuture future, com.google.common.util.concurrent.AsyncFunction<AsyncResultSet, T> transformer) {
 | 
			
		||||
        return Futures.transformAsync(future, new AsyncFunction<AsyncResultSet, T>() {
 | 
			
		||||
    protected <T> ListenableFuture<T> getFutureAsync(TbResultSetFuture future, com.google.common.util.concurrent.AsyncFunction<TbResultSet, T> transformer) {
 | 
			
		||||
        return Futures.transformAsync(future, new AsyncFunction<TbResultSet, T>() {
 | 
			
		||||
            @Nullable
 | 
			
		||||
            @Override
 | 
			
		||||
            public ListenableFuture<T> apply(@Nullable AsyncResultSet input) {
 | 
			
		||||
            public ListenableFuture<T> apply(@Nullable TbResultSet input) {
 | 
			
		||||
                try {
 | 
			
		||||
                    return transformer.apply(input);
 | 
			
		||||
                } catch (Exception e) {
 | 
			
		||||
@ -76,8 +76,4 @@ public abstract class CassandraAbstractAsyncDao extends CassandraAbstractDao {
 | 
			
		||||
        }, readResultsProcessingExecutor);
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    protected ListenableFuture<List<Row>> allRows(AsyncResultSet resultSet) {
 | 
			
		||||
        return ResultSetUtils.allRows(resultSet, readResultsProcessingExecutor);
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@ -39,7 +39,7 @@ import java.util.Map;
 | 
			
		||||
@Component
 | 
			
		||||
@Slf4j
 | 
			
		||||
@NoSqlAnyDao
 | 
			
		||||
public class CassandraBufferedRateExecutor extends AbstractBufferedRateExecutor<CassandraStatementTask, TbResultSetFuture, AsyncResultSet> {
 | 
			
		||||
public class CassandraBufferedRateExecutor extends AbstractBufferedRateExecutor<CassandraStatementTask, TbResultSetFuture, TbResultSet> {
 | 
			
		||||
 | 
			
		||||
    @Autowired
 | 
			
		||||
    private EntityService entityService;
 | 
			
		||||
@ -107,19 +107,22 @@ public class CassandraBufferedRateExecutor extends AbstractBufferedRateExecutor<
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @Override
 | 
			
		||||
    protected SettableFuture<AsyncResultSet> create() {
 | 
			
		||||
    protected SettableFuture<TbResultSet> create() {
 | 
			
		||||
        return SettableFuture.create();
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @Override
 | 
			
		||||
    protected TbResultSetFuture wrap(CassandraStatementTask task, SettableFuture<AsyncResultSet> future) {
 | 
			
		||||
    protected TbResultSetFuture wrap(CassandraStatementTask task, SettableFuture<TbResultSet> future) {
 | 
			
		||||
        return new TbResultSetFuture(future);
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @Override
 | 
			
		||||
    protected ListenableFuture<AsyncResultSet> execute(AsyncTaskContext<CassandraStatementTask, AsyncResultSet> taskCtx) {
 | 
			
		||||
    protected ListenableFuture<TbResultSet> execute(AsyncTaskContext<CassandraStatementTask, TbResultSet> taskCtx) {
 | 
			
		||||
        CassandraStatementTask task = taskCtx.getTask();
 | 
			
		||||
        return task.getSession().executeAsync(task.getStatement());
 | 
			
		||||
        return task.executeAsync(
 | 
			
		||||
                statement ->
 | 
			
		||||
                    this.submit(new CassandraStatementTask(task.getTenantId(), task.getSession(), statement))
 | 
			
		||||
        );
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@ -1,65 +0,0 @@
 | 
			
		||||
/**
 | 
			
		||||
 * Copyright © 2016-2020 The Thingsboard Authors
 | 
			
		||||
 *
 | 
			
		||||
 * Licensed under the Apache License, Version 2.0 (the "License");
 | 
			
		||||
 * you may not use this file except in compliance with the License.
 | 
			
		||||
 * You may obtain a copy of the License at
 | 
			
		||||
 *
 | 
			
		||||
 *     http://www.apache.org/licenses/LICENSE-2.0
 | 
			
		||||
 *
 | 
			
		||||
 * Unless required by applicable law or agreed to in writing, software
 | 
			
		||||
 * distributed under the License is distributed on an "AS IS" BASIS,
 | 
			
		||||
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 | 
			
		||||
 * See the License for the specific language governing permissions and
 | 
			
		||||
 * limitations under the License.
 | 
			
		||||
 */
 | 
			
		||||
package org.thingsboard.server.dao.nosql;
 | 
			
		||||
 | 
			
		||||
import com.datastax.oss.driver.api.core.cql.AsyncResultSet;
 | 
			
		||||
import com.datastax.oss.driver.api.core.cql.Row;
 | 
			
		||||
import com.google.common.collect.Lists;
 | 
			
		||||
import com.google.common.util.concurrent.Futures;
 | 
			
		||||
import com.google.common.util.concurrent.ListenableFuture;
 | 
			
		||||
import com.google.common.util.concurrent.SettableFuture;
 | 
			
		||||
 | 
			
		||||
import java.util.ArrayList;
 | 
			
		||||
import java.util.List;
 | 
			
		||||
import java.util.concurrent.CompletionStage;
 | 
			
		||||
import java.util.concurrent.Executor;
 | 
			
		||||
import java.util.stream.Collectors;
 | 
			
		||||
 | 
			
		||||
public class ResultSetUtils {
 | 
			
		||||
 | 
			
		||||
    public static ListenableFuture<List<Row>> allRows(AsyncResultSet resultSet, Executor executor) {
 | 
			
		||||
        List<ListenableFuture<AsyncResultSet>> futures = new ArrayList<>();
 | 
			
		||||
        futures.add(Futures.immediateFuture(resultSet));
 | 
			
		||||
        while (resultSet.hasMorePages()) {
 | 
			
		||||
            futures.add(toListenable(resultSet.fetchNextPage()));
 | 
			
		||||
        }
 | 
			
		||||
        return Futures.transform( Futures.allAsList(futures),
 | 
			
		||||
                resultSets -> resultSets.stream()
 | 
			
		||||
                        .map(rs -> loadRows(rs))
 | 
			
		||||
                        .flatMap(rows -> rows.stream())
 | 
			
		||||
                        .collect(Collectors.toList()),
 | 
			
		||||
                executor
 | 
			
		||||
        );
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    private static <T> ListenableFuture<T> toListenable(CompletionStage<T> completable) {
 | 
			
		||||
        SettableFuture<T> future = SettableFuture.create();
 | 
			
		||||
        completable.whenComplete(
 | 
			
		||||
                (r, ex) -> {
 | 
			
		||||
                    if (ex != null) {
 | 
			
		||||
                        future.setException(ex);
 | 
			
		||||
                    } else {
 | 
			
		||||
                        future.set(r);
 | 
			
		||||
                    }
 | 
			
		||||
                }
 | 
			
		||||
        );
 | 
			
		||||
        return future;
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    private static List<Row> loadRows(AsyncResultSet resultSet) {
 | 
			
		||||
        return Lists.newArrayList(resultSet.currentPage());
 | 
			
		||||
    }
 | 
			
		||||
}
 | 
			
		||||
@ -29,7 +29,7 @@ import org.thingsboard.server.common.data.kv.JsonDataEntry;
 | 
			
		||||
import org.thingsboard.server.common.data.kv.LongDataEntry;
 | 
			
		||||
import org.thingsboard.server.common.data.kv.StringDataEntry;
 | 
			
		||||
import org.thingsboard.server.common.data.kv.TsKvEntry;
 | 
			
		||||
import org.thingsboard.server.dao.nosql.ResultSetUtils;
 | 
			
		||||
import org.thingsboard.server.dao.nosql.TbResultSet;
 | 
			
		||||
 | 
			
		||||
import javax.annotation.Nullable;
 | 
			
		||||
import java.util.List;
 | 
			
		||||
@ -41,7 +41,7 @@ import java.util.stream.Collectors;
 | 
			
		||||
 * Created by ashvayka on 20.02.17.
 | 
			
		||||
 */
 | 
			
		||||
@Slf4j
 | 
			
		||||
public class AggregatePartitionsFunction implements com.google.common.util.concurrent.AsyncFunction<List<AsyncResultSet>, Optional<TsKvEntry>> {
 | 
			
		||||
public class AggregatePartitionsFunction implements com.google.common.util.concurrent.AsyncFunction<List<TbResultSet>, Optional<TsKvEntry>> {
 | 
			
		||||
 | 
			
		||||
    private static final int LONG_CNT_POS = 0;
 | 
			
		||||
    private static final int DOUBLE_CNT_POS = 1;
 | 
			
		||||
@ -67,14 +67,14 @@ public class AggregatePartitionsFunction implements com.google.common.util.concu
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @Override
 | 
			
		||||
    public ListenableFuture<Optional<TsKvEntry>> apply(@Nullable List<AsyncResultSet> rsList) {
 | 
			
		||||
    public ListenableFuture<Optional<TsKvEntry>> apply(@Nullable List<TbResultSet> rsList) {
 | 
			
		||||
    log.trace("[{}][{}][{}] Going to aggregate data", key, ts, aggregation);
 | 
			
		||||
    if (rsList == null || rsList.isEmpty()) {
 | 
			
		||||
        return Futures.immediateFuture(Optional.empty());
 | 
			
		||||
    }
 | 
			
		||||
    return Futures.transform(
 | 
			
		||||
        Futures.allAsList(
 | 
			
		||||
            rsList.stream().map(rs -> ResultSetUtils.allRows(rs, this.executor))
 | 
			
		||||
            rsList.stream().map(rs -> rs.allRows(this.executor))
 | 
			
		||||
                .collect(Collectors.toList())),
 | 
			
		||||
        rowsList -> {
 | 
			
		||||
            try {
 | 
			
		||||
 | 
			
		||||
@ -52,6 +52,7 @@ import org.thingsboard.server.common.data.kv.StringDataEntry;
 | 
			
		||||
import org.thingsboard.server.common.data.kv.TsKvEntry;
 | 
			
		||||
import org.thingsboard.server.dao.model.ModelConstants;
 | 
			
		||||
import org.thingsboard.server.dao.nosql.CassandraAbstractAsyncDao;
 | 
			
		||||
import org.thingsboard.server.dao.nosql.TbResultSet;
 | 
			
		||||
import org.thingsboard.server.dao.nosql.TbResultSetFuture;
 | 
			
		||||
import org.thingsboard.server.dao.util.NoSqlTsDao;
 | 
			
		||||
 | 
			
		||||
@ -238,14 +239,14 @@ public class CassandraBaseTimeseriesDao extends CassandraAbstractAsyncDao implem
 | 
			
		||||
 | 
			
		||||
            BoundStatement stmt = stmtBuilder.build();
 | 
			
		||||
 | 
			
		||||
            Futures.addCallback(executeAsyncRead(tenantId, stmt), new FutureCallback<AsyncResultSet>() {
 | 
			
		||||
            Futures.addCallback(executeAsyncRead(tenantId, stmt), new FutureCallback<TbResultSet>() {
 | 
			
		||||
                @Override
 | 
			
		||||
                public void onSuccess(@Nullable AsyncResultSet result) {
 | 
			
		||||
                public void onSuccess(@Nullable TbResultSet result) {
 | 
			
		||||
                    if (result == null) {
 | 
			
		||||
                        cursor.addData(convertResultToTsKvEntryList(Collections.emptyList()));
 | 
			
		||||
                        findAllAsyncSequentiallyWithLimit(tenantId, cursor, resultFuture);
 | 
			
		||||
                    } else {
 | 
			
		||||
                        Futures.addCallback(allRows(result), new FutureCallback<List<Row>>() {
 | 
			
		||||
                        Futures.addCallback(result.allRows(readResultsProcessingExecutor), new FutureCallback<List<Row>>() {
 | 
			
		||||
 | 
			
		||||
                            @Override
 | 
			
		||||
                            public void onSuccess(@Nullable List<Row> result) {
 | 
			
		||||
@ -278,21 +279,21 @@ public class CassandraBaseTimeseriesDao extends CassandraAbstractAsyncDao implem
 | 
			
		||||
        final long endTs = query.getEndTs();
 | 
			
		||||
        final long ts = startTs + (endTs - startTs) / 2;
 | 
			
		||||
        ListenableFuture<List<Long>> partitionsListFuture = getPartitionsFuture(tenantId, query, entityId, minPartition, maxPartition);
 | 
			
		||||
        ListenableFuture<List<AsyncResultSet>> aggregationChunks = Futures.transformAsync(partitionsListFuture,
 | 
			
		||||
        ListenableFuture<List<TbResultSet>> aggregationChunks = Futures.transformAsync(partitionsListFuture,
 | 
			
		||||
                getFetchChunksAsyncFunction(tenantId, entityId, key, aggregation, startTs, endTs), readResultsProcessingExecutor);
 | 
			
		||||
 | 
			
		||||
        return Futures.transformAsync(aggregationChunks, new AggregatePartitionsFunction(aggregation, key, ts, readResultsProcessingExecutor), readResultsProcessingExecutor);
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    private AsyncFunction<AsyncResultSet, List<Long>> getPartitionsArrayFunction() {
 | 
			
		||||
    private AsyncFunction<TbResultSet, List<Long>> getPartitionsArrayFunction() {
 | 
			
		||||
        return rs ->
 | 
			
		||||
            Futures.transform(allRows(rs), rows ->
 | 
			
		||||
            Futures.transform(rs.allRows(readResultsProcessingExecutor), rows ->
 | 
			
		||||
                rows.stream()
 | 
			
		||||
                .map(row -> row.getLong(ModelConstants.PARTITION_COLUMN)).collect(Collectors.toList()),
 | 
			
		||||
                    readResultsProcessingExecutor);
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    private AsyncFunction<List<Long>, List<AsyncResultSet>> getFetchChunksAsyncFunction(TenantId tenantId, EntityId entityId, String key, Aggregation aggregation, long startTs, long endTs) {
 | 
			
		||||
    private AsyncFunction<List<Long>, List<TbResultSet>> getFetchChunksAsyncFunction(TenantId tenantId, EntityId entityId, String key, Aggregation aggregation, long startTs, long endTs) {
 | 
			
		||||
        return partitions -> {
 | 
			
		||||
            try {
 | 
			
		||||
                PreparedStatement proto = getFetchStmt(aggregation, DESC_ORDER);
 | 
			
		||||
@ -684,8 +685,8 @@ public class CassandraBaseTimeseriesDao extends CassandraAbstractAsyncDao implem
 | 
			
		||||
        return deletePartitionStmt;
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    private ListenableFuture<List<TsKvEntry>> convertAsyncResultSetToTsKvEntryList(AsyncResultSet rs) {
 | 
			
		||||
        return Futures.transform(this.allRows(rs),
 | 
			
		||||
    private ListenableFuture<List<TsKvEntry>> convertAsyncResultSetToTsKvEntryList(TbResultSet rs) {
 | 
			
		||||
        return Futures.transform(rs.allRows(readResultsProcessingExecutor),
 | 
			
		||||
                rows -> this.convertResultToTsKvEntryList(rows), readResultsProcessingExecutor);
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
@ -8,7 +8,7 @@ cassandra.ssl=false
 | 
			
		||||
 | 
			
		||||
cassandra.jmx=false
 | 
			
		||||
 | 
			
		||||
cassandra.metrics=true
 | 
			
		||||
cassandra.metrics=false
 | 
			
		||||
 | 
			
		||||
cassandra.compression=none
 | 
			
		||||
 | 
			
		||||
@ -60,4 +60,4 @@ cassandra.query.tenant_rate_limits.enabled=false
 | 
			
		||||
cassandra.query.tenant_rate_limits.configuration=5000:1,100000:60
 | 
			
		||||
cassandra.query.tenant_rate_limits.print_tenant_names=false
 | 
			
		||||
 | 
			
		||||
service.type=monolith
 | 
			
		||||
service.type=monolith
 | 
			
		||||
 | 
			
		||||
		Loading…
	
	
			
			x
			
			
		
	
		Reference in New Issue
	
	Block a user