Always use `GetActualSize` to get the part size, not just when encrypted.
Fixes mint test io.minio.MinioClient.uploadPartCopy,
error "Range specified is not valid for source object".
healing code was using incorrect buffers to heal older
objects with 10MiB erasure blockSize, incorrect calculation
of such buffers can lead to incorrect premature closure of
io.Pipe() during healing.
fixes#12410
- it is possible that during I/O failures we might
leave partially written directories, make sure
we purge them after.
- rename current data-dir (null) versionId only after
the newer xl.meta has been written fully.
- attempt removal once for minioMetaTmpBucket/uuid/
as this folder is empty if all previous operations
were successful, this allows avoiding recursive os.Remove()
- for single pool setups usage is not checked.
- for pools, only check the "set" in which it would be placed.
- keep a minimum number of inodes (when we know it).
- ignore for `.minio.sys`.
It makes sense that a node that has multiple disks starts when one
disk fails, returning an i/o error for example. This commit will make this
faulty tolerance available in this specific use case.
Due to incorrect KMS context constructed, we need to add
additional fallbacks and also fix the original root cause
to fix already migrated deployments.
Bonus remove double migration is avoided in gateway mode
for etcd, instead do it once in iam.Init(), also simplify
the migration by not migrating STS users instead let the
clients regenerate them.
- Adds versioning support for S3 based remote tiers that have versioning
enabled. This ensures that when reading or deleting we specify the specific
version ID of the object. In case of deletion, this is important to ensure that
the object version is actually deleted instead of simply being marked for
deletion.
- Stores the remote object's version id in the tier-journal. Tier-journal file
version is not bumped up as serializing the new struct version is
compatible with old journals without the remote object version id.
- `storageRESTVersion` is bumped up as FileInfo struct now includes a
`TransitionRemoteVersionID` member.
- Azure and GCS support for this feature will be added subsequently.
Co-authored-by: Krishnan Parthasarathi <krisis@users.noreply.github.com>
Also adding an API to allow resyncing replication when
existing object replication is enabled and the remote target
is entirely lost. With the `mc replicate reset` command, the
objects that are eligible for replication as per the replication
config will be resynced to target if existing object replication
is enabled on the rule.
This is to ensure that there are no projects
that try to import `minio/minio/pkg` into
their own repo. Any such common packages should
go to `https://github.com/minio/pkg`
IAM not initialized doesn't mean we can't still
read the content from the disk, we should just
allow the request to go-through if object layer
is initialized.
Real-time metrics calculated in-memory rely on the initial
replication metrics saved with data usage. However, this can
lag behind the actual state of the cluster at the time of server
restart leading to inaccurate Pending size/counts reported to
Prometheus. Dropping the Pending metrics as this can be more
reliably monitored by applications with replication notifications.
Signed-off-by: Poorna Krishnamoorthy <poorna@minio.io>
LDAPusername is the simpler form of LDAPUser (userDN),
using a simpler version is convenient from policy
conditions point of view, since these are unique id's
used for LDAP login.
In cases where a cluster is degraded, we do not uphold our consistency
guarantee and we will write fewer erasure codes and rely on healing
to recreate the missing shards.
In some cases replacing known bad disks in practice take days.
We want to change the behavior of a known degraded system to keep
the erasure code promise of the storage class for each object.
This will create the objects with the same confidence as a fully
functional cluster. The tradeoff will be that objects created
during a partial outage will take up slightly more space.
This means that when the storage class is EC:4, there should
always be written 4 parity shards, even if some disks are unavailable.
When an object is created on a set, the disks are immediately
checked. If any disks are unavailable additional parity shards
will be made for each offline disk, up to 50% of the number of disks.
We add an internal metadata field with the actual and intended
erasure code level, this can optionally be picked up later by
the scanner if we decide that data like this should be re-sharded.
Bonus change LDAP settings such as user, group mappings
are now listed as part of `mc admin user list` and
`mc admin group list`
Additionally this PR also deprecates the `/v2` API
that is no longer in use.
A configured audit logger or HTTP logger is validated during MinIO
server startup. Relax the timeout to 10 seconds in that case, otherwise,
both loggers won't be used.
1 second could be too low for a busy HTTP endpoint.
This commit fixes a bug causing the MinIO server to compute
the ETag of a single-part object as MD5 of the compressed
content - not as MD5 of the actual content.
This usually does not affect clients since the MinIO appended
a `-1` to indicate that the ETag belongs to a multipart object.
However, this behavior was problematic since:
- A S3 client being very strict should reject such an ETag since
the client uploaded the object via single-part API but got
a multipart ETag that is not the content MD5.
- The MinIO server leaks (via the ETag) that it compressed the
object.
This commit addresses both cases. Now, the MinIO server returns
an ETag equal to the content MD5 for single-part objects that got
compressed.
Signed-off-by: Andreas Auernhammer <aead@mail.de>
A lot of healing is likely to be on non-existing objects and
locks are very expensive and will slow down scanning
significantly.
In cases where all are valid or, all are broken allow
rejection without locking.
Keep the existing behavior, but move the check for
dangling objects to after the lock has been acquired.
```
_, err = getLatestFileInfo(ctx, partsMetadata, errs)
if err != nil {
return er.purgeObjectDangling(ctx, bucket, object, versionID, partsMetadata, errs, []error{}, opts)
}
```
Revert "heal: Hold lock when reading xl.meta from disks (#12362)"
This reverts commit abd32065aa
This PR fixes two bugs
- Remove fi.Data upon overwrite of objects from inlined-data to non-inlined-data
- Workaround for an existing bug on disk with latest releases to ignore fi.Data
and instead read from the disk for non-inlined-data
- Addtionally add a reserved metadata header to indicate data is inlined for
a given version.
Lock is hold in healObject() after reading xl.meta from disks the first
time. This commit will held the lock since the beginning of HealObject()
Co-authored-by: Anis Elleuch <anis@min.io>
Fixes `testSSES3EncryptedGetObjectReadSeekFunctional` mint test.
```
{
"args": {
"bucketName": "minio-go-test-w53hbpat649nhvws",
"objectName": "6mdswladz4vfpp2oit1pkn3qd11te5"
},
"duration": 7537,
"error": "We encountered an internal error, please try again.: cause(The requested range \"bytes 251717932 -> -116384170 of 135333762\" is not satisfiable.)",
"function": "GetObject(bucketName, objectName)",
"message": "CopyN failed",
"name": "minio-go: testSSES3EncryptedGetObjectReadSeekFunctional",
"status": "FAIL"
}
```
Compressed files always start at the beginning of a part so no additional offset should be added.
Previous PR #12351 added functions to read from the reader
stream to reduce memory usage, use the same technique in
few other places where we are not interested in reading the
data part.
in setups with lots of drives the server
startup is slow, initialize all local drives
in parallel before registering with muxer.
this speeds up when there are multiple pools
and large collection of drives.
This commit adds the `X-Amz-Server-Side-Encryption-Aws-Kms-Key-Id`
response header to the GET, HEAD, PUT and Download API.
Based on AWS documentation [1] AWS S3 returns the KMS key ID as part
of the response headers.
[1] https://docs.aws.amazon.com/AmazonS3/latest/userguide/specifying-kms-encryption.html
Signed-off-by: Andreas Auernhammer <aead@mail.de>
multi-disk clusters initialize buffer pools
per disk, this is perhaps expensive and perhaps
not useful, for a running server instance. As this
may disallow re-use of buffers across sets,
this change ensures that buffers across sets
can be re-used at drive level, this can reduce
quite a lot of memory on large drive setups.
In lieu of new changes coming for server command line, this
change is to deprecate strict requirement for distributed setups
to provide root credentials.
Bonus: remove MINIO_WORM warning from April 2020, it is time to
remove this warning.
However, this slice is also used for closing the writers, so close is never called on these.
Furthermore when an error is returned from a write it is now reported to the reader.
bonus: remove unused heal param from `newBitrotWriter`.
* Remove copy, now that we don't mutate.
At some places bloom filter tracker was getting
updated for `.minio.sys/tmp` bucket, there is no
reason to update bloom filters for those.
And add a missing bloom filter update for MakeBucket()
Bonus: purge unused function deleteEmptyDir()
gracefully start the server, if there are other drives
available - print enough information for administrator
to notice the errors in console.
Bonus: for really large streams use larger buffer for
writes.
- GetObject() should always use a common dataDir to
read from when it starts reading, this allows the
code in erasure decoding to have sane expectations.
- Healing should always heal on the common dataDir, this
allows the code in dangling object detection to purge
dangling content.
These both situations can happen under certain types of
retries during PUT when server is restarting etc, some
namespace entries might be left over.
attempt a delete on remote DNS store first before
attempting locally, because removing at DNS store
is cheaper than deleting locally, in case of
errors locally we can cheaply recreate the
bucket on dnsStore instead of.
This commit adds support for SSE-KMS bucket configurations.
Before, the MinIO server did not support SSE-KMS, and therefore,
it was not possible to specify an SSE-KMS bucket config.
Now, this is possible. For example:
```
mc encrypt set sse-kms some-key <alias>/my-bucket
```
Further, this commit fixes an issue caused by not supporting
SSE-KMS bucket configuration and switching to SSE-KMS as default
SSE method.
Before, the server just checked whether an SSE bucket config was
present (not which type of SSE config) and applied the default
SSE method (which was switched from SSE-S3 to SSE-KMS).
This caused objects to get encrypted with SSE-KMS even though a
SSE-S3 bucket config was present.
This issue is fixed as a side-effect of this commit.
Signed-off-by: Andreas Auernhammer <aead@mail.de>
when bidirectional replication is set up.
If ReplicaModifications is enabled in the replication
configuration, sync metadata updates to source if
replication rules are met. By default, if this
configuration is unset, MinIO automatically sync's
metadata updates on replica back to the source.
This commit adds a check to the MinIO server setup that verifies
that MinIO can reach KES, if configured, and that the default key
exists. If the default key does not exist it will create it
automatically.
Signed-off-by: Andreas Auernhammer <aead@mail.de>
A cache structure will be kept with a tree of usages.
The cache is a tree structure where each keeps track
of its children.
An uncompacted branch contains a count of the files
only directly at the branch level, and contains link to
children branches or leaves.
The leaves are "compacted" based on a number of properties.
A compacted leaf contains the totals of all files beneath it.
A leaf is only scanned once every dataUsageUpdateDirCycles,
rarer if the bloom filter for the path is clean and no lifecycles
are applied. Skipped leaves have their totals transferred from
the previous cycle.
A clean leaf will be included once every healFolderIncludeProb
for partial heal scans. When selected there is a one in
healObjectSelectProb that any object will be chosen for heal scan.
Compaction happens when either:
- The folder (and subfolders) contains less than dataScannerCompactLeastObject objects.
- The folder itself contains more than dataScannerCompactAtFolders folders.
- The folder only contains objects and no subfolders.
- A bucket root will never be compacted.
Furthermore, if a has more than dataScannerCompactAtChildren recursive
children (uncompacted folders) the tree will be recursively scanned and the
branches with the least number of objects will be compacted until the limit
is reached.
This ensures that any branch will never contain an unreasonable amount
of other branches, and also that small branches with few objects don't
take up unreasonable amounts of space.
Whenever a branch is scanned, it is assumed that it will be un-compacted
before it hits any of the above limits. This will make the branch rebalance
itself when scanned if the distribution of objects has changed.
TLDR; With current values: No bucket will ever have more than 10000
child nodes recursively. No single folder will have more than 2500 child
nodes by itself. All subfolders are compacted if they have less than 500
objects in them recursively.
We accumulate the (non-deletemarker) version count for paths as well,
since we are changing the structure anyway.
MRF does not detect when a node is disconnected and reconnected quickly
this change will ensure that MRF is alerted by comparing the last disk
reconnection timestamp with the last MRF check time.
Signed-off-by: Anis Elleuch <anis@min.io>
Co-authored-by: Klaus Post <klauspost@gmail.com>
wait groups are necessary with io.Pipes() to avoid
races when a blocking function may not be expected
and a Write() -> Close() before Read() races on each
other. We should avoid such situations..
Co-authored-by: Klaus Post <klauspost@gmail.com>
This commit replaces the custom KES client implementation
with the KES SDK from https://github.com/minio/kes
The SDK supports multi-server client load-balancing and
requests retry out of the box. Therefore, this change reduces
the overall complexity within the MinIO server and there
is no need to maintain two separate client implementations.
Signed-off-by: Andreas Auernhammer <aead@mail.de>
This commit enforces the usage of AES-256
for config and IAM data en/decryption in FIPS
mode.
Further, it improves the implementation of
`fips.Enabled` by making it a compile time
constant. Now, the compiler is able to evaluate
the any `if fips.Enabled { ... }` at compile time
and eliminate unused code.
Signed-off-by: Andreas Auernhammer <aead@mail.de>
p.writers is a verbatim value of bitrotWriter
backed by a pipe() that should never be nil'ed,
instead use the captured errors to skip the writes.
additionally detect also short writes, and reject
them as errors.
currently GetUser() returns 403 when IAM is not initialized
this can lead to applications crashing, instead return 503
so that the applications can retry and backoff.
fixes#12078
as there is no automatic way to detect if there
is a root disk mounted on / or /var for the container
environments due to how the root disk information
is masked inside overlay root inside container.
this PR brings an environment variable to set
root disk size threshold manually to detect the
root disks in such situations.
This commit fixes a bug in the single-part object decryption
that is triggered in case of SSE-KMS. Before, it was assumed
that the encryption is either SSE-C or SSE-S3. In case of SSE-KMS
the SSE-C branch was executed. This lead to an invalid SSE-C
algorithm error.
This commit fixes this by inverting the `if-else` logic.
Now, the SSE-C branch only gets executed when SSE-C headers
are present.
Signed-off-by: Andreas Auernhammer <aead@mail.de>
This commit fixes a bug introduced by af0c65b.
When there is no / an empty client-provided SSE-KMS
context the `ParseMetadata` may return a nil map
(`kms.Context`).
When unsealing the object key we must check that
the context is nil before assigning a key-value pair.
Signed-off-by: Andreas Auernhammer <aead@mail.de>
UpdateServiceAccount ignores updating fields when not passed from upper
layer, such as empty policy, empty account status, and empty secret key.
This PR will check for a secret key only if it is empty and add more
check on the value of the account status.
Signed-off-by: Anis Elleuch <anis@min.io>
This commit adds basic SSE-KMS support.
Now, a client can specify the SSE-KMS headers
(algorithm, optional key-id, optional context)
such that the object gets encrypted using the
SSE-KMS method. Further, auto-encryption now
defaults to SSE-KMS.
This commit does not try to do any refactoring
and instead tries to implement SSE-KMS as a minimal
change to the code base. However, refactoring the entire
crypto-related code is planned - but needs a separate
effort.
Signed-off-by: Andreas Auernhammer <aead@mail.de>
It is possible in some scenarios that in multiple pools,
two concurrent calls for the same object as a multipart operation
can lead to duplicate entries on two different pools.
This PR fixes this
- hold locks to serialize multiple callers so that we don't race.
- make sure to look for existing objects on the namespace as well
not just for existing uploadIDs
When running MinIO server without LDAP/OpenID, we should error out when
the code tries to create a service account for a non existant regular
user.
Bonus: refactor the check code to be show all cases more clearly
Signed-off-by: Anis Elleuch <anis@min.io>
Co-authored-by: Anis Elleuch <anis@min.io>
This commit adds basic SSE-KMS support.
Now, a client can specify the SSE-KMS headers
(algorithm, optional key-id, optional context)
such that the object gets encrypted using the
SSE-KMS method. Further, auto-encryption now
defaults to SSE-KMS.
This commit does not try to do any refactoring
and instead tries to implement SSE-KMS as a minimal
change to the code base. However, refactoring the entire
crypto-related code is planned - but needs a separate
effort.
Signed-off-by: Andreas Auernhammer <aead@mail.de>
Co-authored-by: Klaus Post <klauspost@gmail.com>
avoid time_wait build up with getObject requests if there are
pending callers and they timeout, can lead to time_wait states
Bonus share the same buffer pool with erasure healing logic,
additionally also fixes a race where parallel readers were
never cleanup during Encode() phase, because pipe.Reader end
was never closed().
Added closer right away upon an error during Encode to make
sure to avoid racy Close() while stream was still being
Read().
This commit fixes a bug when parsing the env. variable
`MINIO_KMS_SECRET_KEY`. Before, the env. variable
name - instead of its value - was parsed. This (obviously)
did not work properly.
This commit fixes this.
Signed-off-by: Andreas Auernhammer <aead@mail.de>
cleanup functions should never be cleaned before the reader is
instantiated, this type of design leads to situations where order
of lockers and places for them to use becomes confusing.
Allow WithCleanupFuncs() if the caller wishes to add cleanupFns
to be run upon close() or an error during initialization of the
reader.
Also make sure streams are closed before we unlock the resources,
this allows for ordered cleanup of resources.
upon errors to acquire lock context would still leak,
since the cancel would never be called. since the lock
is never acquired - proactively clear it before returning.
failed queue should be used for retried requests to
avoid cascading the failures into incoming queue, this
would allow for a more fair retry for failed replicas.
Additionally also avoid taking context in queue task
to avoid confusion, simplifies its usage.
There can be situations where replication completed but the
`X-Amz-Replication-Status` metadata update failed such as
when the server returns 503 under high load. This object version will
continue to be picked up by the scanner and replicateObject would perform
no action since the versions match between source and target.
The metadata would never reflect that replication was successful
without this fix, leading to repeated re-queuing.
This commit enhances the docs about IAM encryption.
It adds a quick-start section that explains how to
get started quickly with `MINIO_KMS_SECRET_KEY`
instead of setting up KES.
It also removes the startup message that gets printed
when the server migrates IAM data to plaintext.
We will point this out in the release notes.
Signed-off-by: Andreas Auernhammer <aead@mail.de>
OpenID connect generated service accounts do not work
properly after console logout, since the parentUser state
is lost - instead use sub+iss claims for parentUser, according
to OIDC spec both the claims provide the necessary stability
across logins etc.
Currently, only credentials could be updated with
`mc admin bucket remote edit`.
Allow updating synchronous replication flag, path,
bandwidth and healthcheck duration on buckets, and
a flag to disable proxying in active-active replication.
* lock: Always cancel the returned Get(R)Lock context
There is a leak with cancel created inside the locking mechanism. The
cancel purpose was to cancel operations such erasure get/put that are
holding non-refreshable locks.
This PR will ensure the created context.Cancel is passed to the unlock
API so it will cleanup and avoid leaks.
* locks: Avoid returning nil cancel in local lockers
Since there is no Refresh mechanism in the local locking mechanism, we
do not generate a new context or cancel. Currently, a nil cancel
function is returned but this can cause a crash. Return a dummy function
instead.
https://github.com/minio/console takes over the functionality for the
future object browser development
Signed-off-by: Harshavardhana <harsha@minio.io>
LDAP DN should be used when allowing setting service accounts
for LDAP users instead of just simple user,
Bonus root owner should be allowed full access
to all service account APIs.
Signed-off-by: Harshavardhana <harsha@minio.io>
Part ETags are not available after multipart finalizes, removing this
check as not useful.
Signed-off-by: Poorna Krishnamoorthy <poorna@minio.io>
Co-authored-by: Harshavardhana <harsha@minio.io>
This commit reverts a change that added support for
parsing base64-encoded keys set via `MINIO_KMS_MASTER_KEY`.
The env. variable `MINIO_KMS_MASTER_KEY` is deprecated and
should ONLY support parsing existing keys - not the new format.
Any new deployment should use `MINIO_KMS_SECRET_KEY`. The legacy
env. variable `MINIO_KMS_MASTER_KEY` will be removed at some point
in time.
Signed-off-by: Andreas Auernhammer <aead@mail.de>
avoid re-read of xl.meta instead just use
the success criteria from PutObjectPart()
and check the ETag matches per Part, if
they match then the parts have been
successfully restored as is.
Signed-off-by: Harshavardhana <harsha@minio.io>
Bonus fix fallback to decrypt previously
encrypted content as well using older master
key ciphertext format.
Signed-off-by: Harshavardhana <harsha@minio.io>
just like replication workers, allow failed replication
workers to be configurable in situations like DR failures
etc to catch up on replication sooner when DR is back
online.
Signed-off-by: Harshavardhana <harsha@minio.io>
peer nodes would not update if policy is unset on
a user, until policies reload every 5minutes. Make
sure to reload the policies properly, if no policy
is found make sure to delete such users and groups
fixes#12074
Signed-off-by: Harshavardhana <harsha@minio.io>
With this change, MinIO's ILM supports transitioning objects to a remote tier.
This change includes support for Azure Blob Storage, AWS S3 compatible object
storage incl. MinIO and Google Cloud Storage as remote tier storage backends.
Some new additions include:
- Admin APIs remote tier configuration management
- Simple journal to track remote objects to be 'collected'
This is used by object API handlers which 'mutate' object versions by
overwriting/replacing content (Put/CopyObject) or removing the version
itself (e.g DeleteObjectVersion).
- Rework of previous ILM transition to fit the new model
In the new model, a storage class (a.k.a remote tier) is defined by the
'remote' object storage type (one of s3, azure, GCS), bucket name and a
prefix.
* Fixed bugs, review comments, and more unit-tests
- Leverage inline small object feature
- Migrate legacy objects to the latest object format before transitioning
- Fix restore to particular version if specified
- Extend SharedDataDirCount to handle transitioned and restored objects
- Restore-object should accept version-id for version-suspended bucket (#12091)
- Check if remote tier creds have sufficient permissions
- Bonus minor fixes to existing error messages
Co-authored-by: Poorna Krishnamoorthy <poorna@minio.io>
Co-authored-by: Krishna Srinivas <krishna@minio.io>
Signed-off-by: Harshavardhana <harsha@minio.io>
instead use expect continue timeout, and have
higher response header timeout, the new higher
timeout satisfies worse case scenarios for total
response time on a CreateFile operation.
Also set the "expect" continue header to satisfy
expect continue timeout behavior.
Some clients seem to cause CreateFile body to be
truncated, leading to no errors which instead
fails with ObjectNotFound on a PUT operation,
this change avoids such failures appropriately.
Signed-off-by: Harshavardhana <harsha@minio.io>
allow restrictions on who can access Prometheus
endpoint, additionally add prometheus as part of
diagnostics canned policy.
Signed-off-by: Harshavardhana <harsha@minio.io>
This commit changes the config/IAM encryption
process. Instead of encrypting config data
(users, policies etc.) with the root credentials
MinIO now encrypts this data with a KMS - if configured.
Therefore, this PR moves the MinIO-KMS configuration (via
env. variables) to a "top-level" configuration.
The KMS configuration cannot be stored in the config file
since it is used to decrypt the config file in the first
place.
As a consequence, this commit also removes support for
Hashicorp Vault - which has been deprecated anyway.
Signed-off-by: Andreas Auernhammer <aead@mail.de>
* fix: pick valid FileInfo additionally based on dataDir
historically we have always relied on modTime
to be consistent and same, we can now add additional
reference to look for the same dataDir value.
A dataDir is the same for an object at a given point in
time for a given version, let's say a `null` version
is overwritten in quorum we do not by mistake pick
up the fileInfo's incorrectly.
* make sure to not preserve fi.Data
Signed-off-by: Harshavardhana <harsha@minio.io>
InfoServiceAccount admin API does not correctly calculate the policy for
a given service account in case if the policy is implied. Fix it.
Signed-off-by: Anis Elleuch <anis@min.io>
avoid potential for duplicates under multi-pool
setup, additionally also make sure CompleteMultipart
is using a more optimal API for uploadID lookup
and never delete the object there is a potential
to create a delete marker during complete multipart.
Signed-off-by: Harshavardhana <harsha@minio.io>
This is an optimization by reducing one extra system call,
and many network operations. This reduction should increase
the performance for small file workloads.
When an error is reported it is ignored and zipping continues with the next object.
However, if there is an error it will write a response to `writeWebErrorResponse(w, err)`, but responses are still being built.
Fixes#12082
Bonus: Exclude common compressed image types.
Thanks to @Alevsk for noticing this nuanced behavior
change between releases from 03-04 to 03-20, make sure
that we handle the legacy path removal as well.
also make sure to close the channel on the producer
side, not in a separate go-routine, this can lead
to races between a writer and a closer.
fixes#12073
This is an optimization to save IOPS. The replication
failures will be re-queued once more to re-attempt
replication. If it still does not succeed, the replication
status is set as `FAILED` and will be caught up on
scanner cycle.
For InfoServiceAccount API, calculating the policy before showing it to
the user was not correctly done (only UX issue, not a security issue)
This commit fixes it.
policy might have an associated mapping with an expired
user key, do not return an error during DeletePolicy
for such situations - proceed normally as its an
expected situation.
This commit introduces a new package `pkg/kms`.
It contains basic types and functions to interact
with various KMS implementations.
This commit also moves KMS-related code from `cmd/crypto`
to `pkg/kms`. Now, it is possible to implement a KMS-based
config data encryption in the `pkg/config` package.
This commit introduces a new package `pkg/fips`
that bundles functionality to handle and configure
cryptographic protocols in case of FIPS 140.
If it is compiled with `--tags=fips` it assumes
that a FIPS 140-2 cryptographic module is used
to implement all FIPS compliant cryptographic
primitives - like AES, SHA-256, ...
In "FIPS mode" it excludes all non-FIPS compliant
cryptographic primitives from the protocol parameters.
- Add 32-bit checksum (32 LSB part of xxhash64) of the serialized metadata.
This will ensure that we always reject corrupted metadata.
- Add automatic repair of inline data, so the data structure can be used.
If data was corrupted, we remove all unreadable entries to ensure that operations
can succeed on the object. Since higher layers add bitrot checks this is not a big problem.
Cannot downgrade to v1.1 metadata, but since that isn't released, no need for a major bump.
only in case of S3 gateway we have a case where we
need to allow for SSE-S3 headers as passthrough,
If SSE-C headers are passed then they are rejected
if KMS is not configured.
This code is necessary for `mc admin update` command
to work with fips compiled binaries, with fips tags
the releaseInfo will automatically point to fips
specific binaries.
This commit fixes a bug in the put-part
implementation. The SSE headers should be
set as specified by AWS - See:
https://docs.aws.amazon.com/AmazonS3/latest/API/API_UploadPart.html
Now, the MinIO server should set SSE-C headers,
like `x-amz-server-side-encryption-customer-algorithm`.
Fixes#11991
locks can get relinquished when Read() sees io.EOF
leading to prematurely closing of the readers
concurrent writes on the same object can have
undesired consequences here when these locks
are relinquished.
EOF may be sent along with data so queue it up and
return it when the buffer is empty.
Also, when reading data without direct io don't add a buffer
that only results in extra memcopy.
Multiple disks from the same set would be writing concurrently.
```
WARNING: DATA RACE
Write at 0x00c002100ce0 by goroutine 166:
github.com/minio/minio/cmd.(*erasureSets).connectDisks.func1()
d:/minio/minio/cmd/erasure-sets.go:254 +0x82f
Previous write at 0x00c002100ce0 by goroutine 129:
github.com/minio/minio/cmd.(*erasureSets).connectDisks.func1()
d:/minio/minio/cmd/erasure-sets.go:254 +0x82f
Goroutine 166 (running) created at:
github.com/minio/minio/cmd.(*erasureSets).connectDisks()
d:/minio/minio/cmd/erasure-sets.go:210 +0x324
github.com/minio/minio/cmd.(*erasureSets).monitorAndConnectEndpoints()
d:/minio/minio/cmd/erasure-sets.go:288 +0x244
Goroutine 129 (finished) created at:
github.com/minio/minio/cmd.(*erasureSets).connectDisks()
d:/minio/minio/cmd/erasure-sets.go:210 +0x324
github.com/minio/minio/cmd.(*erasureSets).monitorAndConnectEndpoints()
d:/minio/minio/cmd/erasure-sets.go:288 +0x244
```
This commit adds a self-test for all bitrot algorithms:
- SHA-256
- BLAKE2b
- HighwayHash
The self-test computes an incremental checksum of pseudo-random
messages. If a bitrot algorithm implementation stops working on
some CPU architecture or with a certain Go version this self-test
will prevent the server from starting and silently corrupting data.
For additional context see: minio/highwayhash#19
Metrics calculation was accumulating inital usage across all nodes
rather than using initial usage only once.
Also fixing:
- bug where all peer traffic was going to the same node.
- reset counters when replication status changes from
PENDING -> FAILED
This PR fixes
- close leaking bandwidth report channel leakage
- remove the closer requirement for bandwidth monitor
instead if Read() fails remember the error and return
error for all subsequent reads.
- use locking for usage-cache.bin updates, with inline
data we cannot afford to have concurrent writes to
usage-cache.bin corrupting xl.meta
implementation in #11949 only catered from single
node, but we need cluster metrics by capturing
from all peers. introduce bucket stats API that
will be used for capturing in-line bucket usage
as well eventually
Current implementation heavily relies on readAllFileInfo
but with the advent of xl.meta inlined with data, we cannot
easily avoid reading data when we are only interested is
updating metadata, this leads to invariably write
amplification during metadata updates, repeatedly reading
data when we are only interested in updating metadata.
This PR ensures that we implement a metadata only update
API at storage layer, that handles updates to metadata alone
for any given version - given the version is valid and
present.
This helps reduce the chattiness for following calls..
- PutObjectTags
- DeleteObjectTags
- PutObjectLegalHold
- PutObjectRetention
- ReplicateObject (updates metadata on replication status)
- collect real time replication metrics for prometheus.
- add pending_count, failed_count metric for total pending/failed replication operations.
- add API to get replication metrics
- add MRF worker to handle spill-over replication operations
- multiple issues found with replication
- fixes an issue when client sends a bucket
name with `/` at the end from SetRemoteTarget
API call make sure to trim the bucket name to
avoid any extra `/`.
- hold write locks in GetObjectNInfo during replication
to ensure that object version stack is not overwritten
while reading the content.
- add additional protection during WriteMetadata() to
ensure that we always write a valid FileInfo{} and avoid
ever writing empty FileInfo{} to the lowest layers.
Co-authored-by: Poorna Krishnamoorthy <poorna@minio.io>
Co-authored-by: Harshavardhana <harsha@minio.io>
current master breaks this important requirement
we need to preserve legacyXLv1 format, this is simply
ignored and overwritten causing a myriad of issues
by leaving stale files on the namespace etc.
for now lets still use the two-phase approach of
writing to `tmp` and then renaming the content to
the actual namespace.
versionID is the one that needs to be preserved and as
well as overwritten in case of replication, transition
etc - dataDir is an ephemeral entity that changes
during overwrites - make sure that versionID is used
to save the object content.
this would break things if you are already running
the latest master, please wipe your current content
and re-do your setup after this change.
upgrading from 2yr old releases is expected to work,
the issue was we were missing checksum info to be
passed down to newBitrotReader() for whole bitrot
calculation
Ensure that we don't use potentially broken algorithms for critical functions, whether it be a runtime problem or implementation problem for a specific platform.
It is inefficient to decide to heal an object before checking its
lifecycle for expiration or transition. This commit will just reverse
the order of action: evaluate lifecycle and heal only if asked and
lifecycle resulted a NoneAction.
replication didn't work as expected when deletion of
delete markers was requested in DeleteMultipleObjects
API, this is due to incorrect lookup elements being
used to look for delete markers.
This allows us to speed up or slow down sleeps
between multiple scanner cycles, helps in testing
as well as some deployments might want to run
scanner more frequently.
This change is also dynamic can be applied on
a running cluster, subsequent cycles pickup
the newly set value.
using Lstat() is causing tiny memory allocations,
that are usually wasted and never used, instead
we can simply uses Access() call that does 0
memory allocations.
This feature brings in support for auto extraction
of objects onto MinIO's namespace from an incoming
tar gzipped stream, the only expected metadata sent
by the client is to set `snowball-auto-extract`.
All the contents from the tar stream are saved as
folders and objects on the namespace.
fixes#8715
service accounts were not inheriting parent policies
anymore due to refactors in the PolicyDBGet() from
the latest release, fix this behavior properly.
The local node name is heavily used in tracing, create a new global
variable to store it. Multiple goroutines can access it since it won't be
changed later.
In #11888 we observe a lot of running, WalkDir calls.
There doesn't appear to be any listerners for these calls, so they should be aborted.
Ensure that WalkDir aborts when upstream cancels the request.
Fixes#11888
The background healing can return NoSuchUpload error, the reason is that
healing code can return errFileNotFound with three parameters. Simplify
the code by returning exact errUploadNotFound error in multipart code.
Also ensure that a typed error is always returned whatever the number of
parameters because it is better than showing internal error.
For large objects taking more than '3 minutes' response
times in a single PUT operation can timeout prematurely
as 'ResponseHeader' timeout hits for 3 minutes. Avoid
this by keeping the connection active during CreateFile
phase.
baseDirFromPrefix(prefix) for object names without
parent directory incorrectly uses empty path, leading
to long listing at various paths that are not useful
for healing - avoid this listing completely if "baseDir"
returns empty simple use the "prefix" as is.
this improves startup performance significantly
For large objects taking more than '3 minutes' response
times in a single PUT operation can timeout prematurely
as 'ResponseHeader' timeout hits for 3 minutes. Avoid
this by keeping the connection active during CreateFile
phase.
some SDKs might incorrectly send duplicate
entries for keys such as "conditions", Go
stdlib unmarshal for JSON does not support
duplicate keys - instead skips the first
duplicate and only preserves the last entry.
This can lead to issues where a policy JSON
while being valid might not properly apply
the required conditions, allowing situations
where POST policy JSON would end up allowing
uploads to unauthorized buckets and paths.
This PR fixes this properly.
This commit adds a `MarshalText` implementation
to the `crypto.Context` type.
The `MarshalText` implementation replaces the
`WriteTo` and `AppendTo` implementation.
It is slightly slower than the `AppendTo` implementation
```
goos: darwin
goarch: arm64
pkg: github.com/minio/minio/cmd/crypto
BenchmarkContext_AppendTo/0-elems-8 381475698 2.892 ns/op 0 B/op 0 allocs/op
BenchmarkContext_AppendTo/1-elems-8 17945088 67.54 ns/op 0 B/op 0 allocs/op
BenchmarkContext_AppendTo/3-elems-8 5431770 221.2 ns/op 72 B/op 2 allocs/op
BenchmarkContext_AppendTo/4-elems-8 3430684 346.7 ns/op 88 B/op 2 allocs/op
```
vs.
```
BenchmarkContext/0-elems-8 135819834 8.658 ns/op 2 B/op 1 allocs/op
BenchmarkContext/1-elems-8 13326243 89.20 ns/op 128 B/op 1 allocs/op
BenchmarkContext/3-elems-8 4935301 243.1 ns/op 200 B/op 3 allocs/op
BenchmarkContext/4-elems-8 2792142 428.2 ns/op 504 B/op 4 allocs/op
goos: darwin
```
However, the `AppendTo` benchmark used a pre-allocated buffer. While
this improves its performance it does not match the actual usage of
`crypto.Context` which is passed to a `KMS` and always encoded into
a newly allocated buffer.
Therefore, this change seems acceptable since it should not impact the
actual performance but reduces the overall code for Context marshaling.
When an object is removed, its parent directory is inspected to check if
it is empty to remove if that is the case.
However, we can use os.Remove() directly since it is only able to remove
a file or an empty directory.
RenameData renames xl.meta and data dir and removes the parent directory
if empty, however, there is a duplicate check for empty dir, since the
parent dir of xl.meta is always the same as the data-dir.
on freshReads if drive returns errInvalidArgument, we
should simply turn-off DirectIO and read normally, there
are situations in k8s like environments where the drives
behave sporadically in a single deployment and may not
have been implemented properly to handle O_DIRECT for
reads.
This PR adds deadlines per Write() calls, such
that slow drives are timed-out appropriately and
the overall responsiveness for Writes() is always
up to a predefined threshold providing applications
sustained latency even if one of the drives is slow
to respond.
MRF was starting to heal when it receives a disk connection event, which
is not good when a node having multiple disks reconnects to the cluster.
Besides, MRF needs Remove healing option to remove stale files.
- write in o_dsync instead of o_direct for smaller
objects to avoid unaligned double Write() situations
that may arise for smaller objects < 128KiB
- avoid fallocate() as its not useful since we do not
use Append() semantics anymore, fallocate is not useful
for streaming I/O we can save on a syscall
- createFile() doesn't need to validate `bucket` name
with a Lstat() call since createFile() is only used
to write at `minioTmpBucket`
- use io.Copy() when writing unAligned writes to allow
usage of ReadFrom() from *os.File providing zero
buffer writes().
```
mc admin info --json
```
provides these details, for now, we shall eventually
expose this at Prometheus level eventually.
Co-authored-by: Harshavardhana <harsha@minio.io>
This commit fixes a security issue in the signature v4 chunked
reader. Before, the reader returned unverified data to the caller
and would only verify the chunk signature once it has encountered
the end of the chunk payload.
Now, the chunk reader reads the entire chunk into an in-memory buffer,
verifies the signature and then returns data to the caller.
In general, this is a common security problem. We verifying data
streams, the verifier MUST NOT return data to the upper layers / its
callers as long as it has not verified the current data chunk / data
segment:
```
func (r *Reader) Read(buffer []byte) {
if err := r.readNext(r.internalBuffer); err != nil {
return err
}
if err := r.verify(r.internalBuffer); err != nil {
return err
}
copy(buffer, r.internalBuffer)
}
```
For operations that require the object to exist make it possible to
detect if the file isn't found in *any* pool.
This will allow these to return the error early without having to re-check.
Cases where we have applications making request
for `//` in object names make sure that all
are normalized to `/` and all such requests that
are prefixed '/' are removed. To ensure a
consistent view from all operations.
Some deployments have low parity (EC:2), but we really do not need to
save our config data with the same parity configuration.
N/2 would be better to keep MinIO configurations intact when unexpected
a number of drives fail.
This commit disables the Hashicorp Vault
support but provides a way to temp. enable
it via the `MINIO_KMS_VAULT_DEPRECATION=off`
Vault support has been deprecated long ago
and this commit just requires users to take
action if they maintain a Vault integration.
major performance improvements in range GETs to avoid large
read amplification when ranges are tiny and random
```
-------------------
Operation: GET
Operations: 142014 -> 339421
Duration: 4m50s -> 4m56s
* Average: +139.41% (+1177.3 MiB/s) throughput, +139.11% (+658.4) obj/s
* Fastest: +125.24% (+1207.4 MiB/s) throughput, +132.32% (+612.9) obj/s
* 50% Median: +139.06% (+1175.7 MiB/s) throughput, +133.46% (+660.9) obj/s
* Slowest: +203.40% (+1267.9 MiB/s) throughput, +198.59% (+753.5) obj/s
```
TTFB from 10MiB BlockSize
```
* First Access TTFB: Avg: 81ms, Median: 61ms, Best: 20ms, Worst: 2.056s
```
TTFB from 1MiB BlockSize
```
* First Access TTFB: Avg: 22ms, Median: 21ms, Best: 8ms, Worst: 91ms
```
Full object reads however do see a slight change which won't be
noticeable in real world, so not doing any comparisons
TTFB still had improvements with full object reads with 1MiB
```
* First Access TTFB: Avg: 68ms, Median: 35ms, Best: 11ms, Worst: 1.16s
```
v/s
TTFB with 10MiB
```
* First Access TTFB: Avg: 388ms, Median: 98ms, Best: 20ms, Worst: 4.156s
```
This change should affect all new uploads, previous uploads should
continue to work with business as usual. But dramatic improvements can
be seen with these changes.