Browse Source

modify block rollback

tags/1.1.2^2^2
zhangshuang 5 years ago
parent
commit
7469f5802d
8 changed files with 416 additions and 54 deletions
  1. +32
    -49
      source/consensus/consensus-bftsmart/src/main/java/com/jd/blockchain/consensus/bftsmart/service/BftsmartNodeServer.java
  2. +8
    -0
      source/consensus/consensus-framework/src/main/java/com/jd/blockchain/consensus/service/MessageHandle.java
  3. +4
    -0
      source/ledger/ledger-core/src/main/java/com/jd/blockchain/ledger/core/LedgerRepositoryImpl.java
  4. +8
    -0
      source/ledger/ledger-core/src/main/java/com/jd/blockchain/ledger/core/TransactionBatchProcessor.java
  5. +10
    -0
      source/ledger/ledger-core/src/main/java/com/jd/blockchain/ledger/core/TransactionEngineImpl.java
  6. +24
    -5
      source/peer/src/main/java/com/jd/blockchain/peer/consensus/ConsensusMessageDispatcher.java
  7. +97
    -0
      source/sdk/sdk-samples/src/test/java/test/com/jd/blockchain/sdk/test/SDK_GateWay_Invalid_Signer_Test_.java
  8. +233
    -0
      source/test/test-integration/src/test/java/test/com/jd/blockchain/intgr/IntegrationTestStorageErrorRollBack.java

+ 32
- 49
source/consensus/consensus-bftsmart/src/main/java/com/jd/blockchain/consensus/bftsmart/service/BftsmartNodeServer.java View File

@@ -358,60 +358,60 @@ public class BftsmartNodeServer extends DefaultRecoverable implements NodeServer
* Used by consensus write phase, pre compute new block hash
*/
public BatchAppResultImpl preComputeAppHash(byte[][] commands) {
String batchId = messageHandle.beginBatch(realmName);
List<AsyncFuture<byte[]>> asyncFutureLinkedList = new ArrayList<>(commands.length);
List<byte[]> responseLinkedList = new ArrayList<>();
StateSnapshot stateSnapshot = null;
BatchAppResultImpl result;
StateSnapshot newStateSnapshot = null;
StateSnapshot preStateSnapshot = null;
BatchAppResultImpl result = null;
String batchId = null;
int msgId = 0;
boolean isOK = true;
TransactionState transactionState = TransactionState.IGNORED_BY_BLOCK_FULL_ROLLBACK;

try {

batchId = messageHandle.beginBatch(realmName);

preStateSnapshot = messageHandle.getStateSnapshot(realmName);

if (preStateSnapshot == null) {
System.out.println("prev state snapshot is null");
}

// System.out.println("last hash = "+preStateSnapshot.getSnapshot());
System.out.println("last height = "+preStateSnapshot.getId());

for (int i = 0; i < commands.length; i++) {
byte[] txContent = commands[i];
AsyncFuture<byte[]> asyncFuture = messageHandle.processOrdered(msgId++, txContent, realmName, batchId);
asyncFutureLinkedList.add(asyncFuture);
}
stateSnapshot = messageHandle.completeBatch(realmName, batchId);
} catch (Exception e) {
LOGGER.error("Error occurred while processing ordered messages or complete batch! --" + e.getMessage(), e);
messageHandle.rollbackBatch(realmName, batchId, TransactionState.IGNORED_BY_CONSENSUS_PHASE_PRECOMPUTE_ROLLBACK.CODE);
isOK = false;
}

if (isOK) {

byte[] blockHashBytes = stateSnapshot.getSnapshot();
newStateSnapshot = messageHandle.completeBatch(realmName, batchId);
// System.out.println("new hash = "+newStateSnapshot.getSnapshot());
System.out.println("new height = "+newStateSnapshot.getId());

for (int i = 0; i < asyncFutureLinkedList.size(); i++) {
responseLinkedList.add(asyncFutureLinkedList.get(i).get());
}

result = new BatchAppResultImpl(responseLinkedList, blockHashBytes, batchId);
result = new BatchAppResultImpl(responseLinkedList, newStateSnapshot.getSnapshot(), batchId);
result.setErrorCode((byte) 0);

return result;
} else {
} catch (Exception e) {
LOGGER.error("Error occurred while pre compute app! --" + e.getMessage(), e);
// messageHandle.rollbackBatch(realmName, batchId, TransactionState.IGNORED_BY_BLOCK_FULL_ROLLBACK.CODE);
for (int i = 0; i < commands.length; i++) {
responseLinkedList.add(createAppResponse(commands[i],transactionState));
responseLinkedList.add(createAppResponse(commands[i],TransactionState.IGNORED_BY_BLOCK_FULL_ROLLBACK));
}

Random random = new Random();
byte[] rand = new byte[4];
random.nextBytes(rand);

result = new BatchAppResultImpl(responseLinkedList, rand, batchId);
result = new BatchAppResultImpl(responseLinkedList,preStateSnapshot.getSnapshot(), batchId);
result.setErrorCode((byte) 1);

return result;
}

return result;
}

// Block full rollback responses, generated in pre compute phase, due to tx fail
public byte[] createAppResponse(byte[] command, TransactionState transactionState) {
// Block full rollback responses, generated in pre compute phase, due to tx exception
private byte[] createAppResponse(byte[] command, TransactionState transactionState) {
TransactionRequest txRequest = BinaryProtocol.decode(command);

TxResponseMessage resp = new TxResponseMessage(txRequest.getTransactionContent().getHash());
@@ -421,25 +421,8 @@ public class BftsmartNodeServer extends DefaultRecoverable implements NodeServer
return BinaryProtocol.encode(resp, TransactionResponse.class);
}



//Pre compute block hash values are inconsistent, update batch messages to new state
public List<byte[]> preCompInconsistentAppResps(List<byte[]> asyncResponseLinkedList) {
List<byte[]> updatedResponses = new ArrayList<>();

for(int i = 0; i < asyncResponseLinkedList.size(); i++) {
TransactionResponse txResponse = BinaryProtocol.decode(asyncResponseLinkedList.get(i));
TxResponseMessage resp = new TxResponseMessage(txResponse.getContentHash());
resp.setExecutionState(TransactionState.IGNORED_BY_CONSENSUS_PHASE_PRECOMPUTE_ROLLBACK);
updatedResponses.add(BinaryProtocol.encode(resp, TransactionResponse.class));
}

return updatedResponses;
}


//Consensus accept phase will terminate, pre compute commit exception occurs, update batch messages execute state to block full rollback
public List<byte[]> blockRollbackAppResps(List<byte[]> asyncResponseLinkedList) {
//update batch messages to block full rollback state
public List<byte[]> updateAppResponses(List<byte[]> asyncResponseLinkedList) {
List<byte[]> updatedResponses = new ArrayList<>();

for(int i = 0; i < asyncResponseLinkedList.size(); i++) {
@@ -472,7 +455,7 @@ public class BftsmartNodeServer extends DefaultRecoverable implements NodeServer
*
*/
public void preComputeAppRollback(String batchId) {
messageHandle.rollbackBatch(realmName, batchId, TransactionState.IGNORED_BY_CONSENSUS_PHASE_PRECOMPUTE_ROLLBACK.CODE);
messageHandle.rollbackBatch(realmName, batchId, TransactionState.IGNORED_BY_BLOCK_FULL_ROLLBACK.CODE);
LOGGER.debug("Rollback of operations that cause inconsistencies in the ledger");
}



+ 8
- 0
source/consensus/consensus-framework/src/main/java/com/jd/blockchain/consensus/service/MessageHandle.java View File

@@ -59,4 +59,12 @@ public interface MessageHandle {
*/
AsyncFuture<byte[]> processUnordered(byte[] message);

/**
* 获得当前最新区块的状态快照
*
* @param realmName
* @return 最新区块的状态快照
*/
StateSnapshot getStateSnapshot(String realmName);

}

+ 4
- 0
source/ledger/ledger-core/src/main/java/com/jd/blockchain/ledger/core/LedgerRepositoryImpl.java View File

@@ -374,6 +374,10 @@ class LedgerRepositoryImpl implements LedgerRepository {
return new LedgerDataset(adminDataset, userAccountSet, dataAccountSet, contractAccountSet, true);
}

public synchronized void resetNextBlockEditor() {
this.nextBlockEditor = null;
}

@Override
public synchronized LedgerEditor createNextBlock() {
if (closed) {


+ 8
- 0
source/ledger/ledger-core/src/main/java/com/jd/blockchain/ledger/core/TransactionBatchProcessor.java View File

@@ -39,6 +39,14 @@ public class TransactionBatchProcessor implements TransactionBatchProcess {

private TransactionBatchResult batchResult;

public byte[] getPrevLatestBlockHash() {
return ledger.getLatestBlockHash().toBytes();
}

public long getPreLatestBlockHeight() {
return ledger.getLatestBlockHeight();
}

public HashDigest getLedgerHash() {
return ledger.getHash();
}


+ 10
- 0
source/ledger/ledger-core/src/main/java/com/jd/blockchain/ledger/core/TransactionEngineImpl.java View File

@@ -48,6 +48,16 @@ public class TransactionEngineImpl implements TransactionEngine {
return batchs.get(ledgerHash);
}

public void freeBatch(HashDigest ledgerHash) {
finishBatch(ledgerHash);
}

public void resetNewBlockEditor(HashDigest ledgerHash) {

LedgerRepository ledgerRepo = ledgerService.getLedger(ledgerHash);
((LedgerRepositoryImpl)ledgerRepo).resetNextBlockEditor();
}

private void finishBatch(HashDigest ledgerHash) {
batchs.remove(ledgerHash);
}


+ 24
- 5
source/peer/src/main/java/com/jd/blockchain/peer/consensus/ConsensusMessageDispatcher.java View File

@@ -6,6 +6,8 @@ import com.jd.blockchain.ledger.OperationResult;
import com.jd.blockchain.ledger.TransactionRequest;
import com.jd.blockchain.ledger.TransactionResponse;
import com.jd.blockchain.ledger.TransactionState;
import com.jd.blockchain.ledger.core.TransactionBatchProcessor;
import com.jd.blockchain.ledger.core.TransactionEngineImpl;
import com.jd.blockchain.service.TransactionBatchProcess;
import com.jd.blockchain.service.TransactionBatchResultHandle;
import com.jd.blockchain.service.TransactionEngine;
@@ -19,6 +21,7 @@ import com.jd.blockchain.consensus.service.MessageHandle;
import com.jd.blockchain.consensus.service.StateSnapshot;
import com.jd.blockchain.crypto.HashDigest;

import javax.swing.plaf.nimbus.State;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
@@ -61,6 +64,17 @@ public class ConsensusMessageDispatcher implements MessageHandle {
return realmProcessor.newBatchId();
}

@Override
public StateSnapshot getStateSnapshot(String realmName) {
RealmProcessor realmProcessor = realmProcessorMap.get(realmName);
if (realmProcessor == null) {
throw new IllegalArgumentException("RealmName is not init!");
}

return realmProcessor.getStateSnapshot();

}

@Override
public AsyncFuture<byte[]> processOrdered(int messageId, byte[] message, String realmName, String batchId) {
// TODO 要求messageId在同一个批次不重复,但目前暂不验证
@@ -191,6 +205,10 @@ public class ConsensusMessageDispatcher implements MessageHandle {
return currBatchId;
}

public StateSnapshot getStateSnapshot() {
return new BlockStateSnapshot(((TransactionBatchProcessor)getTxBatchProcess()).getPreLatestBlockHeight(), ((TransactionBatchProcessor)getTxBatchProcess()).getPrevLatestBlockHash());
}

public AsyncFuture<byte[]> schedule(TransactionRequest txRequest) {
CompletableAsyncFuture<byte[]> asyncTxResult = new CompletableAsyncFuture<>();
TransactionResponse resp = getTxBatchProcess().schedule(txRequest);
@@ -243,16 +261,17 @@ public class ConsensusMessageDispatcher implements MessageHandle {
}

public void rollback(int reasonCode) {

if (batchResultHandle == null) {
return;
}
realmLock.lock();
try {
batchResultHandle.cancel(TransactionState.valueOf((byte)reasonCode));
if (batchResultHandle != null) {
batchResultHandle.cancel(TransactionState.valueOf((byte)reasonCode));
}
currBatchId = null;
txResponseMap = null;
txBatchProcess = null;
batchResultHandle = null;
((TransactionEngineImpl) (txEngine)).freeBatch(ledgerHash);
((TransactionEngineImpl) (txEngine)).resetNewBlockEditor(ledgerHash);
} finally {
realmLock.unlock();
}


+ 97
- 0
source/sdk/sdk-samples/src/test/java/test/com/jd/blockchain/sdk/test/SDK_GateWay_Invalid_Signer_Test_.java View File

@@ -0,0 +1,97 @@
/**
* Copyright: Copyright 2016-2020 JD.COM All Right Reserved
* FileName: test.com.jd.blockchain.sdk.test.SDK_GateWay_InsertData_Test
* Author: shaozhuguang
* Department: 区块链研发部
* Date: 2018/9/4 上午11:06
* Description: 插入数据测试
*/
package test.com.jd.blockchain.sdk.test;

import com.jd.blockchain.binaryproto.DataContractRegistry;
import com.jd.blockchain.crypto.*;
import com.jd.blockchain.ledger.*;
import com.jd.blockchain.sdk.BlockchainService;
import com.jd.blockchain.sdk.client.GatewayServiceFactory;
import com.jd.blockchain.transaction.TxResponseMessage;
import org.junit.Before;
import org.junit.Test;

import static org.junit.Assert.assertTrue;

/**
* 注册具有无效签名的用户账户
* @author zhangshuang
* @create 2019/12/6
* @since 1.0.0
*/

public class SDK_GateWay_Invalid_Signer_Test_ {

private PrivKey privKey;
private PubKey pubKey;

private BlockchainKeypair CLIENT_CERT = null;

private String GATEWAY_IPADDR = null;

private int GATEWAY_PORT;

private boolean SECURE;

private BlockchainService service;

public static final String PASSWORD = "abc";

public static final String PUB_KEYS = "3snPdw7i7Pb3B5AxpSXy6YVruvftugNQ7rB7k2KWukhBwKQhFBFagT";
public static final String PRIV_KEYS = "177gjtSgSdUF3LwRFGhzbpZZxmXXChsnwbuuLCG1V9KYfVuuxLwXGmZCp5FGUvsenhwBQLV";

@Before
public void init() {

privKey = KeyGenUtils.decodePrivKeyWithRawPassword(PRIV_KEYS, PASSWORD);
pubKey = KeyGenUtils.decodePubKey(PUB_KEYS);

CLIENT_CERT = new BlockchainKeypair(SDK_GateWay_KeyPair_Para.pubKey0, SDK_GateWay_KeyPair_Para.privkey0);
GATEWAY_IPADDR = "localhost";
GATEWAY_PORT = 11000;
SECURE = false;
GatewayServiceFactory serviceFactory = GatewayServiceFactory.connect(GATEWAY_IPADDR, GATEWAY_PORT, SECURE,
CLIENT_CERT);
service = serviceFactory.getBlockchainService();

DataContractRegistry.register(TransactionContent.class);
DataContractRegistry.register(TransactionContentBody.class);
DataContractRegistry.register(TransactionRequest.class);
DataContractRegistry.register(NodeRequest.class);
DataContractRegistry.register(EndpointRequest.class);
DataContractRegistry.register(TransactionResponse.class);
}

@Test
public void registerUser_Test() {
HashDigest[] ledgerHashs = service.getLedgerHashs();
// 在本地定义注册账号的 TX;
TransactionTemplate txTemp = service.newTransaction(ledgerHashs[0]);

//existed signer
AsymmetricKeypair keyPair = new BlockchainKeypair(pubKey, privKey);

BlockchainKeypair user = BlockchainKeyGenerator.getInstance().generate();

// 注册
txTemp.users().register(user.getIdentity());

// TX 准备就绪;
PreparedTransaction prepTx = txTemp.prepare();

// 使用私钥进行签名;
prepTx.sign(keyPair);

// 提交交易;
TransactionResponse transactionResponse = prepTx.commit();

assertTrue(transactionResponse.getExecutionState().CODE == TransactionState.IGNORED_BY_BLOCK_FULL_ROLLBACK.CODE);

}
}

+ 233
- 0
source/test/test-integration/src/test/java/test/com/jd/blockchain/intgr/IntegrationTestStorageErrorRollBack.java View File

@@ -0,0 +1,233 @@
//package test.com.jd.blockchain.intgr;
//
//import com.jd.blockchain.consensus.ConsensusProviders;
//import com.jd.blockchain.consensus.bftsmart.BftsmartConsensusSettings;
//import com.jd.blockchain.crypto.*;
//import com.jd.blockchain.gateway.GatewayConfigProperties;
//import com.jd.blockchain.ledger.BlockRollbackException;
//import com.jd.blockchain.ledger.BlockchainKeypair;
//import com.jd.blockchain.ledger.core.LedgerManager;
//import com.jd.blockchain.ledger.core.LedgerQuery;
//import com.jd.blockchain.sdk.BlockchainService;
//import com.jd.blockchain.sdk.client.GatewayServiceFactory;
//import com.jd.blockchain.storage.service.DbConnection;
//import com.jd.blockchain.storage.service.DbConnectionFactory;
//import com.jd.blockchain.storage.service.ExPolicyKVStorage;
//import com.jd.blockchain.storage.service.KVStorageService;
//import com.jd.blockchain.storage.service.utils.MemoryKVStorage;
//import com.jd.blockchain.tools.initializer.LedgerBindingConfig;
//import com.jd.blockchain.utils.Bytes;
//import com.jd.blockchain.utils.concurrent.ThreadInvoker;
//import org.junit.Test;
//import org.mockito.Mockito;
//import org.mockito.invocation.InvocationOnMock;
//import org.mockito.stubbing.Answer;
//import test.com.jd.blockchain.intgr.initializer.LedgerInitializeTest;
//import test.com.jd.blockchain.intgr.initializer.LedgerInitializeWeb4Nodes;
//
//import java.lang.reflect.Method;
//import java.util.concurrent.CountDownLatch;
//import java.util.concurrent.ExecutorService;
//import java.util.concurrent.Executors;
//
//import static org.mockito.Matchers.any;
//import static org.mockito.Matchers.anyLong;
//import static org.mockito.Mockito.doCallRealMethod;
//import static org.mockito.Mockito.doThrow;
//import static test.com.jd.blockchain.intgr.IntegrationBase.*;
//
//public class IntegrationTestStorageErrorRollBack {
//
// private static final boolean isRegisterUser = true;
//
// private static final boolean isRegisterDataAccount = false;
//
// private static final boolean isWriteKv = false;
//
// private static final String DB_TYPE_MEM = "mem";
//
// public static final String BFTSMART_PROVIDER = "com.jd.blockchain.consensus.bftsmart.BftsmartConsensusProvider";
//
// public boolean isRollBack = false;
//
// @Test
// public void test4Memory() {
// test(LedgerInitConsensusConfig.bftsmartProvider, DB_TYPE_MEM, LedgerInitConsensusConfig.memConnectionStrings);
// }
//
// public void test(String[] providers, String dbType, String[] dbConnections) {
//
//
// final ExecutorService sendReqExecutors = Executors.newFixedThreadPool(20);
//
// // 内存账本初始化
// HashDigest ledgerHash = initLedger(dbConnections);
//
// System.out.println("---------------init OK-------------------");
//
// // 启动Peer节点
// PeerTestRunner[] peerNodes = peerNodeStart(ledgerHash, dbType);
//
// System.out.println("---------------peer start OK-------------------");
// String encodedBase58Pwd = KeyGenUtils.encodePasswordAsBase58(LedgerInitializeTest.PASSWORD);
//
// GatewayConfigProperties.KeyPairConfig gwkey0 = new GatewayConfigProperties.KeyPairConfig();
// gwkey0.setPubKeyValue(IntegrationBase.PUB_KEYS[0]);
// gwkey0.setPrivKeyValue(IntegrationBase.PRIV_KEYS[0]);
// gwkey0.setPrivKeyPassword(encodedBase58Pwd);
// GatewayTestRunner gateway = new GatewayTestRunner("127.0.0.1", 11000, gwkey0,
// peerNodes[0].getServiceAddress(), providers,null);
//
// ThreadInvoker.AsyncCallback<Object> gwStarting = gateway.start();
//
// gwStarting.waitReturn();
//
// LedgerQuery[] ledgers = new LedgerQuery[peerNodes.length];
// LedgerManager[] ledgerManagers = new LedgerManager[peerNodes.length];
// LedgerBindingConfig[] ledgerBindingConfigs = new LedgerBindingConfig[peerNodes.length];
// DbConnection[] connections = new DbConnection[peerNodes.length];
// MemoryKVStorage[] storageMocks = new MemoryKVStorage[peerNodes.length];
// for (int i = 0; i < peerNodes.length; i++) {
// ledgerManagers[i] = new LedgerManager();
// ledgerBindingConfigs[i] = peerNodes[i].getLedgerBindingConfig();
// connections[i] = peerNodes[i].getDBConnectionFactory().connect(ledgerBindingConfigs[i].getLedger(ledgerHash).getDbConnection().getUri());
// System.out.printf("StorageService[%s] -> %s \r\n", i, connections[i].getStorageService());
// storageMocks[i] = Mockito.spy((MemoryKVStorage)(connections[i].getStorageService()));
// ledgers[i] = ledgerManagers[i].register(ledgerHash, storageMocks[i]);
// }
//
// final MemoryKVStorage STORAGE_Mock = Mockito.mock(MemoryKVStorage.class);
//
// Answer<String> answers = new Answer() {
//
// @Override
// public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
//
// Method method = invocationOnMock.getMethod();
// if (method.getName().equalsIgnoreCase("set")) {
// Object arg2Obj = invocationOnMock.getArguments()[2];
// if (isRollBack) {
// if (arg2Obj instanceof ExPolicyKVStorage.ExPolicy) {
// return false;
// } else {
// return -1;
// }
// } else {
// if (arg2Obj instanceof ExPolicyKVStorage.ExPolicy) {
// return STORAG.set((Bytes) (invocationOnMock.getArguments()[0]), (byte[]) (invocationOnMock.getArguments()[1]), (ExPolicyKVStorage.ExPolicy) (arg2Obj));
// } else {
// return STORAGE.set((Bytes) (invocationOnMock.getArguments()[0]), (byte[]) (invocationOnMock.getArguments()[1]), (long) (arg2Obj));
// }
// }
// } else if ((method.getName().equalsIgnoreCase("get")) && (method.getParameterCount() == 1)) {
// return STORAGE.get((Bytes) (invocationOnMock.getArguments()[0]));
// } else if ((method.getName().equalsIgnoreCase("get")) && (method.getParameterCount() == 2)) {
// return STORAGE.get((Bytes) (invocationOnMock.getArguments()[0]), (long) (invocationOnMock.getArguments()[1]));
// } else if (method.getName().equalsIgnoreCase("getVersion")) {
// return STORAGE.getVersion((Bytes) (invocationOnMock.getArguments()[0]));
// } else if (method.getName().equalsIgnoreCase("getEntry")) {
// return STORAGE.get((Bytes) (invocationOnMock.getArguments()[0]), (long) (invocationOnMock.getArguments()[1]));
// } else if (method.getName().equalsIgnoreCase("exist")) {
// return STORAGE.get((Bytes) (invocationOnMock.getArguments()[0]));
// } else if (method.getName().equalsIgnoreCase("keySet")) {
// return STORAGE.keySet();
// } else if (method.getName().equalsIgnoreCase("getStorageKeySet")) {
// return STORAGE.getStorageKeySet();
// } else if (method.getName().equalsIgnoreCase("getValue")) {
// return STORAGE.getValue((Bytes) (invocationOnMock.getArguments()[0]));
// } else if (method.getName().equalsIgnoreCase("getStorageCount")) {
// return STORAGE.getStorageCount();
// } else if (method.getName().equalsIgnoreCase("getExPolicyKVStorage")) {
// return STORAGE.getExPolicyKVStorage();
// } else if (method.getName().equalsIgnoreCase("getVersioningKVStorage")) {
// return STORAGE.getVersioningKVStorage();
// }
//
// return null;
// }
// };
//
// when(STORAGE_Mock.set(any(), any(), anyLong())).thenAnswer(answers);
// when(STORAGE_Mock.set(any(), any(), any(ExPolicyKVStorage.ExPolicy.class))).thenAnswer(answers);
// when(STORAGE_Mock.get(any())).thenAnswer(answers);
// when(STORAGE_Mock.get(any(), anyLong())).thenAnswer(answers);
// when(STORAGE_Mock.getVersion(any())).thenAnswer(answers);
// when(STORAGE_Mock.getEntry(any(), anyLong())).thenAnswer(answers);
// when(STORAGE_Mock.exist(any())).thenAnswer(answers);
// when(STORAGE_Mock.keySet()).thenAnswer(answers);
// when(STORAGE_Mock.getStorageKeySet()).thenAnswer(answers);
// when(STORAGE_Mock.getValue(any())).thenAnswer(answers);
// when(STORAGE_Mock.getStorageCount()).thenAnswer(answers);
// when(STORAGE_Mock.getExPolicyKVStorage()).thenAnswer(answers);
// when(STORAGE_Mock.getVersioningKVStorage()).thenAnswer(answers);
//
//
// IntegrationBase.testConsistencyAmongNodes(ledgers);
//
// LedgerQuery ledgerRepository = ledgers[0];
//
// GatewayServiceFactory gwsrvFact = GatewayServiceFactory.connect(gateway.getServiceAddress());
//
// PrivKey privkey0 = KeyGenUtils.decodePrivKeyWithRawPassword(IntegrationBase.PRIV_KEYS[0], IntegrationBase.PASSWORD);
//
// PubKey pubKey0 = KeyGenUtils.decodePubKey(IntegrationBase.PUB_KEYS[0]);
//
// AsymmetricKeypair adminKey = new AsymmetricKeypair(pubKey0, privkey0);
//
// BlockchainService blockchainService = gwsrvFact.getBlockchainService();
//
// int size = 15;
// CountDownLatch countDownLatch = new CountDownLatch(size);
// if (isRegisterUser) {
// for (int i = 0; i < size; i++) {
// sendReqExecutors.execute(() -> {
//
// System.out.printf(" sdk execute time = %s threadId = %s \r\n", System.currentTimeMillis(), Thread.currentThread().getId());
// KeyPairResponse userResponse = IntegrationBase.testSDK_RegisterUser(adminKey, ledgerHash, blockchainService);
//
//// validKeyPair(userResponse, ledgerRepository, IntegrationBase.KeyPairType.USER);
// countDownLatch.countDown();
// });
// }
// }
// try {
// countDownLatch.await();
// } catch (InterruptedException e) {
// e.printStackTrace();
// }
//
// for (int i = 0; i < peerNodes.length; i++) {
// doCallRealMethod().when(storageMocks[i]).set(any(), any(), anyLong());
// }
//
// if (isRegisterDataAccount) {
// KeyPairResponse dataAccountResponse = IntegrationBase.testSDK_RegisterDataAccount(adminKey, ledgerHash, blockchainService);
//
// validKeyPair(dataAccountResponse, ledgerRepository, KeyPairType.DATAACCOUNT);
//
// if (isWriteKv) {
//
// for (int m = 0; m < 13; m++) {
// BlockchainKeypair da = dataAccountResponse.keyPair;
// KvResponse kvResponse = IntegrationBase.testSDK_InsertData(adminKey, ledgerHash, blockchainService, da.getAddress());
// validKvWrite(kvResponse, ledgerRepository, blockchainService);
// }
// }
// }
//
// try {
// System.out.println("----------------- Init Completed -----------------");
// Thread.sleep(Integer.MAX_VALUE);
// } catch (InterruptedException e) {
// e.printStackTrace();
// }
//
// IntegrationBase.testConsistencyAmongNodes(ledgers);
// }
// private HashDigest initLedger(String[] dbConnections) {
// LedgerInitializeWeb4Nodes ledgerInit = new LedgerInitializeWeb4Nodes();
// HashDigest ledgerHash = ledgerInit.testInitWith4Nodes(LedgerInitConsensusConfig.bftsmartConfig, dbConnections);
// System.out.printf("LedgerHash = %s \r\n", ledgerHash.toBase58());
// return ledgerHash;
// }
//}

Loading…
Cancel
Save