fix: import BNS v1 data during event replay (#1301)

* fix: import v1 data during replay

* fix: import names first, subdomains last

* feat: obtain genesis block data from tsv

* fix: v1 import tests

* fix: import route

* fix: api test

* fix: move to for await of

* docs: update README to reflect new replay
This commit is contained in:
Rafael Cárdenas
2022-08-31 08:37:43 -05:00
committed by GitHub
parent 763d99e8d3
commit bc59817aa9
11 changed files with 233 additions and 155 deletions

View File

@@ -98,19 +98,51 @@ For running offline mode set an environment variable `STACKS_API_MODE=offline`
## Event Replay
The stacks-node is only able to emit events live as they happen. This poses a problem in the scenario where the stacks-blockchain-api needs to
be upgraded and its database cannot be migrated to a new schema. One way to handle this upgrade is to wipe the stacks-blockchain-api's database
and stacks-node working directory, and re-sync from scratch.
The stacks-node is only able to emit events live as they happen. This poses a problem in the
scenario where the stacks-blockchain-api needs to be upgraded and its database cannot be migrated to
a new schema. One way to handle this upgrade is to wipe the stacks-blockchain-api's database and
stacks-node working directory, and re-sync from scratch.
Alternatively, an event-replay feature is available where the API records the HTTP POST requests from the stacks-node event emitter, then streams
these events back to itself. Essentially simulating a wipe & full re-sync, but much quicker.
Alternatively, an event-replay feature is available where the API records the HTTP POST requests
from the stacks-node event emitter, then streams these events back to itself. Essentially simulating
a wipe & full re-sync, but much quicker.
The feature can be used via program args. For example, if there are breaking changes in the API's sql schema, like adding a new column which requires
event's to be re-played, the following steps could be ran:
The feature can be used via program args. For example, if there are breaking changes in the API's
sql schema, like adding a new column which requires event's to be re-played, the following steps
could be ran:
### Event Replay Instructions
1. Ensure the API process is not running. When stopping the API, let the process exit gracefully so that any in-progress SQL writes can finish.
#### V1 BNS Data
**Optional but recommended** - If you want the V1 BNS data, there are going to be a few extra steps:
1. Download BNS data:
```shell
curl -L https://storage.googleapis.com/blockstack-v1-migration-data/export-data.tar.gz -o /stacks-node/bns/export-data.tar.gz
```
1. Extract it:
```shell
tar -xzvf ./bns/export-data.tar.gz -C /stacks-node/bns/
```
1. Each file in `./bns` will have a corresponding `sha256` value. To Verify, run a script like the
following to check the sha256sum:
```bash
for file in `ls /stacks-node/bns/* | grep -v sha256 | grep -v .tar.gz`; do
if [ $(sha256sum $file | awk {'print $1'}) == $(cat ${file}.sha256 ) ]; then
echo "sha256 Matched $file"
else
echo "sha256 Mismatch $file"
fi
done
```
1. Set the data's location as the value of `BNS_IMPORT_DIR` in your `.env` file.
#### Export and Import
1. Ensure the API process is not running. When stopping the API, let the process exit gracefully so
that any in-progress SQL writes can finish.
1. Export event data to disk with the `export-events` command:
```shell
@@ -119,19 +151,25 @@ event's to be re-played, the following steps could be ran:
1. Update to the new stacks-blockchain-api version.
1. Perform the event playback using the `import-events` command:
**WARNING**: This will **drop _all_ tables** from the configured Postgres database, including any tables not automatically added by the API.
**WARNING**: This will **drop _all_ tables** from the configured Postgres database, including any
tables not automatically added by the API.
```shell
node ./lib/index.js import-events --file /tmp/stacks-node-events.tsv --wipe-db --force
```
This command has two modes of operation, specified by the `--mode` option:
* `archival` (default): The process will import and ingest *all* blockchain events that have happened since the first block.
* `pruned`: The import process will ignore some prunable events (mempool, microblocks) until the import block height has reached `chain tip - 256` blocks. This saves a considerable amount of time during import, but sacrifices some historical data. You can use this mode if you're mostly interested in running an API that prioritizes real time information.
* `archival` (default): The process will import and ingest *all* blockchain events that have
happened since the first block.
* `pruned`: The import process will ignore some prunable events (mempool, microblocks) until the
import block height has reached `chain tip - 256` blocks. This saves a considerable amount of
time during import, but sacrifices some historical data. You can use this mode if you're mostly
interested in running an API that prioritizes real time information.
Alternatively, instead of performing the `export-events` command in step 1, an environmental variable can be set which enables events to be streamed to a file
as they are received, while the application is running normally. To enable this feature, set the `STACKS_EXPORT_EVENTS_FILE` env var to the file path where
events should be appended. Example:
Alternatively, instead of performing the `export-events` command in step 1, an environmental
variable can be set which enables events to be streamed to a file as they are received, while the
application is running normally. To enable this feature, set the `STACKS_EXPORT_EVENTS_FILE` env var
to the file path where events should be appended. Example:
```
STACKS_EXPORT_EVENTS_FILE=/tmp/stacks-node-events.tsv
```

View File

@@ -78,7 +78,7 @@ Since we'll need to create some files/dirs for persistent data we'll first creat
We'll be using:
```bash
$ mkdir -p ./stacks-node/{persistent-data/postgres,persistent-data/stacks-blockchain,bns,config}
$ mkdir -p ./stacks-node/{persistent-data/postgres,persistent-data/stacks-blockchain,config}
$ docker pull blockstack/stacks-blockchain-api \
&& docker pull blockstack/stacks-blockchain \
&& docker pull postgres:alpine
@@ -86,26 +86,6 @@ $ docker network create stacks-blockchain > /dev/null 2>&1
$ cd ./stacks-node
```
**Optional but recommended**: If you need the v1 BNS data, there are going to be a few extra steps.
1. Download the BNS data:
`curl -L https://storage.googleapis.com/blockstack-v1-migration-data/export-data.tar.gz -o ./bns/export-data.tar.gz`
2. Extract the data:
`tar -xzvf ./bns/export-data.tar.gz -C ./bns/`
3. Each file in `./bns` will have a corresponding `sha256` value.
To Verify, run a script like the following to check the sha256sum:
```bash
for file in `ls ./bns/* | grep -v sha256 | grep -v .tar.gz`; do
if [ $(sha256sum $file | awk {'print $1'}) == $(cat ${file}.sha256 ) ]; then
echo "sha256 Matched $file"
else
echo "sha256 Mismatch $file"
fi
done
```
## Postgres
The `postgres:alpine` image can be run with default settings, the only requirement is that a password Environment Variable is set for the `postgres` user: `POSTGRES_PASSWORD=postgres`
@@ -161,16 +141,9 @@ STACKS_BLOCKCHAIN_API_PORT=3999
STACKS_BLOCKCHAIN_API_HOST=0.0.0.0
STACKS_CORE_RPC_HOST=stacks-blockchain
STACKS_CORE_RPC_PORT=20443
BNS_IMPORT_DIR=/bns-data
API_DOCS_URL=https://docs.hiro.so/api
```
**Note** that here we are importing the bns data with the env var `BNS_IMPORT`.
To Disable this import, simply comment the line: `#BNS_IMPORT_DIR=/bns-data`
***If you leave this enabled***: please allow several minutes for the one-time import to complete before continuing.
The other Environment Variables to pay attention to:
- `PG_HOST`: Set this to your **postgres** instance. In this guide, we'll be using a container named `postgres`.
@@ -184,7 +157,6 @@ docker run -d --rm \
--name stacks-blockchain-api \
--net=stacks-blockchain \
--env-file $(pwd)/.env \
-v $(pwd)/bns:/bns-data \
-p 3700:3700 \
-p 3999:3999 \
blockstack/stacks-blockchain-api

View File

@@ -35,7 +35,7 @@ Since we'll need to create some files/dirs for persistent data,
we'll first create a base directory structure and set some permissions:
```bash
$ sudo mkdir -p /stacks-node/{persistent-data/stacks-blockchain,bns,config,binaries}
$ sudo mkdir -p /stacks-node/{persistent-data/stacks-blockchain,config,binaries}
$ sudo chown -R $(whoami) /stacks-node
$ cd /stacks-node
```
@@ -43,7 +43,7 @@ $ cd /stacks-node
## Install Requirements
```bash
$ PG_VERSION=12 \
$ PG_VERSION=14 \
&& NODE_VERSION=16 \
&& sudo apt-get update \
&& sudo apt-get install -y \
@@ -65,26 +65,6 @@ $ PG_VERSION=12 \
nodejs
```
**Optional but recommended** - If you want the V1 BNS data, there are going to be a few extra steps:
1. Download the BNS data:
`curl -L https://storage.googleapis.com/blockstack-v1-migration-data/export-data.tar.gz -o /stacks-node/bns/export-data.tar.gz`
2. Extract the data:
`tar -xzvf ./bns/export-data.tar.gz -C /stacks-node/bns/`
3. Each file in `./bns` will have a corresponding `sha256` value.
To Verify, run a script like the following to check the sha256sum:
```bash
for file in `ls /stacks-node/bns/* | grep -v sha256 | grep -v .tar.gz`; do
if [ $(sha256sum $file | awk {'print $1'}) == $(cat ${file}.sha256 ) ]; then
echo "sha256 Matched $file"
else
echo "sha256 Mismatch $file"
fi
done
```
## postgres
### postgres permissions
@@ -127,8 +107,6 @@ $ git clone https://github.com/hirosystems/stacks-blockchain-api /stacks-node/st
The stacks blockchain api requires several Environment Variables to be set in order to run properly.
To reduce complexity, we're going to create a `.env` file that we'll use for these env vars.
** Note: ** to enable BNS names, uncomment `BNS_IMPORT_DIR` in the below `.env` file.
Create a new file: `/stacks-node/stacks-blockchain-api/.env` with the following content:
```bash
@@ -148,7 +126,6 @@ STACKS_BLOCKCHAIN_API_PORT=3999
STACKS_BLOCKCHAIN_API_HOST=0.0.0.0
STACKS_CORE_RPC_HOST=localhost
STACKS_CORE_RPC_PORT=20443
#BNS_IMPORT_DIR=/stacks-node/bns
EOF
$ cd /stacks-node/stacks-blockchain-api && nohup node ./lib/index.js &
```

View File

@@ -439,7 +439,7 @@ export interface DataStoreAttachmentData {
blockHeight: number;
}
export interface DataStoreSubdomainBlockData {
export interface DataStoreBnsBlockData {
index_block_hash: string;
parent_index_block_hash: string;
microblock_hash: string;
@@ -449,7 +449,7 @@ export interface DataStoreSubdomainBlockData {
export interface DataStoreAttachmentSubdomainData {
attachment?: DataStoreAttachmentData;
blockData?: DataStoreSubdomainBlockData;
blockData?: DataStoreBnsBlockData;
subdomains?: DbBnsSubdomain[];
}

View File

@@ -101,7 +101,7 @@ import {
DbAssetEventTypeId,
DbTxGlobalStatus,
DataStoreAttachmentData,
DataStoreSubdomainBlockData,
DataStoreBnsBlockData,
DataStoreAttachmentSubdomainData,
} from './common';
import {
@@ -2175,7 +2175,7 @@ export class PgDataStore
);
let isCanonical = true;
let txIndex = -1;
const blockData: DataStoreSubdomainBlockData = {
const blockData: DataStoreBnsBlockData = {
index_block_hash: '',
parent_index_block_hash: '',
microblock_hash: '',
@@ -7222,7 +7222,7 @@ export class PgDataStore
// The `names` and `zonefiles` tables only track latest zonefile changes. We need to check
// `nft_custody` for the latest name owner, but only for names that were NOT imported from v1
// since they did not generate an NFT event for us to track.
if (nameZonefile.rows[0].registered_at !== 0) {
if (nameZonefile.rows[0].registered_at !== 1) {
let value: Buffer;
try {
value = bnsNameCV(name);
@@ -7427,7 +7427,7 @@ export class PgDataStore
names
WHERE
address = $1
AND registered_at = 0
AND registered_at = 1
AND canonical = TRUE
AND microblock_canonical = TRUE
`,

View File

@@ -3,7 +3,8 @@ import * as fs from 'fs';
import { cycleMigrations, dangerousDropAllTables, PgDataStore } from '../datastore/postgres-store';
import { startEventServer } from '../event-stream/event-server';
import { getApiConfiguredChainID, httpPostRequest, logger } from '../helpers';
import { findTsvBlockHeight, getDbBlockHeight } from './helpers';
import { findBnsGenesisBlockData, findTsvBlockHeight, getDbBlockHeight } from './helpers';
import { importV1BnsNames, importV1BnsSubdomains, importV1TokenOfferingData } from '../import-v1';
enum EventImportMode {
/**
@@ -107,6 +108,8 @@ export async function importEventsFromTsv(
if (eventImportMode === EventImportMode.pruned) {
console.log(`Ignoring all prunable events before block height: ${prunedBlockHeight}`);
}
// Look for the TSV's genesis block information for BNS import.
const tsvGenesisBlockData = await findBnsGenesisBlockData(resolvedFilePath);
const db = await PgDataStore.connect({
usageName: 'import-events',
@@ -122,6 +125,18 @@ export async function importEventsFromTsv(
httpLogLevel: 'debug',
});
await importV1TokenOfferingData(db);
// Import V1 BNS names first. Subdomains will be imported after TSV replay is finished in order to
// keep the size of the `subdomains` table small.
if (process.env.BNS_IMPORT_DIR) {
logger.info(`Using BNS export data from: ${process.env.BNS_IMPORT_DIR}`);
await importV1BnsNames(db, process.env.BNS_IMPORT_DIR, tsvGenesisBlockData);
} else {
logger.warn(`Notice: full BNS functionality requires 'BNS_IMPORT_DIR' to be set.`);
}
// Import TSV chain data
const readStream = fs.createReadStream(resolvedFilePath);
const rawEventsIterator = PgDataStore.getRawEventRequests(readStream, status => {
console.log(status);
@@ -163,6 +178,9 @@ export async function importEventsFromTsv(
}
}
await db.finishEventReplay();
if (process.env.BNS_IMPORT_DIR) {
await importV1BnsSubdomains(db, process.env.BNS_IMPORT_DIR, tsvGenesisBlockData);
}
console.log(`Event import and playback successful.`);
await eventServer.closeAsync();
await db.close();

View File

@@ -1,6 +1,15 @@
import * as fs from 'fs';
import * as readline from 'readline';
import { decodeTransaction, TxPayloadTypeID } from 'stacks-encoding-native-js';
import { DataStoreBnsBlockData } from '../datastore/common';
import { PgDataStore } from '../datastore/postgres-store';
import { ReverseFileStream } from './reverse-file-stream';
export type BnsGenesisBlock = DataStoreBnsBlockData & {
tx_id: string;
tx_index: number;
};
/**
* Traverse a TSV file in reverse to find the last received `/new_block` node message and return
* the `block_height` reported by that event. Even though the block produced by that event might
@@ -26,6 +35,47 @@ export async function findTsvBlockHeight(filePath: string): Promise<number> {
return blockHeight;
}
/**
* Traverse a TSV file to find the genesis block and extract its data so we can use it during V1 BNS
* import.
* @param filePath - TSV path
* @returns Genesis block data
*/
export async function findBnsGenesisBlockData(filePath: string): Promise<BnsGenesisBlock> {
const rl = readline.createInterface({
input: fs.createReadStream(filePath),
crlfDelay: Infinity,
});
for await (const line of rl) {
const columns = line.split('\t');
const eventName = columns[2];
if (eventName === '/new_block') {
const payload = JSON.parse(columns[3]);
// Look for block 1
if (payload.block_height === 1) {
for (const tx of payload.transactions) {
const decodedTx = decodeTransaction(tx.raw_tx);
// Look for the only token transfer transaction in the genesis block. This is the one
// that contains all the events, including all BNS name registrations.
if (decodedTx.payload.type_id === TxPayloadTypeID.TokenTransfer) {
rl.close();
return {
index_block_hash: payload.index_block_hash,
parent_index_block_hash: payload.parent_index_block_hash,
microblock_hash: payload.parent_microblock,
microblock_sequence: payload.parent_microblock_sequence,
microblock_canonical: true,
tx_id: decodedTx.tx_id,
tx_index: tx.tx_index,
};
}
}
}
}
}
throw new Error('BNS genesis block data not found');
}
/**
* Get the current block height from the DB. We won't use the `getChainTip` method since that
* adds some conversions from block hashes into strings that we're not interested in. We also can't

View File

@@ -7,7 +7,6 @@ import * as path from 'path';
import * as zlib from 'zlib';
import { bitcoinToStacksAddress } from 'stacks-encoding-native-js';
import * as split2 from 'split2';
import {
DbBnsName,
DbBnsNamespace,
@@ -24,15 +23,8 @@ import {
logger,
REPO_DIR,
} from '../helpers';
import { PoolClient } from 'pg';
const IMPORT_FILES = [
'chainstate.txt',
'name_zonefiles.txt',
'subdomains.csv',
'subdomain_zonefiles.txt',
];
import { BnsGenesisBlock } from '../event-replay/helpers';
const finished = util.promisify(stream.finished);
const pipeline = util.promisify(stream.pipeline);
@@ -87,20 +79,20 @@ class ChainProcessor extends stream.Writable {
namespace: Map<string, DbBnsNamespace>;
db: PgDataStore;
client: PoolClient;
emptyBlockData = {
index_block_hash: '',
parent_index_block_hash: '',
microblock_hash: '',
microblock_sequence: I32_MAX,
microblock_canonical: true,
} as const;
genesisBlock: BnsGenesisBlock;
constructor(client: PoolClient, db: PgDataStore, zhashes: Map<string, string>) {
constructor(
client: PoolClient,
db: PgDataStore,
zhashes: Map<string, string>,
genesisBlock: BnsGenesisBlock
) {
super();
this.zhashes = zhashes;
this.namespace = new Map();
this.client = client;
this.db = db;
this.genesisBlock = genesisBlock;
logger.info(`${this.tag}: importer starting`);
}
@@ -159,16 +151,16 @@ class ChainProcessor extends stream.Writable {
name: parts[0],
address: parts[1],
namespace_id: ns,
registered_at: 0,
registered_at: 1,
expire_block: namespace.lifetime,
zonefile: zonefile,
zonefile_hash: zonefileHash,
tx_id: '',
tx_index: 0,
tx_id: this.genesisBlock.tx_id,
tx_index: this.genesisBlock.tx_index,
canonical: true,
status: 'name-register',
};
await this.db.updateNames(this.client, this.emptyBlockData, obj);
await this.db.updateNames(this.client, this.genesisBlock, obj);
this.rowCount += 1;
if (obj.zonefile === '') {
logger.verbose(
@@ -182,20 +174,20 @@ class ChainProcessor extends stream.Writable {
const obj: DbBnsNamespace = {
namespace_id: parts[0],
address: parts[1],
reveal_block: 0,
ready_block: 0,
reveal_block: 1,
ready_block: 1,
buckets: parts[2],
base: BigInt(parts[3]),
coeff: BigInt(parts[4]),
nonalpha_discount: parseInt(parts[5], 10),
no_vowel_discount: parseInt(parts[6], 10),
lifetime: parseInt(parts[7], 10),
tx_id: '',
tx_index: 0,
tx_id: this.genesisBlock.tx_id,
tx_index: this.genesisBlock.tx_index,
canonical: true,
};
this.namespace.set(obj.namespace_id, obj);
await this.db.updateNamespaces(this.client, this.emptyBlockData, obj);
await this.db.updateNamespaces(this.client, this.genesisBlock, obj);
this.rowCount += 1;
}
}
@@ -239,9 +231,13 @@ function btcToStxAddress(btcAddress: string) {
}
class SubdomainTransform extends stream.Transform {
constructor() {
genesisBlock: BnsGenesisBlock;
constructor(genesisBlock: BnsGenesisBlock) {
super({ objectMode: true, highWaterMark: SUBDOMAIN_BATCH_SIZE });
this.genesisBlock = genesisBlock;
}
_transform(data: string, _encoding: string, callback: stream.TransformCallback) {
const parts = data.split(',');
if (parts[0] !== 'zonefile_hash') {
@@ -258,8 +254,8 @@ class SubdomainTransform extends stream.Transform {
fully_qualified_subdomain: parts[2],
owner: btcToStxAddress(parts[3]), //convert btc address to stx,
block_height: 1, // burn_block_height: parseInt(parts[4], 10)
tx_index: 0,
tx_id: '',
tx_index: this.genesisBlock.tx_index,
tx_id: this.genesisBlock.tx_id,
parent_zonefile_index: parseInt(parts[5], 10),
zonefile_offset: parseInt(parts[6], 10),
resolver: parts[7],
@@ -309,12 +305,12 @@ async function valid(fileName: string): Promise<boolean> {
return true;
}
async function* readSubdomains(importDir: string) {
async function* readSubdomains(importDir: string, genesisBlock: BnsGenesisBlock) {
const metaIter = asyncIterableToGenerator<DbBnsSubdomain>(
stream.pipeline(
fs.createReadStream(path.join(importDir, 'subdomains.csv')),
new LineReaderStream({ highWaterMark: SUBDOMAIN_BATCH_SIZE }),
new SubdomainTransform(),
new SubdomainTransform(genesisBlock),
error => {
if (error) {
console.error('Error reading subdomains.csv');
@@ -396,13 +392,7 @@ class StxVestingTransform extends stream.Transform {
}
}
export async function importV1BnsData(db: PgDataStore, importDir: string) {
const configState = await db.getConfigState();
if (configState.bns_names_onchain_imported && configState.bns_subdomains_imported) {
logger.verbose('Stacks 1.0 BNS data is already imported');
return;
}
async function validateBnsImportDir(importDir: string, importFiles: string[]) {
try {
const statResult = fs.statSync(importDir);
if (!statResult.isDirectory()) {
@@ -413,18 +403,29 @@ export async function importV1BnsData(db: PgDataStore, importDir: string) {
throw error;
}
logger.info('Stacks 1.0 BNS data import started');
logger.info(`Using BNS export data from: ${importDir}`);
// validate contents with their .sha256 files
// check if the files we need can be read
for (const fname of IMPORT_FILES) {
for (const fname of importFiles) {
if (!(await valid(path.join(importDir, fname)))) {
const errMsg = `Cannot read import file due to sha256 mismatch: ${fname}`;
logError(errMsg);
throw new Error(errMsg);
}
}
}
export async function importV1BnsNames(
db: PgDataStore,
importDir: string,
genesisBlock: BnsGenesisBlock
) {
const configState = await db.getConfigState();
if (configState.bns_names_onchain_imported) {
logger.verbose('Stacks 1.0 BNS names are already imported');
return;
}
await validateBnsImportDir(importDir, ['chainstate.txt', 'name_zonefiles.txt']);
logger.info('Stacks 1.0 BNS name import started');
const client = await db.pool.connect();
try {
@@ -433,26 +434,53 @@ export async function importV1BnsData(db: PgDataStore, importDir: string) {
await pipeline(
fs.createReadStream(path.join(importDir, 'chainstate.txt')),
new LineReaderStream({ highWaterMark: 100 }),
new ChainProcessor(client, db, zhashes)
new ChainProcessor(client, db, zhashes, genesisBlock)
);
const blockData = {
index_block_hash: '',
parent_index_block_hash: '',
microblock_hash: '',
microblock_sequence: I32_MAX,
microblock_canonical: true,
const updatedConfigState: DbConfigState = {
...configState,
bns_names_onchain_imported: true,
};
await db.updateConfigState(updatedConfigState, client);
await client.query('COMMIT');
} catch (error) {
await client.query('ROLLBACK');
throw error;
} finally {
client.release();
}
logger.info('Stacks 1.0 BNS name import completed');
}
export async function importV1BnsSubdomains(
db: PgDataStore,
importDir: string,
genesisBlock: BnsGenesisBlock
) {
const configState = await db.getConfigState();
if (configState.bns_subdomains_imported) {
logger.verbose('Stacks 1.0 BNS subdomains are already imported');
return;
}
await validateBnsImportDir(importDir, ['subdomains.csv', 'subdomain_zonefiles.txt']);
logger.info('Stacks 1.0 BNS subdomain import started');
const client = await db.pool.connect();
try {
await client.query('BEGIN');
let subdomainsImported = 0;
const subdomainIter = readSubdomains(importDir);
const subdomainIter = readSubdomains(importDir, genesisBlock);
for await (const subdomainBatch of asyncBatchIterate(
subdomainIter,
SUBDOMAIN_BATCH_SIZE,
false
)) {
await db.updateBatchSubdomains(client, [{ blockData, subdomains: subdomainBatch }]);
await db.updateBatchZonefiles(client, [{ blockData, subdomains: subdomainBatch }]);
await db.updateBatchSubdomains(client, [
{ blockData: genesisBlock, subdomains: subdomainBatch },
]);
await db.updateBatchZonefiles(client, [
{ blockData: genesisBlock, subdomains: subdomainBatch },
]);
subdomainsImported += subdomainBatch.length;
if (subdomainsImported % 10_000 === 0) {
logger.info(`Subdomains imported: ${subdomainsImported}`);
@@ -462,7 +490,6 @@ export async function importV1BnsData(db: PgDataStore, importDir: string) {
const updatedConfigState: DbConfigState = {
...configState,
bns_names_onchain_imported: true,
bns_subdomains_imported: true,
};
await db.updateConfigState(updatedConfigState, client);
@@ -474,7 +501,7 @@ export async function importV1BnsData(db: PgDataStore, importDir: string) {
client.release();
}
logger.info('Stacks 1.0 BNS data import completed');
logger.info('Stacks 1.0 BNS subdomain import completed');
}
/** A passthrough stream which hashes the data as it passes through. */

View File

@@ -18,7 +18,6 @@ import { startEventServer } from './event-stream/event-server';
import { StacksCoreRpcClient } from './core-rpc/client';
import { createServer as createPrometheusServer } from '@promster/server';
import { registerShutdownConfig } from './shutdown-handler';
import { importV1TokenOfferingData, importV1BnsData } from './import-v1';
import { OfflineDummyStore } from './datastore/offline-dummy-store';
import { Socket } from 'net';
import * as getopts from 'getopts';
@@ -123,23 +122,7 @@ async function init(): Promise<void> {
});
if (apiMode !== StacksApiMode.readOnly) {
if (db instanceof PgDataStore) {
if (isProdEnv) {
await importV1TokenOfferingData(db);
} else {
logger.warn(
`Notice: skipping token offering data import because of non-production NODE_ENV`
);
}
if (isProdEnv && !process.env.BNS_IMPORT_DIR) {
logger.warn(`Notice: full BNS functionality requires 'BNS_IMPORT_DIR' to be set.`);
} else if (process.env.BNS_IMPORT_DIR) {
await importV1BnsData(db, process.env.BNS_IMPORT_DIR);
}
}
const configuredChainID = getApiConfiguredChainID();
const eventServer = await startEventServer({
datastore: db,
chainId: configuredChainID,

View File

@@ -414,7 +414,7 @@ describe('BNS API tests', () => {
.build();
await db.update(block);
// Register another name in block 0 (imported from v1, so no nft_event produced)
// Register another name in block 1 (imported from v1, so no nft_event produced)
const dbName2: DbBnsName = {
name: 'imported.btc',
address: address,
@@ -422,7 +422,7 @@ describe('BNS API tests', () => {
expire_block: 10000,
zonefile: 'test-zone-file',
zonefile_hash: 'zonefileHash',
registered_at: 0,
registered_at: 1,
canonical: true,
tx_id: '',
tx_index: 0,

View File

@@ -5,15 +5,18 @@ import * as supertest from 'supertest';
import { startEventServer } from '../event-stream/event-server';
import { Server } from 'net';
import { ChainID } from '@stacks/transactions';
import { importV1BnsData } from '../import-v1';
import { importV1BnsNames, importV1BnsSubdomains } from '../import-v1';
import * as assert from 'assert';
import { TestBlockBuilder } from '../test-utils/test-builders';
import { DataStoreBlockUpdateData } from '../datastore/common';
import { BnsGenesisBlock } from '../event-replay/helpers';
describe('BNS V1 import', () => {
let db: PgDataStore;
let client: PoolClient;
let eventServer: Server;
let api: ApiServer;
let block: DataStoreBlockUpdateData;
beforeEach(async () => {
process.env.PG_DATABASE = 'postgres';
@@ -23,12 +26,22 @@ describe('BNS V1 import', () => {
eventServer = await startEventServer({ datastore: db, chainId: ChainID.Testnet, httpLogLevel: 'silly' });
api = await startApiServer({ datastore: db, chainId: ChainID.Testnet, httpLogLevel: 'silly' });
const block = new TestBlockBuilder().build();
block = new TestBlockBuilder().addTx().build();
await db.update(block);
});
test('v1-import', async () => {
await importV1BnsData(db, 'src/tests-bns/import-test-files');
const genesis: BnsGenesisBlock = {
index_block_hash: block.block.index_block_hash,
parent_index_block_hash: block.block.parent_index_block_hash,
microblock_canonical: true,
microblock_hash: block.block.parent_microblock_hash,
microblock_sequence: block.block.parent_microblock_sequence,
tx_id: block.txs[0].tx.tx_id,
tx_index: block.txs[0].tx.tx_index,
};
await importV1BnsNames(db, 'src/tests-bns/import-test-files', genesis);
await importV1BnsSubdomains(db, 'src/tests-bns/import-test-files', genesis);
// Names
const query1 = await supertest(api.server).get(`/v1/names/zumrai.id`);
@@ -37,8 +50,8 @@ describe('BNS V1 import', () => {
expect(query1.body).toEqual({
address: 'SP29EJ0SVM2TRZ3XGVTZPVTKF4SV1VMD8C0GA5SK5',
blockchain: 'stacks',
expire_block: 52595,
last_txid: '',
expire_block: 52596,
last_txid: '0x1234',
status: 'name-register',
zonefile:
'$ORIGIN zumrai.id\n$TTL 3600\n_http._tcp IN URI 10 1 "https://gaia.blockstack.org/hub/1EPno1VcdGx89ukN2we4iVpnFtkHzw8i5d/profile.json"\n\n',
@@ -121,7 +134,7 @@ describe('BNS V1 import', () => {
expect(query9.body).toEqual({
address: 'SP2S2F9TCAT43KEJT02YTG2NXVCPZXS1426T63D9H',
blockchain: 'stacks',
last_txid: '',
last_txid: '0x1234',
resolver: 'https://registrar.blockstack.org',
status: 'registered_subdomain',
zonefile: