diff --git a/api/clients/codecs/blob_codec.go b/api/clients/codecs/blob_codec.go index 94e19a132..5be519026 100644 --- a/api/clients/codecs/blob_codec.go +++ b/api/clients/codecs/blob_codec.go @@ -6,9 +6,9 @@ import ( type BlobEncodingVersion byte -// All blob encodings are IFFT'd before being dispersed const ( - // This minimal blob encoding includes a version byte, a length uint32, and 31 byte field element mapping. + // This minimal blob encoding contains a 32 byte header = [0x00, version byte, uint32 len of data, 0x00, 0x00,...] + // followed by the encoded data [0x00, 31 bytes of data, 0x00, 31 bytes of data,...] DefaultBlobEncoding BlobEncodingVersion = 0x0 ) @@ -30,6 +30,9 @@ func GenericDecodeBlob(data []byte) ([]byte, error) { if len(data) <= 32 { return nil, fmt.Errorf("data is not of length greater than 32 bytes: %d", len(data)) } + // version byte is stored in [1], because [0] is always 0 to ensure the codecBlobHeader is a valid bn254 element + // see https://github.com/Layr-Labs/eigenda/blob/master/api/clients/codecs/default_blob_codec.go#L21 + // TODO: we should prob be working over a struct with methods such as GetBlobEncodingVersion() to prevent index errors version := BlobEncodingVersion(data[1]) codec, err := BlobEncodingVersionToCodec(version) if err != nil { @@ -38,7 +41,7 @@ func GenericDecodeBlob(data []byte) ([]byte, error) { data, err = codec.DecodeBlob(data) if err != nil { - return nil, err + return nil, fmt.Errorf("unable to decode blob: %w", err) } return data, nil diff --git a/api/clients/config.go b/api/clients/config.go index 16d74742d..e674f8c08 100644 --- a/api/clients/config.go +++ b/api/clients/config.go @@ -2,28 +2,57 @@ package clients import ( "fmt" + "log" "time" "github.com/Layr-Labs/eigenda/api/clients/codecs" ) type EigenDAClientConfig struct { - // RPC is the HTTP provider URL for the Data Availability node. + // RPC is the HTTP provider URL for the EigenDA Disperser RPC string - // The total amount of time that the client will spend waiting for EigenDA to confirm a blob + // Timeout used when making dispersals to the EigenDA Disperser + // TODO: we should change this param as its name is quite confusing + ResponseTimeout time.Duration + + // The total amount of time that the client will spend waiting for EigenDA + // to confirm a blob after it has been dispersed + // Note that reasonable values for this field will depend on the value of WaitForFinalization. StatusQueryTimeout time.Duration // The amount of time to wait between status queries of a newly dispersed blob StatusQueryRetryInterval time.Duration - // The total amount of time that the client will waiting for a response from the EigenDA disperser - ResponseTimeout time.Duration + // The Ethereum RPC URL to use for querying the Ethereum blockchain. + // This is used to make sure that the blob has been confirmed on-chain. + // Only needed when WaitForConfirmationDepth > 0. + EthRpcUrl string + + // The address of the EigenDAServiceManager contract, used to make sure that the blob has been confirmed on-chain. + // Only needed when WaitForConfirmationDepth > 0. + SvcManagerAddr string + + // The number of Ethereum blocks to wait after the blob's batch has been included onchain, before returning from PutBlob calls. + // Only makes sense to wait for < 24 blocks (2 epochs). Otherwise, use WaitForFinalization instead. + // + // When WaitForFinalization is true, this field is ignored. + // + // If WaitForConfirmationDepth > 0, then EthRpcUrl and SvcManagerAddr must be set. + WaitForConfirmationDepth uint64 + + // If true, will wait for the blob to finalize, if false, will wait only for the blob to confirm. + WaitForFinalization bool // The quorum IDs to write blobs to using this client. Should not include default quorums 0 or 1. + // TODO: should we change this to core.QuorumID instead? https://github.com/Layr-Labs/eigenda/blob/style--improve-api-clients-comments/core/data.go#L18 CustomQuorumIDs []uint - // Signer private key in hex encoded format. This key should not be associated with an Ethereum address holding any funds. + // Signer private key in hex encoded format. This key is currently purely used for authn/authz on the disperser. + // For security, it should not be associated with an Ethereum address holding any funds. + // This might change once we introduce payments. + // OPTIONAL: this value is optional, and if set to "", will result in a read-only eigenDA client, + // that can retrieve blobs but cannot disperse blobs. SignerPrivateKeyHex string // Whether to disable TLS for an insecure connection when connecting to a local EigenDA disperser instance. @@ -37,12 +66,29 @@ type EigenDAClientConfig struct { // the commitment. With this mode disabled, you will need to supply the entire blob to perform a verification // that any part of the data matches the KZG commitment. DisablePointVerificationMode bool - - // If true, will wait for the blob to finalize, if false, will wait only for the blob to confirm. - WaitForFinalization bool } func (c *EigenDAClientConfig) CheckAndSetDefaults() error { + if c.WaitForFinalization { + if c.WaitForConfirmationDepth != 0 { + log.Println("Warning: WaitForFinalization is set to true, WaitForConfirmationDepth will be ignored") + } + } else { + if c.WaitForConfirmationDepth > 24 { + log.Printf("Warning: WaitForConfirmationDepth is set to %v > 24 (2 epochs == finality). Consider setting WaitForFinalization to true instead.\n", c.WaitForConfirmationDepth) + } + } + if c.WaitForConfirmationDepth > 0 { + if c.SvcManagerAddr == "" { + return fmt.Errorf("EigenDAClientConfig.SvcManagerAddr not set. Needed because WaitForConfirmationDepth > 0") + } + if c.EthRpcUrl == "" { + return fmt.Errorf("EigenDAClientConfig.EthRpcUrl not set. Needed because WaitForConfirmationDepth > 0") + } + } + if c.SvcManagerAddr == "" && c.WaitForConfirmationDepth > 0 { + return fmt.Errorf("EigenDAClientConfig.SvcManagerAddr not set. Needed because WaitForConfirmationDepth > 0") + } if c.StatusQueryRetryInterval == 0 { c.StatusQueryRetryInterval = 5 * time.Second } diff --git a/api/clients/disperser_client.go b/api/clients/disperser_client.go index 2c5e46a74..55c1de056 100644 --- a/api/clients/disperser_client.go +++ b/api/clients/disperser_client.go @@ -20,6 +20,9 @@ import ( type Config struct { Hostname string Port string + // BlobDispersal Timeouts for both authenticated and unauthenticated dispersals + // GetBlobStatus and RetrieveBlob timeouts are hardcoded to 60seconds + // TODO: do we want to add config timeouts for those separate requests? Timeout time.Duration UseSecureGrpcFlag bool } diff --git a/api/clients/eigenda_client.go b/api/clients/eigenda_client.go index cc7597f8c..0bd9ef2c6 100644 --- a/api/clients/eigenda_client.go +++ b/api/clients/eigenda_client.go @@ -1,32 +1,47 @@ package clients import ( + "bytes" "context" "encoding/base64" "encoding/hex" "fmt" + "math/big" "net" "time" + "github.com/ethereum/go-ethereum/accounts/abi/bind" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/ethclient" + "github.com/ethereum/go-ethereum/log" + "github.com/Layr-Labs/eigenda/api/clients/codecs" grpcdisperser "github.com/Layr-Labs/eigenda/api/grpc/disperser" + edasm "github.com/Layr-Labs/eigenda/contracts/bindings/EigenDAServiceManager" "github.com/Layr-Labs/eigenda/core" "github.com/Layr-Labs/eigenda/core/auth" "github.com/Layr-Labs/eigenda/disperser" - "github.com/ethereum/go-ethereum/log" ) +// IEigenDAClient is a wrapper around the DisperserClient interface which +// encodes blobs before dispersing them, and decodes them after retrieving them. type IEigenDAClient interface { GetBlob(ctx context.Context, batchHeaderHash []byte, blobIndex uint32) ([]byte, error) PutBlob(ctx context.Context, txData []byte) (*grpcdisperser.BlobInfo, error) GetCodec() codecs.BlobCodec } +// EigenDAClient is a wrapper around the DisperserClient which +// encodes blobs before dispersing them, and decodes them after retrieving them. type EigenDAClient struct { - Config EigenDAClientConfig - Log log.Logger - Client DisperserClient - Codec codecs.BlobCodec + // TODO: all of these should be private, to prevent users from using them directly, + // which breaks encapsulation and makes it hard for us to do refactors or changes + Config EigenDAClientConfig + Log log.Logger + Client DisperserClient + ethClient *ethclient.Client + edasmCaller *edasm.ContractEigenDAServiceManagerCaller + Codec codecs.BlobCodec } var _ IEigenDAClient = EigenDAClient{} @@ -37,6 +52,20 @@ func NewEigenDAClient(log log.Logger, config EigenDAClientConfig) (*EigenDAClien return nil, err } + var ethClient *ethclient.Client + var edasmCaller *edasm.ContractEigenDAServiceManagerCaller + if config.WaitForConfirmationDepth > 0 { + ethClient, err = ethclient.Dial(config.EthRpcUrl) + if err != nil { + return nil, fmt.Errorf("failed to dial ETH RPC node: %w", err) + } + edasmCaller, err = edasm.NewContractEigenDAServiceManagerCaller(common.HexToAddress(config.SvcManagerAddr), ethClient) + if err != nil { + return nil, fmt.Errorf("failed to create EigenDAServiceManagerCaller: %w", err) + } + + } + host, port, err := net.SplitHostPort(config.RPC) if err != nil { return nil, fmt.Errorf("failed to parse EigenDA RPC: %w", err) @@ -46,6 +75,7 @@ func NewEigenDAClient(log log.Logger, config EigenDAClientConfig) (*EigenDAClien if len(config.SignerPrivateKeyHex) == 64 { signer = auth.NewLocalBlobRequestSigner(config.SignerPrivateKeyHex) } else if len(config.SignerPrivateKeyHex) == 0 { + // noop signer is used when we need a read-only eigenda client signer = auth.NewLocalNoopSigner() } else { return nil, fmt.Errorf("invalid length for signer private key") @@ -67,13 +97,17 @@ func NewEigenDAClient(log log.Logger, config EigenDAClientConfig) (*EigenDAClien } return &EigenDAClient{ - Log: log, - Config: config, - Client: llClient, - Codec: codec, + Log: log, + Config: config, + Client: llClient, + ethClient: ethClient, + edasmCaller: edasmCaller, + Codec: codec, }, nil } +// Deprecated: do not rely on this function. Do not use m.Codec directly either. +// These will eventually be removed and not exposed. func (m EigenDAClient) GetCodec() codecs.BlobCodec { return m.Codec } @@ -84,27 +118,30 @@ func (m EigenDAClient) GetCodec() codecs.BlobCodec { // data, which is necessary for generating KZG proofs for data's correctness. // The function handles potential errors during blob retrieval, data length // checks, and decoding processes. +// TODO: should we use a pointer receiver instead, to prevent unnecessary copying of the EigenDAClient struct? func (m EigenDAClient) GetBlob(ctx context.Context, batchHeaderHash []byte, blobIndex uint32) ([]byte, error) { data, err := m.Client.RetrieveBlob(ctx, batchHeaderHash, blobIndex) if err != nil { - return nil, err + return nil, fmt.Errorf("could not retrieve blob: %w", err) } if len(data) == 0 { + // TODO: explain when/why/how this can happen return nil, fmt.Errorf("blob has length zero") } decodedData, err := m.Codec.DecodeBlob(data) if err != nil { - return nil, fmt.Errorf("error getting blob: %w", err) + return nil, fmt.Errorf("error decoding blob: %w", err) } return decodedData, nil } -// PutBlob encodes and writes a blob to EigenDA, waiting for it to be finalized -// before returning. This function is resiliant to transient failures and -// timeouts. +// PutBlob encodes and writes a blob to EigenDA, waiting for a desired blob status +// to be reached (guarded by WaitForFinalization config param) before returning. +// This function is resilient to transient failures and timeouts. +// TODO: should we use a pointer receiver instead, to prevent unnecessary copying of the EigenDAClient struct? func (m EigenDAClient) PutBlob(ctx context.Context, data []byte) (*grpcdisperser.BlobInfo, error) { resultChan, errorChan := m.PutBlobAsync(ctx, data) select { // no timeout here because we depend on the configured timeout in PutBlobAsync @@ -115,6 +152,7 @@ func (m EigenDAClient) PutBlob(ctx context.Context, data []byte) (*grpcdisperser } } +// TODO: should we use a pointer receiver instead, to prevent unnecessary copying of the EigenDAClient struct? func (m EigenDAClient) PutBlobAsync(ctx context.Context, data []byte) (resultChan chan *grpcdisperser.BlobInfo, errChan chan error) { resultChan = make(chan *grpcdisperser.BlobInfo, 1) errChan = make(chan error, 1) @@ -142,6 +180,7 @@ func (m EigenDAClient) putBlob(ctx context.Context, rawData []byte, resultChan c customQuorumNumbers[i] = uint8(e) } // disperse blob + // TODO: would be nice to add a trace-id key to the context, to be able to follow requests from batcher->proxy->eigenda blobStatus, requestID, err := m.Client.DisperseBlobAuthenticated(ctx, data, customQuorumNumbers) if err != nil { errChan <- fmt.Errorf("error initializing DisperseBlobAuthenticated() client: %w", err) @@ -150,13 +189,12 @@ func (m EigenDAClient) putBlob(ctx context.Context, rawData []byte, resultChan c // process response if *blobStatus == disperser.Failed { - m.Log.Error("Unable to disperse blob to EigenDA, aborting", "err", err) - errChan <- fmt.Errorf("reply status is %d", blobStatus) + errChan <- fmt.Errorf("unable to disperse blob to eigenda (reply status %d): %w", blobStatus, err) return } base64RequestID := base64.StdEncoding.EncodeToString(requestID) - m.Log.Info("Blob dispersed to EigenDA, now waiting for confirmation", "requestID", base64RequestID) + m.Log.Info("Blob accepted by EigenDA disperser, now polling for status updates", "requestID", base64RequestID) ticker := time.NewTicker(m.Config.StatusQueryRetryInterval) defer ticker.Stop() @@ -166,7 +204,7 @@ func (m EigenDAClient) putBlob(ctx context.Context, rawData []byte, resultChan c defer cancel() alreadyWaitingForDispersal := false - alreadyWaitingForFinalization := false + alreadyWaitingForConfirmationOrFinality := false for { select { case <-ctx.Done(): @@ -175,7 +213,7 @@ func (m EigenDAClient) putBlob(ctx context.Context, rawData []byte, resultChan c case <-ticker.C: statusRes, err := m.Client.GetBlobStatus(ctx, requestID) if err != nil { - m.Log.Error("Unable to retrieve blob dispersal status, will retry", "requestID", base64RequestID, "err", err) + m.Log.Warn("Unable to retrieve blob dispersal status, will retry", "requestID", base64RequestID, "err", err) continue } @@ -183,42 +221,85 @@ func (m EigenDAClient) putBlob(ctx context.Context, rawData []byte, resultChan c case grpcdisperser.BlobStatus_PROCESSING, grpcdisperser.BlobStatus_DISPERSING: // to prevent log clutter, we only log at info level once if alreadyWaitingForDispersal { - m.Log.Debug("Blob submitted, waiting for dispersal from EigenDA", "requestID", base64RequestID) + m.Log.Debug("Blob is being processed by the EigenDA network", "requestID", base64RequestID) } else { - m.Log.Info("Blob submitted, waiting for dispersal from EigenDA", "requestID", base64RequestID) + m.Log.Info("Blob is being processed by the EigenDA network", "requestID", base64RequestID) alreadyWaitingForDispersal = true } case grpcdisperser.BlobStatus_FAILED: - m.Log.Error("EigenDA blob dispersal failed in processing", "requestID", base64RequestID, "err", err) errChan <- fmt.Errorf("EigenDA blob dispersal failed in processing, requestID=%s: %w", base64RequestID, err) return case grpcdisperser.BlobStatus_INSUFFICIENT_SIGNATURES: - m.Log.Error("EigenDA blob dispersal failed in processing with insufficient signatures", "requestID", base64RequestID, "err", err) errChan <- fmt.Errorf("EigenDA blob dispersal failed in processing with insufficient signatures, requestID=%s: %w", base64RequestID, err) return case grpcdisperser.BlobStatus_CONFIRMED: if m.Config.WaitForFinalization { // to prevent log clutter, we only log at info level once - if alreadyWaitingForFinalization { - m.Log.Debug("EigenDA blob confirmed, waiting for finalization", "requestID", base64RequestID) + if alreadyWaitingForConfirmationOrFinality { + m.Log.Debug("EigenDA blob included onchain, waiting for finalization", "requestID", base64RequestID) } else { - m.Log.Info("EigenDA blob confirmed, waiting for finalization", "requestID", base64RequestID) - alreadyWaitingForFinalization = true + m.Log.Info("EigenDA blob included onchain, waiting for finalization", "requestID", base64RequestID) + alreadyWaitingForConfirmationOrFinality = true } } else { - m.Log.Info("EigenDA blob confirmed", "requestID", base64RequestID) - resultChan <- statusRes.Info - return + batchId := statusRes.Info.BlobVerificationProof.GetBatchId() + batchConfirmed, err := m.batchIdConfirmedAtDepth(ctx, batchId, m.Config.WaitForConfirmationDepth) + if err != nil { + m.Log.Warn("Error checking if batch ID is confirmed at depth. Will retry...", "requestID", base64RequestID, "err", err) + } + if batchConfirmed { + m.Log.Info("EigenDA blob confirmed", "requestID", base64RequestID, "confirmationDepth", m.Config.WaitForConfirmationDepth) + resultChan <- statusRes.Info + return + } + // to prevent log clutter, we only log at info level once + if alreadyWaitingForConfirmationOrFinality { + m.Log.Debug("EigenDA blob included onchain, waiting for confirmation", "requestID", base64RequestID, "confirmationDepth", m.Config.WaitForConfirmationDepth) + } else { + m.Log.Info("EigenDA blob included onchain, waiting for confirmation", "requestID", base64RequestID, "confirmationDepth", m.Config.WaitForConfirmationDepth) + alreadyWaitingForConfirmationOrFinality = true + } } case grpcdisperser.BlobStatus_FINALIZED: batchHeaderHashHex := fmt.Sprintf("0x%s", hex.EncodeToString(statusRes.Info.BlobVerificationProof.BatchMetadata.BatchHeaderHash)) - m.Log.Info("Successfully dispersed blob to EigenDA", "requestID", base64RequestID, "batchHeaderHash", batchHeaderHashHex) + m.Log.Info("EigenDA blob finalized", "requestID", base64RequestID, "batchHeaderHash", batchHeaderHashHex) resultChan <- statusRes.Info return default: - errChan <- fmt.Errorf("EigenDA blob dispersal failed in processing with reply status %d", statusRes.Status) + // this should never happen. If it does, the blob is in a heisenberg state... it could either eventually get confirmed or fail + errChan <- fmt.Errorf("unknown reply status %d. ask for assistance from EigenDA team, using requestID %s", statusRes.Status, base64RequestID) return } } } } + +// getConfDeepBlockNumber returns the block number that is `depth` blocks behind the current block number. +func (m EigenDAClient) getConfDeepBlockNumber(ctx context.Context, depth uint64) (*big.Int, error) { + curBlockNumber, err := m.ethClient.BlockNumber(ctx) + if err != nil { + return nil, fmt.Errorf("failed to get latest block number: %w", err) + } + // If curBlock < depth, this will return the genesis block number (0), + // which would cause to accept as confirmed a block that isn't depth deep. + // TODO: there's prob a better way to deal with this, like returning a special error + return new(big.Int).SetUint64(max(curBlockNumber-depth, 0)), nil +} + +// batchIdConfirmedAtDepth checks if a batch ID has been confirmed at a certain depth. +// It returns true if the batch ID has been confirmed at the given depth, and false otherwise, +// or returns an error if any of the network calls fail. +func (m EigenDAClient) batchIdConfirmedAtDepth(ctx context.Context, batchId uint32, depth uint64) (bool, error) { + confDeepBlockNumber, err := m.getConfDeepBlockNumber(ctx, m.Config.WaitForConfirmationDepth) + if err != nil { + return false, fmt.Errorf("failed to get confirmation deep block number: %w", err) + } + onchainBatchMetadataHash, err := m.edasmCaller.BatchIdToBatchMetadataHash(&bind.CallOpts{BlockNumber: confDeepBlockNumber}, batchId) + if err != nil { + return false, fmt.Errorf("failed to get batch metadata hash: %w", err) + } + if bytes.Equal(onchainBatchMetadataHash[:], make([]byte, 32)) { + return false, nil + } + return true, nil +} diff --git a/api/clients/eigenda_client_e2e_test.go b/api/clients/eigenda_client_e2e_test.go index f10a6aec9..173fc3a6d 100644 --- a/api/clients/eigenda_client_e2e_test.go +++ b/api/clients/eigenda_client_e2e_test.go @@ -1,13 +1,14 @@ -package clients_test +package clients import ( "context" "flag" + "math/big" "os" "testing" "time" - "github.com/Layr-Labs/eigenda/api/clients" + "github.com/ethereum/go-ethereum/accounts/abi/bind" "github.com/ethereum/go-ethereum/log" "github.com/stretchr/testify/assert" ) @@ -22,22 +23,61 @@ func TestClientUsingTestnet(t *testing.T) { if !runTestnetIntegrationTests { t.Skip("Skipping testnet integration test") } - logger := log.NewLogger(log.NewTerminalHandler(os.Stderr, true)) - client, err := clients.NewEigenDAClient(logger, clients.EigenDAClientConfig{ - RPC: "disperser-holesky.eigenda.xyz:443", - StatusQueryTimeout: 25 * time.Minute, - StatusQueryRetryInterval: 5 * time.Second, - CustomQuorumIDs: []uint{}, - SignerPrivateKeyHex: "2d23e142a9e86a9175b9dfa213f20ea01f6c1731e09fa6edf895f70fe279cbb1", - WaitForFinalization: true, + + t.Run("PutBlobWaitForFinalityAndGetBlob", func(t *testing.T) { + t.Parallel() + logger := log.NewLogger(log.NewTerminalHandler(os.Stdout, true)) + client, err := NewEigenDAClient(logger, EigenDAClientConfig{ + RPC: "disperser-holesky.eigenda.xyz:443", + StatusQueryTimeout: 25 * time.Minute, + StatusQueryRetryInterval: 5 * time.Second, + CustomQuorumIDs: []uint{}, + SignerPrivateKeyHex: "2d23e142a9e86a9175b9dfa213f20ea01f6c1731e09fa6edf895f70fe279cbb1", + WaitForFinalization: true, + }) + data := "hello world!" + assert.NoError(t, err) + blobInfo, err := client.PutBlob(context.Background(), []byte(data)) + assert.NoError(t, err) + batchHeaderHash := blobInfo.BlobVerificationProof.BatchMetadata.BatchHeaderHash + blobIndex := blobInfo.BlobVerificationProof.BlobIndex + blob, err := client.GetBlob(context.Background(), batchHeaderHash, blobIndex) + assert.NoError(t, err) + assert.Equal(t, data, string(blob)) + }) + + t.Run("PutBlobWaitForConfirmationDepthAndGetBlob", func(t *testing.T) { + t.Parallel() + confDepth := uint64(3) + logger := log.NewLogger(log.NewTerminalHandler(os.Stdout, true)) + client, err := NewEigenDAClient(logger, EigenDAClientConfig{ + RPC: "disperser-holesky.eigenda.xyz:443", + StatusQueryTimeout: 25 * time.Minute, + StatusQueryRetryInterval: 5 * time.Second, + CustomQuorumIDs: []uint{}, + SignerPrivateKeyHex: "2d23e142a9e86a9175b9dfa213f20ea01f6c1731e09fa6edf895f70fe279cbb1", + WaitForFinalization: false, + WaitForConfirmationDepth: confDepth, + SvcManagerAddr: "0xD4A7E1Bd8015057293f0D0A557088c286942e84b", + EthRpcUrl: "https://1rpc.io/holesky", + }) + data := "hello world!" + assert.NoError(t, err) + blobInfo, err := client.PutBlob(context.Background(), []byte(data)) + assert.NoError(t, err) + batchHeaderHash := blobInfo.BlobVerificationProof.BatchMetadata.BatchHeaderHash + blobIndex := blobInfo.BlobVerificationProof.BlobIndex + blob, err := client.GetBlob(context.Background(), batchHeaderHash, blobIndex) + assert.NoError(t, err) + assert.Equal(t, data, string(blob)) + + // assert confirmation depth + blockNumCur, err := client.ethClient.BlockNumber(context.Background()) + assert.NoError(t, err) + blockNumAtDepth := new(big.Int).SetUint64(blockNumCur - confDepth) + batchId := blobInfo.BlobVerificationProof.GetBatchId() + onchainBatchMetadataHash, err := client.edasmCaller.BatchIdToBatchMetadataHash(&bind.CallOpts{BlockNumber: blockNumAtDepth}, batchId) + assert.NoError(t, err) + assert.NotEqual(t, onchainBatchMetadataHash, make([]byte, 32)) }) - data := "hello world!" - assert.NoError(t, err) - blobInfo, err := client.PutBlob(context.Background(), []byte(data)) - assert.NoError(t, err) - batchHeaderHash := blobInfo.BlobVerificationProof.BatchMetadata.BatchHeaderHash - blobIndex := blobInfo.BlobVerificationProof.BlobIndex - blob, err := client.GetBlob(context.Background(), batchHeaderHash, blobIndex) - assert.NoError(t, err) - assert.Equal(t, data, string(blob)) } diff --git a/api/clients/eigenda_client_test.go b/api/clients/eigenda_client_test.go index 8b49fbbb6..79e8556df 100644 --- a/api/clients/eigenda_client_test.go +++ b/api/clients/eigenda_client_test.go @@ -123,6 +123,7 @@ func TestPutRetrieveBlobIFFTNoDecodeSuccess(t *testing.T) { (disperserClient.On("RetrieveBlob", mock.Anything, mock.Anything, mock.Anything). Return(nil, nil).Once()) // pass nil in as the return blob to tell the mock to return the corresponding blob logger := log.NewLogger(log.DiscardHandler()) + ifftCodec := codecs.NewIFFTCodec(codecs.NewDefaultBlobCodec()) eigendaClient := clients.EigenDAClient{ Log: logger, Config: clients.EigenDAClientConfig{ @@ -138,7 +139,7 @@ func TestPutRetrieveBlobIFFTNoDecodeSuccess(t *testing.T) { WaitForFinalization: true, }, Client: disperserClient, - Codec: codecs.NewIFFTCodec(codecs.NewDefaultBlobCodec()), + Codec: ifftCodec, } expectedBlob := []byte("dc49e7df326cfb2e7da5cf68f263e1898443ec2e862350606e7dfbda55ad10b5d61ed1d54baf6ae7a86279c1b4fa9c49a7de721dacb211264c1f5df31bade51c") blobInfo, err := eigendaClient.PutBlob(context.Background(), expectedBlob) @@ -148,7 +149,7 @@ func TestPutRetrieveBlobIFFTNoDecodeSuccess(t *testing.T) { resultBlob, err := eigendaClient.GetBlob(context.Background(), []byte("mock-batch-header-hash"), 100) require.NoError(t, err) - encodedBlob, err := eigendaClient.GetCodec().EncodeBlob(resultBlob) + encodedBlob, err := ifftCodec.EncodeBlob(resultBlob) require.NoError(t, err) resultBlob, err = codecs.NewIFFTCodec(codecs.NewDefaultBlobCodec()).DecodeBlob(encodedBlob) diff --git a/api/grpc/disperser/disperser.pb.go b/api/grpc/disperser/disperser.pb.go index 0737f0e46..36448f8d2 100644 --- a/api/grpc/disperser/disperser.pb.go +++ b/api/grpc/disperser/disperser.pb.go @@ -373,12 +373,12 @@ type DisperseBlobRequest struct { unknownFields protoimpl.UnknownFields // The data to be dispersed. - // The size of data must be <= 2MiB. Every 32 bytes of data chunk is interpreted as an integer in big endian format + // The size of data must be <= 16MiB. Every 32 bytes of data chunk is interpreted as an integer in big endian format // where the lower address has more significant bits. The integer must stay in the valid range to be interpreted // as a field element on the bn254 curve. The valid range is // 0 <= x < 21888242871839275222246405745257275088548364400416034343698204186575808495617 - // containing slightly less than 254 bits and more than 253 bits. If any one of the 32 bytes chunk is outside the range, - // the whole request is deemed as invalid, and rejected. + // If any one of the 32 bytes chunk is outside the range, the whole request is deemed as invalid, and rejected. + // Note that the max allowed field element is a 254 bit number, meaning the first 2 bits of each chunk will always be 00. Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` // The quorums to which the blob will be sent, in addition to the required quorums which are configured // on the EigenDA smart contract. If required quorums are included here, an error will be returned. diff --git a/api/proto/disperser/disperser.proto b/api/proto/disperser/disperser.proto index 536ef182c..393fa5a27 100644 --- a/api/proto/disperser/disperser.proto +++ b/api/proto/disperser/disperser.proto @@ -78,12 +78,12 @@ message AuthenticationData { message DisperseBlobRequest { // The data to be dispersed. - // The size of data must be <= 2MiB. Every 32 bytes of data chunk is interpreted as an integer in big endian format + // The size of data must be <= 16MiB. Every 32 bytes of data chunk is interpreted as an integer in big endian format // where the lower address has more significant bits. The integer must stay in the valid range to be interpreted // as a field element on the bn254 curve. The valid range is // 0 <= x < 21888242871839275222246405745257275088548364400416034343698204186575808495617 - // containing slightly less than 254 bits and more than 253 bits. If any one of the 32 bytes chunk is outside the range, - // the whole request is deemed as invalid, and rejected. + // If any one of the 32 bytes chunk is outside the range, the whole request is deemed as invalid, and rejected. + // Note that the max allowed field element is a 254 bit number, meaning the first 2 bits of each chunk will always be 00. bytes data = 1; // The quorums to which the blob will be sent, in addition to the required quorums which are configured // on the EigenDA smart contract. If required quorums are included here, an error will be returned. diff --git a/encoding/utils/codec/codec.go b/encoding/utils/codec/codec.go index eb08acc47..09659d433 100644 --- a/encoding/utils/codec/codec.go +++ b/encoding/utils/codec/codec.go @@ -5,10 +5,10 @@ import ( ) // ConvertByPaddingEmptyByte takes bytes and insert an empty byte at the front of every 31 byte. -// The empty byte is padded at the low address, because we use big endian to interpret a fiedl element. -// This ensure every 32 bytes are within the valid range of a field element for bn254 curve. -// If the input data is not a multiple of 31, the reminder is added to the output by -// inserting a 0 and the reminder. The output does not necessarily be a multipler of 32 +// The empty byte is padded at the low address, because we use big endian to interpret a field element. +// This ensures every 32 bytes is within the valid range of a field element for bn254 curve. +// If the input data is not a multiple of 31, the remainder is added to the output by +// inserting a 0 and the remainder. The output is thus not necessarily a multiple of 32. func ConvertByPaddingEmptyByte(data []byte) []byte { dataSize := len(data) parseSize := encoding.BYTES_PER_SYMBOL - 1