You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@couchdb.apache.org by rn...@apache.org on 2014/08/28 14:11:55 UTC

[19/50] couch commit: updated refs/heads/master to 9d0ac7d

Move attachment code into couch_att

This is moves a majority of the attachment representation
into the couch_att module. This serves to isolate the
current record to allow easier in-place upgrades as well as
a place to start collecting common attachment related
functionality.

The upgrades are handled lazily to allow rollbacks to older
code if the new attachment format has not yet been required
via storage of any of the new extended attributes supported
by the fetch/store APIs. There are some caveats to this in
that the extended attributes are not enforced by couch_att
at this time so it'd be quite easy to store garbage. As the
extent of attachment concerns becomes more stable, a set of
more permanent fetch_[field]/store_[field] functions may be
added to help enforce both field types as well as common
field names and defaults (all fields will default to
undefined except for those defaults present in the orignal
record definition, which carry over automatically).

Finally, while this patch does move a lot of code to
couch_att, it hasn't refined the interfaces much. These
changes will follow in later patches to improve and
simplify the organization of attachment code. This
includes the addition of more unit tests which currently
only cover some portions of the attachment functionality
related to upgrades and field fetching & storage.


Project: http://git-wip-us.apache.org/repos/asf/couchdb-couch/repo
Commit: http://git-wip-us.apache.org/repos/asf/couchdb-couch/commit/507e5acc
Tree: http://git-wip-us.apache.org/repos/asf/couchdb-couch/tree/507e5acc
Diff: http://git-wip-us.apache.org/repos/asf/couchdb-couch/diff/507e5acc

Branch: refs/heads/master
Commit: 507e5acce25e4640f2dc7a648d83fa030281aac1
Parents: c628f36
Author: Brian Mitchell <br...@p2p.io>
Authored: Wed Dec 11 23:07:50 2013 -0500
Committer: Robert Newson <rn...@apache.org>
Committed: Thu Aug 28 13:00:01 2014 +0100

----------------------------------------------------------------------
 include/couch_db.hrl     |  19 +-
 src/couch_att.erl        | 806 ++++++++++++++++++++++++++++++++++++++++++
 src/couch_db.erl         | 270 +++++---------
 src/couch_db_updater.erl |   2 +-
 src/couch_doc.erl        | 285 +++------------
 src/couch_httpd_db.erl   | 138 ++++----
 6 files changed, 1019 insertions(+), 501 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/couchdb-couch/blob/507e5acc/include/couch_db.hrl
----------------------------------------------------------------------
diff --git a/include/couch_db.hrl b/include/couch_db.hrl
index 570d3db..8bb44a0 100644
--- a/include/couch_db.hrl
+++ b/include/couch_db.hrl
@@ -89,7 +89,7 @@
     % the json body object.
     body = {[]},
 
-    atts = [], % attachments
+    atts = [] :: [couch_att:att()], % attachments
 
     deleted = false,
 
@@ -99,23 +99,6 @@
 }).
 
 
--record(att, {
-    name,
-    type,
-    att_len,
-    disk_len, % length of the attachment in its identity form
-              % (that is, without a content encoding applied to it)
-              % differs from att_len when encoding /= identity
-    md5= <<>>,
-    revpos=0,
-    data,
-    encoding=identity % currently supported values are:
-                      %     identity, gzip
-                      % additional values to support in the future:
-                      %     deflate, compress
-}).
-
-
 -record(user_ctx, {
     name=null,
     roles=[],

http://git-wip-us.apache.org/repos/asf/couchdb-couch/blob/507e5acc/src/couch_att.erl
----------------------------------------------------------------------
diff --git a/src/couch_att.erl b/src/couch_att.erl
new file mode 100644
index 0000000..e011402
--- /dev/null
+++ b/src/couch_att.erl
@@ -0,0 +1,806 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(couch_att).
+
+-export([
+    new/0,
+    new/1,
+    fetch/2,
+    store/2,
+    store/3,
+    transform/3
+]).
+
+-export([
+    is_stub/1,
+    merge_stubs/2
+]).
+
+-export([
+    disk_info/2,
+    to_disk_term/1,
+    from_disk_term/2
+]).
+
+-export([
+    from_json/2,
+    to_json/4
+]).
+
+-export([
+    flush/2,
+    foldl/3,
+    range_foldl/5,
+    foldl_decode/3,
+    to_binary/1
+]).
+
+-export([
+    upgrade/1,
+    downgrade/1
+]).
+
+-export_type([att/0]).
+
+-include_lib("couch/include/couch_db.hrl").
+
+
+%% Legacy attachment record. This is going to be phased out by the new proplist
+%% based structure. It's needed for now to allow code to perform lazy upgrades
+%% while the patch is rolled out to the cluster. Attachments passed as records
+%% will remain so until they are required to be represented as property lists.
+%% Once this has been widely deployed, this record will be removed entirely and
+%% property lists will be the main format.
+-record(att, {
+    name :: binary(),
+    type :: binary(),
+    att_len :: non_neg_integer(),
+
+    %% length of the attachment in its identity form
+    %% (that is, without a content encoding applied to it)
+    %% differs from att_len when encoding /= identity
+    disk_len :: non_neg_integer(),
+
+    md5 = <<>> :: binary(),
+    revpos = 0 :: non_neg_integer(),
+    data :: stub | follows | binary() | {any(), any()} |
+            {follows, pid(), reference()} | fun(() -> binary()),
+
+    %% Encoding of the attachment
+    %% currently supported values are:
+    %%     identity, gzip
+    %% additional values to support in the future:
+    %%     deflate, compress
+    encoding = identity :: identity | gzip
+}).
+
+
+%% Extensible Attachment Type
+%%
+%% The following types describe the known properties for attachment fields
+%% encoded as property lists to allow easier upgrades. Values not in this list
+%% should be accepted at runtime but should be treated as opaque data as might
+%% be used by upgraded code. If you plan on operating on new data, please add
+%% an entry here as documentation.
+
+
+%% The name of the attachment is also used as the mime-part name for file
+%% downloads. These must be unique per document.
+-type name_prop() :: {name, binary()}.
+
+
+%% The mime type of the attachment. This does affect compression of certain
+%% attachments if the type is found to be configured as a compressable type.
+%% This is commonly reserved for text/* types but could include other custom
+%% cases as well. See definition and use of couch_util:compressable_att_type/1.
+-type type_prop() :: {type, binary()}.
+
+
+%% The attachment length is similar to disk-length but ignores additional
+%% encoding that may have occurred.
+-type att_len_prop() :: {att_len, non_neg_integer()}.
+
+
+%% The size of the attachment as stored in a disk stream.
+-type disk_len_prop() :: {disk_len, non_neg_integer()}.
+
+
+%% This is a digest of the original attachment data as uploaded by the client.
+%% it's useful for checking validity of contents against other attachment data
+%% as well as quick digest computation of the enclosing document.
+-type md5_prop() :: {md5, binary()}.
+
+
+-type revpos_prop() :: {revpos, 0}.
+
+
+%% This field is currently overloaded with just about everything. The
+%% {any(), any()} type is just there until I have time to check the actual
+%% values expected. Over time this should be split into more than one property
+%% to allow simpler handling.
+-type data_prop() :: {
+    data, stub | follows | binary() | {any(), any()} |
+    {follows, pid(), reference()} | fun(() -> binary())
+}.
+
+
+%% We will occasionally compress our data. See type_prop() for more information
+%% on when this happens.
+-type encoding_prop() :: {encoding, identity | gzip}.
+
+
+-type attachment() :: [
+    name_prop() | type_prop() |
+    att_len_prop() | disk_len_prop() |
+    md5_prop() | revpos_prop() |
+    data_prop() | encoding_prop()
+].
+
+
+-opaque att() :: #att{} | attachment().
+
+
+new() ->
+    %% We construct a record by default for compatability. This will be
+    %% upgraded on demand. A subtle effect this has on all attachments
+    %% constructed via new is that it will pick up the proper defaults
+    %% from the #att record definition given above. Newer properties do
+    %% not support special default values and will all be treated as
+    %% undefined.
+    #att{}.
+
+
+-spec new([{atom(), any()}]) -> att().
+new(Props) ->
+    store(Props, new()).
+
+
+-spec fetch([atom()], att()) -> [any()];
+           (atom(), att()) -> any().
+fetch(Fields, Att) when is_list(Fields) ->
+    [fetch(Field, Att) || Field <- Fields];
+fetch(Field, Att) when is_list(Att) ->
+    case lists:keyfind(Field, 1, Att) of
+        {Field, Value} -> Value;
+        false -> undefined
+    end;
+fetch(name, #att{name = Name}) ->
+    Name;
+fetch(type, #att{type = Type}) ->
+    Type;
+fetch(att_len, #att{att_len = AttLen}) ->
+    AttLen;
+fetch(disk_len, #att{disk_len = DiskLen}) ->
+    DiskLen;
+fetch(md5, #att{md5 = Digest}) ->
+    Digest;
+fetch(revpos, #att{revpos = RevPos}) ->
+    RevPos;
+fetch(data, #att{data = Data}) ->
+    Data;
+fetch(encoding, #att{encoding = Encoding}) ->
+    Encoding;
+fetch(_, _) ->
+    undefined.
+
+
+-spec store([{atom(), any()}], att()) -> att().
+store(Props, Att0) ->
+    lists:foldl(fun({Field, Value}, Att) ->
+        store(Field, Value, Att)
+    end, Att0, Props).
+
+
+-spec store(atom(), any(), att()) -> att().
+store(Field, undefined, Att) when is_list(Att) ->
+    lists:keydelete(Field, 1, Att);
+store(Field, Value, Att) when is_list(Att) ->
+    lists:keystore(Field, 1, Att, {Field, Value});
+store(name, Name, Att) ->
+    Att#att{name = Name};
+store(type, Type, Att) ->
+    Att#att{type = Type};
+store(att_len, AttLen, Att) ->
+    Att#att{att_len = AttLen};
+store(disk_len, DiskLen, Att) ->
+    Att#att{disk_len = DiskLen};
+store(md5, Digest, Att) ->
+    Att#att{md5 = Digest};
+store(revpos, RevPos, Att) ->
+    Att#att{revpos = RevPos};
+store(data, Data, Att) ->
+    Att#att{data = Data};
+store(encoding, Encoding, Att) ->
+    Att#att{encoding = Encoding};
+store(Field, Value, Att) ->
+    store(Field, Value, upgrade(Att)).
+
+
+-spec transform(atom(), fun(), att()) -> att().
+transform(Field, Fun, Att) ->
+    NewValue = Fun(fetch(Field, Att)),
+    store(Field, NewValue, Att).
+
+
+is_stub(Att) ->
+    stub == fetch(data, Att).
+
+
+%% merge_stubs takes all stub attachments and replaces them with on disk
+%% attachments. It will return {missing, Name} if a stub isn't matched with
+%% an existing attachment on disk. If the revpos is supplied with the stub
+%% it is also only counted to match if is the same as the disk attachment.
+merge_stubs(MemAtts, DiskAtts) ->
+    OnDisk = dict:from_list(
+        [{fetch(name, Att), Att} || Att <- DiskAtts]
+    ),
+    merge_stubs(MemAtts, OnDisk, []).
+
+
+-spec merge_stubs([att()], dict:dict(), [att()]) -> [att()].
+merge_stubs([Att | Rest], OnDisk, Merged) ->
+    case fetch(data, Att) of
+        stub ->
+            [Name, Pos] = fetch([name, revpos], Att),
+            case dict:find(Name, OnDisk) of
+                {ok, DiskAtt} ->
+                    RevPos = fetch(revpos, DiskAtt),
+                    if
+                        %% We want to check for consistency between the stub and
+                        %% disk revpos here. If the stub's revpos is undefined
+                        %% it means it wasn't provided by the user and does not
+                        %% require being matched.
+                        RevPos == Pos orelse Pos == undefined ->
+                            merge_stubs(Rest, OnDisk, [DiskAtt | Merged]);
+                        true ->
+                            {missing, Name}
+                    end;
+                _ ->
+                    {missing, Name}
+            end;
+        _ ->
+            merge_stubs(Rest, OnDisk, [Att | Merged])
+    end;
+merge_stubs([], _, Merged) ->
+    {ok, Merged}.
+
+
+disk_info(_, []) ->
+    {ok, [], []};
+disk_info(ActiveFd, Atts) ->
+    {AttFd, _} = fetch(data, hd(Atts)),
+    if
+        AttFd == ActiveFd ->
+            Tuples = [to_disk_term(Att) || Att <- Atts],
+            Info = lists:map(fun(Att) ->
+                [{_, Pos}, AttLen] = fetch([data, att_len], Att),
+                {Pos, AttLen}
+            end, Atts),
+            {ok, Tuples, Info};
+        true ->
+            ?LOG_ERROR("MISMATCH: ~p ; ~p~n", [ActiveFd, Atts]),
+            file_mismatch
+    end.
+
+
+%% When converting an attachment to disk term format, attempt to stay with the
+%% old format when possible. This should help make the attachment lazy upgrade
+%% as safe as possible, avoiding the need for complicated disk versioning
+%% schemes.
+to_disk_term(#att{} = Att) ->
+    {_, StreamIndex} = fetch(data, Att),
+    {
+        fetch(name, Att),
+        fetch(type, Att),
+        StreamIndex,
+        fetch(att_len, Att),
+        fetch(disk_len, Att),
+        fetch(revpos, Att),
+        fetch(md5, Att),
+        fetch(encoding, Att)
+    };
+to_disk_term(Att) ->
+    BaseProps = [name, type, data, att_len, disk_len, revpos, md5, encoding],
+    {Extended, Base} = lists:foldl(
+        fun
+            (data, {Props, Values}) ->
+                case lists:keytake(data, 1, Props) of
+                    {value, {_, {_Fd, Sp}}, Other} -> {Other, [Sp | Values]};
+                    {value, {_, Value}, Other} -> {Other, [Value | Values]};
+                    false -> {Props, [undefined |Values ]}
+                end;
+            (Key, {Props, Values}) ->
+                case lists:keytake(Key, 1, Props) of
+                    {value, {_, Value}, Other} -> {Other, [Value | Values]};
+                    false -> {Props, [undefined | Values]}
+                end
+        end,
+        {Att, []},
+        BaseProps
+    ),
+    {list_to_tuple(lists:reverse(Base)), Extended}.
+
+
+%% The new disk term format is a simple wrapper around the legacy format. Base
+%% properties will remain in a tuple while the new fields and possibly data from
+%% future extensions will be stored in a list of atom/value pairs. While this is
+%% slightly less efficient, future work should be able to make use of
+%% compression to remove these sorts of common bits (block level compression
+%% with something like a shared dictionary that is checkpointed every now and
+%% then).
+from_disk_term(Fd, {Base, Extended}) when is_tuple(Base), is_list(Extended) ->
+    store(Extended, from_disk_term(Fd, Base));
+from_disk_term(Fd, {Name,Type,Sp,AttLen,DiskLen,RevPos,Md5,Enc}) ->
+    #att{
+        name=Name,
+        type=Type,
+        att_len=AttLen,
+        disk_len=DiskLen,
+        md5=Md5,
+        revpos=RevPos,
+        data={Fd,Sp},
+        encoding=upgrade_encoding(Enc)
+    };
+from_disk_term(Fd, {Name,Type,Sp,AttLen,RevPos,Md5}) ->
+    #att{
+        name=Name,
+        type=Type,
+        att_len=AttLen,
+        disk_len=AttLen,
+        md5=Md5,
+        revpos=RevPos,
+        data={Fd,Sp}
+    };
+from_disk_term(Fd, {Name,{Type,Sp,AttLen}}) ->
+    #att{
+        name=Name,
+        type=Type,
+        att_len=AttLen,
+        disk_len=AttLen,
+        md5= <<>>,
+        revpos=0,
+        data={Fd,Sp}
+    }.
+
+
+%% from_json reads in embedded JSON attachments and creates usable attachment
+%% values. The attachment may be a stub,
+from_json(Name, Props) ->
+    Type = couch_util:get_value(
+        <<"content_type">>, Props, ?DEFAULT_ATTACHMENT_CONTENT_TYPE
+    ),
+    Att = new([{name, Name}, {type, Type}]),
+    IsStub = couch_util:get_value(<<"stub">>, Props),
+    Follows = couch_util:get_value(<<"follows">>, Props),
+    if
+        IsStub -> stub_from_json(Att, Props);
+        Follows -> follow_from_json(Att, Props);
+        true -> inline_from_json(Att, Props)
+    end.
+
+
+stub_from_json(Att, Props) ->
+    {DiskLen, EncodedLen, Encoding} = encoded_lengths_from_json(Props),
+    Digest = digest_from_json(Props),
+    %% We specifically want undefined rather than the default 0 here to skip
+    %% the revpos consistency check on stubs when it's not provided in the
+    %% json object. See merge_stubs/3 for the stub check.
+    RevPos = couch_util:get_value(<<"revpos">>, Props),
+    store([
+        {md5, Digest}, {revpos, RevPos}, {data, stub}, {disk_len, DiskLen},
+        {att_len, EncodedLen}, {encoding, Encoding}
+    ], Att).
+
+
+follow_from_json(Att, Props) ->
+    {DiskLen, EncodedLen, Encoding} = encoded_lengths_from_json(Props),
+    Digest = digest_from_json(Props),
+    store([
+        {md5, Digest}, {data, follows}, {disk_len, DiskLen},
+        {att_len, EncodedLen}, {encoding, Encoding}
+    ], Att).
+
+
+inline_from_json(Att, Props) ->
+    B64Data = couch_util:get_value(<<"data">>, Props),
+    Data = base64:decode(B64Data),
+    Length = size(Data),
+    store([{data, Data}, {disk_len, Length}, {att_len, Length}], Att).
+
+
+encoded_lengths_from_json(Props) ->
+    Len = couch_util:get_value(<<"length">>, Props),
+    case couch_util:get_value(<<"encoding">>, Props) of
+        undefined ->
+            Encoding = identity,
+            EncodedLen = Len;
+        EncodingValue ->
+            EncodedLen = couch_util:get_value(<<"encoded_length">>, Props, Len),
+            Encoding = list_to_existing_atom(binary_to_list(EncodingValue))
+    end,
+    {Len, EncodedLen, Encoding}.
+
+
+digest_from_json(Props) ->
+    case couch_util:get_value(<<"digest">>, Props) of
+        <<"md5-", EncodedMd5/binary>> -> base64:decode(EncodedMd5);
+        _ -> <<>>
+    end.
+
+
+to_json(Att, OutputData, DataToFollow, ShowEncoding) ->
+    [Name, Data, DiskLen, AttLen, Enc, Type, RevPos, Md5] = fetch(
+        [name, data, disk_len, att_len, encoding, type, revpos, md5], Att
+    ),
+    Props = [
+        {<<"content_type">>, Type},
+        {<<"revpos">>, RevPos}
+    ],
+    DigestProp = case base64:encode(Md5) of
+        <<>> -> [];
+        Digest -> [{<<"digest">>, <<"md5-", Digest/binary>>}]
+    end,
+    DataProps = if
+        not OutputData orelse Data == stub ->
+            [{<<"length">>, DiskLen}, {<<"stub">>, true}];
+        DataToFollow ->
+            [{<<"length">>, DiskLen}, {<<"follows">>, true}];
+        true ->
+            AttData = case Enc of
+                gzip -> zlib:gunzip(to_binary(Att));
+                identity -> to_binary(Att)
+            end,
+            [{<<"data">>, base64:encode(AttData)}]
+    end,
+    EncodingProps = if
+        ShowEncoding andalso Enc /= identity ->
+            [
+                {<<"encoding">>, couch_util:to_binary(Enc)},
+                {<<"encoded_length">>, AttLen}
+            ];
+        true ->
+            []
+    end,
+    HeadersProp = case fetch(headers, Att) of
+        undefined -> [];
+        Headers -> [{<<"headers">>, Headers}]
+    end,
+    {Name, {Props ++ DigestProp ++ DataProps ++ EncodingProps ++ HeadersProp}}.
+
+
+flush(Fd, Att) ->
+    flush_data(Fd, fetch(data, Att), Att).
+
+
+flush_data(Fd, {Fd0, _}, Att) when Fd0 == Fd ->
+    % already written to our file, nothing to write
+    Att;
+flush_data(Fd, {OtherFd, StreamPointer}, Att) ->
+    [InMd5, InDiskLen] = fetch([md5, disk_len], Att),
+    {NewStreamData, Len, _IdentityLen, Md5, IdentityMd5} =
+        couch_stream:copy_to_new_stream(OtherFd, StreamPointer, Fd),
+    couch_db:check_md5(IdentityMd5, InMd5),
+    store([
+        {data, {Fd, NewStreamData}},
+        {md5, Md5},
+        {att_len, Len},
+        {disk_len, InDiskLen}
+    ], Att);
+flush_data(Fd, Data, Att) when is_binary(Data) ->
+    couch_db:with_stream(Fd, Att, fun(OutputStream) ->
+        couch_stream:write(OutputStream, Data)
+    end);
+flush_data(Fd, Fun, Att) when is_function(Fun) ->
+    case fetch(att_len, Att) of
+        undefined ->
+            couch_db:with_stream(Fd, Att, fun(OutputStream) ->
+                % Fun(MaxChunkSize, WriterFun) must call WriterFun
+                % once for each chunk of the attachment,
+                Fun(4096,
+                    % WriterFun({Length, Binary}, State)
+                    % WriterFun({0, _Footers}, State)
+                    % Called with Length == 0 on the last time.
+                    % WriterFun returns NewState.
+                    fun({0, Footers}, _) ->
+                        F = mochiweb_headers:from_binary(Footers),
+                        case mochiweb_headers:get_value("Content-MD5", F) of
+                        undefined ->
+                            ok;
+                        Md5 ->
+                            {md5, base64:decode(Md5)}
+                        end;
+                    ({_Length, Chunk}, _) ->
+                        couch_stream:write(OutputStream, Chunk)
+                    end, ok)
+            end);
+        AttLen ->
+            couch_db:with_stream(Fd, Att, fun(OutputStream) ->
+                write_streamed_attachment(OutputStream, Fun, AttLen)
+            end)
+    end.
+
+
+write_streamed_attachment(_Stream, _F, 0) ->
+    ok;
+write_streamed_attachment(_Stream, _F, LenLeft) when LenLeft < 0 ->
+    throw({bad_request, <<"attachment longer than expected">>});
+write_streamed_attachment(Stream, F, LenLeft) when LenLeft > 0 ->
+    Bin = try read_next_chunk(F, LenLeft)
+    catch
+        {mp_parser_died, normal} ->
+            throw({bad_request, <<"attachment shorter than expected">>})
+    end,
+    ok = couch_stream:write(Stream, Bin),
+    write_streamed_attachment(Stream, F, LenLeft - size(Bin)).
+
+read_next_chunk(F, _) when is_function(F, 0) ->
+    F();
+read_next_chunk(F, LenLeft) when is_function(F, 1) ->
+    F(lists:min([LenLeft, 16#2000])).
+
+
+foldl(Att, Fun, Acc) ->
+    foldl(fetch(data, Att), Att, Fun, Acc).
+
+
+foldl(Bin, _Att, Fun, Acc) when is_binary(Bin) ->
+    Fun(Bin, Acc);
+foldl({Fd, Sp}, Att, Fun, Acc) when is_tuple(Sp) orelse Sp == null ->
+    % 09 UPGRADE CODE
+    Len = fetch(att_len, Att),
+    couch_stream:old_foldl(Fd, Sp, Len, Fun, Acc);
+foldl({Fd, Sp}, Att, Fun, Acc) ->
+    Md5 = fetch(md5, Att),
+    couch_stream:foldl(Fd, Sp, Md5, Fun, Acc);
+foldl(DataFun, Att, Fun, Acc) when is_function(DataFun) ->
+    Len = fetch(att_len, Att),
+    fold_streamed_data(DataFun, Len, Fun, Acc).
+
+
+range_foldl(Att, From, To, Fun, Acc) ->
+    {Fd, Sp} = fetch(data, Att),
+    couch_stream:range_foldl(Fd, Sp, From, To, Fun, Acc).
+
+
+foldl_decode(Att, Fun, Acc) ->
+    case fetch([data, encoding], Att) of
+        [{Fd, Sp}, Enc] ->
+            couch_stream:foldl_decode(Fd, Sp, fetch(md5, Att), Enc, Fun, Acc);
+        [Fun2, identity] ->
+            fold_streamed_data(Fun2, fetch(att_len, Att), Fun, Acc)
+    end.
+
+
+to_binary(Att) ->
+    to_binary(fetch(data, Att), Att).
+
+
+to_binary(Bin, _Att) when is_binary(Bin) ->
+    Bin;
+to_binary(Iolist, _Att) when is_list(Iolist) ->
+    iolist_to_binary(Iolist);
+to_binary({_Fd,_Sp}, Att) ->
+    iolist_to_binary(
+        lists:reverse(foldl(Att, fun(Bin,Acc) -> [Bin|Acc] end, []))
+    );
+to_binary(DataFun, Att) when is_function(DataFun)->
+    Len = fetch(att_len, Att),
+    iolist_to_binary(
+        lists:reverse(fold_streamed_data(
+            DataFun,
+            Len,
+            fun(Data, Acc) -> [Data | Acc] end,
+            []
+        ))
+    ).
+
+
+fold_streamed_data(_RcvFun, 0, _Fun, Acc) ->
+    Acc;
+fold_streamed_data(RcvFun, LenLeft, Fun, Acc) when LenLeft > 0->
+    Bin = RcvFun(),
+    ResultAcc = Fun(Bin, Acc),
+    fold_streamed_data(RcvFun, LenLeft - size(Bin), Fun, ResultAcc).
+
+
+%% Upgrade an attachment record to a property list on demand. This is a one-way
+%% operation as downgrading potentially truncates fields with important data.
+-spec upgrade(#att{}) -> attachment().
+upgrade(#att{} = Att) ->
+    Map = lists:zip(
+        record_info(fields, att),
+        lists:seq(2, record_info(size, att))
+    ),
+    %% Don't store undefined elements since that is default
+    [{F, element(I, Att)} || {F, I} <- Map, element(I, Att) /= undefined];
+upgrade(Att) ->
+    Att.
+
+
+%% Downgrade is exposed for interactive convenience. In practice, unless done
+%% manually, upgrades are always one-way.
+downgrade(#att{} = Att) ->
+    Att;
+downgrade(Att) ->
+    #att{
+        name = fetch(name, Att),
+        type = fetch(type, Att),
+        att_len = fetch(att_len, Att),
+        disk_len = fetch(disk_len, Att),
+        md5 = fetch(md5, Att),
+        revpos = fetch(revpos, Att),
+        data = fetch(data, Att),
+        encoding = fetch(encoding, Att)
+    }.
+
+
+upgrade_encoding(true) -> gzip;
+upgrade_encoding(false) -> identity;
+upgrade_encoding(Encoding) -> Encoding.
+
+
+-ifdef(TEST).
+-include_lib("eunit/include/eunit.hrl").
+
+
+%% Test utilities
+
+
+empty_att() -> new().
+
+
+upgraded_empty_att() ->
+    new([{headers, undefined}]).
+
+
+%% Test groups
+
+
+attachment_upgrade_test_() ->
+    {"Lazy record upgrade tests", [
+        {"Existing record fields don't upgrade",
+            {with, empty_att(), [fun test_non_upgrading_fields/1]}
+        },
+        {"New fields upgrade",
+            {with, empty_att(), [fun test_upgrading_fields/1]}
+        }
+    ]}.
+
+
+attachment_defaults_test_() ->
+    {"Attachment defaults tests", [
+        {"Records retain old default values", [
+            {with, empty_att(), [fun test_legacy_defaults/1]}
+        ]},
+        {"Upgraded records inherit defaults", [
+            {with, upgraded_empty_att(), [fun test_legacy_defaults/1]}
+        ]},
+        {"Undefined entries are elided on upgrade", [
+            {with, upgraded_empty_att(), [fun test_elided_entries/1]}
+        ]}
+    ]}.
+
+attachment_field_api_test_() ->
+    {"Basic attachment field api", [
+        fun test_construction/0,
+        fun test_store_and_fetch/0,
+        fun test_transform/0
+    ]}.
+
+
+attachment_disk_term_test_() ->
+    BaseAttachment = new([
+        {name, <<"empty">>},
+        {type, <<"application/octet-stream">>},
+        {att_len, 0},
+        {disk_len, 0},
+        {md5, <<212,29,140,217,143,0,178,4,233,128,9,152,236,248,66,126>>},
+        {revpos, 4},
+        {data, {fake_fd, fake_sp}},
+        {encoding, identity}
+    ]),
+    BaseDiskTerm = {
+        <<"empty">>,
+        <<"application/octet-stream">>,
+        fake_sp,
+        0, 0, 4,
+        <<212,29,140,217,143,0,178,4,233,128,9,152,236,248,66,126>>,
+        identity
+    },
+    Headers = [{<<"X-Foo">>, <<"bar">>}],
+    ExtendedAttachment = store(headers, Headers, BaseAttachment),
+    ExtendedDiskTerm = {BaseDiskTerm, [{headers, Headers}]},
+    {"Disk term tests", [
+        ?_assertEqual(BaseDiskTerm, to_disk_term(BaseAttachment)),
+        ?_assertEqual(BaseAttachment, from_disk_term(fake_fd, BaseDiskTerm)),
+        ?_assertEqual(ExtendedDiskTerm, to_disk_term(ExtendedAttachment)),
+        ?_assertEqual(ExtendedAttachment, from_disk_term(fake_fd, ExtendedDiskTerm))
+    ]}.
+
+
+attachment_json_term_test_() ->
+    %% We need to create a few variations including stubs and inline data.
+    {"JSON term tests", []}.
+
+
+attachment_stub_merge_test_() ->
+    %% Stub merging needs to demonstrate revpos matching, skipping, and missing
+    %% attachment errors.
+    {"Attachment stub merging tests", []}.
+
+
+%% Test generators
+
+
+test_non_upgrading_fields(Attachment) ->
+    Pairs = [
+        {name, "cat.gif"},
+        {type, "text/very-very-plain"},
+        {att_len, 1024},
+        {disk_len, 42},
+        {md5, <<"md5-hashhashhash">>},
+        {revpos, 4},
+        {data, stub},
+        {encoding, gzip}
+    ],
+    lists:foreach(
+        fun({Field, Value}) ->
+            ?assertMatch(#att{}, Attachment),
+            Updated = store(Field, Value, Attachment),
+            ?assertMatch(#att{}, Updated)
+        end,
+    Pairs).
+
+
+test_upgrading_fields(Attachment) ->
+    ?assertMatch(#att{}, Attachment),
+    UpdatedHeaders = store(headers, [{<<"Ans">>, <<"42">>}], Attachment),
+    ?assertMatch(X when is_list(X), UpdatedHeaders),
+    UpdatedHeadersUndefined = store(headers, undefined, Attachment),
+    ?assertMatch(X when is_list(X), UpdatedHeadersUndefined).
+
+
+test_legacy_defaults(Attachment) ->
+    ?assertEqual(<<>>, fetch(md5, Attachment)),
+    ?assertEqual(0, fetch(revpos, Attachment)),
+    ?assertEqual(identity, fetch(encoding, Attachment)).
+
+
+test_elided_entries(Attachment) ->
+    ?assertNot(lists:keymember(name, 1, Attachment)),
+    ?assertNot(lists:keymember(type, 1, Attachment)),
+    ?assertNot(lists:keymember(att_len, 1, Attachment)),
+    ?assertNot(lists:keymember(disk_len, 1, Attachment)),
+    ?assertNot(lists:keymember(data, 1, Attachment)).
+
+
+test_construction() ->
+    ?assert(new() == new()),
+    Initialized = new([{name, <<"foo.bar">>}, {type, <<"application/qux">>}]),
+    ?assertEqual(<<"foo.bar">>, fetch(name, Initialized)),
+    ?assertEqual(<<"application/qux">>, fetch(type, Initialized)).
+
+
+test_store_and_fetch() ->
+    Attachment = empty_att(),
+    ?assertEqual(<<"abc">>, fetch(name, store(name, <<"abc">>, Attachment))),
+    ?assertEqual(42, fetch(ans, store(ans, 42, Attachment))).
+
+
+test_transform() ->
+    Attachment = new([{counter, 0}]),
+    Transformed = transform(counter, fun(Count) -> Count + 1 end, Attachment),
+    ?assertEqual(1, fetch(counter, Transformed)).
+
+
+-endif.

http://git-wip-us.apache.org/repos/asf/couchdb-couch/blob/507e5acc/src/couch_db.erl
----------------------------------------------------------------------
diff --git a/src/couch_db.erl b/src/couch_db.erl
index c9dc8e4..8a9f9e4 100644
--- a/src/couch_db.erl
+++ b/src/couch_db.erl
@@ -31,6 +31,7 @@
 -export([check_is_admin/1, check_is_member/1, get_doc_count/1]).
 -export([reopen/1, is_system_db/1, compression/1, make_doc/5]).
 -export([load_validation_funs/1]).
+-export([check_md5/2, with_stream/3]).
 
 -include_lib("couch/include/couch_db.hrl").
 
@@ -191,12 +192,16 @@ apply_open_options(Else,_Options) ->
 
 apply_open_options2(Doc,[]) ->
     {ok, Doc};
-apply_open_options2(#doc{atts=Atts,revs=Revs}=Doc,
+apply_open_options2(#doc{atts=Atts0,revs=Revs}=Doc,
         [{atts_since, PossibleAncestors}|Rest]) ->
     RevPos = find_ancestor_rev_pos(Revs, PossibleAncestors),
-    apply_open_options2(Doc#doc{atts=[A#att{data=
-        if AttPos>RevPos -> Data; true -> stub end}
-        || #att{revpos=AttPos,data=Data}=A <- Atts]}, Rest);
+    Atts = lists:map(fun(Att) ->
+        [AttPos, Data] = couch_att:fetch([revpos, data], Att),
+        if  AttPos > RevPos -> couch_att:store(data, Data, Att);
+            true -> couch_att:store(data, stub, Att)
+        end
+    end, Atts0),
+    apply_open_options2(Doc#doc{atts=Atts}, Rest);
 apply_open_options2(Doc, [ejson_body | Rest]) ->
     apply_open_options2(couch_doc:with_ejson_body(Doc), Rest);
 apply_open_options2(Doc,[_|Rest]) ->
@@ -784,15 +789,21 @@ prep_and_validate_replicated_updates(Db, [Bucket|RestBuckets], [OldInfo|RestOldI
 
 
 
-new_revid(#doc{body=Body,revs={OldStart,OldRevs},
-        atts=Atts,deleted=Deleted}) ->
-    case [{N, T, M} || #att{name=N,type=T,md5=M} <- Atts, M =/= <<>>] of
-    Atts2 when length(Atts) =/= length(Atts2) ->
-        % We must have old style non-md5 attachments
-        ?l2b(integer_to_list(couch_util:rand32()));
-    Atts2 ->
-        OldRev = case OldRevs of [] -> 0; [OldRev0|_] -> OldRev0 end,
-        couch_util:md5(term_to_binary([Deleted, OldStart, OldRev, Body, Atts2]))
+new_revid(#doc{body=Body, revs={OldStart,OldRevs}, atts=Atts, deleted=Deleted}) ->
+    DigestedAtts = lists:foldl(fun(Att, Acc) ->
+        [N, T, M] = couch_att:fetch([name, type, md5], Att),
+        case M == <<>> of
+            true -> Acc;
+            false -> [{N, T, M} | Acc]
+        end
+    end, [], Atts),
+    case DigestedAtts of
+        Atts2 when length(Atts) =/= length(Atts2) ->
+            % We must have old style non-md5 attachments
+            ?l2b(integer_to_list(couch_util:rand32()));
+        Atts2 ->
+            OldRev = case OldRevs of [] -> 0; [OldRev0|_] -> OldRev0 end,
+            couch_util:md5(term_to_binary([Deleted, OldStart, OldRev, Body, Atts2]))
     end.
 
 new_revs([], OutBuckets, IdRevsAcc) ->
@@ -807,18 +818,15 @@ new_revs([Bucket|RestBuckets], OutBuckets, IdRevsAcc) ->
     new_revs(RestBuckets, [NewBucket|OutBuckets], IdRevsAcc3).
 
 check_dup_atts(#doc{atts=Atts}=Doc) ->
-    Atts2 = lists:sort(fun(#att{name=N1}, #att{name=N2}) -> N1 < N2 end, Atts),
-    check_dup_atts2(Atts2),
+    lists:foldl(fun(Att, Names) ->
+        Name = couch_att:fetch(name, Att),
+        case ordsets:is_element(Name, Names) of
+            true -> throw({bad_request, <<"Duplicate attachments">>});
+            false -> ordsets:add_element(Name, Names)
+        end
+    end, ordsets:new(), Atts),
     Doc.
 
-check_dup_atts2([#att{name=N}, #att{name=N} | _]) ->
-    throw({bad_request, <<"Duplicate attachments">>});
-check_dup_atts2([_ | Rest]) ->
-    check_dup_atts2(Rest);
-check_dup_atts2(_) ->
-    ok.
-
-
 tag_docs([]) ->
     [];
 tag_docs([#doc{meta=Meta}=Doc | Rest]) ->
@@ -1000,11 +1008,10 @@ write_and_commit(#db{main_pid=Pid, user_ctx=Ctx}=Db, DocBuckets1,
 prepare_doc_summaries(Db, BucketList) ->
     [lists:map(
         fun(#doc{body = Body, atts = Atts} = Doc) ->
-            DiskAtts = [{N, T, P, AL, DL, R, M, E} ||
-                #att{name = N, type = T, data = {_, P}, md5 = M, revpos = R,
-                    att_len = AL, disk_len = DL, encoding = E} <- Atts],
+            DiskAtts = [couch_att:to_disk_term(Att) || Att <- Atts],
             AttsFd = case Atts of
-            [#att{data = {Fd, _}} | _] ->
+            [Att | _] ->
+                {Fd, _} = couch_att:fetch(data, Att),
                 Fd;
             [] ->
                 nil
@@ -1025,86 +1032,24 @@ before_docs_update(#db{before_doc_update = Fun} = Db, BucketList) ->
         Bucket) || Bucket <- BucketList].
 
 
-set_new_att_revpos(#doc{revs={RevPos,_Revs},atts=Atts}=Doc) ->
-    Doc#doc{atts= lists:map(fun(#att{data={_Fd,_Sp}}=Att) ->
-            % already commited to disk, do not set new rev
-            Att;
-        (Att) ->
-            Att#att{revpos=RevPos+1}
-        end, Atts)}.
+set_new_att_revpos(#doc{revs={RevPos,_Revs},atts=Atts0}=Doc) ->
+    Atts = lists:map(
+        fun(Att) ->
+            case couch_att:fetch(data, Att) of
+                {_Fd, _Sp} -> Att; % already commited to disk, don't set new rev
+                _ -> couch_att:store(revpos, RevPos+1, Att)
+            end
+        end, Atts0),
+    Doc#doc{atts = Atts}.
 
 
 doc_flush_atts(Doc, Fd) ->
-    Doc#doc{atts=[flush_att(Fd, Att) || Att <- Doc#doc.atts]}.
+    Doc#doc{atts=[couch_att:flush(Fd, Att) || Att <- Doc#doc.atts]}.
 
 check_md5(_NewSig, <<>>) -> ok;
 check_md5(Sig, Sig) -> ok;
 check_md5(_, _) -> throw(md5_mismatch).
 
-flush_att(Fd, #att{data={Fd0, _}}=Att) when Fd0 == Fd ->
-    % already written to our file, nothing to write
-    Att;
-
-flush_att(Fd, #att{data={OtherFd,StreamPointer}, md5=InMd5,
-    disk_len=InDiskLen} = Att) ->
-    {NewStreamData, Len, _IdentityLen, Md5, IdentityMd5} =
-            couch_stream:copy_to_new_stream(OtherFd, StreamPointer, Fd),
-    check_md5(IdentityMd5, InMd5),
-    Att#att{data={Fd, NewStreamData}, md5=Md5, att_len=Len, disk_len=InDiskLen};
-
-flush_att(Fd, #att{data=Data}=Att) when is_binary(Data) ->
-    with_stream(Fd, Att, fun(OutputStream) ->
-        couch_stream:write(OutputStream, Data)
-    end);
-
-flush_att(Fd, #att{data=Fun,att_len=undefined}=Att) when is_function(Fun) ->
-    MaxChunkSize = list_to_integer(
-        config:get("couchdb", "attachment_stream_buffer_size", "4096")),
-    with_stream(Fd, Att, fun(OutputStream) ->
-        % Fun(MaxChunkSize, WriterFun) must call WriterFun
-        % once for each chunk of the attachment,
-        Fun(MaxChunkSize,
-            % WriterFun({Length, Binary}, State)
-            % WriterFun({0, _Footers}, State)
-            % Called with Length == 0 on the last time.
-            % WriterFun returns NewState.
-            fun({0, Footers}, _) ->
-                F = mochiweb_headers:from_binary(Footers),
-                case mochiweb_headers:get_value("Content-MD5", F) of
-                undefined ->
-                    ok;
-                Md5 ->
-                    {md5, base64:decode(Md5)}
-                end;
-            ({_Length, Chunk}, _) ->
-                couch_stream:write(OutputStream, Chunk)
-            end, ok)
-    end);
-
-flush_att(Fd, #att{data=Fun,att_len=AttLen}=Att) when is_function(Fun) ->
-    with_stream(Fd, Att, fun(OutputStream) ->
-        write_streamed_attachment(OutputStream, Fun, AttLen)
-    end);
-
-flush_att(Fd, #att{data={follows, Parser, Ref}}=Att) when is_pid(Parser) ->
-    ParserRef = erlang:monitor(process, Parser),
-    Fun = fun() ->
-        Parser ! {get_bytes, Ref, self()},
-        receive
-            {started_open_doc_revs, NewRef} ->
-                couch_doc:restart_open_doc_revs(Parser, Ref, NewRef);
-            {bytes, Ref, Bytes} ->
-                Bytes;
-            {'DOWN', ParserRef, _, _, Reason} ->
-                throw({mp_parser_died, Reason})
-        end
-    end,
-    try
-        flush_att(Fd, Att#att{data=Fun})
-    after
-        erlang:demonitor(ParserRef, [flush])
-    end.
-
 
 compressible_att_type(MimeType) when is_binary(MimeType) ->
     compressible_att_type(?b2l(MimeType));
@@ -1133,7 +1078,8 @@ compressible_att_type(MimeType) ->
 % is present in the request, but there is no Content-MD5
 % trailer, we're free to ignore this inconsistency and
 % pretend that no Content-MD5 exists.
-with_stream(Fd, #att{md5=InMd5,type=Type,encoding=Enc}=Att, Fun) ->
+with_stream(Fd, Att, Fun) ->
+    [InMd5, Type, Enc] = couch_att:fetch([md5, type, encoding], Att),
     BufferSize = list_to_integer(
         config:get("couchdb", "attachment_stream_buffer_size", "4096")),
     {ok, OutputStream} = case (Enc =:= identity) andalso
@@ -1168,43 +1114,25 @@ with_stream(Fd, #att{md5=InMd5,type=Type,encoding=Enc}=Att, Fun) ->
             {Len, IdentityLen, gzip}
         end;
     gzip ->
-        case {Att#att.att_len, Att#att.disk_len} of
-        {AL, DL} when AL =:= undefined orelse DL =:= undefined ->
-            % Compressed attachment uploaded through the standalone API.
-            {Len, Len, gzip};
-        {AL, DL} ->
-            % This case is used for efficient push-replication, where a
-            % compressed attachment is located in the body of multipart
-            % content-type request.
-            {AL, DL, gzip}
+        case couch_att:fetch([att_len, disk_len], Att) of
+            [AL, DL] when AL =:= undefined orelse DL =:= undefined ->
+                % Compressed attachment uploaded through the standalone API.
+                {Len, Len, gzip};
+            [AL, DL] ->
+                % This case is used for efficient push-replication, where a
+                % compressed attachment is located in the body of multipart
+                % content-type request.
+                {AL, DL, gzip}
         end
     end,
-    Att#att{
-        data={Fd,StreamInfo},
-        att_len=AttLen,
-        disk_len=DiskLen,
-        md5=Md5,
-        encoding=NewEnc
-    }.
-
-
-write_streamed_attachment(_Stream, _F, 0) ->
-    ok;
-write_streamed_attachment(_Stream, _F, LenLeft) when LenLeft < 0 ->
-    throw({bad_request, <<"attachment longer than expected">>});
-write_streamed_attachment(Stream, F, LenLeft) when LenLeft > 0 ->
-    Bin = try read_next_chunk(F, LenLeft)
-    catch
-        {mp_parser_died, normal} ->
-            throw({bad_request, <<"attachment shorter than expected">>})
-    end,
-    ok = couch_stream:write(Stream, Bin),
-    write_streamed_attachment(Stream, F, LenLeft - size(Bin)).
+    couch_att:store([
+        {data, {Fd,StreamInfo}},
+        {att_len, AttLen},
+        {disk_len, DiskLen},
+        {md5, Md5},
+        {encoding, NewEnc}
+    ], Att).
 
-read_next_chunk(F, _) when is_function(F, 0) ->
-    F();
-read_next_chunk(F, LenLeft) when is_function(F, 1) ->
-    F(lists:min([LenLeft, 16#2000])).
 
 enum_docs_since_reduce_to_count(Reds) ->
     couch_btree:final_reduce(
@@ -1367,60 +1295,28 @@ read_doc(#db{fd=Fd}, Pos) ->
     couch_file:pread_term(Fd, Pos).
 
 
-make_doc(#db{fd = Fd} = Db, Id, Deleted, Bp, RevisionPath) ->
-    {BodyData, Atts} =
-    case Bp of
-    nil ->
-        {[], []};
-    _ ->
-        {ok, {BodyData0, Atts00}} = read_doc(Db, Bp),
-        Atts0 = case Atts00 of
-        _ when is_binary(Atts00) ->
-            couch_compress:decompress(Atts00);
-        _ when is_list(Atts00) ->
-            % pre 1.2 format
-            Atts00
-        end,
-        {BodyData0,
-            lists:map(
-                fun({Name,Type,Sp,AttLen,DiskLen,RevPos,Md5,Enc}) ->
-                    #att{name=Name,
-                        type=Type,
-                        att_len=AttLen,
-                        disk_len=DiskLen,
-                        md5=Md5,
-                        revpos=RevPos,
-                        data={Fd,Sp},
-                        encoding=
-                            case Enc of
-                            true ->
-                                % 0110 UPGRADE CODE
-                                gzip;
-                            false ->
-                                % 0110 UPGRADE CODE
-                                identity;
-                            _ ->
-                                Enc
-                            end
-                    };
-                ({Name,Type,Sp,AttLen,RevPos,Md5}) ->
-                    #att{name=Name,
-                        type=Type,
-                        att_len=AttLen,
-                        disk_len=AttLen,
-                        md5=Md5,
-                        revpos=RevPos,
-                        data={Fd,Sp}};
-                ({Name,{Type,Sp,AttLen}}) ->
-                    #att{name=Name,
-                        type=Type,
-                        att_len=AttLen,
-                        disk_len=AttLen,
-                        md5= <<>>,
-                        revpos=0,
-                        data={Fd,Sp}}
-                end, Atts0)}
+make_doc(_Db, Id, Deleted, nil = _Bp, RevisionPath) ->
+    #doc{
+        id = Id,
+        revs = RevisionPath,
+        body = [],
+        atts = [],
+        deleted = Deleted
+    };
+make_doc(#db{fd=Fd}=Db, Id, Deleted, Bp, RevisionPath) ->
+    {BodyData, Atts0} = case Bp of
+        nil ->
+            {[], []};
+        _ ->
+            case read_doc(Db, Bp) of
+                {ok, {BodyData0, Atts1}} when is_binary(Atts1) ->
+                    {BodyData0, couch_compress:decompress(Atts1)};
+                {ok, {BodyData0, Atts1}} when is_list(Atts1) ->
+                    % pre 1.2 format
+                    {BodyData0, Atts1}
+            end
     end,
+    Atts = [couch_att:from_disk_term(Fd, T) || T <- Atts0],
     Doc = #doc{
         id = Id,
         revs = RevisionPath,

http://git-wip-us.apache.org/repos/asf/couchdb-couch/blob/507e5acc/src/couch_db_updater.erl
----------------------------------------------------------------------
diff --git a/src/couch_db_updater.erl b/src/couch_db_updater.erl
index 2d326f6..88fa3b3 100644
--- a/src/couch_db_updater.erl
+++ b/src/couch_db_updater.erl
@@ -591,7 +591,7 @@ flush_trees(#db{fd = Fd} = Db,
                 {ok, NewSummaryPointer, SummarySize} =
                     couch_file:append_raw_chunk(Fd, Summary),
                 TotalSize = lists:foldl(
-                    fun(#att{att_len = L}, A) -> A + L end,
+                    fun(Att, A) -> A + couch_att:fetch(att_len, Att) end,
                     SummarySize, Value#doc.atts),
                 NewValue = #leaf{deleted=IsDeleted, ptr=NewSummaryPointer,
                                  seq=UpdateSeq, size=TotalSize},

http://git-wip-us.apache.org/repos/asf/couchdb-couch/blob/507e5acc/src/couch_doc.erl
----------------------------------------------------------------------
diff --git a/src/couch_doc.erl b/src/couch_doc.erl
index d82d626..74450a4 100644
--- a/src/couch_doc.erl
+++ b/src/couch_doc.erl
@@ -13,9 +13,8 @@
 -module(couch_doc).
 
 -export([to_doc_info/1,to_doc_info_path/1,parse_rev/1,parse_revs/1,rev_to_str/1,revs_to_strs/1]).
--export([att_foldl/3,range_att_foldl/5,att_foldl_decode/3,get_validate_doc_fun/1]).
 -export([from_json_obj/1,to_json_obj/2,has_stubs/1, merge_stubs/2]).
--export([validate_docid/1]).
+-export([validate_docid/1, get_validate_doc_fun/1]).
 -export([doc_from_multi_part_stream/2, doc_from_multi_part_stream/3]).
 -export([doc_to_multi_part_stream/5, len_doc_to_multi_part_stream/4]).
 -export([abort_multi_part_stream/1, restart_open_doc_revs/3]).
@@ -98,50 +97,11 @@ to_json_attachments(Attachments, Options) ->
         lists:member(att_encoding_info, Options)
     ).
 
-to_json_attachments([], _OutputData, _DataToFollow, _ShowEncInfo) ->
+to_json_attachments([], _OutputData, _Follows, _ShowEnc) ->
     [];
-to_json_attachments(Atts, OutputData, DataToFollow, ShowEncInfo) ->
-    AttProps = lists:map(
-        fun(#att{disk_len=DiskLen, att_len=AttLen, encoding=Enc}=Att) ->
-            {Att#att.name, {[
-                {<<"content_type">>, Att#att.type},
-                {<<"revpos">>, Att#att.revpos}] ++
-                case Att#att.md5 of
-                    <<>> ->
-                        [];
-                    Md5 ->
-                        EncodedMd5 = base64:encode(Md5),
-                        [{<<"digest">>, <<"md5-",EncodedMd5/binary>>}]
-                end ++
-                if not OutputData orelse Att#att.data == stub ->
-                    [{<<"length">>, DiskLen}, {<<"stub">>, true}];
-                true ->
-                    if DataToFollow ->
-                        [{<<"length">>, DiskLen}, {<<"follows">>, true}];
-                    true ->
-                        AttData = case Enc of
-                        gzip ->
-                            zlib:gunzip(att_to_bin(Att));
-                        identity ->
-                            att_to_bin(Att)
-                        end,
-                        [{<<"data">>, base64:encode(AttData)}]
-                    end
-                end ++
-                    case {ShowEncInfo, Enc} of
-                    {false, _} ->
-                        [];
-                    {true, identity} ->
-                        [];
-                    {true, _} ->
-                        [
-                            {<<"encoding">>, couch_util:to_binary(Enc)},
-                            {<<"encoded_length">>, AttLen}
-                        ]
-                    end
-            }}
-        end, Atts),
-    [{<<"_attachments">>, {AttProps}}].
+to_json_attachments(Atts, OutputData, Follows, ShowEnc) ->
+    Props = [couch_att:to_json(A, OutputData, Follows, ShowEnc) || A <- Atts],
+    [{<<"_attachments">>, {Props}}].
 
 to_json_obj(Doc, Options) ->
     doc_to_json_obj(with_ejson_body(Doc), Options).
@@ -227,40 +187,7 @@ transfer_fields([{<<"_rev">>, _Rev} | Rest], Doc) ->
     transfer_fields(Rest,Doc);
 
 transfer_fields([{<<"_attachments">>, {JsonBins}} | Rest], Doc) ->
-    Atts = lists:map(fun({Name, {BinProps}}) ->
-        Md5 = case couch_util:get_value(<<"digest">>, BinProps) of
-            <<"md5-",EncodedMd5/binary>> ->
-                base64:decode(EncodedMd5);
-            _ ->
-               <<>>
-        end,
-        case couch_util:get_value(<<"stub">>, BinProps) of
-        true ->
-            Type = couch_util:get_value(<<"content_type">>, BinProps),
-            RevPos = couch_util:get_value(<<"revpos">>, BinProps, nil),
-            DiskLen = couch_util:get_value(<<"length">>, BinProps),
-            {Enc, EncLen} = att_encoding_info(BinProps),
-            #att{name=Name, data=stub, type=Type, att_len=EncLen,
-                disk_len=DiskLen, encoding=Enc, revpos=RevPos, md5=Md5};
-        _ ->
-            Type = couch_util:get_value(<<"content_type">>, BinProps,
-                    ?DEFAULT_ATTACHMENT_CONTENT_TYPE),
-            RevPos = couch_util:get_value(<<"revpos">>, BinProps, 0),
-            case couch_util:get_value(<<"follows">>, BinProps) of
-            true ->
-                DiskLen = couch_util:get_value(<<"length">>, BinProps),
-                {Enc, EncLen} = att_encoding_info(BinProps),
-                #att{name=Name, data=follows, type=Type, encoding=Enc,
-                    att_len=EncLen, disk_len=DiskLen, revpos=RevPos, md5=Md5};
-            _ ->
-                Value = couch_util:get_value(<<"data">>, BinProps),
-                Bin = base64:decode(Value),
-                LenBin = size(Bin),
-                #att{name=Name, data=Bin, type=Type, att_len=LenBin,
-                        disk_len=LenBin, revpos=RevPos}
-            end
-        end
-    end, JsonBins),
+    Atts = [couch_att:from_json(Name, Props) || {Name, {Props}} <- JsonBins],
     transfer_fields(Rest, Doc#doc{atts=Atts});
 
 transfer_fields([{<<"_revisions">>, {Props}} | Rest], Doc) ->
@@ -316,16 +243,6 @@ transfer_fields([{<<"_",Name/binary>>, _} | _], _) ->
 transfer_fields([Field | Rest], #doc{body=Fields}=Doc) ->
     transfer_fields(Rest, Doc#doc{body=[Field|Fields]}).
 
-att_encoding_info(BinProps) ->
-    DiskLen = couch_util:get_value(<<"length">>, BinProps),
-    case couch_util:get_value(<<"encoding">>, BinProps) of
-    undefined ->
-        {identity, DiskLen};
-    Enc ->
-        EncodedLen = couch_util:get_value(<<"encoded_length">>, BinProps, DiskLen),
-        {list_to_existing_atom(?b2l(Enc)), EncodedLen}
-    end.
-
 to_doc_info(FullDocInfo) ->
     {DocInfo, _Path} = to_doc_info_path(FullDocInfo),
     DocInfo.
@@ -387,61 +304,6 @@ is_deleted(Tree) ->
     end.
 
 
-att_foldl(#att{data=Bin}, Fun, Acc) when is_binary(Bin) ->
-    Fun(Bin, Acc);
-att_foldl(#att{data={Fd,Sp},md5=Md5}, Fun, Acc) ->
-    couch_stream:foldl(Fd, Sp, Md5, Fun, Acc);
-att_foldl(#att{data=DataFun,att_len=Len}, Fun, Acc) when is_function(DataFun) ->
-   fold_streamed_data(DataFun, Len, Fun, Acc);
-att_foldl(#att{data={follows, Parser, Ref}}=Att, Fun, Acc) ->
-    ParserRef = erlang:monitor(process, Parser),
-    DataFun = fun() ->
-        Parser ! {get_bytes, Ref, self()},
-        receive
-            {started_open_doc_revs, NewRef} ->
-                couch_doc:restart_open_doc_revs(Parser, Ref, NewRef);
-            {bytes, Ref, Bytes} ->
-                Bytes;
-            {'DOWN', ParserRef, _, _, Reason} ->
-                throw({mp_parser_died, Reason})
-        end
-    end,
-    try
-        att_foldl(Att#att{data=DataFun}, Fun, Acc)
-    after
-        erlang:demonitor(ParserRef, [flush])
-    end.
-
-range_att_foldl(#att{data={Fd,Sp}}, From, To, Fun, Acc) ->
-   couch_stream:range_foldl(Fd, Sp, From, To, Fun, Acc).
-
-att_foldl_decode(#att{data={Fd,Sp},md5=Md5,encoding=Enc}, Fun, Acc) ->
-    couch_stream:foldl_decode(Fd, Sp, Md5, Enc, Fun, Acc);
-att_foldl_decode(#att{data=Fun2,att_len=Len, encoding=identity}, Fun, Acc) ->
-       fold_streamed_data(Fun2, Len, Fun, Acc).
-
-att_to_bin(#att{data=Bin}) when is_binary(Bin) ->
-    Bin;
-att_to_bin(#att{data=Iolist}) when is_list(Iolist) ->
-    iolist_to_binary(Iolist);
-att_to_bin(#att{data={_Fd,_Sp}}=Att) ->
-    iolist_to_binary(
-        lists:reverse(att_foldl(
-                Att,
-                fun(Bin,Acc) -> [Bin|Acc] end,
-                []
-        ))
-    );
-att_to_bin(#att{data=DataFun, att_len=Len}) when is_function(DataFun)->
-    iolist_to_binary(
-        lists:reverse(fold_streamed_data(
-            DataFun,
-            Len,
-            fun(Data, Acc) -> [Data | Acc] end,
-            []
-        ))
-    ).
-
 get_validate_doc_fun({Props}) ->
     get_validate_doc_fun(couch_doc:from_json_obj({Props}));
 get_validate_doc_fun(#doc{body={Props}}=DDoc) ->
@@ -456,50 +318,26 @@ get_validate_doc_fun(#doc{body={Props}}=DDoc) ->
 
 
 has_stubs(#doc{atts=Atts}) ->
-    has_stubs(Atts);
-has_stubs([]) ->
-    false;
-has_stubs([#att{data=stub}|_]) ->
-    true;
-has_stubs([_Att|Rest]) ->
-    has_stubs(Rest).
+    lists:any(fun couch_att:is_stub/1, Atts);
+has_stubs(Atts) ->
+    lists:any(fun couch_att:is_stub/1, Atts).
 
 merge_stubs(#doc{id = Id}, nil) ->
     throw({missing_stub, <<"Previous revision missing for document ", Id/binary>>});
 merge_stubs(#doc{id=Id,atts=MemBins}=StubsDoc, #doc{atts=DiskBins}) ->
-    BinDict = dict:from_list([{Name, Att} || #att{name=Name}=Att <- DiskBins]),
-    MergedBins = lists:map(
-        fun(#att{name=Name, data=stub, revpos=StubRevPos}) ->
-            case dict:find(Name, BinDict) of
-            {ok, #att{revpos=DiskRevPos}=DiskAtt}
-                    when DiskRevPos == StubRevPos orelse StubRevPos == nil ->
-                DiskAtt;
-            _ ->
-                throw({missing_stub,
-                        <<"id:", Id/binary, ", name:", Name/binary>>})
-            end;
-        (Att) ->
-            Att
-        end, MemBins),
-    StubsDoc#doc{atts= MergedBins}.
-
-fold_streamed_data(_RcvFun, 0, _Fun, Acc) ->
-    Acc;
-fold_streamed_data(RcvFun, LenLeft, Fun, Acc) when LenLeft > 0->
-    Bin = RcvFun(),
-    ResultAcc = Fun(Bin, Acc),
-    fold_streamed_data(RcvFun, LenLeft - size(Bin), Fun, ResultAcc).
+    case couch_att:merge_stubs(MemBins, DiskBins) of
+        {ok, MergedBins} ->
+            StubsDoc#doc{atts = MergedBins};
+        {missing, Name} ->
+            throw({missing_stub,
+                <<"Invalid attachment stub in ", Id/binary, " for ", Name/binary>>
+            })
+    end.
 
 len_doc_to_multi_part_stream(Boundary, JsonBytes, Atts, SendEncodedAtts) ->
     AttsSize = lists:foldl(fun(Att, AccAttsSize) ->
-            #att{
-                data=Data,
-                name=Name,
-                att_len=AttLen,
-                disk_len=DiskLen,
-                type=Type,
-                encoding=Encoding
-            } = Att,
+            [Data, Name, AttLen, DiskLen, Type, Encoding] =
+                 couch_att:fetch([data, name, att_len, disk_len, type, encoding], Att),
             case Data of
             stub ->
                 AccAttsSize;
@@ -552,7 +390,7 @@ len_doc_to_multi_part_stream(Boundary, JsonBytes, Atts, SendEncodedAtts) ->
 
 doc_to_multi_part_stream(Boundary, JsonBytes, Atts, WriteFun,
     SendEncodedAtts) ->
-    case lists:any(fun(#att{data=Data})-> Data /= stub end, Atts) of
+    case lists:any(fun(Att)-> couch_att:fetch(data, Att) /= stub end, Atts) of
     true ->
         WriteFun([<<"--", Boundary/binary,
                 "\r\nContent-Type: application/json\r\n\r\n">>,
@@ -564,46 +402,39 @@ doc_to_multi_part_stream(Boundary, JsonBytes, Atts, WriteFun,
 
 atts_to_mp([], _Boundary, WriteFun, _SendEncAtts) ->
     WriteFun(<<"--">>);
-atts_to_mp([#att{data=stub} | RestAtts], Boundary, WriteFun,
-        SendEncodedAtts) ->
-    atts_to_mp(RestAtts, Boundary, WriteFun, SendEncodedAtts);
-atts_to_mp([Att | RestAtts], Boundary, WriteFun,
-        SendEncodedAtts)  ->
-    #att{
-        name=Name,
-        att_len=AttLen,
-        disk_len=DiskLen,
-        type=Type,
-        encoding=Encoding
-    } = Att,
-
-    % write headers
-    LengthBin = case SendEncodedAtts of
-    true -> list_to_binary(integer_to_list(AttLen));
-    false -> list_to_binary(integer_to_list(DiskLen))
-    end,
-    WriteFun(<<"\r\nContent-Disposition: attachment; filename=\"", Name/binary, "\"">>),
-    WriteFun(<<"\r\nContent-Type: ", Type/binary>>),
-    WriteFun(<<"\r\nContent-Length: ", LengthBin/binary>>),
-    case Encoding of
-    identity ->
-        ok;
-    _ ->
-        EncodingBin = atom_to_binary(Encoding, latin1),
-        WriteFun(<<"\r\nContent-Encoding: ", EncodingBin/binary>>)
-    end,
-
-    % write data
-    WriteFun(<<"\r\n\r\n">>),
-    AttFun = case SendEncodedAtts of
-    false ->
-        fun att_foldl_decode/3;
-    true ->
-        fun att_foldl/3
-    end,
-    AttFun(Att, fun(Data, _) -> WriteFun(Data) end, ok),
-    WriteFun(<<"\r\n--", Boundary/binary>>),
-    atts_to_mp(RestAtts, Boundary, WriteFun, SendEncodedAtts).
+atts_to_mp([Att | RestAtts], Boundary, WriteFun, SendEncodedAtts)  ->
+    case couch_att:is_stub(Att) of
+        true ->
+            atts_to_mp(RestAtts, Boundary, WriteFun, SendEncodedAtts);
+        false ->
+            [Name, AttLen, DiskLen, Type, Encoding] =
+                couch_att:fetch([name, att_len, disk_len, type, encoding], Att),
+            % write headers
+            LengthBin = case SendEncodedAtts of
+                true  -> list_to_binary(integer_to_list(AttLen));
+                false -> list_to_binary(integer_to_list(DiskLen))
+            end,
+            WriteFun(<<"\r\nContent-Disposition: attachment; filename=\"", Name/binary, "\"">>),
+            WriteFun(<<"\r\nContent-Type: ", Type/binary>>),
+            WriteFun(<<"\r\nContent-Length: ", LengthBin/binary>>),
+            case Encoding of
+                identity ->
+                    ok;
+                _ ->
+                    EncodingBin = atom_to_binary(Encoding, latin1),
+                    WriteFun(<<"\r\nContent-Encoding: ", EncodingBin/binary>>)
+            end,
+
+            % write data
+            WriteFun(<<"\r\n\r\n">>),
+            AttFun = case SendEncodedAtts of
+                false -> fun couch_att:foldl_decode/3;
+                true  -> fun couch_att:foldl/3
+            end,
+            AttFun(Att, fun(Data, _) -> WriteFun(Data) end, ok),
+            WriteFun(<<"\r\n--", Boundary/binary>>),
+            atts_to_mp(RestAtts, Boundary, WriteFun, SendEncodedAtts)
+    end.
 
 
 doc_from_multi_part_stream(ContentType, DataFun) ->
@@ -630,17 +461,13 @@ doc_from_multi_part_stream(ContentType, DataFun, Ref) ->
         Doc = from_json_obj(?JSON_DECODE(DocBytes)),
         % we'll send the Parser process ID to the remote nodes so they can
         % retrieve their own copies of the attachment data
-        Atts2 = lists:map(
-            fun(#att{data=follows}=A) ->
-                A#att{data={follows, Parser, Ref}};
-            (A) ->
-                A
-            end, Doc#doc.atts),
+        WithParser = fun(follows) -> {follows, Parser, Ref}; (D) -> D end,
+        Atts = [couch_att:transform(data, WithParser, A) || A <- Doc#doc.atts],
         WaitFun = fun() ->
             receive {'DOWN', ParserRef, _, _, _} -> ok end,
             erlang:put(mochiweb_request_recv, true)
         end,
-        {ok, Doc#doc{atts=Atts2}, WaitFun, Parser};
+        {ok, Doc#doc{atts=Atts}, WaitFun, Parser};
     {'DOWN', ParserRef, _, _, normal} ->
         ok;
     {'DOWN', ParserRef, process, Parser, {{nocatch, {Error, Msg}}, _}} ->

http://git-wip-us.apache.org/repos/asf/couchdb-couch/blob/507e5acc/src/couch_httpd_db.erl
----------------------------------------------------------------------
diff --git a/src/couch_httpd_db.erl b/src/couch_httpd_db.erl
index 8689944..77d8788 100644
--- a/src/couch_httpd_db.erl
+++ b/src/couch_httpd_db.erl
@@ -547,16 +547,19 @@ db_doc_req(#httpd{method='POST'}=Req, Db, DocId) ->
         Doc = couch_doc_from_req(Req, DocId, ?JSON_DECODE(Json))
     end,
     UpdatedAtts = [
-        #att{name=validate_attachment_name(Name),
-            type=list_to_binary(ContentType),
-            data=Content} ||
+        couch_att:new([
+            {name, validate_attachment_name(Name)},
+            {type, list_to_binary(ContentType)},
+            {data, Content}
+        ]) ||
         {Name, {ContentType, _}, Content} <-
         proplists:get_all_values("_attachments", Form)
     ],
     #doc{atts=OldAtts} = Doc,
     OldAtts2 = lists:flatmap(
-        fun(#att{name=OldName}=Att) ->
-            case [1 || A <- UpdatedAtts, A#att.name == OldName] of
+        fun(Att) ->
+            OldName = couch_att:fetch(name, Att),
+            case [1 || A <- UpdatedAtts, couch_att:fetch(name, A) == OldName] of
             [] -> [Att]; % the attachment wasn't in the UpdatedAtts, return it
             _ -> [] % the attachment was in the UpdatedAtts, drop it
             end
@@ -684,7 +687,7 @@ send_ranges_multipart(Req, ContentType, Len, Att, Ranges) ->
             <<"\r\nContent-Type: ", ContentType/binary, "\r\n",
             "Content-Range: ", ContentRange/binary, "\r\n",
            "\r\n">>),
-        couch_doc:range_att_foldl(Att, From, To + 1,
+        couch_att:range_foldl(Att, From, To + 1,
             fun(Seg, _) -> send_chunk(Resp, Seg) end, {ok, Resp}),
         couch_httpd:send_chunk(Resp, <<"\r\n--", Boundary/binary>>)
     end, Ranges),
@@ -833,13 +836,14 @@ db_attachment_req(#httpd{method='GET',mochi_req=MochiReq}=Req, Db, DocId, FileNa
     #doc{
         atts=Atts
     } = Doc = couch_doc_open(Db, DocId, Rev, Options),
-    case [A || A <- Atts, A#att.name == FileName] of
+    case [A || A <- Atts, couch_att:fetch(name, A) == FileName] of
     [] ->
         throw({not_found, "Document is missing attachment"});
-    [#att{type=Type, encoding=Enc, disk_len=DiskLen, att_len=AttLen}=Att] ->
-        Etag = case Att#att.md5 of
+    [Att] ->
+        [Type, Enc, DiskLen, AttLen, Md5] = couch_att:fetch([type, encoding, disk_len, att_len, md5], Att),
+        Etag = case Md5 of
             <<>> -> couch_httpd:doc_etag(Doc);
-            Md5 -> "\"" ++ ?b2l(base64:encode(Md5)) ++ "\""
+            _ -> "\"" ++ ?b2l(base64:encode(Md5)) ++ "\""
         end,
         ReqAcceptsAttEnc = lists:member(
            atom_to_list(Enc),
@@ -883,9 +887,9 @@ db_attachment_req(#httpd{method='GET',mochi_req=MochiReq}=Req, Db, DocId, FileNa
         end,
         AttFun = case ReqAcceptsAttEnc of
         false ->
-            fun couch_doc:att_foldl_decode/3;
+            fun couch_att:foldl_decode/3;
         true ->
-            fun couch_doc:att_foldl/3
+            fun couch_att:foldl/3
         end,
         couch_httpd:etag_respond(
             Req,
@@ -903,14 +907,14 @@ db_attachment_req(#httpd{method='GET',mochi_req=MochiReq}=Req, Db, DocId, FileNa
                             Headers1 = [{"Content-Range", make_content_range(From, To, Len)}]
                                 ++ Headers,
                             {ok, Resp} = start_response_length(Req, 206, Headers1, To - From + 1),
-                            couch_doc:range_att_foldl(Att, From, To + 1,
+                            couch_att:range_foldl(Att, From, To + 1,
                                 fun(Seg, _) -> send(Resp, Seg) end, {ok, Resp});
                         {identity, Ranges} when is_list(Ranges) andalso length(Ranges) < 10 ->
                             send_ranges_multipart(Req, Type, Len, Att, Ranges);
                         _ ->
                             Headers1 = Headers ++
                                 if Enc =:= identity orelse ReqAcceptsAttEnc =:= true ->
-                                    [{"Content-MD5", base64:encode(Att#att.md5)}];
+                                    [{"Content-MD5", base64:encode(Md5)}];
                                 true ->
                                     []
                             end,
@@ -929,60 +933,54 @@ db_attachment_req(#httpd{method=Method,mochi_req=MochiReq}=Req, Db, DocId, FileN
                     mochiweb_util:join(
                         lists:map(fun binary_to_list/1,
                             FileNameParts),"/")),
-
     NewAtt = case Method of
         'DELETE' ->
             [];
         _ ->
-            [#att{
-                name = FileName,
-                type = case couch_httpd:header_value(Req,"Content-Type") of
-                    undefined ->
-                        % We could throw an error here or guess by the FileName.
-                        % Currently, just giving it a default.
-                        <<"application/octet-stream">>;
-                    CType ->
-                        list_to_binary(CType)
+            MimeType = case couch_httpd:header_value(Req,"Content-Type") of
+                % We could throw an error here or guess by the FileName.
+                % Currently, just giving it a default.
+                undefined -> <<"application/octet-stream">>;
+                CType -> list_to_binary(CType)
+            end,
+            Data = case couch_httpd:body_length(Req) of
+                undefined ->
+                    <<"">>;
+                {unknown_transfer_encoding, Unknown} ->
+                    exit({unknown_transfer_encoding, Unknown});
+                chunked ->
+                    fun(MaxChunkSize, ChunkFun, InitState) ->
+                        couch_httpd:recv_chunked(
+                            Req, MaxChunkSize, ChunkFun, InitState
+                        )
+                    end;
+                0 ->
+                    <<"">>;
+                Length when is_integer(Length) ->
+                    Expect = case couch_httpd:header_value(Req, "expect") of
+                        undefined ->
+                            undefined;
+                        Value when is_list(Value) ->
+                            string:to_lower(Value)
                     end,
-                data = case couch_httpd:body_length(Req) of
-                    undefined ->
-                        <<"">>;
-                    {unknown_transfer_encoding, Unknown} ->
-                        exit({unknown_transfer_encoding, Unknown});
-                    chunked ->
-                        fun(MaxChunkSize, ChunkFun, InitState) ->
-                            couch_httpd:recv_chunked(Req, MaxChunkSize,
-                                ChunkFun, InitState)
-                        end;
-                    0 ->
-                        <<"">>;
-                    Length when is_integer(Length) ->
-                        Expect = case couch_httpd:header_value(Req, "expect") of
-                                     undefined ->
-                                         undefined;
-                                     Value when is_list(Value) ->
-                                         string:to_lower(Value)
-                                 end,
-                        case Expect of
-                            "100-continue" ->
-                                MochiReq:start_raw_response({100, gb_trees:empty()});
-                            _Else ->
-                                ok
-                        end,
-
-
-                        fun(Size) -> couch_httpd:recv(Req, Size) end
+                    case Expect of
+                        "100-continue" ->
+                            MochiReq:start_raw_response({100, gb_trees:empty()});
+                        _Else ->
+                            ok
                     end,
-                att_len = case couch_httpd:header_value(Req,"Content-Length") of
-                    undefined ->
-                        undefined;
-                    Length ->
-                        list_to_integer(Length)
-                    end,
-                md5 = get_md5_header(Req),
-                encoding = case string:to_lower(string:strip(
-                    couch_httpd:header_value(Req,"Content-Encoding","identity")
-                )) of
+                    fun() -> couch_httpd:recv(Req, 0) end;
+                Length ->
+                    exit({length_not_integer, Length})
+            end,
+            AttLen = case couch_httpd:header_value(Req,"Content-Length") of
+                undefined -> undefined;
+                Len -> list_to_integer(Len)
+            end,
+            ContentEnc = string:to_lower(string:strip(
+                couch_httpd:header_value(Req,"Content-Encoding","identity")
+            )),
+            Encoding = case ContentEnc of
                 "identity" ->
                    identity;
                 "gzip" ->
@@ -992,8 +990,15 @@ db_attachment_req(#httpd{method=Method,mochi_req=MochiReq}=Req, Db, DocId, FileN
                        bad_ctype,
                        "Only gzip and identity content-encodings are supported"
                    })
-                end
-            }]
+            end,
+            [couch_att:new([
+                {name, FileName},
+                {type, MimeType},
+                {data, Data},
+                {att_len, AttLen},
+                {md5, get_md5_header(Req)},
+                {encoding, Encoding}
+            ])]
     end,
 
     Doc = case extract_header_rev(Req, couch_httpd:qs_value(Req, "rev")) of
@@ -1010,7 +1015,7 @@ db_attachment_req(#httpd{method=Method,mochi_req=MochiReq}=Req, Db, DocId, FileN
 
     #doc{atts=Atts} = Doc,
     DocEdited = Doc#doc{
-        atts = NewAtt ++ [A || A <- Atts, A#att.name /= FileName]
+        atts = NewAtt ++ [A || A <- Atts, couch_att:fetch(name, A) /= FileName]
     },
 
     Headers = case Method of
@@ -1216,7 +1221,8 @@ parse_copy_destination_header(Req) ->
     end.
 
 validate_attachment_names(Doc) ->
-    lists:foreach(fun(#att{name=Name}) ->
+    lists:foreach(fun(Att) ->
+        Name = couch_att:fetch(name, Att),
         validate_attachment_name(Name)
     end, Doc#doc.atts).